From 8b832f3c3556ef3f970bac636ef9c70ee9dd260d Mon Sep 17 00:00:00 2001 From: Anu Engineer Date: Thu, 5 Apr 2018 11:24:39 -0700 Subject: HDFS-13405. Ozone: Rename HDSL to HDDS. Contributed by Ajay Kumar, Elek Marton, Mukul Kumar Singh, Shashikant Banerjee and Anu Engineer. --- .../ozone/container/common/helpers/ChunkUtils.java | 346 ++++++ .../container/common/helpers/ContainerData.java | 326 ++++++ .../container/common/helpers/ContainerMetrics.java | 121 +++ .../container/common/helpers/ContainerReport.java | 218 ++++ .../container/common/helpers/ContainerUtils.java | 442 ++++++++ .../helpers/DeletedContainerBlocksSummary.java | 103 ++ .../ozone/container/common/helpers/FileUtils.java | 81 ++ .../ozone/container/common/helpers/KeyUtils.java | 148 +++ .../container/common/helpers/package-info.java | 22 + .../container/common/impl/ChunkManagerImpl.java | 233 ++++ .../common/impl/ContainerManagerImpl.java | 1113 ++++++++++++++++++++ .../common/impl/ContainerReportManagerImpl.java | 90 ++ .../container/common/impl/ContainerStatus.java | 217 ++++ .../common/impl/ContainerStorageLocation.java | 203 ++++ .../ozone/container/common/impl/Dispatcher.java | 713 +++++++++++++ .../container/common/impl/KeyManagerImpl.java | 202 ++++ .../RandomContainerDeletionChoosingPolicy.java | 70 ++ .../common/impl/StorageLocationReport.java | 63 ++ ...TopNOrderedContainerDeletionChoosingPolicy.java | 91 ++ .../ozone/container/common/impl/package-info.java | 22 + .../container/common/interfaces/ChunkManager.java | 76 ++ .../ContainerDeletionChoosingPolicy.java | 46 + .../common/interfaces/ContainerDispatcher.java | 51 + .../interfaces/ContainerLocationManager.java | 58 + .../interfaces/ContainerLocationManagerMXBean.java | 36 + .../common/interfaces/ContainerManager.java | 280 +++++ .../common/interfaces/ContainerReportManager.java | 32 + .../container/common/interfaces/KeyManager.java | 76 ++ .../container/common/interfaces/package-info.java | 20 + .../ozone/container/common/package-info.java | 28 + .../common/statemachine/DatanodeStateMachine.java | 387 +++++++ .../common/statemachine/EndpointStateMachine.java | 294 ++++++ .../statemachine/EndpointStateMachineMBean.java | 34 + .../common/statemachine/SCMConnectionManager.java | 208 ++++ .../statemachine/SCMConnectionManagerMXBean.java | 27 + .../common/statemachine/StateContext.java | 285 +++++ .../background/BlockDeletingService.java | 239 +++++ .../statemachine/background/package-info.java | 18 + .../commandhandler/CloseContainerHandler.java | 112 ++ .../commandhandler/CommandDispatcher.java | 177 ++++ .../commandhandler/CommandHandler.java | 59 ++ .../commandhandler/ContainerReportHandler.java | 114 ++ .../commandhandler/DeleteBlocksCommandHandler.java | 211 ++++ .../statemachine/commandhandler/package-info.java | 18 + .../common/statemachine/package-info.java | 28 + .../container/common/states/DatanodeState.java | 55 + .../common/states/datanode/InitDatanodeState.java | 157 +++ .../states/datanode/RunningDatanodeState.java | 175 +++ .../common/states/datanode/package-info.java | 21 + .../states/endpoint/HeartbeatEndpointTask.java | 267 +++++ .../states/endpoint/RegisterEndpointTask.java | 194 ++++ .../states/endpoint/VersionEndpointTask.java | 68 ++ .../common/states/endpoint/package-info.java | 20 + .../container/common/states/package-info.java | 18 + .../common/transport/server/XceiverServer.java | 130 +++ .../transport/server/XceiverServerHandler.java | 82 ++ .../transport/server/XceiverServerInitializer.java | 62 ++ .../common/transport/server/XceiverServerSpi.java | 43 + .../common/transport/server/package-info.java | 24 + .../server/ratis/ContainerStateMachine.java | 293 ++++++ .../transport/server/ratis/XceiverServerRatis.java | 214 ++++ .../transport/server/ratis/package-info.java | 23 + .../container/common/utils/ContainerCache.java | 168 +++ .../ozone/container/common/utils/package-info.java | 18 + 64 files changed, 9770 insertions(+) create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DeletedContainerBlocksSummary.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/package-info.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerReportManagerImpl.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerStatus.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerStorageLocation.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/package-info.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManager.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManagerMXBean.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerReportManager.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/package-info.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/package-info.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachineMBean.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManagerMXBean.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/package-info.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandDispatcher.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandHandler.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ContainerReportHandler.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/package-info.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/package-info.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/DatanodeState.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/package-info.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/package-info.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/package-info.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServer.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerHandler.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerInitializer.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/package-info.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/package-info.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/package-info.java (limited to 'hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common') 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. + *

