summaryrefslogtreecommitdiff
path: root/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common
diff options
context:
space:
mode:
authorAnu Engineer <aengineer@apache.org>2018-04-05 11:24:39 -0700
committerOwen O'Malley <omalley@apache.org>2018-04-26 05:36:04 -0700
commit8b832f3c3556ef3f970bac636ef9c70ee9dd260d (patch)
tree82ad3bdf945504bf753f3dca20cc41077c2b19c4 /hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common
parent792ac4d08bfdef2ea66630a72bac3a483fe2a547 (diff)
HDFS-13405. Ozone: Rename HDSL to HDDS.
Contributed by Ajay Kumar, Elek Marton, Mukul Kumar Singh, Shashikant Banerjee and Anu Engineer.
Diffstat (limited to 'hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common')
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java346
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java326
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java121
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java218
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java442
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DeletedContainerBlocksSummary.java103
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java81
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java148
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/package-info.java22
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java233
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java1113
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerReportManagerImpl.java90
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerStatus.java217
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerStorageLocation.java203
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java713
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java202
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java70
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java63
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java91
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/package-info.java22
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java76
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java46
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java51
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManager.java58
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManagerMXBean.java36
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java280
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerReportManager.java32
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java76
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/package-info.java20
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/package-info.java28
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java387
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java294
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachineMBean.java34
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java208
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManagerMXBean.java27
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java285
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java239
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/package-info.java18
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java112
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandDispatcher.java177
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandHandler.java59
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ContainerReportHandler.java114
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java211
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/package-info.java18
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/package-info.java28
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/DatanodeState.java55
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java157
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java175
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/package-info.java21
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java267
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java194
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java68
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/package-info.java20
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/package-info.java18
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServer.java130
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerHandler.java82
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerInitializer.java62
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java43
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/package-info.java24
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java293
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java214
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/package-info.java23
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java168
-rw-r--r--hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/package-info.java18
64 files changed, 9770 insertions, 0 deletions
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java
new file mode 100644
index 0000000000..68bf4421f6
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java
@@ -0,0 +1,346 @@
+/*
+ * 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 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.Pipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+ .StorageContainerException;
+import org.apache.hadoop.hdds.protocol.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.proto.ContainerProtos.Result
+ .CHECKSUM_MISMATCH;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .CONTAINER_INTERNAL_ERROR;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .CONTAINER_NOT_FOUND;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .INVALID_WRITE_SIZE;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .IO_EXCEPTION;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .OVERWRITE_FLAG_REQUIRED;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .UNABLE_TO_FIND_CHUNK;
+import static org.apache.hadoop.hdds.protocol.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 pipeline - pipeline.
+ * @param data - container data.
+ * @param info - chunk info.
+ * @return File
+ * @throws StorageContainerException
+ */
+ public static File validateChunk(Pipeline pipeline, ContainerData data,
+ ChunkInfo info) throws StorageContainerException {
+
+ Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
+
+ File chunkFile = getChunkFile(pipeline, 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 pipeline - Container Info.
+ * @param data - Container Data
+ * @param info - Chunk info
+ * @return - File.
+ * @throws StorageContainerException
+ */
+ public static File getChunkFile(Pipeline pipeline, ContainerData data,
+ ChunkInfo info) throws StorageContainerException {
+
+ Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
+ if (data == null) {
+ log.error("Invalid container Name: {}", pipeline.getContainerName());
+ throw new StorageContainerException("Unable to find the container Name:" +
+ " " +
+ pipeline.getContainerName(), 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.getContainerResponse(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.setPipeline(msg.getReadChunk().getPipeline());
+
+ ContainerProtos.ContainerCommandResponseProto.Builder builder =
+ ContainerUtils.getContainerResponse(msg, ContainerProtos.Result
+ .SUCCESS, "");
+ builder.setReadChunk(response);
+ return builder.build();
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java
new file mode 100644
index 0000000000..c29374c07c
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java
@@ -0,0 +1,326 @@
+/*
+ * 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.commons.codec.digest.DigestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.util.Time;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * 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 String containerName;
+ 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 String hash;
+ private AtomicLong bytesUsed;
+ private long maxSize;
+ private Long containerID;
+ private HddsProtos.LifeCycleState state;
+
+ /**
+ * Constructs a ContainerData Object.
+ *
+ * @param containerName - Name
+ */
+ public ContainerData(String containerName, Long containerID,
+ Configuration conf) {
+ this.metadata = new TreeMap<>();
+ this.containerName = containerName;
+ 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 = HddsProtos.LifeCycleState.OPEN;
+ }
+
+ /**
+ * 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.getName(),
+ 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.hasHash()) {
+ data.setHash(protoData.getHash());
+ }
+
+ if (protoData.hasBytesUsed()) {
+ data.setBytesUsed(protoData.getBytesUsed());
+ }
+
+ if (protoData.hasSize()) {
+ data.setMaxSize(protoData.getSize());
+ }
+ return data;
+ }
+
+ /**
+ * Returns a ProtoBuf Message from ContainerData.
+ *
+ * @return Protocol Buffer Message
+ */
+ public ContainerProtos.ContainerData getProtoBufMessage() {
+ ContainerProtos.ContainerData.Builder builder = ContainerProtos
+ .ContainerData.newBuilder();
+ builder.setName(this.getContainerName());
+ builder.setContainerID(this.getContainerID());
+
+ if (this.getDBPath() != null) {
+ builder.setDbPath(this.getDBPath());
+ }
+
+ if (this.getHash() != null) {
+ builder.setHash(this.getHash());
+ }
+
+ if (this.getContainerPath() != null) {
+ builder.setContainerPath(this.getContainerPath());
+ }
+
+ builder.setState(this.getState());
+
+ for (Map.Entry<String, String> entry : metadata.entrySet()) {
+ HddsProtos.KeyValue.Builder keyValBuilder =
+ HddsProtos.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());
+ }
+
+ return builder.build();
+ }
+
+ /**
+ * Returns the name of the container.
+ *
+ * @return - name
+ */
+ public String getContainerName() {
+ return containerName;
+ }
+
+ /**
+ * 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.
+ */
+ public String getName() {
+ return getContainerName();
+ }
+
+ /**
+ * 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(HddsProtos.LifeCycleState state) {
+ this.state = state;
+ }
+
+ public synchronized HddsProtos.LifeCycleState getState() {
+ return this.state;
+ }
+
+ /**
+ * checks if the container is open.
+ * @return - boolean
+ */
+ public synchronized boolean isOpen() {
+ return HddsProtos.LifeCycleState.OPEN == state;
+ }
+
+ /**
+ * Marks this container as closed.
+ */
+ public synchronized void closeContainer() {
+ // TODO: closed or closing here
+ setState(HddsProtos.LifeCycleState.CLOSED);
+
+ // Some thing brain dead for now. name + Time stamp of when we get the close
+ // container message.
+ setHash(DigestUtils.sha256Hex(this.getContainerName() +
+ Long.toString(Time.monotonicNow())));
+ }
+
+ /**
+ * Final hash for this container.
+ * @return - Hash
+ */
+ public String getHash() {
+ return hash;
+ }
+
+ public void setHash(String hash) {
+ this.hash = hash;
+ }
+
+ 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);
+ }
+
+ public long addBytesUsed(long delta) {
+ return this.bytesUsed.addAndGet(delta);
+ }
+
+ public long getBytesUsed() {
+ return bytesUsed.get();
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java
new file mode 100644
index 0000000000..d4d732b8b6
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java
@@ -0,0 +1,121 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableQuantiles;
+import org.apache.hadoop.metrics2.lib.MutableRate;
+
+/**
+ *
+ * This class is for maintaining the various Storage Container
+ * DataNode statistics and publishing them through the metrics interfaces.
+ * This also registers the JMX MBean for RPC.
+ * <p>
+ * This class has a number of metrics variables that are publicly accessible;
+ * these variables (objects) have methods to update their values;
+ * for example:
+ * <p> {@link #numOps}.inc()
+ *
+ */
+@InterfaceAudience.Private
+@Metrics(about="Storage Container DataNode Metrics", context="dfs")
+public class ContainerMetrics {
+ @Metric private MutableCounterLong numOps;
+ private MutableCounterLong[] numOpsArray;
+ private MutableCounterLong[] opsBytesArray;
+ private MutableRate[] opsLatency;
+ private MutableQuantiles[][] opsLatQuantiles;
+ private MetricsRegistry registry = null;
+
+ public ContainerMetrics(int[] intervals) {
+ int numEnumEntries = ContainerProtos.Type.values().length;
+ final int len = intervals.length;
+ this.numOpsArray = new MutableCounterLong[numEnumEntries];
+ this.opsBytesArray = new MutableCounterLong[numEnumEntries];
+ this.opsLatency = new MutableRate[numEnumEntries];
+ this.opsLatQuantiles = new MutableQuantiles[numEnumEntries][len];
+ this.registry = new MetricsRegistry("StorageContainerMetrics");
+ for (int i = 0; i < numEnumEntries; i++) {
+ numOpsArray[i] = registry.newCounter(
+ "num" + ContainerProtos.Type.valueOf(i + 1),
+ "number of " + ContainerProtos.Type.valueOf(i + 1) + " ops",
+ (long) 0);
+ opsBytesArray[i] = registry.newCounter(
+ "bytes" + ContainerProtos.Type.valueOf(i + 1),
+ "bytes used by " + ContainerProtos.Type.valueOf(i + 1) + "op",
+ (long) 0);
+ opsLatency[i] = registry.newRate(
+ "latency" + ContainerProtos.Type.valueOf(i + 1),
+ ContainerProtos.Type.valueOf(i + 1) + " op");
+
+ for (int j = 0; j < len; j++) {
+ int interval = intervals[j];
+ String quantileName = ContainerProtos.Type.valueOf(i + 1) + "Nanos"
+ + interval + "s";
+ opsLatQuantiles[i][j] = registry.newQuantiles(quantileName,
+ "latency of Container ops", "ops", "latency", interval);
+ }
+ }
+ }
+
+ public static ContainerMetrics create(Configuration conf) {
+ MetricsSystem ms = DefaultMetricsSystem.instance();
+ // Percentile measurement is off by default, by watching no intervals
+ int[] intervals =
+ conf.getInts(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY);
+ return ms.register("StorageContainerMetrics",
+ "Storage Container Node Metrics",
+ new ContainerMetrics(intervals));
+ }
+
+ public void incContainerOpcMetrics(ContainerProtos.Type type){
+ numOps.incr();
+ numOpsArray[type.ordinal()].incr();
+ }
+
+ public long getContainerOpsMetrics(ContainerProtos.Type type){
+ return numOpsArray[type.ordinal()].value();
+ }
+
+ public void incContainerOpsLatencies(ContainerProtos.Type type,
+ long latencyNanos) {
+ opsLatency[type.ordinal()].add(latencyNanos);
+ for (MutableQuantiles q: opsLatQuantiles[type.ordinal()]) {
+ q.add(latencyNanos);
+ }
+ }
+
+ public void incContainerBytesStats(ContainerProtos.Type type, long bytes) {
+ opsBytesArray[type.ordinal()].incr(bytes);
+ }
+
+ public long getContainerBytesMetrics(ContainerProtos.Type type){
+ return opsBytesArray[type.ordinal()].value();
+ }
+} \ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
new file mode 100644
index 0000000000..50d2da3975
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
@@ -0,0 +1,218 @@
+/*
+ * 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.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerInfo;
+
+/**
+ * Container Report iterates the closed containers and sends a container report
+ * to SCM.
+ */
+public class ContainerReport {
+ private static final int UNKNOWN = -1;
+ private final String containerName;
+ private final String finalhash;
+ private long size;
+ private long keyCount;
+ private long bytesUsed;
+ private long readCount;
+ private long writeCount;
+ private long readBytes;
+ private long writeBytes;
+ private long containerID;
+
+ public long getContainerID() {
+ return containerID;
+ }
+
+ public void setContainerID(long containerID) {
+ this.containerID = containerID;
+ }
+
+
+
+
+ /**
+ * Constructs the ContainerReport.
+ *
+ * @param containerName - Container Name.
+ * @param finalhash - Final Hash.
+ */
+ public ContainerReport(String containerName, String finalhash) {
+ this.containerName = containerName;
+ this.finalhash = finalhash;
+ this.size = UNKNOWN;
+ this.keyCount = UNKNOWN;
+ this.bytesUsed = 0L;
+ this.readCount = 0L;
+ this.readBytes = 0L;
+ this.writeCount = 0L;
+ this.writeBytes = 0L;
+ }
+
+ /**
+ * Gets a containerReport from protobuf class.
+ *
+ * @param info - ContainerInfo.
+ * @return - ContainerReport.
+ */
+ public static ContainerReport getFromProtoBuf(ContainerInfo info) {
+ Preconditions.checkNotNull(info);
+ ContainerReport report = new ContainerReport(info.getContainerName(),
+ info.getFinalhash());
+ if (info.hasSize()) {
+ report.setSize(info.getSize());
+ }
+ if (info.hasKeyCount()) {
+ report.setKeyCount(info.getKeyCount());
+ }
+ if (info.hasUsed()) {
+ report.setBytesUsed(info.getUsed());
+ }
+ if (info.hasReadCount()) {
+ report.setReadCount(info.getReadCount());
+ }
+ if (info.hasReadBytes()) {
+ report.setReadBytes(info.getReadBytes());
+ }
+ if (info.hasWriteCount()) {
+ report.setWriteCount(info.getWriteCount());
+ }
+ if (info.hasWriteBytes()) {
+ report.setWriteBytes(info.getWriteBytes());
+ }
+
+ report.setContainerID(info.getContainerID());
+ return report;
+ }
+
+ /**
+ * Gets the container name.
+ *
+ * @return - Name
+ */
+ public String getContainerName() {
+ return containerName;
+ }
+
+ /**
+ * Returns the final signature for this container.
+ *
+ * @return - hash
+ */
+ public String getFinalhash() {
+ return finalhash;
+ }
+
+ /**
+ * Returns a positive number it is a valid number, -1 if not known.
+ *
+ * @return size or -1
+ */
+ public long getSize() {
+ return size;
+ }
+
+ /**
+ * Sets the size of the container on disk.
+ *
+ * @param size - int
+ */
+ public void setSize(long size) {
+ this.size = size;
+ }
+
+ /**
+ * Gets number of keys in the container if known.
+ *
+ * @return - Number of keys or -1 for not known.
+ */
+ public long getKeyCount() {
+ return keyCount;
+ }
+
+ /**
+ * Sets the key count.
+ *
+ * @param keyCount - Key Count
+ */
+ public void setKeyCount(long keyCount) {
+ this.keyCount = keyCount;
+ }
+
+ public long getReadCount() {
+ return readCount;
+ }
+
+ public void setReadCount(long readCount) {
+ this.readCount = readCount;
+ }
+
+ public long getWriteCount() {
+ return writeCount;
+ }
+
+ public void setWriteCount(long writeCount) {
+ this.writeCount = writeCount;
+ }
+
+ public long getReadBytes() {
+ return readBytes;
+ }
+
+ public void setReadBytes(long readBytes) {
+ this.readBytes = readBytes;
+ }
+
+ public long getWriteBytes() {
+ return writeBytes;
+ }
+
+ public void setWriteBytes(long writeBytes) {
+ this.writeBytes = writeBytes;
+ }
+
+ public long getBytesUsed() {
+ return bytesUsed;
+ }
+
+ public void setBytesUsed(long bytesUsed) {
+ this.bytesUsed = bytesUsed;
+ }
+
+ /**
+ * Gets a containerInfo protobuf message from ContainerReports.
+ *
+ * @return ContainerInfo
+ */
+ public ContainerInfo getProtoBufMessage() {
+ return ContainerInfo.newBuilder()
+ .setContainerName(this.getContainerName())
+ .setKeyCount(this.getKeyCount())
+ .setSize(this.getSize())
+ .setUsed(this.getBytesUsed())
+ .setReadCount(this.getReadCount())
+ .setReadBytes(this.getReadBytes())
+ .setWriteCount(this.getWriteCount())
+ .setWriteBytes(this.getWriteBytes())
+ .setFinalhash(this.getFinalhash())
+ .setContainerID(this.getContainerID())
+ .build();
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
new file mode 100644
index 0000000000..1818188cb6
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
@@ -0,0 +1,442 @@
+/*
+ * 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.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+ .StorageContainerException;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
+import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.utils.MetadataStoreBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+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.proto.ContainerProtos.Result
+ .INVALID_ARGUMENT;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .UNABLE_TO_FIND_DATA_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_META;
+
+/**
+ * A set of helper functions to create proper responses.
+ */
+public final class ContainerUtils {
+
+ private ContainerUtils() {
+ //never constructed.
+ }
+
+ /**
+ * 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
+ getContainerResponse(ContainerProtos.ContainerCommandRequestProto msg) {
+ ContainerProtos.ContainerCommandResponseProto.Builder builder =
+ getContainerResponse(msg, ContainerProtos.Result.SUCCESS, "");
+ return builder.build();
+ }
+
+ /**
+ * Returns a ReadContainer Response.
+ *
+ * @param msg Request
+ * @param containerData - data
+ * @return Response.
+ */
+ public static ContainerProtos.ContainerCommandResponseProto
+ getReadContainerResponse(ContainerProtos.ContainerCommandRequestProto msg,
+ ContainerData containerData) {
+ Preconditions.checkNotNull(containerData);
+
+ ContainerProtos.ReadContainerResponseProto.Builder response =
+ ContainerProtos.ReadContainerResponseProto.newBuilder();
+ response.setContainerData(containerData.getProtoBufMessage());
+
+ ContainerProtos.ContainerCommandResponseProto.Builder builder =
+ getContainerResponse(msg, ContainerProtos.Result.SUCCESS, "");
+ builder.setReadContainer(response);
+ return builder.build();
+ }
+
+ /**
+ * We found a command type but no associated payload for the command. Hence
+ * return malformed Command as response.
+ *
+ * @param msg - Protobuf message.
+ * @param result - result
+ * @param message - Error message.
+ * @return ContainerCommandResponseProto - MALFORMED_REQUEST.
+ */
+ public static ContainerProtos.ContainerCommandResponseProto.Builder
+ getContainerResponse(ContainerProtos.ContainerCommandRequestProto msg,
+ ContainerProtos.Result result, String message) {
+ return
+ ContainerProtos.ContainerCommandResponseProto.newBuilder()
+ .setCmdType(msg.getCmdType())
+ .setTraceID(msg.getTraceID())
+ .setResult(result)
+ .setMessage(message);
+ }
+
+ /**
+ * Logs the error and returns a response to the caller.
+ *
+ * @param log - Logger
+ * @param ex - Exception
+ * @param msg - Request Object
+ * @return Response
+ */
+ public static ContainerProtos.ContainerCommandResponseProto logAndReturnError(
+ Logger log, StorageContainerException ex,
+ ContainerProtos.ContainerCommandRequestProto msg) {
+ log.info("Operation: {} : Trace ID: {} : Message: {} : Result: {}",
+ msg.getCmdType().name(), msg.getTraceID(),
+ ex.getMessage(), ex.getResult().getValueDescriptor().getName());
+ return getContainerResponse(msg, ex.getResult(), ex.getMessage()).build();
+ }
+
+ /**
+ * Logs the error and returns a response to the caller.
+ *
+ * @param log - Logger
+ * @param ex - Exception
+ * @param msg - Request Object
+ * @return Response
+ */
+ public static ContainerProtos.ContainerCommandResponseProto logAndReturnError(
+ Logger log, RuntimeException ex,
+ ContainerProtos.ContainerCommandRequestProto msg) {
+ log.info("Operation: {} : Trace ID: {} : Message: {} ",
+ msg.getCmdType().name(), msg.getTraceID(), ex.getMessage());
+ return getContainerResponse(msg, INVALID_ARGUMENT, ex.getMessage()).build();
+ }
+
+ /**
+ * We found a command type but no associated payload for the command. Hence
+ * return malformed Command as response.
+ *
+ * @param msg - Protobuf message.
+ * @return ContainerCommandResponseProto - MALFORMED_REQUEST.
+ */
+ public static ContainerProtos.ContainerCommandResponseProto
+ malformedRequest(ContainerProtos.ContainerCommandRequestProto msg) {
+ return getContainerResponse(msg, ContainerProtos.Result.MALFORMED_REQUEST,
+ "Cmd type does not match the payload.").build();
+ }
+
+ /**
+ * We found a command type that is not supported yet.
+ *
+ * @param msg - Protobuf message.
+ * @return ContainerCommandResponseProto - MALFORMED_REQUEST.
+ */
+ public static ContainerProtos.ContainerCommandResponseProto
+ unsupportedRequest(ContainerProtos.ContainerCommandRequestProto msg) {
+ return getContainerResponse(msg, ContainerProtos.Result.UNSUPPORTED_REQUEST,
+ "Server does not support this command yet.").build();
+ }
+
+ /**
+ * get containerName from a container file.
+ *
+ * @param containerFile - File
+ * @return Name of the container.
+ */
+ public static String getContainerNameFromFile(File containerFile) {
+ Preconditions.checkNotNull(containerFile);
+ return Paths.get(containerFile.getParent()).resolve(
+ removeExtension(containerFile.getName())).toString();
+ }
+
+ /**
+ * Verifies that this in indeed a new container.
+ *
+ * @param containerFile - Container File to verify
+ * @param metadataFile - metadata File to verify
+ * @throws IOException
+ */
+ public static void verifyIsNewContainer(File containerFile, File metadataFile)
+ throws IOException {
+ Logger log = LoggerFactory.getLogger(ContainerManagerImpl.class);
+ if (containerFile.exists()) {
+ log.error("container already exists on disk. File: {}",
+ containerFile.toPath());
+ throw new FileAlreadyExistsException("container already exists on " +
+ "disk.");
+ }
+
+ if (metadataFile.exists()) {
+ log.error("metadata found on disk, but missing container. Refusing to" +
+ " write this container. File: {} ", metadataFile.toPath());
+ throw new FileAlreadyExistsException(("metadata found on disk, but " +
+ "missing container. Refusing to write this container."));
+ }
+
+ 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.");
+ }
+
+ if (!metadataFile.createNewFile()) {
+ log.error("creation of the metadata file failed. File: {}",
+ metadataFile.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 Metadata location.
+ *
+ * @param containerData - Data
+ * @param location - Path
+ * @return Path
+ */
+ public static File getMetadataFile(ContainerData containerData,
+ Path location) {
+ return location.resolve(containerData
+ .getContainerName().concat(CONTAINER_META))
+ .toFile();
+ }
+
+ /**
+ * Returns container file location.
+ *
+ * @param containerData - Data
+ * @param location - Root path
+ * @return Path
+ */
+ public static File getContainerFile(ContainerData containerData,
+ Path location) {
+ return location.resolve(containerData
+ .getContainerName().concat(CONTAINER_EXTENSION))
+ .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.",
+ ContainerProtos.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));
+ Path metaPath = Paths.get(rootPath.concat(CONTAINER_META));
+
+ FileUtils.forceDelete(containerPath.toFile());
+ FileUtils.forceDelete(metaPath.toFile());
+ }
+
+ /**
+ * Persistent a {@link DatanodeDetails} to a local file.
+ *
+ * @throws IOException when read/write error occurs
+ */
+ public synchronized static void writeDatanodeDetailsTo(
+ DatanodeDetails datanodeDetails, File path) throws IOException {
+ if (path.exists()) {
+ if (!path.delete() || !path.createNewFile()) {
+ throw new IOException("Unable to overwrite the datanode ID file.");
+ }
+ } else {
+ if(!path.getParentFile().exists() &&
+ !path.getParentFile().mkdirs()) {
+ throw new IOException("Unable to create datanode ID directories.");
+ }
+ }
+ try (FileOutputStream out = new FileOutputStream(path)) {
+ HddsProtos.DatanodeDetailsProto proto =
+ datanodeDetails.getProtoBufMessage();
+ proto.writeTo(out);
+ }
+ }
+
+ /**
+ * Read {@link DatanodeDetails} from a local ID file.
+ *
+ * @param path ID file local path
+ * @return {@link DatanodeDetails}
+ * @throws IOException If the id file is malformed or other I/O exceptions
+ */
+ public synchronized static DatanodeDetails readDatanodeDetailsFrom(File path)
+ throws IOException {
+ if (!path.exists()) {
+ throw new IOException("Datanode ID file not found.");
+ }
+ try(FileInputStream in = new FileInputStream(path)) {
+ return DatanodeDetails.getFromProtoBuf(
+ HddsProtos.DatanodeDetailsProto.parseFrom(in));
+ } catch (IOException e) {
+ throw new IOException("Failed to parse DatanodeDetails from "
+ + path.getAbsolutePath(), e);
+ }
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DeletedContainerBlocksSummary.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DeletedContainerBlocksSummary.java
new file mode 100644
index 0000000000..ade162a263
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DeletedContainerBlocksSummary.java
@@ -0,0 +1,103 @@
+/**
+ * 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.collect.Maps;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * A helper class to wrap the info about under deletion container blocks.
+ */
+public final class DeletedContainerBlocksSummary {
+
+ private final List<DeletedBlocksTransaction> blocks;
+ // key : txID
+ // value : times of this tx has been processed
+ private final Map<Long, Integer> txSummary;
+ // key : container name
+ // value : the number of blocks need to be deleted in this container
+ // if the message contains multiple entries for same block,
+ // blocks will be merged
+ private final Map<String, Integer> blockSummary;
+ // total number of blocks in this message
+ private int numOfBlocks;
+
+ private DeletedContainerBlocksSummary(List<DeletedBlocksTransaction> blocks) {
+ this.blocks = blocks;
+ txSummary = Maps.newHashMap();
+ blockSummary = Maps.newHashMap();
+ blocks.forEach(entry -> {
+ txSummary.put(entry.getTxID(), entry.getCount());
+ if (blockSummary.containsKey(entry.getContainerName())) {
+ blockSummary.put(entry.getContainerName(),
+ blockSummary.get(entry.getContainerName())
+ + entry.getBlockIDCount());
+ } else {
+ blockSummary.put(entry.getContainerName(), entry.getBlockIDCount());
+ }
+ numOfBlocks += entry.getBlockIDCount();
+ });
+ }
+
+ public static DeletedContainerBlocksSummary getFrom(
+ List<DeletedBlocksTransaction> blocks) {
+ return new DeletedContainerBlocksSummary(blocks);
+ }
+
+ public int getNumOfBlocks() {
+ return numOfBlocks;
+ }
+
+ public int getNumOfContainers() {
+ return blockSummary.size();
+ }
+
+ public String getTXIDs() {
+ return String.join(",", txSummary.keySet()
+ .stream().map(String::valueOf).collect(Collectors.toList()));
+ }
+
+ public String getTxIDSummary() {
+ List<String> txSummaryEntry = txSummary.entrySet().stream()
+ .map(entry -> entry.getKey() + "(" + entry.getValue() + ")")
+ .collect(Collectors.toList());
+ return "[" + String.join(",", txSummaryEntry) + "]";
+ }
+
+ @Override public String toString() {
+ StringBuffer sb = new StringBuffer();
+ for (DeletedBlocksTransaction blks : blocks) {
+ sb.append(" ")
+ .append("TXID=")
+ .append(blks.getTxID())
+ .append(", ")
+ .append("TimesProceed=")
+ .append(blks.getCount())
+ .append(", ")
+ .append(blks.getContainerName())
+ .append(" : [")
+ .append(String.join(",", blks.getBlockIDList())).append("]")
+ .append("\n");
+ }
+ return sb.toString();
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java
new file mode 100644
index 0000000000..566db02510
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java
@@ -0,0 +1,81 @@
+/*
+ * 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 com.google.protobuf.ByteString;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
+
+/**
+ * File Utils are helper routines used by putSmallFile and getSmallFile
+ * RPCs.
+ */
+public final class FileUtils {
+ /**
+ * Never Constructed.
+ */
+ private FileUtils() {
+ }
+
+ /**
+ * Gets a response for the putSmallFile RPC.
+ * @param msg - ContainerCommandRequestProto
+ * @return - ContainerCommandResponseProto
+ */
+ public static ContainerProtos.ContainerCommandResponseProto
+ getPutFileResponse(ContainerProtos.ContainerCommandRequestProto msg) {
+ ContainerProtos.PutSmallFileResponseProto.Builder getResponse =
+ ContainerProtos.PutSmallFileResponseProto.newBuilder();
+ ContainerProtos.ContainerCommandResponseProto.Builder builder =
+ ContainerUtils.getContainerResponse(msg, ContainerProtos.Result
+ .SUCCESS, "");
+ builder.setCmdType(ContainerProtos.Type.PutSmallFile);
+ builder.setPutSmallFile(getResponse);
+ return builder.build();
+ }
+
+ /**
+ * Gets a response to the read small file call.
+ * @param msg - Msg
+ * @param data - Data
+ * @param info - Info
+ * @return Response.
+ */
+ public static ContainerProtos.ContainerCommandResponseProto
+ getGetSmallFileResponse(ContainerProtos.ContainerCommandRequestProto msg,
+ byte[] data, ChunkInfo info) {
+ Preconditions.checkNotNull(msg);
+
+ ContainerProtos.ReadChunkResponseProto.Builder readChunkresponse =
+ ContainerProtos.ReadChunkResponseProto.newBuilder();
+ readChunkresponse.setChunkData(info.getProtoBufMessage());
+ readChunkresponse.setData(ByteString.copyFrom(data));
+ readChunkresponse.setPipeline(msg.getGetSmallFile().getKey().getPipeline());
+
+ ContainerProtos.GetSmallFileResponseProto.Builder getSmallFile =
+ ContainerProtos.GetSmallFileResponseProto.newBuilder();
+ getSmallFile.setData(readChunkresponse.build());
+ ContainerProtos.ContainerCommandResponseProto.Builder builder =
+ ContainerUtils.getContainerResponse(msg, ContainerProtos.Result
+ .SUCCESS, "");
+ builder.setCmdType(ContainerProtos.Type.GetSmallFile);
+ builder.setGetSmallFile(getSmallFile);
+ return builder.build();
+ }
+
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java
new file mode 100644
index 0000000000..33eb911d4e
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java
@@ -0,0 +1,148 @@
+/*
+ * 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.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.proto.ContainerProtos.Result
+ .NO_SUCH_KEY;
+import static org.apache.hadoop.hdds.protocol.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.getContainerName(), container.getDBPath());
+ } catch (IOException ex) {
+ String message =
+ String.format("Unable to open DB. DB Name: %s, Path: %s. ex: %s",
+ container.getContainerName(), 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.getContainerName());
+ }
+ /**
+ * 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.getContainerResponse(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.getContainerResponse(msg, ContainerProtos.Result
+ .SUCCESS, "");
+ 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);
+ }
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/package-info.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/package-info.java
new file mode 100644
index 0000000000..21f31e1bbe
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/package-info.java
@@ -0,0 +1,22 @@
+/**
+ * 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;
+/**
+ Contains protocol buffer helper classes and utilites used in
+ impl.
+ **/ \ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java
new file mode 100644
index 0000000000..b0286b96d4
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java
@@ -0,0 +1,233 @@
+/*
+ * 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.Pipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+ .StorageContainerException;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
+import org.apache.hadoop.ozone.OzoneConsts;
+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.proto.ContainerProtos.Result
+ .CONTAINER_INTERNAL_ERROR;
+import static org.apache.hadoop.hdds.protocol.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 pipeline - Name and the set of machines that make this container.
+ * @param keyName - Name of the Key.
+ * @param info - ChunkInfo.
+ * @throws StorageContainerException
+ */
+ @Override
+ public void writeChunk(Pipeline pipeline, String keyName, 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(pipeline, "Pipeline cannot be null");
+ String containerName = pipeline.getContainerName();
+ Preconditions.checkNotNull(containerName,
+ "Container name cannot be null");
+ ContainerData container =
+ containerManager.readContainer(containerName);
+ File chunkFile = ChunkUtils.validateChunk(pipeline, 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, containerName, 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(containerName, newSize - oldSize);
+ containerManager.incrWriteCount(containerName);
+ containerManager.incrWriteBytes(containerName, info.getLen());
+ break;
+ }
+ } 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,
+ String containerName, 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(containerName, sizeDiff);
+ containerManager.incrWriteCount(containerName);
+ containerManager.incrWriteBytes(containerName, chunkLen);
+ }
+
+ /**
+ * reads the data defined by a chunk.
+ *
+ * @param pipeline - container pipeline.
+ * @param keyName - Name of the Key
+ * @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(Pipeline pipeline, String keyName, ChunkInfo info)
+ throws StorageContainerException {
+ containerManager.readLock();
+ try {
+ Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
+ String containerName = pipeline.getContainerName();
+ Preconditions.checkNotNull(containerName,
+ "Container name cannot be null");
+ ContainerData container =
+ containerManager.readContainer(containerName);
+ File chunkFile = ChunkUtils.getChunkFile(pipeline, container, info);
+ ByteBuffer data = ChunkUtils.readData(chunkFile, info);
+ containerManager.incrReadCount(containerName);
+ containerManager.incrReadBytes(containerName, 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 pipeline - Pipeline.
+ * @param keyName - Key Name
+ * @param info - Chunk Info
+ * @throws StorageContainerException
+ */
+ @Override
+ public void deleteChunk(Pipeline pipeline, String keyName, ChunkInfo info)
+ throws StorageContainerException {
+ containerManager.readLock();
+ try {
+ Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
+ String containerName = pipeline.getContainerName();
+ Preconditions.checkNotNull(containerName,
+ "Container name cannot be null");
+ File chunkFile = ChunkUtils.getChunkFile(pipeline, containerManager
+ .readContainer(containerName), info);
+ if ((info.getOffset() == 0) && (info.getLen() == chunkFile.length())) {
+ FileUtil.fullyDelete(chunkFile);
+ containerManager.decrBytesUsed(containerName, 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());
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java
new file mode 100644
index 0000000000..5e7375cd9d
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java
@@ -0,0 +1,1113 @@
+/*
+ * 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.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+ .StorageContainerException;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.ReportState;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.SCMStorageReport;
+import org.apache.hadoop.io.IOUtils;
+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.KeyData;
+import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
+import org.apache.hadoop.ozone.container.common.interfaces.ChunkManager;
+import org.apache.hadoop.ozone.container.common.interfaces
+ .ContainerDeletionChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.interfaces
+ .ContainerLocationManager;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
+import org.apache.hadoop.ozone.container.common.interfaces
+ .ContainerReportManager;
+import org.apache.hadoop.ozone.container.common.interfaces.KeyManager;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.utils.MetadataKeyFilters;
+import org.apache.hadoop.utils.MetadataStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.security.DigestInputStream;
+import java.security.DigestOutputStream;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .CONTAINER_EXISTS;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .CONTAINER_INTERNAL_ERROR;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .CONTAINER_NOT_FOUND;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .ERROR_IN_COMPACT_DB;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .INVALID_CONFIG;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .IO_EXCEPTION;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .NO_SUCH_ALGORITHM;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .UNABLE_TO_READ_METADATA_DB;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .UNCLOSED_CONTAINER_IO;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .UNSUPPORTED_REQUEST;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_META;
+
+/**
+ * A Generic ContainerManagerImpl that will be called from Ozone
+ * ContainerManagerImpl. This allows us to support delta changes to ozone
+ * version without having to rewrite the containerManager.
+ */
+public class ContainerManagerImpl implements ContainerManager {
+ static final Logger LOG =
+ LoggerFactory.getLogger(ContainerManagerImpl.class);
+
+ private final ConcurrentSkipListMap<String, ContainerStatus>
+ containerMap = new ConcurrentSkipListMap<>();
+
+ // Use a non-fair RW lock for better throughput, we may revisit this decision
+ // if this causes fairness issues.
+ private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+ private ContainerLocationManager locationManager;
+ private ChunkManager chunkManager;
+ private KeyManager keyManager;
+ private Configuration conf;
+ private DatanodeDetails datanodeDetails;
+
+ private ContainerDeletionChoosingPolicy containerDeletionChooser;
+ private ContainerReportManager containerReportManager;
+
+ /**
+ * Init call that sets up a container Manager.
+ *
+ * @param config - Configuration.
+ * @param containerDirs - List of Metadata Container locations.
+ * @param dnDetails - DatanodeDetails.
+ * @throws IOException
+ */
+ @Override
+ public void init(
+ Configuration config, List<StorageLocation> containerDirs,
+ DatanodeDetails dnDetails) throws IOException {
+ Preconditions.checkNotNull(config, "Config must not be null");
+ Preconditions.checkNotNull(containerDirs, "Container directories cannot " +
+ "be null");
+ Preconditions.checkNotNull(dnDetails, "Datanode Details cannot " +
+ "be null");
+
+ Preconditions.checkState(containerDirs.size() > 0, "Number of container" +
+ " directories must be greater than zero.");
+
+ this.conf = config;
+ this.datanodeDetails = dnDetails;
+
+ readLock();
+ try {
+ containerDeletionChooser = ReflectionUtils.newInstance(conf.getClass(
+ ScmConfigKeys.OZONE_SCM_CONTAINER_DELETION_CHOOSING_POLICY,
+ TopNOrderedContainerDeletionChoosingPolicy.class,
+ ContainerDeletionChoosingPolicy.class), conf);
+
+ for (StorageLocation path : containerDirs) {
+ File directory = Paths.get(path.getNormalizedUri()).toFile();
+ if (!directory.exists() && !directory.mkdirs()) {
+ LOG.error("Container metadata directory doesn't exist "
+ + "and cannot be created. Path: {}", path.toString());
+ throw new StorageContainerException("Container metadata "
+ + "directory doesn't exist and cannot be created " + path
+ .toString(), INVALID_CONFIG);
+ }
+
+ // TODO: This will fail if any directory is invalid.
+ // We should fix this to handle invalid directories and continue.
+ // Leaving it this way to fail fast for time being.
+ if (!directory.isDirectory()) {
+ LOG.error("Invalid path to container metadata directory. path: {}",
+ path.toString());
+ throw new StorageContainerException("Invalid path to container " +
+ "metadata directory." + path, INVALID_CONFIG);
+ }
+ LOG.info("Loading containers under {}", path);
+ File[] files = directory.listFiles(new ContainerFilter());
+ if (files != null) {
+ for (File containerFile : files) {
+ LOG.debug("Loading container {}", containerFile);
+ String containerPath =
+ ContainerUtils.getContainerNameFromFile(containerFile);
+ Preconditions.checkNotNull(containerPath, "Container path cannot" +
+ " be null");
+ readContainerInfo(containerPath);
+ }
+ }
+ }
+
+ List<StorageLocation> dataDirs = new LinkedList<>();
+ for (String dir : config.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
+ StorageLocation location = StorageLocation.parse(dir);
+ dataDirs.add(location);
+ }
+ this.locationManager =
+ new ContainerLocationManagerImpl(containerDirs, dataDirs, config);
+
+ this.containerReportManager =
+ new ContainerReportManagerImpl(config);
+ } finally {
+ readUnlock();
+ }
+ }
+
+ /**
+ * Reads the Container Info from a file and verifies that checksum match. If
+ * the checksums match, then that file is added to containerMap.
+ *
+ * @param containerName - Name which points to the persisted container.
+ * @throws StorageContainerException
+ */
+ private void readContainerInfo(String containerName)
+ throws StorageContainerException {
+ Preconditions.checkState(containerName.length() > 0,
+ "Container name length cannot be zero.");
+ FileInputStream containerStream = null;
+ DigestInputStream dis = null;
+ FileInputStream metaStream = null;
+ Path cPath = Paths.get(containerName).getFileName();
+ String keyName = null;
+ if (cPath != null) {
+ keyName = cPath.toString();
+ }
+ Preconditions.checkNotNull(keyName,
+ "Container Name to container key mapping is null");
+
+ try {
+ String containerFileName = containerName.concat(CONTAINER_EXTENSION);
+ String metaFileName = containerName.concat(CONTAINER_META);
+
+ containerStream = new FileInputStream(containerFileName);
+
+ metaStream = new FileInputStream(metaFileName);
+
+ MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
+
+ dis = new DigestInputStream(containerStream, sha);
+
+ ContainerProtos.ContainerData containerDataProto =
+ ContainerProtos.ContainerData.parseDelimitedFrom(dis);
+ ContainerData containerData;
+ if (containerDataProto == null) {
+ // Sometimes container metadata might have been created but empty,
+ // when loading the info we get a null, this often means last time
+ // SCM was ending up at some middle phase causing that the metadata
+ // was not populated. Such containers are marked as inactive.
+ containerMap.put(keyName, new ContainerStatus(null));
+ return;
+ }
+ containerData = ContainerData.getFromProtBuf(containerDataProto, conf);
+ ContainerProtos.ContainerMeta meta =
+ ContainerProtos.ContainerMeta.parseDelimitedFrom(metaStream);
+ if (meta != null && !DigestUtils.sha256Hex(sha.digest())
+ .equals(meta.getHash())) {
+ // This means we were not able read data from the disk when booted the
+ // datanode. We are going to rely on SCM understanding that we don't
+ // have valid data for this container when we send container reports.
+ // Hopefully SCM will ask us to delete this container and rebuild it.
+ LOG.error("Invalid SHA found for container data. Name :{}"
+ + "cowardly refusing to read invalid data", containerName);
+ containerMap.put(keyName, new ContainerStatus(null));
+ return;
+ }
+
+ ContainerStatus containerStatus = new ContainerStatus(containerData);
+ // Initialize pending deletion blocks count in in-memory
+ // container status.
+ MetadataStore metadata = KeyUtils.getDB(containerData, conf);
+ List<Map.Entry<byte[], byte[]>> underDeletionBlocks = metadata
+ .getSequentialRangeKVs(null, Integer.MAX_VALUE,
+ MetadataKeyFilters.getDeletingKeyFilter());
+ containerStatus.incrPendingDeletionBlocks(underDeletionBlocks.size());
+
+ List<Map.Entry<byte[], byte[]>> liveKeys = metadata
+ .getRangeKVs(null, Integer.MAX_VALUE,
+ MetadataKeyFilters.getNormalKeyFilter());
+
+ // Get container bytesUsed upon loading container
+ // The in-memory state is updated upon key write or delete
+ // TODO: update containerDataProto and persist it into container MetaFile
+ long bytesUsed = 0;
+ bytesUsed = liveKeys.parallelStream().mapToLong(e-> {
+ KeyData keyData;
+ try {
+ keyData = KeyUtils.getKeyData(e.getValue());
+ return keyData.getSize();
+ } catch (IOException ex) {
+ return 0L;
+ }
+ }).sum();
+ containerStatus.setBytesUsed(bytesUsed);
+
+ containerMap.put(keyName, containerStatus);
+ } catch (IOException | NoSuchAlgorithmException ex) {
+ LOG.error("read failed for file: {} ex: {}", containerName,
+ ex.getMessage());
+
+ // TODO : Add this file to a recovery Queue.
+
+ // Remember that this container is busted and we cannot use it.
+ containerMap.put(keyName, new ContainerStatus(null));
+ throw new StorageContainerException("Unable to read container info",
+ UNABLE_TO_READ_METADATA_DB);
+ } finally {
+ IOUtils.closeStream(dis);
+ IOUtils.closeStream(containerStream);
+ IOUtils.closeStream(metaStream);
+ }
+ }
+
+ /**
+ * Creates a container with the given name.
+ *
+ * @param pipeline -- Nodes which make up this container.
+ * @param containerData - Container Name and metadata.
+ * @throws StorageContainerException - Exception
+ */
+ @Override
+ public void createContainer(Pipeline pipeline, ContainerData containerData)
+ throws StorageContainerException {
+ Preconditions.checkNotNull(containerData, "Container data cannot be null");
+ writeLock();
+ try {
+ if (containerMap.containsKey(containerData.getName())) {
+ LOG.debug("container already exists. {}", containerData.getName());
+ throw new StorageContainerException("container already exists.",
+ CONTAINER_EXISTS);
+ }
+
+ // This is by design. We first write and close the
+ // container Info and metadata to a directory.
+ // Then read back and put that info into the containerMap.
+ // This allows us to make sure that our write is consistent.
+
+ writeContainerInfo(containerData, false);
+ File cFile = new File(containerData.getContainerPath());
+ readContainerInfo(ContainerUtils.getContainerNameFromFile(cFile));
+ } catch (NoSuchAlgorithmException ex) {
+ LOG.error("Internal error: We seem to be running a JVM without a " +
+ "needed hash algorithm.");
+ throw new StorageContainerException("failed to create container",
+ NO_SUCH_ALGORITHM);
+ } finally {
+ writeUnlock();
+ }
+
+ }
+
+ /**
+ * Writes a container to a chosen location and updates the container Map.
+ *
+ * The file formats of ContainerData and Container Meta is the following.
+ *
+ * message ContainerData {
+ * required string name = 1;
+ * repeated KeyValue metadata = 2;
+ * optional string dbPath = 3;
+ * optional string containerPath = 4;
+ * optional int64 bytesUsed = 5;
+ * optional int64 size = 6;
+ * }
+ *
+ * message ContainerMeta {
+ * required string fileName = 1;
+ * required string hash = 2;
+ * }
+ *
+ * @param containerData - container Data
+ * @param overwrite - Whether we are overwriting.
+ * @throws StorageContainerException, NoSuchAlgorithmException
+ */
+ private void writeContainerInfo(ContainerData containerData,
+ boolean overwrite)
+ throws StorageContainerException, NoSuchAlgorithmException {
+
+ Preconditions.checkNotNull(this.locationManager,
+ "Internal error: location manager cannot be null");
+
+ FileOutputStream containerStream = null;
+ DigestOutputStream dos = null;
+ FileOutputStream metaStream = null;
+
+ try {
+ Path metadataPath = null;
+ Path location = (!overwrite) ? locationManager.getContainerPath():
+ Paths.get(containerData.getContainerPath()).getParent();
+ if (location == null) {
+ throw new StorageContainerException(
+ "Failed to get container file path.",
+ CONTAINER_INTERNAL_ERROR);
+ }
+
+ File containerFile = ContainerUtils.getContainerFile(containerData,
+ location);
+ File metadataFile = ContainerUtils.getMetadataFile(containerData,
+ location);
+ String containerName = containerData.getContainerName();
+
+ if(!overwrite) {
+ ContainerUtils.verifyIsNewContainer(containerFile, metadataFile);
+ metadataPath = this.locationManager.getDataPath(containerName);
+ metadataPath = ContainerUtils.createMetadata(metadataPath,
+ containerName, conf);
+ } else {
+ metadataPath = ContainerUtils.getMetadataDirectory(containerData);
+ }
+
+ containerStream = new FileOutputStream(containerFile);
+ metaStream = new FileOutputStream(metadataFile);
+ MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
+
+ dos = new DigestOutputStream(containerStream, sha);
+ containerData.setDBPath(metadataPath.resolve(
+ ContainerUtils.getContainerDbFileName(containerName))
+ .toString());
+ containerData.setContainerPath(containerFile.toString());
+
+ ContainerProtos.ContainerData protoData = containerData
+ .getProtoBufMessage();
+ protoData.writeDelimitedTo(dos);
+
+ ContainerProtos.ContainerMeta protoMeta = ContainerProtos
+ .ContainerMeta.newBuilder()
+ .setFileName(containerFile.toString())
+ .setHash(DigestUtils.sha256Hex(sha.digest()))
+ .build();
+ protoMeta.writeDelimitedTo(metaStream);
+
+ } catch (IOException ex) {
+ // TODO : we need to clean up partially constructed files
+ // The proper way to do would be for a thread
+ // to read all these 3 artifacts and make sure they are
+ // sane. That info needs to come from the replication
+ // pipeline, and if not consistent delete these file.
+
+ // In case of ozone this is *not* a deal breaker since
+ // SCM is guaranteed to generate unique container names.
+ // The saving grace is that we check if we have residue files
+ // lying around when creating a new container. We need to queue
+ // this information to a cleaner thread.
+
+ LOG.error("Creation of container failed. Name: {}, we might need to " +
+ "cleanup partially created artifacts. ",
+ containerData.getContainerName(), ex);
+ throw new StorageContainerException("Container creation failed. ",
+ ex, CONTAINER_INTERNAL_ERROR);
+ } finally {
+ IOUtils.closeStream(dos);
+ IOUtils.closeStream(containerStream);
+ IOUtils.closeStream(metaStream);
+ }
+ }
+
+ /**
+ * Deletes an existing container.
+ *
+ * @param pipeline - nodes that make this container.
+ * @param containerName - name of the container.
+ * @param forceDelete - whether this container should be deleted forcibly.
+ * @throws StorageContainerException
+ */
+ @Override
+ public void deleteContainer(Pipeline pipeline, String containerName,
+ boolean forceDelete) throws StorageContainerException {
+ Preconditions.checkNotNull(containerName, "Container name cannot be null");
+ Preconditions.checkState(containerName.length() > 0,
+ "Container name length cannot be zero.");
+ writeLock();
+ try {
+ if (isOpen(pipeline.getContainerName())) {
+ throw new StorageContainerException(
+ "Deleting an open container is not allowed.",
+ UNCLOSED_CONTAINER_IO);
+ }
+
+ ContainerStatus status = containerMap.get(containerName);
+ if (status == null) {
+ LOG.debug("No such container. Name: {}", containerName);
+ throw new StorageContainerException("No such container. Name : " +
+ containerName, CONTAINER_NOT_FOUND);
+ }
+ if (status.getContainer() == null) {
+ LOG.debug("Invalid container data. Name: {}", containerName);
+ throw new StorageContainerException("Invalid container data. Name : " +
+ containerName, CONTAINER_NOT_FOUND);
+ }
+ ContainerUtils.removeContainer(status.getContainer(), conf, forceDelete);
+ containerMap.remove(containerName);
+ } catch (StorageContainerException e) {
+ throw e;
+ } catch (IOException e) {
+ // TODO : An I/O error during delete can leave partial artifacts on the
+ // disk. We will need the cleaner thread to cleanup this information.
+ LOG.error("Failed to cleanup container. Name: {}", containerName, e);
+ throw new StorageContainerException(containerName, e, IO_EXCEPTION);
+ } finally {
+ writeUnlock();
+ }
+ }
+
+ /**
+ * A simple interface for container Iterations.
+ * <p/>
+ * This call make no guarantees about consistency of the data between
+ * different list calls. It just returns the best known data at that point of
+ * time. It is possible that using this iteration you can miss certain
+ * container from the listing.
+ *
+ * @param prefix - Return keys that match this prefix.
+ * @param count - how many to return
+ * @param prevKey - Previous Key Value or empty String.
+ * @param data - Actual containerData
+ * @throws StorageContainerException
+ */
+ @Override
+ public void listContainer(String prefix, long count, String prevKey,
+ List<ContainerData> data) throws StorageContainerException {
+ // TODO : Support list with Prefix and PrevKey
+ Preconditions.checkNotNull(data,
+ "Internal assertion: data cannot be null");
+ readLock();
+ try {
+ ConcurrentNavigableMap<String, ContainerStatus> map;
+ if (prevKey == null || prevKey.isEmpty()) {
+ map = containerMap.tailMap(containerMap.firstKey(), true);
+ } else {
+ map = containerMap.tailMap(prevKey, false);
+ }
+
+ int currentCount = 0;
+ for (ContainerStatus entry : map.values()) {
+ if (currentCount < count) {
+ data.add(entry.getContainer());
+ currentCount++;
+ } else {
+ return;
+ }
+ }
+ } finally {
+ readUnlock();
+ }
+ }
+
+ /**
+ * Get metadata about a specific container.
+ *
+ * @param containerName - Name of the container
+ * @return ContainerData - Container Data.
+ * @throws StorageContainerException
+ */
+ @Override
+ public ContainerData readContainer(String containerName) throws
+ StorageContainerException {
+ Preconditions.checkNotNull(containerName, "Container name cannot be null");
+ Preconditions.checkState(containerName.length() > 0,
+ "Container name length cannot be zero.");
+ if (!containerMap.containsKey(containerName)) {
+ throw new StorageContainerException("Unable to find the container. Name: "
+ + containerName, CONTAINER_NOT_FOUND);
+ }
+ ContainerData cData = containerMap.get(containerName).getContainer();
+ if (cData == null) {
+ throw new StorageContainerException("Invalid container data. Name: "
+ + containerName, CONTAINER_INTERNAL_ERROR);
+ }
+ return cData;
+ }
+
+ /**
+ * Closes a open container, if it is already closed or does not exist a
+ * StorageContainerException is thrown.
+ *
+ * @param containerName - Name of the container.
+ * @throws StorageContainerException
+ */
+ @Override
+ public void closeContainer(String containerName)
+ throws StorageContainerException, NoSuchAlgorithmException {
+ ContainerData containerData = readContainer(containerName);
+ containerData.closeContainer();
+ writeContainerInfo(containerData, true);
+ MetadataStore db = KeyUtils.getDB(containerData, conf);
+
+ // It is ok if this operation takes a bit of time.
+ // Close container is not expected to be instantaneous.
+ try {
+ db.compactDB();
+ } catch (IOException e) {
+ LOG.error("Error in DB compaction while closing container", e);
+ throw new StorageContainerException(e, ERROR_IN_COMPACT_DB);
+ }
+
+ // Active is different from closed. Closed means it is immutable, active
+ // false means we have some internal error that is happening to this
+ // container. This is a way to track damaged containers if we have an
+ // I/O failure, this allows us to take quick action in case of container
+ // issues.
+
+ ContainerStatus status = new ContainerStatus(containerData);
+ containerMap.put(containerName, status);
+ }
+
+ @Override
+ public void updateContainer(Pipeline pipeline, String containerName,
+ ContainerData data, boolean forceUpdate)
+ throws StorageContainerException {
+ Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
+ Preconditions.checkNotNull(containerName, "Container name cannot be null");
+ Preconditions.checkNotNull(data, "Container data cannot be null");
+ FileOutputStream containerStream = null;
+ DigestOutputStream dos = null;
+ MessageDigest sha = null;
+ File containerFileBK = null, containerFile = null;
+ boolean deleted = false;
+
+ if(!containerMap.containsKey(containerName)) {
+ throw new StorageContainerException("Container doesn't exist. Name :"
+ + containerName, CONTAINER_NOT_FOUND);
+ }
+
+ try {
+ sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
+ } catch (NoSuchAlgorithmException e) {
+ throw new StorageContainerException("Unable to create Message Digest,"
+ + " usually this is a java configuration issue.",
+ NO_SUCH_ALGORITHM);
+ }
+
+ try {
+ Path location = locationManager.getContainerPath();
+ ContainerData orgData = containerMap.get(containerName).getContainer();
+ if (orgData == null) {
+ // updating a invalid container
+ throw new StorageContainerException("Update a container with invalid" +
+ "container meta data", CONTAINER_INTERNAL_ERROR);
+ }
+
+ if (!forceUpdate && !orgData.isOpen()) {
+ throw new StorageContainerException(
+ "Update a closed container is not allowed. Name: " + containerName,
+ UNSUPPORTED_REQUEST);
+ }
+
+ containerFile = ContainerUtils.getContainerFile(orgData, location);
+ // If forceUpdate is true, there is no need to check
+ // whether the container file exists.
+ if (!forceUpdate) {
+ if (!containerFile.exists() || !containerFile.canWrite()) {
+ throw new StorageContainerException(
+ "Container file not exists or corrupted. Name: " + containerName,
+ CONTAINER_INTERNAL_ERROR);
+ }
+
+ // Backup the container file
+ containerFileBK = File.createTempFile(
+ "tmp_" + System.currentTimeMillis() + "_",
+ containerFile.getName(), containerFile.getParentFile());
+ FileUtils.copyFile(containerFile, containerFileBK);
+
+ deleted = containerFile.delete();
+ containerStream = new FileOutputStream(containerFile);
+ dos = new DigestOutputStream(containerStream, sha);
+
+ ContainerProtos.ContainerData protoData = data.getProtoBufMessage();
+ protoData.writeDelimitedTo(dos);
+ }
+
+ // Update the in-memory map
+ ContainerStatus newStatus = new ContainerStatus(data);
+ containerMap.replace(containerName, newStatus);
+ } catch (IOException e) {
+ // Restore the container file from backup
+ if(containerFileBK != null && containerFileBK.exists() && deleted) {
+ if(containerFile.delete()
+ && containerFileBK.renameTo(containerFile)) {
+ throw new StorageContainerException("Container update failed,"
+ + " container data restored from the backup.",
+ CONTAINER_INTERNAL_ERROR);
+ } else {
+ throw new StorageContainerException(
+ "Failed to restore container data from the backup. Name: "
+ + containerName, CONTAINER_INTERNAL_ERROR);
+ }
+ } else {
+ throw new StorageContainerException(
+ e.getMessage(), CONTAINER_INTERNAL_ERROR);
+ }
+ } finally {
+ if (containerFileBK != null && containerFileBK.exists()) {
+ if(!containerFileBK.delete()) {
+ LOG.warn("Unable to delete container file backup : {}.",
+ containerFileBK.getAbsolutePath());
+ }
+ }
+ IOUtils.closeStream(dos);
+ IOUtils.closeStream(containerStream);
+ }
+ }
+
+ @VisibleForTesting
+ protected File getContainerFile(ContainerData data) throws IOException {
+ return ContainerUtils.getContainerFile(data,
+ this.locationManager.getContainerPath());
+ }
+
+ /**
+ * Checks if a container exists.
+ *
+ * @param containerName - Name 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.
+ */
+ @Override
+ public boolean isOpen(String containerName) throws StorageContainerException {
+ final ContainerStatus status = containerMap.get(containerName);
+ if (status == null) {
+ throw new StorageContainerException(
+ "Container status not found: " + containerName, CONTAINER_NOT_FOUND);
+ }
+ final ContainerData cData = status.getContainer();
+ if (cData == null) {
+ throw new StorageContainerException(
+ "Container not found: " + containerName, CONTAINER_NOT_FOUND);
+ }
+ return cData.isOpen();
+ }
+
+ /**
+ * Supports clean shutdown of container.
+ *
+ * @throws IOException
+ */
+ @Override
+ public void shutdown() throws IOException {
+ Preconditions.checkState(this.hasWriteLock(),
+ "Assumption that we are holding the lock violated.");
+ this.containerMap.clear();
+ this.locationManager.shutdown();
+ }
+
+
+ @VisibleForTesting
+ public ConcurrentSkipListMap<String, ContainerStatus> getContainerMap() {
+ return containerMap;
+ }
+
+ /**
+ * Acquire read lock.
+ */
+ @Override
+ public void readLock() {
+ this.lock.readLock().lock();
+
+ }
+
+ @Override
+ public void readLockInterruptibly() throws InterruptedException {
+ this.lock.readLock().lockInterruptibly();
+ }
+
+ /**
+ * Release read lock.
+ */
+ @Override
+ public void readUnlock() {
+ this.lock.readLock().unlock();
+ }
+
+ /**
+ * Check if the current thread holds read lock.
+ */
+ @Override
+ public boolean hasReadLock() {
+ return this.lock.readLock().tryLock();
+ }
+
+ /**
+ * Acquire write lock.
+ */
+ @Override
+ public void writeLock() {
+ this.lock.writeLock().lock();
+ }
+
+ /**
+ * Acquire write lock, unless interrupted while waiting.
+ */
+ @Override
+ public void writeLockInterruptibly() throws InterruptedException {
+ this.lock.writeLock().lockInterruptibly();
+
+ }
+
+ /**
+ * Release write lock.
+ */
+ @Override
+ public void writeUnlock() {
+ this.lock.writeLock().unlock();
+
+ }
+
+ /**
+ * Check if the current thread holds write lock.
+ */
+ @Override
+ public boolean hasWriteLock() {
+ return this.lock.writeLock().isHeldByCurrentThread();
+ }
+
+ public ChunkManager getChunkManager() {
+ return this.chunkManager;
+ }
+
+ /**
+ * Sets the chunk Manager.
+ *
+ * @param chunkManager - Chunk Manager
+ */
+ public void setChunkManager(ChunkManager chunkManager) {
+ this.chunkManager = chunkManager;
+ }
+
+ /**
+ * Gets the Key Manager.
+ *
+ * @return KeyManager.
+ */
+ @Override
+ public KeyManager getKeyManager() {
+ return this.keyManager;
+ }
+
+ /**
+ * Get the node report.
+ * @return node report.
+ */
+ @Override
+ public SCMNodeReport getNodeReport() throws IOException {
+ StorageLocationReport[] reports = locationManager.getLocationReport();
+ SCMNodeReport.Builder nrb = SCMNodeReport.newBuilder();
+ for (int i = 0; i < reports.length; i++) {
+ SCMStorageReport.Builder srb = SCMStorageReport.newBuilder();
+ nrb.addStorageReport(i, srb.setStorageUuid(reports[i].getId())
+ .setCapacity(reports[i].getCapacity())
+ .setScmUsed(reports[i].getScmUsed())
+ .setRemaining(reports[i].getRemaining())
+ .build());
+ }
+ return nrb.build();
+ }
+
+
+ /**
+ * Gets container reports.
+ *
+ * @return List of all closed containers.
+ * @throws IOException
+ */
+ @Override
+ public List<ContainerData> getContainerReports() throws IOException {
+ LOG.debug("Starting container report iteration.");
+ // No need for locking since containerMap is a ConcurrentSkipListMap
+ // And we can never get the exact state since close might happen
+ // after we iterate a point.
+ return containerMap.entrySet().stream()
+ .filter(containerStatus ->
+ !containerStatus.getValue().getContainer().isOpen())
+ .map(containerStatus -> containerStatus.getValue().getContainer())
+ .collect(Collectors.toList());
+ }
+
+ /**
+ * Get container report.
+ *
+ * @return The container report.
+ * @throws IOException
+ */
+ @Override
+ public ContainerReportsRequestProto getContainerReport() throws IOException {
+ LOG.debug("Starting container report iteration.");
+ // No need for locking since containerMap is a ConcurrentSkipListMap
+ // And we can never get the exact state since close might happen
+ // after we iterate a point.
+ List<ContainerStatus> containers = containerMap.values().stream()
+ .collect(Collectors.toList());
+
+ ContainerReportsRequestProto.Builder crBuilder =
+ ContainerReportsRequestProto.newBuilder();
+
+ // TODO: support delta based container report
+ crBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage())
+ .setType(ContainerReportsRequestProto.reportType.fullReport);
+
+ for (ContainerStatus container: containers) {
+ StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
+ StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder();
+ ciBuilder.setContainerName(container.getContainer().getContainerName())
+ .setSize(container.getContainer().getMaxSize())
+ .setUsed(container.getContainer().getBytesUsed())
+ .setKeyCount(container.getContainer().getKeyCount())
+ .setReadCount(container.getReadCount())
+ .setWriteCount(container.getWriteCount())
+ .setReadBytes(container.getReadBytes())
+ .setWriteBytes(container.getWriteBytes())
+ .setContainerID(container.getContainer().getContainerID());
+
+ if (container.getContainer().getHash() != null) {
+ ciBuilder.setFinalhash(container.getContainer().getHash());
+ }
+ crBuilder.addReports(ciBuilder.build());
+ }
+
+ return crBuilder.build();
+ }
+
+ /**
+ * Sets the Key Manager.
+ *
+ * @param keyManager - Key Manager.
+ */
+ @Override
+ public void setKeyManager(KeyManager keyManager) {
+ this.keyManager = keyManager;
+ }
+
+ /**
+ * Filter out only container files from the container metadata dir.
+ */
+ private static class ContainerFilter implements FilenameFilter {
+ /**
+ * Tests if a specified file should be included in a file list.
+ *
+ * @param dir the directory in which the file was found.
+ * @param name the name of the file.
+ * @return <code>true</code> if and only if the name should be included in
+ * the file list; <code>false</code> otherwise.
+ */
+ @Override
+ public boolean accept(File dir, String name) {
+ return name.endsWith(CONTAINER_EXTENSION);
+ }
+ }
+
+ @Override
+ public List<ContainerData> chooseContainerForBlockDeletion(
+ int count) throws StorageContainerException {
+ readLock();
+ try {
+ return containerDeletionChooser.chooseContainerForBlockDeletion(
+ count, containerMap);
+ } finally {
+ readUnlock();
+ }
+ }
+
+ @VisibleForTesting
+ public ContainerDeletionChoosingPolicy getContainerDeletionChooser() {
+ return containerDeletionChooser;
+ }
+
+ @Override
+ public void incrPendingDeletionBlocks(int numBlocks, String containerId) {
+ writeLock();
+ try {
+ ContainerStatus status = containerMap.get(containerId);
+ status.incrPendingDeletionBlocks(numBlocks);
+ } finally {
+ writeUnlock();
+ }
+ }
+
+ @Override
+ public void decrPendingDeletionBlocks(int numBlocks, String containerId) {
+ writeLock();
+ try {
+ ContainerStatus status = containerMap.get(containerId);
+ status.decrPendingDeletionBlocks(numBlocks);
+ } finally {
+ writeUnlock();
+ }
+ }
+
+ /**
+ * Increase the read count of the container.
+ *
+ * @param containerName - Name of the container.
+ */
+ @Override
+ public void incrReadCount(String containerName) {
+ ContainerStatus status = containerMap.get(containerName);
+ status.incrReadCount();
+ }
+
+ public long getReadCount(String containerName) {
+ ContainerStatus status = containerMap.get(containerName);
+ return status.getReadCount();
+ }
+
+ /**
+ * Increse the read counter for bytes read from the container.
+ *
+ * @param containerName - Name of the container.
+ * @param readBytes - bytes read from the container.
+ */
+ @Override
+ public void incrReadBytes(String containerName, long readBytes) {
+ ContainerStatus status = containerMap.get(containerName);
+ status.incrReadBytes(readBytes);
+ }
+
+ public long getReadBytes(String containerName) {
+ readLock();
+ try {
+ ContainerStatus status = containerMap.get(containerName);
+ return status.getReadBytes();
+ } finally {
+ readUnlock();
+ }
+ }
+
+ /**
+ * Increase the write count of the container.
+ *
+ * @param containerName - Name of the container.
+ */
+ @Override
+ public void incrWriteCount(String containerName) {
+ ContainerStatus status = containerMap.get(containerName);
+ status.incrWriteCount();
+ }
+
+ public long getWriteCount(String containerName) {
+ ContainerStatus status = containerMap.get(containerName);
+ return status.getWriteCount();
+ }
+
+ /**
+ * Increse the write counter for bytes write into the container.
+ *
+ * @param containerName - Name of the container.
+ * @param writeBytes - bytes write into the container.
+ */
+ @Override
+ public void incrWriteBytes(String containerName, long writeBytes) {
+ ContainerStatus status = containerMap.get(containerName);
+ status.incrWriteBytes(writeBytes);
+ }
+
+ public long getWriteBytes(String containerName) {
+ ContainerStatus status = containerMap.get(containerName);
+ return status.getWriteBytes();
+ }
+
+ /**
+ * Increase the bytes used by the container.
+ *
+ * @param containerName - Name of the container.
+ * @param used - additional bytes used by the container.
+ * @return the current bytes used.
+ */
+ @Override
+ public long incrBytesUsed(String containerName, long used) {
+ ContainerStatus status = containerMap.get(containerName);
+ return status.incrBytesUsed(used);
+ }
+
+ /**
+ * Decrease the bytes used by the container.
+ *
+ * @param containerName - Name of the container.
+ * @param used - additional bytes reclaimed by the container.
+ * @return the current bytes used.
+ */
+ @Override
+ public long decrBytesUsed(String containerName, long used) {
+ ContainerStatus status = containerMap.get(containerName);
+ return status.decrBytesUsed(used);
+ }
+
+ public long getBytesUsed(String containerName) {
+ ContainerStatus status = containerMap.get(containerName);
+ return status.getBytesUsed();
+ }
+
+ /**
+ * Get the number of keys in the container.
+ *
+ * @param containerName - Name of the container.
+ * @return the current key count.
+ */
+ @Override
+ public long getNumKeys(String containerName) {
+ ContainerStatus status = containerMap.get(containerName);
+ return status.getNumKeys(); }
+
+ /**
+ * Get the container report state to send via HB to SCM.
+ *
+ * @return container report state.
+ */
+ @Override
+ public ReportState getContainerReportState() {
+ return containerReportManager.getContainerReportState();
+ }
+
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerReportManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerReportManagerImpl.java
new file mode 100644
index 0000000000..6c83c66a3a
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerReportManagerImpl.java
@@ -0,0 +1,90 @@
+/*
+ * 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.commons.lang3.RandomUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.ReportState;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.container.common.interfaces
+ .ContainerReportManager;
+import org.apache.hadoop.util.Time;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.hdds.scm.HddsServerUtil.getScmHeartbeatInterval;
+
+/**
+ * Class wraps the container report operations on datanode.
+ * // TODO: support incremental/delta container report
+ */
+public class ContainerReportManagerImpl implements ContainerReportManager {
+ // Last non-empty container report time
+ private long lastContainerReportTime;
+ private final long containerReportInterval;
+ private final long heartbeatInterval;
+ private AtomicLong reportCount;
+ private static final ReportState NO_CONTAINER_REPORTSTATE =
+ ReportState.newBuilder()
+ .setState(ReportState.states.noContainerReports)
+ .setCount(0).build();
+
+ public ContainerReportManagerImpl(Configuration config) {
+ this.lastContainerReportTime = -1;
+ this.reportCount = new AtomicLong(0L);
+ this.containerReportInterval = config.getTimeDuration(
+ OzoneConfigKeys.OZONE_CONTAINER_REPORT_INTERVAL,
+ OzoneConfigKeys.OZONE_CONTAINER_REPORT_INTERVAL_DEFAULT,
+ TimeUnit.MILLISECONDS);
+ this.heartbeatInterval = getScmHeartbeatInterval(config);
+ }
+
+ public ReportState getContainerReportState() {
+ if (lastContainerReportTime < 0) {
+ return getFullContainerReportState();
+ } else {
+ // Add a random delay (0~30s) on top of the container report
+ // interval (60s) so tha the SCM is overwhelmed by the container reports
+ // sent in sync.
+ if (Time.monotonicNow() - lastContainerReportTime >
+ (containerReportInterval + getRandomReportDelay())) {
+ return getFullContainerReportState();
+ } else {
+ return getNoContainerReportState();
+ }
+ }
+ }
+
+ private ReportState getFullContainerReportState() {
+ ReportState.Builder rsBuilder = ReportState.newBuilder();
+ rsBuilder.setState(ReportState.states.completeContinerReport);
+ rsBuilder.setCount(reportCount.incrementAndGet());
+ this.lastContainerReportTime = Time.monotonicNow();
+ return rsBuilder.build();
+ }
+
+ private ReportState getNoContainerReportState() {
+ return NO_CONTAINER_REPORTSTATE;
+ }
+
+ private long getRandomReportDelay() {
+ return RandomUtils.nextLong(0, heartbeatInterval);
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerStatus.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerStatus.java
new file mode 100644
index 0000000000..5577323a10
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerStatus.java
@@ -0,0 +1,217 @@
+/**
+ * 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.ozone.container.common.helpers.ContainerData;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This class represents the state of a container. if the
+ * container reading encountered an error when we boot up we will post that
+ * info to a recovery queue and keep the info in the containerMap.
+ * <p/>
+ * if and when the issue is fixed, the expectation is that this entry will be
+ * deleted by the recovery thread from the containerMap and will insert entry
+ * instead of modifying this class.
+ */
+public class ContainerStatus {
+ private final ContainerData containerData;
+
+ /**
+ * Number of pending deletion blocks in container.
+ */
+ private int numPendingDeletionBlocks;
+
+ private AtomicLong readBytes;
+
+ private AtomicLong writeBytes;
+
+ private AtomicLong readCount;
+
+ private AtomicLong writeCount;
+
+ /**
+ * Creates a Container Status class.
+ *
+ * @param containerData - ContainerData.
+ */
+ ContainerStatus(ContainerData containerData) {
+ this.numPendingDeletionBlocks = 0;
+ this.containerData = containerData;
+ this.readCount = new AtomicLong(0L);
+ this.readBytes = new AtomicLong(0L);
+ this.writeCount = new AtomicLong(0L);
+ this.writeBytes = new AtomicLong(0L);
+ }
+
+ /**
+ * Returns container if it is active. It is not active if we have had an
+ * error and we are waiting for the background threads to fix the issue.
+ *
+ * @return ContainerData.
+ */
+ public ContainerData getContainer() {
+ return containerData;
+ }
+
+ /**
+ * 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;
+ }
+
+ /**
+ * 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();
+ }
+
+ /**
+ * Get the number of bytes used by the container.
+ * @return the number of bytes used by the container.
+ */
+ public long getBytesUsed() {
+ return containerData.getBytesUsed();
+ }
+
+ /**
+ * 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 containerData.addBytesUsed(used);
+ }
+
+ /**
+ * Set the number of bytes used by the container.
+ * @param used the number of bytes used by the container.
+ */
+ public void setBytesUsed(long used) {
+ containerData.setBytesUsed(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.containerData.addBytesUsed(-1L * reclaimed);
+ }
+
+ /**
+ * Get the maximum container size.
+ * @return the maximum container size.
+ */
+ public long getMaxSize() {
+ return containerData.getMaxSize();
+ }
+
+ /**
+ * Set the maximum container size.
+ * @param size the maximum container size.
+ */
+ public void setMaxSize(long size) {
+ this.containerData.setMaxSize(size);
+ }
+
+ /**
+ * Get the number of keys in the container.
+ * @return the number of keys in the container.
+ */
+ public long getNumKeys() {
+ return containerData.getKeyCount();
+ }
+} \ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerStorageLocation.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerStorageLocation.java
new file mode 100644
index 0000000000..7293895295
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerStorageLocation.java
@@ -0,0 +1,203 @@
+/*
+ * 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.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 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);
+ }
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
new file mode 100644
index 0000000000..1c6e39ccdf
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
@@ -0,0 +1,713 @@
+/*
+ * 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.protobuf.ByteString;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+ .StorageContainerException;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
+ .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
+ .ContainerCommandResponseProto;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Type;
+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.common.helpers.FileUtils;
+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.proto.ContainerProtos.Result
+ .CLOSED_CONTAINER_IO;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .GET_SMALL_FILE_ERROR;
+import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
+ .NO_SUCH_ALGORITHM;
+import static org.apache.hadoop.hdds.protocol.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 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.incContainerOpcMetrics(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);
+ } catch (IllegalStateException | NullPointerException e) {
+ 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().getContainerData().getName(),
+ msg.getCmdType().name(),
+ msg.getTraceID(),
+ ex.toString(), ex);
+
+ // TODO : Replace with finer error codes.
+ return ContainerUtils.getContainerResponse(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().getContainerData().getName(),
+ msg.getCmdType().name(),
+ msg.getTraceID(),
+ ex.toString(), ex);
+
+ // TODO : Replace with finer error codes.
+ return ContainerUtils.getContainerResponse(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().getContainerData().getName(),
+ msg.getCmdType().name(),
+ msg.getTraceID(),
+ ex.toString(), ex);
+
+ // TODO : Replace with finer error codes.
+ return ContainerUtils.getContainerResponse(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);
+ }
+
+ Pipeline pipeline = Pipeline.getFromProtoBuf(
+ msg.getUpdateContainer().getPipeline());
+ String containerName = msg.getUpdateContainer()
+ .getContainerData().getName();
+
+ ContainerData data = ContainerData.getFromProtBuf(
+ msg.getUpdateContainer().getContainerData(), conf);
+ boolean forceUpdate = msg.getUpdateContainer().getForceUpdate();
+ this.containerManager.updateContainer(
+ pipeline, containerName, data, forceUpdate);
+ return ContainerUtils.getContainerResponse(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);
+ }
+
+ String name = msg.getReadContainer().getName();
+ ContainerData container = this.containerManager.readContainer(name);
+ 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);
+ }
+
+ Pipeline pipeline = Pipeline.getFromProtoBuf(
+ msg.getDeleteContainer().getPipeline());
+ Preconditions.checkNotNull(pipeline);
+ String name = msg.getDeleteContainer().getName();
+ boolean forceDelete = msg.getDeleteContainer().getForceDelete();
+ this.containerManager.deleteContainer(pipeline, name, forceDelete);
+ return ContainerUtils.getContainerResponse(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 = ContainerData.getFromProtBuf(
+ msg.getCreateContainer().getContainerData(), conf);
+ Preconditions.checkNotNull(cData, "Container data is null");
+
+ Pipeline pipeline = Pipeline.getFromProtoBuf(
+ msg.getCreateContainer().getPipeline());
+ Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
+
+ this.containerManager.createContainer(pipeline, cData);
+ return ContainerUtils.getContainerResponse(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);
+ }
+ Pipeline pipeline = Pipeline.getFromProtoBuf(msg.getCloseContainer()
+ .getPipeline());
+ Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
+ if (!this.containerManager.isOpen(pipeline.getContainerName())) {
+ throw new StorageContainerException("Attempting to close a closed " +
+ "container.", CLOSED_CONTAINER_IO);
+ }
+ this.containerManager.closeContainer(pipeline.getContainerName());
+ return ContainerUtils.getContainerResponse(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);
+ }
+ String keyName = msg.getWriteChunk().getKeyName();
+ Pipeline pipeline = Pipeline.getFromProtoBuf(
+ msg.getWriteChunk().getPipeline());
+ Preconditions.checkNotNull(pipeline);
+ if (!this.containerManager.isOpen(pipeline.getContainerName())) {
+ 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(pipeline, keyName, 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);
+ }
+
+ String keyName = msg.getReadChunk().getKeyName();
+ Pipeline pipeline = Pipeline.getFromProtoBuf(
+ msg.getReadChunk().getPipeline());
+ Preconditions.checkNotNull(pipeline);
+
+ ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getReadChunk()
+ .getChunkData());
+ Preconditions.checkNotNull(chunkInfo);
+ byte[] data = this.containerManager.getChunkManager().readChunk(pipeline,
+ keyName, 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);
+ }
+
+ String keyName = msg.getDeleteChunk().getKeyName();
+ Pipeline pipeline = Pipeline.getFromProtoBuf(
+ msg.getDeleteChunk().getPipeline());
+ Preconditions.checkNotNull(pipeline);
+ if (!this.containerManager.isOpen(pipeline.getContainerName())) {
+ throw new StorageContainerException("Write to closed container.",
+ CLOSED_CONTAINER_IO);
+ }
+ ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getDeleteChunk()
+ .getChunkData());
+ Preconditions.checkNotNull(chunkInfo);
+
+ this.containerManager.getChunkManager().deleteChunk(pipeline, keyName,
+ 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);
+ }
+ Pipeline pipeline = Pipeline.getFromProtoBuf(msg.getPutKey().getPipeline());
+ Preconditions.checkNotNull(pipeline);
+ if (!this.containerManager.isOpen(pipeline.getContainerName())) {
+ throw new StorageContainerException("Write to closed container.",
+ CLOSED_CONTAINER_IO);
+ }
+ KeyData keyData = KeyData.getFromProtoBuf(msg.getPutKey().getKeyData());
+ Preconditions.checkNotNull(keyData);
+ this.containerManager.getKeyManager().putKey(pipeline, 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 = KeyData.getFromProtoBuf(msg.getGetKey().getKeyData());
+ 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);
+ }
+ Pipeline pipeline =
+ Pipeline.getFromProtoBuf(msg.getDeleteKey().getPipeline());
+ Preconditions.checkNotNull(pipeline);
+ if (!this.containerManager.isOpen(pipeline.getContainerName())) {
+ throw new StorageContainerException("Write to closed container.",
+ CLOSED_CONTAINER_IO);
+ }
+ String keyName = msg.getDeleteKey().getName();
+ Preconditions.checkNotNull(keyName);
+ Preconditions.checkState(!keyName.isEmpty());
+ this.containerManager.getKeyManager().deleteKey(pipeline, keyName);
+ 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 {
+
+ Pipeline pipeline =
+ Pipeline.getFromProtoBuf(msg.getPutSmallFile()
+ .getKey().getPipeline());
+
+ Preconditions.checkNotNull(pipeline);
+ if (!this.containerManager.isOpen(pipeline.getContainerName())) {
+ 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(pipeline, keyData
+ .getKeyName(), chunkInfo, data, ContainerProtos.Stage.COMBINED);
+ List<ContainerProtos.ChunkInfo> chunks = new LinkedList<>();
+ chunks.add(chunkInfo.getProtoBufMessage());
+ keyData.setChunks(chunks);
+ this.containerManager.getKeyManager().putKey(pipeline, keyData);
+ return FileUtils.getPutFileResponse(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 {
+ Pipeline pipeline =
+ Pipeline.getFromProtoBuf(msg.getGetSmallFile()
+ .getKey().getPipeline());
+
+ long bytes = 0;
+ Preconditions.checkNotNull(pipeline);
+ KeyData keyData = KeyData.getFromProtoBuf(msg.getGetSmallFile()
+ .getKey().getKeyData());
+ 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(
+ pipeline, keyData.getKeyName(), ChunkInfo.getFromProtoBuf(
+ chunk)));
+ dataBuf = dataBuf.concat(current);
+ c = chunk;
+ }
+ metrics.incContainerBytesStats(Type.GetSmallFile, bytes);
+ return FileUtils.getGetSmallFileResponse(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);
+ }
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java
new file mode 100644
index 0000000000..cf6bf12214
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java
@@ -0,0 +1,202 @@
+/*
+ * 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.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+ .StorageContainerException;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
+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.MetadataKeyFilters.KeyPrefixFilter;
+import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
+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.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(Pipeline pipeline, KeyData data) throws IOException {
+ 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.
+ Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
+ String containerName = pipeline.getContainerName();
+ Preconditions.checkNotNull(containerName,
+ "Container name cannot be null");
+ ContainerData cData = containerManager.readContainer(containerName);
+ 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(data.getKeyName().getBytes(KeyUtils.ENCODING), 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.getContainerName(),
+ "Container name cannot be null");
+ ContainerData cData = containerManager.readContainer(data
+ .getContainerName());
+ 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(data.getKeyName().getBytes(KeyUtils.ENCODING));
+ 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(Pipeline pipeline, String keyName)
+ throws IOException {
+ containerManager.readLock();
+ try {
+ Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
+ String containerName = pipeline.getContainerName();
+ Preconditions.checkNotNull(containerName,
+ "Container name cannot be null");
+ ContainerData cData = containerManager.readContainer(containerName);
+ 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[] kData = db.get(keyName.getBytes(KeyUtils.ENCODING));
+ if (kData == null) {
+ throw new StorageContainerException("Unable to find the key.",
+ NO_SUCH_KEY);
+ }
+ db.delete(keyName.getBytes(KeyUtils.ENCODING));
+ } finally {
+ containerManager.readUnlock();
+ }
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public List<KeyData> listKey(
+ Pipeline pipeline, String prefix, String startKey, int count)
+ throws IOException {
+ Preconditions.checkNotNull(pipeline,
+ "Pipeline cannot be null.");
+ Preconditions.checkArgument(count > 0,
+ "Count must be a positive number.");
+ ContainerData cData = containerManager.readContainer(pipeline
+ .getContainerName());
+ MetadataStore db = KeyUtils.getDB(cData, conf);
+
+ List<KeyData> result = new ArrayList<KeyData>();
+ byte[] startKeyInBytes = startKey == null ? null :
+ DFSUtil.string2Bytes(startKey);
+ MetadataKeyFilter prefixFilter = new KeyPrefixFilter(prefix);
+ List<Map.Entry<byte[], byte[]>> range =
+ db.getSequentialRangeKVs(startKeyInBytes, count, prefixFilter);
+ for (Map.Entry<byte[], byte[]> entry : range) {
+ String keyName = KeyUtils.getKeyName(entry.getKey());
+ KeyData value = KeyUtils.getKeyData(entry.getValue());
+ KeyData data = new KeyData(value.getContainerName(), keyName);
+ 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));
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
new file mode 100644
index 0000000000..3e267d2b37
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
@@ -0,0 +1,70 @@
+/**
+ * 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.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;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Randomly choosing containers for block deletion.
+ */
+public class RandomContainerDeletionChoosingPolicy
+ implements ContainerDeletionChoosingPolicy {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(RandomContainerDeletionChoosingPolicy.class);
+
+ @Override
+ public List<ContainerData> chooseContainerForBlockDeletion(int count,
+ Map<String, ContainerStatus> candidateContainers)
+ throws StorageContainerException {
+ Preconditions.checkNotNull(candidateContainers,
+ "Internal assertion: candidate containers cannot be null");
+
+ int currentCount = 0;
+ List<ContainerData> result = new LinkedList<>();
+ ContainerStatus[] values = new ContainerStatus[candidateContainers.size()];
+ // to get a shuffle list
+ for (ContainerStatus entry : DFSUtil.shuffle(
+ candidateContainers.values().toArray(values))) {
+ if (currentCount < count) {
+ result.add(entry.getContainer());
+ currentCount++;
+
+ LOG.debug("Select container {} for block deletion, "
+ + "pending deletion blocks num: {}.",
+ entry.getContainer().getContainerName(),
+ entry.getNumPendingDeletionBlocks());
+ } else {
+ break;
+ }
+ }
+
+ return result;
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java
new file mode 100644
index 0000000000..7ef91a91f7
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java
@@ -0,0 +1,63 @@
+/**
+ * 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;
+
+/**
+ * Storage location stats of datanodes that provide back store for containers.
+ *
+ */
+public class StorageLocationReport {
+ public static final StorageLocationReport[] EMPTY_ARRAY = {};
+
+ private final String id;
+ private final boolean failed;
+ private final long capacity;
+ private final long scmUsed;
+ private final long remaining;
+
+ public StorageLocationReport(String id, boolean failed,
+ long capacity, long scmUsed, long remaining) {
+ this.id = id;
+ this.failed = failed;
+ this.capacity = capacity;
+ this.scmUsed = scmUsed;
+ this.remaining = remaining;
+ }
+
+ public String getId() {
+ return id;
+ }
+
+ public boolean isFailed() {
+ return failed;
+ }
+
+ public long getCapacity() {
+ return capacity;
+ }
+
+ public long getScmUsed() {
+ return scmUsed;
+ }
+
+ public long getRemaining() {
+ return remaining;
+ }
+
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
new file mode 100644
index 0000000000..0169a96cf9
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
@@ -0,0 +1,91 @@
+/**
+ * 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.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;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * TopN Ordered choosing policy that choosing containers based on pending
+ * deletion blocks' number.
+ */
+public class TopNOrderedContainerDeletionChoosingPolicy
+ implements ContainerDeletionChoosingPolicy {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(TopNOrderedContainerDeletionChoosingPolicy.class);
+
+ /** customized comparator used to compare differentiate container status. **/
+ private static final Comparator<ContainerStatus> CONTAINER_STATUS_COMPARATOR
+ = new Comparator<ContainerStatus>() {
+ @Override
+ public int compare(ContainerStatus c1, ContainerStatus c2) {
+ return Integer.compare(c2.getNumPendingDeletionBlocks(),
+ c1.getNumPendingDeletionBlocks());
+ }
+ };
+
+ @Override
+ public List<ContainerData> chooseContainerForBlockDeletion(int count,
+ Map<String, ContainerStatus> candidateContainers)
+ throws StorageContainerException {
+ Preconditions.checkNotNull(candidateContainers,
+ "Internal assertion: candidate containers cannot be null");
+
+ List<ContainerData> result = new LinkedList<>();
+ List<ContainerStatus> orderedList = new LinkedList<>();
+ orderedList.addAll(candidateContainers.values());
+ Collections.sort(orderedList, CONTAINER_STATUS_COMPARATOR);
+
+ // get top N list ordered by pending deletion blocks' number
+ int currentCount = 0;
+ for (ContainerStatus entry : orderedList) {
+ if (currentCount < count) {
+ if (entry.getNumPendingDeletionBlocks() > 0) {
+ result.add(entry.getContainer());
+ currentCount++;
+
+ LOG.debug(
+ "Select container {} for block deletion, "
+ + "pending deletion blocks num: {}.",
+ entry.getContainer().getContainerName(),
+ entry.getNumPendingDeletionBlocks());
+ } else {
+ LOG.debug("Stop looking for next container, there is no"
+ + " pending deletion block contained in remaining containers.");
+ break;
+ }
+ } else {
+ break;
+ }
+ }
+
+ return result;
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/package-info.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/package-info.java
new file mode 100644
index 0000000000..16da5d9dee
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/package-info.java
@@ -0,0 +1,22 @@
+/**
+ * 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;
+
+/**
+ This package is contains Ozone container implementation.
+**/ \ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java
new file mode 100644
index 0000000000..f55d74ca2b
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java
@@ -0,0 +1,76 @@
+/*
+ * 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.Pipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+ .StorageContainerException;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
+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 pipeline - Name and the set of machines that make this container.
+ * @param keyName - Name of the Key.
+ * @param info - ChunkInfo.
+ * @param stage - Chunk Stage write.
+ * @throws StorageContainerException
+ */
+ void writeChunk(Pipeline pipeline, String keyName,
+ ChunkInfo info, byte[] data, ContainerProtos.Stage stage)
+ throws StorageContainerException;
+
+ /**
+ * reads the data defined by a chunk.
+ * @param pipeline - container pipeline.
+ * @param keyName - Name of the Key
+ * @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(Pipeline pipeline, String keyName, ChunkInfo info) throws
+ StorageContainerException;
+
+ /**
+ * Deletes a given chunk.
+ * @param pipeline - Pipeline.
+ * @param keyName - Key Name
+ * @param info - Chunk Info
+ * @throws StorageContainerException
+ */
+ void deleteChunk(Pipeline pipeline, String keyName, ChunkInfo info) throws
+ StorageContainerException;
+
+ // TODO : Support list operations.
+
+ /**
+ * Shutdown the chunkManager.
+ */
+ void shutdown();
+
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
new file mode 100644
index 0000000000..f7280e2a3c
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.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.ContainerStatus;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This interface is used for choosing desired containers for
+ * block deletion.
+ */
+public interface ContainerDeletionChoosingPolicy {
+
+ /**
+ * Chooses desired containers for block deletion.
+ * @param count
+ * how many to return
+ * @param candidateContainers
+ * candidate containers collection
+ * @return container data list
+ * @throws StorageContainerException
+ */
+ List<ContainerData> chooseContainerForBlockDeletion(int count,
+ Map<String, ContainerStatus> candidateContainers)
+ throws StorageContainerException;
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java
new file mode 100644
index 0000000000..984fe41b91
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java
@@ -0,0 +1,51 @@
+/*
+ * 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.protocol.proto.ContainerProtos
+ .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
+ .ContainerCommandResponseProto;
+
+/**
+ * Dispatcher acts as the bridge between the transport layer and
+ * the actual container layer. This layer is capable of transforming
+ * protobuf objects into corresponding class and issue the function call
+ * into the lower layers.
+ *
+ * The reply from the request is dispatched to the client.
+ */
+public interface ContainerDispatcher {
+ /**
+ * Dispatches commands to container layer.
+ * @param msg - Command Request
+ * @return Command Response
+ */
+ ContainerCommandResponseProto dispatch(ContainerCommandRequestProto msg);
+
+ /**
+ * Initialize the Dispatcher.
+ */
+ void init();
+
+ /**
+ * Shutdown Dispatcher services.
+ */
+ void shutdown();
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManager.java
new file mode 100644
index 0000000000..9c5fcea163
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManager.java
@@ -0,0 +1,58 @@
+/*
+ * 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.ozone.container.common.impl.StorageLocationReport;
+
+import java.io.IOException;
+import java.nio.file.Path;
+
+/**
+ * Returns physical path locations, where the containers will be created.
+ */
+public interface ContainerLocationManager {
+ /**
+ * Returns the path where the container should be placed from a set of
+ * locations.
+ *
+ * @return A path where we should place this container and metadata.
+ * @throws IOException
+ */
+ Path getContainerPath() throws IOException;
+
+ /**
+ * 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
+ */
+ Path getDataPath(String containerName) throws IOException;
+
+ /**
+ * Returns an array of storage location usage report.
+ * @return storage location usage report.
+ */
+ StorageLocationReport[] getLocationReport() throws IOException;
+
+ /**
+ * Supports clean shutdown of container.
+ *
+ * @throws IOException
+ */
+ void shutdown() throws IOException;
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManagerMXBean.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManagerMXBean.java
new file mode 100644
index 0000000000..88e6148630
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManagerMXBean.java
@@ -0,0 +1,36 @@
+/*
+ * 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.ozone.container.common.impl.StorageLocationReport;
+
+import java.io.IOException;
+
+/**
+ * Returns physical path locations, where the containers will be created.
+ */
+public interface ContainerLocationManagerMXBean {
+
+ /**
+ * Returns an array of storage location usage report.
+ *
+ * @return storage location usage report.
+ */
+ StorageLocationReport[] getLocationReport() throws IOException;
+
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java
new file mode 100644
index 0000000000..2ff636e87f
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java
@@ -0,0 +1,280 @@
+/**
+ * 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.Pipeline;
+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.ContainerReportsRequestProto;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.ReportState;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+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 pipeline -- Nodes which make up this container.
+ * @param containerData - Container Name and metadata.
+ * @throws StorageContainerException
+ */
+ void createContainer(Pipeline pipeline, ContainerData containerData)
+ throws StorageContainerException;
+
+ /**
+ * Deletes an existing container.
+ *
+ * @param pipeline - nodes that make this container.
+ * @param containerName - name of the container.
+ * @param forceDelete - whether this container should be deleted forcibly.
+ * @throws StorageContainerException
+ */
+ void deleteContainer(Pipeline pipeline, String containerName,
+ boolean forceDelete) throws StorageContainerException;
+
+ /**
+ * Update an existing container.
+ *
+ * @param pipeline container nodes
+ * @param containerName name of the container
+ * @param data container data
+ * @param forceUpdate if true, update container forcibly.
+ * @throws StorageContainerException
+ */
+ void updateContainer(Pipeline pipeline, String containerName,
+ ContainerData data, boolean forceUpdate) throws StorageContainerException;
+
+ /**
+ * As simple interface for container Iterations.
+ *
+ * @param prefix - Return only values matching this prefix
+ * @param count - how many to return
+ * @param prevKey - Previous key - Server returns results from this point.
+ * @param data - Actual containerData
+ * @throws StorageContainerException
+ */
+ void listContainer(String prefix, long count, String prevKey,
+ 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 containerName - Name of the container
+ * @return ContainerData - Container Data.
+ * @throws StorageContainerException
+ */
+ ContainerData readContainer(String containerName)
+ throws StorageContainerException;
+
+ /**
+ * Closes a open container, if it is already closed or does not exist a
+ * StorageContainerException is thrown.
+ * @param containerName - Name of the container.
+ * @throws StorageContainerException
+ */
+ void closeContainer(String containerName)
+ throws StorageContainerException, NoSuchAlgorithmException;
+
+ /**
+ * Checks if a container exists.
+ * @param containerName - Name 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(String containerName) 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.
+ */
+ SCMNodeReport getNodeReport() throws IOException;
+
+ /**
+ * Gets container report.
+ * @return container report.
+ * @throws IOException
+ */
+ ContainerReportsRequestProto getContainerReport() throws IOException;
+
+ /**
+ * Gets container reports.
+ * @return List of all closed containers.
+ * @throws IOException
+ */
+ List<ContainerData> getContainerReports() throws IOException;
+
+ /**
+ * Increase pending deletion blocks count number of specified container.
+ *
+ * @param numBlocks
+ * increment count number
+ * @param containerId
+ * container id
+ */
+ void incrPendingDeletionBlocks(int numBlocks, String containerId);
+
+ /**
+ * Decrease pending deletion blocks count number of specified container.
+ *
+ * @param numBlocks
+ * decrement count number
+ * @param containerId
+ * container id
+ */
+ void decrPendingDeletionBlocks(int numBlocks, String containerId);
+
+ /**
+ * Increase the read count of the container.
+ * @param containerName - Name of the container.
+ */
+ void incrReadCount(String containerName);
+
+ /**
+ * Increse the read counter for bytes read from the container.
+ * @param containerName - Name of the container.
+ * @param readBytes - bytes read from the container.
+ */
+ void incrReadBytes(String containerName, long readBytes);
+
+
+ /**
+ * Increase the write count of the container.
+ * @param containerName - Name of the container.
+ */
+ void incrWriteCount(String containerName);
+
+ /**
+ * Increase the write counter for bytes write into the container.
+ * @param containerName - Name of the container.
+ * @param writeBytes - bytes write into the container.
+ */
+ void incrWriteBytes(String containerName, long writeBytes);
+
+ /**
+ * Increase the bytes used by the container.
+ * @param containerName - Name of the container.
+ * @param used - additional bytes used by the container.
+ * @return the current bytes used.
+ */
+ long incrBytesUsed(String containerName, long used);
+
+ /**
+ * Decrease the bytes used by the container.
+ * @param containerName - Name of the container.
+ * @param used - additional bytes reclaimed by the container.
+ * @return the current bytes used.
+ */
+ long decrBytesUsed(String containerName, long used);
+
+ /**
+ * Get the bytes used by the container.
+ * @param containerName - Name of the container.
+ * @return the current bytes used by the container.
+ */
+ long getBytesUsed(String containerName);
+
+ /**
+ * Get the number of keys in the container.
+ * @param containerName - Name of the container.
+ * @return the current key count.
+ */
+ long getNumKeys(String containerName);
+
+ /**
+ * Get the container report state to send via HB to SCM.
+ * @return container report state.
+ */
+ ReportState getContainerReportState();
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerReportManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerReportManager.java
new file mode 100644
index 0000000000..4689dfe9e1
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerReportManager.java
@@ -0,0 +1,32 @@
+/*
+ * 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.protocol.proto.StorageContainerDatanodeProtocolProtos.ReportState;
+
+/**
+ * Interface for container report manager operations.
+ */
+public interface ContainerReportManager {
+
+ /**
+ * Get the container report state.
+ * @return the container report state.
+ */
+ ReportState getContainerReportState();
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java
new file mode 100644
index 0000000000..8c27ba94c4
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java
@@ -0,0 +1,76 @@
+/*
+ * 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.Pipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+ .StorageContainerException;
+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 pipeline - Pipeline.
+ * @param data - Key Data.
+ * @throws IOException
+ */
+ void putKey(Pipeline pipeline, 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 pipeline - Pipeline.
+ * @param keyName Key Data.
+ * @throws StorageContainerException
+ */
+ void deleteKey(Pipeline pipeline, String keyName)
+ throws IOException;
+
+ /**
+ * List keys in a container.
+ *
+ * @param pipeline - pipeline.
+ * @param prefix - Prefix in needed.
+ * @param startKey - Key to start from, EMPTY_STRING to begin.
+ * @param count - Number of keys to return.
+ * @return List of Keys that match the criteria.
+ */
+ List<KeyData> listKey(Pipeline pipeline, String prefix, String startKey,
+ int count) throws IOException;
+
+ /**
+ * Shutdown keyManager.
+ */
+ void shutdown();
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/package-info.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/package-info.java
new file mode 100644
index 0000000000..d83bf95c36
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/package-info.java
@@ -0,0 +1,20 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.interfaces;
+/**
+ This package contains common ozone container interfaces.
+ */ \ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/package-info.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/package-info.java
new file mode 100644
index 0000000000..1638a36a13
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/package-info.java
@@ -0,0 +1,28 @@
+/**
+ * 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;
+/**
+ Common Container Layer. At this layer the abstractions are:
+
+ 1. Containers - Both data and metadata containers.
+ 2. Keys - Key/Value pairs that live inside a container.
+ 3. Chunks - Keys can be composed of many chunks.
+
+ Ozone uses these abstractions to build Volumes, Buckets and Keys.
+
+ **/ \ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
new file mode 100644
index 0000000000..8e9482f565
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
@@ -0,0 +1,387 @@
+/**
+ * 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.statemachine;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
+ .CloseContainerHandler;
+import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
+ .CommandDispatcher;
+import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
+ .ContainerReportHandler;
+import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
+ .DeleteBlocksCommandHandler;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.hdds.scm.HddsServerUtil.getScmHeartbeatInterval;
+
+/**
+ * State Machine Class.
+ */
+public class DatanodeStateMachine implements Closeable {
+ @VisibleForTesting
+ static final Logger LOG =
+ LoggerFactory.getLogger(DatanodeStateMachine.class);
+ private final ExecutorService executorService;
+ private final Configuration conf;
+ private final SCMConnectionManager connectionManager;
+ private final long heartbeatFrequency;
+ private StateContext context;
+ private final OzoneContainer container;
+ private DatanodeDetails datanodeDetails;
+ private final CommandDispatcher commandDispatcher;
+ private long commandsHandled;
+ private AtomicLong nextHB;
+ private Thread stateMachineThread = null;
+ private Thread cmdProcessThread = null;
+
+ /**
+ * Constructs a a datanode state machine.
+ *
+ * @param datanodeDetails - DatanodeDetails used to identify a datanode
+ * @param conf - Configuration.
+ */
+ public DatanodeStateMachine(DatanodeDetails datanodeDetails,
+ Configuration conf) throws IOException {
+ this.conf = conf;
+ this.datanodeDetails = datanodeDetails;
+ executorService = HadoopExecutors.newCachedThreadPool(
+ new ThreadFactoryBuilder().setDaemon(true)
+ .setNameFormat("Datanode State Machine Thread - %d").build());
+ connectionManager = new SCMConnectionManager(conf);
+ context = new StateContext(this.conf, DatanodeStates.getInitState(), this);
+ heartbeatFrequency = TimeUnit.SECONDS.toMillis(
+ getScmHeartbeatInterval(conf));
+ container = new OzoneContainer(this.datanodeDetails,
+ new OzoneConfiguration(conf));
+ nextHB = new AtomicLong(Time.monotonicNow());
+
+ // When we add new handlers just adding a new handler here should do the
+ // trick.
+ commandDispatcher = CommandDispatcher.newBuilder()
+ .addHandler(new ContainerReportHandler())
+ .addHandler(new CloseContainerHandler())
+ .addHandler(new DeleteBlocksCommandHandler(
+ container.getContainerManager(), conf))
+ .setConnectionManager(connectionManager)
+ .setContainer(container)
+ .setContext(context)
+ .build();
+ }
+
+ /**
+ *
+ * Return DatanodeDetails if set, return null otherwise.
+ *
+ * @return DatanodeDetails
+ */
+ public DatanodeDetails getDatanodeDetails() {
+ return datanodeDetails;
+ }
+
+
+ /**
+ * Returns the Connection manager for this state machine.
+ *
+ * @return - SCMConnectionManager.
+ */
+ public SCMConnectionManager getConnectionManager() {
+ return connectionManager;
+ }
+
+ public OzoneContainer getContainer() {
+ return this.container;
+ }
+
+ /**
+ * Runs the state machine at a fixed frequency.
+ */
+ private void start() throws IOException {
+ long now = 0;
+
+ container.start();
+ initCommandHandlerThread(conf);
+ while (context.getState() != DatanodeStates.SHUTDOWN) {
+ try {
+ LOG.debug("Executing cycle Number : {}", context.getExecutionCount());
+ nextHB.set(Time.monotonicNow() + heartbeatFrequency);
+ context.setReportState(container.getNodeReport());
+ context.setContainerReportState(container.getContainerReportState());
+ context.execute(executorService, heartbeatFrequency,
+ TimeUnit.MILLISECONDS);
+ now = Time.monotonicNow();
+ if (now < nextHB.get()) {
+ Thread.sleep(nextHB.get() - now);
+ }
+ } catch (InterruptedException e) {
+ // Ignore this exception.
+ } catch (Exception e) {
+ LOG.error("Unable to finish the execution.", e);
+ }
+ }
+ }
+
+ /**
+ * Gets the current context.
+ *
+ * @return StateContext
+ */
+ public StateContext getContext() {
+ return context;
+ }
+
+ /**
+ * Sets the current context.
+ *
+ * @param context - Context
+ */
+ public void setContext(StateContext context) {
+ this.context = context;
+ }
+
+ /**
+ * Closes this stream and releases any system resources associated with it. If
+ * the stream is already closed then invoking this method has no effect.
+ * <p>
+ * <p> As noted in {@link AutoCloseable#close()}, cases where the close may
+ * fail require careful attention. It is strongly advised to relinquish the
+ * underlying resources and to internally <em>mark</em> the {@code Closeable}
+ * as closed, prior to throwing the {@code IOException}.
+ *
+ * @throws IOException if an I/O error occurs
+ */
+ @Override
+ public void close() throws IOException {
+ if (stateMachineThread != null) {
+ stateMachineThread.interrupt();
+ }
+ if (cmdProcessThread != null) {
+ cmdProcessThread.interrupt();
+ }
+ context.setState(DatanodeStates.getLastState());
+ executorService.shutdown();
+ try {
+ if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) {
+ executorService.shutdownNow();
+ }
+
+ if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) {
+ LOG.error("Unable to shutdown state machine properly.");
+ }
+ } catch (InterruptedException e) {
+ LOG.error("Error attempting to shutdown.", e);
+ executorService.shutdownNow();
+ Thread.currentThread().interrupt();
+ }
+
+ if (connectionManager != null) {
+ connectionManager.close();
+ }
+
+ if(container != null) {
+ container.stop();
+ }
+ }
+
+ /**
+ * States that a datanode can be in. GetNextState will move this enum from
+ * getInitState to getLastState.
+ */
+ public enum DatanodeStates {
+ INIT(1),
+ RUNNING(2),
+ SHUTDOWN(3);
+ private final int value;
+
+ /**
+ * Constructs states.
+ *
+ * @param value Enum Value
+ */
+ DatanodeStates(int value) {
+ this.value = value;
+ }
+
+ /**
+ * Returns the first State.
+ *
+ * @return First State.
+ */
+ public static DatanodeStates getInitState() {
+ return INIT;
+ }
+
+ /**
+ * The last state of endpoint states.
+ *
+ * @return last state.
+ */
+ public static DatanodeStates getLastState() {
+ return SHUTDOWN;
+ }
+
+ /**
+ * returns the numeric value associated with the endPoint.
+ *
+ * @return int.
+ */
+ public int getValue() {
+ return value;
+ }
+
+ /**
+ * Returns the next logical state that endPoint should move to. This
+ * function assumes the States are sequentially numbered.
+ *
+ * @return NextState.
+ */
+ public DatanodeStates getNextState() {
+ if (this.value < getLastState().getValue()) {
+ int stateValue = this.getValue() + 1;
+ for (DatanodeStates iter : values()) {
+ if (stateValue == iter.getValue()) {
+ return iter;
+ }
+ }
+ }
+ return getLastState();
+ }
+ }
+
+ /**
+ * Start datanode state machine as a single thread daemon.
+ */
+ public void startDaemon() {
+ Runnable startStateMachineTask = () -> {
+ try {
+ start();
+ LOG.info("Ozone container server started.");
+ } catch (Exception ex) {
+ LOG.error("Unable to start the DatanodeState Machine", ex);
+ }
+ };
+ stateMachineThread = new ThreadFactoryBuilder()
+ .setDaemon(true)
+ .setNameFormat("Datanode State Machine Thread - %d")
+ .build().newThread(startStateMachineTask);
+ stateMachineThread.start();
+ }
+
+ /**
+ * Stop the daemon thread of the datanode state machine.
+ */
+ public synchronized void stopDaemon() {
+ try {
+ context.setState(DatanodeStates.SHUTDOWN);
+ this.close();
+ LOG.info("Ozone container server stopped.");
+ } catch (IOException e) {
+ LOG.error("Stop ozone container server failed.", e);
+ }
+ }
+
+ /**
+ *
+ * Check if the datanode state machine daemon is stopped.
+ *
+ * @return True if datanode state machine daemon is stopped
+ * and false otherwise.
+ */
+ @VisibleForTesting
+ public boolean isDaemonStopped() {
+ return this.executorService.isShutdown()
+ && this.getContext().getExecutionCount() == 0
+ && this.getContext().getState() == DatanodeStates.SHUTDOWN;
+ }
+
+ /**
+ * Create a command handler thread.
+ *
+ * @param config
+ */
+ private void initCommandHandlerThread(Configuration config) {
+
+ /**
+ * Task that periodically checks if we have any outstanding commands.
+ * It is assumed that commands can be processed slowly and in order.
+ * This assumption might change in future. Right now due to this assumption
+ * we have single command queue process thread.
+ */
+ Runnable processCommandQueue = () -> {
+ long now;
+ while (getContext().getState() != DatanodeStates.SHUTDOWN) {
+ SCMCommand command = getContext().getNextCommand();
+ if (command != null) {
+ commandDispatcher.handle(command);
+ commandsHandled++;
+ } else {
+ try {
+ // Sleep till the next HB + 1 second.
+ now = Time.monotonicNow();
+ if (nextHB.get() > now) {
+ Thread.sleep((nextHB.get() - now) + 1000L);
+ }
+ } catch (InterruptedException e) {
+ // Ignore this exception.
+ }
+ }
+ }
+ };
+
+ // We will have only one thread for command processing in a datanode.
+ cmdProcessThread = getCommandHandlerThread(processCommandQueue);
+ cmdProcessThread.start();
+ }
+
+ private Thread getCommandHandlerThread(Runnable processCommandQueue) {
+ Thread handlerThread = new Thread(processCommandQueue);
+ handlerThread.setDaemon(true);
+ handlerThread.setName("Command processor thread");
+ handlerThread.setUncaughtExceptionHandler((Thread t, Throwable e) -> {
+ // Let us just restart this thread after logging a critical error.
+ // if this thread is not running we cannot handle commands from SCM.
+ LOG.error("Critical Error : Command processor thread encountered an " +
+ "error. Thread: {}", t.toString(), e);
+ getCommandHandlerThread(processCommandQueue).start();
+ });
+ return handlerThread;
+ }
+
+ /**
+ * Returns the number of commands handled by the datanode.
+ * @return count
+ */
+ @VisibleForTesting
+ public long getCommandHandled() {
+ return commandsHandled;
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
new file mode 100644
index 0000000000..7e85923d31
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
@@ -0,0 +1,294 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.statemachine;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ozone.protocol.VersionResponse;
+import org.apache.hadoop.ozone.protocolPB
+ .StorageContainerDatanodeProtocolClientSideTranslatorPB;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.time.ZonedDateTime;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static org.apache.hadoop.hdds.scm.HddsServerUtil.getLogWarnInterval;
+import static org.apache.hadoop.hdds.scm.HddsServerUtil.getScmHeartbeatInterval;
+
+/**
+ * Endpoint is used as holder class that keeps state around the RPC endpoint.
+ */
+public class EndpointStateMachine
+ implements Closeable, EndpointStateMachineMBean {
+ static final Logger
+ LOG = LoggerFactory.getLogger(EndpointStateMachine.class);
+ private final StorageContainerDatanodeProtocolClientSideTranslatorPB endPoint;
+ private final AtomicLong missedCount;
+ private final InetSocketAddress address;
+ private final Lock lock;
+ private final Configuration conf;
+ private EndPointStates state;
+ private VersionResponse version;
+ private ZonedDateTime lastSuccessfulHeartbeat;
+
+ /**
+ * Constructs RPC Endpoints.
+ *
+ * @param endPoint - RPC endPoint.
+ */
+ public EndpointStateMachine(InetSocketAddress address,
+ StorageContainerDatanodeProtocolClientSideTranslatorPB endPoint,
+ Configuration conf) {
+ this.endPoint = endPoint;
+ this.missedCount = new AtomicLong(0);
+ this.address = address;
+ state = EndPointStates.getInitState();
+ lock = new ReentrantLock();
+ this.conf = conf;
+ }
+
+ /**
+ * Takes a lock on this EndPoint so that other threads don't use this while we
+ * are trying to communicate via this endpoint.
+ */
+ public void lock() {
+ lock.lock();
+ }
+
+ /**
+ * Unlocks this endpoint.
+ */
+ public void unlock() {
+ lock.unlock();
+ }
+
+ /**
+ * Returns the version that we read from the server if anyone asks .
+ *
+ * @return - Version Response.
+ */
+ public VersionResponse getVersion() {
+ return version;
+ }
+
+ /**
+ * Sets the Version reponse we recieved from the SCM.
+ *
+ * @param version VersionResponse
+ */
+ public void setVersion(VersionResponse version) {
+ this.version = version;
+ }
+
+ /**
+ * Returns the current State this end point is in.
+ *
+ * @return - getState.
+ */
+ public EndPointStates getState() {
+ return state;
+ }
+
+ @Override
+ public int getVersionNumber() {
+ if (version != null) {
+ return version.getProtobufMessage().getSoftwareVersion();
+ } else {
+ return -1;
+ }
+ }
+
+ /**
+ * Sets the endpoint state.
+ *
+ * @param epState - end point state.
+ */
+ public EndPointStates setState(EndPointStates epState) {
+ this.state = epState;
+ return this.state;
+ }
+
+ /**
+ * Closes the connection.
+ *
+ * @throws IOException
+ */
+ @Override
+ public void close() throws IOException {
+ if (endPoint != null) {
+ endPoint.close();
+ }
+ }
+
+ /**
+ * We maintain a count of how many times we missed communicating with a
+ * specific SCM. This is not made atomic since the access to this is always
+ * guarded by the read or write lock. That is, it is serialized.
+ */
+ public void incMissed() {
+ this.missedCount.incrementAndGet();
+ }
+
+ /**
+ * Returns the value of the missed count.
+ *
+ * @return int
+ */
+ public long getMissedCount() {
+ return this.missedCount.get();
+ }
+
+ @Override
+ public String getAddressString() {
+ return getAddress().toString();
+ }
+
+ public void zeroMissedCount() {
+ this.missedCount.set(0);
+ }
+
+ /**
+ * Returns the InetAddress of the endPoint.
+ *
+ * @return - EndPoint.
+ */
+ public InetSocketAddress getAddress() {
+ return this.address;
+ }
+
+ /**
+ * Returns real RPC endPoint.
+ *
+ * @return rpc client.
+ */
+ public StorageContainerDatanodeProtocolClientSideTranslatorPB
+ getEndPoint() {
+ return endPoint;
+ }
+
+ /**
+ * Returns the string that represents this endpoint.
+ *
+ * @return - String
+ */
+ public String toString() {
+ return address.toString();
+ }
+
+ /**
+ * Logs exception if needed.
+ * @param ex - Exception
+ */
+ public void logIfNeeded(Exception ex) {
+ LOG.trace("Incrementing the Missed count. Ex : {}", ex);
+ this.incMissed();
+ if (this.getMissedCount() % getLogWarnInterval(conf) ==
+ 0) {
+ LOG.warn("Unable to communicate to SCM server at {}. We have not been " +
+ "able to communicate to this SCM server for past {} seconds.",
+ this.getAddress().getHostString() + ":" + this.getAddress().getPort(),
+ this.getMissedCount() * getScmHeartbeatInterval(
+ this.conf));
+ }
+ }
+
+
+ /**
+ * States that an Endpoint can be in.
+ * <p>
+ * This is a sorted list of states that EndPoint will traverse.
+ * <p>
+ * GetNextState will move this enum from getInitState to getLastState.
+ */
+ public enum EndPointStates {
+ GETVERSION(1),
+ REGISTER(2),
+ HEARTBEAT(3),
+ SHUTDOWN(4); // if you add value after this please edit getLastState too.
+ private final int value;
+
+ /**
+ * Constructs endPointStates.
+ *
+ * @param value state.
+ */
+ EndPointStates(int value) {
+ this.value = value;
+ }
+
+ /**
+ * Returns the first State.
+ *
+ * @return First State.
+ */
+ public static EndPointStates getInitState() {
+ return GETVERSION;
+ }
+
+ /**
+ * The last state of endpoint states.
+ *
+ * @return last state.
+ */
+ public static EndPointStates getLastState() {
+ return SHUTDOWN;
+ }
+
+ /**
+ * returns the numeric value associated with the endPoint.
+ *
+ * @return int.
+ */
+ public int getValue() {
+ return value;
+ }
+
+ /**
+ * Returns the next logical state that endPoint should move to.
+ * The next state is computed by adding 1 to the current state.
+ *
+ * @return NextState.
+ */
+ public EndPointStates getNextState() {
+ if (this.getValue() < getLastState().getValue()) {
+ int stateValue = this.getValue() + 1;
+ for (EndPointStates iter : values()) {
+ if (stateValue == iter.getValue()) {
+ return iter;
+ }
+ }
+ }
+ return getLastState();
+ }
+ }
+
+ public long getLastSuccessfulHeartbeat() {
+ return lastSuccessfulHeartbeat == null ?
+ 0 :
+ lastSuccessfulHeartbeat.toEpochSecond();
+ }
+
+ public void setLastSuccessfulHeartbeat(
+ ZonedDateTime lastSuccessfulHeartbeat) {
+ this.lastSuccessfulHeartbeat = lastSuccessfulHeartbeat;
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachineMBean.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachineMBean.java
new file mode 100644
index 0000000000..4f64bde0b3
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachineMBean.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.statemachine;
+
+
+/**
+ * JMX representation of an EndpointStateMachine.
+ */
+public interface EndpointStateMachineMBean {
+
+ long getMissedCount();
+
+ String getAddressString();
+
+ EndpointStateMachine.EndPointStates getState();
+
+ int getVersionNumber();
+
+ long getLastSuccessfulHeartbeat();
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
new file mode 100644
index 0000000000..19722f04a5
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
@@ -0,0 +1,208 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.statemachine;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.ozone.protocolPB
+ .StorageContainerDatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolPB;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.management.ObjectName;
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static org.apache.hadoop.hdds.scm.HddsServerUtil
+ .getScmRpcTimeOutInMilliseconds;
+
+/**
+ * SCMConnectionManager - Acts as a class that manages the membership
+ * information of the SCMs that we are working with.
+ */
+public class SCMConnectionManager
+ implements Closeable, SCMConnectionManagerMXBean {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(SCMConnectionManager.class);
+
+ private final ReadWriteLock mapLock;
+ private final Map<InetSocketAddress, EndpointStateMachine> scmMachines;
+
+ private final int rpcTimeout;
+ private final Configuration conf;
+ private final ObjectName jmxBean;
+
+ public SCMConnectionManager(Configuration conf) {
+ this.mapLock = new ReentrantReadWriteLock();
+ Long timeOut = getScmRpcTimeOutInMilliseconds(conf);
+ this.rpcTimeout = timeOut.intValue();
+ this.scmMachines = new HashMap<>();
+ this.conf = conf;
+ jmxBean = MBeans.register("OzoneDataNode",
+ "SCMConnectionManager",
+ this);
+ }
+
+
+ /**
+ * Returns Config.
+ *
+ * @return ozoneConfig.
+ */
+ public Configuration getConf() {
+ return conf;
+ }
+
+ /**
+ * Get RpcTimeout.
+ *
+ * @return - Return RPC timeout.
+ */
+ public int getRpcTimeout() {
+ return rpcTimeout;
+ }
+
+
+ /**
+ * Takes a read lock.
+ */
+ public void readLock() {
+ this.mapLock.readLock().lock();
+ }
+
+ /**
+ * Releases the read lock.
+ */
+ public void readUnlock() {
+ this.mapLock.readLock().unlock();
+ }
+
+ /**
+ * Takes the write lock.
+ */
+ public void writeLock() {
+ this.mapLock.writeLock().lock();
+ }
+
+ /**
+ * Releases the write lock.
+ */
+ public void writeUnlock() {
+ this.mapLock.writeLock().unlock();
+ }
+
+ /**
+ * adds a new SCM machine to the target set.
+ *
+ * @param address - Address of the SCM machine to send heatbeat to.
+ * @throws IOException
+ */
+ public void addSCMServer(InetSocketAddress address) throws IOException {
+ writeLock();
+ try {
+ if (scmMachines.containsKey(address)) {
+ LOG.warn("Trying to add an existing SCM Machine to Machines group. " +
+ "Ignoring the request.");
+ return;
+ }
+ RPC.setProtocolEngine(conf, StorageContainerDatanodeProtocolPB.class,
+ ProtobufRpcEngine.class);
+ long version =
+ RPC.getProtocolVersion(StorageContainerDatanodeProtocolPB.class);
+
+ StorageContainerDatanodeProtocolPB rpcProxy = RPC.getProxy(
+ StorageContainerDatanodeProtocolPB.class, version,
+ address, UserGroupInformation.getCurrentUser(), conf,
+ NetUtils.getDefaultSocketFactory(conf), getRpcTimeout());
+
+ StorageContainerDatanodeProtocolClientSideTranslatorPB rpcClient =
+ new StorageContainerDatanodeProtocolClientSideTranslatorPB(rpcProxy);
+
+ EndpointStateMachine endPoint =
+ new EndpointStateMachine(address, rpcClient, conf);
+ scmMachines.put(address, endPoint);
+ } finally {
+ writeUnlock();
+ }
+ }
+
+ /**
+ * Removes a SCM machine for the target set.
+ *
+ * @param address - Address of the SCM machine to send heatbeat to.
+ * @throws IOException
+ */
+ public void removeSCMServer(InetSocketAddress address) throws IOException {
+ writeLock();
+ try {
+ if (!scmMachines.containsKey(address)) {
+ LOG.warn("Trying to remove a non-existent SCM machine. " +
+ "Ignoring the request.");
+ return;
+ }
+
+ EndpointStateMachine endPoint = scmMachines.get(address);
+ endPoint.close();
+ scmMachines.remove(address);
+ } finally {
+ writeUnlock();
+ }
+ }
+
+ /**
+ * Returns all known RPCEndpoints.
+ *
+ * @return - List of RPC Endpoints.
+ */
+ public Collection<EndpointStateMachine> getValues() {
+ return scmMachines.values();
+ }
+
+ @Override
+ public void close() throws IOException {
+ getValues().forEach(endpointStateMachine
+ -> IOUtils.cleanupWithLogger(LOG, endpointStateMachine));
+ MBeans.unregister(jmxBean);
+ }
+
+ @Override
+ public List<EndpointStateMachineMBean> getSCMServers() {
+ readLock();
+ try {
+ return Collections
+ .unmodifiableList(new ArrayList<>(scmMachines.values()));
+
+ } finally {
+ readUnlock();
+ }
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManagerMXBean.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManagerMXBean.java
new file mode 100644
index 0000000000..25ef16379a
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManagerMXBean.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.statemachine;
+
+import java.util.List;
+
+/**
+ * JMX information about the connected SCM servers.
+ */
+public interface SCMConnectionManagerMXBean {
+
+ List<EndpointStateMachineMBean> getSCMServers();
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
new file mode 100644
index 0000000000..55476fd41e
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
@@ -0,0 +1,285 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.statemachine;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.ReportState;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+import org.apache.hadoop.ozone.container.common.states.DatanodeState;
+import org.apache.hadoop.ozone.container.common.states.datanode
+ .InitDatanodeState;
+import org.apache.hadoop.ozone.container.common.states.datanode
+ .RunningDatanodeState;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.ReportState.states
+ .noContainerReports;
+import static org.apache.hadoop.ozone.OzoneConsts.INVALID_PORT;
+
+/**
+ * Current Context of State Machine.
+ */
+public class StateContext {
+ static final Logger LOG =
+ LoggerFactory.getLogger(StateContext.class);
+ private final Queue<SCMCommand> commandQueue;
+ private final Lock lock;
+ private final DatanodeStateMachine parent;
+ private final AtomicLong stateExecutionCount;
+ private final Configuration conf;
+ private DatanodeStateMachine.DatanodeStates state;
+ private SCMNodeReport nrState;
+ private ReportState reportState;
+ private static final ReportState DEFAULT_REPORT_STATE =
+ ReportState.newBuilder().setState(noContainerReports).setCount(0).build();
+
+ /**
+ * Constructs a StateContext.
+ *
+ * @param conf - Configration
+ * @param state - State
+ * @param parent Parent State Machine
+ */
+ public StateContext(Configuration conf, DatanodeStateMachine.DatanodeStates
+ state, DatanodeStateMachine parent) {
+ this.conf = conf;
+ this.state = state;
+ this.parent = parent;
+ commandQueue = new LinkedList<>();
+ lock = new ReentrantLock();
+ stateExecutionCount = new AtomicLong(0);
+ nrState = SCMNodeReport.getDefaultInstance();
+ }
+
+ /**
+ * Returns the ContainerStateMachine class that holds this state.
+ *
+ * @return ContainerStateMachine.
+ */
+ public DatanodeStateMachine getParent() {
+ return parent;
+ }
+
+ /**
+ * Get the container server port.
+ * @return The container server port if available, return -1 if otherwise
+ */
+ public int getContainerPort() {
+ return parent == null ?
+ INVALID_PORT : parent.getContainer().getContainerServerPort();
+ }
+
+ /**
+ * Gets the Ratis Port.
+ * @return int , return -1 if not valid.
+ */
+ public int getRatisPort() {
+ return parent == null ?
+ INVALID_PORT : parent.getContainer().getRatisContainerServerPort();
+ }
+
+ /**
+ * Returns true if we are entering a new state.
+ *
+ * @return boolean
+ */
+ boolean isEntering() {
+ return stateExecutionCount.get() == 0;
+ }
+
+ /**
+ * Returns true if we are exiting from the current state.
+ *
+ * @param newState - newState.
+ * @return boolean
+ */
+ boolean isExiting(DatanodeStateMachine.DatanodeStates newState) {
+ boolean isExiting = state != newState && stateExecutionCount.get() > 0;
+ if(isExiting) {
+ stateExecutionCount.set(0);
+ }
+ return isExiting;
+ }
+
+ /**
+ * Returns the current state the machine is in.
+ *
+ * @return state.
+ */
+ public DatanodeStateMachine.DatanodeStates getState() {
+ return state;
+ }
+
+ /**
+ * Sets the current state of the machine.
+ *
+ * @param state state.
+ */
+ public void setState(DatanodeStateMachine.DatanodeStates state) {
+ this.state = state;
+ }
+
+ /**
+ * Returns the node report of the datanode state context.
+ * @return the node report.
+ */
+ public SCMNodeReport getNodeReport() {
+ return nrState;
+ }
+
+ /**
+ * Sets the storage location report of the datanode state context.
+ * @param nrReport - node report
+ */
+ public void setReportState(SCMNodeReport nrReport) {
+ this.nrState = nrReport;
+ }
+
+ /**
+ * Returns the next task to get executed by the datanode state machine.
+ * @return A callable that will be executed by the
+ * {@link DatanodeStateMachine}
+ */
+ @SuppressWarnings("unchecked")
+ public DatanodeState<DatanodeStateMachine.DatanodeStates> getTask() {
+ switch (this.state) {
+ case INIT:
+ return new InitDatanodeState(this.conf, parent.getConnectionManager(),
+ this);
+ case RUNNING:
+ return new RunningDatanodeState(this.conf, parent.getConnectionManager(),
+ this);
+ case SHUTDOWN:
+ return null;
+ default:
+ throw new IllegalArgumentException("Not Implemented yet.");
+ }
+ }
+
+ /**
+ * Executes the required state function.
+ *
+ * @param service - Executor Service
+ * @param time - seconds to wait
+ * @param unit - Seconds.
+ * @throws InterruptedException
+ * @throws ExecutionException
+ * @throws TimeoutException
+ */
+ public void execute(ExecutorService service, long time, TimeUnit unit)
+ throws InterruptedException, ExecutionException, TimeoutException {
+ stateExecutionCount.incrementAndGet();
+ DatanodeState<DatanodeStateMachine.DatanodeStates> task = getTask();
+ if (this.isEntering()) {
+ task.onEnter();
+ }
+ task.execute(service);
+ DatanodeStateMachine.DatanodeStates newState = task.await(time, unit);
+ if (this.state != newState) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Task {} executed, state transited from {} to {}",
+ task.getClass().getSimpleName(), this.state, newState);
+ }
+ if (isExiting(newState)) {
+ task.onExit();
+ }
+ this.clearReportState();
+ this.setState(newState);
+ }
+ }
+
+ /**
+ * Returns the next command or null if it is empty.
+ *
+ * @return SCMCommand or Null.
+ */
+ public SCMCommand getNextCommand() {
+ lock.lock();
+ try {
+ return commandQueue.poll();
+ } finally {
+ lock.unlock();
+ }
+ }
+
+ /**
+ * Adds a command to the State Machine queue.
+ *
+ * @param command - SCMCommand.
+ */
+ public void addCommand(SCMCommand command) {
+ lock.lock();
+ try {
+ commandQueue.add(command);
+ } finally {
+ lock.unlock();
+ }
+ }
+
+ /**
+ * Returns the count of the Execution.
+ * @return long
+ */
+ public long getExecutionCount() {
+ return stateExecutionCount.get();
+ }
+
+
+ /**
+ * Gets the ReportState.
+ * @return ReportState.
+ */
+ public synchronized ReportState getContainerReportState() {
+ if (reportState == null) {
+ return DEFAULT_REPORT_STATE;
+ }
+ return reportState;
+ }
+
+ /**
+ * Sets the ReportState.
+ * @param rState - ReportState.
+ */
+ public synchronized void setContainerReportState(ReportState rState) {
+ this.reportState = rState;
+ }
+
+ /**
+ * Clears report state after it has been communicated.
+ */
+ public synchronized void clearReportState() {
+ if(reportState != null) {
+ setContainerReportState(null);
+ }
+ }
+
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java
new file mode 100644
index 0000000000..ac95b2a12c
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java
@@ -0,0 +1,239 @@
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.statemachine.background;
+
+import com.google.common.collect.Lists;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+ .StorageContainerException;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdds.protocol.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;
+import org.apache.hadoop.utils.BackgroundTaskQueue;
+import org.apache.hadoop.utils.BackgroundTaskResult;
+import org.apache.hadoop.utils.BatchOperation;
+import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
+import org.apache.hadoop.utils.MetadataStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys
+ .OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys
+ .OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConfigKeys
+ .OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER;
+import static org.apache.hadoop.ozone.OzoneConfigKeys
+ .OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER_DEFAULT;
+
+/**
+ * A per-datanode container block deleting service takes in charge
+ * of deleting staled ozone blocks.
+ */
+public class BlockDeletingService extends BackgroundService{
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(BlockDeletingService.class);
+
+ private final ContainerManager containerManager;
+ private final Configuration conf;
+
+ // Throttle number of blocks to delete per task,
+ // set to 1 for testing
+ private final int blockLimitPerTask;
+
+ // Throttle the number of containers to process concurrently at a time,
+ private final int containerLimitPerInterval;
+
+ // Task priority is useful when a to-delete block has weight.
+ private final static int TASK_PRIORITY_DEFAULT = 1;
+ // 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, Configuration conf) {
+ super("BlockDeletingService", serviceInterval,
+ TimeUnit.MILLISECONDS, BLOCK_DELETING_SERVICE_CORE_POOL_SIZE,
+ serviceTimeout);
+ this.containerManager = containerManager;
+ this.conf = conf;
+ this.blockLimitPerTask = conf.getInt(
+ OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER,
+ OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER_DEFAULT);
+ this.containerLimitPerInterval = conf.getInt(
+ OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL,
+ OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL_DEFAULT);
+ }
+
+ @Override
+ public BackgroundTaskQueue getTasks() {
+ BackgroundTaskQueue queue = new BackgroundTaskQueue();
+ List<ContainerData> containers = Lists.newArrayList();
+ try {
+ // We at most list a number of containers a time,
+ // in case there are too many containers and start too many workers.
+ // 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(
+ containerLimitPerInterval);
+ LOG.info("Plan to choose {} containers for block deletion, "
+ + "actually returns {} valid containers.",
+ containerLimitPerInterval, containers.size());
+
+ for(ContainerData container : containers) {
+ BlockDeletingTask containerTask =
+ new BlockDeletingTask(container, TASK_PRIORITY_DEFAULT);
+ queue.add(containerTask);
+ }
+ } catch (StorageContainerException e) {
+ LOG.warn("Failed to initiate block deleting tasks, "
+ + "caused by unable to get containers info. "
+ + "Retry in next interval. ", e);
+ } catch (Exception e) {
+ // In case listContainer call throws any uncaught RuntimeException.
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Unexpected error occurs during deleting blocks.", e);
+ }
+ }
+ return queue;
+ }
+
+ private static class ContainerBackgroundTaskResult
+ implements BackgroundTaskResult {
+ private List<String> deletedBlockIds;
+
+ ContainerBackgroundTaskResult() {
+ deletedBlockIds = new LinkedList<>();
+ }
+
+ public void addBlockId(String blockId) {
+ deletedBlockIds.add(blockId);
+ }
+
+ public void addAll(List<String> blockIds) {
+ deletedBlockIds.addAll(blockIds);
+ }
+
+ public List<String> getDeletedBlocks() {
+ return deletedBlockIds;
+ }
+
+ @Override
+ public int getSize() {
+ return deletedBlockIds.size();
+ }
+ }
+
+ private class BlockDeletingTask
+ implements BackgroundTask<BackgroundTaskResult> {
+
+ private final int priority;
+ private final ContainerData containerData;
+
+ BlockDeletingTask(ContainerData containerName, int priority) {
+ this.priority = priority;
+ this.containerData = containerName;
+ }
+
+ @Override
+ public BackgroundTaskResult call() throws Exception {
+ 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);
+ // # of blocks to delete is throttled
+ KeyPrefixFilter filter = new KeyPrefixFilter(
+ OzoneConsts.DELETING_KEY_PREFIX);
+ List<Map.Entry<byte[], byte[]>> toDeleteBlocks =
+ meta.getSequentialRangeKVs(null, blockLimitPerTask, filter);
+ if (toDeleteBlocks.isEmpty()) {
+ LOG.debug("No under deletion block found in container : {}",
+ containerData.getContainerName());
+ }
+
+ List<String> succeedBlocks = new LinkedList<>();
+ LOG.debug("Container : {}, To-Delete blocks : {}",
+ containerData.getContainerName(), toDeleteBlocks.size());
+ File dataDir = ContainerUtils.getDataDirectory(containerData).toFile();
+ if (!dataDir.exists() || !dataDir.isDirectory()) {
+ LOG.error("Invalid container data dir {} : "
+ + "not exist or not a directory", dataDir.getAbsolutePath());
+ return crr;
+ }
+
+ toDeleteBlocks.forEach(entry -> {
+ String blockName = DFSUtil.bytes2String(entry.getKey());
+ LOG.debug("Deleting block {}", blockName);
+ try {
+ ContainerProtos.KeyData data =
+ ContainerProtos.KeyData.parseFrom(entry.getValue());
+ for (ContainerProtos.ChunkInfo chunkInfo : data.getChunksList()) {
+ File chunkFile = dataDir.toPath()
+ .resolve(chunkInfo.getChunkName()).toFile();
+ if (FileUtils.deleteQuietly(chunkFile)) {
+ LOG.debug("block {} chunk {} deleted", blockName,
+ chunkFile.getAbsolutePath());
+ }
+ }
+ succeedBlocks.add(blockName);
+ } catch (InvalidProtocolBufferException e) {
+ LOG.error("Failed to parse block info for block {}", blockName, e);
+ }
+ });
+
+ // Once files are deleted ... clean up DB
+ BatchOperation batch = new BatchOperation();
+ succeedBlocks.forEach(entry ->
+ batch.delete(DFSUtil.string2Bytes(entry)));
+ meta.writeBatch(batch);
+ // update count of pending deletion blocks in in-memory container status
+ containerManager.decrPendingDeletionBlocks(succeedBlocks.size(),
+ containerData.getContainerName());
+
+ if (!succeedBlocks.isEmpty()) {
+ LOG.info("Container: {}, deleted blocks: {}, task elapsed time: {}ms",
+ containerData.getContainerName(), succeedBlocks.size(),
+ Time.monotonicNow() - startTime);
+ }
+ crr.addAll(succeedBlocks);
+ return crr;
+ }
+
+ @Override
+ public int getPriority() {
+ return priority;
+ }
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/package-info.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/package-info.java
new file mode 100644
index 0000000000..a9e202e35e
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * 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.statemachine.background; \ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java
new file mode 100644
index 0000000000..f7b49b7590
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
+
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.SCMCloseContainerCmdResponseProto;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.SCMCmdType;
+import org.apache.hadoop.ozone.container.common.statemachine
+ .SCMConnectionManager;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Container Report handler.
+ */
+public class CloseContainerHandler implements CommandHandler {
+ static final Logger LOG =
+ LoggerFactory.getLogger(CloseContainerHandler.class);
+ private int invocationCount;
+ private long totalTime;
+
+ /**
+ * Constructs a ContainerReport handler.
+ */
+ public CloseContainerHandler() {
+ }
+
+ /**
+ * Handles a given SCM command.
+ *
+ * @param command - SCM Command
+ * @param container - Ozone Container.
+ * @param context - Current Context.
+ * @param connectionManager - The SCMs that we are talking to.
+ */
+ @Override
+ public void handle(SCMCommand command, OzoneContainer container,
+ StateContext context, SCMConnectionManager connectionManager) {
+ LOG.debug("Processing Close Container command.");
+ invocationCount++;
+ long startTime = Time.monotonicNow();
+ String containerName = "UNKNOWN";
+ try {
+
+ SCMCloseContainerCmdResponseProto
+ closeContainerProto =
+ SCMCloseContainerCmdResponseProto
+ .parseFrom(command.getProtoBufMessage());
+ containerName = closeContainerProto.getContainerName();
+
+ container.getContainerManager().closeContainer(containerName);
+
+ } catch (Exception e) {
+ LOG.error("Can't close container " + containerName, e);
+ } finally {
+ long endTime = Time.monotonicNow();
+ totalTime += endTime - startTime;
+ }
+ }
+
+ /**
+ * Returns the command type that this command handler handles.
+ *
+ * @return Type
+ */
+ @Override
+ public SCMCmdType getCommandType() {
+ return SCMCmdType.closeContainerCommand;
+ }
+
+ /**
+ * Returns number of times this handler has been invoked.
+ *
+ * @return int
+ */
+ @Override
+ public int getInvocationCount() {
+ return invocationCount;
+ }
+
+ /**
+ * Returns the average time this function takes to run.
+ *
+ * @return long
+ */
+ @Override
+ public long getAverageRunTime() {
+ if (invocationCount > 0) {
+ return totalTime / invocationCount;
+ }
+ return 0;
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandDispatcher.java
new file mode 100644
index 0000000000..40feca32bd
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandDispatcher.java
@@ -0,0 +1,177 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType;
+import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Dispatches command to the correct handler.
+ */
+public final class CommandDispatcher {
+ static final Logger LOG =
+ LoggerFactory.getLogger(CommandDispatcher.class);
+ private final StateContext context;
+ private final Map<SCMCmdType, CommandHandler> handlerMap;
+ private final OzoneContainer container;
+ private final SCMConnectionManager connectionManager;
+
+ /**
+ * Constructs a command Dispatcher.
+ * @param context - Context.
+ */
+ /**
+ * Constructs a command dispatcher.
+ *
+ * @param container - Ozone Container
+ * @param context - Context
+ * @param handlers - Set of handlers.
+ */
+ private CommandDispatcher(OzoneContainer container, SCMConnectionManager
+ connectionManager, StateContext context,
+ CommandHandler... handlers) {
+ Preconditions.checkNotNull(context);
+ Preconditions.checkNotNull(handlers);
+ Preconditions.checkArgument(handlers.length > 0);
+ Preconditions.checkNotNull(container);
+ Preconditions.checkNotNull(connectionManager);
+ this.context = context;
+ this.container = container;
+ this.connectionManager = connectionManager;
+ handlerMap = new HashMap<>();
+ for (CommandHandler h : handlers) {
+ if(handlerMap.containsKey(h.getCommandType())){
+ LOG.error("Duplicate handler for the same command. Exiting. Handle " +
+ "key : { }", h.getCommandType().getDescriptorForType().getName());
+ throw new IllegalArgumentException("Duplicate handler for the same " +
+ "command.");
+ }
+ handlerMap.put(h.getCommandType(), h);
+ }
+ }
+
+ /**
+ * Dispatch the command to the correct handler.
+ *
+ * @param command - SCM Command.
+ */
+ public void handle(SCMCommand command) {
+ Preconditions.checkNotNull(command);
+ CommandHandler handler = handlerMap.get(command.getType());
+ if (handler != null) {
+ handler.handle(command, container, context, connectionManager);
+ } else {
+ LOG.error("Unknown SCM Command queued. There is no handler for this " +
+ "command. Command: {}", command.getType().getDescriptorForType()
+ .getName());
+ }
+ }
+
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ /**
+ * Helper class to construct command dispatcher.
+ */
+ public static class Builder {
+ private final List<CommandHandler> handlerList;
+ private OzoneContainer container;
+ private StateContext context;
+ private SCMConnectionManager connectionManager;
+
+ public Builder() {
+ handlerList = new LinkedList<>();
+ }
+
+ /**
+ * Adds a handler.
+ *
+ * @param handler - handler
+ * @return Builder
+ */
+ public Builder addHandler(CommandHandler handler) {
+ Preconditions.checkNotNull(handler);
+ handlerList.add(handler);
+ return this;
+ }
+
+ /**
+ * Add the OzoneContainer.
+ *
+ * @param ozoneContainer - ozone container.
+ * @return Builder
+ */
+ public Builder setContainer(OzoneContainer ozoneContainer) {
+ Preconditions.checkNotNull(ozoneContainer);
+ this.container = ozoneContainer;
+ return this;
+ }
+
+ /**
+ * Set the Connection Manager.
+ *
+ * @param scmConnectionManager
+ * @return this
+ */
+ public Builder setConnectionManager(SCMConnectionManager
+ scmConnectionManager) {
+ Preconditions.checkNotNull(scmConnectionManager);
+ this.connectionManager = scmConnectionManager;
+ return this;
+ }
+
+ /**
+ * Sets the Context.
+ *
+ * @param stateContext - StateContext
+ * @return this
+ */
+ public Builder setContext(StateContext stateContext) {
+ Preconditions.checkNotNull(stateContext);
+ this.context = stateContext;
+ return this;
+ }
+
+ /**
+ * Builds a command Dispatcher.
+ * @return Command Dispatcher.
+ */
+ public CommandDispatcher build() {
+ Preconditions.checkNotNull(this.connectionManager, "Missing connection" +
+ " manager.");
+ Preconditions.checkNotNull(this.container, "Missing container.");
+ Preconditions.checkNotNull(this.context, "Missing context.");
+ Preconditions.checkArgument(this.handlerList.size() > 0);
+ return new CommandDispatcher(this.container, this.connectionManager,
+ this.context, handlerList.toArray(
+ new CommandHandler[handlerList.size()]));
+ }
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandHandler.java
new file mode 100644
index 0000000000..13d9f7295d
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandHandler.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
+
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType;
+import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+
+/**
+ * Generic interface for handlers.
+ */
+public interface CommandHandler {
+
+ /**
+ * Handles a given SCM command.
+ * @param command - SCM Command
+ * @param container - Ozone Container.
+ * @param context - Current Context.
+ * @param connectionManager - The SCMs that we are talking to.
+ */
+ void handle(SCMCommand command, OzoneContainer container,
+ StateContext context, SCMConnectionManager connectionManager);
+
+ /**
+ * Returns the command type that this command handler handles.
+ * @return Type
+ */
+ SCMCmdType getCommandType();
+
+ /**
+ * Returns number of times this handler has been invoked.
+ * @return int
+ */
+ int getInvocationCount();
+
+ /**
+ * Returns the average time this function takes to run.
+ * @return long
+ */
+ long getAverageRunTime();
+
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ContainerReportHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ContainerReportHandler.java
new file mode 100644
index 0000000000..ba6b4185df
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ContainerReportHandler.java
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
+
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.SCMCmdType;
+import org.apache.hadoop.ozone.container.common.statemachine
+ .EndpointStateMachine;
+import org.apache.hadoop.ozone.container.common.statemachine
+ .SCMConnectionManager;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * Container Report handler.
+ */
+public class ContainerReportHandler implements CommandHandler {
+ static final Logger LOG =
+ LoggerFactory.getLogger(ContainerReportHandler.class);
+ private int invocationCount;
+ private long totalTime;
+
+ /**
+ * Constructs a ContainerReport handler.
+ */
+ public ContainerReportHandler() {
+ }
+
+ /**
+ * Handles a given SCM command.
+ *
+ * @param command - SCM Command
+ * @param container - Ozone Container.
+ * @param context - Current Context.
+ * @param connectionManager - The SCMs that we are talking to.
+ */
+ @Override
+ public void handle(SCMCommand command, OzoneContainer container,
+ StateContext context, SCMConnectionManager connectionManager) {
+ LOG.debug("Processing Container Report.");
+ invocationCount++;
+ long startTime = Time.monotonicNow();
+ try {
+ ContainerReportsRequestProto contianerReport =
+ container.getContainerReport();
+
+ // TODO : We send this report to all SCMs.Check if it is enough only to
+ // send to the leader once we have RAFT enabled SCMs.
+ for (EndpointStateMachine endPoint : connectionManager.getValues()) {
+ endPoint.getEndPoint().sendContainerReport(contianerReport);
+ }
+ } catch (IOException ex) {
+ LOG.error("Unable to process the Container Report command.", ex);
+ } finally {
+ long endTime = Time.monotonicNow();
+ totalTime += endTime - startTime;
+ }
+ }
+
+ /**
+ * Returns the command type that this command handler handles.
+ *
+ * @return Type
+ */
+ @Override
+ public SCMCmdType getCommandType() {
+ return SCMCmdType.sendContainerReport;
+ }
+
+ /**
+ * Returns number of times this handler has been invoked.
+ *
+ * @return int
+ */
+ @Override
+ public int getInvocationCount() {
+ return invocationCount;
+ }
+
+ /**
+ * Returns the average time this function takes to run.
+ *
+ * @return long
+ */
+ @Override
+ public long getAverageRunTime() {
+ if (invocationCount > 0) {
+ return totalTime / invocationCount;
+ }
+ return 0;
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
new file mode 100644
index 0000000000..f106e3d55f
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
@@ -0,0 +1,211 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto
+ .DeleteBlockTransactionResult;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.SCMCmdType;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
+import org.apache.hadoop.ozone.container.common.helpers
+ .DeletedContainerBlocksSummary;
+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.statemachine
+ .EndpointStateMachine;
+import org.apache.hadoop.ozone.container.common.statemachine
+ .SCMConnectionManager;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.utils.BatchOperation;
+import org.apache.hadoop.utils.MetadataStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Handle block deletion commands.
+ */
+public class DeleteBlocksCommandHandler implements CommandHandler {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(DeleteBlocksCommandHandler.class);
+
+ private ContainerManager containerManager;
+ private Configuration conf;
+ private int invocationCount;
+ private long totalTime;
+
+ public DeleteBlocksCommandHandler(ContainerManager containerManager,
+ Configuration conf) {
+ this.containerManager = containerManager;
+ this.conf = conf;
+ }
+
+ @Override
+ public void handle(SCMCommand command, OzoneContainer container,
+ StateContext context, SCMConnectionManager connectionManager) {
+ if (command.getType() != SCMCmdType.deleteBlocksCommand) {
+ LOG.warn("Skipping handling command, expected command "
+ + "type {} but found {}",
+ SCMCmdType.deleteBlocksCommand, command.getType());
+ return;
+ }
+ LOG.debug("Processing block deletion command.");
+ invocationCount++;
+ long startTime = Time.monotonicNow();
+
+ // move blocks to deleting state.
+ // this is a metadata update, the actual deletion happens in another
+ // recycling thread.
+ DeleteBlocksCommand cmd = (DeleteBlocksCommand) command;
+ List<DeletedBlocksTransaction> containerBlocks = cmd.blocksTobeDeleted();
+
+
+ DeletedContainerBlocksSummary summary =
+ DeletedContainerBlocksSummary.getFrom(containerBlocks);
+ LOG.info("Start to delete container blocks, TXIDs={}, "
+ + "numOfContainers={}, numOfBlocks={}",
+ summary.getTxIDSummary(),
+ summary.getNumOfContainers(),
+ summary.getNumOfBlocks());
+
+ ContainerBlocksDeletionACKProto.Builder resultBuilder =
+ ContainerBlocksDeletionACKProto.newBuilder();
+ containerBlocks.forEach(entry -> {
+ DeleteBlockTransactionResult.Builder txResultBuilder =
+ DeleteBlockTransactionResult.newBuilder();
+ txResultBuilder.setTxID(entry.getTxID());
+ try {
+ deleteContainerBlocks(entry, conf);
+ txResultBuilder.setSuccess(true);
+ } catch (IOException e) {
+ LOG.warn("Failed to delete blocks for container={}, TXID={}",
+ entry.getContainerName(), entry.getTxID(), e);
+ txResultBuilder.setSuccess(false);
+ }
+ resultBuilder.addResults(txResultBuilder.build());
+ });
+ ContainerBlocksDeletionACKProto blockDeletionACK = resultBuilder.build();
+
+ // Send ACK back to SCM as long as meta updated
+ // TODO Or we should wait until the blocks are actually deleted?
+ if (!containerBlocks.isEmpty()) {
+ for (EndpointStateMachine endPoint : connectionManager.getValues()) {
+ try {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Sending following block deletion ACK to SCM");
+ for (DeleteBlockTransactionResult result :
+ blockDeletionACK.getResultsList()) {
+ LOG.debug(result.getTxID() + " : " + result.getSuccess());
+ }
+ }
+ endPoint.getEndPoint()
+ .sendContainerBlocksDeletionACK(blockDeletionACK);
+ } catch (IOException e) {
+ LOG.error("Unable to send block deletion ACK to SCM {}",
+ endPoint.getAddress().toString(), e);
+ }
+ }
+ }
+
+ long endTime = Time.monotonicNow();
+ totalTime += endTime - startTime;
+ }
+
+ /**
+ * Move a bunch of blocks from a container to deleting state.
+ * This is a meta update, the actual deletes happen in async mode.
+ *
+ * @param delTX a block deletion transaction.
+ * @param config configuration.
+ * @throws IOException if I/O error occurs.
+ */
+ private void deleteContainerBlocks(DeletedBlocksTransaction delTX,
+ Configuration config) throws IOException {
+ String containerId = delTX.getContainerName();
+ ContainerData containerInfo = containerManager.readContainer(containerId);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Processing Container : {}, DB path : {}", containerId,
+ containerInfo.getDBPath());
+ }
+
+ int newDeletionBlocks = 0;
+ MetadataStore containerDB = KeyUtils.getDB(containerInfo, config);
+ for (String blk : delTX.getBlockIDList()) {
+ BatchOperation batch = new BatchOperation();
+ byte[] blkBytes = DFSUtil.string2Bytes(blk);
+ byte[] blkInfo = containerDB.get(blkBytes);
+ if (blkInfo != null) {
+ // Found the block in container db,
+ // use an atomic update to change its state to deleting.
+ batch.put(DFSUtil.string2Bytes(OzoneConsts.DELETING_KEY_PREFIX + blk),
+ blkInfo);
+ batch.delete(blkBytes);
+ try {
+ containerDB.writeBatch(batch);
+ newDeletionBlocks++;
+ LOG.debug("Transited Block {} to DELETING state in container {}",
+ blk, containerId);
+ } catch (IOException e) {
+ // if some blocks failed to delete, we fail this TX,
+ // without sending this ACK to SCM, SCM will resend the TX
+ // with a certain number of retries.
+ throw new IOException(
+ "Failed to delete blocks for TXID = " + delTX.getTxID(), e);
+ }
+ } else {
+ LOG.debug("Block {} not found or already under deletion in"
+ + " container {}, skip deleting it.", blk, containerId);
+ }
+ }
+
+ // update pending deletion blocks count in in-memory container status
+ containerManager.incrPendingDeletionBlocks(newDeletionBlocks, containerId);
+ }
+
+ @Override
+ public SCMCmdType getCommandType() {
+ return SCMCmdType.deleteBlocksCommand;
+ }
+
+ @Override
+ public int getInvocationCount() {
+ return this.invocationCount;
+ }
+
+ @Override
+ public long getAverageRunTime() {
+ if (invocationCount > 0) {
+ return totalTime / invocationCount;
+ }
+ return 0;
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/package-info.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/package-info.java
new file mode 100644
index 0000000000..1e9c8dc5ee
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * 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.statemachine.commandhandler; \ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/package-info.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/package-info.java
new file mode 100644
index 0000000000..feb2f812ac
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/package-info.java
@@ -0,0 +1,28 @@
+/**
+ * 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.statemachine;
+/**
+
+ State machine class is used by the container to denote various states a
+ container can be in and also is used for command processing.
+
+ Container has the following states.
+
+ Start - > getVersion -> Register -> Running -> Shutdown
+
+ */ \ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/DatanodeState.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/DatanodeState.java
new file mode 100644
index 0000000000..75142afd10
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/DatanodeState.java
@@ -0,0 +1,55 @@
+/*
+ * 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.states;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * State Interface that allows tasks to maintain states.
+ */
+public interface DatanodeState<T> {
+ /**
+ * Called before entering this state.
+ */
+ void onEnter();
+
+ /**
+ * Called After exiting this state.
+ */
+ void onExit();
+
+ /**
+ * Executes one or more tasks that is needed by this state.
+ *
+ * @param executor - ExecutorService
+ */
+ void execute(ExecutorService executor);
+
+ /**
+ * Wait for execute to finish.
+ *
+ * @param time - Time
+ * @param timeUnit - Unit of time.
+ */
+ T await(long time, TimeUnit timeUnit)
+ throws InterruptedException, ExecutionException, TimeoutException;
+
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java
new file mode 100644
index 0000000000..ac245d511c
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java
@@ -0,0 +1,157 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.states.datanode;
+
+import com.google.common.base.Strings;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.statemachine
+ .DatanodeStateMachine;
+import org.apache.hadoop.ozone.container.common.statemachine
+ .SCMConnectionManager;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.common.states.DatanodeState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.HddsUtils.getSCMAddresses;
+
+/**
+ * Init Datanode State is the task that gets run when we are in Init State.
+ */
+public class InitDatanodeState implements DatanodeState,
+ Callable<DatanodeStateMachine.DatanodeStates> {
+ static final Logger LOG = LoggerFactory.getLogger(InitDatanodeState.class);
+ private final SCMConnectionManager connectionManager;
+ private final Configuration conf;
+ private final StateContext context;
+ private Future<DatanodeStateMachine.DatanodeStates> result;
+
+ /**
+ * Create InitDatanodeState Task.
+ *
+ * @param conf - Conf
+ * @param connectionManager - Connection Manager
+ * @param context - Current Context
+ */
+ public InitDatanodeState(Configuration conf,
+ SCMConnectionManager connectionManager,
+ StateContext context) {
+ this.conf = conf;
+ this.connectionManager = connectionManager;
+ this.context = context;
+ }
+
+ /**
+ * Computes a result, or throws an exception if unable to do so.
+ *
+ * @return computed result
+ * @throws Exception if unable to compute a result
+ */
+ @Override
+ public DatanodeStateMachine.DatanodeStates call() throws Exception {
+ Collection<InetSocketAddress> addresses = null;
+ try {
+ addresses = getSCMAddresses(conf);
+ } catch (IllegalArgumentException e) {
+ if(!Strings.isNullOrEmpty(e.getMessage())) {
+ LOG.error("Failed to get SCM addresses: " + e.getMessage());
+ }
+ return DatanodeStateMachine.DatanodeStates.SHUTDOWN;
+ }
+
+ if (addresses == null || addresses.isEmpty()) {
+ LOG.error("Null or empty SCM address list found.");
+ return DatanodeStateMachine.DatanodeStates.SHUTDOWN;
+ } else {
+ for (InetSocketAddress addr : addresses) {
+ connectionManager.addSCMServer(addr);
+ }
+ }
+
+ // If datanode ID is set, persist it to the ID file.
+ persistContainerDatanodeDetails();
+
+ return this.context.getState().getNextState();
+ }
+
+ /**
+ * Persist DatanodeDetails to datanode.id file.
+ */
+ private void persistContainerDatanodeDetails() throws IOException {
+ String dataNodeIDPath = HddsUtils.getDatanodeIdFilePath(conf);
+ File idPath = new File(dataNodeIDPath);
+ DatanodeDetails datanodeDetails = this.context.getParent()
+ .getDatanodeDetails();
+ if (datanodeDetails != null && !idPath.exists()) {
+ ContainerUtils.writeDatanodeDetailsTo(datanodeDetails, idPath);
+ LOG.info("DatanodeDetails is persisted to {}", dataNodeIDPath);
+ }
+ }
+
+ /**
+ * Called before entering this state.
+ */
+ @Override
+ public void onEnter() {
+ LOG.trace("Entering init container state");
+ }
+
+ /**
+ * Called After exiting this state.
+ */
+ @Override
+ public void onExit() {
+ LOG.trace("Exiting init container state");
+ }
+
+ /**
+ * Executes one or more tasks that is needed by this state.
+ *
+ * @param executor - ExecutorService
+ */
+ @Override
+ public void execute(ExecutorService executor) {
+ result = executor.submit(this);
+ }
+
+ /**
+ * Wait for execute to finish.
+ *
+ * @param time - Time
+ * @param timeUnit - Unit of time.
+ */
+ @Override
+ public DatanodeStateMachine.DatanodeStates await(long time,
+ TimeUnit timeUnit) throws InterruptedException,
+ ExecutionException, TimeoutException {
+ return result.get(time, timeUnit);
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
new file mode 100644
index 0000000000..7a8c17b8c8
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
@@ -0,0 +1,175 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.states.datanode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
+import org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine;
+import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.common.states.DatanodeState;
+import org.apache.hadoop.ozone.container.common.states.endpoint.HeartbeatEndpointTask;
+import org.apache.hadoop.ozone.container.common.states.endpoint.RegisterEndpointTask;
+import org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Class that implements handshake with SCM.
+ */
+public class RunningDatanodeState implements DatanodeState {
+ static final Logger
+ LOG = LoggerFactory.getLogger(RunningDatanodeState.class);
+ private final SCMConnectionManager connectionManager;
+ private final Configuration conf;
+ private final StateContext context;
+ private CompletionService<EndpointStateMachine.EndPointStates> ecs;
+
+ public RunningDatanodeState(Configuration conf,
+ SCMConnectionManager connectionManager,
+ StateContext context) {
+ this.connectionManager = connectionManager;
+ this.conf = conf;
+ this.context = context;
+ }
+
+ /**
+ * Called before entering this state.
+ */
+ @Override
+ public void onEnter() {
+ LOG.trace("Entering handshake task.");
+ }
+
+ /**
+ * Called After exiting this state.
+ */
+ @Override
+ public void onExit() {
+ LOG.trace("Exiting handshake task.");
+ }
+
+ /**
+ * Executes one or more tasks that is needed by this state.
+ *
+ * @param executor - ExecutorService
+ */
+ @Override
+ public void execute(ExecutorService executor) {
+ ecs = new ExecutorCompletionService<>(executor);
+ for (EndpointStateMachine endpoint : connectionManager.getValues()) {
+ Callable<EndpointStateMachine.EndPointStates> endpointTask
+ = getEndPointTask(endpoint);
+ ecs.submit(endpointTask);
+ }
+ }
+ //TODO : Cache some of these tasks instead of creating them
+ //all the time.
+ private Callable<EndpointStateMachine.EndPointStates>
+ getEndPointTask(EndpointStateMachine endpoint) {
+ switch (endpoint.getState()) {
+ case GETVERSION:
+ return new VersionEndpointTask(endpoint, conf);
+ case REGISTER:
+ return RegisterEndpointTask.newBuilder()
+ .setConfig(conf)
+ .setEndpointStateMachine(endpoint)
+ .setDatanodeDetails(context.getParent().getDatanodeDetails())
+ .build();
+ case HEARTBEAT:
+ return HeartbeatEndpointTask.newBuilder()
+ .setConfig(conf)
+ .setEndpointStateMachine(endpoint)
+ .setDatanodeDetails(context.getParent().getDatanodeDetails())
+ .setContext(context)
+ .build();
+ case SHUTDOWN:
+ break;
+ default:
+ throw new IllegalArgumentException("Illegal Argument.");
+ }
+ return null;
+ }
+
+ /**
+ * Computes the next state the container state machine must move to by looking
+ * at all the state of endpoints.
+ * <p>
+ * if any endpoint state has moved to Shutdown, either we have an
+ * unrecoverable error or we have been told to shutdown. Either case the
+ * datanode state machine should move to Shutdown state, otherwise we
+ * remain in the Running state.
+ *
+ * @return next container state.
+ */
+ private DatanodeStateMachine.DatanodeStates
+ computeNextContainerState(
+ List<Future<EndpointStateMachine.EndPointStates>> results) {
+ for (Future<EndpointStateMachine.EndPointStates> state : results) {
+ try {
+ if (state.get() == EndpointStateMachine.EndPointStates.SHUTDOWN) {
+ // if any endpoint tells us to shutdown we move to shutdown state.
+ return DatanodeStateMachine.DatanodeStates.SHUTDOWN;
+ }
+ } catch (InterruptedException | ExecutionException e) {
+ LOG.error("Error in executing end point task.", e);
+ }
+ }
+ return DatanodeStateMachine.DatanodeStates.RUNNING;
+ }
+
+ /**
+ * Wait for execute to finish.
+ *
+ * @param duration - Time
+ * @param timeUnit - Unit of duration.
+ */
+ @Override
+ public DatanodeStateMachine.DatanodeStates
+ await(long duration, TimeUnit timeUnit)
+ throws InterruptedException, ExecutionException, TimeoutException {
+ int count = connectionManager.getValues().size();
+ int returned = 0;
+ long timeLeft = timeUnit.toMillis(duration);
+ long startTime = Time.monotonicNow();
+ List<Future<EndpointStateMachine.EndPointStates>> results = new
+ LinkedList<>();
+
+ while (returned < count && timeLeft > 0) {
+ Future<EndpointStateMachine.EndPointStates> result =
+ ecs.poll(timeLeft, TimeUnit.MILLISECONDS);
+ if (result != null) {
+ results.add(result);
+ returned++;
+ }
+ timeLeft = timeLeft - (Time.monotonicNow() - startTime);
+ }
+ return computeNextContainerState(results);
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/package-info.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/package-info.java
new file mode 100644
index 0000000000..6b8d16c6d3
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.states.datanode;
+/**
+ This package contians files that guide the state transitions from
+ Init->Running->Shutdown for the datanode.
+ */ \ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
new file mode 100644
index 0000000000..5dee10f44b
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
@@ -0,0 +1,267 @@
+/*
+ * 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.states.endpoint;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.SCMCommandResponseProto;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
+import org.apache.hadoop.ozone.container.common.helpers
+ .DeletedContainerBlocksSummary;
+import org.apache.hadoop.ozone.container.common.statemachine
+ .EndpointStateMachine;
+import org.apache.hadoop.ozone.container.common.statemachine
+ .EndpointStateMachine.EndPointStates;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
+import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand;
+import org.apache.hadoop.ozone.protocol.commands.SendContainerCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.time.ZonedDateTime;
+import java.util.concurrent.Callable;
+
+/**
+ * Heartbeat class for SCMs.
+ */
+public class HeartbeatEndpointTask
+ implements Callable<EndpointStateMachine.EndPointStates> {
+ static final Logger LOG =
+ LoggerFactory.getLogger(HeartbeatEndpointTask.class);
+ private final EndpointStateMachine rpcEndpoint;
+ private final Configuration conf;
+ private DatanodeDetailsProto datanodeDetailsProto;
+ private StateContext context;
+
+ /**
+ * Constructs a SCM heart beat.
+ *
+ * @param conf Config.
+ */
+ public HeartbeatEndpointTask(EndpointStateMachine rpcEndpoint,
+ Configuration conf, StateContext context) {
+ this.rpcEndpoint = rpcEndpoint;
+ this.conf = conf;
+ this.context = context;
+ }
+
+ /**
+ * Get the container Node ID proto.
+ *
+ * @return ContainerNodeIDProto
+ */
+ public DatanodeDetailsProto getDatanodeDetailsProto() {
+ return datanodeDetailsProto;
+ }
+
+ /**
+ * Set container node ID proto.
+ *
+ * @param datanodeDetailsProto - the node id.
+ */
+ public void setDatanodeDetailsProto(DatanodeDetailsProto
+ datanodeDetailsProto) {
+ this.datanodeDetailsProto = datanodeDetailsProto;
+ }
+
+ /**
+ * Computes a result, or throws an exception if unable to do so.
+ *
+ * @return computed result
+ * @throws Exception if unable to compute a result
+ */
+ @Override
+ public EndpointStateMachine.EndPointStates call() throws Exception {
+ rpcEndpoint.lock();
+ try {
+ Preconditions.checkState(this.datanodeDetailsProto != null);
+
+ SCMHeartbeatResponseProto reponse = rpcEndpoint.getEndPoint()
+ .sendHeartbeat(datanodeDetailsProto, this.context.getNodeReport(),
+ this.context.getContainerReportState());
+ processResponse(reponse, datanodeDetailsProto);
+ rpcEndpoint.setLastSuccessfulHeartbeat(ZonedDateTime.now());
+ rpcEndpoint.zeroMissedCount();
+ } catch (IOException ex) {
+ rpcEndpoint.logIfNeeded(ex);
+ } finally {
+ rpcEndpoint.unlock();
+ }
+ return rpcEndpoint.getState();
+ }
+
+ /**
+ * Returns a builder class for HeartbeatEndpointTask task.
+ * @return Builder.
+ */
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ /**
+ * Add this command to command processing Queue.
+ *
+ * @param response - SCMHeartbeat response.
+ */
+ private void processResponse(SCMHeartbeatResponseProto response,
+ final DatanodeDetailsProto datanodeDetails) {
+ for (SCMCommandResponseProto commandResponseProto : response
+ .getCommandsList()) {
+ // Verify the response is indeed for this datanode.
+ Preconditions.checkState(commandResponseProto.getDatanodeUUID()
+ .equalsIgnoreCase(datanodeDetails.getUuid()),
+ "Unexpected datanode ID in the response.");
+ switch (commandResponseProto.getCmdType()) {
+ case sendContainerReport:
+ this.context.addCommand(SendContainerCommand.getFromProtobuf(
+ commandResponseProto.getSendReport()));
+ break;
+ case reregisterCommand:
+ if (rpcEndpoint.getState() == EndPointStates.HEARTBEAT) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Received SCM notification to register."
+ + " Interrupt HEARTBEAT and transit to REGISTER state.");
+ }
+ rpcEndpoint.setState(EndPointStates.REGISTER);
+ } else {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Illegal state {} found, expecting {}.",
+ rpcEndpoint.getState().name(), EndPointStates.HEARTBEAT);
+ }
+ }
+ break;
+ case deleteBlocksCommand:
+ DeleteBlocksCommand db = DeleteBlocksCommand
+ .getFromProtobuf(commandResponseProto.getDeleteBlocksProto());
+ if (!db.blocksTobeDeleted().isEmpty()) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(DeletedContainerBlocksSummary
+ .getFrom(db.blocksTobeDeleted())
+ .toString());
+ }
+ this.context.addCommand(db);
+ }
+ break;
+ case closeContainerCommand:
+ CloseContainerCommand closeContainer =
+ CloseContainerCommand.getFromProtobuf(
+ commandResponseProto.getCloseContainerProto());
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Received SCM container close request for container {}",
+ closeContainer.getContainerName());
+ }
+ this.context.addCommand(closeContainer);
+ break;
+ default:
+ throw new IllegalArgumentException("Unknown response : "
+ + commandResponseProto.getCmdType().name());
+ }
+ }
+ }
+
+ /**
+ * Builder class for HeartbeatEndpointTask.
+ */
+ public static class Builder {
+ private EndpointStateMachine endPointStateMachine;
+ private Configuration conf;
+ private DatanodeDetails datanodeDetails;
+ private StateContext context;
+
+ /**
+ * Constructs the builder class.
+ */
+ public Builder() {
+ }
+
+ /**
+ * Sets the endpoint state machine.
+ *
+ * @param rpcEndPoint - Endpoint state machine.
+ * @return Builder
+ */
+ public Builder setEndpointStateMachine(EndpointStateMachine rpcEndPoint) {
+ this.endPointStateMachine = rpcEndPoint;
+ return this;
+ }
+
+ /**
+ * Sets the Config.
+ *
+ * @param config - config
+ * @return Builder
+ */
+ public Builder setConfig(Configuration config) {
+ this.conf = config;
+ return this;
+ }
+
+ /**
+ * Sets the NodeID.
+ *
+ * @param dnDetails - NodeID proto
+ * @return Builder
+ */
+ public Builder setDatanodeDetails(DatanodeDetails dnDetails) {
+ this.datanodeDetails = dnDetails;
+ return this;
+ }
+
+ /**
+ * Sets the context.
+ * @param stateContext - State context.
+ * @return this.
+ */
+ public Builder setContext(StateContext stateContext) {
+ this.context = stateContext;
+ return this;
+ }
+
+ public HeartbeatEndpointTask build() {
+ if (endPointStateMachine == null) {
+ LOG.error("No endpoint specified.");
+ throw new IllegalArgumentException("A valid endpoint state machine is" +
+ " needed to construct HeartbeatEndpointTask task");
+ }
+
+ if (conf == null) {
+ LOG.error("No config specified.");
+ throw new IllegalArgumentException("A valid configration is needed to" +
+ " construct HeartbeatEndpointTask task");
+ }
+
+ if (datanodeDetails == null) {
+ LOG.error("No datanode specified.");
+ throw new IllegalArgumentException("A vaild Node ID is needed to " +
+ "construct HeartbeatEndpointTask task");
+ }
+
+ HeartbeatEndpointTask task = new HeartbeatEndpointTask(this
+ .endPointStateMachine, this.conf, this.context);
+ task.setDatanodeDetailsProto(datanodeDetails.getProtoBufMessage());
+ return task;
+ }
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java
new file mode 100644
index 0000000000..6913896e40
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java
@@ -0,0 +1,194 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.states.endpoint;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
+import org.apache.hadoop.ozone.container.common.statemachine
+ .EndpointStateMachine;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+
+/**
+ * Register a container with SCM.
+ */
+public final class RegisterEndpointTask implements
+ Callable<EndpointStateMachine.EndPointStates> {
+ static final Logger LOG = LoggerFactory.getLogger(RegisterEndpointTask.class);
+
+ private final EndpointStateMachine rpcEndPoint;
+ private final Configuration conf;
+ private Future<EndpointStateMachine.EndPointStates> result;
+ private DatanodeDetailsProto datanodeDetailsProto;
+
+ /**
+ * Creates a register endpoint task.
+ *
+ * @param rpcEndPoint - endpoint
+ * @param conf - conf
+ */
+ @VisibleForTesting
+ public RegisterEndpointTask(EndpointStateMachine rpcEndPoint,
+ Configuration conf) {
+ this.rpcEndPoint = rpcEndPoint;
+ this.conf = conf;
+
+ }
+
+ /**
+ * Get the DatanodeDetailsProto Proto.
+ *
+ * @return DatanodeDetailsProto
+ */
+ public DatanodeDetailsProto getDatanodeDetailsProto() {
+ return datanodeDetailsProto;
+ }
+
+ /**
+ * Set the contiainerNodeID Proto.
+ *
+ * @param datanodeDetailsProto - Container Node ID.
+ */
+ public void setDatanodeDetailsProto(
+ DatanodeDetailsProto datanodeDetailsProto) {
+ this.datanodeDetailsProto = datanodeDetailsProto;
+ }
+
+ /**
+ * Computes a result, or throws an exception if unable to do so.
+ *
+ * @return computed result
+ * @throws Exception if unable to compute a result
+ */
+ @Override
+ public EndpointStateMachine.EndPointStates call() throws Exception {
+
+ if (getDatanodeDetailsProto() == null) {
+ LOG.error("Container ID proto cannot be null in RegisterEndpoint task, " +
+ "shutting down the endpoint.");
+ return rpcEndPoint.setState(EndpointStateMachine.EndPointStates.SHUTDOWN);
+ }
+
+ rpcEndPoint.lock();
+ try {
+
+ // TODO : Add responses to the command Queue.
+ rpcEndPoint.getEndPoint().register(datanodeDetailsProto,
+ conf.getStrings(ScmConfigKeys.OZONE_SCM_NAMES));
+ EndpointStateMachine.EndPointStates nextState =
+ rpcEndPoint.getState().getNextState();
+ rpcEndPoint.setState(nextState);
+ rpcEndPoint.zeroMissedCount();
+ } catch (IOException ex) {
+ rpcEndPoint.logIfNeeded(ex
+ );
+ } finally {
+ rpcEndPoint.unlock();
+ }
+
+ return rpcEndPoint.getState();
+ }
+
+ /**
+ * Returns a builder class for RegisterEndPoint task.
+ *
+ * @return Builder.
+ */
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ /**
+ * Builder class for RegisterEndPoint task.
+ */
+ public static class Builder {
+ private EndpointStateMachine endPointStateMachine;
+ private Configuration conf;
+ private DatanodeDetails datanodeDetails;
+
+ /**
+ * Constructs the builder class.
+ */
+ public Builder() {
+ }
+
+ /**
+ * Sets the endpoint state machine.
+ *
+ * @param rpcEndPoint - Endpoint state machine.
+ * @return Builder
+ */
+ public Builder setEndpointStateMachine(EndpointStateMachine rpcEndPoint) {
+ this.endPointStateMachine = rpcEndPoint;
+ return this;
+ }
+
+ /**
+ * Sets the Config.
+ *
+ * @param config - config
+ * @return Builder.
+ */
+ public Builder setConfig(Configuration config) {
+ this.conf = config;
+ return this;
+ }
+
+ /**
+ * Sets the NodeID.
+ *
+ * @param dnDetails - NodeID proto
+ * @return Builder
+ */
+ public Builder setDatanodeDetails(DatanodeDetails dnDetails) {
+ this.datanodeDetails = dnDetails;
+ return this;
+ }
+
+ public RegisterEndpointTask build() {
+ if (endPointStateMachine == null) {
+ LOG.error("No endpoint specified.");
+ throw new IllegalArgumentException("A valid endpoint state machine is" +
+ " needed to construct RegisterEndPoint task");
+ }
+
+ if (conf == null) {
+ LOG.error("No config specified.");
+ throw new IllegalArgumentException("A valid configration is needed to" +
+ " construct RegisterEndpoint task");
+ }
+
+ if (datanodeDetails == null) {
+ LOG.error("No datanode specified.");
+ throw new IllegalArgumentException("A vaild Node ID is needed to " +
+ "construct RegisterEndpoint task");
+ }
+
+ RegisterEndpointTask task = new RegisterEndpointTask(this
+ .endPointStateMachine, this.conf);
+ task.setDatanodeDetailsProto(datanodeDetails.getProtoBufMessage());
+ return task;
+ }
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
new file mode 100644
index 0000000000..b048ee5b5c
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.states.endpoint;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
+import org.apache.hadoop.ozone.container.common.statemachine
+ .EndpointStateMachine;
+import org.apache.hadoop.ozone.protocol.VersionResponse;
+
+import java.io.IOException;
+import java.util.concurrent.Callable;
+
+/**
+ * Task that returns version.
+ */
+public class VersionEndpointTask implements
+ Callable<EndpointStateMachine.EndPointStates> {
+ private final EndpointStateMachine rpcEndPoint;
+ private final Configuration configuration;
+
+ public VersionEndpointTask(EndpointStateMachine rpcEndPoint,
+ Configuration conf) {
+ this.rpcEndPoint = rpcEndPoint;
+ this.configuration = conf;
+ }
+
+ /**
+ * Computes a result, or throws an exception if unable to do so.
+ *
+ * @return computed result
+ * @throws Exception if unable to compute a result
+ */
+ @Override
+ public EndpointStateMachine.EndPointStates call() throws Exception {
+ rpcEndPoint.lock();
+ try{
+ SCMVersionResponseProto versionResponse =
+ rpcEndPoint.getEndPoint().getVersion(null);
+ rpcEndPoint.setVersion(VersionResponse.getFromProtobuf(versionResponse));
+
+ EndpointStateMachine.EndPointStates nextState =
+ rpcEndPoint.getState().getNextState();
+ rpcEndPoint.setState(nextState);
+ rpcEndPoint.zeroMissedCount();
+ } catch (IOException ex) {
+ rpcEndPoint.logIfNeeded(ex);
+ } finally {
+ rpcEndPoint.unlock();
+ }
+ return rpcEndPoint.getState();
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/package-info.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/package-info.java
new file mode 100644
index 0000000000..112259834d
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/package-info.java
@@ -0,0 +1,20 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.states.endpoint;
+/**
+ This package contains code for RPC endpoints transitions.
+ */ \ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/package-info.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/package-info.java
new file mode 100644
index 0000000000..92c953ff41
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * 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.states;
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServer.java
new file mode 100644
index 0000000000..50e45b45bf
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServer.java
@@ -0,0 +1,130 @@
+/*
+ * 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.transport.server;
+
+import com.google.common.base.Preconditions;
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.handler.logging.LogLevel;
+import io.netty.handler.logging.LoggingHandler;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.SocketAddress;
+
+/**
+ * Creates a netty server endpoint that acts as the communication layer for
+ * Ozone containers.
+ */
+public final class XceiverServer implements XceiverServerSpi {
+ private static final Logger
+ LOG = LoggerFactory.getLogger(XceiverServer.class);
+ private int port;
+ private final ContainerDispatcher storageContainer;
+
+ private EventLoopGroup bossGroup;
+ private EventLoopGroup workerGroup;
+ private Channel channel;
+
+ /**
+ * Constructs a netty server class.
+ *
+ * @param conf - Configuration
+ */
+ public XceiverServer(DatanodeDetails datanodeDetails, Configuration conf,
+ ContainerDispatcher dispatcher) {
+ Preconditions.checkNotNull(conf);
+
+ this.port = conf.getInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
+ OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
+ // Get an available port on current node and
+ // use that as the container port
+ if (conf.getBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT,
+ OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT_DEFAULT)) {
+ try (ServerSocket socket = new ServerSocket()) {
+ socket.setReuseAddress(true);
+ SocketAddress address = new InetSocketAddress(0);
+ socket.bind(address);
+ this.port = socket.getLocalPort();
+ LOG.info("Found a free port for the server : {}", this.port);
+ } catch (IOException e) {
+ LOG.error("Unable find a random free port for the server, "
+ + "fallback to use default port {}", this.port, e);
+ }
+ }
+ datanodeDetails.setContainerPort(port);
+ this.storageContainer = dispatcher;
+ }
+
+ @Override
+ public int getIPCPort() {
+ return this.port;
+ }
+
+ /**
+ * Returns the Replication type supported by this end-point.
+ *
+ * @return enum -- {Stand_Alone, Ratis, Chained}
+ */
+ @Override
+ public HddsProtos.ReplicationType getServerType() {
+ return HddsProtos.ReplicationType.STAND_ALONE;
+ }
+
+ @Override
+ public void start() throws IOException {
+ bossGroup = new NioEventLoopGroup();
+ workerGroup = new NioEventLoopGroup();
+ channel = new ServerBootstrap()
+ .group(bossGroup, workerGroup)
+ .channel(NioServerSocketChannel.class)
+ .handler(new LoggingHandler(LogLevel.INFO))
+ .childHandler(new XceiverServerInitializer(storageContainer))
+ .bind(port)
+ .syncUninterruptibly()
+ .channel();
+ }
+
+ @Override
+ public void stop() {
+ if (storageContainer != null) {
+ storageContainer.shutdown();
+ }
+ if (bossGroup != null) {
+ bossGroup.shutdownGracefully();
+ }
+ if (workerGroup != null) {
+ workerGroup.shutdownGracefully();
+ }
+ if (channel != null) {
+ channel.close().awaitUninterruptibly();
+ }
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerHandler.java
new file mode 100644
index 0000000000..5947dde75d
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerHandler.java
@@ -0,0 +1,82 @@
+/*
+ * 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.transport.server;
+
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
+ .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
+ .ContainerCommandResponseProto;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Netty server handlers that respond to Network events.
+ */
+public class XceiverServerHandler extends
+ SimpleChannelInboundHandler<ContainerCommandRequestProto> {
+
+ static final Logger LOG = LoggerFactory.getLogger(XceiverServerHandler.class);
+ private final ContainerDispatcher dispatcher;
+
+ /**
+ * Constructor for server handler.
+ * @param dispatcher - Dispatcher interface
+ */
+ public XceiverServerHandler(ContainerDispatcher dispatcher) {
+ this.dispatcher = dispatcher;
+ }
+
+ /**
+ * <strong>Please keep in mind that this method will be renamed to {@code
+ * messageReceived(ChannelHandlerContext, I)} in 5.0.</strong>
+ * <p>
+ * Is called for each message of type {@link ContainerCommandRequestProto}.
+ *
+ * @param ctx the {@link ChannelHandlerContext} which this {@link
+ * SimpleChannelInboundHandler} belongs to
+ * @param msg the message to handle
+ * @throws Exception is thrown if an error occurred
+ */
+ @Override
+ public void channelRead0(ChannelHandlerContext ctx,
+ ContainerCommandRequestProto msg) throws
+ Exception {
+ ContainerCommandResponseProto response = this.dispatcher.dispatch(msg);
+ LOG.debug("Writing the reponse back to client.");
+ ctx.writeAndFlush(response);
+
+ }
+
+ /**
+ * Calls {@link ChannelHandlerContext#fireExceptionCaught(Throwable)}
+ * Sub-classes may override this method to change behavior.
+ *
+ * @param ctx - Channel Handler Context
+ * @param cause - Exception
+ */
+ @Override
+ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
+ throws Exception {
+ LOG.error("An exception caught in the pipeline : " + cause.toString());
+ super.exceptionCaught(ctx, cause);
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerInitializer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerInitializer.java
new file mode 100644
index 0000000000..78ba26b4de
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerInitializer.java
@@ -0,0 +1,62 @@
+/*
+ * 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.transport.server;
+
+import com.google.common.base.Preconditions;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.handler.codec.protobuf.ProtobufDecoder;
+import io.netty.handler.codec.protobuf.ProtobufEncoder;
+import io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
+import io.netty.handler.codec.protobuf.ProtobufVarint32LengthFieldPrepender;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
+ .ContainerCommandRequestProto;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
+
+/**
+ * Creates a channel for the XceiverServer.
+ */
+public class XceiverServerInitializer extends ChannelInitializer<SocketChannel>{
+ private final ContainerDispatcher dispatcher;
+ public XceiverServerInitializer(ContainerDispatcher dispatcher) {
+ Preconditions.checkNotNull(dispatcher);
+ this.dispatcher = dispatcher;
+ }
+
+ /**
+ * This method will be called once the Channel is registered. After
+ * the method returns this instance will be removed from the {@link
+ * ChannelPipeline}
+ *
+ * @param ch the which was registered.
+ * @throws Exception is thrown if an error occurs. In that case the channel
+ * will be closed.
+ */
+ @Override
+ protected void initChannel(SocketChannel ch) throws Exception {
+ ChannelPipeline pipeline = ch.pipeline();
+ pipeline.addLast(new ProtobufVarint32FrameDecoder());
+ pipeline.addLast(new ProtobufDecoder(ContainerCommandRequestProto
+ .getDefaultInstance()));
+ pipeline.addLast(new ProtobufVarint32LengthFieldPrepender());
+ pipeline.addLast(new ProtobufEncoder());
+ pipeline.addLast(new XceiverServerHandler(dispatcher));
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java
new file mode 100644
index 0000000000..dad9e9f3fd
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java
@@ -0,0 +1,43 @@
+/*
+ * 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.transport.server;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+
+import java.io.IOException;
+
+/** A server endpoint that acts as the communication layer for Ozone
+ * containers. */
+public interface XceiverServerSpi {
+ /** Starts the server. */
+ void start() throws IOException;
+
+ /** Stops a running server. */
+ void stop();
+
+ /** Get server IPC port. */
+ int getIPCPort();
+
+ /**
+ * Returns the Replication type supported by this end-point.
+ * @return enum -- {Stand_Alone, Ratis, Chained}
+ */
+ HddsProtos.ReplicationType getServerType();
+
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/package-info.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/package-info.java
new file mode 100644
index 0000000000..59c96f1349
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/package-info.java
@@ -0,0 +1,24 @@
+/**
+ * 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.transport.server;
+
+/**
+ * This package contains classes for the server of the storage container
+ * protocol.
+ */
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
new file mode 100644
index 0000000000..1a89e44bd1
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
@@ -0,0 +1,293 @@
+/*
+ * 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.transport.server.ratis;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
+ .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
+ .ContainerCommandResponseProto;
+import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
+ .WriteChunkRequestProto;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.server.storage.RaftStorage;
+import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.shaded.com.google.protobuf.ShadedProtoUtil;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+import org.apache.ratis.statemachine.StateMachineStorage;
+import org.apache.ratis.statemachine.TransactionContext;
+import org.apache.ratis.statemachine.impl.BaseStateMachine;
+import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage;
+import org.apache.ratis.statemachine.impl.TransactionContextImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ThreadPoolExecutor;
+
+/** A {@link org.apache.ratis.statemachine.StateMachine} for containers.
+ *
+ * The stateMachine is responsible for handling different types of container
+ * requests. The container requests can be divided into readonly and write
+ * requests.
+ *
+ * Read only requests are classified in
+ * {@link org.apache.hadoop.hdds.scm.XceiverClientRatis#isReadOnly}
+ * and these readonly requests are replied from the {@link #query(Message)}.
+ *
+ * The write requests can be divided into requests with user data
+ * (WriteChunkRequest) and other request without user data.
+ *
+ * Inorder to optimize the write throughput, the writeChunk request is
+ * processed in 2 phases. The 2 phases are divided in
+ * {@link #startTransaction(RaftClientRequest)}, in the first phase the user
+ * data is written directly into the state machine via
+ * {@link #writeStateMachineData} and in the second phase the
+ * transaction is committed via {@link #applyTransaction(TransactionContext)}
+ *
+ * For the requests with no stateMachine data, the transaction is directly
+ * committed through
+ * {@link #applyTransaction(TransactionContext)}
+ *
+ * There are 2 ordering operation which are enforced right now in the code,
+ * 1) Write chunk operation are executed after the create container operation,
+ * the write chunk operation will fail otherwise as the container still hasn't
+ * been created. Hence the create container operation has been split in the
+ * {@link #startTransaction(RaftClientRequest)}, this will help in synchronizing
+ * the calls in {@link #writeStateMachineData}
+ *
+ * 2) Write chunk commit operation is executed after write chunk state machine
+ * operation. This will ensure that commit operation is sync'd with the state
+ * machine operation.
+ * */
+public class ContainerStateMachine extends BaseStateMachine {
+ static final Logger LOG = LoggerFactory.getLogger(
+ ContainerStateMachine.class);
+ private final SimpleStateMachineStorage storage
+ = new SimpleStateMachineStorage();
+ private final ContainerDispatcher dispatcher;
+ private ThreadPoolExecutor writeChunkExecutor;
+ private final ConcurrentHashMap<Long, CompletableFuture<Message>>
+ writeChunkFutureMap;
+ private final ConcurrentHashMap<String, CompletableFuture<Message>>
+ createContainerFutureMap;
+
+ ContainerStateMachine(ContainerDispatcher dispatcher,
+ ThreadPoolExecutor writeChunkExecutor) {
+ this.dispatcher = dispatcher;
+ this.writeChunkExecutor = writeChunkExecutor;
+ this.writeChunkFutureMap = new ConcurrentHashMap<>();
+ this.createContainerFutureMap = new ConcurrentHashMap<>();
+ }
+
+ @Override
+ public StateMachineStorage getStateMachineStorage() {
+ return storage;
+ }
+
+ @Override
+ public void initialize(
+ RaftPeerId id, RaftProperties properties, RaftStorage raftStorage)
+ throws IOException {
+ super.initialize(id, properties, raftStorage);
+ storage.init(raftStorage);
+ // TODO handle snapshots
+
+ // TODO: Add a flag that tells you that initialize has been called.
+ // Check with Ratis if this feature is done in Ratis.
+ }
+
+ @Override
+ public TransactionContext startTransaction(RaftClientRequest request)
+ throws IOException {
+ final ContainerCommandRequestProto proto =
+ getRequestProto(request.getMessage().getContent());
+
+ final SMLogEntryProto log;
+ if (proto.getCmdType() == ContainerProtos.Type.WriteChunk) {
+ final WriteChunkRequestProto write = proto.getWriteChunk();
+ // create the state machine data proto
+ final WriteChunkRequestProto dataWriteChunkProto =
+ WriteChunkRequestProto
+ .newBuilder(write)
+ .setStage(ContainerProtos.Stage.WRITE_DATA)
+ .build();
+ ContainerCommandRequestProto dataContainerCommandProto =
+ ContainerCommandRequestProto
+ .newBuilder(proto)
+ .setWriteChunk(dataWriteChunkProto)
+ .build();
+
+ // create the log entry proto
+ final WriteChunkRequestProto commitWriteChunkProto =
+ WriteChunkRequestProto.newBuilder()
+ .setPipeline(write.getPipeline())
+ .setKeyName(write.getKeyName())
+ .setChunkData(write.getChunkData())
+ // skipping the data field as it is
+ // already set in statemachine data proto
+ .setStage(ContainerProtos.Stage.COMMIT_DATA)
+ .build();
+ ContainerCommandRequestProto commitContainerCommandProto =
+ ContainerCommandRequestProto
+ .newBuilder(proto)
+ .setWriteChunk(commitWriteChunkProto)
+ .build();
+
+ log = SMLogEntryProto.newBuilder()
+ .setData(getShadedByteString(commitContainerCommandProto))
+ .setStateMachineData(getShadedByteString(dataContainerCommandProto))
+ .build();
+ } else if (proto.getCmdType() == ContainerProtos.Type.CreateContainer) {
+ log = SMLogEntryProto.newBuilder()
+ .setData(request.getMessage().getContent())
+ .setStateMachineData(request.getMessage().getContent())
+ .build();
+ } else {
+ log = SMLogEntryProto.newBuilder()
+ .setData(request.getMessage().getContent())
+ .build();
+ }
+ return new TransactionContextImpl(this, request, log);
+ }
+
+ private ByteString getShadedByteString(ContainerCommandRequestProto proto) {
+ return ShadedProtoUtil.asShadedByteString(proto.toByteArray());
+ }
+
+ private ContainerCommandRequestProto getRequestProto(ByteString request)
+ throws InvalidProtocolBufferException {
+ return ContainerCommandRequestProto.parseFrom(
+ ShadedProtoUtil.asByteString(request));
+ }
+
+ private Message runCommand(ContainerCommandRequestProto requestProto) {
+ LOG.trace("dispatch {}", requestProto);
+ ContainerCommandResponseProto response = dispatcher.dispatch(requestProto);
+ LOG.trace("response {}", response);
+ return () -> ShadedProtoUtil.asShadedByteString(response.toByteArray());
+ }
+
+ private CompletableFuture<Message> handleWriteChunk(
+ ContainerCommandRequestProto requestProto, long entryIndex) {
+ final WriteChunkRequestProto write = requestProto.getWriteChunk();
+ String containerName = write.getPipeline().getContainerName();
+ CompletableFuture<Message> future =
+ createContainerFutureMap.get(containerName);
+ CompletableFuture<Message> writeChunkFuture;
+ if (future != null) {
+ writeChunkFuture = future.thenApplyAsync(
+ v -> runCommand(requestProto), writeChunkExecutor);
+ } else {
+ writeChunkFuture = CompletableFuture.supplyAsync(
+ () -> runCommand(requestProto), writeChunkExecutor);
+ }
+ writeChunkFutureMap.put(entryIndex, writeChunkFuture);
+ return writeChunkFuture;
+ }
+
+ private CompletableFuture<Message> handleCreateContainer(
+ ContainerCommandRequestProto requestProto) {
+ String containerName =
+ requestProto.getCreateContainer().getContainerData().getName();
+ createContainerFutureMap.
+ computeIfAbsent(containerName, k -> new CompletableFuture<>());
+ return CompletableFuture.completedFuture(() -> ByteString.EMPTY);
+ }
+
+ @Override
+ public CompletableFuture<Message> writeStateMachineData(LogEntryProto entry) {
+ try {
+ final ContainerCommandRequestProto requestProto =
+ getRequestProto(entry.getSmLogEntry().getStateMachineData());
+ ContainerProtos.Type cmdType = requestProto.getCmdType();
+ switch (cmdType) {
+ case CreateContainer:
+ return handleCreateContainer(requestProto);
+ case WriteChunk:
+ return handleWriteChunk(requestProto, entry.getIndex());
+ default:
+ throw new IllegalStateException("Cmd Type:" + cmdType
+ + " should not have state machine data");
+ }
+ } catch (IOException e) {
+ return completeExceptionally(e);
+ }
+ }
+
+ @Override
+ public CompletableFuture<Message> query(Message request) {
+ try {
+ final ContainerCommandRequestProto requestProto =
+ getRequestProto(request.getContent());
+ return CompletableFuture.completedFuture(runCommand(requestProto));
+ } catch (IOException e) {
+ return completeExceptionally(e);
+ }
+ }
+
+ @Override
+ public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
+ try {
+ ContainerCommandRequestProto requestProto =
+ getRequestProto(trx.getSMLogEntry().getData());
+ ContainerProtos.Type cmdType = requestProto.getCmdType();
+
+ if (cmdType == ContainerProtos.Type.WriteChunk) {
+ WriteChunkRequestProto write = requestProto.getWriteChunk();
+ // the data field has already been removed in start Transaction
+ Preconditions.checkArgument(!write.hasData());
+ CompletableFuture<Message> stateMachineFuture =
+ writeChunkFutureMap.remove(trx.getLogEntry().getIndex());
+ return stateMachineFuture
+ .thenComposeAsync(v ->
+ CompletableFuture.completedFuture(runCommand(requestProto)));
+ } else {
+ Message message = runCommand(requestProto);
+ if (cmdType == ContainerProtos.Type.CreateContainer) {
+ String containerName =
+ requestProto.getCreateContainer().getContainerData().getName();
+ createContainerFutureMap.remove(containerName).complete(message);
+ }
+ return CompletableFuture.completedFuture(message);
+ }
+ } catch (IOException e) {
+ return completeExceptionally(e);
+ }
+ }
+
+ private static <T> CompletableFuture<T> completeExceptionally(Exception e) {
+ final CompletableFuture<T> future = new CompletableFuture<>();
+ future.completeExceptionally(e);
+ return future;
+ }
+
+ @Override
+ public void close() throws IOException {
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
new file mode 100644
index 0000000000..4bd55f1b99
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
@@ -0,0 +1,214 @@
+/*
+ * 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.transport.server.ratis;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
+import org.apache.hadoop.ozone.container.common.transport.server
+ .XceiverServerSpi;
+import org.apache.ratis.RaftConfigKeys;
+import org.apache.ratis.RatisHelper;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.grpc.GrpcConfigKeys;
+import org.apache.ratis.netty.NettyConfigKeys;
+import org.apache.ratis.rpc.RpcType;
+import org.apache.ratis.rpc.SupportedRpcType;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.util.SizeInBytes;
+import org.apache.ratis.util.TimeDuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.SocketAddress;
+import java.util.Objects;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Creates a ratis server endpoint that acts as the communication layer for
+ * Ozone containers.
+ */
+public final class XceiverServerRatis implements XceiverServerSpi {
+ static final Logger LOG = LoggerFactory.getLogger(XceiverServerRatis.class);
+ private final int port;
+ private final RaftServer server;
+ private ThreadPoolExecutor writeChunkExecutor;
+
+ private XceiverServerRatis(DatanodeDetails dd, int port, String storageDir,
+ ContainerDispatcher dispatcher, Configuration conf) throws IOException {
+
+ final String rpcType = conf.get(
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
+ final RpcType rpc = SupportedRpcType.valueOfIgnoreCase(rpcType);
+ final int raftSegmentSize = conf.getInt(
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY,
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT);
+ final int raftSegmentPreallocatedSize = conf.getInt(
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY,
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT);
+ final int maxChunkSize = OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE;
+ final int numWriteChunkThreads = conf.getInt(
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_KEY,
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_DEFAULT);
+
+ Objects.requireNonNull(dd, "id == null");
+ this.port = port;
+ RaftProperties serverProperties = newRaftProperties(rpc, port,
+ storageDir, maxChunkSize, raftSegmentSize, raftSegmentPreallocatedSize);
+
+ writeChunkExecutor =
+ new ThreadPoolExecutor(numWriteChunkThreads, numWriteChunkThreads,
+ 100, TimeUnit.SECONDS,
+ new ArrayBlockingQueue<>(1024),
+ new ThreadPoolExecutor.CallerRunsPolicy());
+ ContainerStateMachine stateMachine =
+ new ContainerStateMachine(dispatcher, writeChunkExecutor);
+ this.server = RaftServer.newBuilder()
+ .setServerId(RatisHelper.toRaftPeerId(dd))
+ .setGroup(RatisHelper.emptyRaftGroup())
+ .setProperties(serverProperties)
+ .setStateMachine(stateMachine)
+ .build();
+ }
+
+ private static RaftProperties newRaftProperties(
+ RpcType rpc, int port, String storageDir, int scmChunkSize,
+ int raftSegmentSize, int raftSegmentPreallocatedSize) {
+ final RaftProperties properties = new RaftProperties();
+ RaftServerConfigKeys.Log.Appender.setBatchEnabled(properties, true);
+ RaftServerConfigKeys.Log.Appender.setBufferCapacity(properties,
+ SizeInBytes.valueOf(raftSegmentPreallocatedSize));
+ RaftServerConfigKeys.Log.setWriteBufferSize(properties,
+ SizeInBytes.valueOf(scmChunkSize));
+ RaftServerConfigKeys.Log.setPreallocatedSize(properties,
+ SizeInBytes.valueOf(raftSegmentPreallocatedSize));
+ RaftServerConfigKeys.Log.setSegmentSizeMax(properties,
+ SizeInBytes.valueOf(raftSegmentSize));
+ RaftServerConfigKeys.setStorageDir(properties, new File(storageDir));
+ RaftConfigKeys.Rpc.setType(properties, rpc);
+
+ RaftServerConfigKeys.Log.setMaxCachedSegmentNum(properties, 2);
+ GrpcConfigKeys.setMessageSizeMax(properties,
+ SizeInBytes.valueOf(scmChunkSize + raftSegmentPreallocatedSize));
+ RaftServerConfigKeys.Rpc.setTimeoutMin(properties,
+ TimeDuration.valueOf(800, TimeUnit.MILLISECONDS));
+ RaftServerConfigKeys.Rpc.setTimeoutMax(properties,
+ TimeDuration.valueOf(1000, TimeUnit.MILLISECONDS));
+ if (rpc == SupportedRpcType.GRPC) {
+ GrpcConfigKeys.Server.setPort(properties, port);
+ } else if (rpc == SupportedRpcType.NETTY) {
+ NettyConfigKeys.Server.setPort(properties, port);
+ }
+ return properties;
+ }
+
+ public static XceiverServerRatis newXceiverServerRatis(
+ DatanodeDetails datanodeDetails, Configuration ozoneConf,
+ ContainerDispatcher dispatcher) throws IOException {
+ final String ratisDir = File.separator + "ratis";
+ int localPort = ozoneConf.getInt(
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT,
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT_DEFAULT);
+ String storageDir = ozoneConf.get(
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR);
+
+ if (Strings.isNullOrEmpty(storageDir)) {
+ storageDir = ozoneConf.get(OzoneConfigKeys
+ .OZONE_METADATA_DIRS);
+ Preconditions.checkNotNull(storageDir, "ozone.metadata.dirs " +
+ "cannot be null, Please check your configs.");
+ storageDir = storageDir.concat(ratisDir);
+ LOG.warn("Storage directory for Ratis is not configured. Mapping Ratis " +
+ "storage under {}. It is a good idea to map this to an SSD disk.",
+ storageDir);
+ }
+
+ // Get an available port on current node and
+ // use that as the container port
+ if (ozoneConf.getBoolean(OzoneConfigKeys
+ .DFS_CONTAINER_RATIS_IPC_RANDOM_PORT,
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT_DEFAULT)) {
+ try (ServerSocket socket = new ServerSocket()) {
+ socket.setReuseAddress(true);
+ SocketAddress address = new InetSocketAddress(0);
+ socket.bind(address);
+ localPort = socket.getLocalPort();
+ LOG.info("Found a free port for the server : {}", localPort);
+ // If we have random local ports configured this means that it
+ // probably running under MiniOzoneCluster. Ratis locks the storage
+ // directories, so we need to pass different local directory for each
+ // local instance. So we map ratis directories under datanode ID.
+ storageDir =
+ storageDir.concat(File.separator +
+ datanodeDetails.getUuidString());
+ } catch (IOException e) {
+ LOG.error("Unable find a random free port for the server, "
+ + "fallback to use default port {}", localPort, e);
+ }
+ }
+ datanodeDetails.setRatisPort(localPort);
+ return new XceiverServerRatis(datanodeDetails, localPort, storageDir,
+ dispatcher, ozoneConf);
+ }
+
+ @Override
+ public void start() throws IOException {
+ LOG.info("Starting {} {} at port {}", getClass().getSimpleName(),
+ server.getId(), getIPCPort());
+ writeChunkExecutor.prestartAllCoreThreads();
+ server.start();
+ }
+
+ @Override
+ public void stop() {
+ try {
+ writeChunkExecutor.shutdown();
+ server.close();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public int getIPCPort() {
+ return port;
+ }
+
+ /**
+ * Returns the Replication type supported by this end-point.
+ *
+ * @return enum -- {Stand_Alone, Ratis, Chained}
+ */
+ @Override
+ public HddsProtos.ReplicationType getServerType() {
+ return HddsProtos.ReplicationType.RATIS;
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/package-info.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/package-info.java
new file mode 100644
index 0000000000..8debfe0283
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/package-info.java
@@ -0,0 +1,23 @@
+/**
+ * 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.transport.server.ratis;
+
+/**
+ * This package contains classes for the server implementation
+ * using Apache Ratis
+ */
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
new file mode 100644
index 0000000000..6ae45b6d08
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
@@ -0,0 +1,168 @@
+/*
+ * 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.utils;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.collections.MapIterator;
+import org.apache.commons.collections.map.LRUMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.utils.MetadataStoreBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * container cache is a LRUMap that maintains the DB handles.
+ */
+public final class ContainerCache extends LRUMap {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ContainerCache.class);
+ private final Lock lock = new ReentrantLock();
+ private static ContainerCache cache;
+ private static final float LOAD_FACTOR = 0.75f;
+ /**
+ * Constructs a cache that holds DBHandle references.
+ */
+ private ContainerCache(int maxSize, float loadFactor, boolean
+ scanUntilRemovable) {
+ super(maxSize, loadFactor, scanUntilRemovable);
+ }
+
+ /**
+ * Return a singleton instance of {@link ContainerCache}
+ * that holds the DB handlers.
+ *
+ * @param conf - Configuration.
+ * @return A instance of {@link ContainerCache}.
+ */
+ public synchronized static ContainerCache getInstance(Configuration conf) {
+ if (cache == null) {
+ int cacheSize = conf.getInt(OzoneConfigKeys.OZONE_CONTAINER_CACHE_SIZE,
+ OzoneConfigKeys.OZONE_CONTAINER_CACHE_DEFAULT);
+ cache = new ContainerCache(cacheSize, LOAD_FACTOR, true);
+ }
+ return cache;
+ }
+
+ /**
+ * Closes a db instance.
+ *
+ * @param container - name of the container to be closed.
+ * @param db - db instance to close.
+ */
+ private void closeDB(String container, MetadataStore db) {
+ if (db != null) {
+ try {
+ db.close();
+ } catch (IOException e) {
+ LOG.error("Error closing DB. Container: " + container, e);
+ }
+ }
+ }
+
+ /**
+ * Closes all the db instances and resets the cache.
+ */
+ public void shutdownCache() {
+ lock.lock();
+ try {
+ // iterate the cache and close each db
+ MapIterator iterator = cache.mapIterator();
+ while (iterator.hasNext()) {
+ iterator.next();
+ MetadataStore db = (MetadataStore) iterator.getValue();
+ closeDB(iterator.getKey().toString(), db);
+ }
+ // reset the cache
+ cache.clear();
+ } finally {
+ lock.unlock();
+ }
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ protected boolean removeLRU(LinkEntry entry) {
+ lock.lock();
+ try {
+ MetadataStore db = (MetadataStore) entry.getValue();
+ closeDB(entry.getKey().toString(), db);
+ } finally {
+ lock.unlock();
+ }
+ return true;
+ }
+
+ /**
+ * Returns a DB handle if available, create the handler otherwise.
+ *
+ * @param containerName - Name of the container.
+ * @return MetadataStore.
+ */
+ public MetadataStore getDB(String containerName, String containerDBPath)
+ throws IOException {
+ Preconditions.checkNotNull(containerName);
+ Preconditions.checkState(!containerName.isEmpty());
+ lock.lock();
+ try {
+ MetadataStore db = (MetadataStore) this.get(containerName);
+
+ if (db == null) {
+ db = MetadataStoreBuilder.newBuilder()
+ .setDbFile(new File(containerDBPath))
+ .setCreateIfMissing(false)
+ .build();
+ this.put(containerName, db);
+ }
+ return db;
+ } catch (Exception e) {
+ LOG.error("Error opening DB. Container:{} ContainerPath:{}",
+ containerName, containerDBPath, e);
+ throw e;
+ } finally {
+ lock.unlock();
+ }
+ }
+
+ /**
+ * Remove a DB handler from cache.
+ *
+ * @param containerName - Name of the container.
+ */
+ public void removeDB(String containerName) {
+ Preconditions.checkNotNull(containerName);
+ Preconditions.checkState(!containerName.isEmpty());
+ lock.lock();
+ try {
+ MetadataStore db = (MetadataStore)this.get(containerName);
+ closeDB(containerName, db);
+ this.remove(containerName);
+ } finally {
+ lock.unlock();
+ }
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/package-info.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/package-info.java
new file mode 100644
index 0000000000..08264f084a
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * 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.utils; \ No newline at end of file