HDDS-182:CleanUp Reimplemented classes. Contributed by Hansiha Koneru

This commit is contained in:
Bharat Viswanadham 2018-07-05 15:23:39 -07:00
parent a404164939
commit 2c2351e87b
51 changed files with 696 additions and 4798 deletions

View File

@ -183,4 +183,14 @@ public final class OzoneConsts {
private OzoneConsts() {
// Never Constructed
}
// YAML fields for .container files
public static final String CONTAINER_ID = "containerID";
public static final String CONTAINER_TYPE = "containerType";
public static final String STATE = "state";
public static final String METADATA = "metadata";
public static final String MAX_SIZE_GB = "maxSizeGB";
public static final String METADATA_PATH = "metadataPath";
public static final String CHUNKS_PATH = "chunksPath";
public static final String CONTAINER_DB_TYPE = "containerDBType";
}

View File

@ -1,343 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.helpers;
import com.google.common.base.Preconditions;
import org.apache.ratis.shaded.com.google.protobuf.ByteString;
import org.apache.commons.codec.binary.Hex;
import org.apache.commons.codec.digest.DigestUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.AsynchronousFileChannel;
import java.nio.channels.FileLock;
import java.nio.file.StandardOpenOption;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import java.util.concurrent.ExecutionException;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.CHECKSUM_MISMATCH;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.CONTAINER_INTERNAL_ERROR;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.CONTAINER_NOT_FOUND;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.INVALID_WRITE_SIZE;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.IO_EXCEPTION;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.OVERWRITE_FLAG_REQUIRED;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.UNABLE_TO_FIND_CHUNK;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.UNABLE_TO_FIND_DATA_DIR;
/**
* Set of utility functions used by the chunk Manager.
*/
public final class ChunkUtils {
/* Never constructed. */
private ChunkUtils() {
}
/**
* Checks if we are getting a request to overwrite an existing range of
* chunk.
*
* @param chunkFile - File
* @param chunkInfo - Buffer to write
* @return bool
*/
public static boolean isOverWriteRequested(File chunkFile, ChunkInfo
chunkInfo) {
if (!chunkFile.exists()) {
return false;
}
long offset = chunkInfo.getOffset();
return offset < chunkFile.length();
}
/**
* Overwrite is permitted if an only if the user explicitly asks for it. We
* permit this iff the key/value pair contains a flag called
* [OverWriteRequested, true].
*
* @param chunkInfo - Chunk info
* @return true if the user asks for it.
*/
public static boolean isOverWritePermitted(ChunkInfo chunkInfo) {
String overWrite = chunkInfo.getMetadata().get(OzoneConsts.CHUNK_OVERWRITE);
return (overWrite != null) &&
(!overWrite.isEmpty()) &&
(Boolean.valueOf(overWrite));
}
/**
* Validates chunk data and returns a file object to Chunk File that we are
* expected to write data to.
*
* @param data - container data.
* @param info - chunk info.
* @return File
* @throws StorageContainerException
*/
public static File validateChunk(ContainerData data,
ChunkInfo info) throws StorageContainerException {
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
File chunkFile = getChunkFile(data, info);
if (ChunkUtils.isOverWriteRequested(chunkFile, info)) {
if (!ChunkUtils.isOverWritePermitted(info)) {
log.error("Rejecting write chunk request. Chunk overwrite " +
"without explicit request. {}", info.toString());
throw new StorageContainerException("Rejecting write chunk request. " +
"OverWrite flag required." + info.toString(),
OVERWRITE_FLAG_REQUIRED);
}
}
return chunkFile;
}
/**
* Validates that Path to chunk file exists.
*
* @param data - Container Data
* @param info - Chunk info
* @return - File.
* @throws StorageContainerException
*/
public static File getChunkFile(ContainerData data,
ChunkInfo info) throws StorageContainerException {
Preconditions.checkNotNull(data, "Container data can't be null");
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
if (data.getContainerID() < 0) {
log.error("Invalid container id: {}", data.getContainerID());
throw new StorageContainerException("Unable to find the container id:" +
" " +
data.getContainerID(), CONTAINER_NOT_FOUND);
}
File dataDir = ContainerUtils.getDataDirectory(data).toFile();
if (!dataDir.exists()) {
log.error("Unable to find the data directory: {}", dataDir);
throw new StorageContainerException("Unable to find the data directory:" +
" " + dataDir, UNABLE_TO_FIND_DATA_DIR);
}
return dataDir.toPath().resolve(info.getChunkName()).toFile();
}
/**
* Writes the data in chunk Info to the specified location in the chunkfile.
*
* @param chunkFile - File to write data to.
* @param chunkInfo - Data stream to write.
* @param data - The data buffer.
* @throws StorageContainerException
*/
public static void writeData(File chunkFile, ChunkInfo chunkInfo,
byte[] data) throws
StorageContainerException, ExecutionException, InterruptedException,
NoSuchAlgorithmException {
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
if (data.length != chunkInfo.getLen()) {
String err = String.format("data array does not match the length " +
"specified. DataLen: %d Byte Array: %d",
chunkInfo.getLen(), data.length);
log.error(err);
throw new StorageContainerException(err, INVALID_WRITE_SIZE);
}
AsynchronousFileChannel file = null;
FileLock lock = null;
try {
file =
AsynchronousFileChannel.open(chunkFile.toPath(),
StandardOpenOption.CREATE,
StandardOpenOption.WRITE,
StandardOpenOption.SPARSE,
StandardOpenOption.SYNC);
lock = file.lock().get();
if (chunkInfo.getChecksum() != null &&
!chunkInfo.getChecksum().isEmpty()) {
verifyChecksum(chunkInfo, data, log);
}
int size = file.write(ByteBuffer.wrap(data), chunkInfo.getOffset()).get();
if (size != data.length) {
log.error("Invalid write size found. Size:{} Expected: {} ", size,
data.length);
throw new StorageContainerException("Invalid write size found. " +
"Size: " + size + " Expected: " + data.length, INVALID_WRITE_SIZE);
}
} catch (IOException e) {
throw new StorageContainerException(e, IO_EXCEPTION);
} finally {
if (lock != null) {
try {
lock.release();
} catch (IOException e) {
log.error("Unable to release lock ??, Fatal Error.");
throw new StorageContainerException(e, CONTAINER_INTERNAL_ERROR);
}
}
if (file != null) {
try {
file.close();
} catch (IOException e) {
throw new StorageContainerException("Error closing chunk file",
e, CONTAINER_INTERNAL_ERROR);
}
}
}
}
/**
* Verifies the checksum of a chunk against the data buffer.
*
* @param chunkInfo - Chunk Info.
* @param data - data buffer
* @param log - log
* @throws NoSuchAlgorithmException
* @throws StorageContainerException
*/
private static void verifyChecksum(ChunkInfo chunkInfo, byte[] data, Logger
log) throws NoSuchAlgorithmException, StorageContainerException {
MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
sha.update(data);
if (!Hex.encodeHexString(sha.digest()).equals(
chunkInfo.getChecksum())) {
log.error("Checksum mismatch. Provided: {} , computed: {}",
chunkInfo.getChecksum(), DigestUtils.sha256Hex(sha.digest()));
throw new StorageContainerException("Checksum mismatch. Provided: " +
chunkInfo.getChecksum() + " , computed: " +
DigestUtils.sha256Hex(sha.digest()), CHECKSUM_MISMATCH);
}
}
/**
* Reads data from an existing chunk file.
*
* @param chunkFile - file where data lives.
* @param data - chunk definition.
* @return ByteBuffer
* @throws StorageContainerException
* @throws ExecutionException
* @throws InterruptedException
*/
public static ByteBuffer readData(File chunkFile, ChunkInfo data) throws
StorageContainerException, ExecutionException, InterruptedException,
NoSuchAlgorithmException {
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
if (!chunkFile.exists()) {
log.error("Unable to find the chunk file. chunk info : {}",
data.toString());
throw new StorageContainerException("Unable to find the chunk file. " +
"chunk info " +
data.toString(), UNABLE_TO_FIND_CHUNK);
}
AsynchronousFileChannel file = null;
FileLock lock = null;
try {
file =
AsynchronousFileChannel.open(chunkFile.toPath(),
StandardOpenOption.READ);
lock = file.lock(data.getOffset(), data.getLen(), true).get();
ByteBuffer buf = ByteBuffer.allocate((int) data.getLen());
file.read(buf, data.getOffset()).get();
if (data.getChecksum() != null && !data.getChecksum().isEmpty()) {
verifyChecksum(data, buf.array(), log);
}
return buf;
} catch (IOException e) {
throw new StorageContainerException(e, IO_EXCEPTION);
} finally {
if (lock != null) {
try {
lock.release();
} catch (IOException e) {
log.error("I/O error is lock release.");
}
}
if (file != null) {
IOUtils.closeStream(file);
}
}
}
/**
* Returns a CreateContainer Response. This call is used by create and delete
* containers which have null success responses.
*
* @param msg Request
* @return Response.
*/
public static ContainerProtos.ContainerCommandResponseProto
getChunkResponse(ContainerProtos.ContainerCommandRequestProto msg) {
return ContainerUtils.getSuccessResponse(msg);
}
/**
* Gets a response to the read chunk calls.
*
* @param msg - Msg
* @param data - Data
* @param info - Info
* @return Response.
*/
public static ContainerProtos.ContainerCommandResponseProto
getReadChunkResponse(ContainerProtos.ContainerCommandRequestProto msg,
byte[] data, ChunkInfo info) {
Preconditions.checkNotNull(msg);
ContainerProtos.ReadChunkResponseProto.Builder response =
ContainerProtos.ReadChunkResponseProto.newBuilder();
response.setChunkData(info.getProtoBufMessage());
response.setData(ByteString.copyFrom(data));
response.setBlockID(msg.getReadChunk().getBlockID());
ContainerProtos.ContainerCommandResponseProto.Builder builder =
ContainerUtils.getSuccessResponseBuilder(msg);
builder.setReadChunk(response);
return builder.build();
}
}

View File

@ -1,512 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.helpers;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerType;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerLifeCycleState;
import org.apache.hadoop.ozone.OzoneConsts;
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicLong;
import static java.lang.Math.max;
/**
* This class maintains the information about a container in the ozone world.
* <p>
* A container is a name, along with metadata- which is a set of key value
* pair.
*/
public class ContainerData {
private final Map<String, String> metadata;
private String dbPath; // Path to Level DB Store.
// Path to Physical file system where container and checksum are stored.
private String containerFilePath;
private AtomicLong bytesUsed;
private long maxSize;
private long containerID;
private ContainerLifeCycleState state;
private ContainerType containerType;
private String containerDBType;
/**
* Number of pending deletion blocks in container.
*/
private int numPendingDeletionBlocks;
private long deleteTransactionId;
private AtomicLong readBytes;
private AtomicLong writeBytes;
private AtomicLong readCount;
private AtomicLong writeCount;
/**
* Constructs a ContainerData Object.
*
* @param containerID - ID
* @param conf - Configuration
*/
public ContainerData(long containerID,
Configuration conf) {
this.metadata = new TreeMap<>();
this.maxSize = conf.getLong(ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_KEY,
ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_DEFAULT) * OzoneConsts.GB;
this.bytesUsed = new AtomicLong(0L);
this.containerID = containerID;
this.state = ContainerLifeCycleState.OPEN;
this.numPendingDeletionBlocks = 0;
this.deleteTransactionId = 0;
this.readCount = new AtomicLong(0L);
this.readBytes = new AtomicLong(0L);
this.writeCount = new AtomicLong(0L);
this.writeBytes = new AtomicLong(0L);
}
/**
* Constructs a ContainerData Object.
*
* @param containerID - ID
* @param conf - Configuration
* @param state - ContainerLifeCycleState
* @param
*/
public ContainerData(long containerID, Configuration conf,
ContainerLifeCycleState state) {
this.metadata = new TreeMap<>();
this.maxSize = conf.getLong(ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_KEY,
ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_DEFAULT) * OzoneConsts.GB;
this.bytesUsed = new AtomicLong(0L);
this.containerID = containerID;
this.state = state;
this.numPendingDeletionBlocks = 0;
this.deleteTransactionId = 0;
this.readCount = new AtomicLong(0L);
this.readBytes = new AtomicLong(0L);
this.writeCount = new AtomicLong(0L);
this.writeBytes = new AtomicLong(0L);
}
/**
* Constructs a ContainerData object from ProtoBuf classes.
*
* @param protoData - ProtoBuf Message
* @throws IOException
*/
public static ContainerData getFromProtBuf(
ContainerProtos.ContainerData protoData, Configuration conf)
throws IOException {
ContainerData data = new ContainerData(
protoData.getContainerID(), conf);
for (int x = 0; x < protoData.getMetadataCount(); x++) {
data.addMetadata(protoData.getMetadata(x).getKey(),
protoData.getMetadata(x).getValue());
}
if (protoData.hasContainerPath()) {
data.setContainerPath(protoData.getContainerPath());
}
if (protoData.hasDbPath()) {
data.setDBPath(protoData.getDbPath());
}
if (protoData.hasState()) {
data.setState(protoData.getState());
}
if (protoData.hasBytesUsed()) {
data.setBytesUsed(protoData.getBytesUsed());
}
if (protoData.hasSize()) {
data.setMaxSize(protoData.getSize());
}
if(protoData.hasContainerType()) {
data.setContainerType(protoData.getContainerType());
}
if(protoData.hasContainerDBType()) {
data.setContainerDBType(protoData.getContainerDBType());
}
return data;
}
public String getContainerDBType() {
return containerDBType;
}
public void setContainerDBType(String containerDBType) {
this.containerDBType = containerDBType;
}
/**
* Returns a ProtoBuf Message from ContainerData.
*
* @return Protocol Buffer Message
*/
public ContainerProtos.ContainerData getProtoBufMessage() {
ContainerProtos.ContainerData.Builder builder = ContainerProtos
.ContainerData.newBuilder();
builder.setContainerID(this.getContainerID());
if (this.getDBPath() != null) {
builder.setDbPath(this.getDBPath());
}
if (this.getContainerPath() != null) {
builder.setContainerPath(this.getContainerPath());
}
builder.setState(this.getState());
for (Map.Entry<String, String> entry : metadata.entrySet()) {
ContainerProtos.KeyValue.Builder keyValBuilder =
ContainerProtos.KeyValue.newBuilder();
builder.addMetadata(keyValBuilder.setKey(entry.getKey())
.setValue(entry.getValue()).build());
}
if (this.getBytesUsed() >= 0) {
builder.setBytesUsed(this.getBytesUsed());
}
if (this.getKeyCount() >= 0) {
builder.setKeyCount(this.getKeyCount());
}
if (this.getMaxSize() >= 0) {
builder.setSize(this.getMaxSize());
}
if(this.getContainerType() != null) {
builder.setContainerType(containerType);
}
if(this.getContainerDBType() != null) {
builder.setContainerDBType(containerDBType);
}
return builder.build();
}
public void setContainerType(ContainerType containerType) {
this.containerType = containerType;
}
public ContainerType getContainerType() {
return this.containerType;
}
/**
* Adds metadata.
*/
public void addMetadata(String key, String value) throws IOException {
synchronized (this.metadata) {
if (this.metadata.containsKey(key)) {
throw new IOException("This key already exists. Key " + key);
}
metadata.put(key, value);
}
}
/**
* Returns all metadata.
*/
public Map<String, String> getAllMetadata() {
synchronized (this.metadata) {
return Collections.unmodifiableMap(this.metadata);
}
}
/**
* Returns value of a key.
*/
public String getValue(String key) {
synchronized (this.metadata) {
return metadata.get(key);
}
}
/**
* Deletes a metadata entry from the map.
*
* @param key - Key
*/
public void deleteKey(String key) {
synchronized (this.metadata) {
metadata.remove(key);
}
}
/**
* Returns path.
*
* @return - path
*/
public String getDBPath() {
return dbPath;
}
/**
* Sets path.
*
* @param path - String.
*/
public void setDBPath(String path) {
this.dbPath = path;
}
/**
* This function serves as the generic key for ContainerCache class. Both
* ContainerData and ContainerKeyData overrides this function to appropriately
* return the right name that can be used in ContainerCache.
*
* @return String Name.
*/
// TODO: check the ContainerCache class to see if
// we are using the ContainerID instead.
/*
public String getName() {
return getContainerID();
}*/
/**
* Get container file path.
* @return - Physical path where container file and checksum is stored.
*/
public String getContainerPath() {
return containerFilePath;
}
/**
* Set container Path.
* @param containerPath - File path.
*/
public void setContainerPath(String containerPath) {
this.containerFilePath = containerPath;
}
/**
* Get container ID.
* @return - container ID.
*/
public synchronized long getContainerID() {
return containerID;
}
public synchronized void setState(ContainerLifeCycleState state) {
this.state = state;
}
public synchronized ContainerLifeCycleState getState() {
return this.state;
}
/**
* checks if the container is open.
* @return - boolean
*/
public synchronized boolean isOpen() {
return ContainerLifeCycleState.OPEN == state;
}
/**
* checks if the container is invalid.
* @return - boolean
*/
public boolean isValid() {
return !(ContainerLifeCycleState.INVALID == state);
}
/**
* checks if the container is closed.
* @return - boolean
*/
public synchronized boolean isClosed() {
return ContainerLifeCycleState.CLOSED == state;
}
/**
* Marks this container as closed.
*/
public synchronized void closeContainer() {
// TODO: closed or closing here
setState(ContainerLifeCycleState.CLOSED);
}
public void setMaxSize(long maxSize) {
this.maxSize = maxSize;
}
public long getMaxSize() {
return maxSize;
}
public long getKeyCount() {
return metadata.size();
}
public void setBytesUsed(long used) {
this.bytesUsed.set(used);
}
/**
* Get the number of bytes used by the container.
* @return the number of bytes used by the container.
*/
public long getBytesUsed() {
return bytesUsed.get();
}
/**
* Increase the number of bytes used by the container.
* @param used number of bytes used by the container.
* @return the current number of bytes used by the container afert increase.
*/
public long incrBytesUsed(long used) {
return this.bytesUsed.addAndGet(used);
}
/**
* Decrease the number of bytes used by the container.
* @param reclaimed the number of bytes reclaimed from the container.
* @return the current number of bytes used by the container after decrease.
*/
public long decrBytesUsed(long reclaimed) {
return this.bytesUsed.addAndGet(-1L * reclaimed);
}
/**
* Increase the count of pending deletion blocks.
*
* @param numBlocks increment number
*/
public void incrPendingDeletionBlocks(int numBlocks) {
this.numPendingDeletionBlocks += numBlocks;
}
/**
* Decrease the count of pending deletion blocks.
*
* @param numBlocks decrement number
*/
public void decrPendingDeletionBlocks(int numBlocks) {
this.numPendingDeletionBlocks -= numBlocks;
}
/**
* Get the number of pending deletion blocks.
*/
public int getNumPendingDeletionBlocks() {
return this.numPendingDeletionBlocks;
}
/**
* Sets deleteTransactionId to latest delete transactionId for the container.
*
* @param transactionId latest transactionId of the container.
*/
public void updateDeleteTransactionId(long transactionId) {
deleteTransactionId = max(transactionId, deleteTransactionId);
}
/**
* Return the latest deleteTransactionId of the container.
*/
public long getDeleteTransactionId() {
return deleteTransactionId;
}
/**
* Get the number of bytes read from the container.
* @return the number of bytes read from the container.
*/
public long getReadBytes() {
return readBytes.get();
}
/**
* Increase the number of bytes read from the container.
* @param bytes number of bytes read.
*/
public void incrReadBytes(long bytes) {
this.readBytes.addAndGet(bytes);
}
/**
* Get the number of times the container is read.
* @return the number of times the container is read.
*/
public long getReadCount() {
return readCount.get();
}
/**
* Increase the number of container read count by 1.
*/
public void incrReadCount() {
this.readCount.incrementAndGet();
}
/**
* Get the number of bytes write into the container.
* @return the number of bytes write into the container.
*/
public long getWriteBytes() {
return writeBytes.get();
}
/**
* Increase the number of bytes write into the container.
* @param bytes the number of bytes write into the container.
*/
public void incrWriteBytes(long bytes) {
this.writeBytes.addAndGet(bytes);
}
/**
* Get the number of writes into the container.
* @return the number of writes into the container.
*/
public long getWriteCount() {
return writeCount.get();
}
/**
* Increase the number of writes into the container by 1.
*/
public void incrWriteCount() {
this.writeCount.incrementAndGet();
}
}

View File