+ * 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 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 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 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. + *

+ * This class has a number of metrics variables that are publicly accessible; + * these variables (objects) have methods to update their values; + * for example: + *

{@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. + *

+ * There are three things we need to delete. + *

+ * 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 blocks; + // key : txID + // value : times of this tx has been processed + private final Map 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 blockSummary; + // total number of blocks in this message + private int numOfBlocks; + + private DeletedContainerBlocksSummary(List 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 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 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 ".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 + 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 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 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> underDeletionBlocks = metadata + .getSequentialRangeKVs(null, Integer.MAX_VALUE, + MetadataKeyFilters.getDeletingKeyFilter()); + containerStatus.incrPendingDeletionBlocks(underDeletionBlocks.size()); + + List> 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. + *

+ * 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 data) throws StorageContainerException { + // TODO : Support list with Prefix and PrevKey + Preconditions.checkNotNull(data, + "Internal assertion: data cannot be null"); + readLock(); + try { + ConcurrentNavigableMap 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 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 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 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 true if and only if the name should be included in + * the file list; false otherwise. + */ + @Override + public boolean accept(File dir, String name) { + return name.endsWith(CONTAINER_EXTENSION); + } + } + + @Override + public List 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. + *

+ * 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 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 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 result = new ArrayList(); + byte[] startKeyInBytes = startKey == null ? null : + DFSUtil.string2Bytes(startKey); + MetadataKeyFilter prefixFilter = new KeyPrefixFilter(prefix); + List> range = + db.getSequentialRangeKVs(startKeyInBytes, count, prefixFilter); + for (Map.Entry 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 chooseContainerForBlockDeletion(int count, + Map candidateContainers) + throws StorageContainerException { + Preconditions.checkNotNull(candidateContainers, + "Internal assertion: candidate containers cannot be null"); + + int currentCount = 0; + List 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 CONTAINER_STATUS_COMPARATOR + = new Comparator() { + @Override + public int compare(ContainerStatus c1, ContainerStatus c2) { + return Integer.compare(c2.getNumPendingDeletionBlocks(), + c1.getNumPendingDeletionBlocks()); + } + }; + + @Override + public List chooseContainerForBlockDeletion(int count, + Map candidateContainers) + throws StorageContainerException { + Preconditions.checkNotNull(candidateContainers, + "Internal assertion: candidate containers cannot be null"); + + List result = new LinkedList<>(); + List 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 chooseContainerForBlockDeletion(int count, + Map 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 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 data) + throws StorageContainerException; + + /** + * Choose containers for block deletion. + * + * @param count - how many to return + * @throws StorageContainerException + */ + List 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 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 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 + *

+ * 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; +/** + 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. + *

+ *

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 mark 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 + *

+ * 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 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. + *

+ * This is a sorted list of states that EndPoint will traverse. + *

+ * 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 + *

+ * 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; + + +/** + * 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 + *

+ * 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 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 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 getValues() { + return scmMachines.values(); + } + + @Override + public void close() throws IOException { + getValues().forEach(endpointStateMachine + -> IOUtils.cleanupWithLogger(LOG, endpointStateMachine)); + MBeans.unregister(jmxBean); + } + + @Override + public List 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 + *

+ * 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 java.util.List; + +/** + * JMX information about the connected SCM servers. + */ +public interface SCMConnectionManagerMXBean { + + List 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 + *

+ * 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 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 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 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 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 + *

+ * 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; + +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 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 deletedBlockIds; + + ContainerBackgroundTaskResult() { + deletedBlockIds = new LinkedList<>(); + } + + public void addBlockId(String blockId) { + deletedBlockIds.add(blockId); + } + + public void addAll(List blockIds) { + deletedBlockIds.addAll(blockIds); + } + + public List getDeletedBlocks() { + return deletedBlockIds; + } + + @Override + public int getSize() { + return deletedBlockIds.size(); + } + } + + private class BlockDeletingTask + implements BackgroundTask { + + 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> toDeleteBlocks = + meta.getSequentialRangeKVs(null, blockLimitPerTask, filter); + if (toDeleteBlocks.isEmpty()) { + LOG.debug("No under deletion block found in container : {}", + containerData.getContainerName()); + } + + List 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 + *

+ * 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; + +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 + *

+ * 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; + +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 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 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 + *

+ * 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; + +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 + *

+ * 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; + +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 + *

+ * 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; + +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 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 { + /** + * 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 + *

+ * 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.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 { + static final Logger LOG = LoggerFactory.getLogger(InitDatanodeState.class); + private final SCMConnectionManager connectionManager; + private final Configuration conf; + private final StateContext context; + private Future 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 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 + *

+ * 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.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 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 endpointTask + = getEndPointTask(endpoint); + ecs.submit(endpointTask); + } + } + //TODO : Cache some of these tasks instead of creating them + //all the time. + private Callable + 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. + *

+ * 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> results) { + for (Future 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> results = new + LinkedList<>(); + + while (returned < count && timeLeft > 0) { + Future 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 + *

+ * 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.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 { + 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 + *

+ * 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.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 { + static final Logger LOG = LoggerFactory.getLogger(RegisterEndpointTask.class); + + private final EndpointStateMachine rpcEndPoint; + private final Configuration conf; + private Future 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 + *

+ * 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 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 { + 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 + *

+ * 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; +/** + 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 { + + 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; + } + + /** + * Please keep in mind that this method will be renamed to {@code + * messageReceived(ChannelHandlerContext, I)} in 5.0. + *

+ * 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{ + 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> + writeChunkFutureMap; + private final ConcurrentHashMap> + 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 handleWriteChunk( + ContainerCommandRequestProto requestProto, long entryIndex) { + final WriteChunkRequestProto write = requestProto.getWriteChunk(); + String containerName = write.getPipeline().getContainerName(); + CompletableFuture future = + createContainerFutureMap.get(containerName); + CompletableFuture 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 handleCreateContainer( + ContainerCommandRequestProto requestProto) { + String containerName = + requestProto.getCreateContainer().getContainerData().getName(); + createContainerFutureMap. + computeIfAbsent(containerName, k -> new CompletableFuture<>()); + return CompletableFuture.completedFuture(() -> ByteString.EMPTY); + } + + @Override + public CompletableFuture 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 query(Message request) { + try { + final ContainerCommandRequestProto requestProto = + getRequestProto(request.getContent()); + return CompletableFuture.completedFuture(runCommand(requestProto)); + } catch (IOException e) { + return completeExceptionally(e); + } + } + + @Override + public CompletableFuture 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 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 CompletableFuture completeExceptionally(Exception e) { + final CompletableFuture 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 -- cgit v1.2.3