XAttrFeature
+ * Set XAttrFeature
*/
abstract void addXAttrFeature(XAttrFeature xAttrFeature);
-
+
final INode addXAttrFeature(XAttrFeature xAttrFeature, int latestSnapshotId) {
recordModification(latestSnapshotId);
addXAttrFeature(xAttrFeature);
return this;
}
-
+
/**
- * Remove XAttrFeature
+ * Remove XAttrFeature
*/
abstract void removeXAttrFeature();
-
+
final INode removeXAttrFeature(int lastestSnapshotId) {
recordModification(lastestSnapshotId);
removeXAttrFeature();
return this;
}
-
+
/**
* @return if the given snapshot id is {@link Snapshot#CURRENT_STATE_ID},
* return this; otherwise return the corresponding snapshot inode.
@@ -229,7 +230,7 @@ public final boolean isInLatestSnapshot(final int latestSnapshotId) {
latestSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
return false;
}
- // if parent is a reference node, parent must be a renamed node. We can
+ // if parent is a reference node, parent must be a renamed node. We can
// stop the check at the reference node.
if (parent != null && parent.isReference()) {
return true;
@@ -248,7 +249,7 @@ public final boolean isInLatestSnapshot(final int latestSnapshotId) {
return child != null && child.isReference() &&
this == child.asReference().getReferredINode();
}
-
+
/** @return true if the given inode is an ancestor directory of this inode. */
public final boolean isAncestorDirectory(final INodeDirectory dir) {
for(INodeDirectory p = getParent(); p != null; p = p.getParent()) {
@@ -264,7 +265,7 @@ public final boolean isAncestorDirectory(final INodeDirectory dir) {
* this method tells which snapshot the modification should be
* associated with: the snapshot that belongs to the SRC tree of the rename
* operation, or the snapshot belonging to the DST tree.
- *
+ *
* @param latestInDst
* id of the latest snapshot in the DST tree above the reference node
* @return True: the modification should be recorded in the snapshot that
@@ -293,7 +294,7 @@ public final boolean shouldRecordInSrcSnapshot(final int latestInDst) {
* be recorded in the latest snapshot.
*
* @param latestSnapshotId The id of the latest snapshot that has been taken.
- * Note that it is {@link Snapshot#CURRENT_STATE_ID}
+ * Note that it is {@link Snapshot#CURRENT_STATE_ID}
* if no snapshots have been taken.
*/
abstract void recordModification(final int latestSnapshotId);
@@ -352,42 +353,42 @@ public INodeSymlink asSymlink() {
* Clean the subtree under this inode and collect the blocks from the descents
* for further block deletion/update. The current inode can either resides in
* the current tree or be stored as a snapshot copy.
- *
+ *
*
- * In general, we have the following rules.
- * 1. When deleting a file/directory in the current tree, we have different
- * actions according to the type of the node to delete.
- *
- * 1.1 The current inode (this) is an {@link INodeFile}.
- * 1.1.1 If {@code prior} is null, there is no snapshot taken on ancestors
- * before. Thus we simply destroy (i.e., to delete completely, no need to save
- * snapshot copy) the current INode and collect its blocks for further
+ * In general, we have the following rules.
+ * 1. When deleting a file/directory in the current tree, we have different
+ * actions according to the type of the node to delete.
+ *
+ * 1.1 The current inode (this) is an {@link INodeFile}.
+ * 1.1.1 If {@code prior} is null, there is no snapshot taken on ancestors
+ * before. Thus we simply destroy (i.e., to delete completely, no need to save
+ * snapshot copy) the current INode and collect its blocks for further
* cleansing.
* 1.1.2 Else do nothing since the current INode will be stored as a snapshot
* copy.
- *
+ *
* 1.2 The current inode is an {@link INodeDirectory}.
- * 1.2.1 If {@code prior} is null, there is no snapshot taken on ancestors
+ * 1.2.1 If {@code prior} is null, there is no snapshot taken on ancestors
* before. Similarly, we destroy the whole subtree and collect blocks.
- * 1.2.2 Else do nothing with the current INode. Recursively clean its
+ * 1.2.2 Else do nothing with the current INode. Recursively clean its
* children.
- *
+ *
* 1.3 The current inode is a file with snapshot.
* Call recordModification(..) to capture the current states.
* Mark the INode as deleted.
- *
+ *
* 1.4 The current inode is an {@link INodeDirectory} with snapshot feature.
- * Call recordModification(..) to capture the current states.
- * Destroy files/directories created after the latest snapshot
+ * Call recordModification(..) to capture the current states.
+ * Destroy files/directories created after the latest snapshot
* (i.e., the inodes stored in the created list of the latest snapshot).
- * Recursively clean remaining children.
+ * Recursively clean remaining children.
*
* 2. When deleting a snapshot.
* 2.1 To clean {@link INodeFile}: do nothing.
* 2.2 To clean {@link INodeDirectory}: recursively clean its children.
* 2.3 To clean INodeFile with snapshot: delete the corresponding snapshot in
* its diff list.
- * 2.4 To clean {@link INodeDirectory} with snapshot: delete the corresponding
+ * 2.4 To clean {@link INodeDirectory} with snapshot: delete the corresponding
* snapshot in its diff list. Recursively clean its children.
*
*
@@ -424,7 +425,7 @@ public final ContentSummary computeContentSummary(BlockStoragePolicySuite bsps)
}
/**
- * Compute {@link ContentSummary}.
+ * Compute {@link ContentSummary}.
*/
public final ContentSummary computeAndConvertContentSummary(int snapshotId,
ContentSummaryComputationContext summary) {
@@ -515,32 +516,32 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) {
/**
* Count subtree {@link Quota#NAMESPACE} and {@link Quota#STORAGESPACE} usages.
- *
+ *
* With the existence of {@link INodeReference}, the same inode and its
* subtree may be referred by multiple {@link WithName} nodes and a
* {@link DstReference} node. To avoid circles while quota usage computation,
* we have the following rules:
- *
+ *
*
* 1. For a {@link DstReference} node, since the node must be in the current
- * tree (or has been deleted as the end point of a series of rename
- * operations), we compute the quota usage of the referred node (and its
+ * tree (or has been deleted as the end point of a series of rename
+ * operations), we compute the quota usage of the referred node (and its
* subtree) in the regular manner, i.e., including every inode in the current
* tree and in snapshot copies, as well as the size of diff list.
- *
- * 2. For a {@link WithName} node, since the node must be in a snapshot, we
- * only count the quota usage for those nodes that still existed at the
+ *
+ * 2. For a {@link WithName} node, since the node must be in a snapshot, we
+ * only count the quota usage for those nodes that still existed at the
* creation time of the snapshot associated with the {@link WithName} node.
- * We do not count in the size of the diff list.
+ * We do not count in the size of the diff list.
*
*
* @param bsps Block storage policy suite to calculate intended storage type usage
* @param blockStoragePolicyId block storage policy id of the current INode
- * @param useCache Whether to use cached quota usage. Note that
+ * @param useCache Whether to use cached quota usage. Note that
* {@link WithName} node never uses cache for its subtree.
- * @param lastSnapshotId {@link Snapshot#CURRENT_STATE_ID} indicates the
+ * @param lastSnapshotId {@link Snapshot#CURRENT_STATE_ID} indicates the
* computation is in the current tree. Otherwise the id
- * indicates the computation range for a
+ * indicates the computation range for a
* {@link WithName} node.
* @return The subtree quota counts.
*/
@@ -560,7 +561,7 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
*/
public final String getLocalName() {
final byte[] name = getLocalNameBytes();
- return name == null? null: DFSUtil.bytes2String(name);
+ return name == null? null: new String(name, UTF_8);
}
@Override
@@ -593,7 +594,7 @@ public String getFullPathName() {
idx -= name.length;
System.arraycopy(name, 0, path, idx, name.length);
}
- return DFSUtil.bytes2String(path);
+ return new String(path, UTF_8);
}
public byte[][] getPathComponents() {
@@ -803,16 +804,16 @@ public final boolean equals(Object that) {
@Override
public final int hashCode() {
long id = getId();
- return (int)(id^(id>>>32));
+ return (int)(id^(id>>>32));
}
-
+
/**
* Dump the subtree starting from this inode.
* @return a text representation of the tree.
*/
@VisibleForTesting
public final StringBuffer dumpTreeRecursively() {
- final StringWriter out = new StringWriter();
+ final StringWriter out = new StringWriter();
dumpTreeRecursively(new PrintWriter(out, true), new StringBuilder(),
Snapshot.CURRENT_STATE_ID);
return out.getBuffer();
@@ -1022,7 +1023,7 @@ public BlocksMapUpdateInfo() {
toDeleteList = new ChunkedArrayList<>();
toUpdateReplicationInfo = new ChunkedArrayList<>();
}
-
+
/**
* @return The list of blocks that need to be removed from blocksMap
*/
@@ -1057,7 +1058,7 @@ public void clear() {
}
}
- /**
+ /**
* INode feature such as {@link FileUnderConstructionFeature}
* and {@link DirectoryWithQuotaFeature}.
*/
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java
index 2f9bc370daf84..9e6857ac216ae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java
@@ -17,12 +17,13 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
+import static java.nio.charset.StandardCharsets.UTF_8;
+
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
@@ -120,7 +121,7 @@ public INodeAttributes getAttributes(byte[][] components,
INodeAttributes inode) {
String[] elements = new String[components.length];
for (int i = 0; i < elements.length; i++) {
- elements[i] = DFSUtil.bytes2String(components[i]);
+ elements[i] = new String(components[i], UTF_8);
}
return getAttributes(elements, inode);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
index b6e27135feae4..693b47d20454a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
@@ -19,6 +19,7 @@
import java.io.FileNotFoundException;
import java.io.PrintWriter;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
@@ -62,20 +63,20 @@ public static INodeDirectory valueOf(INode inode, Object path
if (!inode.isDirectory()) {
throw new PathIsNotDirectoryException(DFSUtil.path2String(path));
}
- return inode.asDirectory();
+ return inode.asDirectory();
}
protected static final int DEFAULT_FILES_PER_DIRECTORY = 5;
- final static byte[] ROOT_NAME = DFSUtil.string2Bytes("");
+ final static byte[] ROOT_NAME = "".getBytes(UTF_8);
private List children = null;
-
+
/** constructor */
public INodeDirectory(long id, byte[] name, PermissionStatus permissions,
long mtime) {
super(id, name, permissions, mtime, 0L);
}
-
+
/**
* Copy constructor
* @param other The INodeDirectory to be copied
@@ -200,16 +201,16 @@ DirectoryWithQuotaFeature addDirectoryWithQuotaFeature(
int searchChildren(byte[] name) {
return children == null? -1: Collections.binarySearch(children, name);
}
-
+
public DirectoryWithSnapshotFeature addSnapshotFeature(
DirectoryDiffList diffs) {
- Preconditions.checkState(!isWithSnapshot(),
+ Preconditions.checkState(!isWithSnapshot(),
"Directory is already with snapshot");
DirectoryWithSnapshotFeature sf = new DirectoryWithSnapshotFeature(diffs);
addFeature(sf);
return sf;
}
-
+
/**
* If feature list contains a {@link DirectoryWithSnapshotFeature}, return it;
* otherwise, return null.
@@ -227,17 +228,17 @@ public DirectoryDiffList getDiffs() {
DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
return sf != null ? sf.getDiffs() : null;
}
-
+
@Override
public INodeDirectoryAttributes getSnapshotINode(int snapshotId) {
DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
return sf == null ? this : sf.getDiffs().getSnapshotINode(snapshotId, this);
}
-
+
@Override
public String toDetailString() {
DirectoryWithSnapshotFeature sf = this.getDirectoryWithSnapshotFeature();
- return super.toDetailString() + (sf == null ? "" : ", " + sf.getDiffs());
+ return super.toDetailString() + (sf == null ? "" : ", " + sf.getDiffs());
}
public DirectorySnapshottableFeature getDirectorySnapshottableFeature() {
@@ -300,7 +301,7 @@ public void removeSnapshottableFeature() {
}
}
- /**
+ /**
* Replace the given child with a new child. Note that we no longer need to
* replace an normal INodeDirectory or INodeFile into an
* INodeDirectoryWithSnapshot or INodeFileUnderConstruction. The only cases
@@ -315,25 +316,25 @@ public void replaceChild(INode oldChild, final INode newChild,
|| oldChild == children.get(i).asReference().getReferredINode()
.asReference().getReferredINode());
oldChild = children.get(i);
-
+
if (oldChild.isReference() && newChild.isReference()) {
// both are reference nodes, e.g., DstReference -> WithName
- final INodeReference.WithCount withCount =
+ final INodeReference.WithCount withCount =
(WithCount) oldChild.asReference().getReferredINode();
withCount.removeReference(oldChild.asReference());
}
children.set(i, newChild);
-
+
// replace the instance in the created list of the diff list
DirectoryWithSnapshotFeature sf = this.getDirectoryWithSnapshotFeature();
if (sf != null) {
sf.getDiffs().replaceChild(ListType.CREATED, oldChild, newChild);
}
-
+
// update the inodeMap
if (inodeMap != null) {
inodeMap.put(newChild);
- }
+ }
}
INodeReference.WithName replaceChild4ReferenceWithName(INode oldChild,
@@ -373,7 +374,7 @@ public void recordModification(int latestSnapshotId) {
/**
* Save the child to the latest snapshot.
- *
+ *
* @return the child inode, which may be replaced.
*/
public INode saveChild2Snapshot(final INode child, final int latestSnapshotId,
@@ -381,7 +382,7 @@ public INode saveChild2Snapshot(final INode child, final int latestSnapshotId,
if (latestSnapshotId == Snapshot.CURRENT_STATE_ID) {
return child;
}
-
+
// add snapshot feature if necessary
DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
if (sf == null) {
@@ -400,13 +401,13 @@ public INode saveChild2Snapshot(final INode child, final int latestSnapshotId,
*/
public INode getChild(byte[] name, int snapshotId) {
DirectoryWithSnapshotFeature sf;
- if (snapshotId == Snapshot.CURRENT_STATE_ID ||
+ if (snapshotId == Snapshot.CURRENT_STATE_ID ||
(sf = getDirectoryWithSnapshotFeature()) == null) {
ReadOnlyList c = getCurrentChildrenList();
final int i = ReadOnlyList.Util.binarySearch(c, name);
return i < 0 ? null : c.get(i);
}
-
+
return sf.getChild(this, name, snapshotId);
}
@@ -432,7 +433,7 @@ public int searchChild(INode inode) {
return Snapshot.CURRENT_STATE_ID;
}
}
-
+
/**
* @param snapshotId
* if it is not {@link Snapshot#CURRENT_STATE_ID}, get the result
@@ -450,7 +451,7 @@ public ReadOnlyList getChildrenList(final int snapshotId) {
}
return sf.getChildrenList(this, snapshotId);
}
-
+
private ReadOnlyList getCurrentChildrenList() {
return children == null ? ReadOnlyList.Util. emptyList()
: ReadOnlyList.Util.asReadOnlyList(children);
@@ -472,7 +473,7 @@ static int nextChild(ReadOnlyList children, byte[] name) {
}
return -nextPos;
}
-
+
/**
* Remove the specified child from this directory.
*/
@@ -487,13 +488,13 @@ public boolean removeChild(INode child, int latestSnapshotId) {
}
return removeChild(child);
}
-
- /**
+
+ /**
* Remove the specified child from this directory.
* The basic remove method which actually calls children.remove(..).
*
* @param child the child inode to be removed
- *
+ *
* @return true if the child is removed; false if the child is not found.
*/
public boolean removeChild(final INode child) {
@@ -509,12 +510,12 @@ public boolean removeChild(final INode child) {
/**
* Add a child inode to the directory.
- *
+ *
* @param node INode to insert
* @param setModTime set modification time for the parent node
- * not needed when replaying the addition and
+ * not needed when replaying the addition and
* the parent already has the proper mod time
- * @return false if the child with this name already exists;
+ * @return false if the child with this name already exists;
* otherwise, return true;
*/
public boolean addChild(INode node, final boolean setModTime,
@@ -586,7 +587,7 @@ public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
counts.addNameSpace(1);
return counts;
}
-
+
// compute the quota usage in the scope of the current directory tree
final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
if (useCache && q != null && q.isQuotaSet()) { // use the cached quota
@@ -612,7 +613,7 @@ private QuotaCounts computeDirectoryQuotaUsage(BlockStoragePolicySuite bsps,
return computeQuotaUsage4CurrentDirectory(bsps, blockStoragePolicyId,
counts);
}
-
+
/** Add quota usage for this inode excluding children. */
public QuotaCounts computeQuotaUsage4CurrentDirectory(
BlockStoragePolicySuite bsps, byte storagePolicyId, QuotaCounts counts) {
@@ -677,29 +678,29 @@ protected ContentSummaryComputationContext computeDirectoryContentSummary(
summary.yield();
return summary;
}
-
+
/**
* This method is usually called by the undo section of rename.
- *
+ *
* Before calling this function, in the rename operation, we replace the
* original src node (of the rename operation) with a reference node (WithName
* instance) in both the children list and a created list, delete the
* reference node from the children list, and add it to the corresponding
* deleted list.
- *
+ *
* To undo the above operations, we have the following steps in particular:
- *
+ *
*
- * 1) remove the WithName node from the deleted list (if it exists)
- * 2) replace the WithName node in the created list with srcChild
- * 3) add srcChild back as a child of srcParent. Note that we already add
+ * 1) remove the WithName node from the deleted list (if it exists)
+ * 2) replace the WithName node in the created list with srcChild
+ * 3) add srcChild back as a child of srcParent. Note that we already add
* the node into the created list of a snapshot diff in step 2, we do not need
* to add srcChild to the created list of the latest snapshot.
*
- *
- * We do not need to update quota usage because the old child is in the
- * deleted list before.
- *
+ *
+ * We do not need to update quota usage because the old child is in the
+ * deleted list before.
+ *
* @param oldChild
* The reference node to be removed/replaced
* @param newChild
@@ -714,11 +715,11 @@ public void undoRename4ScrParent(final INodeReference oldChild,
sf.getDiffs().replaceChild(ListType.CREATED, oldChild, newChild);
addChild(newChild, true, Snapshot.CURRENT_STATE_ID);
}
-
+
/**
* Undo the rename operation for the dst tree, i.e., if the rename operation
* (with OVERWRITE option) removes a file/dir from the dst tree, add it back
- * and delete possible record in the deleted list.
+ * and delete possible record in the deleted list.
*/
public void undoRename4DstParent(final BlockStoragePolicySuite bsps,
final INode deletedChild,
@@ -756,7 +757,7 @@ public void cleanSubtreeRecursively(
// in case of deletion snapshot, since this call happens after we modify
// the diff list, the snapshot to be deleted has been combined or renamed
// to its latest previous snapshot. (besides, we also need to consider nodes
- // created after prior but before snapshot. this will be done in
+ // created after prior but before snapshot. this will be done in
// DirectoryWithSnapshotFeature)
int s = snapshot != Snapshot.CURRENT_STATE_ID
&& prior != Snapshot.NO_SNAPSHOT_ID ? prior : snapshot;
@@ -785,7 +786,7 @@ public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
clear();
reclaimContext.removedINodes.add(this);
}
-
+
@Override
public void cleanSubtree(ReclaimContext reclaimContext, final int snapshotId,
int priorSnapshotId) {
@@ -813,7 +814,7 @@ public void cleanSubtree(ReclaimContext reclaimContext, final int snapshotId,
}
}
}
-
+
/**
* Compare the metadata with another INodeDirectory
*/
@@ -825,10 +826,10 @@ && getPermissionLong() == other.getPermissionLong()
&& getAclFeature() == other.getAclFeature()
&& getXAttrFeature() == other.getXAttrFeature();
}
-
+
/*
* The following code is to dump the tree recursively for testing.
- *
+ *
* \- foo (INodeDirectory@33dd2717)
* \- sub1 (INodeDirectory@442172)
* +- file1 (INodeFile@78392d4)
@@ -837,7 +838,7 @@ && getAclFeature() == other.getAclFeature()
* \- file3 (INodeFile@78392d6)
* \- z_file4 (INodeFile@45848712)
*/
- static final String DUMPTREE_EXCEPT_LAST_ITEM = "+-";
+ static final String DUMPTREE_EXCEPT_LAST_ITEM = "+-";
static final String DUMPTREE_LAST_ITEM = "\\-";
@VisibleForTesting
@Override
@@ -860,7 +861,7 @@ public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
}
dumpTreeRecursively(out, prefix, new Iterable() {
final Iterator i = getChildrenList(snapshot).iterator();
-
+
@Override
public Iterator iterator() {
return new Iterator() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
index 1223f4ef86cae..d561cd5df89af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
@@ -18,10 +18,10 @@
package org.apache.hadoop.hdfs.server.namenode;
import java.io.PrintWriter;
+import static java.nio.charset.StandardCharsets.UTF_8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
@@ -35,9 +35,9 @@ public class INodeSymlink extends INodeWithAdditionalFields {
INodeSymlink(long id, byte[] name, PermissionStatus permissions,
long mtime, long atime, String symlink) {
super(id, name, permissions, mtime, atime);
- this.symlink = DFSUtil.string2Bytes(symlink);
+ this.symlink = symlink.getBytes(UTF_8);
}
-
+
INodeSymlink(INodeSymlink that) {
super(that);
this.symlink = that.symlink;
@@ -64,13 +64,13 @@ public INodeSymlink asSymlink() {
}
public String getSymlinkString() {
- return DFSUtil.bytes2String(symlink);
+ return new String(symlink, UTF_8);
}
public byte[] getSymlink() {
return symlink;
}
-
+
@Override
public void cleanSubtree(ReclaimContext reclaimContext, final int snapshotId,
int priorSnapshotId) {
@@ -79,7 +79,7 @@ public void cleanSubtree(ReclaimContext reclaimContext, final int snapshotId,
destroyAndCollectBlocks(reclaimContext);
}
}
-
+
@Override
public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
reclaimContext.removedINodes.add(this);
@@ -122,12 +122,12 @@ public void addAclFeature(AclFeature f) {
final XAttrFeature getXAttrFeature(int snapshotId) {
throw new UnsupportedOperationException("XAttrs are not supported on symlinks");
}
-
+
@Override
public void removeXAttrFeature() {
throw new UnsupportedOperationException("XAttrs are not supported on symlinks");
}
-
+
@Override
public void addXAttrFeature(XAttrFeature f) {
throw new UnsupportedOperationException("XAttrs are not supported on symlinks");
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
index b37321d6243ea..050fb39c8eeae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
@@ -21,6 +21,7 @@
import java.util.Collections;
import java.util.List;
import java.util.NoSuchElementException;
+import static java.nio.charset.StandardCharsets.UTF_8;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -84,13 +85,13 @@ static INodesInPath fromComponents(byte[][] components) {
* Example:
* Given the path /c1/c2/c3 where only /c1/c2 exists, resulting in the
* following path components: ["","c1","c2","c3"]
- *
+ *
*
* getExistingPathINodes(["","c1","c2"]) should fill
* the array with [rootINode,c1,c2],
* getExistingPathINodes(["","c1","c2","c3"]) should
* fill the array with [rootINode,c1,c2,null]
- *
+ *
* @param startingDir the starting directory
* @param components array of path component name
* @return the specified number of existing INodes in the path
@@ -127,13 +128,13 @@ static INodesInPath resolve(final INodeDirectory startingDir,
} else if (isRef && isDir && !lastComp) {
// If the curNode is a reference node, need to check its dstSnapshot:
// 1. if the existing snapshot is no later than the dstSnapshot (which
- // is the latest snapshot in dst before the rename), the changes
+ // is the latest snapshot in dst before the rename), the changes
// should be recorded in previous snapshots (belonging to src).
- // 2. however, if the ref node is already the last component, we still
- // need to know the latest snapshot among the ref node's ancestors,
+ // 2. however, if the ref node is already the last component, we still
+ // need to know the latest snapshot among the ref node's ancestors,
// in case of processing a deletion operation. Thus we do not overwrite
// the latest snapshot if lastComp is true. In case of the operation is
- // a modification operation, we do a similar check in corresponding
+ // a modification operation, we do a similar check in corresponding
// recordModification method.
if (!isSnapshot) {
int dstSnapshotId = curNode.asReference().getDstSnapshotId();
@@ -142,7 +143,7 @@ static INodesInPath resolve(final INodeDirectory startingDir,
dstSnapshotId >= snapshotId)) { // the above scenario
int lastSnapshot = CURRENT_STATE_ID;
DirectoryWithSnapshotFeature sf;
- if (curNode.isDirectory() &&
+ if (curNode.isDirectory() &&
(sf = curNode.asDirectory().getDirectoryWithSnapshotFeature()) != null) {
lastSnapshot = sf.getLastSnapshotId();
}
@@ -175,8 +176,9 @@ static INodesInPath resolve(final INodeDirectory startingDir,
// preserved so a path can be reconstructed.
byte[][] componentsCopy =
Arrays.copyOf(components, components.length - 1);
- componentsCopy[count] = DFSUtil.string2Bytes(
- DFSUtil.byteArray2PathString(components, count, 2));
+ componentsCopy[count] = DFSUtil.byteArray2PathString(
+ components, count, 2).getBytes(UTF_8);
+
// shift the remaining components after snapshot name
int start = count + 2;
System.arraycopy(components, start, componentsCopy, count + 1,
@@ -256,8 +258,8 @@ public static INodesInPath append(INodesInPath iip, INode child,
private final boolean isRaw;
/**
- * For snapshot paths, it is the id of the snapshot; or
- * {@link Snapshot#CURRENT_STATE_ID} if the snapshot does not exist. For
+ * For snapshot paths, it is the id of the snapshot; or
+ * {@link Snapshot#CURRENT_STATE_ID} if the snapshot does not exist. For
* non-snapshot paths, it is the id of the latest snapshot found in the path;
* or {@link Snapshot#CURRENT_STATE_ID} if no snapshot is found.
*/
@@ -285,7 +287,7 @@ public int getLatestSnapshotId() {
Preconditions.checkState(!isSnapshot);
return snapshotId;
}
-
+
/**
* For snapshot paths, return the id of the snapshot specified in the path.
* For non-snapshot paths, return {@link Snapshot#CURRENT_STATE_ID}.
@@ -309,7 +311,7 @@ public INode getINode(int i) {
throw new NoSuchElementException("inodes.length == " + inodes.length);
}
}
-
+
/** @return the last inode. */
public INode getLastINode() {
return getINode(-1);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
index fa7bacee84dcc..67c6674d0ef14 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hdfs.server.namenode.snapshot;
import java.io.PrintWriter;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
@@ -26,7 +27,6 @@
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.protocol.SnapshotException;
import org.apache.hadoop.hdfs.server.namenode.Content;
@@ -118,12 +118,12 @@ public void renameSnapshot(String path, String oldName, String newName)
if (newName.equals(oldName)) {
return;
}
- final int indexOfOld = searchSnapshot(DFSUtil.string2Bytes(oldName));
+ final int indexOfOld = searchSnapshot(oldName.getBytes(UTF_8));
if (indexOfOld < 0) {
throw new SnapshotException("The snapshot " + oldName
+ " does not exist for directory " + path);
} else {
- final byte[] newNameBytes = DFSUtil.string2Bytes(newName);
+ final byte[] newNameBytes = newName.getBytes(UTF_8);
int indexOfNew = searchSnapshot(newNameBytes);
if (indexOfNew >= 0) {
throw new SnapshotException("The snapshot " + newName
@@ -204,7 +204,7 @@ public Snapshot addSnapshot(INodeDirectory snapshotRoot, int id, String name)
public Snapshot removeSnapshot(
INode.ReclaimContext reclaimContext, INodeDirectory snapshotRoot,
String snapshotName) throws SnapshotException {
- final int i = searchSnapshot(DFSUtil.string2Bytes(snapshotName));
+ final int i = searchSnapshot(snapshotName.getBytes(UTF_8));
if (i < 0) {
throw new SnapshotException("Cannot delete snapshot " + snapshotName
+ " from path " + snapshotRoot.getFullPathName()
@@ -268,7 +268,7 @@ private Snapshot getSnapshotByName(INodeDirectory snapshotRoot,
String snapshotName) throws SnapshotException {
Snapshot s = null;
if (snapshotName != null && !snapshotName.isEmpty()) {
- final int index = searchSnapshot(DFSUtil.string2Bytes(snapshotName));
+ final int index = searchSnapshot(snapshotName.getBytes(UTF_8));
if (index < 0) {
throw new SnapshotException("Cannot find the snapshot of directory "
+ snapshotRoot.getFullPathName() + " with name " + snapshotName);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
index 832a3391f5b0d..0342973d54568 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
@@ -20,6 +20,7 @@
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.text.SimpleDateFormat;
import java.util.Arrays;
import java.util.Comparator;
@@ -27,7 +28,6 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.namenode.AclFeature;
import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
@@ -50,13 +50,13 @@ public class Snapshot implements Comparable {
*/
public static final int CURRENT_STATE_ID = Integer.MAX_VALUE - 1;
public static final int NO_SNAPSHOT_ID = -1;
-
+
/**
* The pattern for generating the default snapshot name.
* E.g. s20130412-151029.033
*/
private static final String DEFAULT_SNAPSHOT_NAME_PATTERN = "'s'yyyyMMdd-HHmmss.SSS";
-
+
public static String generateDefaultSnapshotName() {
return new SimpleDateFormat(DEFAULT_SNAPSHOT_NAME_PATTERN).format(new Date());
}
@@ -72,7 +72,7 @@ public static String getSnapshotPath(String snapshottableDir,
.append(snapshotRelativePath)
.toString();
}
-
+
/**
* Get the name of the given snapshot.
* @param s The given snapshot.
@@ -81,7 +81,7 @@ public static String getSnapshotPath(String snapshottableDir,
static String getSnapshotName(Snapshot s) {
return s != null ? s.getRoot().getLocalName() : "";
}
-
+
public static int getSnapshotId(Snapshot s) {
return s == null ? CURRENT_STATE_ID : s.getId();
}
@@ -107,7 +107,7 @@ public int compare(Snapshot left, Snapshot right) {
= new Comparator() {
@Override
public int compare(Integer left, Integer right) {
- // Snapshot.CURRENT_STATE_ID means the current state, thus should be the
+ // Snapshot.CURRENT_STATE_ID means the current state, thus should be the
// largest
return left - right;
}
@@ -116,12 +116,12 @@ public int compare(Integer left, Integer right) {
/**
* Find the latest snapshot that 1) covers the given inode (which means the
* snapshot was either taken on the inode or taken on an ancestor of the
- * inode), and 2) was taken before the given snapshot (if the given snapshot
+ * inode), and 2) was taken before the given snapshot (if the given snapshot
* is not null).
- *
+ *
* @param inode the given inode that the returned snapshot needs to cover
* @param anchor the returned snapshot should be taken before this given id.
- * @return id of the latest snapshot that covers the given inode and was taken
+ * @return id of the latest snapshot that covers the given inode and was taken
* before the the given snapshot (if it is not null).
*/
public static int findLatestSnapshot(INode inode, final int anchor) {
@@ -136,7 +136,7 @@ public static int findLatestSnapshot(INode inode, final int anchor) {
}
return latest;
}
-
+
static Snapshot read(DataInput in, FSImageFormat.Loader loader)
throws IOException {
final int snapshotId = in.readInt();
@@ -153,13 +153,13 @@ static public class Root extends INodeDirectory {
@Override
public boolean apply(Feature input) {
- if (AclFeature.class.isInstance(input)
+ if (AclFeature.class.isInstance(input)
|| XAttrFeature.class.isInstance(input)) {
return true;
}
return false;
}
-
+
}))
.toArray(new Feature[0]));
}
@@ -202,7 +202,7 @@ public String getRootFullPathName() {
Snapshot(int id, String name, INodeDirectory dir) {
this(id, dir, dir);
- this.root.setLocalName(DFSUtil.string2Bytes(name));
+ this.root.setLocalName(name.getBytes(UTF_8));
}
Snapshot(int id, INodeDirectory dir, INodeDirectory parent) {
@@ -210,7 +210,7 @@ public String getRootFullPathName() {
this.root = new Root(dir);
this.root.setParent(parent);
}
-
+
public int getId() {
return id;
}
@@ -224,7 +224,7 @@ public Root getRoot() {
public int compareTo(byte[] bytes) {
return root.compareTo(bytes);
}
-
+
@Override
public boolean equals(Object that) {
if (this == that) {
@@ -234,12 +234,12 @@ public boolean equals(Object that) {
}
return this.id == ((Snapshot)that).id;
}
-
+
@Override
public int hashCode() {
return id;
}
-
+
@Override
public String toString() {
return getClass().getSimpleName() + "." + root.getLocalName() + "(id=" + id + ")";
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
index fcab53a858181..cc0cb6959e0cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
@@ -20,12 +20,12 @@
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -126,11 +126,11 @@ private static FileDiff loadFileDiff(FileDiff posterior, DataInput in,
// 2. Load file size
final long fileSize = in.readLong();
-
- // 3. Load snapshotINode
+
+ // 3. Load snapshotINode
final INodeFileAttributes snapshotINode = in.readBoolean()?
loader.loadINodeFileAttributes(in): null;
-
+
return new FileDiff(snapshot.getId(), snapshotINode, posterior, fileSize);
}
@@ -150,18 +150,18 @@ public static INode loadCreated(byte[] createdNodeName,
if (d != null) {
return d;
} // else go to the next SnapshotDiff
- }
+ }
// use the current child
INode currentChild = parent.getChild(createdNodeName,
Snapshot.CURRENT_STATE_ID);
if (currentChild == null) {
throw new IOException("Cannot find an INode associated with the INode "
- + DFSUtil.bytes2String(createdNodeName)
+ + new String(createdNodeName, UTF_8)
+ " in created list while loading FSImage.");
}
return currentChild;
}
-
+
/**
* Load the created list from fsimage.
* @param parent The directory that the created list belongs to.
@@ -180,12 +180,12 @@ private static List loadCreatedList(INodeDirectory parent,
}
return createdList;
}
-
+
/**
* Load the deleted list from the fsimage.
- *
+ *
* @param parent The directory that the deleted list belongs to.
- * @param createdList The created list associated with the deleted list in
+ * @param createdList The created list associated with the deleted list in
* the same Diff.
* @param in The {@link DataInput} to read.
* @param loader The {@link Loader} instance.
@@ -199,8 +199,8 @@ private static List loadDeletedList(INodeDirectory parent,
for (int i = 0; i < deletedSize; i++) {
final INode deleted = loader.loadINodeWithLocalName(true, in, true);
deletedList.add(deleted);
- // set parent: the parent field of an INode in the deleted list is not
- // useful, but set the parent here to be consistent with the original
+ // set parent: the parent field of an INode in the deleted list is not
+ // useful, but set the parent here to be consistent with the original
// fsdir tree.
deleted.setParent(parent);
if (deleted.isFile()) {
@@ -209,7 +209,7 @@ private static List loadDeletedList(INodeDirectory parent,
}
return deletedList;
}
-
+
/**
* Load snapshots and snapshotQuota for a Snapshottable directory.
*
@@ -270,7 +270,7 @@ private static INodeDirectoryAttributes loadSnapshotINodeInDirectoryDiff(
Snapshot snapshot, DataInput in, FSImageFormat.Loader loader)
throws IOException {
// read the boolean indicating whether snapshotINode == Snapshot.Root
- boolean useRoot = in.readBoolean();
+ boolean useRoot = in.readBoolean();
if (useRoot) {
return snapshot.getRoot();
} else {
@@ -278,12 +278,12 @@ private static INodeDirectoryAttributes loadSnapshotINodeInDirectoryDiff(
return in.readBoolean()? loader.loadINodeDirectoryAttributes(in): null;
}
}
-
+
/**
* Load {@link DirectoryDiff} from fsimage.
* @param parent The directory that the SnapshotDiff belongs to.
* @param in The {@link DataInput} instance to read.
- * @param loader The {@link Loader} instance that this loading procedure is
+ * @param loader The {@link Loader} instance that this loading procedure is
* using.
* @return A {@link DirectoryDiff}.
*/
@@ -294,17 +294,17 @@ private static DirectoryDiff loadDirectoryDiff(INodeDirectory parent,
// 2. Load DirectoryDiff#childrenSize
int childrenSize = in.readInt();
-
- // 3. Load DirectoryDiff#snapshotINode
+
+ // 3. Load DirectoryDiff#snapshotINode
INodeDirectoryAttributes snapshotINode = loadSnapshotINodeInDirectoryDiff(
snapshot, in, loader);
-
+
// 4. Load the created list in SnapshotDiff#Diff
List createdList = loadCreatedList(parent, in);
-
+
// 5. Load the deleted list in SnapshotDiff#Diff
List deletedList = loadDeletedList(parent, createdList, in, loader);
-
+
// 6. Compose the SnapshotDiff
List diffs = parent.getDiffs().asList();
DirectoryDiff sdiff = new DirectoryDiff(snapshot.getId(), snapshotINode,
@@ -312,7 +312,7 @@ private static DirectoryDiff loadDirectoryDiff(INodeDirectory parent,
deletedList, snapshotINode == snapshot.getRoot());
return sdiff;
}
-
+
/** A reference map for fsimage serialization. */
public static class ReferenceMap {
@@ -322,7 +322,7 @@ public static class ReferenceMap {
private final Map referenceMap
= new HashMap();
/**
- * Used to record whether the subtree of the reference node has been saved
+ * Used to record whether the subtree of the reference node has been saved
*/
private final Map dirMap = new HashMap();
@@ -342,7 +342,7 @@ public void writeINodeReferenceWithCount(
out.writeLong(id);
}
}
-
+
public boolean toProcessSubtree(long id) {
if (dirMap.containsKey(id)) {
return false;
@@ -351,7 +351,7 @@ public boolean toProcessSubtree(long id) {
return true;
}
}
-
+
public INodeReference.WithCount loadINodeReferenceWithCount(
boolean isSnapshotINode, DataInput in, FSImageFormat.Loader loader
) throws IOException {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
index 8ad78249a8964..bc1dc7829536e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
@@ -20,6 +20,7 @@
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@@ -29,7 +30,6 @@
import javax.management.ObjectName;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.SnapshotException;
@@ -49,11 +49,11 @@
/**
* Manage snapshottable directories and their snapshots.
- *
+ *
* This class includes operations that create, access, modify snapshots and/or
* snapshot-related data. In general, the locking structure of snapshot
* operations is:
- *
+ *
* 1. Lock the {@link FSNamesystem} lock in {@link FSNamesystem} before calling
* into {@link SnapshotManager} methods.
* 2. Lock the {@link FSDirectory} lock for the {@link SnapshotManager} methods
@@ -67,7 +67,7 @@ public class SnapshotManager implements SnapshotStatsMXBean {
private final AtomicInteger numSnapshots = new AtomicInteger();
private int snapshotCounter = 0;
-
+
/** All snapshottable directories in the namesystem. */
private final Map snapshottables =
new HashMap();
@@ -123,7 +123,7 @@ public void setSnapshottable(final String path, boolean checkNestedSnapshottable
}
addSnapshottable(d);
}
-
+
/** Add the given snapshottable directory to {@link #snapshottables}. */
public void addSnapshottable(INodeDirectory dir) {
Preconditions.checkArgument(dir.isSnapshottable());
@@ -134,7 +134,7 @@ public void addSnapshottable(INodeDirectory dir) {
private void removeSnapshottable(INodeDirectory s) {
snapshottables.remove(s.getId());
}
-
+
/** Remove snapshottable directories from {@link #snapshottables} */
public void removeSnapshottable(List toRemove) {
if (toRemove != null) {
@@ -146,7 +146,7 @@ public void removeSnapshottable(List toRemove) {
/**
* Set the given snapshottable directory to non-snapshottable.
- *
+ *
* @throws SnapshotException if there are snapshots in the directory.
*/
public void resetSnapshottable(final String path) throws IOException {
@@ -217,13 +217,13 @@ public String createSnapshot(final INodesInPath iip, String snapshotRoot,
}
srcRoot.addSnapshot(snapshotCounter, snapshotName);
-
+
//create success, update id
snapshotCounter++;
numSnapshots.getAndIncrement();
return Snapshot.getSnapshotPath(snapshotRoot, snapshotName);
}
-
+
/**
* Delete a snapshot for a snapshottable directory
* @param snapshotName Name of the snapshot to be deleted
@@ -255,7 +255,7 @@ public void renameSnapshot(final INodesInPath iip, final String snapshotRoot,
final INodeDirectory srcRoot = getSnapshottableRoot(iip);
srcRoot.renameSnapshot(snapshotRoot, oldSnapshotName, newSnapshotName);
}
-
+
public int getNumSnapshottableDirs() {
return snapshottables.size();
}
@@ -297,7 +297,7 @@ public void write(DataOutput out) throws IOException {
}
}
}
-
+
/**
* Read values of {@link #snapshotCounter}, {@link #numSnapshots}, and
* all snapshots from the DataInput
@@ -306,7 +306,7 @@ public Map read(DataInput in, FSImageFormat.Loader loader
) throws IOException {
snapshotCounter = in.readInt();
numSnapshots.set(in.readInt());
-
+
// read snapshots
final Map snapshotMap = new HashMap();
for(int i = 0; i < numSnapshots.get(); i++) {
@@ -315,7 +315,7 @@ public Map read(DataInput in, FSImageFormat.Loader loader
}
return snapshotMap;
}
-
+
/**
* List all the snapshottable directories that are owned by the current user.
* @param userName Current user name.
@@ -328,20 +328,20 @@ public SnapshottableDirectoryStatus[] getSnapshottableDirListing(
if (snapshottables.isEmpty()) {
return null;
}
-
- List statusList =
+
+ List statusList =
new ArrayList();
for (INodeDirectory dir : snapshottables.values()) {
if (userName == null || userName.equals(dir.getUserName())) {
SnapshottableDirectoryStatus status = new SnapshottableDirectoryStatus(
dir.getModificationTime(), dir.getAccessTime(),
dir.getFsPermission(), dir.getUserName(), dir.getGroupName(),
- dir.getLocalNameBytes(), dir.getId(),
+ dir.getLocalNameBytes(), dir.getId(),
dir.getChildrenNum(Snapshot.CURRENT_STATE_ID),
dir.getDirectorySnapshottableFeature().getNumSnapshots(),
dir.getDirectorySnapshottableFeature().getSnapshotQuota(),
dir.getParent() == null ? DFSUtilClient.EMPTY_BYTES :
- DFSUtil.string2Bytes(dir.getParent().getFullPathName()));
+ dir.getParent().getFullPathName().getBytes(UTF_8));
statusList.add(status);
}
}
@@ -349,7 +349,7 @@ public SnapshottableDirectoryStatus[] getSnapshottableDirListing(
return statusList.toArray(
new SnapshottableDirectoryStatus[statusList.size()]);
}
-
+
/**
* Compute the difference between two snapshots of a directory, or between a
* snapshot of the directory and its current tree.
@@ -372,7 +372,7 @@ public SnapshotDiffReport diff(final INodesInPath iip,
return diffs != null ? diffs.generateReport() : new SnapshotDiffReport(
snapshotRootPath, from, to, Collections. emptyList());
}
-
+
public void clearSnapshottableDirs() {
snapshottables.clear();
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 7f26b03f92ba3..0b6ec9db51491 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -45,6 +45,7 @@
import java.io.InterruptedIOException;
import java.io.PrintStream;
import java.io.RandomAccessFile;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
import java.net.HttpURLConnection;
@@ -177,18 +178,18 @@
public class DFSTestUtil {
private static final Log LOG = LogFactory.getLog(DFSTestUtil.class);
-
+
private static final Random gen = new Random();
private static final String[] dirNames = {
"zero", "one", "two", "three", "four", "five", "six", "seven", "eight", "nine"
};
-
+
private final int maxLevels;
private final int maxSize;
private final int minSize;
private final int nFiles;
private MyFile[] files;
-
+
/** Creates a new instance of DFSTestUtil
*
* @param nFiles Number of files to be created
@@ -218,7 +219,7 @@ public DFSTestUtil(String testName, int nFiles, int maxLevels, int maxSize,
this.maxSize = maxSize;
this.minSize = minSize;
}
-
+
/**
* when formatting a namenode - we must provide clusterid.
* @param conf
@@ -282,11 +283,11 @@ public static void setEditLogForTesting(FSNamesystem fsn, FSEditLog newLog) {
* a single file.
*/
private class MyFile {
-
+
private String name = "";
private final int size;
private final long seed;
-
+
MyFile() {
int nLevels = gen.nextInt(maxLevels);
if (nLevels != 0) {
@@ -307,7 +308,7 @@ private class MyFile {
size = minSize + gen.nextInt(maxSize - minSize);
seed = gen.nextLong();
}
-
+
String getName() { return name; }
int getSize() { return size; }
long getSeed() { return seed; }
@@ -323,20 +324,20 @@ public static byte[] readFileAsBytes(FileSystem fs, Path fileName) throws IOExce
return os.toByteArray();
}
}
-
+
/** create nFiles with random names and directory hierarchies
* with random (but reproducible) data in them.
*/
public void createFiles(FileSystem fs, String topdir,
short replicationFactor) throws IOException {
files = new MyFile[nFiles];
-
+
for (int idx = 0; idx < nFiles; idx++) {
files[idx] = new MyFile();
}
-
+
Path root = new Path(topdir);
-
+
for (int idx = 0; idx < nFiles; idx++) {
createFile(fs, new Path(root, files[idx].getName()), files[idx].getSize(),
replicationFactor, files[idx].getSeed());
@@ -349,7 +350,7 @@ public static String readFile(FileSystem fs, Path fileName)
return new String(buf, 0, buf.length);
}
- public static byte[] readFileBuffer(FileSystem fs, Path fileName)
+ public static byte[] readFileBuffer(FileSystem fs, Path fileName)
throws IOException {
try (ByteArrayOutputStream os = new ByteArrayOutputStream();
FSDataInputStream in = fs.open(fileName)) {
@@ -357,13 +358,13 @@ public static byte[] readFileBuffer(FileSystem fs, Path fileName)
return os.toByteArray();
}
}
-
- public static void createFile(FileSystem fs, Path fileName, long fileLen,
+
+ public static void createFile(FileSystem fs, Path fileName, long fileLen,
short replFactor, long seed) throws IOException {
createFile(fs, fileName, 1024, fileLen, fs.getDefaultBlockSize(fileName),
replFactor, seed);
}
-
+
public static void createFile(FileSystem fs, Path fileName, int bufferLen,
long fileLen, long blockSize, short replFactor, long seed)
throws IOException {
@@ -418,18 +419,18 @@ public static void createFile(FileSystem fs, Path fileName,
}
}
}
-
+
public static byte[] calculateFileContentsFromSeed(long seed, int length) {
Random rb = new Random(seed);
byte val[] = new byte[length];
rb.nextBytes(val);
return val;
}
-
+
/** check if the files have been copied correctly. */
public boolean checkFiles(FileSystem fs, String topdir) throws IOException {
Path root = new Path(topdir);
-
+
for (int idx = 0; idx < nFiles; idx++) {
Path fPath = new Path(root, files[idx].getName());
try (FSDataInputStream in = fs.open(fPath)) {
@@ -445,11 +446,11 @@ public boolean checkFiles(FileSystem fs, String topdir) throws IOException {
}
}
}
-
+
return true;
}
- void setReplication(FileSystem fs, String topdir, short value)
+ void setReplication(FileSystem fs, String topdir, short value)
throws IOException {
Path root = new Path(topdir);
for (int idx = 0; idx < nFiles; idx++) {
@@ -462,7 +463,7 @@ void setReplication(FileSystem fs, String topdir, short value)
* Waits for the replication factor of all files to reach the
* specified target.
*/
- public void waitReplication(FileSystem fs, String topdir, short value)
+ public void waitReplication(FileSystem fs, String topdir, short value)
throws IOException, InterruptedException, TimeoutException {
Path root = new Path(topdir);
@@ -582,7 +583,7 @@ public static void waitCorruptReplicas(FileSystem fs, FSNamesystem ns,
/*
* Wait up to 20s for the given DN (IP:port) to be decommissioned
*/
- public static void waitForDecommission(FileSystem fs, String name)
+ public static void waitForDecommission(FileSystem fs, String name)
throws IOException, InterruptedException, TimeoutException {
DatanodeInfo dn = null;
int count = 0;
@@ -646,10 +647,10 @@ public static long getDatanodeCapacity(DatanodeManager dm, int index) {
}
/*
- * Wait for the given # live/dead DNs, total capacity, and # vol failures.
+ * Wait for the given # live/dead DNs, total capacity, and # vol failures.
*/
- public static void waitForDatanodeStatus(DatanodeManager dm, int expectedLive,
- int expectedDead, long expectedVolFails, long expectedTotalCapacity,
+ public static void waitForDatanodeStatus(DatanodeManager dm, int expectedLive,
+ int expectedDead, long expectedVolFails, long expectedTotalCapacity,
long timeout) throws InterruptedException, TimeoutException {
final List live = new ArrayList();
final List dead = new ArrayList();
@@ -689,7 +690,7 @@ public static void waitForDatanodeStatus(DatanodeManager dm, int expectedLive,
/*
* Wait for the given DN to consider itself dead.
*/
- public static void waitForDatanodeDeath(DataNode dn)
+ public static void waitForDatanodeDeath(DataNode dn)
throws InterruptedException, TimeoutException {
final int ATTEMPTS = 10;
int count = 0;
@@ -702,7 +703,7 @@ public static void waitForDatanodeDeath(DataNode dn)
throw new TimeoutException("Timed out waiting for DN to die");
}
}
-
+
/** return list of filenames created as part of createFiles */
public String[] getFileNames(String topDir) {
if (nFiles == 0)
@@ -754,20 +755,20 @@ public static void waitReplication(FileSystem fs, Path fileName, short replFacto
" to reach " + replFactor + " replicas");
}
}
-
+
/** delete directory and everything underneath it.*/
public void cleanup(FileSystem fs, String topdir) throws IOException {
Path root = new Path(topdir);
fs.delete(root, true);
files = null;
}
-
+
public static ExtendedBlock getFirstBlock(FileSystem fs, Path path) throws IOException {
try (HdfsDataInputStream in = (HdfsDataInputStream) fs.open(path)) {
in.readByte();
return in.getCurrentBlock();
}
- }
+ }
public static List getAllBlocks(FSDataInputStream in)
throws IOException {
@@ -823,7 +824,7 @@ public static void writeFile(FileSystem fs, Path p, String s)
}
/* Append the given string to the given file */
- public static void appendFile(FileSystem fs, Path p, String s)
+ public static void appendFile(FileSystem fs, Path p, String s)
throws IOException {
assert fs.exists(p);
try (InputStream is = new ByteArrayInputStream(s.getBytes());
@@ -831,7 +832,7 @@ public static void appendFile(FileSystem fs, Path p, String s)
IOUtils.copyBytes(is, os, s.length());
}
}
-
+
/**
* Append specified length of bytes to a given file
* @param fs The file system
@@ -850,30 +851,30 @@ public static void appendFile(FileSystem fs, Path p, int length)
out.write(toAppend);
}
}
-
+
/**
* @return url content as string (UTF-8 encoding assumed)
*/
public static String urlGet(URL url) throws IOException {
return new String(urlGetBytes(url), Charsets.UTF_8);
}
-
+
/**
* @return URL contents as a byte array
*/
public static byte[] urlGetBytes(URL url) throws IOException {
URLConnection conn = url.openConnection();
HttpURLConnection hc = (HttpURLConnection)conn;
-
+
assertEquals(HttpURLConnection.HTTP_OK, hc.getResponseCode());
ByteArrayOutputStream out = new ByteArrayOutputStream();
IOUtils.copyBytes(conn.getInputStream(), out, 4096, true);
return out.toByteArray();
}
-
+
/**
* mock class to get group mapping for fake users
- *
+ *
*/
static class MockUnixGroupsMapping extends ShellBasedUnixGroupsMapping {
static Map fakeUser2GroupsMap;
@@ -883,14 +884,14 @@ static class MockUnixGroupsMapping extends ShellBasedUnixGroupsMapping {
defaultGroups.add("supergroup");
fakeUser2GroupsMap = new HashMap();
}
-
+
@Override
public List getGroups(String user) throws IOException {
boolean found = false;
-
+
// check to see if this is one of fake users
List l = new ArrayList();
- for(String u : fakeUser2GroupsMap.keySet()) {
+ for(String u : fakeUser2GroupsMap.keySet()) {
if(user.equals(u)) {
found = true;
for(String gr : fakeUser2GroupsMap.get(u)) {
@@ -898,12 +899,12 @@ public List getGroups(String user) throws IOException {
}
}
}
-
+
// default
if(!found) {
l = super.getGroups(user);
if(l.size() == 0) {
- System.out.println("failed to get real group for " + user +
+ System.out.println("failed to get real group for " + user +
"; using default");
return defaultGroups;
}
@@ -911,7 +912,7 @@ public List getGroups(String user) throws IOException {
return l;
}
}
-
+
/**
* update the configuration with fake class for mapping user to groups
* @param conf
@@ -922,18 +923,18 @@ public List getGroups(String user) throws IOException {
if(map!=null) {
MockUnixGroupsMapping.fakeUser2GroupsMap = map;
}
-
+
// fake mapping user to groups
conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
DFSTestUtil.MockUnixGroupsMapping.class,
ShellBasedUnixGroupsMapping.class);
-
+
}
-
+
/**
* Get a FileSystem instance as specified user in a doAs block.
*/
- static public FileSystem getFileSystemAs(UserGroupInformation ugi,
+ static public FileSystem getFileSystemAs(UserGroupInformation ugi,
final Configuration conf) throws IOException {
try {
return ugi.doAs(new PrivilegedExceptionAction() {
@@ -956,7 +957,7 @@ public static byte[] generateSequentialBytes(int start, int length) {
return result;
}
-
+
public static Statistics getStatistics(FileSystem fs) {
return FileSystem.getStatistics(fs.getUri().getScheme(), fs.getClass());
}
@@ -974,7 +975,7 @@ public static byte[] loadFile(String filename) throws IOException {
}
/** For {@link TestTransferRbw} */
- public static BlockOpResponseProto transferRbw(final ExtendedBlock b,
+ public static BlockOpResponseProto transferRbw(final ExtendedBlock b,
final DFSClient dfsClient, final DatanodeInfo... datanodes) throws IOException {
assertEquals(2, datanodes.length);
final long writeTimeout = dfsClient.getDatanodeWriteTimeout(datanodes.length);
@@ -993,7 +994,7 @@ public static BlockOpResponseProto transferRbw(final ExtendedBlock b,
return BlockOpResponseProto.parseDelimitedFrom(in);
}
}
-
+
public static void setFederatedConfiguration(MiniDFSCluster cluster,
Configuration conf) {
Set nameservices = new HashSet();
@@ -1041,7 +1042,7 @@ public static void setFederatedHAConfiguration(MiniDFSCluster cluster,
conf.set(DFSConfigKeys.DFS_NAMESERVICES, Joiner.on(",")
.join(nameservices.keySet()));
}
-
+
private static DatanodeID getDatanodeID(String ipAddr) {
return new DatanodeID(ipAddr, "localhost",
UUID.randomUUID().toString(),
@@ -1072,12 +1073,12 @@ public static DatanodeInfo getLocalDatanodeInfo() {
public static DatanodeInfo getDatanodeInfo(String ipAddr) {
return new DatanodeInfo(getDatanodeID(ipAddr));
}
-
+
public static DatanodeInfo getLocalDatanodeInfo(int port) {
return new DatanodeInfo(getLocalDatanodeID(port));
}
- public static DatanodeInfo getDatanodeInfo(String ipAddr,
+ public static DatanodeInfo getDatanodeInfo(String ipAddr,
String host, int port) {
return new DatanodeInfo(new DatanodeID(ipAddr, host,
UUID.randomUUID().toString(), port,
@@ -1101,10 +1102,10 @@ public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr,
return getDatanodeDescriptor(ipAddr, DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT,
rackLocation);
}
-
+
public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr,
String rackLocation, String hostname) {
- return getDatanodeDescriptor(ipAddr,
+ return getDatanodeDescriptor(ipAddr,
DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT, rackLocation, hostname);
}
@@ -1112,15 +1113,15 @@ public static DatanodeStorageInfo createDatanodeStorageInfo(
String storageID, String ip) {
return createDatanodeStorageInfo(storageID, ip, "defaultRack", "host");
}
-
+
public static DatanodeStorageInfo[] createDatanodeStorageInfos(String[] racks) {
return createDatanodeStorageInfos(racks, null);
}
-
+
public static DatanodeStorageInfo[] createDatanodeStorageInfos(String[] racks, String[] hostnames) {
return createDatanodeStorageInfos(racks.length, racks, hostnames);
}
-
+
public static DatanodeStorageInfo[] createDatanodeStorageInfos(int n) {
return createDatanodeStorageInfos(n, null, null);
}
@@ -1184,17 +1185,17 @@ public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr,
DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT);
return new DatanodeDescriptor(dnId, rackLocation);
}
-
+
public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr,
int port, String rackLocation) {
return getDatanodeDescriptor(ipAddr, port, rackLocation, "host");
}
-
+
public static DatanodeRegistration getLocalDatanodeRegistration() {
return new DatanodeRegistration(getLocalDatanodeID(), new StorageInfo(
NodeType.DATA_NODE), new ExportedBlockKeys(), VersionInfo.getVersion());
}
-
+
/** Copy one file's contents into the other **/
public static void copyFile(File src, File dest) throws IOException {
FileUtils.copyFile(src, dest);
@@ -1205,10 +1206,10 @@ public static class Builder {
private int maxSize = 8*1024;
private int minSize = 1;
private int nFiles = 1;
-
+
public Builder() {
}
-
+
public Builder setName(String string) {
return this;
}
@@ -1217,7 +1218,7 @@ public Builder setNumFiles(int nFiles) {
this.nFiles = nFiles;
return this;
}
-
+
public Builder setMaxLevels(int maxLevels) {
this.maxLevels = maxLevels;
return this;
@@ -1232,21 +1233,21 @@ public Builder setMinSize(int minSize) {
this.minSize = minSize;
return this;
}
-
+
public DFSTestUtil build() {
return new DFSTestUtil(nFiles, maxLevels, maxSize, minSize);
}
}
-
+
/**
* Run a set of operations and generate all edit logs
*/
public static void runOperations(MiniDFSCluster cluster,
- DistributedFileSystem filesystem, Configuration conf, long blockSize,
+ DistributedFileSystem filesystem, Configuration conf, long blockSize,
int nnIndex) throws IOException {
// create FileContext for rename2
FileContext fc = FileContext.getFileContext(cluster.getURI(0), conf);
-
+
// OP_ADD 0
final Path pathFileCreate = new Path("/file_create");
FSDataOutputStream s = filesystem.create(pathFileCreate);
@@ -1309,7 +1310,7 @@ public static void runOperations(MiniDFSCluster cluster,
long atime = mtime;
filesystem.setTimes(pathFileCreate, mtime, atime);
// OP_SET_QUOTA 14
- filesystem.setQuota(pathDirectoryMkdir, 1000L,
+ filesystem.setQuota(pathDirectoryMkdir, 1000L,
HdfsConstants.QUOTA_DONT_SET);
// OP_SET_QUOTA_BY_STORAGETYPE
filesystem.setQuotaByStorageType(pathDirectoryMkdir, StorageType.SSD, 888L);
@@ -1341,7 +1342,7 @@ public static void runOperations(MiniDFSCluster cluster,
// OP_SYMLINK 17
Path pathSymlink = new Path("/file_symlink");
fc.createSymlink(pathConcatTarget, pathSymlink, false);
-
+
// OP_REASSIGN_LEASE 22
String filePath = "/hard-lease-recovery-test";
byte[] bytes = "foo-bar-baz".getBytes();
@@ -1410,9 +1411,9 @@ public static void runOperations(MiniDFSCluster cluster,
.build());
filesystem.setAcl(pathConcatTarget, aclEntryList);
// OP_SET_XATTR
- filesystem.setXAttr(pathConcatTarget, "user.a1",
+ filesystem.setXAttr(pathConcatTarget, "user.a1",
new byte[]{0x31, 0x32, 0x33});
- filesystem.setXAttr(pathConcatTarget, "user.a2",
+ filesystem.setXAttr(pathConcatTarget, "user.a2",
new byte[]{0x37, 0x38, 0x39});
// OP_REMOVE_XATTR
filesystem.removeXAttr(pathConcatTarget, "user.a2");
@@ -1435,7 +1436,7 @@ public static long verifyExpectedCacheUsage(final long expectedCacheUsed,
final long expectedBlocks, final FsDatasetSpi> fsd) throws Exception {
GenericTestUtils.waitFor(new Supplier() {
private int tries = 0;
-
+
@Override
public Boolean get() {
long curCacheUsed = fsd.getCacheUsed();
@@ -1483,8 +1484,8 @@ public static void checkComponentsEquals(byte[][] expected, byte[][] actual) {
int i = 0;
for (byte[] e : expected) {
byte[] actualComponent = actual[i++];
- assertTrue("expected: " + DFSUtil.bytes2String(e) + ", actual: "
- + DFSUtil.bytes2String(actualComponent),
+ assertTrue("expected: " + new String(e, UTF_8) + ", actual: "
+ + new String(actualComponent, UTF_8),
Arrays.equals(e, actualComponent));
}
}
@@ -1498,7 +1499,7 @@ public static class ShortCircuitTestContext implements Closeable {
private final TemporarySocketDirectory sockDir;
private boolean closed = false;
private final boolean formerTcpReadsDisabled;
-
+
public ShortCircuitTestContext(String testName) {
this.testName = testName;
this.sockDir = new TemporarySocketDirectory();
@@ -1506,7 +1507,7 @@ public ShortCircuitTestContext(String testName) {
formerTcpReadsDisabled = DFSInputStream.tcpReadsDisabledForTesting;
Assume.assumeTrue(DomainSocket.getLoadingFailureReason() == null);
}
-
+
public Configuration newConfiguration() {
Configuration conf = new Configuration();
conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
@@ -1703,7 +1704,7 @@ public static void FsShellRun(String cmd, int retcode, String contain,
Configuration conf) throws Exception {
FsShell shell = new FsShell(new Configuration(conf));
toolRun(shell, cmd, retcode, contain);
- }
+ }
public static void DFSAdminRun(String cmd, int retcode, String contain,
Configuration conf) throws Exception {
@@ -1843,7 +1844,7 @@ public static void resetLastUpdatesWithOffset(DatanodeInfo dn, long offset) {
dn.setLastUpdate(Time.now() + offset);
dn.setLastUpdateMonotonic(Time.monotonicNow() + offset);
}
-
+
/**
* This method takes a set of block locations and fills the provided buffer
* with expected bytes based on simulated content from
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
index 1256eb1f42d78..29f697b5e1717 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
@@ -28,6 +28,7 @@
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.RandomAccessFile;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
@@ -49,7 +50,6 @@
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirType;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -74,9 +74,9 @@
* Utility functions for testing fsimage storage.
*/
public abstract class FSImageTestUtil {
-
+
public static final Log LOG = LogFactory.getLog(FSImageTestUtil.class);
-
+
/**
* The position in the fsimage header where the txid is
* written.
@@ -85,14 +85,14 @@ public abstract class FSImageTestUtil {
/**
* This function returns a md5 hash of a file.
- *
+ *
* @param file input file
* @return The md5 string
*/
public static String getFileMD5(File file) throws IOException {
return MD5FileUtils.computeMd5ForFile(file).toString();
}
-
+
/**
* Calculate the md5sum of an image after zeroing out the transaction ID
* field in the header. This is useful for tests that want to verify
@@ -116,7 +116,7 @@ public static String getImageFileMD5IgnoringTxId(File imageFile)
tmpFile.delete();
}
}
-
+
public static StorageDirectory mockStorageDirectory(
File currentDir, NameNodeDirType type) {
// Mock the StorageDirectory interface to just point to this file
@@ -129,7 +129,7 @@ public static StorageDirectory mockStorageDirectory(
Mockito.doReturn(mockFile(false)).when(sd).getPreviousDir();
return sd;
}
-
+
/**
* Make a mock storage directory that returns some set of file contents.
* @param type type of storage dir
@@ -141,37 +141,37 @@ static StorageDirectory mockStorageDirectory(
boolean previousExists,
String... fileNames) {
StorageDirectory sd = mock(StorageDirectory.class);
-
+
doReturn(type).when(sd).getStorageDirType();
-
+
// Version file should always exist
doReturn(mockFile(true)).when(sd).getVersionFile();
doReturn(mockFile(true)).when(sd).getRoot();
// Previous dir optionally exists
doReturn(mockFile(previousExists))
- .when(sd).getPreviousDir();
-
+ .when(sd).getPreviousDir();
+
// Return a mock 'current' directory which has the given paths
File[] files = new File[fileNames.length];
for (int i = 0; i < fileNames.length; i++) {
files[i] = new File(fileNames[i]);
}
-
+
File mockDir = Mockito.spy(new File("/dir/current"));
doReturn(files).when(mockDir).listFiles();
doReturn(mockDir).when(sd).getCurrentDir();
-
+
return sd;
}
-
+
static File mockFile(boolean exists) {
File mockFile = mock(File.class);
doReturn(exists).when(mockFile).exists();
return mockFile;
}
-
+
public static FSImageTransactionalStorageInspector inspectStorageDirectory(
File dir, NameNodeDirType dirType) throws IOException {
FSImageTransactionalStorageInspector inspector =
@@ -180,7 +180,7 @@ public static FSImageTransactionalStorageInspector inspectStorageDirectory(
return inspector;
}
-
+
/**
* Return a standalone instance of FSEditLog that will log into the given
* log directory. The returned instance is not yet opened.
@@ -192,20 +192,20 @@ public static FSEditLog createStandaloneEditLog(File logDir)
throw new IOException("Unable to delete contents of " + logDir);
}
NNStorage storage = Mockito.mock(NNStorage.class);
- StorageDirectory sd
+ StorageDirectory sd
= FSImageTestUtil.mockStorageDirectory(logDir, NameNodeDirType.EDITS);
List sds = Lists.newArrayList(sd);
Mockito.doReturn(sds).when(storage).dirIterable(NameNodeDirType.EDITS);
Mockito.doReturn(sd).when(storage)
.getStorageDirectory(Matchers.anyObject());
- FSEditLog editLog = new FSEditLog(new Configuration(),
+ FSEditLog editLog = new FSEditLog(new Configuration(),
storage,
ImmutableList.of(logDir.toURI()));
editLog.initJournalsForWrite();
return editLog;
}
-
+
/**
* Create an aborted in-progress log in the given directory, containing
@@ -216,13 +216,13 @@ public static void createAbortedLogWithMkdirs(File editsLogDir, int numDirs,
FSEditLog editLog = FSImageTestUtil.createStandaloneEditLog(editsLogDir);
editLog.setNextTxId(firstTxId);
editLog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-
+
PermissionStatus perms = PermissionStatus.createImmutable("fakeuser", "fakegroup",
FsPermission.createImmutable((short)0755));
for (int i = 1; i <= numDirs; i++) {
String dirName = "dir" + i;
INodeDirectory dir = new INodeDirectory(newInodeId + i - 1,
- DFSUtil.string2Bytes(dirName), perms, 0L);
+ dirName.getBytes(UTF_8), perms, 0L);
editLog.logMkDir("/" + dirName, dir);
}
editLog.logSync();
@@ -251,7 +251,7 @@ public static EnumMap> countEditLogOpTypes(
EditLogInputStream elis) throws IOException {
EnumMap> opCounts =
new EnumMap>(FSEditLogOpCodes.class);
-
+
FSEditLogOp op;
while ((op = elis.readOp()) != null) {
Holder i = opCounts.get(op.opCode);
@@ -270,9 +270,9 @@ public static EnumMap> countEditLogOpTypes(
*/
public static void assertSameNewestImage(List dirs) throws Exception {
if (dirs.size() < 2) return;
-
+
long imageTxId = -1;
-
+
List imageFiles = new ArrayList();
for (File dir : dirs) {
FSImageTransactionalStorageInspector inspector =
@@ -287,16 +287,16 @@ public static void assertSameNewestImage(List dirs) throws Exception {
imageTxId = thisTxId;
imageFiles.add(inspector.getLatestImages().get(0).getFile());
}
-
+
assertFileContentsSame(imageFiles.toArray(new File[0]));
}
-
+
/**
* Given a list of directories, assert that any files that are named
* the same thing have the same contents. For example, if a file
* named "fsimage_1" shows up in more than one directory, then it must
* be the same.
- * @throws Exception
+ * @throws Exception
*/
public static void assertParallelFilesAreIdentical(List dirs,
Set ignoredFileNames) throws Exception {
@@ -306,7 +306,7 @@ public static void assertParallelFilesAreIdentical(List dirs,
if (ignoredFileNames.contains(f.getName())) {
continue;
}
-
+
List fileList = groupedByName.get(f.getName());
if (fileList == null) {
fileList = new ArrayList();
@@ -330,7 +330,7 @@ public static void assertParallelFilesAreIdentical(List dirs,
assertFileContentsSame(sameNameList.toArray(new File[0]));
}
}
- }
+ }
}
/**
@@ -359,7 +359,7 @@ public static void assertPropertiesFilesSame(File[] propFiles)
public static void assertPropertiesFilesSame(
File[] propFiles, Set ignoredProperties) throws IOException {
Set> prevProps = null;
-
+
for (File f : propFiles) {
Properties props;
FileInputStream is = new FileInputStream(f);
@@ -389,11 +389,11 @@ public static void assertPropertiesFilesSame(
/**
* Assert that all of the given paths have the exact same
- * contents
+ * contents
*/
public static void assertFileContentsSame(File... files) throws Exception {
if (files.length < 2) return;
-
+
Map md5s = getFileMD5s(files);
if (Sets.newHashSet(md5s.values()).size() > 1) {
fail("File contents differed:\n " +
@@ -402,7 +402,7 @@ public static void assertFileContentsSame(File... files) throws Exception {
.join(md5s));
}
}
-
+
/**
* Assert that the given files are not all the same, and in fact that
* they have expectedUniqueHashes unique contents.
@@ -419,7 +419,7 @@ public static void assertFileContentsDifferent(
.join(md5s));
}
}
-
+
public static Map getFileMD5s(File... files) throws Exception {
Map ret = Maps.newHashMap();
for (File f : files) {
@@ -431,7 +431,7 @@ public static Map getFileMD5s(File... files) throws Exception {
/**
* @return a List which contains the "current" dir for each storage
- * directory of the given type.
+ * directory of the given type.
*/
public static List getCurrentDirs(NNStorage storage,
NameNodeDirType type) {
@@ -451,7 +451,7 @@ public static File findLatestImageFile(StorageDirectory sd)
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
inspector.inspectDirectory(sd);
-
+
return inspector.getLatestImages().get(0).getFile();
}
@@ -479,7 +479,7 @@ public static void assertNNHasCheckpoints(MiniDFSCluster cluster,
List txids) {
assertNNHasCheckpoints(cluster, 0, txids);
}
-
+
public static void assertNNHasCheckpoints(MiniDFSCluster cluster,
int nnIdx, List txids) {
@@ -512,7 +512,7 @@ public static List getNameNodeCurrentDirs(MiniDFSCluster cluster, int nnId
public static EditLogFile findLatestEditsLog(StorageDirectory sd)
throws IOException {
File currentDir = sd.getCurrentDir();
- List foundEditLogs
+ List foundEditLogs
= Lists.newArrayList(FileJournalManager.matchEditLogs(currentDir));
return Collections.max(foundEditLogs, EditLogFile.COMPARE_BY_START_TXID);
}
@@ -520,7 +520,7 @@ public static EditLogFile findLatestEditsLog(StorageDirectory sd)
/**
* Corrupt the given VERSION file by replacing a given
* key with a new value and re-writing the file.
- *
+ *
* @param versionFile the VERSION file to corrupt
* @param key the key to replace
* @param value the new value for this key
@@ -533,19 +533,19 @@ public static void corruptVersionFile(File versionFile, String key, String value
try {
props.load(fis);
IOUtils.closeStream(fis);
-
+
if (value == null || value.isEmpty()) {
props.remove(key);
} else {
props.setProperty(key, value);
}
-
+
out = new FileOutputStream(versionFile);
props.store(out, null);
-
+
} finally {
IOUtils.cleanup(null, fis, out);
- }
+ }
}
public static void assertReasonableNameCurrentDir(File curDir)
@@ -565,11 +565,11 @@ public static void logStorageContents(Log LOG, NNStorage storage) {
File[] files = curDir.listFiles();
Arrays.sort(files);
for (File f : files) {
- LOG.info(" file " + f.getAbsolutePath() + "; len = " + f.length());
+ LOG.info(" file " + f.getAbsolutePath() + "; len = " + f.length());
}
}
}
-
+
/** get the fsImage*/
public static FSImage getFSImage(NameNode node) {
return node.getFSImage();
@@ -581,12 +581,12 @@ public static FSImage getFSImage(NameNode node) {
public static long getNSQuota(FSNamesystem ns) {
return ns.dir.rootDir.getQuotaCounts().getNameSpace();
}
-
+
public static void assertNNFilesMatch(MiniDFSCluster cluster) throws Exception {
List curDirs = Lists.newArrayList();
curDirs.addAll(FSImageTestUtil.getNameNodeCurrentDirs(cluster, 0));
curDirs.addAll(FSImageTestUtil.getNameNodeCurrentDirs(cluster, 1));
-
+
// Ignore seen_txid file, since the newly bootstrapped standby
// will have a higher seen_txid than the one it bootstrapped from.
Set ignoredFiles = ImmutableSet.of("seen_txid");
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
index 82f5cfb169917..6feaa8abedd11 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
@@ -22,6 +22,7 @@
import java.io.File;
import java.io.IOException;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.List;
@@ -33,7 +34,6 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
@@ -75,7 +75,7 @@ public class TestFSImageWithSnapshot {
MiniDFSCluster cluster;
FSNamesystem fsn;
DistributedFileSystem hdfs;
-
+
@Before
public void setUp() throws Exception {
conf = new Configuration();
@@ -104,8 +104,8 @@ private File getImageFile(String dir, long imageTxId) {
return new File(dir, String.format("%s_%019d", NameNodeFile.IMAGE,
imageTxId));
}
-
- /**
+
+ /**
* Create a temp file for dumping the fsdir
* @param dir directory for the temp file
* @param suffix suffix of of the temp file
@@ -114,8 +114,8 @@ private File getImageFile(String dir, long imageTxId) {
private File getDumpTreeFile(String dir, String suffix) {
return new File(dir, String.format("dumpTree_%s", suffix));
}
-
- /**
+
+ /**
* Dump the fsdir tree to a temp file
* @param fileSuffix suffix of the temp file for dumping
* @return the temp file
@@ -125,7 +125,7 @@ private File dumpTree2File(String fileSuffix) throws IOException {
SnapshotTestHelper.dumpTree2File(fsn.getFSDirectory(), file);
return file;
}
-
+
/** Append a file without closing the output stream */
private HdfsDataOutputStream appendFileWithoutClosing(Path file, int length)
throws IOException {
@@ -136,7 +136,7 @@ private HdfsDataOutputStream appendFileWithoutClosing(Path file, int length)
out.write(toAppend);
return out;
}
-
+
/** Save the fsimage to a temp file */
private File saveFSImageToTempFile() throws IOException {
SaveNamespaceContext context = new SaveNamespaceContext(fsn, txid,
@@ -152,7 +152,7 @@ private File saveFSImageToTempFile() throws IOException {
}
return imageFile;
}
-
+
/** Load the fsimage from a temp file */
private void loadFSImageFromTempFile(File imageFile) throws IOException {
FSImageFormat.LoaderDelegator loader = FSImageFormat.newLoader(conf, fsn);
@@ -166,7 +166,7 @@ private void loadFSImageFromTempFile(File imageFile) throws IOException {
fsn.writeUnlock();
}
}
-
+
/**
* Test when there is snapshot taken on root
*/
@@ -175,14 +175,14 @@ public void testSnapshotOnRoot() throws Exception {
final Path root = new Path("/");
hdfs.allowSnapshot(root);
hdfs.createSnapshot(root, "s1");
-
+
cluster.shutdown();
cluster = new MiniDFSCluster.Builder(conf).format(false)
.numDataNodes(NUM_DATANODES).build();
cluster.waitActive();
fsn = cluster.getNamesystem();
hdfs = cluster.getFileSystem();
-
+
// save namespace and restart cluster
hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
hdfs.saveNamespace();
@@ -193,23 +193,23 @@ public void testSnapshotOnRoot() throws Exception {
cluster.waitActive();
fsn = cluster.getNamesystem();
hdfs = cluster.getFileSystem();
-
+
INodeDirectory rootNode = fsn.dir.getINode4Write(root.toString())
.asDirectory();
- assertTrue("The children list of root should be empty",
+ assertTrue("The children list of root should be empty",
rootNode.getChildrenList(Snapshot.CURRENT_STATE_ID).isEmpty());
// one snapshot on root: s1
List diffList = rootNode.getDiffs().asList();
assertEquals(1, diffList.size());
- Snapshot s1 = rootNode.getSnapshot(DFSUtil.string2Bytes("s1"));
+ Snapshot s1 = rootNode.getSnapshot("s1".getBytes(UTF_8));
assertEquals(s1.getId(), diffList.get(0).getSnapshotId());
-
+
// check SnapshotManager's snapshottable directory list
assertEquals(1, fsn.getSnapshotManager().getNumSnapshottableDirs());
SnapshottableDirectoryStatus[] sdirs = fsn.getSnapshotManager()
.getSnapshottableDirListing(null);
assertEquals(root, sdirs[0].getFullPath());
-
+
// save namespace and restart cluster
hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
hdfs.saveNamespace();
@@ -252,7 +252,7 @@ public void testSaveLoadImage() throws Exception {
DFSTestUtil.createFile(hdfs, sub1file1, BLOCKSIZE, (short) 1, seed);
DFSTestUtil.createFile(hdfs, sub1file2, BLOCKSIZE, (short) 1, seed);
checkImage(s);
-
+
hdfs.createSnapshot(dir, "s" + ++s);
Path sub2 = new Path(dir, "sub2");
Path sub2file1 = new Path(sub2, "sub2file1");
@@ -267,14 +267,14 @@ public void testSaveLoadImage() throws Exception {
hdfs.setOwner(sub2, "dr.who", "unknown");
hdfs.delete(sub2file1, true);
checkImage(s);
-
+
hdfs.createSnapshot(dir, "s" + ++s);
Path sub1_sub2file2 = new Path(sub1, "sub2file2");
hdfs.rename(sub2file2, sub1_sub2file2);
-
+
hdfs.rename(sub1file1, sub2file1);
checkImage(s);
-
+
hdfs.rename(sub2file1, sub2file2);
checkImage(s);
}
@@ -284,10 +284,10 @@ void checkImage(int s) throws IOException {
// dump the fsdir tree
File fsnBefore = dumpTree2File(name + "_before");
-
+
// save the namesystem to a temp file
File imageFile = saveFSImageToTempFile();
-
+
long numSdirBefore = fsn.getNumSnapshottableDirs();
long numSnapshotBefore = fsn.getNumSnapshots();
SnapshottableDirectoryStatus[] dirBefore = hdfs.getSnapshottableDirListing();
@@ -305,20 +305,20 @@ void checkImage(int s) throws IOException {
cluster.waitActive();
fsn = cluster.getNamesystem();
hdfs = cluster.getFileSystem();
-
+
// load the namesystem from the temp file
loadFSImageFromTempFile(imageFile);
-
+
// dump the fsdir tree again
File fsnAfter = dumpTree2File(name + "_after");
-
+
// compare two dumped tree
SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnAfter, true);
-
+
long numSdirAfter = fsn.getNumSnapshottableDirs();
long numSnapshotAfter = fsn.getNumSnapshots();
SnapshottableDirectoryStatus[] dirAfter = hdfs.getSnapshottableDirListing();
-
+
Assert.assertEquals(numSdirBefore, numSdirAfter);
Assert.assertEquals(numSnapshotBefore, numSnapshotAfter);
Assert.assertEquals(dirBefore.length, dirAfter.length);
@@ -330,7 +330,7 @@ void checkImage(int s) throws IOException {
Assert.assertTrue(pathListBefore.contains(sAfter.getFullPath().toString()));
}
}
-
+
/**
* Test the fsimage saving/loading while file appending.
*/
@@ -341,11 +341,11 @@ public void testSaveLoadImageWithAppending() throws Exception {
Path sub1file2 = new Path(sub1, "sub1file2");
DFSTestUtil.createFile(hdfs, sub1file1, BLOCKSIZE, (short) 1, seed);
DFSTestUtil.createFile(hdfs, sub1file2, BLOCKSIZE, (short) 1, seed);
-
+
// 1. create snapshot s0
hdfs.allowSnapshot(dir);
hdfs.createSnapshot(dir, "s0");
-
+
// 2. create snapshot s1 before appending sub1file1 finishes
HdfsDataOutputStream out = appendFileWithoutClosing(sub1file1, BLOCKSIZE);
out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
@@ -353,13 +353,13 @@ public void testSaveLoadImageWithAppending() throws Exception {
DFSTestUtil.appendFile(hdfs, sub1file2, BLOCKSIZE);
hdfs.createSnapshot(dir, "s1");
out.close();
-
+
// 3. create snapshot s2 before appending finishes
out = appendFileWithoutClosing(sub1file1, BLOCKSIZE);
out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
hdfs.createSnapshot(dir, "s2");
out.close();
-
+
// 4. save fsimage before appending finishes
out = appendFileWithoutClosing(sub1file1, BLOCKSIZE);
out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
@@ -367,7 +367,7 @@ public void testSaveLoadImageWithAppending() throws Exception {
File fsnBefore = dumpTree2File("before");
// save the namesystem to a temp file
File imageFile = saveFSImageToTempFile();
-
+
// 5. load fsimage and compare
// first restart the cluster, and format the cluster
out.close();
@@ -379,14 +379,14 @@ public void testSaveLoadImageWithAppending() throws Exception {
hdfs = cluster.getFileSystem();
// then load the fsimage
loadFSImageFromTempFile(imageFile);
-
+
// dump the fsdir tree again
File fsnAfter = dumpTree2File("after");
-
+
// compare two dumped tree
SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnAfter, true);
}
-
+
/**
* Test the fsimage loading while there is file under construction.
*/
@@ -397,18 +397,18 @@ public void testLoadImageWithAppending() throws Exception {
Path sub1file2 = new Path(sub1, "sub1file2");
DFSTestUtil.createFile(hdfs, sub1file1, BLOCKSIZE, (short) 1, seed);
DFSTestUtil.createFile(hdfs, sub1file2, BLOCKSIZE, (short) 1, seed);
-
+
hdfs.allowSnapshot(dir);
hdfs.createSnapshot(dir, "s0");
-
+
HdfsDataOutputStream out = appendFileWithoutClosing(sub1file1, BLOCKSIZE);
- out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
-
+ out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
+
// save namespace and restart cluster
hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
hdfs.saveNamespace();
hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
-
+
cluster.shutdown();
cluster = new MiniDFSCluster.Builder(conf).format(false)
.numDataNodes(NUM_DATANODES).build();
@@ -416,7 +416,7 @@ public void testLoadImageWithAppending() throws Exception {
fsn = cluster.getNamesystem();
hdfs = cluster.getFileSystem();
}
-
+
/**
* Test fsimage loading when 1) there is an empty file loaded from fsimage,
* and 2) there is later an append operation to be applied from edit log.
@@ -427,28 +427,28 @@ public void testLoadImageWithEmptyFile() throws Exception {
Path file = new Path(dir, "file");
FSDataOutputStream out = hdfs.create(file);
out.close();
-
+
// save namespace
hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
hdfs.saveNamespace();
hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
-
+
// append to the empty file
out = hdfs.append(file);
out.write(1);
out.close();
-
+
// restart cluster
cluster.shutdown();
cluster = new MiniDFSCluster.Builder(conf).format(false)
.numDataNodes(NUM_DATANODES).build();
cluster.waitActive();
hdfs = cluster.getFileSystem();
-
+
FileStatus status = hdfs.getFileStatus(file);
assertEquals(1, status.getLen());
}
-
+
/**
* Testing a special case with snapshots. When the following steps happen:
*
@@ -460,8 +460,8 @@ public void testLoadImageWithEmptyFile() throws Exception {
*
* When we merge the diff from s2 to s1 (since we deleted s2), we need to make
* sure all the files/dirs created after s1 should be destroyed. Otherwise
- * we may save these files/dirs to the fsimage, and cause FileNotFound
- * Exception while loading fsimage.
+ * we may save these files/dirs to the fsimage, and cause FileNotFound
+ * Exception while loading fsimage.
*/
@Test (timeout=300000)
public void testSaveLoadImageAfterSnapshotDeletion()
@@ -471,25 +471,25 @@ public void testSaveLoadImageAfterSnapshotDeletion()
Path subDir = new Path(dir, "subdir");
Path subsubDir = new Path(subDir, "subsubdir");
hdfs.mkdirs(subsubDir);
-
+
// take snapshots on subdir and dir
SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
-
+
// create new dir under initial dir
Path newDir = new Path(subsubDir, "newdir");
Path newFile = new Path(newDir, "newfile");
hdfs.mkdirs(newDir);
DFSTestUtil.createFile(hdfs, newFile, BLOCKSIZE, (short) 1, seed);
-
+
// create another snapshot
SnapshotTestHelper.createSnapshot(hdfs, dir, "s2");
-
+
// delete subsubdir
hdfs.delete(subsubDir, true);
-
+
// delete snapshot s2
hdfs.deleteSnapshot(dir, "s2");
-
+
// restart cluster
cluster.shutdown();
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES)
@@ -497,12 +497,12 @@ public void testSaveLoadImageAfterSnapshotDeletion()
cluster.waitActive();
fsn = cluster.getNamesystem();
hdfs = cluster.getFileSystem();
-
+
// save namespace to fsimage
hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
hdfs.saveNamespace();
hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
-
+
cluster.shutdown();
cluster = new MiniDFSCluster.Builder(conf).format(false)
.numDataNodes(NUM_DATANODES).build();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 81a25fdd25034..f530937df9282 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -42,6 +42,7 @@
import java.io.RandomAccessFile;
import java.io.StringWriter;
import java.io.Writer;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.nio.channels.FileChannel;
@@ -76,7 +77,6 @@
import org.apache.hadoop.hdfs.DFSInputStream;
import org.apache.hadoop.hdfs.DFSOutputStream;
import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.fs.StorageType;
@@ -136,20 +136,20 @@ public class TestFsck {
static final String AUDITLOG_FILE =
GenericTestUtils.getTempPath("TestFsck-audit.log");
-
- // Pattern for:
+
+ // Pattern for:
// allowed=true ugi=name ip=/address cmd=FSCK src=/ dst=null perm=null
static final Pattern FSCK_PATTERN = Pattern.compile(
"allowed=.*?\\s" +
- "ugi=.*?\\s" +
- "ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\s" +
- "cmd=fsck\\ssrc=\\/\\sdst=null\\s" +
+ "ugi=.*?\\s" +
+ "ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\s" +
+ "cmd=fsck\\ssrc=\\/\\sdst=null\\s" +
"perm=null\\s" + "proto=.*");
static final Pattern GET_FILE_INFO_PATTERN = Pattern.compile(
"allowed=.*?\\s" +
- "ugi=.*?\\s" +
- "ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\s" +
- "cmd=getfileinfo\\ssrc=\\/\\sdst=null\\s" +
+ "ugi=.*?\\s" +
+ "ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\s" +
+ "cmd=getfileinfo\\ssrc=\\/\\sdst=null\\s" +
"perm=null\\s" + "proto=.*");
static final Pattern NUM_MISSING_BLOCKS_PATTERN = Pattern.compile(
@@ -157,11 +157,11 @@ public class TestFsck {
static final Pattern NUM_CORRUPT_BLOCKS_PATTERN = Pattern.compile(
".*Corrupt blocks:\t\t([0123456789]*).*");
-
+
private static final String LINE_SEPARATOR =
System.getProperty("line.separator");
- static String runFsck(Configuration conf, int expectedErrCode,
+ static String runFsck(Configuration conf, int expectedErrCode,
boolean checkErrorCode, String... path)
throws Exception {
ByteArrayOutputStream bStream = new ByteArrayOutputStream();
@@ -250,7 +250,7 @@ private void setupAuditLogs() throws IOException {
new RollingFileAppender(layout, AUDITLOG_FILE);
logger.addAppender(appender);
}
-
+
private void verifyAuditLogs() throws IOException {
// Turn off the logs
Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger();
@@ -286,7 +286,7 @@ private void verifyAuditLogs() throws IOException {
}
}
}
-
+
@Test
public void testFsckNonExistent() throws Exception {
DFSTestUtil util = new DFSTestUtil.Builder().setName("TestFsck").
@@ -449,7 +449,7 @@ static private class CorruptedTestFile {
final private int numDataNodes;
final private int blockSize;
final private byte[] initialContents;
-
+
CorruptedTestFile(String name, Set blocksToCorrupt,
DFSClient dfsClient, int numDataNodes, int blockSize)
throws IOException {
@@ -477,7 +477,7 @@ private byte[] cacheInitialContents() throws IOException {
}
return content;
}
-
+
public void removeBlocks(MiniDFSCluster cluster)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
@@ -555,7 +555,7 @@ public void checkSalvagedRemains() throws IOException {
}
}
}
-
+
@Test
public void testFsckMoveAndDelete() throws Exception {
final int maxMoveTries = 5;
@@ -621,7 +621,7 @@ public void testFsckMoveAndDelete() throws Exception {
assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
util.cleanup(fs, topDir);
}
-
+
@Test
public void testFsckOpenFiles() throws Exception {
DFSTestUtil util = new DFSTestUtil.Builder().setName("TestFsck").
@@ -762,7 +762,7 @@ public void testCorruptBlock() throws Exception {
outStr = runFsck(conf, 0, true, "/");
System.out.println(outStr);
assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
-
+
// corrupt replicas
File blockFile = cluster.getBlockFile(0, block);
if (blockFile != null && blockFile.exists()) {
@@ -956,7 +956,7 @@ public void testFsckReplicaDetails() throws Exception {
}
/** Test if fsck can return -1 in case of failure.
- *
+ *
* @throws Exception
*/
@Test
@@ -986,7 +986,7 @@ public void testFsckError() throws Exception {
// clean up file system
fs.delete(filePath, true);
}
-
+
/** check if option -list-corruptfiles of fsck command works properly. */
@Test
public void testFsckListCorruptFilesBlocks() throws Exception {
@@ -1049,7 +1049,7 @@ public void testFsckListCorruptFilesBlocks() throws Exception {
util.cleanup(fs, "/corruptData");
util.cleanup(fs, "/goodData");
}
-
+
/**
* Test for checking fsck command on illegal arguments should print the proper
* usage.
@@ -1078,7 +1078,7 @@ public void testToCheckTheFsckCommandOnIllegalArguments() throws Exception {
// clean up file system
fs.delete(filePath, true);
}
-
+
/**
* Tests that the # of missing block replicas and expected replicas is
* correct.
@@ -1095,11 +1095,11 @@ public void testFsckMissingReplicas() throws IOException {
final short numBlocks = 3;
// Set a small-ish blocksize
final long blockSize = 512;
-
+
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
-
+
DistributedFileSystem dfs = null;
-
+
// Startup a minicluster
cluster =
new MiniDFSCluster.Builder(conf).numDataNodes(numReplicas).build();
@@ -1138,7 +1138,7 @@ public void testFsckMissingReplicas() throws IOException {
(numBlocks*replFactor) - (numBlocks*numReplicas));
assertEquals(replRes.numExpectedReplicas, numBlocks*replFactor);
}
-
+
/**
* Tests that the # of misreplaced replicas is correct.
* @throws IOException
@@ -1153,14 +1153,14 @@ public void testFsckMisPlacedReplicas() throws IOException {
final short numBlocks = 3;
// Set a small-ish blocksize
final long blockSize = 512;
-
+
String[] racks = {"/rack1", "/rack1"};
String[] hosts = {"host1", "host2"};
-
+
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
-
+
DistributedFileSystem dfs = null;
-
+
// Startup a minicluster
cluster =
new MiniDFSCluster.Builder(conf).numDataNodes(numDn).hosts(hosts)
@@ -1244,7 +1244,7 @@ public void testFsckFileNotFound() throws Exception {
String owner = "foo";
String group = "bar";
byte[] symlink = null;
- byte[] path = DFSUtil.string2Bytes(pathString);
+ byte[] path = pathString.getBytes(UTF_8);
long fileId = 312321L;
int numChildren = 1;
byte storagePolicy = 0;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
index 3492c2a23463d..8abb3eb48670f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
@@ -26,6 +26,7 @@
import java.io.FileNotFoundException;
import java.io.IOException;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.ArrayList;
import java.util.List;
@@ -51,7 +52,6 @@
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -184,7 +184,7 @@ public void testPreferredBlockSizeBelowLowerBound ()
replication = 3;
preferredBlockSize = -1;
createINodeFile(replication, preferredBlockSize);
- }
+ }
/**
* IllegalArgumentException is expected for setting above upper bound
@@ -204,25 +204,25 @@ public void testGetFullPathName() {
replication = 3;
preferredBlockSize = 128*1024*1024;
INodeFile inf = createINodeFile(replication, preferredBlockSize);
- inf.setLocalName(DFSUtil.string2Bytes("f"));
+ inf.setLocalName("f".getBytes(UTF_8));
INodeDirectory root = new INodeDirectory(HdfsConstants.GRANDFATHER_INODE_ID,
INodeDirectory.ROOT_NAME, perm, 0L);
INodeDirectory dir = new INodeDirectory(HdfsConstants.GRANDFATHER_INODE_ID,
- DFSUtil.string2Bytes("d"), perm, 0L);
+ "d".getBytes(UTF_8), perm, 0L);
assertEquals("f", inf.getFullPathName());
dir.addChild(inf);
assertEquals("d"+Path.SEPARATOR+"f", inf.getFullPathName());
-
+
root.addChild(dir);
assertEquals(Path.SEPARATOR+"d"+Path.SEPARATOR+"f", inf.getFullPathName());
assertEquals(Path.SEPARATOR+"d", dir.getFullPathName());
assertEquals(Path.SEPARATOR, root.getFullPathName());
}
-
+
/**
* FSDirectory#unprotectedSetQuota creates a new INodeDirectoryWithQuota to
* replace the original INodeDirectory. Before HDFS-4243, the parent field of
@@ -252,14 +252,14 @@ public void testGetFullPathNameAfterSetQuota() throws Exception {
// Check the full path name of the INode associating with the file
INode fnode = fsdir.getINode(file.toString());
assertEquals(file.toString(), fnode.getFullPathName());
-
+
// Call FSDirectory#unprotectedSetQuota which calls
// INodeDirectory#replaceChild
dfs.setQuota(dir, Long.MAX_VALUE - 1, replication * fileLen * 10);
INodeDirectory dirNode = getDir(fsdir, dir);
assertEquals(dir.toString(), dirNode.getFullPathName());
assertTrue(dirNode.isWithQuota());
-
+
final Path newDir = new Path("/newdir");
final Path newFile = new Path(newDir, "file");
// Also rename dir
@@ -275,7 +275,7 @@ public void testGetFullPathNameAfterSetQuota() throws Exception {
}
}
}
-
+
@Test
public void testConcatBlocks() {
INodeFile origFile = createINodeFiles(1, "origfile")[0];
@@ -286,8 +286,8 @@ public void testConcatBlocks() {
origFile.concatBlocks(appendFiles, bm);
assertEquals("Number of blocks didn't match", origFile.numBlocks(), 5L);
}
-
- /**
+
+ /**
* Creates the required number of files with one block each
* @param nCount Number of INodes to create
* @return Array of INode files
@@ -302,18 +302,18 @@ private INodeFile[] createINodeFiles(int nCount, String fileNamePrefix) {
for (int i = 0; i < nCount; i++) {
iNodes[i] = new INodeFile(i, null, perm, 0L, 0L, null, replication,
preferredBlockSize);
- iNodes[i].setLocalName(DFSUtil.string2Bytes(fileNamePrefix + i));
+ iNodes[i].setLocalName((fileNamePrefix + i).getBytes(UTF_8));
BlockInfo newblock = new BlockInfoContiguous(replication);
iNodes[i].addBlock(newblock);
}
-
+
return iNodes;
}
/**
* Test for the static {@link INodeFile#valueOf(INode, String)}
* and {@link INodeFileUnderConstruction#valueOf(INode, String)} methods.
- * @throws IOException
+ * @throws IOException
*/
@Test
public void testValueOf () throws IOException {
@@ -361,7 +361,7 @@ public void testValueOf () throws IOException {
HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication,
1024L);
from.asFile().toUnderConstruction("client", "machine");
-
+
//cast to INodeFile, should success
final INodeFile f = INodeFile.valueOf(from, path);
assertTrue(f == from);
@@ -431,7 +431,7 @@ public void testInodeId() throws IOException {
DFSTestUtil.createFile(fs, new Path("/test1/file"), 1024, (short) 1, 0);
assertEquals(++expectedLastInodeId, fsn.dir.getLastInodeId());
assertEquals(++inodeCount, fsn.dir.getInodeMapSize());
-
+
// Ensure right inode ID is returned in file status
HdfsFileStatus fileStatus = nnrpc.getFileInfo("/test1/file");
assertEquals(expectedLastInodeId, fileStatus.getFileId());
@@ -442,12 +442,12 @@ public void testInodeId() throws IOException {
assertTrue(fs.rename(path, renamedPath));
assertEquals(expectedLastInodeId, fsn.dir.getLastInodeId());
assertEquals(inodeCount, fsn.dir.getInodeMapSize());
-
+
// Delete test2/file and test2 and ensure inode map size decreases
assertTrue(fs.delete(renamedPath, true));
inodeCount -= 2;
assertEquals(inodeCount, fsn.dir.getInodeMapSize());
-
+
// Create and concat /test/file1 /test/file2
// Create /test1/file1 and /test1/file2
String file1 = "/test1/file1";
@@ -467,7 +467,7 @@ public void testInodeId() throws IOException {
inodeCount -= 2; // test1 and file2 is deleted
assertEquals(inodeCount, fsn.dir.getInodeMapSize());
- // Make sure editlog is loaded correctly
+ // Make sure editlog is loaded correctly
cluster.restartNameNode();
cluster.waitActive();
fsn = cluster.getNamesystem();
@@ -544,7 +544,7 @@ public void testWriteToDeletedFile() throws IOException {
cluster.shutdown();
}
}
-
+
private Path getInodePath(long inodeId, String remainingPath) {
StringBuilder b = new StringBuilder();
b.append(Path.SEPARATOR).append(FSDirectory.DOT_RESERVED_STRING)
@@ -555,7 +555,7 @@ private Path getInodePath(long inodeId, String remainingPath) {
LOG.info("Inode path is " + p);
return p;
}
-
+
/**
* Tests for addressing files using /.reserved/.inodes/ in file system
* operations.
@@ -572,44 +572,44 @@ public void testInodeIdBasedPaths() throws Exception {
cluster.waitActive();
DistributedFileSystem fs = cluster.getFileSystem();
NamenodeProtocols nnRpc = cluster.getNameNodeRpc();
-
+
// FileSystem#mkdirs "/testInodeIdBasedPaths"
Path baseDir = getInodePath(INodeId.ROOT_INODE_ID, "testInodeIdBasedPaths");
Path baseDirRegPath = new Path("/testInodeIdBasedPaths");
fs.mkdirs(baseDir);
fs.exists(baseDir);
long baseDirFileId = nnRpc.getFileInfo(baseDir.toString()).getFileId();
-
+
// FileSystem#create file and FileSystem#close
Path testFileInodePath = getInodePath(baseDirFileId, "test1");
Path testFileRegularPath = new Path(baseDir, "test1");
final int testFileBlockSize = 1024;
FileSystemTestHelper.createFile(fs, testFileInodePath, 1, testFileBlockSize);
assertTrue(fs.exists(testFileInodePath));
-
+
// FileSystem#setPermission
FsPermission perm = new FsPermission((short)0666);
fs.setPermission(testFileInodePath, perm);
-
+
// FileSystem#getFileStatus and FileSystem#getPermission
FileStatus fileStatus = fs.getFileStatus(testFileInodePath);
assertEquals(perm, fileStatus.getPermission());
-
+
// FileSystem#setOwner
fs.setOwner(testFileInodePath, fileStatus.getOwner(), fileStatus.getGroup());
-
+
// FileSystem#setTimes
fs.setTimes(testFileInodePath, 0, 0);
fileStatus = fs.getFileStatus(testFileInodePath);
assertEquals(0, fileStatus.getModificationTime());
assertEquals(0, fileStatus.getAccessTime());
-
+
// FileSystem#setReplication
fs.setReplication(testFileInodePath, (short)3);
fileStatus = fs.getFileStatus(testFileInodePath);
assertEquals(3, fileStatus.getReplication());
fs.setReplication(testFileInodePath, (short)1);
-
+
// ClientProtocol#getPreferredBlockSize
assertEquals(testFileBlockSize,
nnRpc.getPreferredBlockSize(testFileInodePath.toString()));
@@ -627,31 +627,31 @@ public void testInodeIdBasedPaths() throws Exception {
fs.listXAttrs(testFileInodePath);
fs.access(testFileInodePath, FsAction.READ_WRITE);
}
-
+
// symbolic link related tests
-
+
// Reserved path is not allowed as a target
String invalidTarget = new Path(baseDir, "invalidTarget").toString();
String link = new Path(baseDir, "link").toString();
testInvalidSymlinkTarget(nnRpc, invalidTarget, link);
-
+
// Test creating a link using reserved inode path
String validTarget = "/validtarget";
testValidSymlinkTarget(nnRpc, validTarget, link);
-
+
// FileSystem#append
fs.append(testFileInodePath);
// DistributedFileSystem#recoverLease
-
+
fs.recoverLease(testFileInodePath);
-
+
// Namenode#getBlockLocations
LocatedBlocks l1 = nnRpc.getBlockLocations(testFileInodePath.toString(),
0, Long.MAX_VALUE);
LocatedBlocks l2 = nnRpc.getBlockLocations(testFileRegularPath.toString(),
0, Long.MAX_VALUE);
checkEquals(l1, l2);
-
+
// FileSystem#rename - both the variants
Path renameDst = getInodePath(baseDirFileId, "test2");
fileStatus = fs.getFileStatus(testFileInodePath);
@@ -659,20 +659,20 @@ public void testInodeIdBasedPaths() throws Exception {
fs.rename(testFileInodePath, renameDst);
fs.rename(renameDst, testFileInodePath);
assertEquals(fileStatus, fs.getFileStatus(testFileInodePath));
-
+
// Rename variant 2: rename and rename bacck
fs.rename(testFileInodePath, renameDst, Rename.OVERWRITE);
fs.rename(renameDst, testFileInodePath, Rename.OVERWRITE);
assertEquals(fileStatus, fs.getFileStatus(testFileInodePath));
-
+
// FileSystem#getContentSummary
assertEquals(fs.getContentSummary(testFileRegularPath).toString(),
fs.getContentSummary(testFileInodePath).toString());
-
+
// FileSystem#listFiles
checkEquals(fs.listFiles(baseDirRegPath, false),
fs.listFiles(baseDir, false));
-
+
// FileSystem#delete
fs.delete(testFileInodePath, true);
assertFalse(fs.exists(testFileInodePath));
@@ -682,7 +682,7 @@ public void testInodeIdBasedPaths() throws Exception {
}
}
}
-
+
private void testInvalidSymlinkTarget(NamenodeProtocols nnRpc,
String invalidTarget, String link) throws IOException {
try {
@@ -700,12 +700,12 @@ private void testValidSymlinkTarget(NamenodeProtocols nnRpc, String target,
nnRpc.createSymlink(target, link, perm, false);
assertEquals(target, nnRpc.getLinkTarget(link));
}
-
+
private static void checkEquals(LocatedBlocks l1, LocatedBlocks l2) {
List list1 = l1.getLocatedBlocks();
List list2 = l2.getLocatedBlocks();
assertEquals(list1.size(), list2.size());
-
+
for (int i = 0; i < list1.size(); i++) {
LocatedBlock b1 = list1.get(i);
LocatedBlock b2 = list2.get(i);
@@ -718,7 +718,7 @@ private static void checkEquals(RemoteIterator i1,
RemoteIterator i2) throws IOException {
while (i1.hasNext()) {
assertTrue(i2.hasNext());
-
+
// Compare all the fields but the path name, which is relative
// to the original path from listFiles.
LocatedFileStatus l1 = i1.next();
@@ -734,7 +734,7 @@ private static void checkEquals(RemoteIterator i1,
}
assertFalse(i2.hasNext());
}
-
+
/**
* Check /.reserved path is reserved and cannot be created.
*/
@@ -747,12 +747,12 @@ public void testReservedFileNames() throws IOException {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
cluster.waitActive();
FileSystem fs = cluster.getFileSystem();
-
+
// Creation of directory or file with reserved path names is disallowed
ensureReservedFileNamesCannotBeCreated(fs, "/.reserved", false);
ensureReservedFileNamesCannotBeCreated(fs, "/.reserved", false);
Path reservedPath = new Path("/.reserved");
-
+
// Loading of fsimage or editlog with /.reserved directory should fail
// Mkdir "/.reserved reserved path with reserved path check turned off
FSDirectory.CHECK_RESERVED_FILE_NAMES = false;
@@ -774,7 +774,7 @@ public void testReservedFileNames() throws IOException {
}
}
}
-
+
private void ensureReservedFileNamesCannotBeCreated(FileSystem fs, String name,
boolean isDir) {
// Creation of directory or file with reserved path names is disallowed
@@ -790,7 +790,7 @@ private void ensureReservedFileNamesCannotBeCreated(FileSystem fs, String name,
// ignored
}
}
-
+
private void ensureReservedFileNamesCannotBeLoaded(MiniDFSCluster cluster)
throws IOException {
// Turn on reserved file name checking. Loading of edits should fail
@@ -805,7 +805,7 @@ private void ensureReservedFileNamesCannotBeLoaded(MiniDFSCluster cluster)
FSDirectory.CHECK_RESERVED_FILE_NAMES = true;
ensureClusterRestartFails(cluster);
}
-
+
private void ensureClusterRestartFails(MiniDFSCluster cluster) {
try {
cluster.restartNameNode();
@@ -815,14 +815,14 @@ private void ensureClusterRestartFails(MiniDFSCluster cluster) {
}
assertFalse(cluster.isClusterUp());
}
-
+
private void ensureClusterRestartSucceeds(MiniDFSCluster cluster)
throws IOException {
cluster.restartNameNode();
cluster.waitActive();
assertTrue(cluster.isClusterUp());
}
-
+
/**
* For a given path, build a tree of INodes and return the leaf node.
*/
@@ -830,7 +830,7 @@ private INode createTreeOfInodes(String path) throws QuotaExceededException {
byte[][] components = INode.getPathComponents(path);
FsPermission perm = FsPermission.createImmutable((short)0755);
PermissionStatus permstatus = PermissionStatus.createImmutable("", "", perm);
-
+
long id = 0;
INodeDirectory prev = new INodeDirectory(++id, new byte[0], permstatus, 0);
INodeDirectory dir = null;
@@ -838,14 +838,14 @@ private INode createTreeOfInodes(String path) throws QuotaExceededException {
if (component.length == 0) {
continue;
}
- System.out.println("Adding component " + DFSUtil.bytes2String(component));
+ System.out.println("Adding component " + new String(component, UTF_8));
dir = new INodeDirectory(++id, component, permstatus, 0);
prev.addChild(dir, false, Snapshot.CURRENT_STATE_ID);
prev = dir;
}
return dir; // Last Inode in the chain
}
-
+
/**
* Test for {@link FSDirectory#getPathComponents(INode)}
*/
@@ -961,7 +961,7 @@ public void testInodeReplacement() throws Exception {
}
}
}
-
+
@Test
public void testDotdotInodePath() throws Exception {
final Configuration conf = new Configuration();
@@ -982,12 +982,12 @@ public void testDotdotInodePath() throws Exception {
client = new DFSClient(DFSUtilClient.getNNAddress(conf), conf);
HdfsFileStatus status = client.getFileInfo(testPath);
assertTrue(parentId == status.getFileId());
-
+
// Test root's parent is still root
testPath = "/.reserved/.inodes/" + parentId + "/..";
status = client.getFileInfo(testPath);
assertTrue(parentId == status.getFileId());
-
+
} finally {
IOUtils.cleanup(LOG, client);
if (cluster != null) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathComponents.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathComponents.java
index 189f34cab0f4b..ef6d8d7e66c4d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathComponents.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathComponents.java
@@ -18,13 +18,14 @@
package org.apache.hadoop.hdfs.server.namenode;
import static org.junit.Assert.assertEquals;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.Arrays;
import org.apache.hadoop.hdfs.DFSUtil;
import org.junit.Test;
/**
- *
+ *
*/
public class TestPathComponents {
@@ -96,7 +97,7 @@ public void testString(String path, String[] expected) throws Exception {
String[] actual = new String[components.length];
for (int i=0; i < components.length; i++) {
if (components[i] != null) {
- actual[i] = DFSUtil.bytes2String(components[i]);
+ actual[i] = new String(components[i], UTF_8);
}
}
assertEquals(Arrays.asList(expected), Arrays.asList(actual));
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
index d1d915e2ea4b0..442e972a69d3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
@@ -22,13 +22,13 @@
import static org.junit.Assert.assertTrue;
import java.io.FileNotFoundException;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.SnapshotException;
@@ -47,7 +47,7 @@ public class TestSnapshotPathINodes {
private static final short REPLICATION = 3;
static private final Path dir = new Path("/TestSnapshot");
-
+
static private final Path sub1 = new Path(dir, "sub1");
static private final Path file1 = new Path(sub1, "file1");
static private final Path file2 = new Path(sub1, "file2");
@@ -67,7 +67,7 @@ public static void setUp() throws Exception {
FSNamesystem fsn = cluster.getNamesystem();
fsdir = fsn.getFSDirectory();
-
+
hdfs = cluster.getFileSystem();
}
@@ -89,7 +89,7 @@ public static void tearDown() throws Exception {
public void testAllowSnapshot() throws Exception {
final String pathStr = sub1.toString();
final INode before = fsdir.getINode(pathStr);
-
+
// Before a directory is snapshottable
Assert.assertFalse(before.asDirectory().isSnapshottable());
@@ -100,21 +100,21 @@ public void testAllowSnapshot() throws Exception {
final INode after = fsdir.getINode(pathStr);
Assert.assertTrue(after.asDirectory().isSnapshottable());
}
-
+
hdfs.disallowSnapshot(path);
{
final INode after = fsdir.getINode(pathStr);
Assert.assertFalse(after.asDirectory().isSnapshottable());
}
}
-
+
static Snapshot getSnapshot(INodesInPath inodesInPath, String name,
int index) {
if (name == null) {
return null;
}
final INode inode = inodesInPath.getINode(index - 1);
- return inode.asDirectory().getSnapshot(DFSUtil.string2Bytes(name));
+ return inode.asDirectory().getSnapshot(name.getBytes(UTF_8));
}
static void assertSnapshot(INodesInPath inodesInPath, boolean isSnapshot,
@@ -136,7 +136,7 @@ static void assertINodeFile(INode inode, Path path) {
assertEquals(INodeFile.class, inode.getClass());
}
- /**
+ /**
* for normal (non-snapshot) file.
*/
@Test (timeout=15000)
@@ -178,8 +178,8 @@ public void testNonSnapshotPathINodes() throws Exception {
assertSnapshot(nodesInPath, false, null, -1);
assertEquals(nodesInPath.getLastINode().getFullPathName(), file1.toString());
}
-
- /**
+
+ /**
* for snapshot file.
*/
@Test (timeout=15000)
@@ -195,13 +195,13 @@ public void testSnapshotPathINodes() throws Exception {
INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir,
components, false);
// Length of inodes should be (components.length - 1), since we will ignore
- // ".snapshot"
+ // ".snapshot"
assertEquals(nodesInPath.length(), components.length - 1);
// SnapshotRootIndex should be 3: {root, Testsnapshot, sub1, s1, file1}
final Snapshot snapshot = getSnapshot(nodesInPath, "s1", 3);
assertSnapshot(nodesInPath, true, snapshot, 3);
assertEquals(".snapshot/s1",
- DFSUtil.bytes2String(nodesInPath.getPathComponent(3)));
+ new String(nodesInPath.getPathComponent(3), UTF_8));
assertTrue(nodesInPath.getINode(3) instanceof Snapshot.Root);
assertEquals("s1", nodesInPath.getINode(3).getLocalName());
@@ -209,7 +209,7 @@ public void testSnapshotPathINodes() throws Exception {
INode snapshotFileNode = nodesInPath.getLastINode();
assertINodeFile(snapshotFileNode, file1);
assertTrue(snapshotFileNode.getParent().isWithSnapshot());
-
+
// Call getExistingPathINodes and request only one INode.
nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, false);
assertEquals(nodesInPath.length(), components.length - 1);
@@ -217,7 +217,7 @@ public void testSnapshotPathINodes() throws Exception {
// Check the INode for file1 (snapshot file)
assertINodeFile(nodesInPath.getLastINode(), file1);
- // Resolve the path "/TestSnapshot/sub1/.snapshot"
+ // Resolve the path "/TestSnapshot/sub1/.snapshot"
String dotSnapshotPath = sub1.toString() + "/.snapshot";
components = INode.getPathComponents(dotSnapshotPath);
nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, false);
@@ -225,19 +225,19 @@ public void testSnapshotPathINodes() throws Exception {
// since we put a null in the inode array for ".snapshot"
assertEquals(nodesInPath.length(), components.length);
assertEquals(".snapshot",
- DFSUtil.bytes2String(nodesInPath.getLastLocalName()));
+ new String(nodesInPath.getLastLocalName(), UTF_8));
assertNull(nodesInPath.getLastINode());
// ensure parent inodes can strip the .snapshot
assertEquals(sub1.toString(),
nodesInPath.getParentINodesInPath().getPath());
- // No SnapshotRoot dir is included in the resolved inodes
+ // No SnapshotRoot dir is included in the resolved inodes
assertSnapshot(nodesInPath, true, snapshot, -1);
// The last INode should be null, the last but 1 should be sub1
assertNull(nodesInPath.getLastINode());
assertEquals(nodesInPath.getINode(-2).getFullPathName(), sub1.toString());
assertTrue(nodesInPath.getINode(-2).isDirectory());
-
+
String[] invalidPathComponent = {"invalidDir", "foo", ".snapshot", "bar"};
Path invalidPath = new Path(invalidPathComponent[0]);
for(int i = 1; i < invalidPathComponent.length; i++) {
@@ -252,8 +252,8 @@ public void testSnapshotPathINodes() throws Exception {
hdfs.deleteSnapshot(sub1, "s1");
hdfs.disallowSnapshot(sub1);
}
-
- /**
+
+ /**
* for snapshot file after deleting the original file.
*/
@Test (timeout=15000)
@@ -262,10 +262,10 @@ public void testSnapshotPathINodesAfterDeletion() throws Exception {
// pointing to a snapshot file
hdfs.allowSnapshot(sub1);
hdfs.createSnapshot(sub1, "s2");
-
+
// Delete the original file /TestSnapshot/sub1/file1
hdfs.delete(file1, false);
-
+
final Snapshot snapshot;
{
// Resolve the path for the snapshot file
@@ -275,12 +275,12 @@ public void testSnapshotPathINodesAfterDeletion() throws Exception {
INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir,
components, false);
// Length of inodes should be (components.length - 1), since we will ignore
- // ".snapshot"
+ // ".snapshot"
assertEquals(nodesInPath.length(), components.length - 1);
// SnapshotRootIndex should be 3: {root, Testsnapshot, sub1, s2, file1}
snapshot = getSnapshot(nodesInPath, "s2", 3);
assertSnapshot(nodesInPath, true, snapshot, 3);
-
+
// Check the INode for file1 (snapshot file)
final INode inode = nodesInPath.getLastINode();
assertEquals(file1.getName(), inode.getLocalName());
@@ -328,7 +328,7 @@ public void testSnapshotPathINodesWithAddedFile() throws Exception {
// pointing to a snapshot file
hdfs.allowSnapshot(sub1);
hdfs.createSnapshot(sub1, "s4");
-
+
// Add a new file /TestSnapshot/sub1/file3
final Path file3 = new Path(sub1, "file3");
DFSTestUtil.createFile(hdfs, file3, 1024, REPLICATION, seed);
@@ -341,7 +341,7 @@ public void testSnapshotPathINodesWithAddedFile() throws Exception {
INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir,
components, false);
// Length of inodes should be (components.length - 1), since we will ignore
- // ".snapshot"
+ // ".snapshot"
assertEquals(nodesInPath.length(), components.length - 1);
// The number of non-null inodes should be components.length - 2, since
// snapshot of file3 does not exist
@@ -350,7 +350,7 @@ public void testSnapshotPathINodesWithAddedFile() throws Exception {
// SnapshotRootIndex should still be 3: {root, Testsnapshot, sub1, s4, null}
assertSnapshot(nodesInPath, true, s4, 3);
-
+
// Check the last INode in inodes, which should be null
assertNull(nodesInPath.getINode(nodesInPath.length() - 1));
}
@@ -375,8 +375,8 @@ public void testSnapshotPathINodesWithAddedFile() throws Exception {
hdfs.deleteSnapshot(sub1, "s4");
hdfs.disallowSnapshot(sub1);
}
-
- /**
+
+ /**
* for snapshot file while modifying file after snapshot.
*/
@Test (timeout=15000)
@@ -394,12 +394,12 @@ public void testSnapshotPathINodesAfterModification() throws Exception {
// record the modification time of the inode
final long modTime = nodesInPath.getINode(nodesInPath.length() - 1)
.getModificationTime();
-
+
// Create a snapshot for the dir, and check the inodes for the path
// pointing to a snapshot file
hdfs.allowSnapshot(sub1);
hdfs.createSnapshot(sub1, "s3");
-
+
// Modify file1
DFSTestUtil.appendFile(hdfs, file1, "the content for appending");
@@ -409,7 +409,7 @@ public void testSnapshotPathINodesAfterModification() throws Exception {
INodesInPath ssNodesInPath = INodesInPath.resolve(fsdir.rootDir,
components, false);
// Length of ssInodes should be (components.length - 1), since we will
- // ignore ".snapshot"
+ // ignore ".snapshot"
assertEquals(ssNodesInPath.length(), components.length - 1);
final Snapshot s3 = getSnapshot(ssNodesInPath, "s3", 3);
assertSnapshot(ssNodesInPath, true, s3, 3);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java
index 5b7ed41f42e95..070ca7486dd88 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java
@@ -21,6 +21,7 @@
import static org.junit.Assert.assertTrue;
import java.io.IOException;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.Random;
import java.util.regex.Pattern;
@@ -30,7 +31,6 @@
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -61,11 +61,11 @@ public class TestNestedSnapshots {
private static final short REPLICATION = 3;
private static final long BLOCKSIZE = 1024;
-
+
private static final Configuration conf = new Configuration();
private static MiniDFSCluster cluster;
private static DistributedFileSystem hdfs;
-
+
@Before
public void setUp() throws Exception {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
@@ -81,12 +81,12 @@ public void tearDown() throws Exception {
cluster = null;
}
}
-
+
/**
* Create a snapshot for /test/foo and create another snapshot for
* /test/foo/bar. Files created before the snapshots should appear in both
* snapshots and the files created after the snapshots should not appear in
- * any of the snapshots.
+ * any of the snapshots.
*/
@Test (timeout=300000)
public void testNestedSnapshots() throws Exception {
@@ -99,14 +99,14 @@ public void testNestedSnapshots() throws Exception {
print("create file " + file1);
final String s1name = "foo-s1";
- final Path s1path = SnapshotTestHelper.getSnapshotRoot(foo, s1name);
+ final Path s1path = SnapshotTestHelper.getSnapshotRoot(foo, s1name);
hdfs.allowSnapshot(foo);
print("allow snapshot " + foo);
hdfs.createSnapshot(foo, s1name);
print("create snapshot " + s1name);
final String s2name = "bar-s2";
- final Path s2path = SnapshotTestHelper.getSnapshotRoot(bar, s2name);
+ final Path s2path = SnapshotTestHelper.getSnapshotRoot(bar, s2name);
hdfs.allowSnapshot(bar);
print("allow snapshot " + bar);
hdfs.createSnapshot(bar, s2name);
@@ -115,7 +115,7 @@ public void testNestedSnapshots() throws Exception {
final Path file2 = new Path(bar, "file2");
DFSTestUtil.createFile(hdfs, file2, BLOCKSIZE, REPLICATION, SEED);
print("create file " + file2);
-
+
assertFile(s1path, s2path, file1, true, true, true);
assertFile(s1path, s2path, file2, true, false, false);
@@ -130,11 +130,11 @@ public void testNestedSnapshots() throws Exception {
print("delete snapshot " + rootSnapshot);
hdfs.disallowSnapshot(rootPath);
print("disallow snapshot " + rootStr);
-
+
//change foo to non-snapshottable
hdfs.deleteSnapshot(foo, s1name);
hdfs.disallowSnapshot(foo);
-
+
//test disallow nested snapshots
cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(false);
try {
@@ -170,7 +170,7 @@ public void testNestedSnapshots() throws Exception {
se, "ancestor");
}
}
-
+
static void assertNestedSnapshotException(SnapshotException se, String substring) {
Assert.assertTrue(se.getMessage().startsWith(
"Nested snapshottable directories not allowed"));
@@ -212,7 +212,7 @@ public void testSnapshotLimit() throws Exception {
final String snapshotName = "s" + s;
hdfs.createSnapshot(dir, snapshotName);
- //create a file occasionally
+ //create a file occasionally
if (s % step == 0) {
final Path file = new Path(dirStr, "f" + s);
DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, SEED);
@@ -257,7 +257,7 @@ public void testSnapshotName() throws Exception {
final Path snapshotPath = hdfs.createSnapshot(dir);
//check snapshot path and the default snapshot name
- final String snapshotName = snapshotPath.getName();
+ final String snapshotName = snapshotPath.getName();
Assert.assertTrue("snapshotName=" + snapshotName, Pattern.matches(
"s\\d\\d\\d\\d\\d\\d\\d\\d-\\d\\d\\d\\d\\d\\d\\.\\d\\d\\d",
snapshotName));
@@ -275,7 +275,7 @@ public void testIdCmp() {
final PermissionStatus perm = PermissionStatus.createImmutable(
"user", "group", FsPermission.createImmutable((short)0));
final INodeDirectory snapshottable = new INodeDirectory(0,
- DFSUtil.string2Bytes("foo"), perm, 0L);
+ "foo".getBytes(UTF_8), perm, 0L);
snapshottable.addSnapshottableFeature();
final Snapshot[] snapshots = {
new Snapshot(1, "s1", snapshottable),
@@ -288,7 +288,7 @@ public void testIdCmp() {
for(Snapshot s : snapshots) {
Assert.assertTrue(Snapshot.ID_COMPARATOR.compare(null, s) > 0);
Assert.assertTrue(Snapshot.ID_COMPARATOR.compare(s, null) < 0);
-
+
for(Snapshot t : snapshots) {
final int expected = s.getRoot().getLocalName().compareTo(
t.getRoot().getLocalName());
@@ -299,7 +299,7 @@ public void testIdCmp() {
}
}
}
-
+
/**
* When we have nested snapshottable directories and if we try to reset the
* snapshottable descendant back to an regular directory, we need to replace
@@ -312,19 +312,19 @@ public void testDisallowNestedSnapshottableDir() throws Exception {
final Path dir = new Path("/dir");
final Path sub = new Path(dir, "sub");
hdfs.mkdirs(sub);
-
+
SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
final Path file = new Path(sub, "file");
DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, SEED);
-
+
FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
INode subNode = fsdir.getINode(sub.toString());
assertTrue(subNode.asDirectory().isWithSnapshot());
-
+
hdfs.allowSnapshot(sub);
subNode = fsdir.getINode(sub.toString());
assertTrue(subNode.isDirectory() && subNode.asDirectory().isSnapshottable());
-
+
hdfs.disallowSnapshot(sub);
subNode = fsdir.getINode(sub.toString());
assertTrue(subNode.asDirectory().isWithSnapshot());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
index d1b3aa6f870c6..a02a391246799 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
@@ -29,6 +29,7 @@
import java.io.File;
import java.io.IOException;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.EnumSet;
import java.util.List;
import java.util.Random;
@@ -46,7 +47,6 @@
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSOutputStream;
import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
@@ -84,13 +84,13 @@ public class TestRenameWithSnapshots {
SnapshotTestHelper.disableLogs();
}
private static final Log LOG = LogFactory.getLog(TestRenameWithSnapshots.class);
-
+
private static final long SEED = 0;
private static final short REPL = 3;
private static final short REPL_1 = 2;
private static final short REPL_2 = 1;
private static final long BLOCKSIZE = 1024;
-
+
private static final Configuration conf = new Configuration();
private static MiniDFSCluster cluster;
private static FSNamesystem fsn;
@@ -106,7 +106,7 @@ public class TestRenameWithSnapshots {
static private final String snap1 = "snap1";
static private final String snap2 = "snap2";
-
+
@Before
public void setUp() throws Exception {
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
@@ -139,7 +139,7 @@ public void testRenameFromSDir2NonSDir() throws Exception {
final Path foo = new Path(abc, "foo");
DFSTestUtil.createFile(hdfs, foo, BLOCKSIZE, REPL, SEED);
hdfs.createSnapshot(abc, "s0");
-
+
try {
hdfs.rename(abc, new Path(dirStr, "tmp"));
fail("Expect exception since " + abc
@@ -154,7 +154,7 @@ public void testRenameFromSDir2NonSDir() throws Exception {
hdfs.mkdirs(xyz, new FsPermission((short)0777));
final Path bar = new Path(xyz, "bar");
hdfs.rename(foo, bar);
-
+
final INode fooRef = fsdir.getINode(
SnapshotTestHelper.getSnapshotPath(abc, "s0", "foo").toString());
Assert.assertTrue(fooRef.isReference());
@@ -168,23 +168,23 @@ public void testRenameFromSDir2NonSDir() throws Exception {
Assert.assertTrue(barRef.isReference());
Assert.assertSame(withCount, barRef.asReference().getReferredINode());
-
+
hdfs.delete(bar, false);
Assert.assertEquals(1, withCount.getReferenceCount());
}
-
+
private static boolean existsInDiffReport(List entries,
DiffType type, String sourcePath, String targetPath) {
for (DiffReportEntry entry : entries) {
- if (entry.equals(new DiffReportEntry(type, DFSUtil
- .string2Bytes(sourcePath), targetPath == null ? null : DFSUtil
- .string2Bytes(targetPath)))) {
+ if(entry.equals(new DiffReportEntry(type,
+ sourcePath.getBytes(UTF_8),
+ targetPath == null ? null : targetPath.getBytes(UTF_8)))) {
return true;
}
}
return false;
}
-
+
/**
* Rename a file under a snapshottable directory, file does not exist
* in a snapshot.
@@ -235,12 +235,12 @@ public void testRenameTwiceInSnapshot() throws Exception {
DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, REPL, SEED);
hdfs.createSnapshot(sub1, snap1);
hdfs.rename(file1, file2);
-
+
hdfs.createSnapshot(sub1, snap2);
hdfs.rename(file2, file3);
SnapshotDiffReport diffReport;
-
+
// Query the diff report and make sure it looks as expected.
diffReport = hdfs.getSnapshotDiffReport(sub1, snap1, snap2);
LOG.info("DiffList is " + diffReport.toString());
@@ -249,7 +249,7 @@ public void testRenameTwiceInSnapshot() throws Exception {
assertTrue(existsInDiffReport(entries, DiffType.MODIFY, "", null));
assertTrue(existsInDiffReport(entries, DiffType.RENAME, file1.getName(),
file2.getName()));
-
+
diffReport = hdfs.getSnapshotDiffReport(sub1, snap2, "");
LOG.info("DiffList is " + diffReport.toString());
entries = diffReport.getDiffList();
@@ -257,7 +257,7 @@ public void testRenameTwiceInSnapshot() throws Exception {
assertTrue(existsInDiffReport(entries, DiffType.MODIFY, "", null));
assertTrue(existsInDiffReport(entries, DiffType.RENAME, file2.getName(),
file3.getName()));
-
+
diffReport = hdfs.getSnapshotDiffReport(sub1, snap1, "");
LOG.info("DiffList is " + diffReport.toString());
entries = diffReport.getDiffList();
@@ -266,14 +266,14 @@ public void testRenameTwiceInSnapshot() throws Exception {
assertTrue(existsInDiffReport(entries, DiffType.RENAME, file1.getName(),
file3.getName()));
}
-
+
@Test (timeout=60000)
public void testRenameFileInSubDirOfDirWithSnapshot() throws Exception {
final Path sub2 = new Path(sub1, "sub2");
final Path sub2file1 = new Path(sub2, "sub2file1");
final Path sub2file2 = new Path(sub2, "sub2file2");
final String sub1snap1 = "sub1snap1";
-
+
hdfs.mkdirs(sub1);
hdfs.mkdirs(sub2);
DFSTestUtil.createFile(hdfs, sub2file1, BLOCKSIZE, REPL, SEED);
@@ -299,15 +299,15 @@ public void testRenameDirectoryInSnapshot() throws Exception {
final Path sub3 = new Path(sub1, "sub3");
final Path sub2file1 = new Path(sub2, "sub2file1");
final String sub1snap1 = "sub1snap1";
-
+
hdfs.mkdirs(sub1);
hdfs.mkdirs(sub2);
DFSTestUtil.createFile(hdfs, sub2file1, BLOCKSIZE, REPL, SEED);
SnapshotTestHelper.createSnapshot(hdfs, sub1, sub1snap1);
-
+
// First rename the sub-directory.
hdfs.rename(sub2, sub3);
-
+
// Query the diff report and make sure it looks as expected.
SnapshotDiffReport diffReport = hdfs.getSnapshotDiffReport(sub1, sub1snap1,
"");
@@ -318,7 +318,7 @@ public void testRenameDirectoryInSnapshot() throws Exception {
assertTrue(existsInDiffReport(entries, DiffType.RENAME, sub2.getName(),
sub3.getName()));
}
-
+
/**
* After the following steps:
*
@@ -328,7 +328,7 @@ public void testRenameDirectoryInSnapshot() throws Exception {
* 4. Take snapshot s3 on /dir1 at time t3.
* 5. Rename /dir2/foo/ to /dir1/foo/.
*
- * When changes happening on foo, the diff should be recorded in snapshot s2.
+ * When changes happening on foo, the diff should be recorded in snapshot s2.
*/
@Test (timeout=60000)
public void testRenameDirAcrossSnapshottableDirs() throws Exception {
@@ -341,29 +341,29 @@ public void testRenameDirAcrossSnapshottableDirs() throws Exception {
final Path bar2 = new Path(foo, "bar2");
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
DFSTestUtil.createFile(hdfs, bar2, BLOCKSIZE, REPL, SEED);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
-
+
hdfs.setReplication(bar2, REPL_1);
hdfs.delete(bar, true);
-
+
hdfs.createSnapshot(sdir1, "s3");
-
+
final Path newfoo = new Path(sdir1, "foo");
hdfs.rename(foo, newfoo);
-
- // still can visit the snapshot copy of bar through
+
+ // still can visit the snapshot copy of bar through
// /dir2/.snapshot/s2/foo/bar
final Path snapshotBar = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
"foo/bar");
assertTrue(hdfs.exists(snapshotBar));
-
+
// delete bar2
final Path newBar2 = new Path(newfoo, "bar2");
assertTrue(hdfs.exists(newBar2));
hdfs.delete(newBar2, true);
-
+
// /dir2/.snapshot/s2/foo/bar2 should still work
final Path bar2_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
"foo/bar2");
@@ -374,7 +374,7 @@ public void testRenameDirAcrossSnapshottableDirs() throws Exception {
"foo/bar2");
assertFalse(hdfs.exists(bar2_s3));
}
-
+
/**
* Rename a single file across snapshottable dirs.
*/
@@ -386,33 +386,33 @@ public void testRenameFileAcrossSnapshottableDirs() throws Exception {
hdfs.mkdirs(sdir2);
final Path foo = new Path(sdir2, "foo");
DFSTestUtil.createFile(hdfs, foo, BLOCKSIZE, REPL, SEED);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
hdfs.createSnapshot(sdir1, "s3");
-
+
final Path newfoo = new Path(sdir1, "foo");
hdfs.rename(foo, newfoo);
-
+
// change the replication factor of foo
hdfs.setReplication(newfoo, REPL_1);
-
+
// /dir2/.snapshot/s2/foo should still work
final Path foo_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
"foo");
assertTrue(hdfs.exists(foo_s2));
FileStatus status = hdfs.getFileStatus(foo_s2);
assertEquals(REPL, status.getReplication());
-
+
final Path foo_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3",
"foo");
assertFalse(hdfs.exists(foo_s3));
INodeDirectory sdir2Node = fsdir.getINode(sdir2.toString()).asDirectory();
- Snapshot s2 = sdir2Node.getSnapshot(DFSUtil.string2Bytes("s2"));
+ Snapshot s2 = sdir2Node.getSnapshot("s2".getBytes(UTF_8));
INodeFile sfoo = fsdir.getINode(newfoo.toString()).asFile();
assertEquals(s2.getId(), sfoo.getDiffs().getLastSnapshotId());
}
-
+
/**
* Test renaming a dir and then delete snapshots.
*/
@@ -427,23 +427,23 @@ public void testRenameDirAndDeleteSnapshot_1() throws Exception {
final Path bar2 = new Path(foo, "bar2");
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
DFSTestUtil.createFile(hdfs, bar2, BLOCKSIZE, REPL, SEED);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
hdfs.createSnapshot(sdir1, "s3");
-
+
final Path newfoo = new Path(sdir1, "foo");
hdfs.rename(foo, newfoo);
-
+
final Path newbar = new Path(newfoo, bar.getName());
final Path newbar2 = new Path(newfoo, bar2.getName());
final Path newbar3 = new Path(newfoo, "bar3");
DFSTestUtil.createFile(hdfs, newbar3, BLOCKSIZE, REPL, SEED);
-
+
hdfs.createSnapshot(sdir1, "s4");
hdfs.delete(newbar, true);
hdfs.delete(newbar3, true);
-
+
assertFalse(hdfs.exists(newbar3));
assertFalse(hdfs.exists(bar));
final Path bar_s4 = SnapshotTestHelper.getSnapshotPath(sdir1, "s4",
@@ -452,14 +452,14 @@ public void testRenameDirAndDeleteSnapshot_1() throws Exception {
"foo/bar3");
assertTrue(hdfs.exists(bar_s4));
assertTrue(hdfs.exists(bar3_s4));
-
+
hdfs.createSnapshot(sdir1, "s5");
hdfs.delete(newbar2, true);
assertFalse(hdfs.exists(bar2));
final Path bar2_s5 = SnapshotTestHelper.getSnapshotPath(sdir1, "s5",
"foo/bar2");
assertTrue(hdfs.exists(bar2_s5));
-
+
// delete snapshot s5. The diff of s5 should be combined to s4
hdfs.deleteSnapshot(sdir1, "s5");
restartClusterAndCheckImage(true);
@@ -467,11 +467,11 @@ public void testRenameDirAndDeleteSnapshot_1() throws Exception {
final Path bar2_s4 = SnapshotTestHelper.getSnapshotPath(sdir1, "s4",
"foo/bar2");
assertTrue(hdfs.exists(bar2_s4));
-
+
// delete snapshot s4. The diff of s4 should be combined to s2 instead of
// s3.
hdfs.deleteSnapshot(sdir1, "s4");
-
+
assertFalse(hdfs.exists(bar_s4));
Path bar_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3", "foo/bar");
assertFalse(hdfs.exists(bar_s3));
@@ -480,7 +480,7 @@ public void testRenameDirAndDeleteSnapshot_1() throws Exception {
final Path bar_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
"foo/bar");
assertTrue(hdfs.exists(bar_s2));
-
+
assertFalse(hdfs.exists(bar2_s4));
Path bar2_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3", "foo/bar2");
assertFalse(hdfs.exists(bar2_s3));
@@ -489,7 +489,7 @@ public void testRenameDirAndDeleteSnapshot_1() throws Exception {
final Path bar2_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
"foo/bar2");
assertTrue(hdfs.exists(bar2_s2));
-
+
assertFalse(hdfs.exists(bar3_s4));
Path bar3_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3", "foo/bar3");
assertFalse(hdfs.exists(bar3_s3));
@@ -498,15 +498,15 @@ public void testRenameDirAndDeleteSnapshot_1() throws Exception {
final Path bar3_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
"foo/bar3");
assertFalse(hdfs.exists(bar3_s2));
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
-
+
// delete snapshot s2.
hdfs.deleteSnapshot(sdir2, "s2");
assertFalse(hdfs.exists(bar_s2));
assertFalse(hdfs.exists(bar2_s2));
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
hdfs.deleteSnapshot(sdir1, "s3");
@@ -514,13 +514,13 @@ public void testRenameDirAndDeleteSnapshot_1() throws Exception {
hdfs.deleteSnapshot(sdir1, "s1");
restartClusterAndCheckImage(true);
}
-
+
private void restartClusterAndCheckImage(boolean compareQuota)
throws IOException {
File fsnBefore = new File(testDir, "dumptree_before");
File fsnMiddle = new File(testDir, "dumptree_middle");
File fsnAfter = new File(testDir, "dumptree_after");
-
+
SnapshotTestHelper.dumpTree2File(fsdir, fsnBefore);
cluster.shutdown(false, false);
@@ -530,9 +530,9 @@ private void restartClusterAndCheckImage(boolean compareQuota)
fsn = cluster.getNamesystem();
fsdir = fsn.getFSDirectory();
hdfs = cluster.getFileSystem();
- // later check fsnMiddle to see if the edit log is applied correctly
+ // later check fsnMiddle to see if the edit log is applied correctly
SnapshotTestHelper.dumpTree2File(fsdir, fsnMiddle);
-
+
// save namespace and restart cluster
hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
hdfs.saveNamespace();
@@ -546,13 +546,13 @@ private void restartClusterAndCheckImage(boolean compareQuota)
hdfs = cluster.getFileSystem();
// dump the namespace loaded from fsimage
SnapshotTestHelper.dumpTree2File(fsdir, fsnAfter);
-
+
SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnMiddle,
compareQuota);
SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnAfter,
compareQuota);
}
-
+
/**
* Test renaming a file and then delete snapshots.
*/
@@ -564,40 +564,40 @@ public void testRenameFileAndDeleteSnapshot() throws Exception {
hdfs.mkdirs(sdir2);
final Path foo = new Path(sdir2, "foo");
DFSTestUtil.createFile(hdfs, foo, BLOCKSIZE, REPL, SEED);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
hdfs.createSnapshot(sdir1, "s3");
-
+
final Path newfoo = new Path(sdir1, "foo");
hdfs.rename(foo, newfoo);
-
+
hdfs.setReplication(newfoo, REPL_1);
-
+
hdfs.createSnapshot(sdir1, "s4");
hdfs.setReplication(newfoo, REPL_2);
-
+
FileStatus status = hdfs.getFileStatus(newfoo);
assertEquals(REPL_2, status.getReplication());
final Path foo_s4 = SnapshotTestHelper.getSnapshotPath(sdir1, "s4", "foo");
status = hdfs.getFileStatus(foo_s4);
assertEquals(REPL_1, status.getReplication());
-
+
hdfs.createSnapshot(sdir1, "s5");
final Path foo_s5 = SnapshotTestHelper.getSnapshotPath(sdir1, "s5", "foo");
status = hdfs.getFileStatus(foo_s5);
assertEquals(REPL_2, status.getReplication());
-
+
// delete snapshot s5.
hdfs.deleteSnapshot(sdir1, "s5");
restartClusterAndCheckImage(true);
assertFalse(hdfs.exists(foo_s5));
status = hdfs.getFileStatus(foo_s4);
assertEquals(REPL_1, status.getReplication());
-
+
// delete snapshot s4.
hdfs.deleteSnapshot(sdir1, "s4");
-
+
assertFalse(hdfs.exists(foo_s4));
Path foo_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3", "foo");
assertFalse(hdfs.exists(foo_s3));
@@ -607,20 +607,20 @@ public void testRenameFileAndDeleteSnapshot() throws Exception {
assertTrue(hdfs.exists(foo_s2));
status = hdfs.getFileStatus(foo_s2);
assertEquals(REPL, status.getReplication());
-
+
INodeFile snode = fsdir.getINode(newfoo.toString()).asFile();
assertEquals(1, snode.getDiffs().asList().size());
INodeDirectory sdir2Node = fsdir.getINode(sdir2.toString()).asDirectory();
- Snapshot s2 = sdir2Node.getSnapshot(DFSUtil.string2Bytes("s2"));
+ Snapshot s2 = sdir2Node.getSnapshot("s2".getBytes(UTF_8));
assertEquals(s2.getId(), snode.getDiffs().getLastSnapshotId());
-
+
// restart cluster
restartClusterAndCheckImage(true);
-
+
// delete snapshot s2.
hdfs.deleteSnapshot(sdir2, "s2");
assertFalse(hdfs.exists(foo_s2));
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
hdfs.deleteSnapshot(sdir1, "s3");
@@ -628,11 +628,11 @@ public void testRenameFileAndDeleteSnapshot() throws Exception {
hdfs.deleteSnapshot(sdir1, "s1");
restartClusterAndCheckImage(true);
}
-
+
/**
- * Test rename a dir and a file multiple times across snapshottable
+ * Test rename a dir and a file multiple times across snapshottable
* directories: /dir1/foo -> /dir2/foo -> /dir3/foo -> /dir2/foo -> /dir1/foo
- *
+ *
* Only create snapshots in the beginning (before the rename).
*/
@Test
@@ -643,31 +643,31 @@ public void testRenameMoreThanOnceAcrossSnapDirs() throws Exception {
hdfs.mkdirs(sdir1);
hdfs.mkdirs(sdir2);
hdfs.mkdirs(sdir3);
-
+
final Path foo_dir1 = new Path(sdir1, "foo");
final Path bar1_dir1 = new Path(foo_dir1, "bar1");
final Path bar2_dir1 = new Path(sdir1, "bar");
DFSTestUtil.createFile(hdfs, bar1_dir1, BLOCKSIZE, REPL, SEED);
DFSTestUtil.createFile(hdfs, bar2_dir1, BLOCKSIZE, REPL, SEED);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
SnapshotTestHelper.createSnapshot(hdfs, sdir3, "s3");
-
+
// 1. /dir1/foo -> /dir2/foo, /dir1/bar -> /dir2/bar
final Path foo_dir2 = new Path(sdir2, "foo");
hdfs.rename(foo_dir1, foo_dir2);
final Path bar2_dir2 = new Path(sdir2, "bar");
hdfs.rename(bar2_dir1, bar2_dir2);
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
-
+
// modification on /dir2/foo and /dir2/bar
final Path bar1_dir2 = new Path(foo_dir2, "bar1");
hdfs.setReplication(bar1_dir2, REPL_1);
hdfs.setReplication(bar2_dir2, REPL_1);
-
+
// check
final Path bar1_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
"foo/bar1");
@@ -689,21 +689,21 @@ public void testRenameMoreThanOnceAcrossSnapDirs() throws Exception {
assertEquals(REPL, statusBar2.getReplication());
statusBar2 = hdfs.getFileStatus(bar2_dir2);
assertEquals(REPL_1, statusBar2.getReplication());
-
+
// 2. /dir2/foo -> /dir3/foo, /dir2/bar -> /dir3/bar
final Path foo_dir3 = new Path(sdir3, "foo");
hdfs.rename(foo_dir2, foo_dir3);
final Path bar2_dir3 = new Path(sdir3, "bar");
hdfs.rename(bar2_dir2, bar2_dir3);
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
-
+
// modification on /dir3/foo and /dir3/bar
final Path bar1_dir3 = new Path(foo_dir3, "bar1");
hdfs.setReplication(bar1_dir3, REPL_2);
hdfs.setReplication(bar2_dir3, REPL_2);
-
+
// check
final Path bar1_s3 = SnapshotTestHelper.getSnapshotPath(sdir3, "s3",
"foo/bar1");
@@ -723,18 +723,18 @@ public void testRenameMoreThanOnceAcrossSnapDirs() throws Exception {
assertEquals(REPL, statusBar2.getReplication());
statusBar2 = hdfs.getFileStatus(bar2_dir3);
assertEquals(REPL_2, statusBar2.getReplication());
-
+
// 3. /dir3/foo -> /dir2/foo, /dir3/bar -> /dir2/bar
hdfs.rename(foo_dir3, foo_dir2);
hdfs.rename(bar2_dir3, bar2_dir2);
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
-
+
// modification on /dir2/foo
hdfs.setReplication(bar1_dir2, REPL);
hdfs.setReplication(bar2_dir2, REPL);
-
+
// check
assertTrue(hdfs.exists(bar1_s1));
assertTrue(hdfs.exists(bar2_s1));
@@ -750,11 +750,11 @@ public void testRenameMoreThanOnceAcrossSnapDirs() throws Exception {
assertEquals(REPL, statusBar2.getReplication());
statusBar2 = hdfs.getFileStatus(bar2_dir2);
assertEquals(REPL, statusBar2.getReplication());
-
+
// 4. /dir2/foo -> /dir1/foo, /dir2/bar -> /dir1/bar
hdfs.rename(foo_dir2, foo_dir1);
hdfs.rename(bar2_dir2, bar2_dir1);
-
+
// check the internal details
INodeReference fooRef = fsdir.getINode4Write(foo_dir1.toString())
.asReference();
@@ -766,13 +766,13 @@ public void testRenameMoreThanOnceAcrossSnapDirs() throws Exception {
INodeDirectory foo = fooWithCount.asDirectory();
assertEquals(1, foo.getDiffs().asList().size());
INodeDirectory sdir1Node = fsdir.getINode(sdir1.toString()).asDirectory();
- Snapshot s1 = sdir1Node.getSnapshot(DFSUtil.string2Bytes("s1"));
+ Snapshot s1 = sdir1Node.getSnapshot("s1".getBytes(UTF_8));
assertEquals(s1.getId(), foo.getDirectoryWithSnapshotFeature()
.getLastSnapshotId());
INodeFile bar1 = fsdir.getINode4Write(bar1_dir1.toString()).asFile();
assertEquals(1, bar1.getDiffs().asList().size());
assertEquals(s1.getId(), bar1.getDiffs().getLastSnapshotId());
-
+
INodeReference barRef = fsdir.getINode4Write(bar2_dir1.toString())
.asReference();
INodeReference.WithCount barWithCount = (WithCount) barRef
@@ -781,18 +781,18 @@ public void testRenameMoreThanOnceAcrossSnapDirs() throws Exception {
INodeFile bar = barWithCount.asFile();
assertEquals(1, bar.getDiffs().asList().size());
assertEquals(s1.getId(), bar.getDiffs().getLastSnapshotId());
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
-
+
// delete foo
hdfs.delete(foo_dir1, true);
restartClusterAndCheckImage(true);
hdfs.delete(bar2_dir1, true);
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
-
+
// check
assertTrue(hdfs.exists(bar1_s1));
assertTrue(hdfs.exists(bar2_s1));
@@ -807,21 +807,21 @@ public void testRenameMoreThanOnceAcrossSnapDirs() throws Exception {
assertEquals(REPL, statusBar1.getReplication());
statusBar2 = hdfs.getFileStatus(bar2_s1);
assertEquals(REPL, statusBar2.getReplication());
-
+
final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1", "foo");
fooRef = fsdir.getINode(foo_s1.toString()).asReference();
fooWithCount = (WithCount) fooRef.getReferredINode();
assertEquals(1, fooWithCount.getReferenceCount());
-
+
barRef = fsdir.getINode(bar2_s1.toString()).asReference();
barWithCount = (WithCount) barRef.getReferredINode();
assertEquals(1, barWithCount.getReferenceCount());
}
-
+
/**
- * Test rename a dir multiple times across snapshottable directories:
+ * Test rename a dir multiple times across snapshottable directories:
* /dir1/foo -> /dir2/foo -> /dir3/foo -> /dir2/foo -> /dir1/foo
- *
+ *
* Create snapshots after each rename.
*/
@Test
@@ -832,55 +832,55 @@ public void testRenameMoreThanOnceAcrossSnapDirs_2() throws Exception {
hdfs.mkdirs(sdir1);
hdfs.mkdirs(sdir2);
hdfs.mkdirs(sdir3);
-
+
final Path foo_dir1 = new Path(sdir1, "foo");
final Path bar1_dir1 = new Path(foo_dir1, "bar1");
final Path bar_dir1 = new Path(sdir1, "bar");
DFSTestUtil.createFile(hdfs, bar1_dir1, BLOCKSIZE, REPL, SEED);
DFSTestUtil.createFile(hdfs, bar_dir1, BLOCKSIZE, REPL, SEED);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
SnapshotTestHelper.createSnapshot(hdfs, sdir3, "s3");
-
+
// 1. /dir1/foo -> /dir2/foo, /dir1/bar -> /dir2/bar
final Path foo_dir2 = new Path(sdir2, "foo");
hdfs.rename(foo_dir1, foo_dir2);
final Path bar_dir2 = new Path(sdir2, "bar");
hdfs.rename(bar_dir1, bar_dir2);
-
+
// modification on /dir2/foo and /dir2/bar
final Path bar1_dir2 = new Path(foo_dir2, "bar1");
hdfs.setReplication(bar1_dir2, REPL_1);
hdfs.setReplication(bar_dir2, REPL_1);
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
-
+
// create snapshots
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s11");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s22");
SnapshotTestHelper.createSnapshot(hdfs, sdir3, "s33");
-
+
// 2. /dir2/foo -> /dir3/foo
final Path foo_dir3 = new Path(sdir3, "foo");
hdfs.rename(foo_dir2, foo_dir3);
final Path bar_dir3 = new Path(sdir3, "bar");
hdfs.rename(bar_dir2, bar_dir3);
-
+
// modification on /dir3/foo
final Path bar1_dir3 = new Path(foo_dir3, "bar1");
hdfs.setReplication(bar1_dir3, REPL_2);
hdfs.setReplication(bar_dir3, REPL_2);
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
-
+
// create snapshots
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s111");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s222");
SnapshotTestHelper.createSnapshot(hdfs, sdir3, "s333");
-
+
// check
final Path bar1_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
"foo/bar1");
@@ -900,7 +900,7 @@ public void testRenameMoreThanOnceAcrossSnapDirs_2() throws Exception {
assertTrue(hdfs.exists(bar_s1));
assertTrue(hdfs.exists(bar_s22));
assertTrue(hdfs.exists(bar_s333));
-
+
FileStatus statusBar1 = hdfs.getFileStatus(bar1_s1);
assertEquals(REPL, statusBar1.getReplication());
statusBar1 = hdfs.getFileStatus(bar1_dir3);
@@ -909,7 +909,7 @@ public void testRenameMoreThanOnceAcrossSnapDirs_2() throws Exception {
assertEquals(REPL_1, statusBar1.getReplication());
statusBar1 = hdfs.getFileStatus(bar1_s333);
assertEquals(REPL_2, statusBar1.getReplication());
-
+
FileStatus statusBar = hdfs.getFileStatus(bar_s1);
assertEquals(REPL, statusBar.getReplication());
statusBar = hdfs.getFileStatus(bar_dir3);
@@ -918,22 +918,22 @@ public void testRenameMoreThanOnceAcrossSnapDirs_2() throws Exception {
assertEquals(REPL_1, statusBar.getReplication());
statusBar = hdfs.getFileStatus(bar_s333);
assertEquals(REPL_2, statusBar.getReplication());
-
+
// 3. /dir3/foo -> /dir2/foo
hdfs.rename(foo_dir3, foo_dir2);
hdfs.rename(bar_dir3, bar_dir2);
-
+
// modification on /dir2/foo
hdfs.setReplication(bar1_dir2, REPL);
hdfs.setReplication(bar_dir2, REPL);
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
-
+
// create snapshots
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1111");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2222");
-
+
// check
final Path bar1_s2222 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2222",
"foo/bar1");
@@ -947,7 +947,7 @@ public void testRenameMoreThanOnceAcrossSnapDirs_2() throws Exception {
assertTrue(hdfs.exists(bar_s22));
assertTrue(hdfs.exists(bar_s333));
assertTrue(hdfs.exists(bar_s2222));
-
+
statusBar1 = hdfs.getFileStatus(bar1_s1);
assertEquals(REPL, statusBar1.getReplication());
statusBar1 = hdfs.getFileStatus(bar1_dir2);
@@ -958,7 +958,7 @@ public void testRenameMoreThanOnceAcrossSnapDirs_2() throws Exception {
assertEquals(REPL_2, statusBar1.getReplication());
statusBar1 = hdfs.getFileStatus(bar1_s2222);
assertEquals(REPL, statusBar1.getReplication());
-
+
statusBar = hdfs.getFileStatus(bar_s1);
assertEquals(REPL, statusBar.getReplication());
statusBar = hdfs.getFileStatus(bar_dir2);
@@ -969,16 +969,16 @@ public void testRenameMoreThanOnceAcrossSnapDirs_2() throws Exception {
assertEquals(REPL_2, statusBar.getReplication());
statusBar = hdfs.getFileStatus(bar_s2222);
assertEquals(REPL, statusBar.getReplication());
-
+
// 4. /dir2/foo -> /dir1/foo
hdfs.rename(foo_dir2, foo_dir1);
hdfs.rename(bar_dir2, bar_dir1);
-
+
// check the internal details
INodeDirectory sdir1Node = fsdir.getINode(sdir1.toString()).asDirectory();
INodeDirectory sdir2Node = fsdir.getINode(sdir2.toString()).asDirectory();
INodeDirectory sdir3Node = fsdir.getINode(sdir3.toString()).asDirectory();
-
+
INodeReference fooRef = fsdir.getINode4Write(foo_dir1.toString())
.asReference();
INodeReference.WithCount fooWithCount = (WithCount) fooRef.getReferredINode();
@@ -987,12 +987,12 @@ public void testRenameMoreThanOnceAcrossSnapDirs_2() throws Exception {
INodeDirectory foo = fooWithCount.asDirectory();
List fooDiffs = foo.getDiffs().asList();
assertEquals(4, fooDiffs.size());
-
- Snapshot s2222 = sdir2Node.getSnapshot(DFSUtil.string2Bytes("s2222"));
- Snapshot s333 = sdir3Node.getSnapshot(DFSUtil.string2Bytes("s333"));
- Snapshot s22 = sdir2Node.getSnapshot(DFSUtil.string2Bytes("s22"));
- Snapshot s1 = sdir1Node.getSnapshot(DFSUtil.string2Bytes("s1"));
-
+
+ Snapshot s2222 = sdir2Node.getSnapshot("s2222".getBytes(UTF_8));
+ Snapshot s333 = sdir3Node.getSnapshot("s333".getBytes(UTF_8));
+ Snapshot s22 = sdir2Node.getSnapshot("s22".getBytes(UTF_8));
+ Snapshot s1 = sdir1Node.getSnapshot("s1".getBytes(UTF_8));
+
assertEquals(s2222.getId(), fooDiffs.get(3).getSnapshotId());
assertEquals(s333.getId(), fooDiffs.get(2).getSnapshotId());
assertEquals(s22.getId(), fooDiffs.get(1).getSnapshotId());
@@ -1003,7 +1003,7 @@ public void testRenameMoreThanOnceAcrossSnapDirs_2() throws Exception {
assertEquals(s333.getId(), bar1Diffs.get(2).getSnapshotId());
assertEquals(s22.getId(), bar1Diffs.get(1).getSnapshotId());
assertEquals(s1.getId(), bar1Diffs.get(0).getSnapshotId());
-
+
INodeReference barRef = fsdir.getINode4Write(bar_dir1.toString())
.asReference();
INodeReference.WithCount barWithCount = (WithCount) barRef.getReferredINode();
@@ -1016,17 +1016,17 @@ public void testRenameMoreThanOnceAcrossSnapDirs_2() throws Exception {
assertEquals(s333.getId(), barDiffs.get(2).getSnapshotId());
assertEquals(s22.getId(), barDiffs.get(1).getSnapshotId());
assertEquals(s1.getId(), barDiffs.get(0).getSnapshotId());
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
-
+
// delete foo
hdfs.delete(foo_dir1, true);
hdfs.delete(bar_dir1, true);
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
-
+
// check
final Path bar1_s1111 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1111",
"foo/bar1");
@@ -1042,7 +1042,7 @@ public void testRenameMoreThanOnceAcrossSnapDirs_2() throws Exception {
assertTrue(hdfs.exists(bar_s333));
assertTrue(hdfs.exists(bar_s2222));
assertFalse(hdfs.exists(bar_s1111));
-
+
final Path foo_s2222 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2222",
"foo");
fooRef = fsdir.getINode(foo_s2222.toString()).asReference();
@@ -1055,7 +1055,7 @@ public void testRenameMoreThanOnceAcrossSnapDirs_2() throws Exception {
bar1Diffs = bar1.getDiffs().asList();
assertEquals(3, bar1Diffs.size());
assertEquals(s333.getId(), bar1Diffs.get(2).getSnapshotId());
-
+
barRef = fsdir.getINode(bar_s2222.toString()).asReference();
barWithCount = (WithCount) barRef.getReferredINode();
assertEquals(4, barWithCount.getReferenceCount());
@@ -1064,7 +1064,7 @@ public void testRenameMoreThanOnceAcrossSnapDirs_2() throws Exception {
assertEquals(4, barDiffs.size());
assertEquals(s2222.getId(), barDiffs.get(3).getSnapshotId());
}
-
+
/**
* Test rename from a non-snapshottable dir to a snapshottable dir
*/
@@ -1077,19 +1077,19 @@ public void testRenameFromNonSDir2SDir() throws Exception {
final Path foo = new Path(sdir1, "foo");
final Path bar = new Path(foo, "bar");
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir2, snap1);
-
+
final Path newfoo = new Path(sdir2, "foo");
hdfs.rename(foo, newfoo);
-
+
INode fooNode = fsdir.getINode4Write(newfoo.toString());
assertTrue(fooNode instanceof INodeDirectory);
}
-
+
/**
- * Test rename where the src/dst directories are both snapshottable
- * directories without snapshots. In such case we need to update the
+ * Test rename where the src/dst directories are both snapshottable
+ * directories without snapshots. In such case we need to update the
* snapshottable dir list in SnapshotManager.
*/
@Test (timeout=60000)
@@ -1100,14 +1100,14 @@ public void testRenameAndUpdateSnapshottableDirs() throws Exception {
final Path bar = new Path(sdir2, "bar");
hdfs.mkdirs(foo);
hdfs.mkdirs(bar);
-
+
hdfs.allowSnapshot(foo);
SnapshotTestHelper.createSnapshot(hdfs, bar, snap1);
assertEquals(2, fsn.getSnapshottableDirListing().length);
-
+
INodeDirectory fooNode = fsdir.getINode4Write(foo.toString()).asDirectory();
long fooId = fooNode.getId();
-
+
try {
hdfs.rename(foo, bar, Rename.OVERWRITE);
fail("Expect exception since " + bar
@@ -1116,7 +1116,7 @@ public void testRenameAndUpdateSnapshottableDirs() throws Exception {
GenericTestUtils.assertExceptionContains(bar.toString()
+ " is snapshottable and already has snapshots", e);
}
-
+
hdfs.deleteSnapshot(bar, snap1);
hdfs.rename(foo, bar, Rename.OVERWRITE);
SnapshottableDirectoryStatus[] dirs = fsn.getSnapshottableDirListing();
@@ -1124,7 +1124,7 @@ public void testRenameAndUpdateSnapshottableDirs() throws Exception {
assertEquals(bar, dirs[0].getFullPath());
assertEquals(fooId, dirs[0].getDirStatus().getFileId());
}
-
+
/**
* After rename, delete the snapshot in src
*/
@@ -1137,35 +1137,35 @@ public void testRenameDirAndDeleteSnapshot_2() throws Exception {
final Path foo = new Path(sdir2, "foo");
final Path bar = new Path(foo, "bar");
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s3");
-
+
final Path newfoo = new Path(sdir1, "foo");
hdfs.rename(foo, newfoo);
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
-
+
final Path bar2 = new Path(newfoo, "bar2");
DFSTestUtil.createFile(hdfs, bar2, BLOCKSIZE, REPL, SEED);
-
+
hdfs.createSnapshot(sdir1, "s4");
hdfs.delete(newfoo, true);
-
+
final Path bar2_s4 = SnapshotTestHelper.getSnapshotPath(sdir1, "s4",
"foo/bar2");
assertTrue(hdfs.exists(bar2_s4));
final Path bar_s4 = SnapshotTestHelper.getSnapshotPath(sdir1, "s4",
"foo/bar");
assertTrue(hdfs.exists(bar_s4));
-
+
// delete snapshot s4. The diff of s4 should be combined to s3
hdfs.deleteSnapshot(sdir1, "s4");
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
-
+
Path bar_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3", "foo/bar");
assertFalse(hdfs.exists(bar_s3));
bar_s3 = SnapshotTestHelper.getSnapshotPath(sdir2, "s3", "foo/bar");
@@ -1174,16 +1174,16 @@ public void testRenameDirAndDeleteSnapshot_2() throws Exception {
assertFalse(hdfs.exists(bar2_s3));
bar2_s3 = SnapshotTestHelper.getSnapshotPath(sdir2, "s3", "foo/bar2");
assertFalse(hdfs.exists(bar2_s3));
-
+
// delete snapshot s3
hdfs.deleteSnapshot(sdir2, "s3");
final Path bar_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
"foo/bar");
assertTrue(hdfs.exists(bar_s2));
-
+
// check internal details
INodeDirectory sdir2Node = fsdir.getINode(sdir2.toString()).asDirectory();
- Snapshot s2 = sdir2Node.getSnapshot(DFSUtil.string2Bytes("s2"));
+ Snapshot s2 = sdir2Node.getSnapshot("s2".getBytes(UTF_8));
final Path foo_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2", "foo");
INodeReference fooRef = fsdir.getINode(foo_s2.toString()).asReference();
assertTrue(fooRef instanceof INodeReference.WithName);
@@ -1193,10 +1193,10 @@ public void testRenameDirAndDeleteSnapshot_2() throws Exception {
List diffs = fooDir.getDiffs().asList();
assertEquals(1, diffs.size());
assertEquals(s2.getId(), diffs.get(0).getSnapshotId());
-
+
// restart the cluster and check fsimage
restartClusterAndCheckImage(true);
-
+
// delete snapshot s2.
hdfs.deleteSnapshot(sdir2, "s2");
assertFalse(hdfs.exists(bar_s2));
@@ -1205,16 +1205,16 @@ public void testRenameDirAndDeleteSnapshot_2() throws Exception {
QuotaCounts q = fsdir.getRoot().getDirectoryWithQuotaFeature().getSpaceConsumed();
assertEquals(3, q.getNameSpace());
assertEquals(0, q.getStorageSpace());
-
+
hdfs.deleteSnapshot(sdir1, "s1");
restartClusterAndCheckImage(true);
q = fsdir.getRoot().getDirectoryWithQuotaFeature().getSpaceConsumed();
assertEquals(3, q.getNameSpace());
assertEquals(0, q.getStorageSpace());
}
-
+
/**
- * Rename a file and then append the same file.
+ * Rename a file and then append the same file.
*/
@Test
public void testRenameAndAppend() throws Exception {
@@ -1222,18 +1222,18 @@ public void testRenameAndAppend() throws Exception {
final Path sdir2 = new Path("/dir2");
hdfs.mkdirs(sdir1);
hdfs.mkdirs(sdir2);
-
+
final Path foo = new Path(sdir1, "foo");
DFSTestUtil.createFile(hdfs, foo, BLOCKSIZE, REPL, SEED);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, snap1);
final Path foo2 = new Path(sdir2, "foo");
hdfs.rename(foo, foo2);
-
+
INode fooRef = fsdir.getINode4Write(foo2.toString());
assertTrue(fooRef instanceof INodeReference.DstReference);
-
+
FSDataOutputStream out = hdfs.append(foo2);
try {
byte[] content = new byte[1024];
@@ -1249,18 +1249,18 @@ public void testRenameAndAppend() throws Exception {
out.close();
}
}
-
+
fooRef = fsdir.getINode4Write(foo2.toString());
assertTrue(fooRef instanceof INodeReference.DstReference);
INodeFile fooNode = fooRef.asFile();
assertTrue(fooNode.isWithSnapshot());
assertFalse(fooNode.isUnderConstruction());
-
+
restartClusterAndCheckImage(true);
}
-
+
/**
- * Test the undo section of rename. Before the rename, we create the renamed
+ * Test the undo section of rename. Before the rename, we create the renamed
* file/dir before taking the snapshot.
*/
@Test
@@ -1274,24 +1274,24 @@ public void testRenameUndo_1() throws Exception {
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
final Path dir2file = new Path(sdir2, "file");
DFSTestUtil.createFile(hdfs, dir2file, BLOCKSIZE, REPL, SEED);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
-
+
INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
INodeDirectory mockDir2 = spy(dir2);
doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
Mockito.anyInt());
INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
root.replaceChild(dir2, mockDir2, fsdir.getINodeMap());
-
+
final Path newfoo = new Path(sdir2, "foo");
boolean result = hdfs.rename(foo, newfoo);
assertFalse(result);
-
+
// check the current internal details
INodeDirectory dir1Node = fsdir.getINode4Write(sdir1.toString())
.asDirectory();
- Snapshot s1 = dir1Node.getSnapshot(DFSUtil.string2Bytes("s1"));
+ Snapshot s1 = dir1Node.getSnapshot("s1".getBytes(UTF_8));
ReadOnlyList dir1Children = dir1Node
.getChildrenList(Snapshot.CURRENT_STATE_ID);
assertEquals(1, dir1Children.size());
@@ -1299,23 +1299,23 @@ public void testRenameUndo_1() throws Exception {
List dir1Diffs = dir1Node.getDiffs().asList();
assertEquals(1, dir1Diffs.size());
assertEquals(s1.getId(), dir1Diffs.get(0).getSnapshotId());
-
+
// after the undo of rename, both the created and deleted list of sdir1
// should be empty
ChildrenDiff childrenDiff = dir1Diffs.get(0).getChildrenDiff();
assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
assertEquals(0, childrenDiff.getList(ListType.CREATED).size());
-
+
INode fooNode = fsdir.getINode4Write(foo.toString());
assertTrue(fooNode.isDirectory() && fooNode.asDirectory().isWithSnapshot());
List fooDiffs = fooNode.asDirectory().getDiffs().asList();
assertEquals(1, fooDiffs.size());
assertEquals(s1.getId(), fooDiffs.get(0).getSnapshotId());
-
+
final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1", "foo");
INode fooNode_s1 = fsdir.getINode(foo_s1.toString());
assertTrue(fooNode_s1 == fooNode);
-
+
// check sdir2
assertFalse(hdfs.exists(newfoo));
INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
@@ -1328,7 +1328,7 @@ public void testRenameUndo_1() throws Exception {
}
/**
- * Test the undo section of rename. Before the rename, we create the renamed
+ * Test the undo section of rename. Before the rename, we create the renamed
* file/dir after taking the snapshot.
*/
@Test
@@ -1339,29 +1339,29 @@ public void testRenameUndo_2() throws Exception {
hdfs.mkdirs(sdir2);
final Path dir2file = new Path(sdir2, "file");
DFSTestUtil.createFile(hdfs, dir2file, BLOCKSIZE, REPL, SEED);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
-
+
// create foo after taking snapshot
final Path foo = new Path(sdir1, "foo");
final Path bar = new Path(foo, "bar");
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
-
+
INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
INodeDirectory mockDir2 = spy(dir2);
doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
Mockito.anyInt());
INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
root.replaceChild(dir2, mockDir2, fsdir.getINodeMap());
-
+
final Path newfoo = new Path(sdir2, "foo");
boolean result = hdfs.rename(foo, newfoo);
assertFalse(result);
-
+
// check the current internal details
INodeDirectory dir1Node = fsdir.getINode4Write(sdir1.toString())
.asDirectory();
- Snapshot s1 = dir1Node.getSnapshot(DFSUtil.string2Bytes("s1"));
+ Snapshot s1 = dir1Node.getSnapshot("s1".getBytes(UTF_8));
ReadOnlyList dir1Children = dir1Node
.getChildrenList(Snapshot.CURRENT_STATE_ID);
assertEquals(1, dir1Children.size());
@@ -1369,20 +1369,20 @@ public void testRenameUndo_2() throws Exception {
List dir1Diffs = dir1Node.getDiffs().asList();
assertEquals(1, dir1Diffs.size());
assertEquals(s1.getId(), dir1Diffs.get(0).getSnapshotId());
-
- // after the undo of rename, the created list of sdir1 should contain
+
+ // after the undo of rename, the created list of sdir1 should contain
// 1 element
ChildrenDiff childrenDiff = dir1Diffs.get(0).getChildrenDiff();
assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
assertEquals(1, childrenDiff.getList(ListType.CREATED).size());
-
+
INode fooNode = fsdir.getINode4Write(foo.toString());
assertTrue(fooNode instanceof INodeDirectory);
assertTrue(childrenDiff.getList(ListType.CREATED).get(0) == fooNode);
-
+
final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1", "foo");
assertFalse(hdfs.exists(foo_s1));
-
+
// check sdir2
assertFalse(hdfs.exists(newfoo));
INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
@@ -1393,7 +1393,7 @@ public void testRenameUndo_2() throws Exception {
assertEquals(1, dir2Children.size());
assertEquals(dir2file.getName(), dir2Children.get(0).getLocalName());
}
-
+
/**
* Test the undo section of the second-time rename.
*/
@@ -1408,30 +1408,30 @@ public void testRenameUndo_3() throws Exception {
final Path foo = new Path(sdir1, "foo");
final Path bar = new Path(foo, "bar");
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
-
+
INodeDirectory dir3 = fsdir.getINode4Write(sdir3.toString()).asDirectory();
INodeDirectory mockDir3 = spy(dir3);
doReturn(false).when(mockDir3).addChild((INode) anyObject(), anyBoolean(),
Mockito.anyInt());
INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
-
+
final Path foo_dir2 = new Path(sdir2, "foo2");
final Path foo_dir3 = new Path(sdir3, "foo3");
hdfs.rename(foo, foo_dir2);
boolean result = hdfs.rename(foo_dir2, foo_dir3);
assertFalse(result);
-
+
// check the current internal details
INodeDirectory dir1Node = fsdir.getINode4Write(sdir1.toString())
.asDirectory();
- Snapshot s1 = dir1Node.getSnapshot(DFSUtil.string2Bytes("s1"));
+ Snapshot s1 = dir1Node.getSnapshot("s1".getBytes(UTF_8));
INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
.asDirectory();
- Snapshot s2 = dir2Node.getSnapshot(DFSUtil.string2Bytes("s2"));
+ Snapshot s2 = dir2Node.getSnapshot("s2".getBytes(UTF_8));
ReadOnlyList dir2Children = dir2Node
.getChildrenList(Snapshot.CURRENT_STATE_ID);
assertEquals(1, dir2Children.size());
@@ -1443,14 +1443,14 @@ public void testRenameUndo_3() throws Exception {
assertEquals(1, childrenDiff.getList(ListType.CREATED).size());
final Path foo_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2", "foo2");
assertFalse(hdfs.exists(foo_s2));
-
+
INode fooNode = fsdir.getINode4Write(foo_dir2.toString());
assertTrue(childrenDiff.getList(ListType.CREATED).get(0) == fooNode);
assertTrue(fooNode instanceof INodeReference.DstReference);
List fooDiffs = fooNode.asDirectory().getDiffs().asList();
assertEquals(1, fooDiffs.size());
assertEquals(s1.getId(), fooDiffs.get(0).getSnapshotId());
-
+
// create snapshot on sdir2 and rename again
hdfs.createSnapshot(sdir2, "s3");
result = hdfs.rename(foo_dir2, foo_dir3);
@@ -1458,7 +1458,7 @@ public void testRenameUndo_3() throws Exception {
// check internal details again
dir2Node = fsdir.getINode4Write(sdir2.toString()).asDirectory();
- Snapshot s3 = dir2Node.getSnapshot(DFSUtil.string2Bytes("s3"));
+ Snapshot s3 = dir2Node.getSnapshot("s3".getBytes(UTF_8));
fooNode = fsdir.getINode4Write(foo_dir2.toString());
dir2Children = dir2Node.getChildrenList(Snapshot.CURRENT_STATE_ID);
assertEquals(1, dir2Children.size());
@@ -1466,27 +1466,27 @@ public void testRenameUndo_3() throws Exception {
assertEquals(2, dir2Diffs.size());
assertEquals(s2.getId(), dir2Diffs.get(0).getSnapshotId());
assertEquals(s3.getId(), dir2Diffs.get(1).getSnapshotId());
-
+
childrenDiff = dir2Diffs.get(0).getChildrenDiff();
assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
assertEquals(1, childrenDiff.getList(ListType.CREATED).size());
assertTrue(childrenDiff.getList(ListType.CREATED).get(0) == fooNode);
-
+
childrenDiff = dir2Diffs.get(1).getChildrenDiff();
assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
assertEquals(0, childrenDiff.getList(ListType.CREATED).size());
-
+
final Path foo_s3 = SnapshotTestHelper.getSnapshotPath(sdir2, "s3", "foo2");
assertFalse(hdfs.exists(foo_s2));
assertTrue(hdfs.exists(foo_s3));
-
+
assertTrue(fooNode instanceof INodeReference.DstReference);
fooDiffs = fooNode.asDirectory().getDiffs().asList();
assertEquals(2, fooDiffs.size());
assertEquals(s1.getId(), fooDiffs.get(0).getSnapshotId());
assertEquals(s3.getId(), fooDiffs.get(1).getSnapshotId());
}
-
+
/**
* Test undo where dst node being overwritten is a reference node
*/
@@ -1498,35 +1498,35 @@ public void testRenameUndo_4() throws Exception {
hdfs.mkdirs(sdir1);
hdfs.mkdirs(sdir2);
hdfs.mkdirs(sdir3);
-
+
final Path foo = new Path(sdir1, "foo");
final Path bar = new Path(foo, "bar");
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
-
+
final Path foo2 = new Path(sdir2, "foo2");
hdfs.mkdirs(foo2);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
-
+
// rename foo2 to foo3, so that foo3 will be a reference node
final Path foo3 = new Path(sdir3, "foo3");
hdfs.rename(foo2, foo3);
-
+
INode foo3Node = fsdir.getINode4Write(foo3.toString());
assertTrue(foo3Node.isReference());
-
+
INodeDirectory dir3 = fsdir.getINode4Write(sdir3.toString()).asDirectory();
INodeDirectory mockDir3 = spy(dir3);
// fail the rename but succeed in undo
doReturn(false).when(mockDir3).addChild((INode) Mockito.isNull(),
anyBoolean(), Mockito.anyInt());
- Mockito.when(mockDir3.addChild((INode) Mockito.isNotNull(), anyBoolean(),
+ Mockito.when(mockDir3.addChild((INode) Mockito.isNotNull(), anyBoolean(),
Mockito.anyInt())).thenReturn(false).thenCallRealMethod();
INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
foo3Node.setParent(mockDir3);
-
+
try {
hdfs.rename(foo, foo3, Rename.OVERWRITE);
fail("the rename from " + foo + " to " + foo3 + " should fail");
@@ -1534,7 +1534,7 @@ public void testRenameUndo_4() throws Exception {
GenericTestUtils.assertExceptionContains("rename from " + foo + " to "
+ foo3 + " failed.", e);
}
-
+
// make sure the undo is correct
final INode foo3Node_undo = fsdir.getINode4Write(foo3.toString());
assertSame(foo3Node, foo3Node_undo);
@@ -1543,7 +1543,7 @@ public void testRenameUndo_4() throws Exception {
assertEquals(2, foo3_wc.getReferenceCount());
assertSame(foo3Node, foo3_wc.getParentReference());
}
-
+
/**
* Test rename while the rename operation will exceed the quota in the dst
* tree.
@@ -1556,18 +1556,18 @@ public void testRenameUndo_5() throws Exception {
final Path subdir2 = new Path(dir2, "subdir2");
hdfs.mkdirs(dir1);
hdfs.mkdirs(subdir2);
-
+
final Path foo = new Path(dir1, "foo");
final Path bar = new Path(foo, "bar");
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
-
+
SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
SnapshotTestHelper.createSnapshot(hdfs, dir2, "s2");
-
+
// set ns quota of dir2 to 4, so the current remaining is 2 (already has
// dir2, and subdir2)
hdfs.setQuota(dir2, 4, Long.MAX_VALUE - 1);
-
+
final Path foo2 = new Path(subdir2, foo.getName());
FSDirectory fsdir2 = Mockito.spy(fsdir);
Mockito.doThrow(new NSQuotaExceededException("fake exception")).when(fsdir2)
@@ -1576,13 +1576,13 @@ public void testRenameUndo_5() throws Exception {
(FsPermission) Mockito.anyObject(),
Mockito.anyBoolean());
Whitebox.setInternalState(fsn, "dir", fsdir2);
- // rename /test/dir1/foo to /test/dir2/subdir2/foo.
+ // rename /test/dir1/foo to /test/dir2/subdir2/foo.
// FSDirectory#verifyQuota4Rename will pass since the remaining quota is 2.
// However, the rename operation will fail since we let addLastINode throw
// NSQuotaExceededException
boolean rename = hdfs.rename(foo, foo2);
assertFalse(rename);
-
+
// check the undo
assertTrue(hdfs.exists(foo));
assertTrue(hdfs.exists(bar));
@@ -1602,7 +1602,7 @@ public void testRenameUndo_5() throws Exception {
DirectoryDiff diff = diffList.get(0);
assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
-
+
// check dir2
INodeDirectory dir2Node = fsdir2.getINode4Write(dir2.toString()).asDirectory();
assertTrue(dir2Node.isSnapshottable());
@@ -1621,7 +1621,7 @@ public void testRenameUndo_5() throws Exception {
assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
}
-
+
/**
* Test the rename undo when removing dst node fails
*/
@@ -1634,13 +1634,13 @@ public void testRenameUndo_6() throws Exception {
final Path subsub_dir2 = new Path(sub_dir2, "subdir");
hdfs.mkdirs(dir1);
hdfs.mkdirs(subsub_dir2);
-
+
final Path foo = new Path(dir1, "foo");
hdfs.mkdirs(foo);
-
+
SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
SnapshotTestHelper.createSnapshot(hdfs, dir2, "s2");
-
+
// set ns quota of dir2 to 4, so the current remaining is 1 (already has
// dir2, sub_dir2, and subsub_dir2)
hdfs.setQuota(dir2, 4, Long.MAX_VALUE - 1);
@@ -1648,8 +1648,8 @@ public void testRenameUndo_6() throws Exception {
Mockito.doThrow(new RuntimeException("fake exception")).when(fsdir2)
.removeLastINode((INodesInPath) Mockito.anyObject());
Whitebox.setInternalState(fsn, "dir", fsdir2);
- // rename /test/dir1/foo to /test/dir2/sub_dir2/subsub_dir2.
- // FSDirectory#verifyQuota4Rename will pass since foo only be counted
+ // rename /test/dir1/foo to /test/dir2/sub_dir2/subsub_dir2.
+ // FSDirectory#verifyQuota4Rename will pass since foo only be counted
// as 1 in NS quota. However, the rename operation will fail when removing
// subsub_dir2.
try {
@@ -1659,7 +1659,7 @@ public void testRenameUndo_6() throws Exception {
String msg = "fake exception";
GenericTestUtils.assertExceptionContains(msg, e);
}
-
+
// check the undo
assertTrue(hdfs.exists(foo));
INodeDirectory dir1Node = fsdir2.getINode4Write(dir1.toString())
@@ -1676,7 +1676,7 @@ public void testRenameUndo_6() throws Exception {
DirectoryDiff diff = diffList.get(0);
assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
-
+
// check dir2
INodeDirectory dir2Node = fsdir2.getINode4Write(dir2.toString()).asDirectory();
assertTrue(dir2Node.isSnapshottable());
@@ -1692,14 +1692,14 @@ public void testRenameUndo_6() throws Exception {
INode subsubdir2Node = fsdir2.getINode4Write(subsub_dir2.toString());
assertTrue(subsubdir2Node.getClass() == INodeDirectory.class);
assertSame(subdir2Node, subsubdir2Node.getParent());
-
+
diffList = ( dir2Node).getDiffs().asList();
assertEquals(1, diffList.size());
diff = diffList.get(0);
assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
}
-
+
/**
* Test rename to an invalid name (xxx/.snapshot)
*/
@@ -1709,10 +1709,10 @@ public void testRenameUndo_7() throws Exception {
final Path foo = new Path(root, "foo");
final Path bar = new Path(foo, "bar");
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
-
+
// create a snapshot on root
SnapshotTestHelper.createSnapshot(hdfs, root, snap1);
-
+
// rename bar to /foo/.snapshot which is invalid
final Path invalid = new Path(foo, HdfsConstants.DOT_SNAPSHOT_DIR);
try {
@@ -1722,7 +1722,7 @@ public void testRenameUndo_7() throws Exception {
GenericTestUtils.assertExceptionContains("\"" +
HdfsConstants.DOT_SNAPSHOT_DIR + "\" is a reserved name", e);
}
-
+
// check
INodeDirectory rootNode = fsdir.getINode4Write(root.toString())
.asDirectory();
@@ -1734,12 +1734,12 @@ public void testRenameUndo_7() throws Exception {
assertEquals(1, diffList.size());
DirectoryDiff diff = diffList.get(0);
// this diff is generated while renaming
- Snapshot s1 = rootNode.getSnapshot(DFSUtil.string2Bytes(snap1));
+ Snapshot s1 = rootNode.getSnapshot(snap1.getBytes(UTF_8));
assertEquals(s1.getId(), diff.getSnapshotId());
// after undo, the diff should be empty
assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
-
+
// bar was converted to filewithsnapshot while renaming
INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
assertSame(barNode, children.get(0));
@@ -1748,10 +1748,10 @@ public void testRenameUndo_7() throws Exception {
assertEquals(1, barDiffList.size());
FileDiff barDiff = barDiffList.get(0);
assertEquals(s1.getId(), barDiff.getSnapshotId());
-
+
// restart cluster multiple times to make sure the fsimage and edits log are
- // correct. Note that when loading fsimage, foo and bar will be converted
- // back to normal INodeDirectory and INodeFile since they do not store any
+ // correct. Note that when loading fsimage, foo and bar will be converted
+ // back to normal INodeDirectory and INodeFile since they do not store any
// snapshot data
hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
hdfs.saveNamespace();
@@ -1762,7 +1762,7 @@ public void testRenameUndo_7() throws Exception {
cluster.waitActive();
restartClusterAndCheckImage(true);
}
-
+
/**
* Test the rename undo when quota of dst tree is exceeded after rename.
*/
@@ -1775,24 +1775,24 @@ public void testRenameExceedQuota() throws Exception {
final Path subfile_dir2 = new Path(sub_dir2, "subfile");
hdfs.mkdirs(dir1);
DFSTestUtil.createFile(hdfs, subfile_dir2, BLOCKSIZE, REPL, SEED);
-
+
final Path foo = new Path(dir1, "foo");
DFSTestUtil.createFile(hdfs, foo, BLOCKSIZE, REPL, SEED);
-
+
SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
SnapshotTestHelper.createSnapshot(hdfs, dir2, "s2");
-
+
// set ns quota of dir2 to 4, so the current remaining is 1 (already has
// dir2, sub_dir2, subfile_dir2, and s2)
hdfs.setQuota(dir2, 5, Long.MAX_VALUE - 1);
-
- // rename /test/dir1/foo to /test/dir2/sub_dir2/subfile_dir2.
- // FSDirectory#verifyQuota4Rename will pass since foo only be counted
- // as 1 in NS quota. The rename operation will succeed while the real quota
+
+ // rename /test/dir1/foo to /test/dir2/sub_dir2/subfile_dir2.
+ // FSDirectory#verifyQuota4Rename will pass since foo only be counted
+ // as 1 in NS quota. The rename operation will succeed while the real quota
// of dir2 will become 7 (dir2, s2 in dir2, sub_dir2, s2 in sub_dir2,
// subfile_dir2 in deleted list, new subfile, s1 in new subfile).
hdfs.rename(foo, subfile_dir2, Rename.OVERWRITE);
-
+
// check dir2
INode dir2Node = fsdir.getINode4Write(dir2.toString());
assertTrue(dir2Node.asDirectory().isSnapshottable());
@@ -1801,7 +1801,7 @@ public void testRenameExceedQuota() throws Exception {
assertEquals(4, counts.getNameSpace());
assertEquals(BLOCKSIZE * REPL * 2, counts.getStorageSpace());
}
-
+
@Test
public void testRename2PreDescendant() throws Exception {
final Path sdir1 = new Path("/dir1");
@@ -1810,25 +1810,25 @@ public void testRename2PreDescendant() throws Exception {
final Path bar = new Path(foo, "bar");
hdfs.mkdirs(bar);
hdfs.mkdirs(sdir2);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, snap1);
-
+
// /dir1/foo/bar -> /dir2/bar
final Path bar2 = new Path(sdir2, "bar");
hdfs.rename(bar, bar2);
-
+
// /dir1/foo -> /dir2/bar/foo
final Path foo2 = new Path(bar2, "foo");
hdfs.rename(foo, foo2);
-
+
restartClusterAndCheckImage(true);
-
+
// delete snap1
hdfs.deleteSnapshot(sdir1, snap1);
-
+
restartClusterAndCheckImage(true);
}
-
+
/**
* move a directory to its prior descendant
*/
@@ -1845,43 +1845,43 @@ public void testRename2PreDescendant_2() throws Exception {
hdfs.mkdirs(sdir2);
DFSTestUtil.createFile(hdfs, file1InBar, BLOCKSIZE, REPL, SEED);
DFSTestUtil.createFile(hdfs, file2InBar, BLOCKSIZE, REPL, SEED);
-
+
hdfs.setQuota(sdir1, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
hdfs.setQuota(sdir2, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
hdfs.setQuota(foo, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
hdfs.setQuota(bar, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
-
+
// create snapshot on root
SnapshotTestHelper.createSnapshot(hdfs, root, snap1);
// delete file1InBar
hdfs.delete(file1InBar, true);
-
+
// create another snapshot on root
SnapshotTestHelper.createSnapshot(hdfs, root, snap2);
// delete file2InBar
hdfs.delete(file2InBar, true);
-
+
// /dir1/foo/bar -> /dir2/bar
final Path bar2 = new Path(sdir2, "bar2");
hdfs.rename(bar, bar2);
-
+
// /dir1/foo -> /dir2/bar/foo
final Path foo2 = new Path(bar2, "foo2");
hdfs.rename(foo, foo2);
-
+
restartClusterAndCheckImage(true);
-
+
// delete snapshot snap2
hdfs.deleteSnapshot(root, snap2);
-
- // after deleteing snap2, the WithName node "bar", which originally was
- // stored in the deleted list of "foo" for snap2, is moved to its deleted
- // list for snap1. In that case, it will not be counted when calculating
- // quota for "foo". However, we do not update this quota usage change while
+
+ // after deleteing snap2, the WithName node "bar", which originally was
+ // stored in the deleted list of "foo" for snap2, is moved to its deleted
+ // list for snap1. In that case, it will not be counted when calculating
+ // quota for "foo". However, we do not update this quota usage change while
// deleting snap2.
restartClusterAndCheckImage(false);
}
-
+
/**
* move a directory to its prior descedant
*/
@@ -1896,40 +1896,40 @@ public void testRename2PreDescendant_3() throws Exception {
hdfs.mkdirs(bar);
hdfs.mkdirs(sdir2);
DFSTestUtil.createFile(hdfs, fileInBar, BLOCKSIZE, REPL, SEED);
-
+
hdfs.setQuota(sdir1, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
hdfs.setQuota(sdir2, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
hdfs.setQuota(foo, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
hdfs.setQuota(bar, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
-
+
// create snapshot on root
SnapshotTestHelper.createSnapshot(hdfs, root, snap1);
// delete fileInBar
hdfs.delete(fileInBar, true);
// create another snapshot on root
SnapshotTestHelper.createSnapshot(hdfs, root, snap2);
-
+
// /dir1/foo/bar -> /dir2/bar
final Path bar2 = new Path(sdir2, "bar2");
hdfs.rename(bar, bar2);
-
+
// /dir1/foo -> /dir2/bar/foo
final Path foo2 = new Path(bar2, "foo2");
hdfs.rename(foo, foo2);
-
+
restartClusterAndCheckImage(true);
-
+
// delete snapshot snap1
hdfs.deleteSnapshot(root, snap1);
-
+
restartClusterAndCheckImage(true);
}
-
+
/**
* After the following operations:
* Rename a dir -> create a snapshot s on dst tree -> delete the renamed dir
* -> delete snapshot s on dst tree
- *
+ *
* Make sure we destroy everything created after the rename under the renamed
* dir.
*/
@@ -1941,27 +1941,27 @@ public void testRenameDirAndDeleteSnapshot_3() throws Exception {
final Path bar = new Path(foo, "bar");
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
hdfs.mkdirs(sdir2);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
-
+
final Path foo2 = new Path(sdir2, "foo");
hdfs.rename(foo, foo2);
-
+
// create two new files under foo2
final Path bar2 = new Path(foo2, "bar2");
DFSTestUtil.createFile(hdfs, bar2, BLOCKSIZE, REPL, SEED);
final Path bar3 = new Path(foo2, "bar3");
DFSTestUtil.createFile(hdfs, bar3, BLOCKSIZE, REPL, SEED);
-
+
// create a new snapshot on sdir2
hdfs.createSnapshot(sdir2, "s3");
-
+
// delete foo2
hdfs.delete(foo2, true);
// delete s3
hdfs.deleteSnapshot(sdir2, "s3");
-
+
// check
final INodeDirectory dir1Node = fsdir.getINode4Write(sdir1.toString())
.asDirectory();
@@ -1971,12 +1971,12 @@ public void testRenameDirAndDeleteSnapshot_3() throws Exception {
.asDirectory();
QuotaCounts q2 = dir2Node.getDirectoryWithQuotaFeature().getSpaceConsumed();
assertEquals(1, q2.getNameSpace());
-
+
final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
foo.getName());
INode fooRef = fsdir.getINode(foo_s1.toString());
assertTrue(fooRef instanceof INodeReference.WithName);
- INodeReference.WithCount wc =
+ INodeReference.WithCount wc =
(WithCount) fooRef.asReference().getReferredINode();
assertEquals(1, wc.getReferenceCount());
INodeDirectory fooNode = wc.getReferredINode().asDirectory();
@@ -1986,20 +1986,20 @@ public void testRenameDirAndDeleteSnapshot_3() throws Exception {
assertEquals(bar.getName(), children.get(0).getLocalName());
List diffList = fooNode.getDiffs().asList();
assertEquals(1, diffList.size());
- Snapshot s1 = dir1Node.getSnapshot(DFSUtil.string2Bytes("s1"));
+ Snapshot s1 = dir1Node.getSnapshot("s1".getBytes(UTF_8));
assertEquals(s1.getId(), diffList.get(0).getSnapshotId());
ChildrenDiff diff = diffList.get(0).getChildrenDiff();
assertEquals(0, diff.getList(ListType.CREATED).size());
assertEquals(0, diff.getList(ListType.DELETED).size());
-
+
restartClusterAndCheckImage(true);
}
-
+
/**
* After the following operations:
* Rename a dir -> create a snapshot s on dst tree -> rename the renamed dir
* again -> delete snapshot s on dst tree
- *
+ *
* Make sure we only delete the snapshot s under the renamed dir.
*/
@Test
@@ -2010,27 +2010,27 @@ public void testRenameDirAndDeleteSnapshot_4() throws Exception {
final Path bar = new Path(foo, "bar");
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
hdfs.mkdirs(sdir2);
-
+
SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
-
+
final Path foo2 = new Path(sdir2, "foo");
hdfs.rename(foo, foo2);
-
+
// create two new files under foo2
final Path bar2 = new Path(foo2, "bar2");
DFSTestUtil.createFile(hdfs, bar2, BLOCKSIZE, REPL, SEED);
final Path bar3 = new Path(foo2, "bar3");
DFSTestUtil.createFile(hdfs, bar3, BLOCKSIZE, REPL, SEED);
-
+
// create a new snapshot on sdir2
hdfs.createSnapshot(sdir2, "s3");
-
+
// rename foo2 again
hdfs.rename(foo2, foo);
// delete snapshot s3
hdfs.deleteSnapshot(sdir2, "s3");
-
+
// check
final INodeDirectory dir1Node = fsdir.getINode4Write(sdir1.toString())
.asDirectory();
@@ -2041,12 +2041,12 @@ public void testRenameDirAndDeleteSnapshot_4() throws Exception {
.asDirectory();
QuotaCounts q2 = dir2Node.getDirectoryWithQuotaFeature().getSpaceConsumed();
assertEquals(1, q2.getNameSpace());
-
+
final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
foo.getName());
final INode fooRef = fsdir.getINode(foo_s1.toString());
assertTrue(fooRef instanceof INodeReference.WithName);
- INodeReference.WithCount wc =
+ INodeReference.WithCount wc =
(WithCount) fooRef.asReference().getReferredINode();
assertEquals(2, wc.getReferenceCount());
INodeDirectory fooNode = wc.getReferredINode().asDirectory();
@@ -2058,27 +2058,27 @@ public void testRenameDirAndDeleteSnapshot_4() throws Exception {
assertEquals(bar3.getName(), children.get(2).getLocalName());
List diffList = fooNode.getDiffs().asList();
assertEquals(1, diffList.size());
- Snapshot s1 = dir1Node.getSnapshot(DFSUtil.string2Bytes("s1"));
+ Snapshot s1 = dir1Node.getSnapshot("s1".getBytes(UTF_8));
assertEquals(s1.getId(), diffList.get(0).getSnapshotId());
ChildrenDiff diff = diffList.get(0).getChildrenDiff();
// bar2 and bar3 in the created list
assertEquals(2, diff.getList(ListType.CREATED).size());
assertEquals(0, diff.getList(ListType.DELETED).size());
-
+
final INode fooRef2 = fsdir.getINode4Write(foo.toString());
assertTrue(fooRef2 instanceof INodeReference.DstReference);
- INodeReference.WithCount wc2 =
+ INodeReference.WithCount wc2 =
(WithCount) fooRef2.asReference().getReferredINode();
assertSame(wc, wc2);
assertSame(fooRef2, wc.getParentReference());
-
+
restartClusterAndCheckImage(true);
}
-
+
/**
- * This test demonstrates that
+ * This test demonstrates that
* {@link INodeDirectory#removeChild}
- * and
+ * and
* {@link INodeDirectory#addChild}
* should use {@link INode#isInLatestSnapshot} to check if the
* added/removed child should be recorded in snapshots.
@@ -2091,7 +2091,7 @@ public void testRenameDirAndDeleteSnapshot_5() throws Exception {
hdfs.mkdirs(dir1);
hdfs.mkdirs(dir2);
hdfs.mkdirs(dir3);
-
+
final Path foo = new Path(dir1, "foo");
hdfs.mkdirs(foo);
SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
@@ -2100,7 +2100,7 @@ public void testRenameDirAndDeleteSnapshot_5() throws Exception {
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
// delete snapshot s1. now foo is not in any snapshot
hdfs.deleteSnapshot(dir1, "s1");
-
+
SnapshotTestHelper.createSnapshot(hdfs, dir2, "s2");
// rename /dir1/foo to /dir2/foo
final Path foo2 = new Path(dir2, foo.getName());
@@ -2109,19 +2109,19 @@ public void testRenameDirAndDeleteSnapshot_5() throws Exception {
final Path bar2 = new Path(dir2, "foo/bar");
final Path bar3 = new Path(dir3, "bar");
hdfs.rename(bar2, bar3);
-
- // delete /dir2/foo. Since it is not in any snapshot, we will call its
+
+ // delete /dir2/foo. Since it is not in any snapshot, we will call its
// destroy function. If we do not use isInLatestSnapshot in removeChild and
- // addChild methods in INodeDirectory (with snapshot), the file bar will be
+ // addChild methods in INodeDirectory (with snapshot), the file bar will be
// stored in the deleted list of foo, and will be destroyed.
hdfs.delete(foo2, true);
-
+
// check if /dir3/bar still exists
assertTrue(hdfs.exists(bar3));
INodeFile barNode = (INodeFile) fsdir.getINode4Write(bar3.toString());
assertSame(fsdir.getINode4Write(dir3.toString()), barNode.getParent());
}
-
+
/**
* Rename and deletion snapshot under the same the snapshottable directory.
*/
@@ -2132,31 +2132,31 @@ public void testRenameDirAndDeleteSnapshot_6() throws Exception {
final Path dir2 = new Path(test, "dir2");
hdfs.mkdirs(dir1);
hdfs.mkdirs(dir2);
-
+
final Path foo = new Path(dir2, "foo");
final Path bar = new Path(foo, "bar");
final Path file = new Path(bar, "file");
DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPL, SEED);
-
+
// take a snapshot on /test
SnapshotTestHelper.createSnapshot(hdfs, test, "s0");
-
- // delete /test/dir2/foo/bar/file after snapshot s0, so that there is a
+
+ // delete /test/dir2/foo/bar/file after snapshot s0, so that there is a
// snapshot copy recorded in bar
hdfs.delete(file, true);
-
+
// rename foo from dir2 to dir1
final Path newfoo = new Path(dir1, foo.getName());
hdfs.rename(foo, newfoo);
-
+
final Path foo_s0 = SnapshotTestHelper.getSnapshotPath(test, "s0",
"dir2/foo");
assertTrue("the snapshot path " + foo_s0 + " should exist",
hdfs.exists(foo_s0));
-
- // delete snapshot s0. The deletion will first go down through dir1, and
+
+ // delete snapshot s0. The deletion will first go down through dir1, and
// find foo in the created list of dir1. Then it will use null as the prior
- // snapshot and continue the snapshot deletion process in the subtree of
+ // snapshot and continue the snapshot deletion process in the subtree of
// foo. We need to make sure the snapshot s0 can be deleted cleanly in the
// foo subtree.
hdfs.deleteSnapshot(test, "s0");
@@ -2168,7 +2168,7 @@ public void testRenameDirAndDeleteSnapshot_6() throws Exception {
assertTrue("the diff list of " + dir2
+ " should be empty after deleting s0", dir2Node.getDiffs().asList()
.isEmpty());
-
+
assertTrue(hdfs.exists(newfoo));
INode fooRefNode = fsdir.getINode4Write(newfoo.toString());
assertTrue(fooRefNode instanceof INodeReference.DstReference);
@@ -2179,14 +2179,14 @@ public void testRenameDirAndDeleteSnapshot_6() throws Exception {
assertTrue(fooNode.getDiffs().asList().isEmpty());
INodeDirectory barNode = fooNode.getChildrenList(Snapshot.CURRENT_STATE_ID)
.get(0).asDirectory();
- // bar should also be INodeDirectory (With Snapshot), and both of its diff
- // list and children list are empty
+ // bar should also be INodeDirectory (With Snapshot), and both of its diff
+ // list and children list are empty
assertTrue(barNode.getDiffs().asList().isEmpty());
assertTrue(barNode.getChildrenList(Snapshot.CURRENT_STATE_ID).isEmpty());
-
+
restartClusterAndCheckImage(true);
}
-
+
/**
* Unit test for HDFS-4842.
*/
@@ -2198,29 +2198,29 @@ public void testRenameDirAndDeleteSnapshot_7() throws Exception {
final Path dir2 = new Path(test, "dir2");
hdfs.mkdirs(dir1);
hdfs.mkdirs(dir2);
-
+
final Path foo = new Path(dir2, "foo");
final Path bar = new Path(foo, "bar");
final Path file = new Path(bar, "file");
DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPL, SEED);
-
+
// take a snapshot s0 and s1 on /test
SnapshotTestHelper.createSnapshot(hdfs, test, "s0");
SnapshotTestHelper.createSnapshot(hdfs, test, "s1");
// delete file so we have a snapshot copy for s1 in bar
hdfs.delete(file, true);
-
+
// create another snapshot on dir2
SnapshotTestHelper.createSnapshot(hdfs, dir2, "s2");
-
+
// rename foo from dir2 to dir1
final Path newfoo = new Path(dir1, foo.getName());
hdfs.rename(foo, newfoo);
-
+
// delete snapshot s1
hdfs.deleteSnapshot(test, "s1");
-
- // make sure the snapshot copy of file in s1 is merged to s0. For
+
+ // make sure the snapshot copy of file in s1 is merged to s0. For
// HDFS-4842, we need to make sure that we do not wrongly use s2 as the
// prior snapshot of s1.
final Path file_s2 = SnapshotTestHelper.getSnapshotPath(dir2, "s2",
@@ -2229,7 +2229,7 @@ public void testRenameDirAndDeleteSnapshot_7() throws Exception {
final Path file_s0 = SnapshotTestHelper.getSnapshotPath(test, "s0",
"dir2/foo/bar/file");
assertTrue(hdfs.exists(file_s0));
-
+
// check dir1: foo should be in the created list of s0
INodeDirectory dir1Node = fsdir.getINode4Write(dir1.toString())
.asDirectory();
@@ -2244,7 +2244,7 @@ public void testRenameDirAndDeleteSnapshot_7() throws Exception {
INode cNode = cList.get(0);
INode fooNode = fsdir.getINode4Write(newfoo.toString());
assertSame(cNode, fooNode);
-
+
// check foo and its subtree
final Path newbar = new Path(newfoo, bar.getName());
INodeDirectory barNode = fsdir.getINode4Write(newbar.toString())
@@ -2256,12 +2256,12 @@ public void testRenameDirAndDeleteSnapshot_7() throws Exception {
DirectoryDiff diff = barDiffList.get(0);
INodeDirectory testNode = fsdir.getINode4Write(test.toString())
.asDirectory();
- Snapshot s0 = testNode.getSnapshot(DFSUtil.string2Bytes("s0"));
+ Snapshot s0 = testNode.getSnapshot("s0".getBytes(UTF_8));
assertEquals(s0.getId(), diff.getSnapshotId());
// and file should be stored in the deleted list of this snapshot diff
assertEquals("file", diff.getChildrenDiff().getList(ListType.DELETED)
.get(0).getLocalName());
-
+
// check dir2: a WithName instance for foo should be in the deleted list
// of the snapshot diff for s2
INodeDirectory dir2Node = fsdir.getINode4Write(dir2.toString())
@@ -2271,19 +2271,19 @@ public void testRenameDirAndDeleteSnapshot_7() throws Exception {
assertEquals(1, dir2DiffList.size());
dList = dir2DiffList.get(0).getChildrenDiff().getList(ListType.DELETED);
assertEquals(1, dList.size());
- final Path foo_s2 = SnapshotTestHelper.getSnapshotPath(dir2, "s2",
+ final Path foo_s2 = SnapshotTestHelper.getSnapshotPath(dir2, "s2",
foo.getName());
- INodeReference.WithName fooNode_s2 =
+ INodeReference.WithName fooNode_s2 =
(INodeReference.WithName) fsdir.getINode(foo_s2.toString());
assertSame(dList.get(0), fooNode_s2);
assertSame(fooNode.asReference().getReferredINode(),
fooNode_s2.getReferredINode());
-
+
restartClusterAndCheckImage(true);
}
-
+
/**
- * Make sure we clean the whole subtree under a DstReference node after
+ * Make sure we clean the whole subtree under a DstReference node after
* deleting a snapshot.
* see HDFS-5476.
*/
@@ -2294,7 +2294,7 @@ public void testCleanDstReference() throws Exception {
final Path bar = new Path(foo, "bar");
hdfs.mkdirs(bar);
SnapshotTestHelper.createSnapshot(hdfs, test, "s0");
-
+
// create file after s0 so that the file should not be included in s0
final Path fileInBar = new Path(bar, "file");
DFSTestUtil.createFile(hdfs, fileInBar, BLOCKSIZE, REPL, SEED);
@@ -2306,16 +2306,16 @@ public void testCleanDstReference() throws Exception {
// delete bar and foo2
hdfs.delete(new Path(foo2, "bar"), true);
hdfs.delete(foo2, true);
-
+
final Path sfileInBar = SnapshotTestHelper.getSnapshotPath(test, "s1",
"foo2/bar/file");
assertTrue(hdfs.exists(sfileInBar));
-
+
hdfs.deleteSnapshot(test, "s1");
assertFalse(hdfs.exists(sfileInBar));
-
+
restartClusterAndCheckImage(true);
- // make sure the file under bar is deleted
+ // make sure the file under bar is deleted
final Path barInS0 = SnapshotTestHelper.getSnapshotPath(test, "s0",
"foo/bar");
INodeDirectory barNode = fsdir.getINode(barInS0.toString()).asDirectory();
@@ -2347,9 +2347,9 @@ public void testRenameUCFileInSnapshot() throws Exception {
// save namespace and restart
restartClusterAndCheckImage(true);
}
-
+
/**
- * Similar with testRenameUCFileInSnapshot, but do renaming first and then
+ * Similar with testRenameUCFileInSnapshot, but do renaming first and then
* append file without closing it. Unit test for HDFS-5425.
*/
@Test
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java
index c5ac26ed95a32..99af85a74c9e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java
@@ -22,13 +22,13 @@
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertTrue;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -49,13 +49,13 @@ public class TestSetQuotaWithSnapshot {
protected static final long seed = 0;
protected static final short REPLICATION = 3;
protected static final long BLOCKSIZE = 1024;
-
+
protected Configuration conf;
protected MiniDFSCluster cluster;
protected FSNamesystem fsn;
protected FSDirectory fsdir;
protected DistributedFileSystem hdfs;
-
+
@Rule
public ExpectedException exception = ExpectedException.none();
@@ -79,14 +79,14 @@ public void tearDown() throws Exception {
cluster = null;
}
}
-
+
@Test (timeout=60000)
public void testSetQuota() throws Exception {
final Path dir = new Path("/TestSnapshot");
hdfs.mkdirs(dir);
// allow snapshot on dir and create snapshot s1
SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
-
+
Path sub = new Path(dir, "sub");
hdfs.mkdirs(sub);
Path fileInSub = new Path(sub, "file");
@@ -95,13 +95,13 @@ public void testSetQuota() throws Exception {
fsdir.getINode(sub.toString()), sub);
// subNode should be a INodeDirectory, but not an INodeDirectoryWithSnapshot
assertFalse(subNode.isWithSnapshot());
-
+
hdfs.setQuota(sub, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
subNode = INodeDirectory.valueOf(fsdir.getINode(sub.toString()), sub);
assertTrue(subNode.isQuotaSet());
assertFalse(subNode.isWithSnapshot());
}
-
+
/**
* Test clear quota of a snapshottable dir or a dir with snapshot.
*/
@@ -109,28 +109,28 @@ public void testSetQuota() throws Exception {
public void testClearQuota() throws Exception {
final Path dir = new Path("/TestSnapshot");
hdfs.mkdirs(dir);
-
+
hdfs.allowSnapshot(dir);
hdfs.setQuota(dir, HdfsConstants.QUOTA_DONT_SET,
HdfsConstants.QUOTA_DONT_SET);
INodeDirectory dirNode = fsdir.getINode4Write(dir.toString()).asDirectory();
assertTrue(dirNode.isSnapshottable());
assertEquals(0, dirNode.getDiffs().asList().size());
-
+
hdfs.setQuota(dir, HdfsConstants.QUOTA_DONT_SET - 1,
HdfsConstants.QUOTA_DONT_SET - 1);
dirNode = fsdir.getINode4Write(dir.toString()).asDirectory();
assertTrue(dirNode.isSnapshottable());
assertEquals(0, dirNode.getDiffs().asList().size());
-
+
hdfs.setQuota(dir, HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_RESET);
dirNode = fsdir.getINode4Write(dir.toString()).asDirectory();
assertTrue(dirNode.isSnapshottable());
assertEquals(0, dirNode.getDiffs().asList().size());
-
+
// allow snapshot on dir and create snapshot s1
SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
-
+
// clear quota of dir
hdfs.setQuota(dir, HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_RESET);
// dir should still be a snapshottable directory
@@ -140,7 +140,7 @@ public void testClearQuota() throws Exception {
SnapshottableDirectoryStatus[] status = hdfs.getSnapshottableDirListing();
assertEquals(1, status.length);
assertEquals(dir, status[0].getFullPath());
-
+
final Path subDir = new Path(dir, "sub");
hdfs.mkdirs(subDir);
hdfs.createSnapshot(dir, "s2");
@@ -151,7 +151,7 @@ public void testClearQuota() throws Exception {
assertTrue(subNode.asDirectory().isWithSnapshot());
List diffList = subNode.asDirectory().getDiffs().asList();
assertEquals(1, diffList.size());
- Snapshot s2 = dirNode.getSnapshot(DFSUtil.string2Bytes("s2"));
+ Snapshot s2 = dirNode.getSnapshot("s2".getBytes(UTF_8));
assertEquals(s2.getId(), diffList.get(0).getSnapshotId());
List createdList = diffList.get(0).getChildrenDiff().getList(ListType.CREATED);
assertEquals(1, createdList.size());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
index ca53788e98d05..aef7ba7b128d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
@@ -24,6 +24,7 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.PrintStream;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.security.PrivilegedAction;
import org.apache.hadoop.conf.Configuration;
@@ -32,7 +33,6 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
@@ -70,18 +70,18 @@ public class TestSnapshotDeletion {
protected static final short REPLICATION = 3;
protected static final short REPLICATION_1 = 2;
protected static final long BLOCKSIZE = 1024;
-
+
private final Path dir = new Path("/TestSnapshot");
private final Path sub = new Path(dir, "sub1");
private final Path subsub = new Path(sub, "subsub1");
-
+
protected Configuration conf;
protected MiniDFSCluster cluster;
protected FSNamesystem fsn;
protected FSDirectory fsdir;
protected BlockManager blockmanager;
protected DistributedFileSystem hdfs;
-
+
@Rule
public ExpectedException exception = ExpectedException.none();
@@ -105,7 +105,7 @@ public void tearDown() throws Exception {
cluster = null;
}
}
-
+
/**
* Deleting snapshottable directory with snapshots must fail.
*/
@@ -173,7 +173,7 @@ public void testDeleteDirectoryWithSnapshot2() throws Exception {
Path file1 = new Path(sub, "file1");
DFSTestUtil.createFile(hdfs, file0, BLOCKSIZE, REPLICATION, seed);
DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, REPLICATION, seed);
-
+
Path subfile1 = new Path(subsub, "file0");
Path subfile2 = new Path(subsub, "file1");
DFSTestUtil.createFile(hdfs, subfile1, BLOCKSIZE, REPLICATION, seed);
@@ -190,7 +190,7 @@ public void testDeleteDirectoryWithSnapshot2() throws Exception {
exception.expectMessage(error);
hdfs.delete(dir, true);
}
-
+
private static INodeDirectory getDir(final FSDirectory fsdir, final Path dir)
throws IOException {
final String dirStr = dir.toString();
@@ -212,14 +212,14 @@ private void checkQuotaUsageComputation(final Path dirPath,
assertEquals(dirNode.dumpTreeRecursively().toString(), expectedDs,
counts.getStorageSpace());
}
-
+
/**
* Test deleting a directory which is a descendant of a snapshottable
* directory. In the test we need to cover the following cases:
- *
+ *
*
* 1. Delete current INodeFile/INodeDirectory without taking any snapshot.
- * 2. Delete current INodeFile/INodeDirectory while snapshots have been taken
+ * 2. Delete current INodeFile/INodeDirectory while snapshots have been taken
* on ancestor(s).
* 3. Delete current INodeFileWithSnapshot.
* 4. Delete current INodeDirectoryWithSnapshot.
@@ -243,7 +243,7 @@ public void testDeleteCurrentFileDirectory() throws Exception {
// another file, created under noChangeDir, whose metadata will be changed
Path metaChangeFile2 = new Path(noChangeDir, "metaChangeFile2");
DFSTestUtil.createFile(hdfs, metaChangeFile2, BLOCKSIZE, REPLICATION, seed);
-
+
// Case 1: delete deleteDir before taking snapshots
hdfs.setQuota(dir, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
checkQuotaUsageComputation(dir, 10, BLOCKSIZE * REPLICATION * 4);
@@ -252,7 +252,7 @@ public void testDeleteCurrentFileDirectory() throws Exception {
// create snapshot s0
SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
-
+
// after creating snapshot s0, create a directory tempdir under dir and then
// delete dir immediately
Path tempDir = new Path(dir, "tempdir");
@@ -268,27 +268,27 @@ public void testDeleteCurrentFileDirectory() throws Exception {
for (BlockInfo b : blocks) {
assertEquals(INVALID_INODE_ID, b.getBlockCollectionId());
}
-
+
// make a change: create a new file under subsub
Path newFileAfterS0 = new Path(subsub, "newFile");
DFSTestUtil.createFile(hdfs, newFileAfterS0, BLOCKSIZE, REPLICATION, seed);
// further change: change the replicator factor of metaChangeFile
hdfs.setReplication(metaChangeFile1, REPLICATION_1);
hdfs.setReplication(metaChangeFile2, REPLICATION_1);
-
+
// create snapshot s1
SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
// check dir's quota usage
checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 4);
-
+
// get two snapshots for later use
Snapshot snapshot0 = fsdir.getINode(dir.toString()).asDirectory()
- .getSnapshot(DFSUtil.string2Bytes("s0"));
+ .getSnapshot("s0".getBytes(UTF_8));
Snapshot snapshot1 = fsdir.getINode(dir.toString()).asDirectory()
- .getSnapshot(DFSUtil.string2Bytes("s1"));
-
+ .getSnapshot("s1".getBytes(UTF_8));
+
// Case 2 + Case 3: delete noChangeDirParent, noChangeFile, and
- // metaChangeFile2. Note that when we directly delete a directory, the
+ // metaChangeFile2. Note that when we directly delete a directory, the
// directory will be converted to an INodeDirectoryWithSnapshot. To make
// sure the deletion goes through an INodeDirectory, we delete the parent
// of noChangeDir
@@ -296,12 +296,12 @@ public void testDeleteCurrentFileDirectory() throws Exception {
// while deletion, we add a diff for metaChangeFile2 as its snapshot copy
// for s1, we also add diffs for both sub and noChangeDirParent
checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 4);
-
- // check the snapshot copy of noChangeDir
+
+ // check the snapshot copy of noChangeDir
Path snapshotNoChangeDir = SnapshotTestHelper.getSnapshotPath(dir, "s1",
sub.getName() + "/" + noChangeDirParent.getName() + "/"
+ noChangeDir.getName());
- INodeDirectory snapshotNode =
+ INodeDirectory snapshotNode =
(INodeDirectory) fsdir.getINode(snapshotNoChangeDir.toString());
// should still be an INodeDirectory
assertEquals(INodeDirectory.class, snapshotNode.getClass());
@@ -314,14 +314,14 @@ public void testDeleteCurrentFileDirectory() throws Exception {
assertEquals(INodeFile.class, noChangeFileSCopy.getClass());
TestSnapshotBlocksMap.assertBlockCollection(new Path(snapshotNoChangeDir,
noChangeFileSCopy.getLocalName()).toString(), 1, fsdir, blockmanager);
-
+
INodeFile metaChangeFile2SCopy = children.get(0).asFile();
assertEquals(metaChangeFile2.getName(), metaChangeFile2SCopy.getLocalName());
assertTrue(metaChangeFile2SCopy.isWithSnapshot());
assertFalse(metaChangeFile2SCopy.isUnderConstruction());
TestSnapshotBlocksMap.assertBlockCollection(new Path(snapshotNoChangeDir,
metaChangeFile2SCopy.getLocalName()).toString(), 1, fsdir, blockmanager);
-
+
// check the replication factor of metaChangeFile2SCopy
assertEquals(REPLICATION_1,
metaChangeFile2SCopy.getFileReplication(Snapshot.CURRENT_STATE_ID));
@@ -329,7 +329,7 @@ public void testDeleteCurrentFileDirectory() throws Exception {
metaChangeFile2SCopy.getFileReplication(snapshot1.getId()));
assertEquals(REPLICATION,
metaChangeFile2SCopy.getFileReplication(snapshot0.getId()));
-
+
// Case 4: delete directory sub
// before deleting sub, we first create a new file under sub
Path newFile = new Path(sub, "newFile");
@@ -345,7 +345,7 @@ public void testDeleteCurrentFileDirectory() throws Exception {
for (BlockInfo b : blocks) {
assertEquals(INVALID_INODE_ID, b.getBlockCollectionId());
}
-
+
// make sure the whole subtree of sub is stored correctly in snapshot
Path snapshotSub = SnapshotTestHelper.getSnapshotPath(dir, "s1",
sub.getName());
@@ -356,9 +356,9 @@ public void testDeleteCurrentFileDirectory() throws Exception {
// newFile should have been destroyed
assertEquals(1, snapshotNode4Sub.getChildrenList(Snapshot.CURRENT_STATE_ID)
.size());
- // but should have two children, subsub and noChangeDir, when s1 was taken
+ // but should have two children, subsub and noChangeDir, when s1 was taken
assertEquals(2, snapshotNode4Sub.getChildrenList(snapshot1.getId()).size());
-
+
// check the snapshot copy of subsub, which is contained in the subtree of
// sub's snapshot copy
INode snapshotNode4Subsub = snapshotNode4Sub.getChildrenList(
@@ -371,7 +371,7 @@ public void testDeleteCurrentFileDirectory() throws Exception {
assertEquals(2, children.size());
assertEquals(children.get(0).getLocalName(), metaChangeFile1.getName());
assertEquals(children.get(1).getLocalName(), newFileAfterS0.getName());
- // only one child before snapshot s0
+ // only one child before snapshot s0
children = snapshotSubsubDir.getChildrenList(snapshot0.getId());
assertEquals(1, children.size());
INode child = children.get(0);
@@ -387,9 +387,9 @@ public void testDeleteCurrentFileDirectory() throws Exception {
assertEquals(REPLICATION,
metaChangeFile1SCopy.getFileReplication(snapshot0.getId()));
}
-
+
/**
- * Test deleting the earliest (first) snapshot. In this simplest scenario, the
+ * Test deleting the earliest (first) snapshot. In this simplest scenario, the
* snapshots are taken on the same directory, and we do not need to combine
* snapshot diffs.
*/
@@ -400,7 +400,7 @@ public void testDeleteEarliestSnapshot1() throws Exception {
Path file1 = new Path(sub, "file1");
DFSTestUtil.createFile(hdfs, file0, BLOCKSIZE, REPLICATION, seed);
DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, REPLICATION, seed);
-
+
String snapshotName = "s1";
try {
hdfs.deleteSnapshot(sub, snapshotName);
@@ -410,7 +410,7 @@ public void testDeleteEarliestSnapshot1() throws Exception {
GenericTestUtils.assertExceptionContains(
"Directory is not a snapshottable directory: " + sub, e);
}
-
+
// make sub snapshottable
hdfs.allowSnapshot(sub);
try {
@@ -422,7 +422,7 @@ public void testDeleteEarliestSnapshot1() throws Exception {
+ snapshotName + " from path " + sub.toString()
+ ": the snapshot does not exist.", e);
}
-
+
// create snapshot s1 for sub
SnapshotTestHelper.createSnapshot(hdfs, sub, snapshotName);
// check quota usage computation
@@ -433,7 +433,7 @@ public void testDeleteEarliestSnapshot1() throws Exception {
// now we can create a snapshot with the same name
hdfs.createSnapshot(sub, snapshotName);
checkQuotaUsageComputation(sub, 3, BLOCKSIZE * REPLICATION * 2);
-
+
// create a new file under sub
Path newFile = new Path(sub, "newFile");
DFSTestUtil.createFile(hdfs, newFile, BLOCKSIZE, REPLICATION, seed);
@@ -454,16 +454,16 @@ public void testDeleteEarliestSnapshot1() throws Exception {
assertEquals(statusBeforeDeletion.toString(),
statusAfterDeletion.toString());
}
-
+
/**
- * Test deleting the earliest (first) snapshot. In this more complicated
+ * Test deleting the earliest (first) snapshot. In this more complicated
* scenario, the snapshots are taken across directories.
*
* The test covers the following scenarios:
* 1. delete the first diff in the diff list of a directory
* 2. delete the first diff in the diff list of a file
*
- * Also, the recursive cleanTree process should cover both INodeFile and
+ * Also, the recursive cleanTree process should cover both INodeFile and
* INodeDirectory.
*/
@Test (timeout=300000)
@@ -476,15 +476,15 @@ public void testDeleteEarliestSnapshot2() throws Exception {
DFSTestUtil.createFile(hdfs, noChangeFile, BLOCKSIZE, REPLICATION, seed);
DFSTestUtil.createFile(hdfs, metaChangeFile, BLOCKSIZE, REPLICATION, seed);
DFSTestUtil.createFile(hdfs, toDeleteFile, BLOCKSIZE, REPLICATION, seed);
-
+
final INodeFile toDeleteFileNode = TestSnapshotBlocksMap
.assertBlockCollection(toDeleteFile.toString(), 1, fsdir, blockmanager);
BlockInfo[] blocks = toDeleteFileNode.getBlocks();
-
+
// create snapshot s0 on dir
SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
checkQuotaUsageComputation(dir, 7, 3 * BLOCKSIZE * REPLICATION);
-
+
// delete /TestSnapshot/sub/noChangeDir/metaChangeDir/toDeleteFile
hdfs.delete(toDeleteFile, true);
// the deletion adds diff of toDeleteFile and metaChangeDir
@@ -494,43 +494,43 @@ public void testDeleteEarliestSnapshot2() throws Exception {
hdfs.setReplication(metaChangeFile, REPLICATION_1);
hdfs.setOwner(metaChangeDir, "unknown", "unknown");
checkQuotaUsageComputation(dir, 7, 3 * BLOCKSIZE * REPLICATION);
-
+
// create snapshot s1 on dir
hdfs.createSnapshot(dir, "s1");
checkQuotaUsageComputation(dir, 7, 3 * BLOCKSIZE * REPLICATION);
-
+
// delete snapshot s0
hdfs.deleteSnapshot(dir, "s0");
- // namespace: remove toDeleteFile and its diff, metaChangeFile's diff,
- // metaChangeDir's diff, dir's diff. diskspace: remove toDeleteFile, and
+ // namespace: remove toDeleteFile and its diff, metaChangeFile's diff,
+ // metaChangeDir's diff, dir's diff. diskspace: remove toDeleteFile, and
// metaChangeFile's replication factor decreases
checkQuotaUsageComputation(dir, 6, 2 * BLOCKSIZE * REPLICATION - BLOCKSIZE);
for (BlockInfo b : blocks) {
assertEquals(INVALID_INODE_ID, b.getBlockCollectionId());
}
-
+
// check 1. there is no snapshot s0
final INodeDirectory dirNode = fsdir.getINode(dir.toString()).asDirectory();
- Snapshot snapshot0 = dirNode.getSnapshot(DFSUtil.string2Bytes("s0"));
+ Snapshot snapshot0 = dirNode.getSnapshot("s0".getBytes(UTF_8));
assertNull(snapshot0);
- Snapshot snapshot1 = dirNode.getSnapshot(DFSUtil.string2Bytes("s1"));
+ Snapshot snapshot1 = dirNode.getSnapshot("s1".getBytes(UTF_8));
DirectoryDiffList diffList = dirNode.getDiffs();
assertEquals(1, diffList.asList().size());
assertEquals(snapshot1.getId(), diffList.getLast().getSnapshotId());
diffList = fsdir.getINode(metaChangeDir.toString()).asDirectory()
.getDiffs();
assertEquals(0, diffList.asList().size());
-
+
// check 2. noChangeDir and noChangeFile are still there
- final INodeDirectory noChangeDirNode =
+ final INodeDirectory noChangeDirNode =
(INodeDirectory) fsdir.getINode(noChangeDir.toString());
assertEquals(INodeDirectory.class, noChangeDirNode.getClass());
- final INodeFile noChangeFileNode =
+ final INodeFile noChangeFileNode =
(INodeFile) fsdir.getINode(noChangeFile.toString());
assertEquals(INodeFile.class, noChangeFileNode.getClass());
TestSnapshotBlocksMap.assertBlockCollection(noChangeFile.toString(), 1,
fsdir, blockmanager);
-
+
// check 3: current metadata of metaChangeFile and metaChangeDir
FileStatus status = hdfs.getFileStatus(metaChangeDir);
assertEquals("unknown", status.getOwner());
@@ -539,7 +539,7 @@ public void testDeleteEarliestSnapshot2() throws Exception {
assertEquals(REPLICATION_1, status.getReplication());
TestSnapshotBlocksMap.assertBlockCollection(metaChangeFile.toString(), 1,
fsdir, blockmanager);
-
+
// check 4: no snapshot copy for toDeleteFile
try {
hdfs.getFileStatus(toDeleteFile);
@@ -548,7 +548,7 @@ public void testDeleteEarliestSnapshot2() throws Exception {
GenericTestUtils.assertExceptionContains("File does not exist: "
+ toDeleteFile.toString(), e);
}
-
+
final Path toDeleteFileInSnapshot = SnapshotTestHelper.getSnapshotPath(dir,
"s0", toDeleteFile.toString().substring(dir.toString().length()));
try {
@@ -642,7 +642,7 @@ public void testDeleteSnapshot2() throws Exception {
public void testCombineSnapshotDiff1() throws Exception {
testCombineSnapshotDiffImpl(sub, "", 1);
}
-
+
/**
* Test deleting snapshots in more complicated scenarios (snapshot diffs are
* distributed in the directory sub-tree)
@@ -651,9 +651,9 @@ public void testCombineSnapshotDiff1() throws Exception {
public void testCombineSnapshotDiff2() throws Exception {
testCombineSnapshotDiffImpl(sub, "subsub1/subsubsub1/", 3);
}
-
+
/**
- * When combine two snapshots, make sure files/directories created after the
+ * When combine two snapshots, make sure files/directories created after the
* prior snapshot get destroyed.
*/
@Test (timeout=300000)
@@ -665,34 +665,34 @@ public void testCombineSnapshotDiff3() throws Exception {
hdfs.mkdirs(subDir2);
Path subsubDir = new Path(subDir1, "subsubdir");
hdfs.mkdirs(subsubDir);
-
+
// take snapshots on subdir and dir
SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
-
+
// create new dir under initial dir
Path newDir = new Path(subsubDir, "newdir");
Path newFile = new Path(newDir, "newfile");
DFSTestUtil.createFile(hdfs, newFile, BLOCKSIZE, REPLICATION, seed);
Path newFile2 = new Path(subDir2, "newfile");
DFSTestUtil.createFile(hdfs, newFile2, BLOCKSIZE, REPLICATION, seed);
-
+
// create another snapshot
SnapshotTestHelper.createSnapshot(hdfs, dir, "s2");
-
+
checkQuotaUsageComputation(dir, 7, BLOCKSIZE * 2 * REPLICATION);
-
+
// delete subsubdir and subDir2
hdfs.delete(subsubDir, true);
hdfs.delete(subDir2, true);
-
+
// add diff of s2 to subDir1, subsubDir, and subDir2
checkQuotaUsageComputation(dir, 7, BLOCKSIZE * 2 * REPLICATION);
-
+
// delete snapshot s2
hdfs.deleteSnapshot(dir, "s2");
-
+
// delete s2 diff in dir, subDir2, and subsubDir. Delete newFile, newDir,
- // and newFile2. Rename s2 diff to s1 for subDir1
+ // and newFile2. Rename s2 diff to s1 for subDir1
checkQuotaUsageComputation(dir, 4, 0);
// Check rename of snapshot diff in subDir1
Path subdir1_s1 = SnapshotTestHelper.getSnapshotPath(dir, "s1",
@@ -702,12 +702,12 @@ public void testCombineSnapshotDiff3() throws Exception {
assertTrue(hdfs.exists(subdir1_s1));
assertFalse(hdfs.exists(subdir1_s2));
}
-
+
/**
* Test snapshot deletion
* @param snapshotRoot The dir where the snapshots are created
- * @param modDirStr The snapshotRoot itself or one of its sub-directory,
- * where the modifications happen. It is represented as a relative
+ * @param modDirStr The snapshotRoot itself or one of its sub-directory,
+ * where the modifications happen. It is represented as a relative
* path to the snapshotRoot.
*/
private void testCombineSnapshotDiffImpl(Path snapshotRoot, String modDirStr,
@@ -728,55 +728,55 @@ private void testCombineSnapshotDiffImpl(Path snapshotRoot, String modDirStr,
// create snapshot s1 for snapshotRoot
SnapshotTestHelper.createSnapshot(hdfs, snapshotRoot, "s1");
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 4, 8 * BLOCKSIZE);
-
+
// delete file11
hdfs.delete(file11, true);
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 4, 8 * BLOCKSIZE);
-
+
// modify file12
hdfs.setReplication(file12, REPLICATION);
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 4, 9 * BLOCKSIZE);
-
+
// modify file13
hdfs.setReplication(file13, REPLICATION);
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 4, 10 * BLOCKSIZE);
-
+
// create file14
DFSTestUtil.createFile(hdfs, file14, BLOCKSIZE, REPLICATION, seed);
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 5, 13 * BLOCKSIZE);
-
+
// create file15
DFSTestUtil.createFile(hdfs, file15, BLOCKSIZE, REPLICATION, seed);
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 6, 16 * BLOCKSIZE);
-
+
// create snapshot s2 for snapshotRoot
hdfs.createSnapshot(snapshotRoot, "s2");
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 6, 16 * BLOCKSIZE);
-
+
// create file11 again: (0, d) + (c, 0)
DFSTestUtil.createFile(hdfs, file11, BLOCKSIZE, REPLICATION, seed);
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 19 * BLOCKSIZE);
-
+
// delete file12
hdfs.delete(file12, true);
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 19 * BLOCKSIZE);
-
+
// modify file13
hdfs.setReplication(file13, (short) (REPLICATION - 2));
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 19 * BLOCKSIZE);
-
+
// delete file14: (c, 0) + (0, d)
hdfs.delete(file14, true);
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 19 * BLOCKSIZE);
-
+
// modify file15
hdfs.setReplication(file15, REPLICATION_1);
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 19 * BLOCKSIZE);
-
+
// create snapshot s3 for snapshotRoot
hdfs.createSnapshot(snapshotRoot, "s3");
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 19 * BLOCKSIZE);
-
+
// modify file10, to check if the posterior diff was set correctly
hdfs.setReplication(file10, REPLICATION);
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 20 * BLOCKSIZE);
@@ -802,11 +802,11 @@ private void testCombineSnapshotDiffImpl(Path snapshotRoot, String modDirStr,
BlockInfo[] blocks_14 = file14Node.getBlocks();
TestSnapshotBlocksMap.assertBlockCollection(file15_s2.toString(), 1, fsdir,
blockmanager);
-
+
// delete s2, in which process we need to combine the diff in s2 to s1
hdfs.deleteSnapshot(snapshotRoot, "s2");
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 6, 14 * BLOCKSIZE);
-
+
// check the correctness of s1
FileStatus statusAfterDeletion10 = hdfs.getFileStatus(file10_s1);
FileStatus statusAfterDeletion11 = hdfs.getFileStatus(file11_s1);
@@ -852,44 +852,44 @@ private void testCombineSnapshotDiffImpl(Path snapshotRoot, String modDirStr,
assertEquals(REPLICATION_1, b.getReplication());
}
}
-
- /** Test deleting snapshots with modification on the metadata of directory */
+
+ /** Test deleting snapshots with modification on the metadata of directory */
@Test (timeout=300000)
public void testDeleteSnapshotWithDirModification() throws Exception {
Path file = new Path(sub, "file");
DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed);
hdfs.setOwner(sub, "user1", "group1");
-
+
// create snapshot s1 for sub1, and change the metadata of sub1
SnapshotTestHelper.createSnapshot(hdfs, sub, "s1");
checkQuotaUsageComputation(sub, 2, BLOCKSIZE * 3);
hdfs.setOwner(sub, "user2", "group2");
checkQuotaUsageComputation(sub, 2, BLOCKSIZE * 3);
-
+
// create snapshot s2 for sub1, but do not modify sub1 afterwards
hdfs.createSnapshot(sub, "s2");
checkQuotaUsageComputation(sub, 2, BLOCKSIZE * 3);
-
+
// create snapshot s3 for sub1, and change the metadata of sub1
hdfs.createSnapshot(sub, "s3");
checkQuotaUsageComputation(sub, 2, BLOCKSIZE * 3);
hdfs.setOwner(sub, "user3", "group3");
checkQuotaUsageComputation(sub, 2, BLOCKSIZE * 3);
-
+
// delete snapshot s3
hdfs.deleteSnapshot(sub, "s3");
checkQuotaUsageComputation(sub, 2, BLOCKSIZE * 3);
-
+
// check sub1's metadata in snapshot s2
FileStatus statusOfS2 = hdfs.getFileStatus(new Path(sub,
HdfsConstants.DOT_SNAPSHOT_DIR + "/s2"));
assertEquals("user2", statusOfS2.getOwner());
assertEquals("group2", statusOfS2.getGroup());
-
+
// delete snapshot s2
hdfs.deleteSnapshot(sub, "s2");
checkQuotaUsageComputation(sub, 2, BLOCKSIZE * 3);
-
+
// check sub1's metadata in snapshot s1
FileStatus statusOfS1 = hdfs.getFileStatus(new Path(sub,
HdfsConstants.DOT_SNAPSHOT_DIR + "/s1"));
@@ -930,9 +930,9 @@ public Object run() {
});
}
- /**
- * A test covering the case where the snapshot diff to be deleted is renamed
- * to its previous snapshot.
+ /**
+ * A test covering the case where the snapshot diff to be deleted is renamed
+ * to its previous snapshot.
*/
@Test (timeout=300000)
public void testRenameSnapshotDiff() throws Exception {
@@ -943,7 +943,7 @@ public void testRenameSnapshotDiff() throws Exception {
DFSTestUtil.createFile(hdfs, subFile0, BLOCKSIZE, REPLICATION, seed);
DFSTestUtil.createFile(hdfs, subsubFile0, BLOCKSIZE, REPLICATION, seed);
hdfs.setOwner(subsub, "owner", "group");
-
+
// create snapshot s0 on sub
SnapshotTestHelper.createSnapshot(hdfs, sub, "s0");
checkQuotaUsageComputation(sub, 4, BLOCKSIZE * 6);
@@ -953,28 +953,28 @@ public void testRenameSnapshotDiff() throws Exception {
DFSTestUtil.createFile(hdfs, subFile1, BLOCKSIZE, REPLICATION_1, seed);
DFSTestUtil.createFile(hdfs, subsubFile1, BLOCKSIZE, REPLICATION, seed);
checkQuotaUsageComputation(sub, 6, BLOCKSIZE * 11);
-
+
// create snapshot s1 on sub
SnapshotTestHelper.createSnapshot(hdfs, sub, "s1");
checkQuotaUsageComputation(sub, 6, BLOCKSIZE * 11);
-
+
// create snapshot s2 on dir
SnapshotTestHelper.createSnapshot(hdfs, dir, "s2");
checkQuotaUsageComputation(dir, 7, BLOCKSIZE * 11);
checkQuotaUsageComputation(sub, 6, BLOCKSIZE * 11);
-
+
// make changes on subsub and subsubFile1
hdfs.setOwner(subsub, "unknown", "unknown");
hdfs.setReplication(subsubFile1, REPLICATION_1);
checkQuotaUsageComputation(dir, 7, BLOCKSIZE * 11);
checkQuotaUsageComputation(sub, 6, BLOCKSIZE * 11);
-
+
// make changes on sub
hdfs.delete(subFile1, true);
checkQuotaUsageComputation(new Path("/"), 8, BLOCKSIZE * 11);
checkQuotaUsageComputation(dir, 7, BLOCKSIZE * 11);
checkQuotaUsageComputation(sub, 6, BLOCKSIZE * 11);
-
+
Path subsubSnapshotCopy = SnapshotTestHelper.getSnapshotPath(dir, "s2",
sub.getName() + Path.SEPARATOR + subsub.getName());
Path subsubFile1SCopy = SnapshotTestHelper.getSnapshotPath(dir, "s2",
@@ -989,13 +989,13 @@ public void testRenameSnapshotDiff() throws Exception {
assertEquals(REPLICATION, subsubFile1Status.getReplication());
FileStatus subFile1Status = hdfs.getFileStatus(subFile1SCopy);
assertEquals(REPLICATION_1, subFile1Status.getReplication());
-
+
// delete snapshot s2
hdfs.deleteSnapshot(dir, "s2");
checkQuotaUsageComputation(new Path("/"), 8, BLOCKSIZE * 11);
checkQuotaUsageComputation(dir, 7, BLOCKSIZE * 11);
checkQuotaUsageComputation(sub, 6, BLOCKSIZE * 11);
-
+
// no snapshot copy for s2
try {
hdfs.getFileStatus(subsubSnapshotCopy);
@@ -1018,7 +1018,7 @@ public void testRenameSnapshotDiff() throws Exception {
GenericTestUtils.assertExceptionContains("File does not exist: "
+ subFile1SCopy.toString(), e);
}
-
+
// the snapshot copy of s2 should now be renamed to s1 under sub
subsubSnapshotCopy = SnapshotTestHelper.getSnapshotPath(sub, "s1",
subsub.getName());
@@ -1031,12 +1031,12 @@ public void testRenameSnapshotDiff() throws Exception {
assertEquals("group", subsubStatus.getGroup());
subsubFile1Status = hdfs.getFileStatus(subsubFile1SCopy);
assertEquals(REPLICATION, subsubFile1Status.getReplication());
- // also subFile1's snapshot copy should have been moved to diff of s1 as
+ // also subFile1's snapshot copy should have been moved to diff of s1 as
// combination
subFile1Status = hdfs.getFileStatus(subFile1SCopy);
assertEquals(REPLICATION_1, subFile1Status.getReplication());
}
-
+
@Test
public void testDeleteSnapshotCommandWithIllegalArguments() throws Exception {
ByteArrayOutputStream out = new ByteArrayOutputStream();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java
index 453afacddac6a..9bd2317135b98 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java
@@ -22,13 +22,13 @@
import static org.junit.Assert.fail;
import java.io.IOException;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.HashMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -48,14 +48,14 @@ public class TestSnapshotDiffReport {
protected static final short REPLICATION_1 = 2;
protected static final long BLOCKSIZE = 1024;
public static final int SNAPSHOTNUMBER = 10;
-
+
private final Path dir = new Path("/TestSnapshot");
private final Path sub1 = new Path(dir, "sub1");
-
+
protected Configuration conf;
protected MiniDFSCluster cluster;
protected DistributedFileSystem hdfs;
-
+
private final HashMap snapshotNumberMap = new HashMap();
@Before
@@ -74,7 +74,7 @@ public void tearDown() throws Exception {
cluster = null;
}
}
-
+
private String genSnapshotName(Path snapshotDir) {
int sNum = -1;
if (snapshotNumberMap.containsKey(snapshotDir)) {
@@ -83,11 +83,11 @@ private String genSnapshotName(Path snapshotDir) {
snapshotNumberMap.put(snapshotDir, ++sNum);
return "s" + sNum;
}
-
+
/**
* Create/modify/delete files under a given directory, also create snapshots
* of directories.
- */
+ */
private void modifyAndCreateSnapshot(Path modifyDir, Path[] snapshotDirs)
throws Exception {
Path file10 = new Path(modifyDir, "file10");
@@ -108,7 +108,7 @@ private void modifyAndCreateSnapshot(Path modifyDir, Path[] snapshotDirs)
hdfs.allowSnapshot(snapshotDir);
hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
}
-
+
// delete file11
hdfs.delete(file11, true);
// modify file12
@@ -121,12 +121,12 @@ private void modifyAndCreateSnapshot(Path modifyDir, Path[] snapshotDirs)
DFSTestUtil.createFile(hdfs, file14, BLOCKSIZE, REPLICATION, seed);
// create file15
DFSTestUtil.createFile(hdfs, file15, BLOCKSIZE, REPLICATION, seed);
-
+
// create snapshot
for (Path snapshotDir : snapshotDirs) {
hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
}
-
+
// create file11 again
DFSTestUtil.createFile(hdfs, file11, BLOCKSIZE, REPLICATION, seed);
// delete file12
@@ -139,7 +139,7 @@ private void modifyAndCreateSnapshot(Path modifyDir, Path[] snapshotDirs)
hdfs.delete(file14, true);
// modify file15
hdfs.setReplication(file15, (short) (REPLICATION - 1));
-
+
// create snapshot
for (Path snapshotDir : snapshotDirs) {
hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
@@ -147,7 +147,7 @@ private void modifyAndCreateSnapshot(Path modifyDir, Path[] snapshotDirs)
// modify file10
hdfs.setReplication(file10, (short) (REPLICATION + 1));
}
-
+
/** check the correctness of the diff reports */
private void verifyDiffReport(Path dir, String from, String to,
DiffReportEntry... entries) throws IOException {
@@ -157,10 +157,10 @@ private void verifyDiffReport(Path dir, String from, String to,
.getSnapshotDiffReport(dir, to, from);
System.out.println(report.toString());
System.out.println(inverseReport.toString() + "\n");
-
+
assertEquals(entries.length, report.getDiffList().size());
assertEquals(entries.length, inverseReport.getDiffList().size());
-
+
for (DiffReportEntry entry : entries) {
if (entry.getType() == DiffType.MODIFY) {
assertTrue(report.getDiffList().contains(entry));
@@ -176,7 +176,7 @@ private void verifyDiffReport(Path dir, String from, String to,
}
}
}
-
+
/** Test the computation and representation of diff between snapshots */
@Test (timeout=60000)
public void testDiffReport() throws Exception {
@@ -187,7 +187,7 @@ public void testDiffReport() throws Exception {
hdfs.mkdirs(subsubsub1);
modifyAndCreateSnapshot(sub1, new Path[]{sub1, subsubsub1});
modifyAndCreateSnapshot(subsubsub1, new Path[]{sub1, subsubsub1});
-
+
try {
hdfs.getSnapshotDiffReport(subsub1, "s1", "s2");
fail("Expect exception when getting snapshot diff report: " + subsub1
@@ -196,7 +196,7 @@ public void testDiffReport() throws Exception {
GenericTestUtils.assertExceptionContains(
"Directory is not a snapshottable directory: " + subsub1, e);
}
-
+
final String invalidName = "invalid";
try {
hdfs.getSnapshotDiffReport(sub1, invalidName, invalidName);
@@ -206,16 +206,16 @@ public void testDiffReport() throws Exception {
"Cannot find the snapshot of directory " + sub1 + " with name "
+ invalidName, e);
}
-
+
// diff between the same snapshot
SnapshotDiffReport report = hdfs.getSnapshotDiffReport(sub1, "s0", "s0");
System.out.println(report);
assertEquals(0, report.getDiffList().size());
-
+
report = hdfs.getSnapshotDiffReport(sub1, "", "");
System.out.println(report);
assertEquals(0, report.getDiffList().size());
-
+
report = hdfs.getSnapshotDiffReport(subsubsub1, "s0", "s2");
System.out.println(report);
assertEquals(0, report.getDiffList().size());
@@ -225,75 +225,75 @@ public void testDiffReport() throws Exception {
System.out.println(report);
assertEquals(0, report.getDiffList().size());
- verifyDiffReport(sub1, "s0", "s2",
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
- new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file15")),
- new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file12")),
- new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file11")),
- new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file11")),
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("file13")),
- new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("link13")),
- new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("link13")));
-
- verifyDiffReport(sub1, "s0", "s5",
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
- new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file15")),
- new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file12")),
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("file10")),
- new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file11")),
- new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file11")),
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("file13")),
- new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("link13")),
- new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("link13")),
+ verifyDiffReport(sub1, "s0", "s2",
+ new DiffReportEntry(DiffType.MODIFY, "".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.CREATE, "file15".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.DELETE, "file12".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.DELETE, "file11".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.CREATE, "file11".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.MODIFY, "file13".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.DELETE, "link13".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.CREATE, "link13".getBytes(UTF_8)));
+
+ verifyDiffReport(sub1, "s0", "s5",
+ new DiffReportEntry(DiffType.MODIFY, "".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.CREATE, "file15".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.DELETE, "file12".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.MODIFY, "file10".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.DELETE, "file11".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.CREATE, "file11".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.MODIFY, "file13".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.DELETE, "link13".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.CREATE, "link13".getBytes(UTF_8)),
new DiffReportEntry(DiffType.MODIFY,
- DFSUtil.string2Bytes("subsub1/subsubsub1")),
+ "subsub1/subsubsub1".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file10")),
+ "subsub1/subsubsub1/file10".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file11")),
+ "subsub1/subsubsub1/file11".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file13")),
+ "subsub1/subsubsub1/file13".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/link13")),
+ "subsub1/subsubsub1/link13".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file15")));
-
+ "subsub1/subsubsub1/file15".getBytes(UTF_8)));
+
verifyDiffReport(sub1, "s2", "s5",
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("file10")),
+ new DiffReportEntry(DiffType.MODIFY, "file10".getBytes(UTF_8)),
new DiffReportEntry(DiffType.MODIFY,
- DFSUtil.string2Bytes("subsub1/subsubsub1")),
+ "subsub1/subsubsub1".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file10")),
+ "subsub1/subsubsub1/file10".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file11")),
+ "subsub1/subsubsub1/file11".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file13")),
+ "subsub1/subsubsub1/file13".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/link13")),
+ "subsub1/subsubsub1/link13".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file15")));
-
+ "subsub1/subsubsub1/file15".getBytes(UTF_8)));
+
verifyDiffReport(sub1, "s3", "",
new DiffReportEntry(DiffType.MODIFY,
- DFSUtil.string2Bytes("subsub1/subsubsub1")),
+ "subsub1/subsubsub1".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file15")),
+ "subsub1/subsubsub1/file15".getBytes(UTF_8)),
new DiffReportEntry(DiffType.DELETE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file12")),
+ "subsub1/subsubsub1/file12".getBytes(UTF_8)),
new DiffReportEntry(DiffType.MODIFY,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file10")),
+ "subsub1/subsubsub1/file10".getBytes(UTF_8)),
new DiffReportEntry(DiffType.DELETE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file11")),
+ "subsub1/subsubsub1/file11".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file11")),
+ "subsub1/subsubsub1/file11".getBytes(UTF_8)),
new DiffReportEntry(DiffType.MODIFY,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file13")),
+ "subsub1/subsubsub1/file13".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/link13")),
+ "subsub1/subsubsub1/link13".getBytes(UTF_8)),
new DiffReportEntry(DiffType.DELETE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/link13")));
+ "subsub1/subsubsub1/link13".getBytes(UTF_8)));
}
-
+
/**
* Make changes under a sub-directory, then delete the sub-directory. Make
* sure the diff report computation correctly retrieve the diff from the
@@ -305,31 +305,31 @@ public void testDiffReport2() throws Exception {
Path subsubsub1 = new Path(subsub1, "subsubsub1");
hdfs.mkdirs(subsubsub1);
modifyAndCreateSnapshot(subsubsub1, new Path[]{sub1});
-
+
// delete subsub1
hdfs.delete(subsub1, true);
// check diff report between s0 and s2
- verifyDiffReport(sub1, "s0", "s2",
+ verifyDiffReport(sub1, "s0", "s2",
new DiffReportEntry(DiffType.MODIFY,
- DFSUtil.string2Bytes("subsub1/subsubsub1")),
- new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file15")),
+ "subsub1/subsubsub1".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.CREATE,
+ "subsub1/subsubsub1/file15".getBytes(UTF_8)),
new DiffReportEntry(DiffType.DELETE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file12")),
+ "subsub1/subsubsub1/file12".getBytes(UTF_8)),
new DiffReportEntry(DiffType.DELETE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file11")),
+ "subsub1/subsubsub1/file11".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file11")),
+ "subsub1/subsubsub1/file11".getBytes(UTF_8)),
new DiffReportEntry(DiffType.MODIFY,
- DFSUtil.string2Bytes("subsub1/subsubsub1/file13")),
+ "subsub1/subsubsub1/file13".getBytes(UTF_8)),
new DiffReportEntry(DiffType.CREATE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/link13")),
+ "subsub1/subsubsub1/link13".getBytes(UTF_8)),
new DiffReportEntry(DiffType.DELETE,
- DFSUtil.string2Bytes("subsub1/subsubsub1/link13")));
+ "subsub1/subsubsub1/link13".getBytes(UTF_8)));
// check diff report between s0 and the current status
- verifyDiffReport(sub1, "s0", "",
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
- new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("subsub1")));
+ verifyDiffReport(sub1, "s0", "",
+ new DiffReportEntry(DiffType.MODIFY, "".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.DELETE, "subsub1".getBytes(UTF_8)));
}
/**
@@ -361,16 +361,15 @@ public void testDiffReportWithRename() throws Exception {
hdfs.delete(sdir2, true);
verifyDiffReport(root, "s1", "s2",
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir1")),
- new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir1/foo"),
- DFSUtil.string2Bytes("dir2/bar/foo")),
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir2")),
- new DiffReportEntry(DiffType.MODIFY,
- DFSUtil.string2Bytes("dir1/foo/bar")),
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir1/foo")),
- new DiffReportEntry(DiffType.RENAME, DFSUtil
- .string2Bytes("dir1/foo/bar"), DFSUtil.string2Bytes("dir2/bar")));
+ new DiffReportEntry(DiffType.MODIFY, "".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.MODIFY, "dir1".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.RENAME, "dir1/foo".getBytes(UTF_8),
+ "dir2/bar/foo".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.MODIFY, "dir2".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.MODIFY, "dir1/foo/bar".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.MODIFY, "dir1/foo".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.RENAME, "dir1/foo/bar".getBytes(UTF_8),
+ "dir2/bar".getBytes(UTF_8)));
}
/**
@@ -401,11 +400,9 @@ public void testDiffReportWithRenameOutside() throws Exception {
SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
verifyDiffReport(dir1, "s0", "s1",
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
- new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes(newBar
- .getName())),
- new DiffReportEntry(DiffType.DELETE,
- DFSUtil.string2Bytes(foo.getName())));
+ new DiffReportEntry(DiffType.MODIFY, "".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.CREATE, newBar.getName().getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.DELETE, foo.getName().getBytes(UTF_8)));
}
/**
@@ -428,25 +425,22 @@ public void testDiffReportWithRenameAndDelete() throws Exception {
hdfs.rename(fileInFoo, fileInBar, Rename.OVERWRITE);
SnapshotTestHelper.createSnapshot(hdfs, root, "s1");
verifyDiffReport(root, "s0", "s1",
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir1/foo")),
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir2/bar")),
- new DiffReportEntry(DiffType.DELETE, DFSUtil
- .string2Bytes("dir2/bar/file")),
- new DiffReportEntry(DiffType.RENAME,
- DFSUtil.string2Bytes("dir1/foo/file"),
- DFSUtil.string2Bytes("dir2/bar/file")));
+ new DiffReportEntry(DiffType.MODIFY, "".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.MODIFY, "dir1/foo".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.MODIFY, "dir2/bar".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.DELETE, "dir2/bar/file".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.RENAME, "dir1/foo/file".getBytes(UTF_8),
+ "dir2/bar/file".getBytes(UTF_8)));
// delete bar
hdfs.delete(bar, true);
SnapshotTestHelper.createSnapshot(hdfs, root, "s2");
verifyDiffReport(root, "s0", "s2",
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir1/foo")),
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir2")),
- new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("dir2/bar")),
- new DiffReportEntry(DiffType.DELETE,
- DFSUtil.string2Bytes("dir1/foo/file")));
+ new DiffReportEntry(DiffType.MODIFY, "".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.MODIFY, "dir1/foo".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.MODIFY, "dir2".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.DELETE, "dir2/bar".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.DELETE, "dir1/foo/file".getBytes(UTF_8)));
}
@Test
@@ -464,11 +458,11 @@ public void testDiffReportWithRenameToNewDir() throws Exception {
SnapshotTestHelper.createSnapshot(hdfs, root, "s1");
verifyDiffReport(root, "s0", "s1",
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("foo")),
- new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("bar")),
- new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("foo/file"),
- DFSUtil.string2Bytes("bar/file")));
+ new DiffReportEntry(DiffType.MODIFY, "".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.MODIFY, "foo".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.CREATE, "bar".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.RENAME, "foo/file".getBytes(UTF_8),
+ "bar/file".getBytes(UTF_8)));
}
/**
@@ -488,10 +482,10 @@ public void testDiffReportWithRenameAndAppend() throws Exception {
// we always put modification on the file before rename
verifyDiffReport(root, "s0", "s1",
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("foo")),
- new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("foo"),
- DFSUtil.string2Bytes("bar")));
+ new DiffReportEntry(DiffType.MODIFY, "".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.MODIFY, "foo".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.RENAME, "foo".getBytes(UTF_8),
+ "bar".getBytes(UTF_8)));
}
/**
@@ -524,9 +518,9 @@ public void testDiffReportWithRenameAndSnapshotDeletion() throws Exception {
// we always put modification on the file before rename
verifyDiffReport(root, "s1", "",
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
- new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("foo2")),
- new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("foo2/bar"),
- DFSUtil.string2Bytes("foo2/bar-new")));
+ new DiffReportEntry(DiffType.MODIFY, "".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.MODIFY, "foo2".getBytes(UTF_8)),
+ new DiffReportEntry(DiffType.RENAME, "foo2/bar".getBytes(UTF_8),
+ "foo2/bar-new".getBytes(UTF_8)));
}
-}
\ No newline at end of file
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestDiff.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestDiff.java
index 9c6839c99b7de..652c4a9eb4f7a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestDiff.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestDiff.java
@@ -17,13 +17,13 @@
*/
package org.apache.hadoop.hdfs.util;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
import org.apache.hadoop.hdfs.util.Diff;
@@ -85,8 +85,8 @@ void runDiffTest(int startSize, int numModifications) {
// make modifications to current and record the diff
final List current = new ArrayList(previous);
-
- final List> diffs =
+
+ final List> diffs =
new ArrayList>();
for(int j = 0; j < 5; j++) {
diffs.add(new Diff());
@@ -199,7 +199,7 @@ void runDiffTest(int startSize, int numModifications) {
final Container r = combined.accessCurrent(inode.getKey());
final INode computed;
if (r != null) {
- computed = r.getElement();
+ computed = r.getElement();
} else {
final int i = Diff.search(previous, inode.getKey());
computed = i < 0? null: previous.get(i);
@@ -243,7 +243,7 @@ static int findWidth(int max) {
}
static INode newINode(int n, int width) {
- byte[] name = DFSUtil.string2Bytes(String.format("n%0" + width + "d", n));
+ byte[] name = String.format("n%0" + width + "d", n).getBytes(UTF_8);
return new INodeDirectory(n, name, PERM, 0L);
}
@@ -331,7 +331,7 @@ static void modify(INode inode, final List current,
}
}
}
-
+
static void assertDiff(String s, Diff diff) {
Assert.assertEquals(s, diff.toString());
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
index 94cec8596bd9c..3ac4233ab2461 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
@@ -23,6 +23,7 @@
import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.*;
import java.io.IOException;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
@@ -36,7 +37,6 @@
import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.XAttrHelper;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -64,7 +64,7 @@ public void testHdfsFileStatus() throws IOException {
final String parent = "/dir";
final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26,
now, now + 10, new FsPermission((short) 0644), "user", "group",
- DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"),
+ "bar".getBytes(UTF_8), "foo".getBytes(UTF_8),
HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0, null);
final FileStatus fstatus = toFileStatus(status, parent);
System.out.println("status = " + status);
@@ -79,11 +79,11 @@ public void testHdfsFileStatus() throws IOException {
System.out.println("fs2 = " + fs2);
Assert.assertEquals(fstatus, fs2);
}
-
+
@Test
public void testToDatanodeInfoWithoutSecurePort() throws Exception {
Map response = new HashMap();
-
+
response.put("ipAddr", "127.0.0.1");
response.put("hostName", "localhost");
response.put("storageID", "fake-id");
@@ -101,7 +101,7 @@ public void testToDatanodeInfoWithoutSecurePort() throws Exception {
response.put("adminState", "NORMAL");
response.put("cacheCapacity", 123l);
response.put("cacheUsed", 321l);
-
+
JsonUtilClient.toDatanodeInfo(response);
}
@@ -154,7 +154,7 @@ public void testToDatanodeInfoWithName() throws Exception {
response.put("ipAddr", "127.0.0.1");
checkDecodeFailure(response);
}
-
+
@Test
public void testToAclStatus() throws IOException {
String jsonString =
@@ -196,10 +196,10 @@ public void testToJsonFromAclStatus() {
JsonUtil.toJsonString(aclStatusBuilder.build()));
}
-
+
@Test
public void testToJsonFromXAttrs() throws IOException {
- String jsonString =
+ String jsonString =
"{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
"{\"name\":\"user.a2\",\"value\":\"0x313131\"}]}";
XAttr xAttr1 = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.USER).
@@ -209,14 +209,14 @@ public void testToJsonFromXAttrs() throws IOException {
List xAttrs = Lists.newArrayList();
xAttrs.add(xAttr1);
xAttrs.add(xAttr2);
-
- Assert.assertEquals(jsonString, JsonUtil.toJsonString(xAttrs,
+
+ Assert.assertEquals(jsonString, JsonUtil.toJsonString(xAttrs,
XAttrCodec.HEX));
}
-
+
@Test
public void testToXAttrMap() throws IOException {
- String jsonString =
+ String jsonString =
"{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
"{\"name\":\"user.a2\",\"value\":\"0x313131\"}]}";
ObjectReader reader = new ObjectMapper().reader(Map.class);
@@ -230,19 +230,19 @@ public void testToXAttrMap() throws IOException {
xAttrs.add(xAttr2);
Map xAttrMap = XAttrHelper.buildXAttrMap(xAttrs);
Map parsedXAttrMap = JsonUtilClient.toXAttrs(json);
-
+
Assert.assertEquals(xAttrMap.size(), parsedXAttrMap.size());
Iterator> iter = xAttrMap.entrySet().iterator();
while(iter.hasNext()) {
Entry entry = iter.next();
- Assert.assertArrayEquals(entry.getValue(),
+ Assert.assertArrayEquals(entry.getValue(),
parsedXAttrMap.get(entry.getKey()));
}
}
-
+
@Test
public void testGetXAttrFromJson() throws IOException {
- String jsonString =
+ String jsonString =
"{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
"{\"name\":\"user.a2\",\"value\":\"0x313131\"}]}";
ObjectReader reader = new ObjectMapper().reader(Map.class);
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpSync.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpSync.java
index f1fae11bd2293..e9ffc333fc5db 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpSync.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpSync.java
@@ -22,7 +22,6 @@
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -30,6 +29,7 @@
import java.io.FileNotFoundException;
import java.io.IOException;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.Arrays;
import java.util.List;
import java.util.Random;
@@ -215,13 +215,13 @@ private boolean getAllDiffs() throws IOException {
dt == SnapshotDiffReport.DiffType.CREATE ||
dt == SnapshotDiffReport.DiffType.DELETE) {
final Path source =
- new Path(DFSUtilClient.bytes2String(entry.getSourcePath()));
+ new Path(new String(entry.getSourcePath(), UTF_8));
list.add(new DiffInfo(source, null, dt));
} else if (dt == SnapshotDiffReport.DiffType.RENAME) {
final Path source =
- new Path(DFSUtilClient.bytes2String(entry.getSourcePath()));
+ new Path(new String(entry.getSourcePath(), UTF_8));
final Path target =
- new Path(DFSUtilClient.bytes2String(entry.getTargetPath()));
+ new Path(new String(entry.getTargetPath(), UTF_8));
list.add(new DiffInfo(source, target, dt));
}
}