You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by bo...@apache.org on 2011/04/21 20:13:46 UTC
svn commit: r1095789 - in /hadoop/hdfs/trunk: ./ src/java/
src/java/org/apache/hadoop/hdfs/ src/java/org/apache/hadoop/hdfs/protocol/
src/java/org/apache/hadoop/hdfs/server/namenode/
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/
Author: boryas
Date: Thu Apr 21 18:13:46 2011
New Revision: 1095789
URL: http://svn.apache.org/viewvc?rev=1095789&view=rev
Log:
HDFS-1751. Intrinsic limits for HDFS files, directories
Added:
hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/protocol/FSLimitException.java
hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java
Modified:
hadoop/hdfs/trunk/CHANGES.txt
hadoop/hdfs/trunk/src/java/hdfs-default.xml
hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
Modified: hadoop/hdfs/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/CHANGES.txt?rev=1095789&r1=1095788&r2=1095789&view=diff
==============================================================================
--- hadoop/hdfs/trunk/CHANGES.txt (original)
+++ hadoop/hdfs/trunk/CHANGES.txt Thu Apr 21 18:13:46 2011
@@ -34,6 +34,8 @@ Trunk (unreleased changes)
HDFS-1070. Speedup namenode image loading and saving by storing only
local file names. (hairong)
+
+ HDFS-1751. Intrinsic limits for HDFS files, directories (daryn via boryas).
IMPROVEMENTS
Modified: hadoop/hdfs/trunk/src/java/hdfs-default.xml
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/hdfs-default.xml?rev=1095789&r1=1095788&r2=1095789&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/hdfs-default.xml (original)
+++ hadoop/hdfs/trunk/src/java/hdfs-default.xml Thu Apr 21 18:13:46 2011
@@ -195,6 +195,20 @@ creations/deletions), or "all".</descrip
</property>
<property>
+ <name>dfs.namenode.fs-limits.max-component-length</name>
+ <value>0</value>
+ <description>Defines the maximum number of characters in each component
+ of a path. A value of 0 will disable the check.</description>
+</property>
+
+<property>
+ <name>dfs.namenode.fs-limits.max-directory-items</name>
+ <value>0</value>
+ <description>Defines the maximum number of items that a directory may
+ contain. A value of 0 will disable the check.</description>
+</property>
+
+<property>
<name>dfs.namenode.edits.dir</name>
<value>${dfs.namenode.name.dir}</value>
<description>Determines where on the local filesystem the DFS name node
Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/DFSConfigKeys.java?rev=1095789&r1=1095788&r2=1095789&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/DFSConfigKeys.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/DFSConfigKeys.java Thu Apr 21 18:13:46 2011
@@ -120,6 +120,12 @@ public class DFSConfigKeys extends Commo
public static final String DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY = "dfs.namenode.delegation.token.max-lifetime";
public static final long DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT = 7*24*60*60*1000; // 7 days
+ //Filesystem limit keys
+ public static final String DFS_NAMENODE_MAX_COMPONENT_LENGTH_KEY = "dfs.namenode.fs-limits.max-component-length";
+ public static final int DFS_NAMENODE_MAX_COMPONENT_LENGTH_DEFAULT = 0; // no limit
+ public static final String DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY = "dfs.namenode.fs-limits.max-directory-items";
+ public static final int DFS_NAMENODE_MAX_DIRECTORY_ITEMS_DEFAULT = 0; // no limit
+
//Following keys have no defaults
public static final String DFS_DATANODE_DATA_DIR_KEY = "dfs.datanode.data.dir";
public static final String DFS_NAMENODE_HTTPS_ADDRESS_KEY = "dfs.namenode.https-address";
Added: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/protocol/FSLimitException.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/protocol/FSLimitException.java?rev=1095789&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/protocol/FSLimitException.java (added)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/protocol/FSLimitException.java Thu Apr 21 18:13:46 2011
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+/**
+ * Abstract class for deriving exceptions related to filesystem constraints
+ */
+public abstract class FSLimitException extends QuotaExceededException {
+ protected static final long serialVersionUID = 1L;
+
+ protected FSLimitException() {}
+
+ protected FSLimitException(String msg) {
+ super(msg);
+ }
+
+ protected FSLimitException(long quota, long count) {
+ super(quota, count);
+ }
+
+ /**
+ * Path component length is too long
+ */
+ public static final
+ class PathComponentTooLongException extends FSLimitException {
+ protected static final long serialVersionUID = 1L;
+
+ protected PathComponentTooLongException() {}
+
+ protected PathComponentTooLongException(String msg) {
+ super(msg);
+ }
+
+ public PathComponentTooLongException(long quota, long count) {
+ super(quota, count);
+ }
+
+ @Override
+ public String getMessage() {
+ Path violator = new Path(pathName);
+ return "The maximum path component name limit of " + violator.getName() +
+ " in directory " + violator.getParent() +
+ " is exceeded: limit=" + quota + " length=" + count;
+ }
+ }
+
+ /**
+ * Directory has too many items
+ */
+ public static final
+ class MaxDirectoryItemsExceededException extends FSLimitException {
+ protected static final long serialVersionUID = 1L;
+
+ protected MaxDirectoryItemsExceededException() {}
+
+ protected MaxDirectoryItemsExceededException(String msg) {
+ super(msg);
+ }
+
+ public MaxDirectoryItemsExceededException(long quota, long count) {
+ super(quota, count);
+ }
+
+ @Override
+ public String getMessage() {
+ return "The directory item limit of " + pathName +
+ " is exceeded: limit=" + quota + " items=" + count;
+ }
+ }
+}
Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java?rev=1095789&r1=1095788&r2=1095789&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java Thu Apr 21 18:13:46 2011
@@ -42,6 +42,8 @@ import org.apache.hadoop.hdfs.Distribute
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.FSLimitException;
+import org.apache.hadoop.hdfs.protocol.FSLimitException.*;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -68,6 +70,8 @@ class FSDirectory implements Closeable {
FSImage fsImage;
private volatile boolean ready = false;
private static final long UNKNOWN_DISK_SPACE = -1;
+ private final int maxComponentLength;
+ private final int maxDirItems;
private final int lsLimit; // max list limit
// lock to protect BlockMap.
@@ -119,6 +123,14 @@ class FSDirectory implements Closeable {
this.lsLimit = configuredLimit>0 ?
configuredLimit : DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT;
+ // filesystem limits
+ this.maxComponentLength = conf.getInt(
+ DFSConfigKeys.DFS_NAMENODE_MAX_COMPONENT_LENGTH_KEY,
+ DFSConfigKeys.DFS_NAMENODE_MAX_COMPONENT_LENGTH_DEFAULT);
+ this.maxDirItems = conf.getInt(
+ DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY,
+ DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_DEFAULT);
+
int threshold = conf.getInt(
DFSConfigKeys.DFS_NAMENODE_NAME_CACHE_THRESHOLD_KEY,
DFSConfigKeys.DFS_NAMENODE_NAME_CACHE_THRESHOLD_DEFAULT);
@@ -158,7 +170,7 @@ class FSDirectory implements Closeable {
}
writeLock();
try {
- this.ready = true;
+ setReady(true);
this.nameCache.initialized();
cond.signalAll();
} finally {
@@ -166,6 +178,11 @@ class FSDirectory implements Closeable {
}
}
+ // exposed for unit tests
+ protected void setReady(boolean flag) {
+ ready = flag;
+ }
+
private void incrDeletedFileCount(int count) {
if (getFSNamesystem() != null)
NameNode.getNameNodeMetrics().numFilesDeleted.inc(count);
@@ -1613,12 +1630,58 @@ class FSDirectory implements Closeable {
commonAncestor);
}
+ /**
+ * Verify that filesystem limit constraints are not violated
+ * @throws PathComponentTooLongException child's name is too long
+ * @throws MaxDirectoryItemsExceededException items per directory is exceeded
+ */
+ protected <T extends INode> void verifyFsLimits(INode[] pathComponents,
+ int pos, T child) throws FSLimitException {
+ boolean includeChildName = false;
+ try {
+ if (maxComponentLength != 0) {
+ int length = child.getLocalName().length();
+ if (length > maxComponentLength) {
+ includeChildName = true;
+ throw new PathComponentTooLongException(maxComponentLength, length);
+ }
+ }
+ if (maxDirItems != 0) {
+ INodeDirectory parent = (INodeDirectory)pathComponents[pos-1];
+ int count = parent.getChildren().size();
+ if (count >= maxDirItems) {
+ throw new MaxDirectoryItemsExceededException(maxDirItems, count);
+ }
+ }
+ } catch (FSLimitException e) {
+ String badPath = getFullPathName(pathComponents, pos-1);
+ if (includeChildName) {
+ badPath += Path.SEPARATOR + child.getLocalName();
+ }
+ e.setPathName(badPath);
+ // Do not throw if edits log is still being processed
+ if (ready) throw(e);
+ // log pre-existing paths that exceed limits
+ NameNode.LOG.error("FSDirectory.verifyFsLimits - " + e.getLocalizedMessage());
+ }
+ }
+
/** Add a node child to the inodes at index pos.
* Its ancestors are stored at [0, pos-1].
* QuotaExceededException is thrown if it violates quota limit */
private <T extends INode> T addChild(INode[] pathComponents, int pos,
T child, long childDiskspace, boolean inheritPermission,
boolean checkQuota) throws QuotaExceededException {
+ // The filesystem limits are not really quotas, so this check may appear
+ // odd. It's because a rename operation deletes the src, tries to add
+ // to the dest, if that fails, re-adds the src from whence it came.
+ // The rename code disables the quota when it's restoring to the
+ // original location becase a quota violation would cause the the item
+ // to go "poof". The fs limits must be bypassed for the same reason.
+ if (checkQuota) {
+ verifyFsLimits(pathComponents, pos, child);
+ }
+
INode.DirCounts counts = new INode.DirCounts();
child.spaceConsumedInTree(counts);
if (childDiskspace < 0) {
Added: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java?rev=1095789&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java (added)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java Thu Apr 21 18:13:46 2011
@@ -0,0 +1,167 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.FSLimitException;
+import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
+import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFsLimits {
+ static Configuration conf;
+ static INode[] inodes;
+ static FSDirectory fs;
+ static boolean fsIsReady;
+
+ static PermissionStatus perms
+ = new PermissionStatus("admin", "admin", FsPermission.getDefault());
+
+ static INodeDirectoryWithQuota rootInode;
+
+ static private FSNamesystem getMockNamesystem() {
+ FSNamesystem fsn = mock(FSNamesystem.class);
+ when(
+ fsn.createFsOwnerPermissions((FsPermission)anyObject())
+ ).thenReturn(
+ new PermissionStatus("root", "wheel", FsPermission.getDefault())
+ );
+ return fsn;
+ }
+
+ private static class TestFSDirectory extends FSDirectory {
+ public TestFSDirectory() throws IOException {
+ super(new FSImage(), getMockNamesystem(), conf);
+ setReady(fsIsReady);
+ }
+
+ @Override
+ public <T extends INode> void verifyFsLimits(INode[] pathComponents,
+ int pos, T child) throws FSLimitException {
+ super.verifyFsLimits(pathComponents, pos, child);
+ }
+ }
+
+ @Before
+ public void setUp() {
+ conf = new Configuration();
+ rootInode = new INodeDirectoryWithQuota(INodeDirectory.ROOT_NAME, perms, 0L, 0L);
+ inodes = new INode[]{ rootInode, null };
+ fs = null;
+ fsIsReady = true;
+ }
+
+ @Test
+ public void testDefaultMaxComponentLength() {
+ int maxComponentLength = conf.getInt(
+ DFSConfigKeys.DFS_NAMENODE_MAX_COMPONENT_LENGTH_KEY,
+ DFSConfigKeys.DFS_NAMENODE_MAX_COMPONENT_LENGTH_DEFAULT);
+ assertEquals(0, maxComponentLength);
+ }
+
+ @Test
+ public void testDefaultMaxDirItems() {
+ int maxDirItems = conf.getInt(
+ DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY,
+ DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_DEFAULT);
+ assertEquals(0, maxDirItems);
+ }
+
+ @Test
+ public void testNoLimits() throws Exception {
+ addChildWithName("1", null);
+ addChildWithName("22", null);
+ addChildWithName("333", null);
+ addChildWithName("4444", null);
+ addChildWithName("55555", null);
+ }
+
+ @Test
+ public void testMaxComponentLength() throws Exception {
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_COMPONENT_LENGTH_KEY, 2);
+
+ addChildWithName("1", null);
+ addChildWithName("22", null);
+ addChildWithName("333", PathComponentTooLongException.class);
+ addChildWithName("4444", PathComponentTooLongException.class);
+ }
+
+ @Test
+ public void testMaxDirItems() throws Exception {
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY, 2);
+
+ addChildWithName("1", null);
+ addChildWithName("22", null);
+ addChildWithName("333", MaxDirectoryItemsExceededException.class);
+ addChildWithName("4444", MaxDirectoryItemsExceededException.class);
+ }
+
+ @Test
+ public void testMaxComponentsAndMaxDirItems() throws Exception {
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_COMPONENT_LENGTH_KEY, 3);
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY, 2);
+
+ addChildWithName("1", null);
+ addChildWithName("22", null);
+ addChildWithName("333", MaxDirectoryItemsExceededException.class);
+ addChildWithName("4444", PathComponentTooLongException.class);
+ }
+
+ @Test
+ public void testDuringEditLogs() throws Exception {
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_COMPONENT_LENGTH_KEY, 3);
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY, 2);
+ fsIsReady = false;
+
+ addChildWithName("1", null);
+ addChildWithName("22", null);
+ addChildWithName("333", null);
+ addChildWithName("4444", null);
+ }
+
+ private void addChildWithName(String name, Class<?> expected)
+ throws Exception {
+ // have to create after the caller has had a chance to set conf values
+ if (fs == null) fs = new TestFSDirectory();
+
+ INode child = new INodeDirectory(name, perms);
+ child.setLocalName(name);
+
+ Class<?> generated = null;
+ try {
+ fs.verifyFsLimits(inodes, 1, child);
+ rootInode.addChild(child, false, false);
+ } catch (QuotaExceededException e) {
+ generated = e.getClass();
+ }
+ assertEquals(expected, generated);
+ }
+}