@ -19,8 +19,6 @@
package org.apache.hadoop.ozone.container.common.helpers;
import com.google.common.base.Preconditions;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@ -34,10 +32,8 @@ import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.ozone.OzoneConsts;
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.utils.MetadataStore;
import org.apache.hadoop.utils.MetadataStoreBuilder;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -49,14 +45,6 @@ import java.nio.file.Path;
import java.nio.file.Paths;
import static org.apache.commons.io.FilenameUtils.removeExtension;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.CLOSED_CONTAINER_IO;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.INVALID_CONTAINER_STATE;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.SUCCESS;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.UNABLE_TO_FIND_DATA_DIR;
/**
* A set of helper functions to create proper responses.
@ -203,83 +191,23 @@ public final class ContainerUtils {
* @param containerFile - Container File to verify
* @throws IOException
*/
public static void verifyIsNewContainer(File containerFile)
throws IOException {
Logger log = LoggerFactory.getLogger(ContainerManagerImpl.class);
if (containerFile.exists()) {
log.error("container already exists on disk. File: {}",
containerFile.toPath());
public static void verifyIsNewContainer(File containerFile) throws
FileAlreadyExistsException {
Logger log = LoggerFactory.getLogger(ContainerSet.class);
Preconditions.checkNotNull(containerFile, "containerFile Should not be " +
"null");
if (containerFile.getParentFile().exists()) {
log.error("Container already exists on disk. File: {}", containerFile
.toPath());
throw new FileAlreadyExistsException("container already exists on " +
"disk.");
}
File parentPath = new File(containerFile.getParent());
if (!parentPath.exists() && !parentPath.mkdirs()) {
log.error("Unable to create parent path. Path: {}",
parentPath.toString());
throw new IOException("Unable to create container directory.");
}
if (!containerFile.createNewFile()) {
log.error("creation of a new container file failed. File: {}",
containerFile.toPath());
throw new IOException("creation of a new container file failed.");
}
}
public static String getContainerDbFileName(String containerName) {
return containerName + OzoneConsts.DN_CONTAINER_DB;
}
/**
* creates a Metadata DB for the specified container.
*
* @param containerPath - Container Path.
* @throws IOException
*/
public static Path createMetadata(Path containerPath, String containerName,
Configuration conf)
throws IOException {
Logger log = LoggerFactory.getLogger(ContainerManagerImpl.class);
Preconditions.checkNotNull(containerPath);
Path metadataPath = containerPath.resolve(OzoneConsts.CONTAINER_META_PATH);
if (!metadataPath.toFile().mkdirs()) {
log.error("Unable to create directory for metadata storage. Path: {}",
metadataPath);
throw new IOException("Unable to create directory for metadata storage." +
" Path: " + metadataPath);
}
MetadataStore store = MetadataStoreBuilder.newBuilder()
.setConf(conf)
.setCreateIfMissing(true)
.setDbFile(metadataPath
.resolve(getContainerDbFileName(containerName)).toFile())
.build();
// we close since the SCM pre-creates containers.
// we will open and put Db handle into a cache when keys are being created
// in a container.
store.close();
Path dataPath = containerPath.resolve(OzoneConsts.CONTAINER_DATA_PATH);
if (!dataPath.toFile().mkdirs()) {
// If we failed to create data directory, we cleanup the
// metadata directory completely. That is, we will delete the
// whole directory including LevelDB file.
log.error("Unable to create directory for data storage. cleaning up the" +
" container path: {} dataPath: {}",
containerPath, dataPath);
FileUtils.deleteDirectory(containerPath.toFile());
throw new IOException("Unable to create directory for data storage." +
" Path: " + dataPath);
}
return metadataPath;
}
/**
* Returns container file location.
*
@ -294,93 +222,6 @@ public final class ContainerUtils {
.toFile();
}
/**
* Container metadata directory -- here is where the level DB lives.
*
* @param cData - cData.
* @return Path to the parent directory where the DB lives.
*/
public static Path getMetadataDirectory(ContainerData cData) {
Path dbPath = Paths.get(cData.getDBPath());
Preconditions.checkNotNull(dbPath);
Preconditions.checkState(dbPath.toString().length() > 0);
return dbPath.getParent();
}
/**
* Returns the path where data or chunks live for a given container.
*
* @param cData - cData container
* @return - Path
* @throws StorageContainerException
*/
public static Path getDataDirectory(ContainerData cData)
throws StorageContainerException {
Path path = getMetadataDirectory(cData);
Preconditions.checkNotNull(path);
Path parentPath = path.getParent();
if (parentPath == null) {
throw new StorageContainerException("Unable to get Data directory."
+ path, UNABLE_TO_FIND_DATA_DIR);
}
return parentPath.resolve(OzoneConsts.CONTAINER_DATA_PATH);
}
/**
* remove Container if it is empty.
* <p/>
* There are three things we need to delete.
* <p/>
* 1. Container file and metadata file. 2. The Level DB file 3. The path that
* we created on the data location.
*
* @param containerData - Data of the container to remove.
* @param conf - configuration of the cluster.
* @param forceDelete - whether this container should be deleted forcibly.
* @throws IOException
*/
public static void removeContainer(ContainerData containerData,
Configuration conf, boolean forceDelete) throws IOException {
Preconditions.checkNotNull(containerData);
Path dbPath = Paths.get(containerData.getDBPath());
MetadataStore db = KeyUtils.getDB(containerData, conf);
// If the container is not empty and cannot be deleted forcibly,
// then throw a SCE to stop deleting.
if(!forceDelete && !db.isEmpty()) {
throw new StorageContainerException(
"Container cannot be deleted because it is not empty.",
Result.ERROR_CONTAINER_NOT_EMPTY);
}
// Close the DB connection and remove the DB handler from cache
KeyUtils.removeDB(containerData, conf);
// Delete the DB File.
FileUtils.forceDelete(dbPath.toFile());
dbPath = dbPath.getParent();
// Delete all Metadata in the Data directories for this containers.
if (dbPath != null) {
FileUtils.deleteDirectory(dbPath.toFile());
dbPath = dbPath.getParent();
}
// now delete the container directory, this means that all key data dirs
// will be removed too.
if (dbPath != null) {
FileUtils.deleteDirectory(dbPath.toFile());
}
// Delete the container metadata from the metadata locations.
String rootPath = getContainerNameFromFile(new File(containerData
.getContainerPath()));
Path containerPath = Paths.get(rootPath.concat(CONTAINER_EXTENSION));
FileUtils.forceDelete(containerPath.toFile());
}
/**
* Persistent a {@link DatanodeDetails} to a local file.
*

View File

@ -1,148 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.helpers;
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
import org.apache.hadoop.utils.MetadataStore;
import java.io.IOException;
import java.nio.charset.Charset;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.NO_SUCH_KEY;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.UNABLE_TO_READ_METADATA_DB;
/**
* Utils functions to help key functions.
*/
public final class KeyUtils {
public static final String ENCODING_NAME = "UTF-8";
public static final Charset ENCODING = Charset.forName(ENCODING_NAME);
/**
* Never Constructed.
*/
private KeyUtils() {
}
/**
* Get a DB handler for a given container.
* If the handler doesn't exist in cache yet, first create one and
* add into cache. This function is called with containerManager
* ReadLock held.
*
* @param container container.
* @param conf configuration.
* @return MetadataStore handle.
* @throws StorageContainerException
*/
public static MetadataStore getDB(ContainerData container,
Configuration conf) throws StorageContainerException {
Preconditions.checkNotNull(container);
ContainerCache cache = ContainerCache.getInstance(conf);
Preconditions.checkNotNull(cache);
try {
return cache.getDB(container.getContainerID(), container
.getContainerDBType(), container.getDBPath());
} catch (IOException ex) {
String message =
String.format("Unable to open DB. DB Name: %s, Path: %s. ex: %s",
container.getContainerID(), container.getDBPath(), ex.getMessage());
throw new StorageContainerException(message, UNABLE_TO_READ_METADATA_DB);
}
}
/**
* Remove a DB handler from cache.
*
* @param container - Container data.
* @param conf - Configuration.
*/
public static void removeDB(ContainerData container,
Configuration conf) {
Preconditions.checkNotNull(container);
ContainerCache cache = ContainerCache.getInstance(conf);
Preconditions.checkNotNull(cache);
cache.removeDB(container.getContainerID());
}
/**
* Shutdown all DB Handles.
*
* @param cache - Cache for DB Handles.
*/
@SuppressWarnings("unchecked")
public static void shutdownCache(ContainerCache cache) {
cache.shutdownCache();
}
/**
* Returns successful keyResponse.
* @param msg - Request.
* @return Response.
*/
public static ContainerProtos.ContainerCommandResponseProto
getKeyResponse(ContainerProtos.ContainerCommandRequestProto msg) {
return ContainerUtils.getSuccessResponse(msg);
}
public static ContainerProtos.ContainerCommandResponseProto
getKeyDataResponse(ContainerProtos.ContainerCommandRequestProto msg,
KeyData data) {
ContainerProtos.GetKeyResponseProto.Builder getKey = ContainerProtos
.GetKeyResponseProto.newBuilder();
getKey.setKeyData(data.getProtoBufMessage());
ContainerProtos.ContainerCommandResponseProto.Builder builder =
ContainerUtils.getSuccessResponseBuilder(msg);
builder.setGetKey(getKey);
return builder.build();
}
/**
* Parses the key name from a bytes array.
* @param bytes key name in bytes.
* @return key name string.
*/
public static String getKeyName(byte[] bytes) {
return new String(bytes, ENCODING);
}
/**
* Parses the {@link KeyData} from a bytes array.
*
* @param bytes key data in bytes.
* @return key data.
* @throws IOException if the bytes array is malformed or invalid.
*/
public static KeyData getKeyData(byte[] bytes) throws IOException {
try {
ContainerProtos.KeyData kd = ContainerProtos.KeyData.parseFrom(bytes);
KeyData data = KeyData.getFromProtoBuf(kd);
return data;
} catch (IOException e) {
throw new StorageContainerException("Failed to parse key data from the" +
" bytes array.", NO_SUCH_KEY);
}
}
}

View File

@ -1,233 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.impl;
import com.google.common.base.Preconditions;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.hadoop.ozone.container.common.helpers.ChunkUtils;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.interfaces.ChunkManager;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.file.Files;
import java.nio.file.StandardCopyOption;
import java.security.NoSuchAlgorithmException;
import java.util.concurrent.ExecutionException;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.CONTAINER_INTERNAL_ERROR;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.UNSUPPORTED_REQUEST;
/**
* An implementation of ChunkManager that is used by default in ozone.
*/
public class ChunkManagerImpl implements ChunkManager {
static final Logger LOG =
LoggerFactory.getLogger(ChunkManagerImpl.class);
private final ContainerManager containerManager;
/**
* Constructs a ChunkManager.
*
* @param manager - ContainerManager.
*/
public ChunkManagerImpl(ContainerManager manager) {
this.containerManager = manager;
}
/**
* writes a given chunk.
*
* @param blockID - ID of the block.
* @param info - ChunkInfo.
* @throws StorageContainerException
*/
@Override
public void writeChunk(BlockID blockID, ChunkInfo info,
byte[] data, ContainerProtos.Stage stage)
throws StorageContainerException {
// we don't want container manager to go away while we are writing chunks.
containerManager.readLock();
// TODO : Take keyManager Write lock here.
try {
Preconditions.checkNotNull(blockID, "Block ID cannot be null.");
long containerID = blockID.getContainerID();
Preconditions.checkState(containerID >= 0,
"Container ID cannot be negative");
ContainerData container =
containerManager.readContainer(containerID);
File chunkFile = ChunkUtils.validateChunk(container, info);
File tmpChunkFile = getTmpChunkFile(chunkFile, info);
LOG.debug("writing chunk:{} chunk stage:{} chunk file:{} tmp chunk file",
info.getChunkName(), stage, chunkFile, tmpChunkFile);
switch (stage) {
case WRITE_DATA:
ChunkUtils.writeData(tmpChunkFile, info, data);
break;
case COMMIT_DATA:
commitChunk(tmpChunkFile, chunkFile, containerID, info.getLen());
break;
case COMBINED:
// directly write to the chunk file
long oldSize = chunkFile.length();
ChunkUtils.writeData(chunkFile, info, data);
long newSize = chunkFile.length();
containerManager.incrBytesUsed(containerID, newSize - oldSize);
containerManager.incrWriteCount(containerID);
containerManager.incrWriteBytes(containerID, info.getLen());
break;
default:
throw new IOException("Can not identify write operation.");
}
} catch (ExecutionException | NoSuchAlgorithmException | IOException e) {
LOG.error("write data failed. error: {}", e);
throw new StorageContainerException("Internal error: ", e,
CONTAINER_INTERNAL_ERROR);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
LOG.error("write data failed. error: {}", e);
throw new StorageContainerException("Internal error: ", e,
CONTAINER_INTERNAL_ERROR);
} finally {
containerManager.readUnlock();
}
}
// Create a temporary file in the same container directory
// in the format "<chunkname>.tmp"
private static File getTmpChunkFile(File chunkFile, ChunkInfo info)
throws StorageContainerException {
return new File(chunkFile.getParent(),
chunkFile.getName() +
OzoneConsts.CONTAINER_CHUNK_NAME_DELIMITER +
OzoneConsts.CONTAINER_TEMPORARY_CHUNK_PREFIX);
}
// Commit the chunk by renaming the temporary chunk file to chunk file
private void commitChunk(File tmpChunkFile, File chunkFile,
long containerID, long chunkLen) throws IOException {
long sizeDiff = tmpChunkFile.length() - chunkFile.length();
// It is safe to replace here as the earlier chunk if existing should be
// caught as part of validateChunk
Files.move(tmpChunkFile.toPath(), chunkFile.toPath(),
StandardCopyOption.REPLACE_EXISTING);
containerManager.incrBytesUsed(containerID, sizeDiff);
containerManager.incrWriteCount(containerID);
containerManager.incrWriteBytes(containerID, chunkLen);
}
/**
* reads the data defined by a chunk.
*
* @param blockID - ID of the block.
* @param info - ChunkInfo.
* @return byte array
* @throws StorageContainerException
* TODO: Right now we do not support partial reads and writes of chunks.
* TODO: Explore if we need to do that for ozone.
*/
@Override
public byte[] readChunk(BlockID blockID, ChunkInfo info)
throws StorageContainerException {
containerManager.readLock();
try {
Preconditions.checkNotNull(blockID, "Block ID cannot be null.");
long containerID = blockID.getContainerID();
Preconditions.checkState(containerID >= 0,
"Container ID cannot be negative");
ContainerData container =
containerManager.readContainer(containerID);
File chunkFile = ChunkUtils.getChunkFile(container, info);
ByteBuffer data = ChunkUtils.readData(chunkFile, info);
containerManager.incrReadCount(containerID);
containerManager.incrReadBytes(containerID, chunkFile.length());
return data.array();
} catch (ExecutionException | NoSuchAlgorithmException e) {
LOG.error("read data failed. error: {}", e);
throw new StorageContainerException("Internal error: ",
e, CONTAINER_INTERNAL_ERROR);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
LOG.error("read data failed. error: {}", e);
throw new StorageContainerException("Internal error: ",
e, CONTAINER_INTERNAL_ERROR);
} finally {
containerManager.readUnlock();
}
}
/**
* Deletes a given chunk.
*
* @param blockID - ID of the block.
* @param info - Chunk Info
* @throws StorageContainerException
*/
@Override
public void deleteChunk(BlockID blockID, ChunkInfo info)
throws StorageContainerException {
containerManager.readLock();
try {
Preconditions.checkNotNull(blockID, "Block ID cannot be null.");
long containerID = blockID.getContainerID();
Preconditions.checkState(containerID >= 0,
"Container ID cannot be negative");
File chunkFile = ChunkUtils.getChunkFile(containerManager
.readContainer(containerID), info);
if ((info.getOffset() == 0) && (info.getLen() == chunkFile.length())) {
FileUtil.fullyDelete(chunkFile);
containerManager.decrBytesUsed(containerID, chunkFile.length());
} else {
LOG.error("Not Supported Operation. Trying to delete a " +
"chunk that is in shared file. chunk info : " + info.toString());
throw new StorageContainerException("Not Supported Operation. " +
"Trying to delete a chunk that is in shared file. chunk info : "
+ info.toString(), UNSUPPORTED_REQUEST);
}
} finally {
containerManager.readUnlock();
}
}
/**
* Shutdown the chunkManager.
*
* In the chunkManager we haven't acquired any resources, so nothing to do
* here. This call is made with containerManager Writelock held.
*/
@Override
public void shutdown() {
Preconditions.checkState(this.containerManager.hasWriteLock());
}
}

View File

@ -17,6 +17,7 @@
*/
package org.apache.hadoop.ozone.container.common.impl;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
ContainerType;
@ -28,6 +29,7 @@ import java.io.IOException;
import java.util.Collections;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
/**
@ -41,7 +43,10 @@ public class ContainerData {
private final ContainerType containerType;
// Unique identifier for the container
private final long containerId;
private final long containerID;
// Path to container root dir.
private String containerPath;
// Layout version of the container data
private final int layOutVersion;
@ -65,6 +70,10 @@ public class ContainerData {
private HddsVolume volume;
/**
* Number of pending deletion blocks in container.
*/
private final AtomicInteger numPendingDeletionBlocks;
/**
* Creates a ContainerData Object, which holds metadata of the container.
@ -73,18 +82,8 @@ public class ContainerData {
* @param size - container maximum size
*/
public ContainerData(ContainerType type, long containerId, int size) {
this.containerType = type;
this.containerId = containerId;
this.layOutVersion = ChunkLayOutVersion.getLatestVersion().getVersion();
this.metadata = new TreeMap<>();
this.state = ContainerLifeCycleState.OPEN;
this.readCount = new AtomicLong(0L);
this.readBytes = new AtomicLong(0L);
this.writeCount = new AtomicLong(0L);
this.writeBytes = new AtomicLong(0L);
this.bytesUsed = new AtomicLong(0L);
this.keyCount = new AtomicLong(0L);
this.maxSizeGB = size;
this(type, containerId,
ChunkLayOutVersion.getLatestVersion().getVersion(), size);
}
/**
@ -94,10 +93,12 @@ public class ContainerData {
* @param layOutVersion - Container layOutVersion
* @param size - Container maximum size
*/
public ContainerData(ContainerType type, long containerId, int
layOutVersion, int size) {
public ContainerData(ContainerType type, long containerId,
int layOutVersion, int size) {
Preconditions.checkNotNull(type);
this.containerType = type;
this.containerId = containerId;
this.containerID = containerId;
this.layOutVersion = layOutVersion;
this.metadata = new TreeMap<>();
this.state = ContainerLifeCycleState.OPEN;
@ -108,13 +109,30 @@ public class ContainerData {
this.bytesUsed = new AtomicLong(0L);
this.keyCount = new AtomicLong(0L);
this.maxSizeGB = size;
this.numPendingDeletionBlocks = new AtomicInteger(0);
}
/**
* Returns the containerId.
* Returns the containerID.
*/
public long getContainerId() {
return containerId;
public long getContainerID() {
return containerID;
}
/**
* Returns the path to base dir of the container.
* @return Path to base dir.
*/
public String getContainerPath() {
return containerPath;
}
/**
* Set the base dir path of the container.
* @param baseDir path to base dir
*/
public void setContainerPath(String baseDir) {
this.containerPath = baseDir;
}
/**
@ -163,9 +181,6 @@ public class ContainerData {
*/
public void addMetadata(String key, String value) throws IOException {
synchronized (this.metadata) {
if (this.metadata.containsKey(key)) {
throw new IOException("This key already exists. Key " + key);
}
metadata.put(key, value);
}
}
@ -299,7 +314,6 @@ public class ContainerData {
return this.bytesUsed.addAndGet(used);
}
/**
* Decrease the number of bytes used by the container.
* @param reclaimed the number of bytes reclaimed from the container.
@ -356,4 +370,75 @@ public class ContainerData {
this.keyCount.set(count);
}
/**
* Returns container metadata path.
*/
public String getMetadataPath() {
return null;
}
/**
* Returns container data path.
*/
public String getDataPath() {
return null;
}
/**
* Increase the count of pending deletion blocks.
*
* @param numBlocks increment number
*/
public void incrPendingDeletionBlocks(int numBlocks) {
this.numPendingDeletionBlocks.addAndGet(numBlocks);
}
/**
* Decrease the count of pending deletion blocks.
*
* @param numBlocks decrement number
*/
public void decrPendingDeletionBlocks(int numBlocks) {
this.numPendingDeletionBlocks.addAndGet(-1 * numBlocks);
}
/**
* Get the number of pending deletion blocks.
*/
public int getNumPendingDeletionBlocks() {
return this.numPendingDeletionBlocks.get();
}
/**
* Returns a ProtoBuf Message from ContainerData.
*
* @return Protocol Buffer Message
*/
public ContainerProtos.ContainerData getProtoBufMessage() {
ContainerProtos.ContainerData.Builder builder =
ContainerProtos.ContainerData.newBuilder();
builder.setContainerID(this.getContainerID());
if (this.containerPath != null) {
builder.setContainerPath(this.containerPath);
}
builder.setState(this.getState());
for (Map.Entry<String, String> entry : metadata.entrySet()) {
ContainerProtos.KeyValue.Builder keyValBuilder =
ContainerProtos.KeyValue.newBuilder();
builder.addMetadata(keyValBuilder.setKey(entry.getKey())
.setValue(entry.getValue()).build());
}
if (this.getBytesUsed() >= 0) {
builder.setBytesUsed(this.getBytesUsed());
}
builder.setContainerType(containerType);
return builder.build();
}
}

View File

@ -21,6 +21,8 @@ package org.apache.hadoop.ozone.container.common.impl;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.yaml.snakeyaml.Yaml;
@ -48,7 +50,8 @@ import org.yaml.snakeyaml.nodes.ScalarNode;
import org.yaml.snakeyaml.nodes.Tag;
import org.yaml.snakeyaml.representer.Representer;
import static org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData.YAML_TAG;
import static org.apache.hadoop.ozone.container.keyvalue
.KeyValueContainerData.KEYVALUE_YAML_TAG;
/**
* Class for creating and reading .container files.
@ -84,7 +87,7 @@ public final class ContainerDataYaml {
Representer representer = new ContainerDataRepresenter();
representer.setPropertyUtils(propertyUtils);
representer.addClassTag(KeyValueContainerData.class,
KeyValueContainerData.YAML_TAG);
KeyValueContainerData.KEYVALUE_YAML_TAG);
Constructor keyValueDataConstructor = new ContainerDataConstructor();
@ -171,7 +174,8 @@ public final class ContainerDataYaml {
//Adding our own specific constructors for tags.
// When a new Container type is added, we need to add yamlConstructor
// for that
this.yamlConstructors.put(YAML_TAG, new ConstructKeyValueContainerData());
this.yamlConstructors.put(
KEYVALUE_YAML_TAG, new ConstructKeyValueContainerData());
this.yamlConstructors.put(Tag.INT, new ConstructLong());
}
@ -181,20 +185,21 @@ public final class ContainerDataYaml {
Map<Object, Object> nodes = constructMapping(mnode);
//Needed this, as TAG.INT type is by default converted to Long.
long layOutVersion = (long) nodes.get("layOutVersion");
long layOutVersion = (long) nodes.get(OzoneConsts.LAYOUTVERSION);
int lv = (int) layOutVersion;
long size = (long) nodes.get("maxSizeGB");
long size = (long) nodes.get(OzoneConsts.MAX_SIZE_GB);
int maxSize = (int) size;
//When a new field is added, it needs to be added here.
KeyValueContainerData kvData = new KeyValueContainerData((long) nodes
.get("containerId"), lv, maxSize);
kvData.setContainerDBType((String)nodes.get("containerDBType"));
.get(OzoneConsts.CONTAINER_ID), lv, maxSize);
kvData.setContainerDBType((String)nodes.get(
OzoneConsts.CONTAINER_DB_TYPE));
kvData.setMetadataPath((String) nodes.get(
"metadataPath"));
kvData.setChunksPath((String) nodes.get("chunksPath"));
Map<String, String> meta = (Map) nodes.get("metadata");
OzoneConsts.METADATA_PATH));
kvData.setChunksPath((String) nodes.get(OzoneConsts.CHUNKS_PATH));
Map<String, String> meta = (Map) nodes.get(OzoneConsts.METADATA);
meta.forEach((key, val) -> {
try {
kvData.addMetadata(key, val);
@ -204,7 +209,7 @@ public final class ContainerDataYaml {
"for containerId " + (long) nodes.get("containerId"));
}
});
String state = (String) nodes.get("state");
String state = (String) nodes.get(OzoneConsts.STATE);
switch (state) {
case "OPEN":
kvData.setState(ContainerProtos.ContainerLifeCycleState.OPEN);
@ -218,7 +223,7 @@ public final class ContainerDataYaml {
default:
throw new IllegalStateException("Unexpected " +
"ContainerLifeCycleState " + state + " for the containerId " +
(long) nodes.get("containerId"));
(long) nodes.get(OzoneConsts.CONTAINER_ID));
}
return kvData;
}

View File

@ -1,158 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.impl;
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.interfaces
.ContainerLocationManager;
import org.apache.hadoop.ozone.container.common.interfaces
.ContainerLocationManagerMXBean;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.management.ObjectName;
import java.io.IOException;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.LinkedList;
import java.util.List;
/**
* A class that tells the ContainerManager where to place the containers.
* Please note : There is *no* one-to-one correlation between metadata
* Locations and data Locations.
*
* For example : A user could map all container files to a
* SSD but leave data/metadata on bunch of other disks.
*/
public class ContainerLocationManagerImpl implements ContainerLocationManager,
ContainerLocationManagerMXBean {
private static final Logger LOG =
LoggerFactory.getLogger(ContainerLocationManagerImpl.class);
private final List<ContainerStorageLocation> dataLocations;
private int currentIndex;
private final List<StorageLocation> metadataLocations;
private final ObjectName jmxbean;
/**
* Constructs a Location Manager.
* @param metadataLocations - Refers to the metadataLocations
* where we store the container metadata.
* @param dataDirs - metadataLocations where we store the actual
* data or chunk files.
* @param conf - configuration.
* @throws IOException
*/
public ContainerLocationManagerImpl(List<StorageLocation> metadataLocations,
List<StorageLocation> dataDirs, Configuration conf)
throws IOException {
dataLocations = new LinkedList<>();
for (StorageLocation dataDir : dataDirs) {
dataLocations.add(new ContainerStorageLocation(dataDir, conf));
}
this.metadataLocations = metadataLocations;
jmxbean = MBeans.register("OzoneDataNode",
ContainerLocationManager.class.getSimpleName(), this);
}
/**
* Returns the path where the container should be placed from a set of
* metadataLocations.
*
* @return A path where we should place this container and metadata.
* @throws IOException
*/
@Override
public Path getContainerPath()
throws IOException {
Preconditions.checkState(metadataLocations.size() > 0);
int index = currentIndex % metadataLocations.size();
return Paths.get(metadataLocations.get(index).getNormalizedUri());
}
/**
* Returns the path where the container Data file are stored.
*
* @return a path where we place the LevelDB and data files of a container.
* @throws IOException
*/
@Override
public Path getDataPath(String containerName) throws IOException {
Path currentPath = Paths.get(
dataLocations.get(currentIndex++ % dataLocations.size())
.getNormalizedUri());
currentPath = currentPath.resolve(OzoneConsts.CONTAINER_PREFIX);
return currentPath.resolve(containerName);
}
@Override
public StorageLocationReport[] getLocationReport() throws IOException {
boolean failed;
StorageLocationReport[] reports =
new StorageLocationReport[dataLocations.size()];
for (int idx = 0; idx < dataLocations.size(); idx++) {
ContainerStorageLocation loc = dataLocations.get(idx);
long scmUsed = 0;
long remaining = 0;
failed = false;
try {
scmUsed = loc.getScmUsed();
remaining = loc.getAvailable();
} catch (IOException ex) {
LOG.warn("Failed to get scmUsed and remaining for container " +
"storage location {}", loc.getNormalizedUri());
// reset scmUsed and remaining if df/du failed.
scmUsed = 0;
remaining = 0;
failed = true;
}
StorageLocationReport.Builder builder =
StorageLocationReport.newBuilder();
builder.setStorageLocation(loc.getStorageLocation())
.setId(loc.getStorageUuId())
.setFailed(failed)
.setCapacity(loc.getCapacity())
.setRemaining(remaining)
.setScmUsed(scmUsed)
.setStorageType(loc.getStorageType());
StorageLocationReport r = builder.build();
reports[idx] = r;
}
return reports;
}
/**
* Supports clean shutdown of container location du threads.
*
* @throws IOException
*/
@Override
public void shutdown() throws IOException {
for (ContainerStorageLocation loc: dataLocations) {
loc.shutdown();
}
MBeans.unregister(jmxbean);
}
}

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.container.common.impl;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto
@ -63,7 +64,7 @@ public class ContainerSet {
StorageContainerException {
Preconditions.checkNotNull(container, "container cannot be null");
long containerId = container.getContainerData().getContainerId();
long containerId = container.getContainerData().getContainerID();
if(containerMap.putIfAbsent(containerId, container) == null) {
LOG.debug("Container with container Id {} is added to containerMap",
containerId);
@ -133,6 +134,13 @@ public class ContainerSet {
return containerMap.entrySet().iterator();
}
/**
* Return a copy of the containerMap
* @return containerMap
*/
public Map<Long, Container> getContainerMap() {
return ImmutableMap.copyOf(containerMap);
}
/**
* A simple interface for container Iterations.
@ -196,7 +204,7 @@ public class ContainerSet {
for (Container container: containers) {
long containerId = container.getContainerData().getContainerId();
long containerId = container.getContainerData().getContainerID();
ContainerInfo.Builder ciBuilder = ContainerInfo.newBuilder();
ContainerData containerData = container.getContainerData();
ciBuilder.setContainerID(containerId)
@ -234,9 +242,14 @@ public class ContainerSet {
break;
default:
throw new StorageContainerException("Invalid Container state found: " +
containerData.getContainerId(), INVALID_CONTAINER_STATE);
containerData.getContainerID(), INVALID_CONTAINER_STATE);
}
return state;
}
// TODO: Implement BlockDeletingService
public List<ContainerData> chooseContainerForBlockDeletion(
int count) throws StorageContainerException {
return null;
}
}

View File

@ -1,212 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.impl;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CachingGetSpaceUsed;
import org.apache.hadoop.fs.DF;
import org.apache.hadoop.fs.GetSpaceUsed;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.util.ShutdownHookManager;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStreamWriter;
import java.net.URI;
import java.nio.charset.StandardCharsets;
import java.nio.file.Paths;
import java.util.Scanner;
import static org.apache.hadoop.util.RunJar.SHUTDOWN_HOOK_PRIORITY;
/**
* Class that wraps the space usage of the Datanode Container Storage Location
* by SCM containers.
*/
public class ContainerStorageLocation {
private static final Logger LOG =
LoggerFactory.getLogger(ContainerStorageLocation.class);
private static final String DU_CACHE_FILE = "scmUsed";
private volatile boolean scmUsedSaved = false;
private final StorageLocation dataLocation;
private final String storageUuId;
private final DF usage;
private final GetSpaceUsed scmUsage;
private final File scmUsedFile;
public ContainerStorageLocation(StorageLocation dataLoc, Configuration conf)
throws IOException {
this.dataLocation = dataLoc;
this.storageUuId = DatanodeStorage.generateUuid();
File dataDir = Paths.get(dataLoc.getNormalizedUri()).resolve(
OzoneConsts.CONTAINER_PREFIX).toFile();
// Initialize container data root if it does not exist as required by DF/DU
if (!dataDir.exists()) {
if (!dataDir.mkdirs()) {
LOG.error("Unable to create the container storage location at : {}",
dataDir);
throw new IllegalArgumentException("Unable to create the container" +
" storage location at : " + dataDir);
}
}
scmUsedFile = new File(dataDir, DU_CACHE_FILE);
// get overall disk usage
this.usage = new DF(dataDir, conf);
// get SCM specific usage
this.scmUsage = new CachingGetSpaceUsed.Builder().setPath(dataDir)
.setConf(conf)
.setInitialUsed(loadScmUsed())
.build();
// Ensure scm usage is saved during shutdown.
ShutdownHookManager.get().addShutdownHook(
new Runnable() {
@Override
public void run() {
if (!scmUsedSaved) {
saveScmUsed();
}
}
}, SHUTDOWN_HOOK_PRIORITY);
}
public URI getNormalizedUri() {
return dataLocation.getNormalizedUri();
}
public String getStorageUuId() {
return storageUuId;
}
public long getCapacity() {
long capacity = usage.getCapacity();
return (capacity > 0) ? capacity : 0;
}
public long getAvailable() throws IOException {
long remaining = getCapacity() - getScmUsed();
long available = usage.getAvailable();
if (remaining > available) {
remaining = available;
}
return (remaining > 0) ? remaining : 0;
}
public long getScmUsed() throws IOException{
return scmUsage.getUsed();
}
public String getStorageLocation() {
return getNormalizedUri().getRawPath();
}
public StorageType getStorageType() {
return dataLocation.getStorageType();
}
public void shutdown() {
saveScmUsed();
scmUsedSaved = true;
if (scmUsage instanceof CachingGetSpaceUsed) {
IOUtils.cleanupWithLogger(null, ((CachingGetSpaceUsed) scmUsage));
}
}
/**
* Read in the cached DU value and return it if it is less than 600 seconds
* old (DU update interval). Slight imprecision of scmUsed is not critical
* and skipping DU can significantly shorten the startup time.
* If the cached value is not available or too old, -1 is returned.
*/
long loadScmUsed() {
long cachedScmUsed;
long mtime;
Scanner sc;
try {
sc = new Scanner(scmUsedFile, "UTF-8");
} catch (FileNotFoundException fnfe) {
return -1;
}
try {
// Get the recorded scmUsed from the file.
if (sc.hasNextLong()) {
cachedScmUsed = sc.nextLong();
} else {
return -1;
}
// Get the recorded mtime from the file.
if (sc.hasNextLong()) {
mtime = sc.nextLong();
} else {
return -1;
}
// Return the cached value if mtime is okay.
if (mtime > 0 && (Time.now() - mtime < 600000L)) {
LOG.info("Cached ScmUsed found for {} : {} ", dataLocation,
cachedScmUsed);
return cachedScmUsed;
}
return -1;
} finally {
sc.close();
}
}
/**
* Write the current scmUsed to the cache file.
*/
void saveScmUsed() {
if (scmUsedFile.exists() && !scmUsedFile.delete()) {
LOG.warn("Failed to delete old scmUsed file in {}.", dataLocation);
}
OutputStreamWriter out = null;
try {
long used = getScmUsed();
if (used > 0) {
out = new OutputStreamWriter(new FileOutputStream(scmUsedFile),
StandardCharsets.UTF_8);
// mtime is written last, so that truncated writes won't be valid.
out.write(Long.toString(used) + " " + Long.toString(Time.now()));
out.flush();
out.close();
out = null;
}
} catch (IOException ioe) {
// If write failed, the volume might be bad. Since the cache file is
// not critical, log the error and continue.
LOG.warn("Failed to write scmUsed to " + scmUsedFile, ioe);
} finally {
IOUtils.cleanupWithLogger(null, out);
}
}
}

View File

@ -1,695 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.impl;
import com.google.common.base.Preconditions;
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
import org.apache.ratis.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandResponseProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.hadoop.ozone.container.common.helpers.ChunkUtils;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.keyvalue.helpers.SmallFileUtils;
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.security.NoSuchAlgorithmException;
import java.util.LinkedList;
import java.util.List;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.CLOSED_CONTAINER_IO;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.GET_SMALL_FILE_ERROR;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.NO_SUCH_ALGORITHM;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.PUT_SMALL_FILE_ERROR;
/**
* Ozone Container dispatcher takes a call from the netty server and routes it
* to the right handler function.
*/
public class Dispatcher implements ContainerDispatcher {
static final Logger LOG = LoggerFactory.getLogger(Dispatcher.class);
private final ContainerManager containerManager;
private ContainerMetrics metrics;
private Configuration conf;
/**
* Constructs an OzoneContainer that receives calls from
* XceiverServerHandler.
*
* @param containerManager - A class that manages containers.
*/
public Dispatcher(ContainerManager containerManager, Configuration config) {
Preconditions.checkNotNull(containerManager);
this.containerManager = containerManager;
this.metrics = null;
this.conf = config;
}
@Override
public void init() {
this.metrics = ContainerMetrics.create(conf);
}
@Override
public void shutdown() {
}
@Override
public Handler getHandler(ContainerProtos.ContainerType containerType) {
return null;
}
@Override
public void setScmId(String scmId) {
// DO nothing, this will be removed when cleanup.
}
@Override
public ContainerCommandResponseProto dispatch(
ContainerCommandRequestProto msg) {
LOG.trace("Command {}, trace ID: {} ", msg.getCmdType().toString(),
msg.getTraceID());
long startNanos = System.nanoTime();
ContainerCommandResponseProto resp = null;
try {
Preconditions.checkNotNull(msg);
Type cmdType = msg.getCmdType();
metrics.incContainerOpsMetrics(cmdType);
if ((cmdType == Type.CreateContainer) ||
(cmdType == Type.DeleteContainer) ||
(cmdType == Type.ReadContainer) ||
(cmdType == Type.ListContainer) ||
(cmdType == Type.UpdateContainer) ||
(cmdType == Type.CloseContainer)) {
resp = containerProcessHandler(msg);
}
if ((cmdType == Type.PutKey) ||
(cmdType == Type.GetKey) ||
(cmdType == Type.DeleteKey) ||
(cmdType == Type.ListKey)) {
resp = keyProcessHandler(msg);
}
if ((cmdType == Type.WriteChunk) ||
(cmdType == Type.ReadChunk) ||
(cmdType == Type.DeleteChunk)) {
resp = chunkProcessHandler(msg);
}
if ((cmdType == Type.PutSmallFile) ||
(cmdType == Type.GetSmallFile)) {
resp = smallFileHandler(msg);
}
if (resp != null) {
metrics.incContainerOpsLatencies(cmdType,
System.nanoTime() - startNanos);
return resp;
}
return ContainerUtils.unsupportedRequest(msg);
} catch (StorageContainerException e) {
// This useful since the trace ID will allow us to correlate failures.
return ContainerUtils.logAndReturnError(LOG, e, msg);
}
}
public ContainerMetrics getContainerMetrics() {
return metrics;
}
/**
* Handles the all Container related functionality.
*
* @param msg - command
* @return - response
* @throws StorageContainerException
*/
private ContainerCommandResponseProto containerProcessHandler(
ContainerCommandRequestProto msg) throws StorageContainerException {
try {
switch (msg.getCmdType()) {
case CreateContainer:
return handleCreateContainer(msg);
case DeleteContainer:
return handleDeleteContainer(msg);
case ListContainer:
// TODO : Support List Container.
return ContainerUtils.unsupportedRequest(msg);
case UpdateContainer:
return handleUpdateContainer(msg);
case ReadContainer:
return handleReadContainer(msg);
case CloseContainer:
return handleCloseContainer(msg);
default:
return ContainerUtils.unsupportedRequest(msg);
}
} catch (StorageContainerException e) {
return ContainerUtils.logAndReturnError(LOG, e, msg);
} catch (IOException ex) {
LOG.warn("Container operation failed. " +
"Container: {} Operation: {} trace ID: {} Error: {}",
msg.getCreateContainer().getContainerID(),
msg.getCmdType().name(),
msg.getTraceID(),
ex.toString(), ex);
// TODO : Replace with finer error codes.
return ContainerUtils.getContainerCommandResponse(msg,
ContainerProtos.Result.CONTAINER_INTERNAL_ERROR,
ex.toString()).build();
}
}
/**
* Handles the all key related functionality.
*
* @param msg - command
* @return - response
* @throws StorageContainerException
*/
private ContainerCommandResponseProto keyProcessHandler(
ContainerCommandRequestProto msg) throws StorageContainerException {
try {
switch (msg.getCmdType()) {
case PutKey:
return handlePutKey(msg);
case GetKey:
return handleGetKey(msg);
case DeleteKey:
return handleDeleteKey(msg);
case ListKey:
return ContainerUtils.unsupportedRequest(msg);
default:
return ContainerUtils.unsupportedRequest(msg);
}
} catch (StorageContainerException e) {
return ContainerUtils.logAndReturnError(LOG, e, msg);
} catch (IOException ex) {
LOG.warn("Container operation failed. " +
"Container: {} Operation: {} trace ID: {} Error: {}",
msg.getCreateContainer().getContainerID(),
msg.getCmdType().name(),
msg.getTraceID(),
ex.toString(), ex);
// TODO : Replace with finer error codes.
return ContainerUtils.getContainerCommandResponse(msg,
ContainerProtos.Result.CONTAINER_INTERNAL_ERROR,
ex.toString()).build();
}
}
/**
* Handles the all chunk related functionality.
*
* @param msg - command
* @return - response
* @throws StorageContainerException
*/
private ContainerCommandResponseProto chunkProcessHandler(
ContainerCommandRequestProto msg) throws StorageContainerException {
try {
switch (msg.getCmdType()) {
case WriteChunk:
return handleWriteChunk(msg);
case ReadChunk:
return handleReadChunk(msg);
case DeleteChunk:
return handleDeleteChunk(msg);
case ListChunk:
return ContainerUtils.unsupportedRequest(msg);
default:
return ContainerUtils.unsupportedRequest(msg);
}
} catch (StorageContainerException e) {
return ContainerUtils.logAndReturnError(LOG, e, msg);
} catch (IOException ex) {
LOG.warn("Container operation failed. " +
"Container: {} Operation: {} trace ID: {} Error: {}",
msg.getCreateContainer().getContainerID(),
msg.getCmdType().name(),
msg.getTraceID(),
ex.toString(), ex);
// TODO : Replace with finer error codes.
return ContainerUtils.getContainerCommandResponse(msg,
ContainerProtos.Result.CONTAINER_INTERNAL_ERROR,
ex.toString()).build();
}
}
/**
* Dispatch calls to small file hanlder.
* @param msg - request
* @return response
* @throws StorageContainerException
*/
private ContainerCommandResponseProto smallFileHandler(
ContainerCommandRequestProto msg) throws StorageContainerException {
switch (msg.getCmdType()) {
case PutSmallFile:
return handlePutSmallFile(msg);
case GetSmallFile:
return handleGetSmallFile(msg);
default:
return ContainerUtils.unsupportedRequest(msg);
}
}
/**
* Update an existing container with the new container data.
*
* @param msg Request
* @return ContainerCommandResponseProto
* @throws IOException
*/
private ContainerCommandResponseProto handleUpdateContainer(
ContainerCommandRequestProto msg)
throws IOException {
if (!msg.hasUpdateContainer()) {
LOG.debug("Malformed read container request. trace ID: {}",
msg.getTraceID());
return ContainerUtils.malformedRequest(msg);
}
long containerID = msg.getUpdateContainer().getContainerID();
ContainerData data = new ContainerData(msg.getUpdateContainer()
.getContainerID(), conf);
boolean forceUpdate = msg.getUpdateContainer().getForceUpdate();
this.containerManager.updateContainer(containerID,
data, forceUpdate);
return ContainerUtils.getSuccessResponse(msg);
}
/**
* Calls into container logic and returns appropriate response.
*
* @param msg - Request
* @return ContainerCommandResponseProto
* @throws IOException
*/
private ContainerCommandResponseProto handleReadContainer(
ContainerCommandRequestProto msg)
throws IOException {
if (!msg.hasReadContainer()) {
LOG.debug("Malformed read container request. trace ID: {}",
msg.getTraceID());
return ContainerUtils.malformedRequest(msg);
}
long containerID = msg.getReadContainer().getContainerID();
ContainerData container = this.containerManager.
readContainer(containerID);
return ContainerUtils.getReadContainerResponse(msg, container);
}
/**
* Calls into container logic and returns appropriate response.
*
* @param msg - Request
* @return Response.
* @throws IOException
*/
private ContainerCommandResponseProto handleDeleteContainer(
ContainerCommandRequestProto msg) throws IOException {
if (!msg.hasDeleteContainer()) {
LOG.debug("Malformed delete container request. trace ID: {}",
msg.getTraceID());
return ContainerUtils.malformedRequest(msg);
}
long containerID = msg.getDeleteContainer().getContainerID();
boolean forceDelete = msg.getDeleteContainer().getForceDelete();
this.containerManager.deleteContainer(containerID, forceDelete);
return ContainerUtils.getSuccessResponse(msg);
}
/**
* Calls into container logic and returns appropriate response.
*
* @param msg - Request
* @return Response.
* @throws IOException
*/
private ContainerCommandResponseProto handleCreateContainer(
ContainerCommandRequestProto msg) throws IOException {
if (!msg.hasCreateContainer()) {
LOG.debug("Malformed create container request. trace ID: {}",
msg.getTraceID());
return ContainerUtils.malformedRequest(msg);
}
ContainerData cData = new ContainerData(
msg.getCreateContainer().getContainerID(), conf);
this.containerManager.createContainer(cData);
return ContainerUtils.getSuccessResponse(msg);
}
/**
* closes an open container.
*
* @param msg -
* @return
* @throws IOException
*/
private ContainerCommandResponseProto handleCloseContainer(
ContainerCommandRequestProto msg) throws IOException {
try {
if (!msg.hasCloseContainer()) {
LOG.debug("Malformed close Container request. trace ID: {}",
msg.getTraceID());
return ContainerUtils.malformedRequest(msg);
}
long containerID = msg.getCloseContainer().getContainerID();
if (!this.containerManager.isOpen(containerID)) {
throw new StorageContainerException("Attempting to close a closed " +
"container.", CLOSED_CONTAINER_IO);
}
this.containerManager.closeContainer(containerID);
return ContainerUtils.getSuccessResponse(msg);
} catch (NoSuchAlgorithmException e) {
throw new StorageContainerException("No such Algorithm", e,
NO_SUCH_ALGORITHM);
}
}
/**
* Calls into chunk manager to write a chunk.
*
* @param msg - Request.
* @return Response.
* @throws IOException
*/
private ContainerCommandResponseProto handleWriteChunk(
ContainerCommandRequestProto msg) throws IOException {
if (!msg.hasWriteChunk()) {
LOG.debug("Malformed write chunk request. trace ID: {}",
msg.getTraceID());
return ContainerUtils.malformedRequest(msg);
}
BlockID blockID = BlockID.getFromProtobuf(
msg.getWriteChunk().getBlockID());
if (!this.containerManager.isOpen(blockID.getContainerID())) {
throw new StorageContainerException("Write to closed container.",
CLOSED_CONTAINER_IO);
}
ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getWriteChunk()
.getChunkData());
Preconditions.checkNotNull(chunkInfo);
byte[] data = null;
if (msg.getWriteChunk().getStage() == ContainerProtos.Stage.WRITE_DATA
|| msg.getWriteChunk().getStage() == ContainerProtos.Stage.COMBINED) {
data = msg.getWriteChunk().getData().toByteArray();
metrics.incContainerBytesStats(Type.WriteChunk, data.length);
}
this.containerManager.getChunkManager()
.writeChunk(blockID, chunkInfo,
data, msg.getWriteChunk().getStage());
return ChunkUtils.getChunkResponse(msg);
}
/**
* Calls into chunk manager to read a chunk.
*
* @param msg - Request.
* @return - Response.
* @throws IOException
*/
private ContainerCommandResponseProto handleReadChunk(
ContainerCommandRequestProto msg) throws IOException {
if (!msg.hasReadChunk()) {
LOG.debug("Malformed read chunk request. trace ID: {}",
msg.getTraceID());
return ContainerUtils.malformedRequest(msg);
}
BlockID blockID = BlockID.getFromProtobuf(
msg.getReadChunk().getBlockID());
ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getReadChunk()
.getChunkData());
Preconditions.checkNotNull(chunkInfo);
byte[] data = this.containerManager.getChunkManager().
readChunk(blockID, chunkInfo);
metrics.incContainerBytesStats(Type.ReadChunk, data.length);
return ChunkUtils.getReadChunkResponse(msg, data, chunkInfo);
}
/**
* Calls into chunk manager to write a chunk.
*
* @param msg - Request.
* @return Response.
* @throws IOException
*/
private ContainerCommandResponseProto handleDeleteChunk(
ContainerCommandRequestProto msg) throws IOException {
if (!msg.hasDeleteChunk()) {
LOG.debug("Malformed delete chunk request. trace ID: {}",
msg.getTraceID());
return ContainerUtils.malformedRequest(msg);
}
BlockID blockID = BlockID.getFromProtobuf(msg.getDeleteChunk()
.getBlockID());
long containerID = blockID.getContainerID();
if (!this.containerManager.isOpen(containerID)) {
throw new StorageContainerException("Write to closed container.",
CLOSED_CONTAINER_IO);
}
ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getDeleteChunk()
.getChunkData());
Preconditions.checkNotNull(chunkInfo);
this.containerManager.getChunkManager().deleteChunk(blockID,
chunkInfo);
return ChunkUtils.getChunkResponse(msg);
}
/**
* Put Key handler.
*
* @param msg - Request.
* @return - Response.
* @throws IOException
*/
private ContainerCommandResponseProto handlePutKey(
ContainerCommandRequestProto msg) throws IOException {
if (!msg.hasPutKey()) {
LOG.debug("Malformed put key request. trace ID: {}",
msg.getTraceID());
return ContainerUtils.malformedRequest(msg);
}
BlockID blockID = BlockID.getFromProtobuf(
msg.getPutKey().getKeyData().getBlockID());
long containerID = blockID.getContainerID();
if (!this.containerManager.isOpen(containerID)) {
throw new StorageContainerException("Write to closed container.",
CLOSED_CONTAINER_IO);
}
KeyData keyData = KeyData.getFromProtoBuf(msg.getPutKey().getKeyData());
Preconditions.checkNotNull(keyData);
this.containerManager.getKeyManager().putKey(keyData);
long numBytes = keyData.getProtoBufMessage().toByteArray().length;
metrics.incContainerBytesStats(Type.PutKey, numBytes);
return KeyUtils.getKeyResponse(msg);
}
/**
* Handle Get Key.
*
* @param msg - Request.
* @return - Response.
* @throws IOException
*/
private ContainerCommandResponseProto handleGetKey(
ContainerCommandRequestProto msg) throws IOException {
if (!msg.hasGetKey()) {
LOG.debug("Malformed get key request. trace ID: {}",
msg.getTraceID());
return ContainerUtils.malformedRequest(msg);
}
KeyData keyData = new KeyData(
BlockID.getFromProtobuf(msg.getGetKey().getBlockID()));
Preconditions.checkNotNull(keyData);
KeyData responseData =
this.containerManager.getKeyManager().getKey(keyData);
long numBytes = responseData.getProtoBufMessage().toByteArray().length;
metrics.incContainerBytesStats(Type.GetKey, numBytes);
return KeyUtils.getKeyDataResponse(msg, responseData);
}
/**
* Handle Delete Key.
*
* @param msg - Request.
* @return - Response.
* @throws IOException
*/
private ContainerCommandResponseProto handleDeleteKey(
ContainerCommandRequestProto msg) throws IOException {
if (!msg.hasDeleteKey()) {
LOG.debug("Malformed delete key request. trace ID: {}",
msg.getTraceID());
return ContainerUtils.malformedRequest(msg);
}
BlockID blockID = BlockID.getFromProtobuf(msg.getDeleteKey()
.getBlockID());
Preconditions.checkNotNull(blockID);
long containerID = blockID.getContainerID();
if (!this.containerManager.isOpen(containerID)) {
throw new StorageContainerException("Write to closed container.",
CLOSED_CONTAINER_IO);
}
this.containerManager.getKeyManager().deleteKey(blockID);
return KeyUtils.getKeyResponse(msg);
}
/**
* Handles writing a chunk and associated key using single RPC.
*
* @param msg - Message.
* @return ContainerCommandResponseProto
* @throws StorageContainerException
*/
private ContainerCommandResponseProto handlePutSmallFile(
ContainerCommandRequestProto msg) throws StorageContainerException {
if (!msg.hasPutSmallFile()) {
LOG.debug("Malformed put small file request. trace ID: {}",
msg.getTraceID());
return ContainerUtils.malformedRequest(msg);
}
try {
BlockID blockID = BlockID.getFromProtobuf(msg.
getPutSmallFile().getKey().getKeyData().getBlockID());
long containerID = blockID.getContainerID();
if (!this.containerManager.isOpen(containerID)) {
throw new StorageContainerException("Write to closed container.",
CLOSED_CONTAINER_IO);
}
KeyData keyData = KeyData.getFromProtoBuf(msg.getPutSmallFile().getKey()
.getKeyData());
ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getPutSmallFile()
.getChunkInfo());
byte[] data = msg.getPutSmallFile().getData().toByteArray();
metrics.incContainerBytesStats(Type.PutSmallFile, data.length);
this.containerManager.getChunkManager().writeChunk(blockID,
chunkInfo, data, ContainerProtos.Stage.COMBINED);
List<ContainerProtos.ChunkInfo> chunks = new LinkedList<>();
chunks.add(chunkInfo.getProtoBufMessage());
keyData.setChunks(chunks);
this.containerManager.getKeyManager().putKey(keyData);
return SmallFileUtils.getPutFileResponseSuccess(msg);
} catch (StorageContainerException e) {
return ContainerUtils.logAndReturnError(LOG, e, msg);
} catch (IOException e) {
throw new StorageContainerException("Put Small File Failed.", e,
PUT_SMALL_FILE_ERROR);
}
}
/**
* Handles getting a data stream using a key. This helps in reducing the RPC
* overhead for small files.
*
* @param msg - ContainerCommandRequestProto
* @return ContainerCommandResponseProto
* @throws StorageContainerException
*/
private ContainerCommandResponseProto handleGetSmallFile(
ContainerCommandRequestProto msg) throws StorageContainerException {
ByteString dataBuf = ByteString.EMPTY;
if (!msg.hasGetSmallFile()) {
LOG.debug("Malformed get small file request. trace ID: {}",
msg.getTraceID());
return ContainerUtils.malformedRequest(msg);
}
try {
long bytes = 0;
KeyData keyData = new KeyData(BlockID.getFromProtobuf(
msg.getGetSmallFile().getKey().getBlockID()));
KeyData data = this.containerManager.getKeyManager().getKey(keyData);
ContainerProtos.ChunkInfo c = null;
for (ContainerProtos.ChunkInfo chunk : data.getChunks()) {
bytes += chunk.getSerializedSize();
ByteString current =
ByteString.copyFrom(this.containerManager.getChunkManager()
.readChunk(keyData.getBlockID(),
ChunkInfo.getFromProtoBuf(chunk)));
dataBuf = dataBuf.concat(current);
c = chunk;
}
metrics.incContainerBytesStats(Type.GetSmallFile, bytes);
return SmallFileUtils.getGetSmallFileResponseSuccess(
msg, dataBuf.toByteArray(), ChunkInfo.getFromProtoBuf(c));
} catch (StorageContainerException e) {
return ContainerUtils.logAndReturnError(LOG, e, msg);
} catch (IOException e) {
throw new StorageContainerException("Get Small File Failed", e,
GET_SMALL_FILE_ERROR);
}
}
}

View File

@ -1,204 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.impl;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Longs;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
import org.apache.hadoop.ozone.container.common.interfaces.KeyManager;
import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
import org.apache.hadoop.utils.MetadataStore;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.NO_SUCH_KEY;
/**
* Key Manager impl.
*/
public class KeyManagerImpl implements KeyManager {
static final Logger LOG =
LoggerFactory.getLogger(KeyManagerImpl.class);
private static final float LOAD_FACTOR = 0.75f;
private final ContainerManager containerManager;
private final Configuration conf;
/**
* Constructs a key Manager.
*
* @param containerManager - Container Manager.
*/
public KeyManagerImpl(ContainerManager containerManager, Configuration conf) {
Preconditions.checkNotNull(containerManager, "Container manager cannot be" +
" null");
Preconditions.checkNotNull(conf, "Config cannot be null");
this.containerManager = containerManager;
this.conf = conf;
}
/**
* {@inheritDoc}
*/
@Override
public void putKey(KeyData data) throws IOException {
Preconditions.checkNotNull(data,
"KeyData cannot be null for put operation.");
Preconditions.checkState(data.getContainerID() >= 0,
"Container ID cannot be negative");
containerManager.readLock();
try {
// We are not locking the key manager since LevelDb serializes all actions
// against a single DB. We rely on DB level locking to avoid conflicts.
ContainerData cData = containerManager.readContainer(
data.getContainerID());
MetadataStore db = KeyUtils.getDB(cData, conf);
// This is a post condition that acts as a hint to the user.
// Should never fail.
Preconditions.checkNotNull(db, "DB cannot be null here");
db.put(Longs.toByteArray(data.getLocalID()), data
.getProtoBufMessage().toByteArray());
} finally {
containerManager.readUnlock();
}
}
/**
* {@inheritDoc}
*/
@Override
public KeyData getKey(KeyData data) throws IOException {
containerManager.readLock();
try {
Preconditions.checkNotNull(data, "Key data cannot be null");
Preconditions.checkNotNull(data.getContainerID(),
"Container name cannot be null");
ContainerData cData = containerManager.readContainer(data
.getContainerID());
MetadataStore db = KeyUtils.getDB(cData, conf);
// This is a post condition that acts as a hint to the user.
// Should never fail.
Preconditions.checkNotNull(db, "DB cannot be null here");
byte[] kData = db.get(Longs.toByteArray(data.getLocalID()));
if (kData == null) {
throw new StorageContainerException("Unable to find the key.",
NO_SUCH_KEY);
}
ContainerProtos.KeyData keyData =
ContainerProtos.KeyData.parseFrom(kData);
return KeyData.getFromProtoBuf(keyData);
} finally {
containerManager.readUnlock();
}
}
/**
* {@inheritDoc}
*/
@Override
public void deleteKey(BlockID blockID)
throws IOException {
Preconditions.checkNotNull(blockID, "block ID cannot be null.");
Preconditions.checkState(blockID.getContainerID() >= 0,
"Container ID cannot be negative.");
Preconditions.checkState(blockID.getLocalID() >= 0,
"Local ID cannot be negative.");
containerManager.readLock();
try {
ContainerData cData = containerManager
.readContainer(blockID.getContainerID());
MetadataStore db = KeyUtils.getDB(cData, conf);
// This is a post condition that acts as a hint to the user.
// Should never fail.
Preconditions.checkNotNull(db, "DB cannot be null here");
// Note : There is a race condition here, since get and delete
// are not atomic. Leaving it here since the impact is refusing
// to delete a key which might have just gotten inserted after
// the get check.
byte[] kKey = Longs.toByteArray(blockID.getLocalID());
byte[] kData = db.get(kKey);
if (kData == null) {
throw new StorageContainerException("Unable to find the key.",
NO_SUCH_KEY);
}
db.delete(kKey);
} finally {
containerManager.readUnlock();
}
}
/**
* {@inheritDoc}
*/
@Override
public List<KeyData> listKey(
long containerID, long startLocalID, int count)
throws IOException {
Preconditions.checkState(containerID >= 0,
"Container ID cannot be negative");
Preconditions.checkState(startLocalID >= 0,
"startLocal ID cannot be negative");
Preconditions.checkArgument(count > 0,
"Count must be a positive number.");
ContainerData cData = containerManager.readContainer(containerID);
MetadataStore db = KeyUtils.getDB(cData, conf);
List<KeyData> result = new ArrayList<>();
byte[] startKeyInBytes = Longs.toByteArray(startLocalID);
List<Map.Entry<byte[], byte[]>> range =
db.getSequentialRangeKVs(startKeyInBytes, count, null);
for (Map.Entry<byte[], byte[]> entry : range) {
KeyData value = KeyUtils.getKeyData(entry.getValue());
KeyData data = new KeyData(value.getBlockID());
result.add(data);
}
return result;
}
/**
* Shutdown keyManager.
*/
@Override
public void shutdown() {
Preconditions.checkState(this.containerManager.hasWriteLock(), "asserts " +
"that we are holding the container manager lock when shutting down.");
KeyUtils.shutdownCache(ContainerCache.getInstance(conf));
}
}

View File

@ -21,7 +21,6 @@ import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.interfaces
.ContainerDeletionChoosingPolicy;
import org.slf4j.Logger;

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.ozone.container.common.impl;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.interfaces
.ContainerDeletionChoosingPolicy;
import org.slf4j.Logger;

View File

@ -1,73 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.interfaces;
import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
/**
* Chunk Manager allows read, write, delete and listing of chunks in
* a container.
*/
public interface ChunkManager {
/**
* writes a given chunk.
* @param blockID - ID of the block.
* @param info - ChunkInfo.
* @param stage - Chunk Stage write.
* @throws StorageContainerException
*/
void writeChunk(BlockID blockID,
ChunkInfo info, byte[] data, ContainerProtos.Stage stage)
throws StorageContainerException;
/**
* reads the data defined by a chunk.
* @param blockID - ID of the block.
* @param info - ChunkInfo.
* @return byte array
* @throws StorageContainerException
*
* TODO: Right now we do not support partial reads and writes of chunks.
* TODO: Explore if we need to do that for ozone.
*/
byte[] readChunk(BlockID blockID, ChunkInfo info) throws
StorageContainerException;
/**
* Deletes a given chunk.
* @param blockID - ID of the block.
* @param info - Chunk Info
* @throws StorageContainerException
*/
void deleteChunk(BlockID blockID, ChunkInfo info) throws
StorageContainerException;
// TODO : Support list operations.
/**
* Shutdown the chunkManager.
*/
void shutdown();
}

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.ozone.container.common.interfaces;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerLifeCycleState;

View File

@ -19,7 +19,7 @@ package org.apache.hadoop.ozone.container.common.interfaces;
import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import java.util.List;
import java.util.Map;
@ -28,6 +28,7 @@ import java.util.Map;
* This interface is used for choosing desired containers for
* block deletion.
*/
// TODO: Fix ContainerDeletionChoosingPolicy to work with new StorageLayer
public interface ContainerDeletionChoosingPolicy {
/**

View File

@ -1,267 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.interfaces;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.hdfs.util.RwLock;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import java.io.IOException;
import java.security.NoSuchAlgorithmException;
import java.util.List;
/**
* Interface for container operations.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public interface ContainerManager extends RwLock {
/**
* Init call that sets up a container Manager.
*
* @param config - Configuration.
* @param containerDirs - List of Metadata Container locations.
* @param datanodeDetails - DatanodeDetails
* @throws StorageContainerException
*/
void init(Configuration config, List<StorageLocation> containerDirs,
DatanodeDetails datanodeDetails) throws IOException;
/**
* Creates a container with the given name.
*
* @param containerData - Container Name and metadata.
* @throws StorageContainerException
*/
void createContainer(ContainerData containerData)
throws StorageContainerException;
/**
* Deletes an existing container.
*
* @param containerID - ID of the container.
* @param forceDelete - whether this container should be deleted forcibly.
* @throws StorageContainerException
*/
void deleteContainer(long containerID,
boolean forceDelete) throws StorageContainerException;
/**
* Update an existing container.
*
* @param containerID ID of the container
* @param data container data
* @param forceUpdate if true, update container forcibly.
* @throws StorageContainerException
*/
void updateContainer(long containerID, ContainerData data,
boolean forceUpdate) throws StorageContainerException;
/**
* As simple interface for container Iterations.
*
* @param startContainerID - Return containers with ID >= startContainerID.
* @param count - how many to return
* @param data - Actual containerData
* @throws StorageContainerException
*/
void listContainer(long startContainerID, long count,
List<ContainerData> data) throws StorageContainerException;
/**
* Choose containers for block deletion.
*
* @param count - how many to return
* @throws StorageContainerException
*/
List<ContainerData> chooseContainerForBlockDeletion(int count)
throws StorageContainerException;
/**
* Get metadata about a specific container.
*
* @param containerID - ID of the container.
* @return ContainerData - Container Data.
* @throws StorageContainerException
*/
ContainerData readContainer(long containerID)
throws StorageContainerException;
/**
* Closes a open container, if it is already closed or does not exist a
* StorageContainerException is thrown.
* @param containerID - ID of the container.
* @throws StorageContainerException
*/
void closeContainer(long containerID)
throws StorageContainerException, NoSuchAlgorithmException;
/**
* Checks if a container exists.
* @param containerID - ID of the container.
* @return true if the container is open false otherwise.
* @throws StorageContainerException - Throws Exception if we are not
* able to find the container.
*/
boolean isOpen(long containerID) throws StorageContainerException;
/**
* Supports clean shutdown of container.
*
* @throws StorageContainerException
*/
void shutdown() throws IOException;
/**
* Sets the Chunk Manager.
*
* @param chunkManager - ChunkManager.
*/
void setChunkManager(ChunkManager chunkManager);
/**
* Gets the Chunk Manager.
*
* @return ChunkManager.
*/
ChunkManager getChunkManager();
/**
* Sets the Key Manager.
*
* @param keyManager - Key Manager.
*/
void setKeyManager(KeyManager keyManager);
/**
* Gets the Key Manager.
*
* @return KeyManager.
*/
KeyManager getKeyManager();
/**
* Get the Node Report of container storage usage.
* @return node report.
*/
NodeReportProto getNodeReport() throws IOException;
/**
* Gets container report.
* @return container report.
* @throws IOException
*/
ContainerReportsProto getContainerReport() throws IOException;
/**
* Gets container reports.
* @return List of all closed containers.
* @throws IOException
*/
List<ContainerData> getClosedContainerReports() throws IOException;
/**
* Increase pending deletion blocks count number of specified container.
*
* @param numBlocks
* increment count number
* @param containerId
* container id
*/
void incrPendingDeletionBlocks(int numBlocks, long containerId);
/**
* Decrease pending deletion blocks count number of specified container.
*
* @param numBlocks
* decrement count number
* @param containerId
* container id
*/
void decrPendingDeletionBlocks(int numBlocks, long containerId);
/**
* Increase the read count of the container.
* @param containerId - ID of the container.
*/
void incrReadCount(long containerId);
/**
* Increse the read counter for bytes read from the container.
* @param containerId - ID of the container.
* @param readBytes - bytes read from the container.
*/
void incrReadBytes(long containerId, long readBytes);
/**
* Increase the write count of the container.
* @param containerId - ID of the container.
*/
void incrWriteCount(long containerId);
/**
* Increase the write counter for bytes write into the container.
* @param containerId - ID of the container.
* @param writeBytes - bytes write into the container.
*/
void incrWriteBytes(long containerId, long writeBytes);
/**
* Increase the bytes used by the container.
* @param containerId - ID of the container.
* @param used - additional bytes used by the container.
* @return the current bytes used.
*/
long incrBytesUsed(long containerId, long used);
/**
* Decrease the bytes used by the container.
* @param containerId - ID of the container.
* @param used - additional bytes reclaimed by the container.
* @return the current bytes used.
*/
long decrBytesUsed(long containerId, long used);
/**
* Get the bytes used by the container.
* @param containerId - ID of the container.
* @return the current bytes used by the container.
*/
long getBytesUsed(long containerId);
/**
* Get the number of keys in the container.
* @param containerId - ID of the container.
* @return the current key count.
*/
long getNumKeys(long containerId);
}

View File

@ -1,73 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.interfaces;
import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
import java.io.IOException;
import java.util.List;
/**
* KeyManager deals with Key Operations in the container Level.
*/
public interface KeyManager {
/**
* Puts or overwrites a key.
*
* @param data - Key Data.
* @throws IOException
*/
void putKey(KeyData data) throws IOException;
/**
* Gets an existing key.
*
* @param data - Key Data.
* @return Key Data.
* @throws IOException
*/
KeyData getKey(KeyData data) throws IOException;
/**
* Deletes an existing Key.
*
* @param blockID - ID of the block.
* @throws StorageContainerException
*/
void deleteKey(BlockID blockID)
throws IOException;
/**
* List keys in a container.
*
* @param containerID - ID of the container.
* @param startLocalID - Key to start from, 0 to begin.
* @param count - Number of keys to return.
* @return List of Keys that match the criteria.
*/
List<KeyData> listKey(long containerID, long startLocalID,
int count) throws IOException;
/**
* Shutdown keyManager.
*/
void shutdown();
}

View File

@ -86,20 +86,17 @@ public class VolumeSet {
*/
private final AutoCloseableLock volumeSetLock;
private final DatanodeDetails dnDetails;
private String datanodeUuid;
private final String datanodeUuid;
private String clusterID;
public VolumeSet(DatanodeDetails datanodeDetails, Configuration conf)
public VolumeSet(String dnUuid, Configuration conf)
throws DiskOutOfSpaceException {
this(datanodeDetails, null, conf);
this(dnUuid, null, conf);
}
public VolumeSet(DatanodeDetails datanodeDetails, String clusterID,
Configuration conf)
public VolumeSet(String dnUuid, String clusterID, Configuration conf)
throws DiskOutOfSpaceException {
this.dnDetails = datanodeDetails;
this.datanodeUuid = datanodeDetails.getUuidString();
this.datanodeUuid = dnUuid;
this.clusterID = clusterID;
this.conf = conf;
this.volumeSetLock = new AutoCloseableLock(

View File

@ -31,6 +31,8 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
@ -112,7 +114,7 @@ public class KeyValueContainer implements Container {
.getVolumesList(), maxSize);
String containerBasePath = containerVolume.getHddsRootDir().toString();
long containerId = containerData.getContainerId();
long containerId = containerData.getContainerID();
String containerName = Long.toString(containerId);
containerMetaDataPath = KeyValueContainerLocationUtil
@ -127,7 +129,7 @@ public class KeyValueContainer implements Container {
containerMetaDataPath, containerName);
// Check if it is new Container.
KeyValueContainerUtil.verifyIsNewContainer(containerMetaDataPath);
ContainerUtils.verifyIsNewContainer(containerMetaDataPath);
//Create Metadata path chunks path and metadata db
KeyValueContainerUtil.createContainerMetaData(containerMetaDataPath,
@ -184,7 +186,7 @@ public class KeyValueContainer implements Container {
File tempCheckSumFile = null;
FileOutputStream containerCheckSumStream = null;
Writer writer = null;
long containerId = containerData.getContainerId();
long containerId = containerData.getContainerID();
try {
tempContainerFile = createTempFile(containerFile);
tempCheckSumFile = createTempFile(containerCheckSumFile);
@ -238,7 +240,7 @@ public class KeyValueContainer implements Container {
File containerBkpFile = null;
File checkSumBkpFile = null;
long containerId = containerData.getContainerId();
long containerId = containerData.getContainerID();
try {
if (containerFile.exists() && containerCheckSumFile.exists()) {
@ -251,8 +253,8 @@ public class KeyValueContainer implements Container {
} else {
containerData.setState(ContainerProtos.ContainerLifeCycleState.INVALID);
throw new StorageContainerException("Container is an Inconsistent " +
"state, missing required files(.container, .chksm)",
INVALID_CONTAINER_STATE);
"state, missing required files(.container, .chksm). ContainerID: " +
containerId, INVALID_CONTAINER_STATE);
}
} catch (StorageContainerException ex) {
throw ex;
@ -303,7 +305,7 @@ public class KeyValueContainer implements Container {
@Override
public void delete(boolean forceDelete)
throws StorageContainerException {
long containerId = containerData.getContainerId();
long containerId = containerData.getContainerID();
try {
KeyValueContainerUtil.removeContainer(containerData, config, forceDelete);
} catch (StorageContainerException ex) {
@ -326,11 +328,11 @@ public class KeyValueContainer implements Container {
// complete this action
try {
writeLock();
long containerId = containerData.getContainerId();
long containerId = containerData.getContainerID();
if(!containerData.isValid()) {
LOG.debug("Invalid container data. Container Id: {}", containerId);
throw new StorageContainerException("Invalid container data. Name : " +
containerId, INVALID_CONTAINER_STATE);
throw new StorageContainerException("Invalid container data. " +
"ContainerID: " + containerId, INVALID_CONTAINER_STATE);
}
containerData.closeContainer();
File containerFile = getContainerFile();
@ -380,16 +382,16 @@ public class KeyValueContainer implements Container {
// TODO: Now, when writing the updated data to .container file, we are
// holding lock and writing data to disk. We can have async implementation
// to flush the update container data to disk.
long containerId = containerData.getContainerId();
long containerId = containerData.getContainerID();
if(!containerData.isValid()) {
LOG.debug("Invalid container data. ID: {}", containerId);
LOG.debug("Invalid container data. ContainerID: {}", containerId);
throw new StorageContainerException("Invalid container data. " +
"Container Name : " + containerId, INVALID_CONTAINER_STATE);
"ContainerID: " + containerId, INVALID_CONTAINER_STATE);
}
if (!forceUpdate && !containerData.isOpen()) {
throw new StorageContainerException(
"Updating a closed container is not allowed. ID: " + containerId,
UNSUPPORTED_REQUEST);
"Updating a closed container without force option is not allowed. " +
"ContainerID: " + containerId, UNSUPPORTED_REQUEST);
}
try {
for (Map.Entry<String, String> entry : metadata.entrySet()) {
@ -482,7 +484,7 @@ public class KeyValueContainer implements Container {
*/
private File getContainerFile() {
return new File(containerData.getMetadataPath(), containerData
.getContainerId() + OzoneConsts.CONTAINER_EXTENSION);
.getContainerID() + OzoneConsts.CONTAINER_EXTENSION);
}
/**
@ -491,7 +493,7 @@ public class KeyValueContainer implements Container {
*/
private File getContainerCheckSumFile() {
return new File(containerData.getMetadataPath(), containerData
.getContainerId() + OzoneConsts.CONTAINER_FILE_CHECKSUM_EXTENSION);
.getContainerID() + OzoneConsts.CONTAINER_FILE_CHECKSUM_EXTENSION);
}
/**

View File

@ -18,16 +18,29 @@
package org.apache.hadoop.ozone.container.keyvalue;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Lists;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.yaml.snakeyaml.nodes.Tag;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import static org.apache.hadoop.ozone.OzoneConsts.CHUNKS_PATH;
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_TYPE;
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ID;
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_TYPE;
import static org.apache.hadoop.ozone.OzoneConsts.LAYOUTVERSION;
import static org.apache.hadoop.ozone.OzoneConsts.MAX_SIZE_GB;
import static org.apache.hadoop.ozone.OzoneConsts.METADATA;
import static org.apache.hadoop.ozone.OzoneConsts.METADATA_PATH;
import static org.apache.hadoop.ozone.OzoneConsts.STATE;
/**
* This class represents the KeyValueContainer metadata, which is the
* in-memory representation of container metadata and is represented on disk
@ -36,12 +49,20 @@ import java.util.Map;
public class KeyValueContainerData extends ContainerData {
// Yaml Tag used for KeyValueContainerData.
public static final Tag YAML_TAG = new Tag("KeyValueContainerData");
public static final Tag KEYVALUE_YAML_TAG = new Tag("KeyValueContainerData");
// Fields need to be stored in .container file.
private static final List<String> YAML_FIELDS = Lists.newArrayList(
"containerType", "containerId", "layOutVersion", "state", "metadata",
"metadataPath", "chunksPath", "containerDBType", "maxSizeGB");
private static final List<String> YAML_FIELDS =
Lists.newArrayList(
CONTAINER_TYPE,
CONTAINER_ID,
LAYOUTVERSION,
STATE,
METADATA,
METADATA_PATH,
CHUNKS_PATH,
CONTAINER_DB_TYPE,
MAX_SIZE_GB);
// Path to Container metadata Level DB/RocksDB Store and .container file.
private String metadataPath;
@ -96,11 +117,11 @@ public class KeyValueContainerData extends ContainerData {
public File getDbFile() {
return dbFile;
}
/**
* Returns container metadata path.
*
* @return - path
*/
@Override
public String getMetadataPath() {
return metadataPath;
}
@ -122,6 +143,14 @@ public class KeyValueContainerData extends ContainerData {
return chunksPath;
}
/**
* Returns container chunks path.
*/
@Override
public String getDataPath() {
return chunksPath;
}
/**
* Set chunks Path.
* @param chunkPath - File path.
@ -181,7 +210,7 @@ public class KeyValueContainerData extends ContainerData {
public ContainerProtos.ContainerData getProtoBufMessage() {
ContainerProtos.ContainerData.Builder builder = ContainerProtos
.ContainerData.newBuilder();
builder.setContainerID(this.getContainerId());
builder.setContainerID(this.getContainerID());
builder.setDbPath(this.getDbFile().getPath());
builder.setContainerPath(this.getMetadataPath());
builder.setState(this.getState());
@ -211,4 +240,41 @@ public class KeyValueContainerData extends ContainerData {
public static List<String> getYamlFields() {
return YAML_FIELDS;
}
/**
* Constructs a KeyValueContainerData object from ProtoBuf classes.
*
* @param protoData - ProtoBuf Message
* @throws IOException
*/
@VisibleForTesting
public static KeyValueContainerData getFromProtoBuf(
ContainerProtos.ContainerData protoData) throws IOException {
// TODO: Add containerMaxSize to ContainerProtos.ContainerData
KeyValueContainerData data = new KeyValueContainerData(
protoData.getContainerID(),
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
for (int x = 0; x < protoData.getMetadataCount(); x++) {
data.addMetadata(protoData.getMetadata(x).getKey(),
protoData.getMetadata(x).getValue());
}
if (protoData.hasContainerPath()) {
data.setContainerPath(protoData.getContainerPath());
}
if (protoData.hasState()) {
data.setState(protoData.getState());
}
if (protoData.hasBytesUsed()) {
data.setBytesUsed(protoData.getBytesUsed());
}
if(protoData.hasContainerDBType()) {
data.setContainerDBType(protoData.getContainerDBType());
}
return data;
}
}

View File

@ -288,7 +288,7 @@ public class KeyValueHandler extends Handler {
DELETE_ON_OPEN_CONTAINER);
} else {
containerSet.removeContainer(
kvContainer.getContainerData().getContainerId());
kvContainer.getContainerData().getContainerID());
// Release the lock first.
// Avoid holding write locks for disk operations
kvContainer.writeUnlock();

View File

@ -33,8 +33,8 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerImpl;
import org.apache.ratis.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.ozone.container.common.volume.VolumeIOStats;
import org.apache.hadoop.util.Time;
@ -229,17 +229,16 @@ public final class ChunkUtils {
* Validates chunk data and returns a file object to Chunk File that we are
* expected to write data to.
*
* @param data - container data.
* @param chunkFile - chunkFile to write data into.
* @param info - chunk info.
* @return File
* @return boolean isOverwrite
* @throws StorageContainerException
*/
public static File validateChunk(KeyValueContainerData data, ChunkInfo info)
throws StorageContainerException {
public static boolean validateChunkForOverwrite(File chunkFile,
ChunkInfo info) throws StorageContainerException {
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
File chunkFile = getChunkFile(data, info);
if (isOverWriteRequested(chunkFile, info)) {
if (!isOverWritePermitted(info)) {
log.error("Rejecting write chunk request. Chunk overwrite " +
@ -248,8 +247,9 @@ public final class ChunkUtils {
"OverWrite flag required." + info.toString(),
OVERWRITE_FLAG_REQUIRED);
}
return true;
}
return chunkFile;
return false;
}
/**
@ -340,8 +340,8 @@ public final class ChunkUtils {
public static ContainerCommandResponseProto getReadChunkResponse(
ContainerCommandRequestProto msg, byte[] data, ChunkInfo info) {
Preconditions.checkNotNull(msg);
Preconditions.checkNotNull("Chunk data is null", data);
Preconditions.checkNotNull("Chunk Info is null", info);
Preconditions.checkNotNull(data, "Chunk data is null");
Preconditions.checkNotNull(info, "Chunk Info is null");
ReadChunkResponseProto.Builder response =
ReadChunkResponseProto.newBuilder();

View File

@ -57,24 +57,25 @@ public final class KeyUtils {
* add into cache. This function is called with containerManager
* ReadLock held.
*
* @param container container.
* @param containerData containerData.
* @param conf configuration.
* @return MetadataStore handle.
* @throws StorageContainerException
*/
public static MetadataStore getDB(KeyValueContainerData container,
public static MetadataStore getDB(KeyValueContainerData containerData,
Configuration conf) throws
StorageContainerException {
Preconditions.checkNotNull(container);
Preconditions.checkNotNull(containerData);
ContainerCache cache = ContainerCache.getInstance(conf);
Preconditions.checkNotNull(cache);
Preconditions.checkNotNull(container.getDbFile());
Preconditions.checkNotNull(containerData.getDbFile());
try {
return cache.getDB(container.getContainerId(), container
.getContainerDBType(), container.getDbFile().getAbsolutePath());
return cache.getDB(containerData.getContainerID(), containerData
.getContainerDBType(), containerData.getDbFile().getAbsolutePath());
} catch (IOException ex) {
String message = String.format("Unable to open DB Path: " +
"%s. ex: %s", container.getDbFile(), ex.getMessage());
String message = String.format("Error opening DB. Container:%s " +
"ContainerPath:%s", containerData.getContainerID(), containerData
.getDbFile().getPath());
throw new StorageContainerException(message, UNABLE_TO_READ_METADATA_DB);
}
}
@ -89,7 +90,7 @@ public final class KeyUtils {
Preconditions.checkNotNull(container);
ContainerCache cache = ContainerCache.getInstance(conf);
Preconditions.checkNotNull(cache);
cache.removeDB(container.getContainerId());
cache.removeDB(container.getContainerID());
}
/**

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.utils.MetadataKeyFilters;
import org.apache.hadoop.utils.MetadataStore;
@ -68,19 +69,6 @@ public final class KeyValueContainerUtil {
private static final Logger LOG = LoggerFactory.getLogger(
KeyValueContainerUtil.class);
public static void verifyIsNewContainer(File containerFile) throws
FileAlreadyExistsException {
Preconditions.checkNotNull(containerFile, "containerFile Should not be " +
"null");
if (containerFile.getParentFile().exists()) {
LOG.error("container already exists on disk. File: {}", containerFile
.toPath());
throw new FileAlreadyExistsException("container already exists on " +
"disk.");
}
}
/**
* creates metadata path, chunks path and metadata DB for the specified
* container.
@ -271,7 +259,7 @@ public final class KeyValueContainerUtil {
Preconditions.checkNotNull(dbFile, "dbFile cannot be null");
Preconditions.checkNotNull(config, "ozone config cannot be null");
long containerId = containerData.getContainerId();
long containerId = containerData.getContainerID();
String containerName = String.valueOf(containerId);
File metadataPath = new File(containerData.getMetadataPath());
@ -282,7 +270,7 @@ public final class KeyValueContainerUtil {
// Verify Checksum
String checksum = KeyValueContainerUtil.computeCheckSum(
containerData.getContainerId(), containerFile);
containerData.getContainerID(), containerFile);
KeyValueContainerUtil.verifyCheckSum(containerId, checksumFile, checksum);
containerData.setDbFile(dbFile);
@ -305,4 +293,34 @@ public final class KeyValueContainerUtil {
containerData.setKeyCount(liveKeys.size());
}
/**
* Returns the path where data or chunks live for a given container.
*
* @param kvContainerData - KeyValueContainerData
* @return - Path to the chunks directory
*/
public static Path getDataDirectory(KeyValueContainerData kvContainerData) {
String chunksPath = kvContainerData.getChunksPath();
Preconditions.checkNotNull(chunksPath);
return Paths.get(chunksPath);
}
/**
* Container metadata directory -- here is where the level DB and
* .container file lives.
*
* @param kvContainerData - KeyValueContainerData
* @return Path to the metadata directory
*/
public static Path getMetadataDirectory(
KeyValueContainerData kvContainerData) {
String metadataPath = kvContainerData.getMetadataPath();
Preconditions.checkNotNull(metadataPath);
return Paths.get(metadataPath);
}
}

View File

@ -76,7 +76,10 @@ public class ChunkManagerImpl implements ChunkManager {
HddsVolume volume = containerData.getVolume();
VolumeIOStats volumeIOStats = volume.getVolumeIOStats();
File chunkFile = ChunkUtils.validateChunk(containerData, info);
File chunkFile = ChunkUtils.getChunkFile(containerData, info);
boolean isOverwrite = ChunkUtils.validateChunkForOverwrite(
chunkFile, info);
File tmpChunkFile = getTmpChunkFile(chunkFile, info);
LOG.debug("writing chunk:{} chunk stage:{} chunk file:{} tmp chunk file",
@ -101,8 +104,9 @@ public class ChunkManagerImpl implements ChunkManager {
case COMBINED:
// directly write to the chunk file
ChunkUtils.writeData(chunkFile, info, data, volumeIOStats);
// Increment container stats here, as we directly write to chunk file.
containerData.incrBytesUsed(info.getLen());
if (!isOverwrite) {
containerData.incrBytesUsed(info.getLen());
}
containerData.incrWriteCount();
containerData.incrWriteBytes(info.getLen());
break;

View File

@ -16,9 +16,13 @@
* the License.
*/
package org.apache.hadoop.ozone.container.common.statemachine.background;
package org.apache.hadoop.ozone.container.keyvalue.statemachine.background;
import com.google.common.collect.Lists;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
import org.apache.ratis.shaded.com.google.protobuf
.InvalidProtocolBufferException;
import org.apache.commons.io.FileUtils;
@ -28,10 +32,6 @@ import org.apache.hadoop.hdds.scm.container.common.helpers
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.utils.BackgroundService;
import org.apache.hadoop.utils.BackgroundTask;
@ -62,12 +62,13 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
* A per-datanode container block deleting service takes in charge
* of deleting staled ozone blocks.
*/
// TODO: Fix BlockDeletingService to work with new StorageLayer
public class BlockDeletingService extends BackgroundService{
private static final Logger LOG =
LoggerFactory.getLogger(BlockDeletingService.class);
private final ContainerManager containerManager;
ContainerSet containerSet;
private final Configuration conf;
// Throttle number of blocks to delete per task,
@ -82,12 +83,12 @@ public class BlockDeletingService extends BackgroundService{
// Core pool size for container tasks
private final static int BLOCK_DELETING_SERVICE_CORE_POOL_SIZE = 10;
public BlockDeletingService(ContainerManager containerManager,
long serviceInterval, long serviceTimeout, TimeUnit unit,
Configuration conf) {
super("BlockDeletingService", serviceInterval, unit,
BLOCK_DELETING_SERVICE_CORE_POOL_SIZE, serviceTimeout);
this.containerManager = containerManager;
public BlockDeletingService(ContainerSet containerSet,
long serviceInterval, long serviceTimeout, Configuration conf) {
super("BlockDeletingService", serviceInterval,
TimeUnit.MILLISECONDS, BLOCK_DELETING_SERVICE_CORE_POOL_SIZE,
serviceTimeout);
this.containerSet = containerSet;
this.conf = conf;
this.blockLimitPerTask = conf.getInt(
OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER,
@ -108,7 +109,7 @@ public class BlockDeletingService extends BackgroundService{
// We must ensure there is no empty container in this result.
// The chosen result depends on what container deletion policy is
// configured.
containers = containerManager.chooseContainerForBlockDeletion(
containers = containerSet.chooseContainerForBlockDeletion(
containerLimitPerInterval);
LOG.info("Plan to choose {} containers for block deletion, "
+ "actually returns {} valid containers.",
@ -174,7 +175,8 @@ public class BlockDeletingService extends BackgroundService{
ContainerBackgroundTaskResult crr = new ContainerBackgroundTaskResult();
long startTime = Time.monotonicNow();
// Scan container's db and get list of under deletion blocks
MetadataStore meta = KeyUtils.getDB(containerData, conf);
MetadataStore meta = KeyUtils.getDB(
(KeyValueContainerData) containerData, conf);
// # of blocks to delete is throttled
KeyPrefixFilter filter =
new KeyPrefixFilter().addFilter(OzoneConsts.DELETING_KEY_PREFIX);
@ -188,7 +190,7 @@ public class BlockDeletingService extends BackgroundService{
List<String> succeedBlocks = new LinkedList<>();
LOG.debug("Container : {}, To-Delete blocks : {}",
containerData.getContainerID(), toDeleteBlocks.size());
File dataDir = ContainerUtils.getDataDirectory(containerData).toFile();
File dataDir = new File(containerData.getDataPath());
if (!dataDir.exists() || !dataDir.isDirectory()) {
LOG.error("Invalid container data dir {} : "
+ "not exist or not a directory", dataDir.getAbsolutePath());
@ -227,8 +229,7 @@ public class BlockDeletingService extends BackgroundService{
});
meta.writeBatch(batch);
// update count of pending deletion blocks in in-memory container status
containerManager.decrPendingDeletionBlocks(succeedBlocks.size(),
containerData.getContainerID());
containerData.decrPendingDeletionBlocks(succeedBlocks.size());
if (!succeedBlocks.isEmpty()) {
LOG.info("Container: {}, deleted blocks: {}, task elapsed time: {}ms",

View File

@ -15,4 +15,4 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.statemachine.background;
package org.apache.hadoop.ozone.container.keyvalue.statemachine.background;

View File

@ -73,7 +73,7 @@ public class OzoneContainer {
conf) throws IOException {
this.dnDetails = datanodeDetails;
this.config = conf;
this.volumeSet = new VolumeSet(datanodeDetails, conf);
this.volumeSet = new VolumeSet(datanodeDetails.getUuidString(), conf);
this.containerSet = new ContainerSet();
boolean useGrpc = this.config.getBoolean(
ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_KEY,

View File

@ -47,7 +47,7 @@ public class TestKeyValueContainerData {
MAXSIZE);
assertEquals(containerType, kvData.getContainerType());
assertEquals(containerId, kvData.getContainerId());
assertEquals(containerId, kvData.getContainerID());
assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN, kvData
.getState());
assertEquals(0, kvData.getMetadata().size());

View File

@ -64,7 +64,7 @@ public class TestContainerDataYaml {
// Read from .container file, and verify data.
KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
.readContainerFile(containerFile);
assertEquals(Long.MAX_VALUE, kvData.getContainerId());
assertEquals(Long.MAX_VALUE, kvData.getContainerID());
assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
.getContainerType());
assertEquals("RocksDB", kvData.getContainerDBType());
@ -92,7 +92,7 @@ public class TestContainerDataYaml {
containerFile);
// verify data.
assertEquals(Long.MAX_VALUE, kvData.getContainerId());
assertEquals(Long.MAX_VALUE, kvData.getContainerID());
assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
.getContainerType());
assertEquals("RocksDB", kvData.getContainerDBType());
@ -150,7 +150,7 @@ public class TestContainerDataYaml {
assertEquals("RocksDB", kvData.getContainerDBType());
assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
.getContainerType());
assertEquals(9223372036854775807L, kvData.getContainerId());
assertEquals(9223372036854775807L, kvData.getContainerID());
assertEquals("/hdds/current/aed-fg4-hji-jkl/containerdir0/1", kvData
.getChunksPath());
assertEquals("/hdds/current/aed-fg4-hji-jkl/containerdir0/1", kvData

View File

@ -74,7 +74,7 @@ public class TestContainerSet {
.getContainer(containerId);
KeyValueContainerData keyValueContainerData = (KeyValueContainerData)
container.getContainerData();
assertEquals(containerId, keyValueContainerData.getContainerId());
assertEquals(containerId, keyValueContainerData.getContainerID());
assertEquals(state, keyValueContainerData.getState());
assertNull(containerSet.getContainer(1000L));
@ -97,7 +97,7 @@ public class TestContainerSet {
while(containerIterator.hasNext()) {
Container kv = containerIterator.next();
ContainerData containerData = kv.getContainerData();
long containerId = containerData.getContainerId();
long containerId = containerData.getContainerID();
if (containerId%2 == 0) {
assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED,
containerData.getState());
@ -117,7 +117,7 @@ public class TestContainerSet {
while (containerMapIterator.hasNext()) {
Container kv = containerMapIterator.next().getValue();
ContainerData containerData = kv.getContainerData();
long containerId = containerData.getContainerId();
long containerId = containerData.getContainerID();
if (containerId%2 == 0) {
assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED,
containerData.getState());
@ -155,8 +155,8 @@ public class TestContainerSet {
assertEquals(5, result.size());
for(ContainerData containerData : result) {
assertTrue(containerData.getContainerId() >=2 && containerData
.getContainerId()<=6);
assertTrue(containerData.getContainerID() >=2 && containerData
.getContainerID()<=6);
}
}

View File

@ -54,11 +54,7 @@ public class TestRoundRobinVolumeChoosingPolicy {
conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDirKey);
policy = ReflectionUtils.newInstance(
RoundRobinVolumeChoosingPolicy.class, null);
DatanodeDetails datanodeDetails = DatanodeDetails.newBuilder()
.setUuid(UUID.randomUUID().toString())
.setIpAddress(DUMMY_IP_ADDR)
.build();
VolumeSet volumeSet = new VolumeSet(datanodeDetails, conf);
VolumeSet volumeSet = new VolumeSet(UUID.randomUUID().toString(), conf);
volumes = volumeSet.getVolumesList();
}

View File

@ -53,11 +53,7 @@ public class TestVolumeSet {
private static final String DUMMY_IP_ADDR = "0.0.0.0";
private void initializeVolumeSet() throws Exception {
DatanodeDetails datanodeDetails = DatanodeDetails.newBuilder()
.setUuid(UUID.randomUUID().toString())
.setIpAddress(DUMMY_IP_ADDR)
.build();
volumeSet = new VolumeSet(datanodeDetails, conf);
volumeSet = new VolumeSet(UUID.randomUUID().toString(), conf);
}
@Rule

View File

@ -244,24 +244,6 @@ public class TestKeyValueContainer {
}
@Test
public void testUpdateContainerInvalidMetadata() throws IOException {
try {
keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
Map<String, String> metadata = new HashMap<>();
metadata.put("VOLUME", "ozone");
keyValueContainer.update(metadata, true);
//Trying to update again with same metadata
keyValueContainer.update(metadata, true);
fail("testUpdateContainerInvalidMetadata failed");
} catch (StorageContainerException ex) {
GenericTestUtils.assertExceptionContains("Container Metadata update " +
"error", ex);
assertEquals(ContainerProtos.Result.CONTAINER_METADATA_ERROR, ex
.getResult());
}
}
@Test
public void testUpdateContainerUnsupportedRequest() throws Exception {
try {
@ -275,7 +257,7 @@ public class TestKeyValueContainer {
fail("testUpdateContainerUnsupportedRequest failed");
} catch (StorageContainerException ex) {
GenericTestUtils.assertExceptionContains("Updating a closed container " +
"is not allowed", ex);
"without force option is not allowed", ex);
assertEquals(ContainerProtos.Result.UNSUPPORTED_REQUEST, ex
.getResult());
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.ozone.container.keyvalue;
import com.google.common.base.Supplier;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@ -30,6 +31,7 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
@ -71,12 +73,7 @@ public class TestKeyValueHandler {
conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, volume);
this.containerSet = new ContainerSet();
DatanodeDetails datanodeDetails = DatanodeDetails.newBuilder()
.setUuid(DATANODE_UUID)
.setHostName("localhost")
.setIpAddress("127.0.0.1")
.build();
this.volumeSet = new VolumeSet(datanodeDetails, conf);
this.volumeSet = new VolumeSet(DATANODE_UUID, conf);
this.dispatcher = new HddsDispatcher(conf, containerSet, volumeSet);
this.handler = (KeyValueHandler) dispatcher.getHandler(
@ -246,7 +243,7 @@ public class TestKeyValueHandler {
// Verify that new container is added to containerSet.
Container container = containerSet.getContainer(contId);
Assert.assertEquals(contId, container.getContainerData().getContainerId());
Assert.assertEquals(contId, container.getContainerData().getContainerID());
Assert.assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN,
container.getContainerState());
}

View File

@ -62,7 +62,7 @@ public class TestOzoneContainer {
conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, folder.getRoot()
.getAbsolutePath() + "," + folder.newFolder().getAbsolutePath());
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, folder.newFolder().getAbsolutePath());
volumeSet = new VolumeSet(datanodeDetails, conf);
volumeSet = new VolumeSet(datanodeDetails.getUuidString(), conf);
volumeChoosingPolicy = new RoundRobinVolumeChoosingPolicy();
for (int i=0; i<10; i++) {

View File

@ -19,8 +19,8 @@ package org.apache.hadoop.ozone.container.testutils;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
import org.apache.hadoop.ozone.container.common.statemachine.background
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.keyvalue.statemachine.background
.BlockDeletingService;
import java.util.concurrent.CountDownLatch;
@ -42,10 +42,9 @@ public class BlockDeletingServiceTestImpl
private Thread testingThread;
private AtomicInteger numOfProcessed = new AtomicInteger(0);
public BlockDeletingServiceTestImpl(ContainerManager containerManager,
public BlockDeletingServiceTestImpl(ContainerSet containerSet,
int serviceInterval, Configuration conf) {
super(containerManager, serviceInterval, SERVICE_TIMEOUT_IN_MILLISECONDS,
TimeUnit.MILLISECONDS, conf);
super(containerSet, serviceInterval, SERVICE_TIMEOUT_IN_MILLISECONDS, conf);
}
@VisibleForTesting

View File

@ -1,7 +1,7 @@
!<KeyValueContainerData>
containerDBType: RocksDB
chunksPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
containerId: 9223372036854775807
containerID: 9223372036854775807
containerType: KeyValueContainer
metadataPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
layOutVersion: 1

View File

@ -1,7 +1,7 @@
!<KeyValueContainerData>
containerDBType: RocksDB
chunksPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
containerId: 9223372036854775807
containerID: 9223372036854775807
containerType: KeyValueContainer
metadataPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
layOutVersion: 1

View File

@ -58,6 +58,8 @@ public final class ContainerTestHelper {
ContainerTestHelper.class);
private static Random r = new Random();
public static final int CONTAINER_MAX_SIZE_GB = 1;
/**
* Never constructed.
*/

View File

@ -29,14 +29,17 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
import org.apache.hadoop.ozone.container.testutils.BlockDeletingServiceTestImpl;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
import org.apache.hadoop.ozone.container.common.impl.RandomContainerDeletionChoosingPolicy;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
import org.apache.hadoop.ozone.container.common.statemachine.background.BlockDeletingService;
import org.apache.hadoop.ozone.container.keyvalue.statemachine.background
.BlockDeletingService;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
@ -44,6 +47,7 @@ import org.apache.hadoop.utils.BackgroundService;
import org.apache.hadoop.utils.MetadataKeyFilters;
import org.apache.hadoop.utils.MetadataStore;
import org.junit.Assert;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.BeforeClass;
import org.junit.Before;
@ -70,6 +74,8 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
/**
* Tests to test block deleting service.
*/
// TODO: Fix BlockDeletingService to work with new StorageLayer
@Ignore
public class TestBlockDeletingService {
private static final Logger LOG =
@ -101,36 +107,22 @@ public class TestBlockDeletingService {
FileUtils.deleteDirectory(testRoot);
}
private ContainerManager createContainerManager(Configuration conf)
throws Exception {
// use random container choosing policy for testing
conf.set(ScmConfigKeys.OZONE_SCM_CONTAINER_DELETION_CHOOSING_POLICY,
RandomContainerDeletionChoosingPolicy.class.getName());
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
containersDir.getAbsolutePath());
if (containersDir.exists()) {
FileUtils.deleteDirectory(containersDir);
}
ContainerManager containerManager = new ContainerManagerImpl();
List<StorageLocation> pathLists = new LinkedList<>();
pathLists.add(StorageLocation.parse(containersDir.getAbsolutePath()));
containerManager.init(conf, pathLists, TestUtils.getDatanodeDetails());
return containerManager;
}
/**
* A helper method to create some blocks and put them under deletion
* state for testing. This method directly updates container.db and
* creates some fake chunk files for testing.
*/
private void createToDeleteBlocks(ContainerManager mgr,
private void createToDeleteBlocks(ContainerSet containerSet,
Configuration conf, int numOfContainers, int numOfBlocksPerContainer,
int numOfChunksPerBlock, File chunkDir) throws IOException {
for (int x = 0; x < numOfContainers; x++) {
long containerID = ContainerTestHelper.getTestContainerID();
ContainerData data = new ContainerData(containerID, conf);
mgr.createContainer(data);
data = mgr.readContainer(containerID);
KeyValueContainerData data = new KeyValueContainerData(containerID,
ContainerTestHelper.CONTAINER_MAX_SIZE_GB);
Container container = new KeyValueContainer(data, conf);
containerSet.addContainer(container);
data = (KeyValueContainerData) containerSet.getContainer(
containerID).getContainerData();
MetadataStore metadata = KeyUtils.getDB(data, conf);
for (int j = 0; j<numOfBlocksPerContainer; j++) {
BlockID blockID =
@ -198,29 +190,28 @@ public class TestBlockDeletingService {
Configuration conf = new OzoneConfiguration();
conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
ContainerManager containerManager = createContainerManager(conf);
createToDeleteBlocks(containerManager, conf, 1, 3, 1, chunksDir);
ContainerSet containerSet = new ContainerSet();
createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
BlockDeletingServiceTestImpl svc =
new BlockDeletingServiceTestImpl(containerManager, 1000, conf);
new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
svc.start();
GenericTestUtils.waitFor(() -> svc.isStarted(), 100, 3000);
// Ensure 1 container was created
List<ContainerData> containerData = Lists.newArrayList();
containerManager.listContainer(0L, 1, containerData);
containerSet.listContainer(0L, 1, containerData);
Assert.assertEquals(1, containerData.size());
MetadataStore meta = KeyUtils.getDB(containerData.get(0), conf);
Map<Long, ContainerData> containerMap =
((ContainerManagerImpl) containerManager).getContainerMap();
long transactionId =
containerMap.get(containerData.get(0).getContainerID())
.getDeleteTransactionId();
MetadataStore meta = KeyUtils.getDB(
(KeyValueContainerData) containerData.get(0), conf);
Map<Long, Container> containerMap = containerSet.getContainerMap();
// Number of deleted blocks in container should be equal to 0 before
// block delete
Assert.assertEquals(0, transactionId);
// TODO : Implement deleteTransactionID in ContainerData.
// Assert.assertEquals(0, transactionId);
// Ensure there are 3 blocks under deletion and 0 deleted blocks
Assert.assertEquals(3, getUnderDeletionBlocksCount(meta));
@ -240,7 +231,6 @@ public class TestBlockDeletingService {
Assert.assertEquals(3, getDeletedBlocksCount(meta));
svc.shutdown();
shutdownContainerMangaer(containerManager);
}
@Test
@ -250,12 +240,12 @@ public class TestBlockDeletingService {
TimeUnit.MILLISECONDS);
conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 10);
ContainerManager containerManager = createContainerManager(conf);
ContainerSet containerSet = new ContainerSet();
// Create 1 container with 100 blocks
createToDeleteBlocks(containerManager, conf, 1, 100, 1, chunksDir);
createToDeleteBlocks(containerSet, conf, 1, 100, 1, chunksDir);
BlockDeletingServiceTestImpl service =
new BlockDeletingServiceTestImpl(containerManager, 1000, conf);
new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
service.start();
GenericTestUtils.waitFor(() -> service.isStarted(), 100, 3000);
@ -269,7 +259,6 @@ public class TestBlockDeletingService {
// Shutdown service and verify all threads are stopped
service.shutdown();
GenericTestUtils.waitFor(() -> service.getThreadCount() == 0, 100, 1000);
shutdownContainerMangaer(containerManager);
}
@Test
@ -277,14 +266,13 @@ public class TestBlockDeletingService {
Configuration conf = new OzoneConfiguration();
conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
ContainerManager containerManager = createContainerManager(conf);
createToDeleteBlocks(containerManager, conf, 1, 3, 1, chunksDir);
ContainerSet containerSet = new ContainerSet();
createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
// set timeout value as 1ns to trigger timeout behavior
long timeout = 1;
BlockDeletingService svc = new BlockDeletingService(containerManager,
TimeUnit.MILLISECONDS.toNanos(1000), timeout, TimeUnit.NANOSECONDS,
conf);
BlockDeletingService svc =
new BlockDeletingService(containerSet, 1000, timeout, conf);
svc.start();
LogCapturer log = LogCapturer.captureLogs(BackgroundService.LOG);
@ -303,16 +291,15 @@ public class TestBlockDeletingService {
// test for normal case that doesn't have timeout limitation
timeout = 0;
createToDeleteBlocks(containerManager, conf, 1, 3, 1, chunksDir);
svc = new BlockDeletingService(containerManager,
TimeUnit.MILLISECONDS.toNanos(1000), timeout, TimeUnit.NANOSECONDS,
conf);
createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
svc = new BlockDeletingService(containerSet, 1000, timeout, conf);
svc.start();
// get container meta data
List<ContainerData> containerData = Lists.newArrayList();
containerManager.listContainer(0L, 1, containerData);
MetadataStore meta = KeyUtils.getDB(containerData.get(0), conf);
containerSet.listContainer(0L, 1, containerData);
MetadataStore meta = KeyUtils.getDB(
(KeyValueContainerData) containerData.get(0), conf);
LogCapturer newLog = LogCapturer.captureLogs(BackgroundService.LOG);
GenericTestUtils.waitFor(() -> {
@ -331,7 +318,6 @@ public class TestBlockDeletingService {
Assert.assertTrue(!newLog.getOutput().contains(
"Background task executes timed out, retrying in next interval"));
svc.shutdown();
shutdownContainerMangaer(containerManager);
}
@Test(timeout = 30000)
@ -349,11 +335,11 @@ public class TestBlockDeletingService {
// Process 1 container per interval
conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 1);
conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 1);
ContainerManager containerManager = createContainerManager(conf);
createToDeleteBlocks(containerManager, conf, 2, 1, 10, chunksDir);
ContainerSet containerSet = new ContainerSet();
createToDeleteBlocks(containerSet, conf, 2, 1, 10, chunksDir);
BlockDeletingServiceTestImpl service =
new BlockDeletingServiceTestImpl(containerManager, 1000, conf);
new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
service.start();
try {
@ -363,7 +349,6 @@ public class TestBlockDeletingService {
Assert.assertEquals(10, chunksDir.listFiles().length);
} finally {
service.shutdown();
shutdownContainerMangaer(containerManager);
}
}
@ -383,14 +368,14 @@ public class TestBlockDeletingService {
Configuration conf = new OzoneConfiguration();
conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
ContainerManager containerManager = createContainerManager(conf);
createToDeleteBlocks(containerManager, conf, 5, 3, 1, chunksDir);
ContainerSet containerSet = new ContainerSet();
createToDeleteBlocks(containerSet, conf, 5, 3, 1, chunksDir);
// Make sure chunks are created
Assert.assertEquals(15, chunksDir.listFiles().length);
BlockDeletingServiceTestImpl service =
new BlockDeletingServiceTestImpl(containerManager, 1000, conf);
new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
service.start();
try {
@ -407,17 +392,6 @@ public class TestBlockDeletingService {
Assert.assertEquals(0, chunksDir.listFiles().length);
} finally {
service.shutdown();
shutdownContainerMangaer(containerManager);
}
}
private void shutdownContainerMangaer(ContainerManager mgr)
throws IOException {
mgr.writeLock();
try {
mgr.shutdown();
} finally {
mgr.writeUnlock();
}
}
}

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.ozone.container.common.impl;
import static org.apache.hadoop.ozone.container.ContainerTestHelper.createSingleNodePipeline;
import java.io.File;
import java.io.IOException;
import java.util.HashMap;
@ -36,23 +34,25 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.utils.MetadataStore;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
/**
* The class for testing container deletion choosing policy.
*/
@Ignore
public class TestContainerDeletionChoosingPolicy {
private static String path;
private static ContainerManagerImpl containerManager;
private static ContainerSet containerSet;
private static OzoneConfiguration conf;
@Before
@ -65,18 +65,6 @@ public class TestContainerDeletionChoosingPolicy {
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
}
@After
public void shutdown() throws IOException {
FileUtils.deleteDirectory(new File(path));
containerManager.writeLock();
try{
containerManager.shutdown();
} finally {
containerManager.writeUnlock();
}
}
@Test
public void testRandomChoosingPolicy() throws IOException {
File containerDir = new File(path);
@ -89,25 +77,26 @@ public class TestContainerDeletionChoosingPolicy {
RandomContainerDeletionChoosingPolicy.class.getName());
List<StorageLocation> pathLists = new LinkedList<>();
pathLists.add(StorageLocation.parse(containerDir.getAbsolutePath()));
containerManager = new ContainerManagerImpl();
containerManager.init(conf, pathLists, TestUtils.getDatanodeDetails());
containerSet = new ContainerSet();
int numContainers = 10;
for (int i = 0; i < numContainers; i++) {
ContainerData data = new ContainerData(new Long(i), conf);
containerManager.createContainer(data);
KeyValueContainerData data = new KeyValueContainerData(new Long(i),
ContainerTestHelper.CONTAINER_MAX_SIZE_GB);
KeyValueContainer container = new KeyValueContainer(data, conf);
containerSet.addContainer(container);
Assert.assertTrue(
containerManager.getContainerMap().containsKey(data.getContainerID()));
containerSet.getContainerMap().containsKey(data.getContainerID()));
}
List<ContainerData> result0 = containerManager
List<ContainerData> result0 = containerSet
.chooseContainerForBlockDeletion(5);
Assert.assertEquals(5, result0.size());
// test random choosing
List<ContainerData> result1 = containerManager
List<ContainerData> result1 = containerSet
.chooseContainerForBlockDeletion(numContainers);
List<ContainerData> result2 = containerManager
List<ContainerData> result2 = containerSet
.chooseContainerForBlockDeletion(numContainers);
boolean hasShuffled = false;
@ -133,9 +122,8 @@ public class TestContainerDeletionChoosingPolicy {
TopNOrderedContainerDeletionChoosingPolicy.class.getName());
List<StorageLocation> pathLists = new LinkedList<>();
pathLists.add(StorageLocation.parse(containerDir.getAbsolutePath()));
containerManager = new ContainerManagerImpl();
containerSet = new ContainerSet();
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
containerManager.init(conf, pathLists, datanodeDetails);
int numContainers = 10;
Random random = new Random();
@ -143,10 +131,12 @@ public class TestContainerDeletionChoosingPolicy {
// create [numContainers + 1] containers
for (int i = 0; i <= numContainers; i++) {
long containerId = RandomUtils.nextLong();
ContainerData data = new ContainerData(containerId, conf);
containerManager.createContainer(data);
KeyValueContainerData data = new KeyValueContainerData(new Long(i),
ContainerTestHelper.CONTAINER_MAX_SIZE_GB);
KeyValueContainer container = new KeyValueContainer(data, conf);
containerSet.addContainer(container);
Assert.assertTrue(
containerManager.getContainerMap().containsKey(containerId));
containerSet.getContainerMap().containsKey(containerId));
// don't create deletion blocks in the last container.
if (i == numContainers) {
@ -167,16 +157,11 @@ public class TestContainerDeletionChoosingPolicy {
}
}
containerManager.writeLock();
containerManager.shutdown();
containerManager.writeUnlock();
containerManager.init(conf, pathLists, datanodeDetails);
List<ContainerData> result0 = containerManager
List<ContainerData> result0 = containerSet
.chooseContainerForBlockDeletion(5);
Assert.assertEquals(5, result0.size());
List<ContainerData> result1 = containerManager
List<ContainerData> result1 = containerSet
.chooseContainerForBlockDeletion(numContainers + 1);
// the empty deletion blocks container should not be chosen
Assert.assertEquals(numContainers, result1.size());

View File

@ -17,26 +17,38 @@
package org.apache.hadoop.ozone.container.common.impl;
import com.google.common.collect.Maps;
import org.apache.commons.codec.binary.Hex;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
import org.apache.hadoop.ozone.container.common.volume
.RoundRobinVolumeChoosingPolicy;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
import org.apache.hadoop.ozone.container.keyvalue.helpers
.KeyValueContainerLocationUtil;
import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerImpl;
import org.apache.hadoop.ozone.container.keyvalue.impl.KeyManagerImpl;
import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
import org.apache.hadoop.ozone.container.keyvalue.interfaces.KeyManager;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.utils.MetadataStore;
import org.junit.After;
import org.junit.AfterClass;
@ -51,7 +63,6 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.nio.file.DirectoryStream;
import java.nio.file.Files;
@ -65,11 +76,10 @@ import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.ArrayList;
import java.util.UUID;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ROOT_PREFIX;
import static org.apache.hadoop.ozone.container.ContainerTestHelper
.createSingleNodePipeline;
import static org.apache.hadoop.ozone.container.ContainerTestHelper.getChunk;
import static org.apache.hadoop.ozone.container.ContainerTestHelper.getData;
import static org.apache.hadoop.ozone.container.ContainerTestHelper
@ -95,40 +105,42 @@ public class TestContainerPersistence {
private static Logger log =
LoggerFactory.getLogger(TestContainerPersistence.class);
private static String hddsPath;
private static String path;
private static ContainerManagerImpl containerManager;
private static ChunkManagerImpl chunkManager;
private static KeyManagerImpl keyManager;
private static OzoneConfiguration conf;
private static List<StorageLocation> pathLists = new LinkedList<>();
private Long containerID = 8888L;;
private static final String datanodeUuid = UUID.randomUUID().toString();
private static final String scmId = UUID.randomUUID().toString();
private static ContainerSet containerSet;
private static VolumeSet volumeSet;
private static VolumeChoosingPolicy volumeChoosingPolicy;
private static KeyManager keyManager;
private static ChunkManager chunkManager;
@BeforeClass
public static void init() throws Throwable {
conf = new OzoneConfiguration();
path = GenericTestUtils
hddsPath = GenericTestUtils
.getTempPath(TestContainerPersistence.class.getSimpleName());
path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
path = hddsPath + conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, hddsPath);
File containerDir = new File(path);
if (containerDir.exists()) {
FileUtils.deleteDirectory(new File(path));
}
Assert.assertTrue(containerDir.mkdirs());
containerManager = new ContainerManagerImpl();
chunkManager = new ChunkManagerImpl(containerManager);
containerManager.setChunkManager(chunkManager);
keyManager = new KeyManagerImpl(containerManager, conf);
containerManager.setKeyManager(keyManager);
volumeChoosingPolicy = new RoundRobinVolumeChoosingPolicy();
}
@AfterClass
public static void shutdown() throws IOException {
FileUtils.deleteDirectory(new File(path));
FileUtils.deleteDirectory(new File(hddsPath));
}
@Before
@ -140,7 +152,10 @@ public class TestContainerPersistence {
Paths.get(path).resolve(CONTAINER_ROOT_PREFIX).toString());
pathLists.clear();
containerManager.getContainerMap().clear();
containerSet = new ContainerSet();
volumeSet = new VolumeSet(datanodeUuid, conf);
keyManager = new KeyManagerImpl(conf);
chunkManager = new ChunkManagerImpl();
if (!new File(loc.getNormalizedUri()).mkdirs()) {
throw new IOException("unable to create paths. " +
@ -152,26 +167,18 @@ public class TestContainerPersistence {
StorageLocation location = StorageLocation.parse(dir);
FileUtils.forceMkdir(new File(location.getNormalizedUri()));
}
containerManager.init(conf, pathLists, TestUtils.getDatanodeDetails());
}
}
@After
public void cleanupDir() throws IOException {
// Shutdown containerManager
containerManager.writeLock();
try {
containerManager.shutdown();
} finally {
containerManager.writeUnlock();
}
// Clean up SCM metadata
log.info("Deleting {}", path);
FileUtils.deleteDirectory(new File(path));
log.info("Deleting {}", hddsPath);
FileUtils.deleteDirectory(new File(hddsPath));
// Clean up SCM datanode container metadata/data
for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
for (String dir : conf.getStrings(ScmConfigKeys.HDDS_DATANODE_DIR_KEY)) {
StorageLocation location = StorageLocation.parse(dir);
FileUtils.deleteDirectory(new File(location.getNormalizedUri()));
}
@ -181,32 +188,39 @@ public class TestContainerPersistence {
return ContainerTestHelper.getTestContainerID();
}
private Container addContainer(ContainerSet containerSet, long containerID)
throws IOException {
KeyValueContainerData data = new KeyValueContainerData(containerID,
ContainerTestHelper.CONTAINER_MAX_SIZE_GB);
data.addMetadata("VOLUME", "shire");
data.addMetadata("owner)", "bilbo");
KeyValueContainer container = new KeyValueContainer(data, conf);
container.create(volumeSet, volumeChoosingPolicy, scmId);
containerSet.addContainer(container);
return container;
}
@Test
public void testCreateContainer() throws Exception {
long testContainerID = getTestContainerID();
ContainerData data = new ContainerData(testContainerID, conf);
data.addMetadata("VOLUME", "shire");
data.addMetadata("owner)", "bilbo");
containerManager.createContainer(data);
Assert.assertTrue(containerManager.getContainerMap()
addContainer(containerSet, testContainerID);
Assert.assertTrue(containerSet.getContainerMap()
.containsKey(testContainerID));
ContainerData cData = containerManager
.getContainerMap().get(testContainerID);
KeyValueContainerData kvData =
(KeyValueContainerData) containerSet.getContainer(testContainerID)
.getContainerData();
Assert.assertNotNull(cData);
Assert.assertNotNull(cData.getContainerPath());
Assert.assertNotNull(cData.getDBPath());
Assert.assertNotNull(kvData);
Assert.assertTrue(new File(kvData.getMetadataPath()).exists());
Assert.assertTrue(new File(kvData.getChunksPath()).exists());
Assert.assertTrue(kvData.getDbFile().exists());
Assert.assertTrue(new File(cData.getContainerPath())
.exists());
Path meta = Paths.get(cData.getDBPath()).getParent();
Path meta = kvData.getDbFile().toPath().getParent();
Assert.assertTrue(meta != null && Files.exists(meta));
MetadataStore store = null;
try {
store = KeyUtils.getDB(cData, conf);
store = KeyUtils.getDB(kvData, conf);
Assert.assertNotNull(store);
} finally {
if (store != null) {
@ -219,12 +233,9 @@ public class TestContainerPersistence {
public void testCreateDuplicateContainer() throws Exception {
long testContainerID = getTestContainerID();
ContainerData data = new ContainerData(testContainerID, conf);
data.addMetadata("VOLUME", "shire");
data.addMetadata("owner)", "bilbo");
containerManager.createContainer(data);
Container container = addContainer(containerSet, testContainerID);
try {
containerManager.createContainer(data);
containerSet.addContainer(container);
fail("Expected Exception not thrown.");
} catch (IOException ex) {
Assert.assertNotNull(ex);
@ -237,54 +248,40 @@ public class TestContainerPersistence {
Thread.sleep(100);
long testContainerID2 = getTestContainerID();
ContainerData data = new ContainerData(testContainerID1, conf);
data.addMetadata("VOLUME", "shire");
data.addMetadata("owner)", "bilbo");
containerManager.createContainer(data);
containerManager.closeContainer(testContainerID1);
Container container1 = addContainer(containerSet, testContainerID1);
container1.close();
data = new ContainerData(testContainerID2, conf);
data.addMetadata("VOLUME", "shire");
data.addMetadata("owner)", "bilbo");
containerManager.createContainer(data);
containerManager.closeContainer(testContainerID2);
Container container2 = addContainer(containerSet, testContainerID2);
Assert.assertTrue(containerManager.getContainerMap()
Assert.assertTrue(containerSet.getContainerMap()
.containsKey(testContainerID1));
Assert.assertTrue(containerManager.getContainerMap()
Assert.assertTrue(containerSet.getContainerMap()
.containsKey(testContainerID2));
containerManager.deleteContainer(testContainerID1, false);
Assert.assertFalse(containerManager.getContainerMap()
container1.delete(false);
containerSet.removeContainer(testContainerID1);
Assert.assertFalse(containerSet.getContainerMap()
.containsKey(testContainerID1));
// Let us make sure that we are able to re-use a container name after
// delete.
data = new ContainerData(testContainerID1, conf);
data.addMetadata("VOLUME", "shire");
data.addMetadata("owner)", "bilbo");
containerManager.createContainer(data);
containerManager.closeContainer(testContainerID1);
// Assert we still have both containers.
Assert.assertTrue(containerManager.getContainerMap()
.containsKey(testContainerID1));
Assert.assertTrue(containerManager.getContainerMap()
.containsKey(testContainerID2));
// Add some key to a container and then delete.
// Delete should fail because the container is no longer empty.
// Adding key to a deleted container should fail.
exception.expect(StorageContainerException.class);
exception.expectMessage("Error opening DB.");
BlockID blockID1 = ContainerTestHelper.getTestBlockID(testContainerID1);
KeyData someKey = new KeyData(blockID1);
someKey.setChunks(new LinkedList<ContainerProtos.ChunkInfo>());
keyManager.putKey(someKey);
KeyData someKey1 = new KeyData(blockID1);
someKey1.setChunks(new LinkedList<ContainerProtos.ChunkInfo>());
keyManager.putKey(container1, someKey1);
// Deleting a non-empty container should fail.
BlockID blockID2 = ContainerTestHelper.getTestBlockID(testContainerID2);
KeyData someKey2 = new KeyData(blockID2);
someKey2.setChunks(new LinkedList<ContainerProtos.ChunkInfo>());
keyManager.putKey(container2, someKey2);
exception.expect(StorageContainerException.class);
exception.expectMessage(
"Container cannot be deleted because it is not empty.");
containerManager.deleteContainer(testContainerID1, false);
Assert.assertTrue(containerManager.getContainerMap()
container2.delete(false);
Assert.assertTrue(containerSet.getContainerMap()
.containsKey(testContainerID1));
}
@ -295,21 +292,21 @@ public class TestContainerPersistence {
for (int i = 0; i < count; i++) {
long testContainerID = getTestContainerID();
ContainerData data = new ContainerData(testContainerID, conf);
containerManager.createContainer(data);
Container container = addContainer(containerSet, testContainerID);
// Close a bunch of containers.
// Put closed container names to a list.
if (i%3 == 0) {
containerManager.closeContainer(testContainerID);
containerIDs.add(testContainerID);
container.close();
}
containerIDs.add(testContainerID);
}
// The container report only returns reports of closed containers.
List<ContainerData> reports = containerManager.getClosedContainerReports();
Assert.assertEquals(4, reports.size());
for(ContainerData report : reports) {
// ContainerSet#getContainerReport currently returns all containers (open
// and closed) reports.
List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports =
containerSet.getContainerReport().getReportsList();
Assert.assertEquals(10, reports.size());
for(StorageContainerDatanodeProtocolProtos.ContainerInfo report : reports) {
long actualContainerID = report.getContainerID();
Assert.assertTrue(containerIDs.remove(actualContainerID));
}
@ -324,24 +321,21 @@ public class TestContainerPersistence {
*/
@Test
public void testListContainer() throws IOException {
final int count = 50;
final int count = 10;
final int step = 5;
Map<Long, ContainerData> testMap = new HashMap<>();
for (int x = 0; x < count; x++) {
long testContainerID = getTestContainerID();
ContainerData data = new ContainerData(testContainerID, conf);
data.addMetadata("VOLUME", "shire");
data.addMetadata("owner)", "bilbo");
containerManager.createContainer(data);
testMap.put(testContainerID, data);
Container container = addContainer(containerSet, testContainerID);
testMap.put(testContainerID, container.getContainerData());
}
int counter = 0;
long prevKey = 0;
List<ContainerData> results = new LinkedList<>();
while (counter < count) {
containerManager.listContainer(prevKey, step, results);
containerSet.listContainer(prevKey, step, results);
for (int y = 0; y < results.size(); y++) {
testMap.remove(results.get(y).getContainerID());
}
@ -350,7 +344,7 @@ public class TestContainerPersistence {
//Assert that container is returning results in a sorted fashion.
Assert.assertTrue(prevKey < nextKey);
prevKey = nextKey;
prevKey = nextKey + 1;
results.clear();
}
// Assert that we listed all the keys that we had put into
@ -358,22 +352,18 @@ public class TestContainerPersistence {
Assert.assertTrue(testMap.isEmpty());
}
private ChunkInfo writeChunkHelper(BlockID blockID,
Pipeline pipeline) throws IOException,
NoSuchAlgorithmException {
private ChunkInfo writeChunkHelper(BlockID blockID)
throws IOException, NoSuchAlgorithmException {
final int datalen = 1024;
long testContainerID = blockID.getContainerID();
ContainerData cData = new ContainerData(testContainerID, conf);
cData.addMetadata("VOLUME", "shire");
cData.addMetadata("owner", "bilbo");
if(!containerManager.getContainerMap()
.containsKey(testContainerID)) {
containerManager.createContainer(cData);
Container container = containerSet.getContainer(testContainerID);
if (container == null) {
container = addContainer(containerSet, testContainerID);
}
ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
byte[] data = getData(datalen);
setDataChecksum(info, data);
chunkManager.writeChunk(blockID, info, data, COMBINED);
chunkManager.writeChunk(container, blockID, info, data, COMBINED);
return info;
}
@ -389,8 +379,7 @@ public class TestContainerPersistence {
NoSuchAlgorithmException {
BlockID blockID = ContainerTestHelper.
getTestBlockID(getTestContainerID());
Pipeline pipeline = createSingleNodePipeline();
writeChunkHelper(blockID, pipeline);
writeChunkHelper(blockID);
}
/**
@ -407,27 +396,22 @@ public class TestContainerPersistence {
final int chunkCount = 1024;
long testContainerID = getTestContainerID();
Container container = addContainer(containerSet, testContainerID);
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
Map<String, ChunkInfo> fileHashMap = new HashMap<>();
ContainerData cData = new ContainerData(testContainerID, conf);
cData.addMetadata("VOLUME", "shire");
cData.addMetadata("owner)", "bilbo");
containerManager.createContainer(cData);
BlockID blockID = ContainerTestHelper.
getTestBlockID(testContainerID);
for (int x = 0; x < chunkCount; x++) {
ChunkInfo info = getChunk(blockID.getLocalID(), x, 0, datalen);
byte[] data = getData(datalen);
setDataChecksum(info, data);
chunkManager.writeChunk(blockID, info, data, COMBINED);
chunkManager.writeChunk(container, blockID, info, data, COMBINED);
String fileName = String.format("%s.data.%d", blockID.getLocalID(), x);
fileHashMap.put(fileName, info);
}
ContainerData cNewData = containerManager.readContainer(testContainerID);
ContainerData cNewData = container.getContainerData();
Assert.assertNotNull(cNewData);
Path dataDir = ContainerUtils.getDataDirectory(cNewData);
Path dataDir = Paths.get(cNewData.getDataPath());
String globFormat = String.format("%s.data.*", blockID.getLocalID());
MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
@ -451,7 +435,7 @@ public class TestContainerPersistence {
for (int x = 0; x < chunkCount; x++) {
String fileName = String.format("%s.data.%d", blockID.getLocalID(), x);
ChunkInfo info = fileHashMap.get(fileName);
byte[] data = chunkManager.readChunk(blockID, info);
byte[] data = chunkManager.readChunk(container, blockID, info);
sha.update(data);
Assert.assertEquals(Hex.encodeHexString(sha.digest()),
info.getChecksum());
@ -472,23 +456,19 @@ public class TestContainerPersistence {
final int length = datalen/2;
long testContainerID = getTestContainerID();
BlockID blockID = ContainerTestHelper.
getTestBlockID(testContainerID);
Container container = addContainer(containerSet, testContainerID);
ContainerData cData = new ContainerData(testContainerID, conf);
cData.addMetadata("VOLUME", "shire");
cData.addMetadata("owner)", "bilbo");
containerManager.createContainer(cData);
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
byte[] data = getData(datalen);
setDataChecksum(info, data);
chunkManager.writeChunk(blockID, info, data, COMBINED);
chunkManager.writeChunk(container, blockID, info, data, COMBINED);
byte[] readData = chunkManager.readChunk(blockID, info);
byte[] readData = chunkManager.readChunk(container, blockID, info);
assertTrue(Arrays.equals(data, readData));
ChunkInfo info2 = getChunk(blockID.getLocalID(), 0, start, length);
byte[] readData2 = chunkManager.readChunk(blockID, info2);
byte[] readData2 = chunkManager.readChunk(container, blockID, info2);
assertEquals(length, readData2.length);
assertTrue(Arrays.equals(
Arrays.copyOfRange(data, start, start + length), readData2));
@ -507,31 +487,29 @@ public class TestContainerPersistence {
final int datalen = 1024;
long testContainerID = getTestContainerID();
BlockID blockID = ContainerTestHelper.
getTestBlockID(testContainerID);
Container container = addContainer(containerSet, testContainerID);
ContainerData cData = new ContainerData(testContainerID, conf);
cData.addMetadata("VOLUME", "shire");
cData.addMetadata("owner)", "bilbo");
containerManager.createContainer(cData);
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
byte[] data = getData(datalen);
setDataChecksum(info, data);
chunkManager.writeChunk(blockID, info, data, COMBINED);
chunkManager.writeChunk(container, blockID, info, data, COMBINED);
try {
chunkManager.writeChunk(blockID, info, data, COMBINED);
} catch (IOException ex) {
Assert.assertTrue(ex.getCause().getMessage().contains(
chunkManager.writeChunk(container, blockID, info, data, COMBINED);
} catch (StorageContainerException ex) {
Assert.assertTrue(ex.getMessage().contains(
"Rejecting write chunk request. OverWrite flag required"));
Assert.assertEquals(ex.getResult(),
ContainerProtos.Result.OVERWRITE_FLAG_REQUIRED);
}
// With the overwrite flag it should work now.
info.addMetadata(OzoneConsts.CHUNK_OVERWRITE, "true");
chunkManager.writeChunk(blockID, info, data, COMBINED);
long bytesUsed = containerManager.getBytesUsed(testContainerID);
chunkManager.writeChunk(container, blockID, info, data, COMBINED);
long bytesUsed = container.getContainerData().getBytesUsed();
Assert.assertEquals(datalen, bytesUsed);
long bytesWrite = containerManager.getWriteBytes(testContainerID);
long bytesWrite = container.getContainerData().getWriteBytes();
Assert.assertEquals(datalen * 2, bytesWrite);
}
@ -549,13 +527,9 @@ public class TestContainerPersistence {
final int chunkCount = 1024;
long testContainerID = getTestContainerID();
BlockID blockID = ContainerTestHelper.
getTestBlockID(testContainerID);
Container container = addContainer(containerSet, testContainerID);
ContainerData cData = new ContainerData(testContainerID, conf);
cData.addMetadata("VOLUME", "shire");
cData.addMetadata("owner)", "bilbo");
containerManager.createContainer(cData);
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
MessageDigest oldSha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
for (int x = 0; x < chunkCount; x++) {
// we are writing to the same chunk file but at different offsets.
@ -564,12 +538,12 @@ public class TestContainerPersistence {
byte[] data = getData(datalen);
oldSha.update(data);
setDataChecksum(info, data);
chunkManager.writeChunk(blockID, info, data, COMBINED);
chunkManager.writeChunk(container, blockID, info, data, COMBINED);
}
// Request to read the whole data in a single go.
ChunkInfo largeChunk = getChunk(blockID.getLocalID(), 0, 0, datalen * chunkCount);
byte[] newdata = chunkManager.readChunk(blockID, largeChunk);
byte[] newdata = chunkManager.readChunk(container, blockID, largeChunk);
MessageDigest newSha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
newSha.update(newdata);
Assert.assertEquals(Hex.encodeHexString(oldSha.digest()),
@ -587,21 +561,17 @@ public class TestContainerPersistence {
NoSuchAlgorithmException {
final int datalen = 1024;
long testContainerID = getTestContainerID();
BlockID blockID = ContainerTestHelper.
getTestBlockID(testContainerID);
Container container = addContainer(containerSet, testContainerID);
ContainerData cData = new ContainerData(testContainerID, conf);
cData.addMetadata("VOLUME", "shire");
cData.addMetadata("owner)", "bilbo");
containerManager.createContainer(cData);
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
byte[] data = getData(datalen);
setDataChecksum(info, data);
chunkManager.writeChunk(blockID, info, data, COMBINED);
chunkManager.deleteChunk(blockID, info);
chunkManager.writeChunk(container, blockID, info, data, COMBINED);
chunkManager.deleteChunk(container, blockID, info);
exception.expect(StorageContainerException.class);
exception.expectMessage("Unable to find the chunk file.");
chunkManager.readChunk(blockID, info);
chunkManager.readChunk(container, blockID, info);
}
/**
@ -613,16 +583,16 @@ public class TestContainerPersistence {
@Test
public void testPutKey() throws IOException, NoSuchAlgorithmException {
long testContainerID = getTestContainerID();
BlockID blockID = ContainerTestHelper.
getTestBlockID(testContainerID);
Pipeline pipeline = createSingleNodePipeline();
ChunkInfo info = writeChunkHelper(blockID, pipeline);
Container container = addContainer(containerSet, testContainerID);
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
ChunkInfo info = writeChunkHelper(blockID);
KeyData keyData = new KeyData(blockID);
List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
chunkList.add(info.getProtoBufMessage());
keyData.setChunks(chunkList);
keyManager.putKey(keyData);
KeyData readKeyData = keyManager.getKey(keyData);
keyManager.putKey(container, keyData);
KeyData readKeyData = keyManager.getKey(container, keyData.getBlockID());
ChunkInfo readChunk =
ChunkInfo.getFromProtoBuf(readKeyData.getChunks().get(0));
Assert.assertEquals(info.getChecksum(), readChunk.getChecksum());
@ -641,11 +611,10 @@ public class TestContainerPersistence {
final int datalen = 1024;
long totalSize = 0L;
long testContainerID = getTestContainerID();
BlockID blockID = ContainerTestHelper.
getTestBlockID(testContainerID);
Pipeline pipeline = createSingleNodePipeline();
Container container = addContainer(containerSet, testContainerID);
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
List<ChunkInfo> chunkList = new LinkedList<>();
ChunkInfo info = writeChunkHelper(blockID, pipeline);
ChunkInfo info = writeChunkHelper(blockID);
totalSize += datalen;
chunkList.add(info);
for (int x = 1; x < chunkCount; x++) {
@ -653,18 +622,18 @@ public class TestContainerPersistence {
info = getChunk(blockID.getLocalID(), x, x * datalen, datalen);
byte[] data = getData(datalen);
setDataChecksum(info, data);
chunkManager.writeChunk(blockID, info, data, COMBINED);
totalSize += datalen * (x + 1);
chunkManager.writeChunk(container, blockID, info, data, COMBINED);
totalSize += datalen;
chunkList.add(info);
}
long bytesUsed = containerManager.getBytesUsed(testContainerID);
long bytesUsed = container.getContainerData().getBytesUsed();
Assert.assertEquals(totalSize, bytesUsed);
long writeBytes = containerManager.getWriteBytes(testContainerID);
long writeBytes = container.getContainerData().getWriteBytes();
Assert.assertEquals(chunkCount * datalen, writeBytes);
long readCount = containerManager.getReadCount(testContainerID);
long readCount = container.getContainerData().getReadCount();
Assert.assertEquals(0, readCount);
long writeCount = containerManager.getWriteCount(testContainerID);
long writeCount = container.getContainerData().getWriteCount();
Assert.assertEquals(chunkCount, writeCount);
KeyData keyData = new KeyData(blockID);
@ -673,8 +642,8 @@ public class TestContainerPersistence {
chunkProtoList.add(i.getProtoBufMessage());
}
keyData.setChunks(chunkProtoList);
keyManager.putKey(keyData);
KeyData readKeyData = keyManager.getKey(keyData);
keyManager.putKey(container, keyData);
KeyData readKeyData = keyManager.getKey(container, keyData.getBlockID());
ChunkInfo lastChunk = chunkList.get(chunkList.size() - 1);
ChunkInfo readChunk =
ChunkInfo.getFromProtoBuf(readKeyData.getChunks().get(readKeyData
@ -691,18 +660,18 @@ public class TestContainerPersistence {
@Test
public void testDeleteKey() throws IOException, NoSuchAlgorithmException {
long testContainerID = getTestContainerID();
Container container = addContainer(containerSet, testContainerID);
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
Pipeline pipeline = createSingleNodePipeline();
ChunkInfo info = writeChunkHelper(blockID, pipeline);
ChunkInfo info = writeChunkHelper(blockID);
KeyData keyData = new KeyData(blockID);
List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
chunkList.add(info.getProtoBufMessage());
keyData.setChunks(chunkList);
keyManager.putKey(keyData);
keyManager.deleteKey(blockID);
keyManager.putKey(container, keyData);
keyManager.deleteKey(container, blockID);
exception.expect(StorageContainerException.class);
exception.expectMessage("Unable to find the key.");
keyManager.getKey(keyData);
keyManager.getKey(container, keyData.getBlockID());
}
/**
@ -715,19 +684,18 @@ public class TestContainerPersistence {
public void testDeleteKeyTwice() throws IOException,
NoSuchAlgorithmException {
long testContainerID = getTestContainerID();
BlockID blockID = ContainerTestHelper.
getTestBlockID(testContainerID);
Pipeline pipeline = createSingleNodePipeline();
ChunkInfo info = writeChunkHelper(blockID, pipeline);
Container container = addContainer(containerSet, testContainerID);
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
ChunkInfo info = writeChunkHelper(blockID);
KeyData keyData = new KeyData(blockID);
List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
chunkList.add(info.getProtoBufMessage());
keyData.setChunks(chunkList);
keyManager.putKey(keyData);
keyManager.deleteKey(blockID);
keyManager.putKey(container, keyData);
keyManager.deleteKey(container, blockID);
exception.expect(StorageContainerException.class);
exception.expectMessage("Unable to find the key.");
keyManager.deleteKey(blockID);
keyManager.deleteKey(container, blockID);
}
/**
@ -738,88 +706,86 @@ public class TestContainerPersistence {
*/
@Test
public void testUpdateContainer() throws IOException {
long testContainerID = ContainerTestHelper.
getTestContainerID();
ContainerData data = new ContainerData(testContainerID, conf);
data.addMetadata("VOLUME", "shire");
data.addMetadata("owner", "bilbo");
long testContainerID = ContainerTestHelper.getTestContainerID();
Container container = addContainer(containerSet, testContainerID);
containerManager.createContainer(data);
File orgContainerFile = containerManager.getContainerFile(data);
File orgContainerFile = KeyValueContainerLocationUtil.getContainerFile(
new File(container.getContainerData().getMetadataPath()),
String.valueOf(testContainerID));
Assert.assertTrue(orgContainerFile.exists());
ContainerData newData = new ContainerData(testContainerID, conf);
newData.addMetadata("VOLUME", "shire_new");
newData.addMetadata("owner", "bilbo_new");
Map<String, String> newMetadata = Maps.newHashMap();
newMetadata.put("VOLUME", "shire_new");
newMetadata.put("owner", "bilbo_new");
containerManager.updateContainer(testContainerID, newData, false);
container.update(newMetadata, false);
Assert.assertEquals(1, containerManager.getContainerMap().size());
Assert.assertTrue(containerManager.getContainerMap()
Assert.assertEquals(1, containerSet.getContainerMap().size());
Assert.assertTrue(containerSet.getContainerMap()
.containsKey(testContainerID));
// Verify in-memory map
ContainerData actualNewData = containerManager.getContainerMap()
.get(testContainerID);
ContainerData actualNewData =
containerSet.getContainer(testContainerID).getContainerData();
Assert.assertEquals("shire_new",
actualNewData.getAllMetadata().get("VOLUME"));
actualNewData.getMetadata().get("VOLUME"));
Assert.assertEquals("bilbo_new",
actualNewData.getAllMetadata().get("owner"));
actualNewData.getMetadata().get("owner"));
// Verify container data on disk
File newContainerFile = containerManager.getContainerFile(actualNewData);
File newContainerFile = KeyValueContainerLocationUtil.getContainerFile(
new File(actualNewData.getMetadataPath()),
String.valueOf(testContainerID));
Assert.assertTrue("Container file should exist.",
newContainerFile.exists());
Assert.assertEquals("Container file should be in same location.",
orgContainerFile.getAbsolutePath(),
newContainerFile.getAbsolutePath());
try (FileInputStream newIn = new FileInputStream(newContainerFile)) {
ContainerProtos.ContainerData actualContainerDataProto =
ContainerProtos.ContainerData.parseDelimitedFrom(newIn);
ContainerData actualContainerData = ContainerData
.getFromProtBuf(actualContainerDataProto, conf);
Assert.assertEquals("shire_new",
actualContainerData.getAllMetadata().get("VOLUME"));
Assert.assertEquals("bilbo_new",
actualContainerData.getAllMetadata().get("owner"));
}
ContainerData actualContainerData = ContainerDataYaml.readContainerFile(
newContainerFile);
Assert.assertEquals("shire_new",
actualContainerData.getMetadata().get("VOLUME"));
Assert.assertEquals("bilbo_new",
actualContainerData.getMetadata().get("owner"));
// Test force update flag.
// Delete container file then try to update without force update flag.
FileUtil.fullyDelete(newContainerFile);
// Close the container and then try to update without force update flag.
container.close();
try {
containerManager.updateContainer(testContainerID, newData, false);
container.update(newMetadata, false);
} catch (StorageContainerException ex) {
Assert.assertEquals("Container file not exists or "
+ "corrupted. ID: " + testContainerID, ex.getMessage());
Assert.assertEquals("Updating a closed container without force option " +
"is not allowed. ContainerID: " + testContainerID, ex.getMessage());
}
// Update with force flag, it should be success.
newData = new ContainerData(testContainerID, conf);
newData.addMetadata("VOLUME", "shire_new_1");
newData.addMetadata("owner", "bilbo_new_1");
containerManager.updateContainer(testContainerID, newData, true);
newMetadata.put("VOLUME", "shire_new_1");
newMetadata.put("owner", "bilbo_new_1");
container.update(newMetadata, true);
// Verify in-memory map
actualNewData = containerManager.getContainerMap()
.get(testContainerID);
actualNewData =
containerSet.getContainer(testContainerID).getContainerData();
Assert.assertEquals("shire_new_1",
actualNewData.getAllMetadata().get("VOLUME"));
actualNewData.getMetadata().get("VOLUME"));
Assert.assertEquals("bilbo_new_1",
actualNewData.getAllMetadata().get("owner"));
actualNewData.getMetadata().get("owner"));
// Update a non-existing container
exception.expect(StorageContainerException.class);
exception.expectMessage("Container doesn't exist.");
containerManager.updateContainer(RandomUtils.nextLong(),
newData, false);
exception.expectMessage("Container is an Inconsistent state, missing " +
"required files(.container, .chksm).");
Container nonExistentContainer = new KeyValueContainer(
new KeyValueContainerData(RandomUtils.nextLong(),
ContainerTestHelper.CONTAINER_MAX_SIZE_GB), conf);
nonExistentContainer.update(newMetadata, false);
}
private KeyData writeKeyHelper(Pipeline pipeline, BlockID blockID)
private KeyData writeKeyHelper(BlockID blockID)
throws IOException, NoSuchAlgorithmException {
ChunkInfo info = writeChunkHelper(blockID, pipeline);
ChunkInfo info = writeChunkHelper(blockID);
KeyData keyData = new KeyData(blockID);
List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
chunkList.add(info.getProtoBufMessage());
@ -829,20 +795,18 @@ public class TestContainerPersistence {
@Test
public void testListKey() throws Exception {
long testContainerID = getTestContainerID();
Pipeline pipeline = createSingleNodePipeline();
Container container = addContainer(containerSet, testContainerID);
List<BlockID> expectedKeys = new ArrayList<>();
for (int i = 0; i < 10; i++) {
BlockID blockID = new BlockID(
testContainerID, i);
BlockID blockID = new BlockID(testContainerID, i);
expectedKeys.add(blockID);
KeyData kd = writeKeyHelper(pipeline, blockID);
keyManager.putKey(kd);
KeyData kd = writeKeyHelper(blockID);
keyManager.putKey(container, kd);
}
// List all keys
List<KeyData> result = keyManager.listKey(testContainerID, 0, 100);
List<KeyData> result = keyManager.listKey(container, 0, 100);
Assert.assertEquals(10, result.size());
int index = 0;
@ -855,7 +819,7 @@ public class TestContainerPersistence {
// List key with startKey filter
long k6 = expectedKeys.get(6).getLocalID();
result = keyManager.listKey(testContainerID, k6, 100);
result = keyManager.listKey(container, k6, 100);
Assert.assertEquals(4, result.size());
for (int i = 6; i < 10; i++) {
@ -866,6 +830,6 @@ public class TestContainerPersistence {
// Count must be >0
exception.expect(IllegalArgumentException.class);
exception.expectMessage("Count must be a positive number.");
keyManager.listKey(testContainerID, 0, -1);
keyManager.listKey(container, 0, -1);
}
}

View File

@ -74,7 +74,8 @@ public class TestContainerMetrics {
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, path);
VolumeSet volumeSet = new VolumeSet(datanodeDetails, conf);
VolumeSet volumeSet = new VolumeSet(
datanodeDetails.getUuidString(), conf);
ContainerSet containerSet = new ContainerSet();
HddsDispatcher dispatcher = new HddsDispatcher(conf, containerSet,
volumeSet);

View File

@ -18,7 +18,10 @@
package org.apache.hadoop.ozone.container.server;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.ratis.shaded.io.netty.channel.embedded.EmbeddedChannel;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@ -32,9 +35,7 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.RatisTestHelper;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.container.common.impl.Dispatcher;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServer;
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerHandler;
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi;
@ -204,7 +205,6 @@ public class TestContainerServer {
public void testClientServerWithContainerDispatcher() throws Exception {
XceiverServer server = null;
XceiverClient client = null;
String containerName = OzoneUtils.getRequestID();
try {
Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline();
@ -213,8 +213,8 @@ public class TestContainerServer {
pipeline.getLeader()
.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
Dispatcher dispatcher =
new Dispatcher(mock(ContainerManager.class), conf);
HddsDispatcher dispatcher = new HddsDispatcher(
conf, mock(ContainerSet.class), mock(VolumeSet.class));
dispatcher.init();
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
server = new XceiverServer(datanodeDetails, conf, dispatcher);
@ -229,10 +229,6 @@ public class TestContainerServer {
ContainerCommandResponseProto response = client.sendCommand(request);
Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
Assert.assertTrue(dispatcher.
getContainerMetrics().
getContainerOpsMetrics(
ContainerProtos.Type.CreateContainer)== 1);
} finally {
if (client != null) {
client.close();

View File

@ -24,8 +24,6 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.cli.ResultCode;
import org.apache.hadoop.hdds.scm.cli.SCMCLI;
@ -35,9 +33,14 @@ import org.apache.hadoop.hdds.scm.client.ScmClient;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.Timeout;
@ -56,9 +59,11 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
/**
* This class tests the CLI of SCM.
*/
@Ignore ("Needs to be fixed for new SCM and Storage design")
public class TestSCMCli {
private static SCMCLI cli;
@ -161,11 +166,12 @@ public class TestSCMCli {
.createContainer(xceiverClientManager.getType(),
HddsProtos.ReplicationFactor.ONE, containerOwner);
ContainerData cdata = ContainerData
.getFromProtBuf(containerOperationClient.readContainer(
container.getContainerID(), container.getPipeline()), conf);
KeyUtils.getDB(cdata, conf).put(Longs.toByteArray(container.getContainerID()),
"someKey".getBytes());
KeyValueContainerData kvData = KeyValueContainerData
.getFromProtoBuf(containerOperationClient.readContainer(
container.getContainerID(), container.getPipeline()));
KeyUtils.getDB(kvData, conf)
.put(Longs.toByteArray(container.getContainerID()),
"someKey".getBytes());
Assert.assertTrue(containerExist(container.getContainerID()));
// Gracefully delete a container should fail because it is open.
@ -272,10 +278,10 @@ public class TestSCMCli {
ContainerInfo container = containerOperationClient
.createContainer(xceiverClientManager.getType(),
HddsProtos.ReplicationFactor.ONE, containerOwner);
ContainerData data = ContainerData
.getFromProtBuf(containerOperationClient.
KeyValueContainerData data = KeyValueContainerData
.getFromProtoBuf(containerOperationClient.
readContainer(container.getContainerID(),
container.getPipeline()), conf);
container.getPipeline()));
info = new String[] { "-container", "-info", "-c",
Long.toString(container.getContainerID()) };
@ -287,7 +293,7 @@ public class TestSCMCli {
String openStatus = data.isOpen() ? "OPEN" : "CLOSED";
String expected =
String.format(formatStr, container.getContainerID(), openStatus,
data.getDBPath(), data.getContainerPath(), "",
data.getDbFile().getPath(), data.getContainerPath(), "",
datanodeDetails.getHostName(), datanodeDetails.getHostName());
assertEquals(expected, out.toString());
@ -297,9 +303,9 @@ public class TestSCMCli {
container = containerOperationClient
.createContainer(xceiverClientManager.getType(),
HddsProtos.ReplicationFactor.ONE, containerOwner);
data = ContainerData
.getFromProtBuf(containerOperationClient.readContainer(
container.getContainerID(), container.getPipeline()), conf);
data = KeyValueContainerData
.getFromProtoBuf(containerOperationClient.readContainer(
container.getContainerID(), container.getPipeline()));
KeyUtils.getDB(data, conf)
.put(containerID.getBytes(), "someKey".getBytes());
@ -310,7 +316,7 @@ public class TestSCMCli {
openStatus = data.isOpen() ? "OPEN" : "CLOSED";
expected = String.format(formatStr, container.getContainerID(), openStatus,
data.getDBPath(), data.getContainerPath(), "",
data.getDbFile().getPath(), data.getContainerPath(), "",
datanodeDetails.getHostName(), datanodeDetails.getHostName());
assertEquals(expected, out.toString());
@ -325,14 +331,14 @@ public class TestSCMCli {
Long.toString(container.getContainerID()) };
exitCode = runCommandAndGetOutput(info, out, null);
assertEquals(ResultCode.SUCCESS, exitCode);
data = ContainerData
.getFromProtBuf(containerOperationClient.readContainer(
container.getContainerID(), container.getPipeline()), conf);
data = KeyValueContainerData
.getFromProtoBuf(containerOperationClient.readContainer(
container.getContainerID(), container.getPipeline()));
openStatus = data.isOpen() ? "OPEN" : "CLOSED";
expected = String
.format(formatStr, container.getContainerID(), openStatus,
data.getDBPath(), data.getContainerPath(), "",
data.getDbFile().getPath(), data.getContainerPath(), "",
datanodeDetails.getHostName(), datanodeDetails.getHostName());
assertEquals(expected, out.toString());
}

View File

@ -18,6 +18,9 @@
package org.apache.hadoop.ozone.genesis;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.ratis.shaded.com.google.protobuf.ByteString;
import org.apache.commons.codec.digest.DigestUtils;
import org.apache.commons.io.FileUtils;
@ -27,11 +30,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
import org.apache.hadoop.ozone.container.common.impl.Dispatcher;
import org.apache.hadoop.ozone.container.common.impl.KeyManagerImpl;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
import org.apache.hadoop.util.Time;
import org.openjdk.jmh.annotations.Benchmark;
@ -65,8 +63,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.PutKeyRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.GetKeyRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerData;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
@ -77,8 +73,8 @@ public class BenchMarkDatanodeDispatcher {
private String baseDir;
private String datanodeUuid;
private Dispatcher dispatcher;
private Pipeline pipeline;
private HddsDispatcher dispatcher;
private ByteString data;
private Random random;
private AtomicInteger containerCount;
@ -104,7 +100,6 @@ public class BenchMarkDatanodeDispatcher {
data = ByteString.copyFromUtf8(RandomStringUtils.randomAscii(1048576));
random = new Random();
Configuration conf = new OzoneConfiguration();
ContainerManager manager = new ContainerManagerImpl();
baseDir = System.getProperty("java.io.tmpdir") + File.separator +
datanodeUuid;
@ -113,15 +108,12 @@ public class BenchMarkDatanodeDispatcher {
// metadata directory
StorageLocation metadataDir = StorageLocation.parse(
baseDir+ File.separator + CONTAINER_ROOT_PREFIX);
List<StorageLocation> locations = Arrays.asList(metadataDir);
baseDir + File.separator + CONTAINER_ROOT_PREFIX);
manager
.init(conf, locations, GenesisUtil.createDatanodeDetails(datanodeUuid));
manager.setChunkManager(new ChunkManagerImpl(manager));
manager.setKeyManager(new KeyManagerImpl(manager, conf));
ContainerSet containerSet = new ContainerSet();
VolumeSet volumeSet = new VolumeSet(datanodeUuid, conf);
dispatcher = new Dispatcher(manager, conf);
dispatcher = new HddsDispatcher(conf, containerSet, volumeSet);
dispatcher.init();
containerCount = new AtomicInteger();