|
@@ -17,7 +17,7 @@
|
|
|
|
|
|
package org.apache.hadoop.ozone.scm.block;
|
|
|
|
|
|
-import org.apache.commons.lang.NotImplementedException;
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
@@ -28,9 +28,11 @@ import org.apache.hadoop.ozone.scm.node.NodeManager;
|
|
|
import org.apache.hadoop.scm.ScmConfigKeys;
|
|
|
import org.apache.hadoop.scm.container.common.helpers.AllocatedBlock;
|
|
|
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.utils.LevelDBStore;
|
|
|
import org.iq80.leveldb.DBIterator;
|
|
|
import org.iq80.leveldb.Options;
|
|
|
+import org.iq80.leveldb.WriteBatch;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
@@ -49,18 +51,20 @@ import java.util.UUID;
|
|
|
|
|
|
import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_DB;
|
|
|
import static org.apache.hadoop.ozone.OzoneConsts.OPEN_CONTAINERS_DB;
|
|
|
-import static org.apache.hadoop.ozone.scm.exceptions.SCMException
|
|
|
- .ResultCodes.CHILL_MODE_EXCEPTION;
|
|
|
-import static org.apache.hadoop.ozone.scm.exceptions.SCMException
|
|
|
- .ResultCodes.FAILED_TO_ALLOCATE_CONTAINER;
|
|
|
-import static org.apache.hadoop.ozone.scm.exceptions.SCMException
|
|
|
- .ResultCodes.FAILED_TO_FIND_CONTAINER;
|
|
|
-import static org.apache.hadoop.ozone.scm.exceptions.SCMException
|
|
|
- .ResultCodes.FAILED_TO_FIND_CONTAINER_WITH_SAPCE;
|
|
|
-import static org.apache.hadoop.ozone.scm.exceptions.SCMException
|
|
|
- .ResultCodes.FAILED_TO_LOAD_OPEN_CONTAINER;
|
|
|
-import static org.apache.hadoop.ozone.scm.exceptions.SCMException
|
|
|
- .ResultCodes.INVALID_BLOCK_SIZE;
|
|
|
+import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
|
|
|
+ CHILL_MODE_EXCEPTION;
|
|
|
+import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
|
|
|
+ FAILED_TO_ALLOCATE_CONTAINER;
|
|
|
+import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
|
|
|
+ FAILED_TO_FIND_CONTAINER;
|
|
|
+import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
|
|
|
+ FAILED_TO_FIND_CONTAINER_WITH_SAPCE;
|
|
|
+import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
|
|
|
+ FAILED_TO_FIND_BLOCK;
|
|
|
+import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
|
|
|
+ FAILED_TO_LOAD_OPEN_CONTAINER;
|
|
|
+import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
|
|
|
+ INVALID_BLOCK_SIZE;
|
|
|
|
|
|
/**
|
|
|
* Block Manager manages the block access for SCM.
|
|
@@ -290,8 +294,8 @@ public class BlockManagerImpl implements BlockManager {
|
|
|
try {
|
|
|
byte[] containerBytes = blockStore.get(DFSUtil.string2Bytes(key));
|
|
|
if (containerBytes == null) {
|
|
|
- throw new IOException("Specified block key does not exist. key : " +
|
|
|
- key);
|
|
|
+ throw new SCMException("Specified block key does not exist. key : " +
|
|
|
+ key, FAILED_TO_FIND_BLOCK);
|
|
|
}
|
|
|
return containerManager.getContainer(
|
|
|
DFSUtil.bytes2String(containerBytes));
|
|
@@ -307,7 +311,41 @@ public class BlockManagerImpl implements BlockManager {
|
|
|
*/
|
|
|
@Override
|
|
|
public void deleteBlock(final String key) throws IOException {
|
|
|
- throw new NotImplementedException("deleteBlock is not supported");
|
|
|
+ if (!nodeManager.isOutOfNodeChillMode()) {
|
|
|
+ throw new SCMException("Unable to delete block while in chill mode",
|
|
|
+ CHILL_MODE_EXCEPTION);
|
|
|
+ }
|
|
|
+
|
|
|
+ lock.lock();
|
|
|
+ try {
|
|
|
+ byte[] containerBytes = blockStore.get(DFSUtil.string2Bytes(key));
|
|
|
+ if (containerBytes == null) {
|
|
|
+ throw new SCMException("Specified block key does not exist. key : " +
|
|
|
+ key, FAILED_TO_FIND_BLOCK);
|
|
|
+ }
|
|
|
+ try (WriteBatch wb = blockStore.createWriteBatch()) {
|
|
|
+ containerManager.getContainer(
|
|
|
+ DFSUtil.bytes2String(containerBytes));
|
|
|
+ String deletedKeyName = getDeletedKeyName(key);
|
|
|
+ // Add a tombstone for the deleted key
|
|
|
+ wb.put(DFSUtil.string2Bytes(deletedKeyName), containerBytes);
|
|
|
+ // Delete the block key
|
|
|
+ wb.delete(DFSUtil.string2Bytes(key));
|
|
|
+ blockStore.commitWriteBatch(wb);
|
|
|
+ // TODO: Add async tombstone clean thread to send delete command to
|
|
|
+ // datanodes in the pipeline to clean up the blocks from containers.
|
|
|
+ // TODO: Container report handling of the deleted blocks:
|
|
|
+ // Remove tombstone and update open container usage.
|
|
|
+ // We will revisit this when the closed container replication is done.
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ lock.unlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public String getDeletedKeyName(String key) {
|
|
|
+ return StringUtils.format(".Deleted/%s", key);
|
|
|
}
|
|
|
|
|
|
/**
|