Merge branch 'HDFS-6584' into trunk

Conflicts:
	hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
	hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
	hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyConsiderLoad.java
This commit is contained in:
Tsz-Wo Nicholas Sze 2014-09-19 07:11:19 +08:00
parent fd3aa38231
commit 022474c807
94 changed files with 5663 additions and 813 deletions

View File

@ -67,6 +67,19 @@ export HADOOP_SECURE_DN_USER=${HADOOP_SECURE_DN_USER}
# Where log files are stored in the secure data environment. # Where log files are stored in the secure data environment.
export HADOOP_SECURE_DN_LOG_DIR=${HADOOP_LOG_DIR}/${HADOOP_HDFS_USER} export HADOOP_SECURE_DN_LOG_DIR=${HADOOP_LOG_DIR}/${HADOOP_HDFS_USER}
###
# HDFS Mover specific parameters
###
# Specify the JVM options to be used when starting the HDFS Mover.
# These options will be appended to the options specified as HADOOP_OPTS
# and therefore may override any similar flags set in HADOOP_OPTS
#
# export HADOOP_MOVER_OPTS=""
###
# Advanced Users Only!
###
# The directory where pid files are stored. /tmp by default. # The directory where pid files are stored. /tmp by default.
# NOTE: this should be set to a directory that can only be written to by # NOTE: this should be set to a directory that can only be written to by
# the user that will run the hadoop daemons. Otherwise there is the # the user that will run the hadoop daemons. Otherwise there is the

View File

@ -1,5 +1,81 @@
Hadoop HDFS Change Log Hadoop HDFS Change Log
HDFS-6584: Archival Storage
HDFS-6677. Change INodeFile and FSImage to support storage policy ID.
(szetszwo)
HDFS-6670. Add block storage policy support with default HOT, WARM and COLD
policies. (szetszwo)
HDFS-6671. Change BlockPlacementPolicy to consider block storage policy
in replicaiton. (szetszwo)
HDFS-6710. Change BlockPlacementPolicy to consider block storage policy
in replica deletion. (szetszwo)
HDFS-6679. Bump NameNodeLayoutVersion and update editsStored test files.
(vinayakumarb via szetszwo)
HDFS-6686. Change BlockPlacementPolicy to use fallback when some storage
types are unavailable. (szetszwo)
HDFS-6835. Add a new API to set storage policy. (jing9)
HDFS-6847. Support storage policy on directories and include storage policy
in HdfsFileStatus. (Jing Zhao via szetszwo)
HDFS-6801. Add a new data migration tool, Mover, for archiving data.
(szetszwo via jing9)
HDFS-6863. Support migration for snapshot paths. (jing9)
HDFS-6906. Add more tests for BlockStoragePolicy. (szetszwo via jing9)
HDFS-6911. check if a block is already scheduled in Mover.
(szetszwo via jing9)
HDFS-6920. Check the storage type of delNodeHintStorage when deleting
a replica. (szetszwo via jing9)
HDFS-6944. Add retry and termination logic for Mover. (jing9)
HDFS-6969. INode#getStoragePolicyID should always return the latest
storage policy. (jing9)
HDFS-6961. BlockPlacementPolicy#chooseTarget should check each valid
storage type in each choosing round. (jing9)
HDFS-6876. support set/get storage policy in DFSAdmin. (jing9)
HDFS-6997. Add more tests for data migration and replicaion. (szetszwo)
HDFS-6875. Support migration for a list of specified paths. (jing9)
HDFS-7027. Mover does not terminate when some storage type is out of space.
(szetszwo via jing9)
HDFS-7029. Fix TestDFSInotifyEventInputStream and TestDistributedFileSystem.
(szetszwo via jing9)
HDFS-7028. FSDirectory should not get storage policy id from symlinks.
(szetszwo)
HDFS-7034. Fix TestBlockPlacement and TestStorageMover. (jing9)
HDFS-7039. Fix Balancer tests. (szetszwo via jing9)
HDFS-7062. Skip under construction block for migration. (jing9)
HDFS-7052. Add Mover into hdfs script. (jing9)
HDFS-7072. Fix TestBlockManager and TestStorageMover. (jing9 via szetszwo)
HDFS-6864. Archival Storage: add user documentation. (szetszwo via jing9)
HDFS-7088. Archival Storage: fix TestBalancer and
TestBalancerWithMultipleNameNodes. (szetszwo via jing9)
Release 2.6.0 - UNRELEASED Release 2.6.0 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -48,6 +48,8 @@ function print_usage(){
echo " fsck run a DFS filesystem checking utility" echo " fsck run a DFS filesystem checking utility"
echo " balancer run a cluster balancing utility" echo " balancer run a cluster balancing utility"
echo " jmxget get JMX exported values from NameNode or DataNode." echo " jmxget get JMX exported values from NameNode or DataNode."
echo " mover run a utility to move block replicas across"
echo " storage types"
echo " oiv apply the offline fsimage viewer to an fsimage" echo " oiv apply the offline fsimage viewer to an fsimage"
echo " oiv_legacy apply the offline fsimage viewer to an legacy fsimage" echo " oiv_legacy apply the offline fsimage viewer to an legacy fsimage"
echo " oev apply the offline edits viewer to an edits file" echo " oev apply the offline edits viewer to an edits file"
@ -159,6 +161,9 @@ elif [ "$COMMAND" = "fsck" ] ; then
elif [ "$COMMAND" = "balancer" ] ; then elif [ "$COMMAND" = "balancer" ] ; then
CLASS=org.apache.hadoop.hdfs.server.balancer.Balancer CLASS=org.apache.hadoop.hdfs.server.balancer.Balancer
HADOOP_OPTS="$HADOOP_OPTS $HADOOP_BALANCER_OPTS" HADOOP_OPTS="$HADOOP_OPTS $HADOOP_BALANCER_OPTS"
elif [ "$COMMAND" = "mover" ] ; then
CLASS=org.apache.hadoop.hdfs.server.mover.Mover
HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_MOVER_OPTS}"
elif [ "$COMMAND" = "jmxget" ] ; then elif [ "$COMMAND" = "jmxget" ] ; then
CLASS=org.apache.hadoop.hdfs.tools.JMXGet CLASS=org.apache.hadoop.hdfs.tools.JMXGet
elif [ "$COMMAND" = "oiv" ] ; then elif [ "$COMMAND" = "oiv" ] ; then

View File

@ -47,7 +47,7 @@ if "%1" == "--config" (
goto print_usage goto print_usage
) )
set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover
for %%i in ( %hdfscommands% ) do ( for %%i in ( %hdfscommands% ) do (
if %hdfs-command% == %%i set hdfscommand=true if %hdfs-command% == %%i set hdfscommand=true
) )
@ -150,6 +150,11 @@ goto :eof
set CLASS=org.apache.hadoop.hdfs.tools.CacheAdmin set CLASS=org.apache.hadoop.hdfs.tools.CacheAdmin
goto :eof goto :eof
:mover
set CLASS=org.apache.hadoop.hdfs.server.mover.Mover
set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_MOVER_OPTS%
goto :eof
@rem This changes %1, %2 etc. Hence those cannot be used after calling this. @rem This changes %1, %2 etc. Hence those cannot be used after calling this.
:make_command_arguments :make_command_arguments
if "%1" == "--config" ( if "%1" == "--config" (
@ -198,6 +203,7 @@ goto :eof
@echo lsSnapshottableDir list all snapshottable dirs owned by the current user @echo lsSnapshottableDir list all snapshottable dirs owned by the current user
@echo Use -help to see options @echo Use -help to see options
@echo cacheadmin configure the HDFS cache @echo cacheadmin configure the HDFS cache
@echo mover run a utility to move block replicas across storage types
@echo. @echo.
@echo Most commands print help when invoked w/o parameters. @echo Most commands print help when invoked w/o parameters.

View File

@ -0,0 +1,21 @@
<?xml version="1.0" encoding="UTF-8"?>
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<!--
Licensed 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. See accompanying LICENSE file.
-->
<!-- Put site-specific property overrides in this file. -->
<configuration>
</configuration>

View File

@ -16,6 +16,7 @@
<!-- Put site-specific property overrides in this file. --> <!-- Put site-specific property overrides in this file. -->
<configuration> <configuration xmlns:xi="http://www.w3.org/2001/XInclude">
<xi:include href="blockStoragePolicy-site.xml" />
</configuration> </configuration>

View File

@ -0,0 +1,419 @@
/**
* 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;
import java.util.Arrays;
import java.util.EnumSet;
import java.util.LinkedList;
import java.util.List;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.XAttr;
import org.apache.hadoop.fs.XAttr.NameSpace;
/**
* A block storage policy describes how to select the storage types
* for the replicas of a block.
*/
@InterfaceAudience.Private
public class BlockStoragePolicy {
public static final Log LOG = LogFactory.getLog(BlockStoragePolicy.class);
public static final String DFS_BLOCK_STORAGE_POLICIES_KEY
= "dfs.block.storage.policies";
public static final String DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX
= "dfs.block.storage.policy.";
public static final String DFS_BLOCK_STORAGE_POLICY_CREATION_FALLBACK_KEY_PREFIX
= "dfs.block.storage.policy.creation-fallback.";
public static final String DFS_BLOCK_STORAGE_POLICY_REPLICATION_FALLBACK_KEY_PREFIX
= "dfs.block.storage.policy.replication-fallback.";
public static final String STORAGE_POLICY_XATTR_NAME = "bsp";
/** set the namespace to TRUSTED so that only privilege users can access */
public static final NameSpace XAttrNS = NameSpace.TRUSTED;
public static final int ID_BIT_LENGTH = 4;
public static final int ID_MAX = (1 << ID_BIT_LENGTH) - 1;
public static final byte ID_UNSPECIFIED = 0;
private static final Suite DEFAULT_SUITE = createDefaultSuite();
private static Suite createDefaultSuite() {
final BlockStoragePolicy[] policies = new BlockStoragePolicy[1 << ID_BIT_LENGTH];
final StorageType[] storageTypes = {StorageType.DISK};
final byte defaultPolicyId = 12;
policies[defaultPolicyId] = new BlockStoragePolicy(defaultPolicyId, "HOT",
storageTypes, StorageType.EMPTY_ARRAY, StorageType.EMPTY_ARRAY);
return new Suite(defaultPolicyId, policies);
}
/** A block storage policy suite. */
public static class Suite {
private final byte defaultPolicyID;
private final BlockStoragePolicy[] policies;
private Suite(byte defaultPolicyID, BlockStoragePolicy[] policies) {
this.defaultPolicyID = defaultPolicyID;
this.policies = policies;
}
/** @return the corresponding policy. */
public BlockStoragePolicy getPolicy(byte id) {
// id == 0 means policy not specified.
return id == 0? getDefaultPolicy(): policies[id];
}
/** @return the default policy. */
public BlockStoragePolicy getDefaultPolicy() {
return getPolicy(defaultPolicyID);
}
public BlockStoragePolicy getPolicy(String policyName) {
if (policies != null) {
for (BlockStoragePolicy policy : policies) {
if (policy != null && policy.name.equals(policyName)) {
return policy;
}
}
}
return null;
}
}
/** A 4-bit policy ID */
private final byte id;
/** Policy name */
private final String name;
/** The storage types to store the replicas of a new block. */
private final StorageType[] storageTypes;
/** The fallback storage type for block creation. */
private final StorageType[] creationFallbacks;
/** The fallback storage type for replication. */
private final StorageType[] replicationFallbacks;
@VisibleForTesting
public BlockStoragePolicy(byte id, String name, StorageType[] storageTypes,
StorageType[] creationFallbacks, StorageType[] replicationFallbacks) {
this.id = id;
this.name = name;
this.storageTypes = storageTypes;
this.creationFallbacks = creationFallbacks;
this.replicationFallbacks = replicationFallbacks;
}
/**
* @return a list of {@link StorageType}s for storing the replicas of a block.
*/
public List<StorageType> chooseStorageTypes(final short replication) {
final List<StorageType> types = new LinkedList<StorageType>();
int i = 0;
for(; i < replication && i < storageTypes.length; i++) {
types.add(storageTypes[i]);
}
final StorageType last = storageTypes[storageTypes.length - 1];
for(; i < replication; i++) {
types.add(last);
}
return types;
}
/**
* Choose the storage types for storing the remaining replicas, given the
* replication number and the storage types of the chosen replicas.
*
* @param replication the replication number.
* @param chosen the storage types of the chosen replicas.
* @return a list of {@link StorageType}s for storing the replicas of a block.
*/
public List<StorageType> chooseStorageTypes(final short replication,
final Iterable<StorageType> chosen) {
return chooseStorageTypes(replication, chosen, null);
}
private List<StorageType> chooseStorageTypes(final short replication,
final Iterable<StorageType> chosen, final List<StorageType> excess) {
final List<StorageType> types = chooseStorageTypes(replication);
diff(types, chosen, excess);
return types;
}
/**
* Choose the storage types for storing the remaining replicas, given the
* replication number, the storage types of the chosen replicas and
* the unavailable storage types. It uses fallback storage in case that
* the desired storage type is unavailable.
*
* @param replication the replication number.
* @param chosen the storage types of the chosen replicas.
* @param unavailables the unavailable storage types.
* @param isNewBlock Is it for new block creation?
* @return a list of {@link StorageType}s for storing the replicas of a block.
*/
public List<StorageType> chooseStorageTypes(final short replication,
final Iterable<StorageType> chosen,
final EnumSet<StorageType> unavailables,
final boolean isNewBlock) {
final List<StorageType> excess = new LinkedList<StorageType>();
final List<StorageType> storageTypes = chooseStorageTypes(
replication, chosen, excess);
final int expectedSize = storageTypes.size() - excess.size();
final List<StorageType> removed = new LinkedList<StorageType>();
for(int i = storageTypes.size() - 1; i >= 0; i--) {
// replace/remove unavailable storage types.
final StorageType t = storageTypes.get(i);
if (unavailables.contains(t)) {
final StorageType fallback = isNewBlock?
getCreationFallback(unavailables)
: getReplicationFallback(unavailables);
if (fallback == null) {
removed.add(storageTypes.remove(i));
} else {
storageTypes.set(i, fallback);
}
}
}
// remove excess storage types after fallback replacement.
diff(storageTypes, excess, null);
if (storageTypes.size() < expectedSize) {
LOG.warn("Failed to place enough replicas: expected size is " + expectedSize
+ " but only " + storageTypes.size() + " storage types can be selected "
+ "(replication=" + replication
+ ", selected=" + storageTypes
+ ", unavailable=" + unavailables
+ ", removed=" + removed
+ ", policy=" + this + ")");
}
return storageTypes;
}
/**
* Compute the list difference t = t - c.
* Further, if e is not null, set e = e + c - t;
*/
private static void diff(List<StorageType> t, Iterable<StorageType> c,
List<StorageType> e) {
for(StorageType storagetype : c) {
final int i = t.indexOf(storagetype);
if (i >= 0) {
t.remove(i);
} else if (e != null) {
e.add(storagetype);
}
}
}
/**
* Choose excess storage types for deletion, given the
* replication number and the storage types of the chosen replicas.
*
* @param replication the replication number.
* @param chosen the storage types of the chosen replicas.
* @return a list of {@link StorageType}s for deletion.
*/
public List<StorageType> chooseExcess(final short replication,
final Iterable<StorageType> chosen) {
final List<StorageType> types = chooseStorageTypes(replication);
final List<StorageType> excess = new LinkedList<StorageType>();
diff(types, chosen, excess);
return excess;
}
/** @return the fallback {@link StorageType} for creation. */
public StorageType getCreationFallback(EnumSet<StorageType> unavailables) {
return getFallback(unavailables, creationFallbacks);
}
/** @return the fallback {@link StorageType} for replication. */
public StorageType getReplicationFallback(EnumSet<StorageType> unavailables) {
return getFallback(unavailables, replicationFallbacks);
}
@Override
public int hashCode() {
return Byte.valueOf(id).hashCode();
}
@Override
public boolean equals(Object obj) {
if (obj == this) {
return true;
} else if (obj == null || !(obj instanceof BlockStoragePolicy)) {
return false;
}
final BlockStoragePolicy that = (BlockStoragePolicy)obj;
return this.id == that.id;
}
@Override
public String toString() {
return getClass().getSimpleName() + "{" + name + ":" + id
+ ", storageTypes=" + Arrays.asList(storageTypes)
+ ", creationFallbacks=" + Arrays.asList(creationFallbacks)
+ ", replicationFallbacks=" + Arrays.asList(replicationFallbacks);
}
public byte getId() {
return id;
}
public String getName() {
return name;
}
private static StorageType getFallback(EnumSet<StorageType> unavailables,
StorageType[] fallbacks) {
for(StorageType fb : fallbacks) {
if (!unavailables.contains(fb)) {
return fb;
}
}
return null;
}
private static byte parseID(String idString, String element, Configuration conf) {
byte id = 0;
try {
id = Byte.parseByte(idString);
} catch(NumberFormatException nfe) {
throwIllegalArgumentException("Failed to parse policy ID \"" + idString
+ "\" to a " + ID_BIT_LENGTH + "-bit integer", conf);
}
if (id < 0) {
throwIllegalArgumentException("Invalid policy ID: id = " + id
+ " < 1 in \"" + element + "\"", conf);
} else if (id == 0) {
throw new IllegalArgumentException("Policy ID 0 is reserved: " + element);
} else if (id > ID_MAX) {
throwIllegalArgumentException("Invalid policy ID: id = " + id
+ " > MAX = " + ID_MAX + " in \"" + element + "\"", conf);
}
return id;
}
private static StorageType[] parseStorageTypes(String[] strings) {
if (strings == null || strings.length == 0) {
return StorageType.EMPTY_ARRAY;
}
final StorageType[] types = new StorageType[strings.length];
for(int i = 0; i < types.length; i++) {
types[i] = StorageType.valueOf(strings[i].trim().toUpperCase());
}
return types;
}
private static StorageType[] readStorageTypes(byte id, String keyPrefix,
Configuration conf) {
final String key = keyPrefix + id;
final String[] values = conf.getStrings(key);
try {
return parseStorageTypes(values);
} catch(Exception e) {
throw new IllegalArgumentException("Failed to parse " + key
+ " \"" + conf.get(key), e);
}
}
private static BlockStoragePolicy readBlockStoragePolicy(byte id, String name,
Configuration conf) {
final StorageType[] storageTypes = readStorageTypes(id,
DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX, conf);
if (storageTypes.length == 0) {
throw new IllegalArgumentException(
DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX + id + " is missing or is empty.");
}
final StorageType[] creationFallbacks = readStorageTypes(id,
DFS_BLOCK_STORAGE_POLICY_CREATION_FALLBACK_KEY_PREFIX, conf);
final StorageType[] replicationFallbacks = readStorageTypes(id,
DFS_BLOCK_STORAGE_POLICY_REPLICATION_FALLBACK_KEY_PREFIX, conf);
return new BlockStoragePolicy(id, name, storageTypes, creationFallbacks,
replicationFallbacks);
}
/** Read {@link Suite} from conf. */
public static Suite readBlockStorageSuite(Configuration conf) {
final BlockStoragePolicy[] policies = new BlockStoragePolicy[1 << ID_BIT_LENGTH];
final String[] values = conf.getStrings(DFS_BLOCK_STORAGE_POLICIES_KEY);
if (values == null) {
// conf property is missing, use default suite.
return DEFAULT_SUITE;
}
byte firstID = -1;
for(String v : values) {
v = v.trim();
final int i = v.indexOf(':');
if (i < 0) {
throwIllegalArgumentException("Failed to parse element \"" + v
+ "\" (expected format is NAME:ID)", conf);
} else if (i == 0) {
throwIllegalArgumentException("Policy name is missing in \"" + v + "\"", conf);
} else if (i == v.length() - 1) {
throwIllegalArgumentException("Policy ID is missing in \"" + v + "\"", conf);
}
final String name = v.substring(0, i).trim();
for(int j = 1; j < policies.length; j++) {
if (policies[j] != null && policies[j].name.equals(name)) {
throwIllegalArgumentException("Policy name duplication: \""
+ name + "\" appears more than once", conf);
}
}
final byte id = parseID(v.substring(i + 1).trim(), v, conf);
if (policies[id] != null) {
throwIllegalArgumentException("Policy duplication: ID " + id
+ " appears more than once", conf);
}
policies[id] = readBlockStoragePolicy(id, name, conf);
String prefix = "";
if (firstID == -1) {
firstID = id;
prefix = "(default) ";
}
LOG.info(prefix + policies[id]);
}
if (firstID == -1) {
throwIllegalArgumentException("Empty list is not allowed", conf);
}
return new Suite(firstID, policies);
}
public static String buildXAttrName() {
return XAttrNS.toString().toLowerCase() + "." + STORAGE_POLICY_XATTR_NAME;
}
public static XAttr buildXAttr(byte policyId) {
final String name = buildXAttrName();
return XAttrHelper.buildXAttr(name, new byte[] { policyId });
}
public static boolean isStoragePolicyXAttr(XAttr xattr) {
return xattr != null && xattr.getNameSpace() == BlockStoragePolicy.XAttrNS
&& xattr.getName().equals(BlockStoragePolicy.STORAGE_POLICY_XATTR_NAME);
}
private static void throwIllegalArgumentException(String message,
Configuration conf) {
throw new IllegalArgumentException(message + " in "
+ DFS_BLOCK_STORAGE_POLICIES_KEY + " \""
+ conf.get(DFS_BLOCK_STORAGE_POLICIES_KEY) + "\".");
}
}

View File

@ -1760,6 +1760,25 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
} }
} }
/**
* Set storage policy for an existing file/directory
* @param src file/directory name
* @param policyName name of the storage policy
*/
public void setStoragePolicy(String src, String policyName)
throws IOException {
try {
namenode.setStoragePolicy(src, policyName);
} catch (RemoteException e) {
throw e.unwrapRemoteException(AccessControlException.class,
FileNotFoundException.class,
SafeModeException.class,
NSQuotaExceededException.class,
UnresolvedPathException.class,
SnapshotAccessControlException.class);
}
}
/** /**
* Rename file or directory. * Rename file or directory.
* @see ClientProtocol#rename(String, String) * @see ClientProtocol#rename(String, String)

View File

@ -371,6 +371,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final int DFS_BALANCER_MOVERTHREADS_DEFAULT = 1000; public static final int DFS_BALANCER_MOVERTHREADS_DEFAULT = 1000;
public static final String DFS_BALANCER_DISPATCHERTHREADS_KEY = "dfs.balancer.dispatcherThreads"; public static final String DFS_BALANCER_DISPATCHERTHREADS_KEY = "dfs.balancer.dispatcherThreads";
public static final int DFS_BALANCER_DISPATCHERTHREADS_DEFAULT = 200; public static final int DFS_BALANCER_DISPATCHERTHREADS_DEFAULT = 200;
public static final String DFS_MOVER_MOVEDWINWIDTH_KEY = "dfs.mover.movedWinWidth";
public static final long DFS_MOVER_MOVEDWINWIDTH_DEFAULT = 5400*1000L;
public static final String DFS_MOVER_MOVERTHREADS_KEY = "dfs.mover.moverThreads";
public static final int DFS_MOVER_MOVERTHREADS_DEFAULT = 1000;
public static final String DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address"; public static final String DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
public static final int DFS_DATANODE_DEFAULT_PORT = 50010; public static final int DFS_DATANODE_DEFAULT_PORT = 50010;
public static final String DFS_DATANODE_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_DATANODE_DEFAULT_PORT; public static final String DFS_DATANODE_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_DATANODE_DEFAULT_PORT;
@ -438,6 +444,15 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final Class<BlockPlacementPolicyDefault> DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT = BlockPlacementPolicyDefault.class; public static final Class<BlockPlacementPolicyDefault> DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT = BlockPlacementPolicyDefault.class;
public static final String DFS_REPLICATION_MAX_KEY = "dfs.replication.max"; public static final String DFS_REPLICATION_MAX_KEY = "dfs.replication.max";
public static final int DFS_REPLICATION_MAX_DEFAULT = 512; public static final int DFS_REPLICATION_MAX_DEFAULT = 512;
public static final String DFS_BLOCK_STORAGE_POLICIES_KEY
= BlockStoragePolicy.DFS_BLOCK_STORAGE_POLICIES_KEY;
public static final String DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX
= BlockStoragePolicy.DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX;
public static final String DFS_BLOCK_STORAGE_POLICY_CREATION_FALLBACK_KEY_PREFIX
= BlockStoragePolicy.DFS_BLOCK_STORAGE_POLICY_CREATION_FALLBACK_KEY_PREFIX;
public static final String DFS_BLOCK_STORAGE_POLICY_REPLICATION_FALLBACK_KEY_PREFIX
= BlockStoragePolicy.DFS_BLOCK_STORAGE_POLICY_REPLICATION_FALLBACK_KEY_PREFIX;
public static final String DFS_DF_INTERVAL_KEY = "dfs.df.interval"; public static final String DFS_DF_INTERVAL_KEY = "dfs.df.interval";
public static final int DFS_DF_INTERVAL_DEFAULT = 60000; public static final int DFS_DF_INTERVAL_DEFAULT = 60000;
public static final String DFS_BLOCKREPORT_INTERVAL_MSEC_KEY = "dfs.blockreport.intervalMsec"; public static final String DFS_BLOCKREPORT_INTERVAL_MSEC_KEY = "dfs.blockreport.intervalMsec";

View File

@ -472,6 +472,38 @@ public class DistributedFileSystem extends FileSystem {
}.resolve(this, absF); }.resolve(this, absF);
} }
/**
* Set the source path to the specified storage policy.
*
* @param src The source path referring to either a directory or a file.
* @param policyName The name of the storage policy.
*/
public void setStoragePolicy(final Path src, final String policyName)
throws IOException {
statistics.incrementWriteOps(1);
Path absF = fixRelativePart(src);
new FileSystemLinkResolver<Void>() {
@Override
public Void doCall(final Path p)
throws IOException, UnresolvedLinkException {
dfs.setStoragePolicy(getPathName(p), policyName);
return null;
}
@Override
public Void next(final FileSystem fs, final Path p)
throws IOException {
if (fs instanceof DistributedFileSystem) {
((DistributedFileSystem) fs).setStoragePolicy(p, policyName);
return null;
} else {
throw new UnsupportedOperationException(
"Cannot perform setStoragePolicy on a non-DistributedFileSystem: "
+ src + " -> " + p);
}
}
}.resolve(this, absF);
}
/** /**
* Move blocks from srcs to trg and delete srcs afterwards. * Move blocks from srcs to trg and delete srcs afterwards.
* The file block sizes must be the same. * The file block sizes must be the same.

View File

@ -32,9 +32,11 @@ import org.apache.hadoop.classification.InterfaceStability;
@InterfaceStability.Unstable @InterfaceStability.Unstable
public enum StorageType { public enum StorageType {
DISK, DISK,
SSD; SSD,
ARCHIVE;
public static final StorageType DEFAULT = DISK; public static final StorageType DEFAULT = DISK;
public static final StorageType[] EMPTY_ARRAY = {}; public static final StorageType[] EMPTY_ARRAY = {};
private static final StorageType[] VALUES = values(); private static final StorageType[] VALUES = values();

View File

@ -259,6 +259,20 @@ public interface ClientProtocol {
FileNotFoundException, SafeModeException, UnresolvedLinkException, FileNotFoundException, SafeModeException, UnresolvedLinkException,
SnapshotAccessControlException, IOException; SnapshotAccessControlException, IOException;
/**
* Set the storage policy for a file/directory
* @param src Path of an existing file/directory.
* @param policyName The name of the storage policy
* @throws SnapshotAccessControlException If access is denied
* @throws UnresolvedLinkException if <code>src</code> contains a symlink
* @throws FileNotFoundException If file/dir <code>src</code> is not found
* @throws QuotaExceededException If changes violate the quota restriction
*/
@Idempotent
public void setStoragePolicy(String src, String policyName)
throws SnapshotAccessControlException, UnresolvedLinkException,
FileNotFoundException, QuotaExceededException, IOException;
/** /**
* Set permissions for an existing file/directory. * Set permissions for an existing file/directory.
* *

View File

@ -162,4 +162,7 @@ public class HdfsConstants {
public static final String SEPARATOR_DOT_SNAPSHOT_DIR public static final String SEPARATOR_DOT_SNAPSHOT_DIR
= Path.SEPARATOR + DOT_SNAPSHOT_DIR; = Path.SEPARATOR + DOT_SNAPSHOT_DIR;
public static final String SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR
= Path.SEPARATOR + DOT_SNAPSHOT_DIR + Path.SEPARATOR;
} }

View File

@ -50,6 +50,7 @@ public class HdfsFileStatus {
// Used by dir, not including dot and dotdot. Always zero for a regular file. // Used by dir, not including dot and dotdot. Always zero for a regular file.
private final int childrenNum; private final int childrenNum;
private final byte storagePolicy;
public static final byte[] EMPTY_NAME = new byte[0]; public static final byte[] EMPTY_NAME = new byte[0];
@ -71,7 +72,8 @@ public class HdfsFileStatus {
public HdfsFileStatus(long length, boolean isdir, int block_replication, public HdfsFileStatus(long length, boolean isdir, int block_replication,
long blocksize, long modification_time, long access_time, long blocksize, long modification_time, long access_time,
FsPermission permission, String owner, String group, byte[] symlink, FsPermission permission, String owner, String group, byte[] symlink,
byte[] path, long fileId, int childrenNum, FileEncryptionInfo feInfo) { byte[] path, long fileId, int childrenNum, FileEncryptionInfo feInfo,
byte storagePolicy) {
this.length = length; this.length = length;
this.isdir = isdir; this.isdir = isdir;
this.block_replication = (short)block_replication; this.block_replication = (short)block_replication;
@ -90,6 +92,7 @@ public class HdfsFileStatus {
this.fileId = fileId; this.fileId = fileId;
this.childrenNum = childrenNum; this.childrenNum = childrenNum;
this.feInfo = feInfo; this.feInfo = feInfo;
this.storagePolicy = storagePolicy;
} }
/** /**
@ -251,6 +254,11 @@ public class HdfsFileStatus {
return childrenNum; return childrenNum;
} }
/** @return the storage policy id */
public final byte getStoragePolicy() {
return storagePolicy;
}
public final FileStatus makeQualified(URI defaultUri, Path path) { public final FileStatus makeQualified(URI defaultUri, Path path) {
return new FileStatus(getLen(), isDir(), getReplication(), return new FileStatus(getLen(), isDir(), getReplication(),
getBlockSize(), getModificationTime(), getBlockSize(), getModificationTime(),

View File

@ -58,10 +58,10 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
int block_replication, long blocksize, long modification_time, int block_replication, long blocksize, long modification_time,
long access_time, FsPermission permission, String owner, String group, long access_time, FsPermission permission, String owner, String group,
byte[] symlink, byte[] path, long fileId, LocatedBlocks locations, byte[] symlink, byte[] path, long fileId, LocatedBlocks locations,
int childrenNum, FileEncryptionInfo feInfo) { int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy) {
super(length, isdir, block_replication, blocksize, modification_time, super(length, isdir, block_replication, blocksize, modification_time,
access_time, permission, owner, group, symlink, path, fileId, access_time, permission, owner, group, symlink, path, fileId,
childrenNum, feInfo); childrenNum, feInfo, storagePolicy);
this.locations = locations; this.locations = locations;
} }

View File

@ -24,6 +24,7 @@ import java.util.Date;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
/** /**
@ -61,7 +62,7 @@ public class SnapshottableDirectoryStatus {
int snapshotNumber, int snapshotQuota, byte[] parentFullPath) { int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time, this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
access_time, permission, owner, group, null, localName, inodeId, access_time, permission, owner, group, null, localName, inodeId,
childrenNum, null); childrenNum, null, BlockStoragePolicy.ID_UNSPECIFIED);
this.snapshotNumber = snapshotNumber; this.snapshotNumber = snapshotNumber;
this.snapshotQuota = snapshotQuota; this.snapshotQuota = snapshotQuota;
this.parentFullPath = parentFullPath; this.parentFullPath = parentFullPath;

View File

@ -173,6 +173,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetRep
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
@ -236,6 +238,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
static final GetSnapshottableDirListingResponseProto static final GetSnapshottableDirListingResponseProto
NULL_GET_SNAPSHOTTABLE_DIR_LISTING_RESPONSE = NULL_GET_SNAPSHOTTABLE_DIR_LISTING_RESPONSE =
GetSnapshottableDirListingResponseProto.newBuilder().build(); GetSnapshottableDirListingResponseProto.newBuilder().build();
static final SetStoragePolicyResponseProto VOID_SET_STORAGE_POLICY_RESPONSE =
SetStoragePolicyResponseProto.newBuilder().build();
private static final CreateResponseProto VOID_CREATE_RESPONSE = private static final CreateResponseProto VOID_CREATE_RESPONSE =
CreateResponseProto.newBuilder().build(); CreateResponseProto.newBuilder().build();
@ -1413,6 +1417,18 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
return VOID_CHECKACCESS_RESPONSE; return VOID_CHECKACCESS_RESPONSE;
} }
@Override
public SetStoragePolicyResponseProto setStoragePolicy(
RpcController controller, SetStoragePolicyRequestProto request)
throws ServiceException {
try {
server.setStoragePolicy(request.getSrc(), request.getPolicyName());
} catch (IOException e) {
throw new ServiceException(e);
}
return VOID_SET_STORAGE_POLICY_RESPONSE;
}
public GetCurrentEditLogTxidResponseProto getCurrentEditLogTxid(RpcController controller, public GetCurrentEditLogTxidResponseProto getCurrentEditLogTxid(RpcController controller,
GetCurrentEditLogTxidRequestProto req) throws ServiceException { GetCurrentEditLogTxidRequestProto req) throws ServiceException {
try { try {

View File

@ -64,7 +64,9 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException; import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo; import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto; import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
@ -152,6 +154,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTim
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
@ -1435,6 +1438,19 @@ public class ClientNamenodeProtocolTranslatorPB implements
} }
} }
@Override
public void setStoragePolicy(String src, String policyName)
throws SnapshotAccessControlException, UnresolvedLinkException,
FileNotFoundException, QuotaExceededException, IOException {
SetStoragePolicyRequestProto req = SetStoragePolicyRequestProto
.newBuilder().setSrc(src).setPolicyName(policyName).build();
try {
rpcProxy.setStoragePolicy(null, req);
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
public long getCurrentEditLogTxid() throws IOException { public long getCurrentEditLogTxid() throws IOException {
GetCurrentEditLogTxidRequestProto req = GetCurrentEditLogTxidRequestProto GetCurrentEditLogTxidRequestProto req = GetCurrentEditLogTxidRequestProto
.getDefaultInstance(); .getDefaultInstance();

View File

@ -43,6 +43,8 @@ import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.inotify.Event; import org.apache.hadoop.hdfs.inotify.Event;
@ -175,6 +177,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
import org.apache.hadoop.hdfs.server.namenode.INodeId; import org.apache.hadoop.hdfs.server.namenode.INodeId;
@ -892,9 +895,25 @@ public class PBHelper {
} }
builder.addAllTargets(convert(cmd.getTargets())) builder.addAllTargets(convert(cmd.getTargets()))
.addAllTargetStorageUuids(convert(cmd.getTargetStorageIDs())); .addAllTargetStorageUuids(convert(cmd.getTargetStorageIDs()));
StorageType[][] types = cmd.getTargetStorageTypes();
if (types != null) {
builder.addAllTargetStorageTypes(convert(types));
}
return builder.build(); return builder.build();
} }
private static List<StorageTypesProto> convert(StorageType[][] types) {
List<StorageTypesProto> list = Lists.newArrayList();
if (types != null) {
for (StorageType[] ts : types) {
StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
builder.addAllStorageTypes(convertStorageTypes(ts));
list.add(builder.build());
}
}
return list;
}
public static BlockIdCommandProto convert(BlockIdCommand cmd) { public static BlockIdCommandProto convert(BlockIdCommand cmd) {
BlockIdCommandProto.Builder builder = BlockIdCommandProto.newBuilder() BlockIdCommandProto.Builder builder = BlockIdCommandProto.newBuilder()
.setBlockPoolId(cmd.getBlockPoolId()); .setBlockPoolId(cmd.getBlockPoolId());
@ -1023,7 +1042,7 @@ public class PBHelper {
} else { } else {
for(int i = 0; i < targetStorageTypes.length; i++) { for(int i = 0; i < targetStorageTypes.length; i++) {
List<StorageTypeProto> p = targetStorageTypesList.get(i).getStorageTypesList(); List<StorageTypeProto> p = targetStorageTypesList.get(i).getStorageTypesList();
targetStorageTypes[i] = p.toArray(new StorageType[p.size()]); targetStorageTypes[i] = convertStorageTypes(p, targets[i].length);
} }
} }
@ -1329,8 +1348,9 @@ public class PBHelper {
fs.hasFileId()? fs.getFileId(): INodeId.GRANDFATHER_INODE_ID, fs.hasFileId()? fs.getFileId(): INodeId.GRANDFATHER_INODE_ID,
fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null, fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null,
fs.hasChildrenNum() ? fs.getChildrenNum() : -1, fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
null); fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
: BlockStoragePolicy.ID_UNSPECIFIED);
} }
public static SnapshottableDirectoryStatus convert( public static SnapshottableDirectoryStatus convert(
@ -1376,7 +1396,8 @@ public class PBHelper {
setGroup(fs.getGroup()). setGroup(fs.getGroup()).
setFileId(fs.getFileId()). setFileId(fs.getFileId()).
setChildrenNum(fs.getChildrenNum()). setChildrenNum(fs.getChildrenNum()).
setPath(ByteString.copyFrom(fs.getLocalNameInBytes())); setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
setStoragePolicy(fs.getStoragePolicy());
if (fs.isSymlink()) { if (fs.isSymlink()) {
builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes())); builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
} }
@ -1384,7 +1405,8 @@ public class PBHelper {
builder.setFileEncryptionInfo(convert(fs.getFileEncryptionInfo())); builder.setFileEncryptionInfo(convert(fs.getFileEncryptionInfo()));
} }
if (fs instanceof HdfsLocatedFileStatus) { if (fs instanceof HdfsLocatedFileStatus) {
LocatedBlocks locations = ((HdfsLocatedFileStatus)fs).getBlockLocations(); final HdfsLocatedFileStatus lfs = (HdfsLocatedFileStatus) fs;
LocatedBlocks locations = lfs.getBlockLocations();
if (locations != null) { if (locations != null) {
builder.setLocations(PBHelper.convert(locations)); builder.setLocations(PBHelper.convert(locations));
} }
@ -1697,6 +1719,8 @@ public class PBHelper {
return StorageTypeProto.DISK; return StorageTypeProto.DISK;
case SSD: case SSD:
return StorageTypeProto.SSD; return StorageTypeProto.SSD;
case ARCHIVE:
return StorageTypeProto.ARCHIVE;
default: default:
throw new IllegalStateException( throw new IllegalStateException(
"BUG: StorageType not found, type=" + type); "BUG: StorageType not found, type=" + type);
@ -1725,6 +1749,8 @@ public class PBHelper {
return StorageType.DISK; return StorageType.DISK;
case SSD: case SSD:
return StorageType.SSD; return StorageType.SSD;
case ARCHIVE:
return StorageType.ARCHIVE;
default: default:
throw new IllegalStateException( throw new IllegalStateException(
"BUG: StorageTypeProto not found, type=" + type); "BUG: StorageTypeProto not found, type=" + type);

View File

@ -23,7 +23,6 @@ import java.io.IOException;
import java.io.PrintStream; import java.io.PrintStream;
import java.net.URI; import java.net.URI;
import java.text.DateFormat; import java.text.DateFormat;
import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
@ -54,6 +53,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault
import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.Tool;
@ -271,7 +271,7 @@ public class Balancer {
// over-utilized, above-average, below-average and under-utilized. // over-utilized, above-average, below-average and under-utilized.
long overLoadedBytes = 0L, underLoadedBytes = 0L; long overLoadedBytes = 0L, underLoadedBytes = 0L;
for(DatanodeStorageReport r : reports) { for(DatanodeStorageReport r : reports) {
final DDatanode dn = dispatcher.newDatanode(r); final DDatanode dn = dispatcher.newDatanode(r.getDatanodeInfo());
for(StorageType t : StorageType.asList()) { for(StorageType t : StorageType.asList()) {
final Double utilization = policy.getUtilization(r, t); final Double utilization = policy.getUtilization(r, t);
if (utilization == null) { // datanode does not have such storage type if (utilization == null) { // datanode does not have such storage type
@ -295,7 +295,7 @@ public class Balancer {
} }
g = s; g = s;
} else { } else {
g = dn.addStorageGroup(t, maxSize2Move); g = dn.addTarget(t, maxSize2Move);
if (thresholdDiff <= 0) { // within threshold if (thresholdDiff <= 0) { // within threshold
belowAvgUtilized.add(g); belowAvgUtilized.add(g);
} else { } else {
@ -549,15 +549,10 @@ public class Balancer {
final Formatter formatter = new Formatter(System.out); final Formatter formatter = new Formatter(System.out);
System.out.println("Time Stamp Iteration# Bytes Already Moved Bytes Left To Move Bytes Being Moved"); System.out.println("Time Stamp Iteration# Bytes Already Moved Bytes Left To Move Bytes Being Moved");
final List<NameNodeConnector> connectors List<NameNodeConnector> connectors = Collections.emptyList();
= new ArrayList<NameNodeConnector>(namenodes.size());
try { try {
for (URI uri : namenodes) { connectors = NameNodeConnector.newNameNodeConnectors(namenodes,
final NameNodeConnector nnc = new NameNodeConnector( Balancer.class.getSimpleName(), BALANCER_ID_PATH, conf);
Balancer.class.getSimpleName(), uri, BALANCER_ID_PATH, conf);
nnc.getKeyManager().startBlockKeyUpdater();
connectors.add(nnc);
}
boolean done = false; boolean done = false;
for(int iteration = 0; !done; iteration++) { for(int iteration = 0; !done; iteration++) {
@ -582,7 +577,7 @@ public class Balancer {
} }
} finally { } finally {
for(NameNodeConnector nnc : connectors) { for(NameNodeConnector nnc : connectors) {
nnc.close(); IOUtils.cleanup(LOG, nnc);
} }
} }
return ExitStatus.SUCCESS.getExitCode(); return ExitStatus.SUCCESS.getExitCode();

View File

@ -43,11 +43,15 @@ import java.util.concurrent.Executors;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -85,7 +89,11 @@ public class Dispatcher {
private static final long MAX_BLOCKS_SIZE_TO_FETCH = 2 * GB; private static final long MAX_BLOCKS_SIZE_TO_FETCH = 2 * GB;
private static final int MAX_NO_PENDING_MOVE_ITERATIONS = 5; private static final int MAX_NO_PENDING_MOVE_ITERATIONS = 5;
private static final long DELAY_AFTER_ERROR = 10 * 1000L; // 10 seconds /**
* the period of time to delay the usage of a DataNode after hitting
* errors when using it for migrating data
*/
private static long delayAfterErrors = 10 * 1000;
private final NameNodeConnector nnc; private final NameNodeConnector nnc;
private final SaslDataTransferClient saslClient; private final SaslDataTransferClient saslClient;
@ -102,12 +110,14 @@ public class Dispatcher {
private final MovedBlocks<StorageGroup> movedBlocks; private final MovedBlocks<StorageGroup> movedBlocks;
/** Map (datanodeUuid,storageType -> StorageGroup) */ /** Map (datanodeUuid,storageType -> StorageGroup) */
private final StorageGroupMap storageGroupMap = new StorageGroupMap(); private final StorageGroupMap<StorageGroup> storageGroupMap
= new StorageGroupMap<StorageGroup>();
private NetworkTopology cluster; private NetworkTopology cluster;
private final ExecutorService moveExecutor; private final ExecutorService moveExecutor;
private final ExecutorService dispatchExecutor; private final ExecutorService dispatchExecutor;
/** The maximum number of concurrent blocks moves at a datanode */ /** The maximum number of concurrent blocks moves at a datanode */
private final int maxConcurrentMovesPerNode; private final int maxConcurrentMovesPerNode;
@ -139,18 +149,18 @@ public class Dispatcher {
} }
} }
static class StorageGroupMap { public static class StorageGroupMap<G extends StorageGroup> {
private static String toKey(String datanodeUuid, StorageType storageType) { private static String toKey(String datanodeUuid, StorageType storageType) {
return datanodeUuid + ":" + storageType; return datanodeUuid + ":" + storageType;
} }
private final Map<String, StorageGroup> map = new HashMap<String, StorageGroup>(); private final Map<String, G> map = new HashMap<String, G>();
StorageGroup get(String datanodeUuid, StorageType storageType) { public G get(String datanodeUuid, StorageType storageType) {
return map.get(toKey(datanodeUuid, storageType)); return map.get(toKey(datanodeUuid, storageType));
} }
void put(StorageGroup g) { public void put(G g) {
final String key = toKey(g.getDatanodeInfo().getDatanodeUuid(), g.storageType); final String key = toKey(g.getDatanodeInfo().getDatanodeUuid(), g.storageType);
final StorageGroup existing = map.put(key, g); final StorageGroup existing = map.put(key, g);
Preconditions.checkState(existing == null); Preconditions.checkState(existing == null);
@ -163,24 +173,32 @@ public class Dispatcher {
void clear() { void clear() {
map.clear(); map.clear();
} }
public Collection<G> values() {
return map.values();
}
} }
/** This class keeps track of a scheduled block move */ /** This class keeps track of a scheduled block move */
private class PendingMove { public class PendingMove {
private DBlock block; private DBlock block;
private Source source; private Source source;
private DDatanode proxySource; private DDatanode proxySource;
private StorageGroup target; private StorageGroup target;
private PendingMove() { private PendingMove(Source source, StorageGroup target) {
this.source = source;
this.target = target;
} }
@Override @Override
public String toString() { public String toString() {
final Block b = block.getBlock(); final Block b = block != null ? block.getBlock() : null;
return b + " with size=" + b.getNumBytes() + " from " String bStr = b != null ? (b + " with size=" + b.getNumBytes() + " ")
+ source.getDisplayName() + " to " + target.getDisplayName() : " ";
+ " through " + proxySource.datanode; return bStr + "from " + source.getDisplayName() + " to " + target
.getDisplayName() + " through " + (proxySource != null ? proxySource
.datanode : "");
} }
/** /**
@ -190,9 +208,11 @@ public class Dispatcher {
* @return true if a block and its proxy are chosen; false otherwise * @return true if a block and its proxy are chosen; false otherwise
*/ */
private boolean chooseBlockAndProxy() { private boolean chooseBlockAndProxy() {
// source and target must have the same storage type
final StorageType t = source.getStorageType();
// iterate all source's blocks until find a good one // iterate all source's blocks until find a good one
for (Iterator<DBlock> i = source.getBlockIterator(); i.hasNext();) { for (Iterator<DBlock> i = source.getBlockIterator(); i.hasNext();) {
if (markMovedIfGoodBlock(i.next())) { if (markMovedIfGoodBlock(i.next(), t)) {
i.remove(); i.remove();
return true; return true;
} }
@ -203,10 +223,10 @@ public class Dispatcher {
/** /**
* @return true if the given block is good for the tentative move. * @return true if the given block is good for the tentative move.
*/ */
private boolean markMovedIfGoodBlock(DBlock block) { private boolean markMovedIfGoodBlock(DBlock block, StorageType targetStorageType) {
synchronized (block) { synchronized (block) {
synchronized (movedBlocks) { synchronized (movedBlocks) {
if (isGoodBlockCandidate(source, target, block)) { if (isGoodBlockCandidate(source, target, targetStorageType, block)) {
this.block = block; this.block = block;
if (chooseProxySource()) { if (chooseProxySource()) {
movedBlocks.put(block); movedBlocks.put(block);
@ -299,12 +319,13 @@ public class Dispatcher {
LOG.info("Successfully moved " + this); LOG.info("Successfully moved " + this);
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Failed to move " + this + ": " + e.getMessage()); LOG.warn("Failed to move " + this + ": " + e.getMessage());
target.getDDatanode().setHasFailure();
// Proxy or target may have some issues, delay before using these nodes // Proxy or target may have some issues, delay before using these nodes
// further in order to avoid a potential storm of "threads quota // further in order to avoid a potential storm of "threads quota
// exceeded" warnings when the dispatcher gets out of sync with work // exceeded" warnings when the dispatcher gets out of sync with work
// going on in datanodes. // going on in datanodes.
proxySource.activateDelay(DELAY_AFTER_ERROR); proxySource.activateDelay(delayAfterErrors);
target.getDDatanode().activateDelay(DELAY_AFTER_ERROR); target.getDDatanode().activateDelay(delayAfterErrors);
} finally { } finally {
IOUtils.closeStream(out); IOUtils.closeStream(out);
IOUtils.closeStream(in); IOUtils.closeStream(in);
@ -355,10 +376,23 @@ public class Dispatcher {
} }
/** A class for keeping track of block locations in the dispatcher. */ /** A class for keeping track of block locations in the dispatcher. */
private static class DBlock extends MovedBlocks.Locations<StorageGroup> { public static class DBlock extends MovedBlocks.Locations<StorageGroup> {
DBlock(Block block) { public DBlock(Block block) {
super(block); super(block);
} }
@Override
public synchronized boolean isLocatedOn(StorageGroup loc) {
// currently we only check if replicas are located on the same DataNodes
// since we do not have the capability to store two replicas in the same
// DataNode even though they are on two different storage types
for (StorageGroup existing : locations) {
if (existing.getDatanodeInfo().equals(loc.getDatanodeInfo())) {
return true;
}
}
return false;
}
} }
/** The class represents a desired move. */ /** The class represents a desired move. */
@ -377,10 +411,10 @@ public class Dispatcher {
} }
/** A class that keeps track of a datanode. */ /** A class that keeps track of a datanode. */
static class DDatanode { public static class DDatanode {
/** A group of storages in a datanode with the same storage type. */ /** A group of storages in a datanode with the same storage type. */
class StorageGroup { public class StorageGroup {
final StorageType storageType; final StorageType storageType;
final long maxSize2Move; final long maxSize2Move;
private long scheduledSize = 0L; private long scheduledSize = 0L;
@ -390,17 +424,25 @@ public class Dispatcher {
this.maxSize2Move = maxSize2Move; this.maxSize2Move = maxSize2Move;
} }
public StorageType getStorageType() {
return storageType;
}
private DDatanode getDDatanode() { private DDatanode getDDatanode() {
return DDatanode.this; return DDatanode.this;
} }
DatanodeInfo getDatanodeInfo() { public DatanodeInfo getDatanodeInfo() {
return DDatanode.this.datanode; return DDatanode.this.datanode;
} }
/** Decide if still need to move more bytes */ /** Decide if still need to move more bytes */
synchronized boolean hasSpaceForScheduling() { boolean hasSpaceForScheduling() {
return availableSizeToMove() > 0L; return hasSpaceForScheduling(0L);
}
synchronized boolean hasSpaceForScheduling(long size) {
return availableSizeToMove() > size;
} }
/** @return the total number of bytes that need to be moved */ /** @return the total number of bytes that need to be moved */
@ -409,7 +451,7 @@ public class Dispatcher {
} }
/** increment scheduled size */ /** increment scheduled size */
synchronized void incScheduledSize(long size) { public synchronized void incScheduledSize(long size) {
scheduledSize += size; scheduledSize += size;
} }
@ -423,6 +465,18 @@ public class Dispatcher {
scheduledSize = 0L; scheduledSize = 0L;
} }
private PendingMove addPendingMove(DBlock block, final PendingMove pm) {
if (getDDatanode().addPendingBlock(pm)) {
if (pm.markMovedIfGoodBlock(block, getStorageType())) {
incScheduledSize(pm.block.getNumBytes());
return pm;
} else {
getDDatanode().removePendingBlock(pm);
}
}
return null;
}
/** @return the name for display */ /** @return the name for display */
String getDisplayName() { String getDisplayName() {
return datanode + ":" + storageType; return datanode + ":" + storageType;
@ -435,38 +489,46 @@ public class Dispatcher {
} }
final DatanodeInfo datanode; final DatanodeInfo datanode;
final EnumMap<StorageType, StorageGroup> storageMap private final EnumMap<StorageType, Source> sourceMap
= new EnumMap<StorageType, Source>(StorageType.class);
private final EnumMap<StorageType, StorageGroup> targetMap
= new EnumMap<StorageType, StorageGroup>(StorageType.class); = new EnumMap<StorageType, StorageGroup>(StorageType.class);
protected long delayUntil = 0L; protected long delayUntil = 0L;
/** blocks being moved but not confirmed yet */ /** blocks being moved but not confirmed yet */
private final List<PendingMove> pendings; private final List<PendingMove> pendings;
private volatile boolean hasFailure = false;
private final int maxConcurrentMoves; private final int maxConcurrentMoves;
@Override @Override
public String toString() { public String toString() {
return getClass().getSimpleName() + ":" + datanode + ":" + storageMap.values(); return getClass().getSimpleName() + ":" + datanode;
} }
private DDatanode(DatanodeStorageReport r, int maxConcurrentMoves) { private DDatanode(DatanodeInfo datanode, int maxConcurrentMoves) {
this.datanode = r.getDatanodeInfo(); this.datanode = datanode;
this.maxConcurrentMoves = maxConcurrentMoves; this.maxConcurrentMoves = maxConcurrentMoves;
this.pendings = new ArrayList<PendingMove>(maxConcurrentMoves); this.pendings = new ArrayList<PendingMove>(maxConcurrentMoves);
} }
private void put(StorageType storageType, StorageGroup g) { public DatanodeInfo getDatanodeInfo() {
final StorageGroup existing = storageMap.put(storageType, g); return datanode;
}
private static <G extends StorageGroup> void put(StorageType storageType,
G g, EnumMap<StorageType, G> map) {
final StorageGroup existing = map.put(storageType, g);
Preconditions.checkState(existing == null); Preconditions.checkState(existing == null);
} }
StorageGroup addStorageGroup(StorageType storageType, long maxSize2Move) { public StorageGroup addTarget(StorageType storageType, long maxSize2Move) {
final StorageGroup g = new StorageGroup(storageType, maxSize2Move); final StorageGroup g = new StorageGroup(storageType, maxSize2Move);
put(storageType, g); put(storageType, g, targetMap);
return g; return g;
} }
Source addSource(StorageType storageType, long maxSize2Move, Dispatcher d) { public Source addSource(StorageType storageType, long maxSize2Move, Dispatcher d) {
final Source s = d.new Source(storageType, maxSize2Move, this); final Source s = d.new Source(storageType, maxSize2Move, this);
put(storageType, s); put(storageType, s, sourceMap);
return s; return s;
} }
@ -504,10 +566,14 @@ public class Dispatcher {
synchronized boolean removePendingBlock(PendingMove pendingBlock) { synchronized boolean removePendingBlock(PendingMove pendingBlock) {
return pendings.remove(pendingBlock); return pendings.remove(pendingBlock);
} }
void setHasFailure() {
this.hasFailure = true;
}
} }
/** A node that can be the sources of a block move */ /** A node that can be the sources of a block move */
class Source extends DDatanode.StorageGroup { public class Source extends DDatanode.StorageGroup {
private final List<Task> tasks = new ArrayList<Task>(2); private final List<Task> tasks = new ArrayList<Task>(2);
private long blocksToReceive = 0L; private long blocksToReceive = 0L;
@ -575,8 +641,11 @@ public class Dispatcher {
/** Decide if the given block is a good candidate to move or not */ /** Decide if the given block is a good candidate to move or not */
private boolean isGoodBlockCandidate(DBlock block) { private boolean isGoodBlockCandidate(DBlock block) {
// source and target must have the same storage type
final StorageType sourceStorageType = getStorageType();
for (Task t : tasks) { for (Task t : tasks) {
if (Dispatcher.this.isGoodBlockCandidate(this, t.target, block)) { if (Dispatcher.this.isGoodBlockCandidate(this, t.target,
sourceStorageType, block)) {
return true; return true;
} }
} }
@ -596,11 +665,9 @@ public class Dispatcher {
for (Iterator<Task> i = tasks.iterator(); i.hasNext();) { for (Iterator<Task> i = tasks.iterator(); i.hasNext();) {
final Task task = i.next(); final Task task = i.next();
final DDatanode target = task.target.getDDatanode(); final DDatanode target = task.target.getDDatanode();
PendingMove pendingBlock = new PendingMove(); final PendingMove pendingBlock = new PendingMove(this, task.target);
if (target.addPendingBlock(pendingBlock)) { if (target.addPendingBlock(pendingBlock)) {
// target is not busy, so do a tentative block allocation // target is not busy, so do a tentative block allocation
pendingBlock.source = this;
pendingBlock.target = task.target;
if (pendingBlock.chooseBlockAndProxy()) { if (pendingBlock.chooseBlockAndProxy()) {
long blockSize = pendingBlock.block.getNumBytes(); long blockSize = pendingBlock.block.getNumBytes();
incScheduledSize(-blockSize); incScheduledSize(-blockSize);
@ -618,6 +685,11 @@ public class Dispatcher {
return null; return null;
} }
/** Add a pending move */
public PendingMove addPendingMove(DBlock block, StorageGroup target) {
return target.addPendingMove(block, new PendingMove(this, target));
}
/** Iterate all source's blocks to remove moved ones */ /** Iterate all source's blocks to remove moved ones */
private void removeMovedBlocks() { private void removeMovedBlocks() {
for (Iterator<DBlock> i = getBlockIterator(); i.hasNext();) { for (Iterator<DBlock> i = getBlockIterator(); i.hasNext();) {
@ -655,13 +727,7 @@ public class Dispatcher {
if (p != null) { if (p != null) {
// Reset no pending move counter // Reset no pending move counter
noPendingMoveIteration=0; noPendingMoveIteration=0;
// move the block executePendingMove(p);
moveExecutor.execute(new Runnable() {
@Override
public void run() {
p.dispatch();
}
});
continue; continue;
} }
@ -717,7 +783,8 @@ public class Dispatcher {
this.cluster = NetworkTopology.getInstance(conf); this.cluster = NetworkTopology.getInstance(conf);
this.moveExecutor = Executors.newFixedThreadPool(moverThreads); this.moveExecutor = Executors.newFixedThreadPool(moverThreads);
this.dispatchExecutor = Executors.newFixedThreadPool(dispatcherThreads); this.dispatchExecutor = dispatcherThreads == 0? null
: Executors.newFixedThreadPool(dispatcherThreads);
this.maxConcurrentMovesPerNode = maxConcurrentMovesPerNode; this.maxConcurrentMovesPerNode = maxConcurrentMovesPerNode;
this.saslClient = new SaslDataTransferClient( this.saslClient = new SaslDataTransferClient(
@ -725,11 +792,15 @@ public class Dispatcher {
TrustedChannelResolver.getInstance(conf), nnc.fallbackToSimpleAuth); TrustedChannelResolver.getInstance(conf), nnc.fallbackToSimpleAuth);
} }
StorageGroupMap getStorageGroupMap() { public DistributedFileSystem getDistributedFileSystem() {
return nnc.getDistributedFileSystem();
}
public StorageGroupMap<StorageGroup> getStorageGroupMap() {
return storageGroupMap; return storageGroupMap;
} }
NetworkTopology getCluster() { public NetworkTopology getCluster() {
return cluster; return cluster;
} }
@ -777,7 +848,7 @@ public class Dispatcher {
} }
/** Get live datanode storage reports and then build the network topology. */ /** Get live datanode storage reports and then build the network topology. */
List<DatanodeStorageReport> init() throws IOException { public List<DatanodeStorageReport> init() throws IOException {
final DatanodeStorageReport[] reports = nnc.getLiveDatanodeStorageReport(); final DatanodeStorageReport[] reports = nnc.getLiveDatanodeStorageReport();
final List<DatanodeStorageReport> trimmed = new ArrayList<DatanodeStorageReport>(); final List<DatanodeStorageReport> trimmed = new ArrayList<DatanodeStorageReport>();
// create network topology and classify utilization collections: // create network topology and classify utilization collections:
@ -793,8 +864,18 @@ public class Dispatcher {
return trimmed; return trimmed;
} }
public DDatanode newDatanode(DatanodeStorageReport r) { public DDatanode newDatanode(DatanodeInfo datanode) {
return new DDatanode(r, maxConcurrentMovesPerNode); return new DDatanode(datanode, maxConcurrentMovesPerNode);
}
public void executePendingMove(final PendingMove p) {
// move the block
moveExecutor.execute(new Runnable() {
@Override
public void run() {
p.dispatch();
}
});
} }
public boolean dispatchAndCheckContinue() throws InterruptedException { public boolean dispatchAndCheckContinue() throws InterruptedException {
@ -834,7 +915,7 @@ public class Dispatcher {
} }
// wait for all block moving to be done // wait for all block moving to be done
waitForMoveCompletion(); waitForMoveCompletion(targets);
return bytesMoved.get() - bytesLastMoved; return bytesMoved.get() - bytesLastMoved;
} }
@ -842,23 +923,25 @@ public class Dispatcher {
/** The sleeping period before checking if block move is completed again */ /** The sleeping period before checking if block move is completed again */
static private long blockMoveWaitTime = 30000L; static private long blockMoveWaitTime = 30000L;
/** set the sleeping period for block move completion check */ /**
static void setBlockMoveWaitTime(long time) { * Wait for all block move confirmations.
blockMoveWaitTime = time; * @return true if there is failed move execution
} */
public static boolean waitForMoveCompletion(
/** Wait for all block move confirmations. */ Iterable<? extends StorageGroup> targets) {
private void waitForMoveCompletion() { boolean hasFailure = false;
for(;;) { for(;;) {
boolean empty = true; boolean empty = true;
for (StorageGroup t : targets) { for (StorageGroup t : targets) {
if (!t.getDDatanode().isPendingQEmpty()) { if (!t.getDDatanode().isPendingQEmpty()) {
empty = false; empty = false;
break; break;
} else {
hasFailure |= t.getDDatanode().hasFailure;
} }
} }
if (empty) { if (empty) {
return; //all pending queues are empty return hasFailure; // all pending queues are empty
} }
try { try {
Thread.sleep(blockMoveWaitTime); Thread.sleep(blockMoveWaitTime);
@ -874,9 +957,9 @@ public class Dispatcher {
* 2. the block does not have a replica on the target; * 2. the block does not have a replica on the target;
* 3. doing the move does not reduce the number of racks that the block has * 3. doing the move does not reduce the number of racks that the block has
*/ */
private boolean isGoodBlockCandidate(Source source, StorageGroup target, private boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target,
DBlock block) { StorageType targetStorageType, DBlock block) {
if (source.storageType != target.storageType) { if (target.storageType != targetStorageType) {
return false; return false;
} }
// check if the block is moved or not // check if the block is moved or not
@ -887,7 +970,7 @@ public class Dispatcher {
return false; return false;
} }
if (cluster.isNodeGroupAware() if (cluster.isNodeGroupAware()
&& isOnSameNodeGroupWithReplicas(target, block, source)) { && isOnSameNodeGroupWithReplicas(source, target, block)) {
return false; return false;
} }
if (reduceNumOfRacks(source, target, block)) { if (reduceNumOfRacks(source, target, block)) {
@ -900,7 +983,7 @@ public class Dispatcher {
* Determine whether moving the given block replica from source to target * Determine whether moving the given block replica from source to target
* would reduce the number of racks of the block replicas. * would reduce the number of racks of the block replicas.
*/ */
private boolean reduceNumOfRacks(Source source, StorageGroup target, private boolean reduceNumOfRacks(StorageGroup source, StorageGroup target,
DBlock block) { DBlock block) {
final DatanodeInfo sourceDn = source.getDatanodeInfo(); final DatanodeInfo sourceDn = source.getDatanodeInfo();
if (cluster.isOnSameRack(sourceDn, target.getDatanodeInfo())) { if (cluster.isOnSameRack(sourceDn, target.getDatanodeInfo())) {
@ -937,8 +1020,8 @@ public class Dispatcher {
* @return true if there are any replica (other than source) on the same node * @return true if there are any replica (other than source) on the same node
* group with target * group with target
*/ */
private boolean isOnSameNodeGroupWithReplicas( private boolean isOnSameNodeGroupWithReplicas(StorageGroup source,
StorageGroup target, DBlock block, Source source) { StorageGroup target, DBlock block) {
final DatanodeInfo targetDn = target.getDatanodeInfo(); final DatanodeInfo targetDn = target.getDatanodeInfo();
for (StorageGroup g : block.getLocations()) { for (StorageGroup g : block.getLocations()) {
if (g != source && cluster.isOnSameNodeGroup(g.getDatanodeInfo(), targetDn)) { if (g != source && cluster.isOnSameNodeGroup(g.getDatanodeInfo(), targetDn)) {
@ -958,9 +1041,22 @@ public class Dispatcher {
movedBlocks.cleanup(); movedBlocks.cleanup();
} }
/** set the sleeping period for block move completion check */
@VisibleForTesting
public static void setBlockMoveWaitTime(long time) {
blockMoveWaitTime = time;
}
@VisibleForTesting
public static void setDelayAfterErrors(long time) {
delayAfterErrors = time;
}
/** shutdown thread pools */ /** shutdown thread pools */
void shutdownNow() { public void shutdownNow() {
if (dispatchExecutor != null) {
dispatchExecutor.shutdownNow(); dispatchExecutor.shutdownNow();
}
moveExecutor.shutdownNow(); moveExecutor.shutdownNow();
} }

View File

@ -31,6 +31,11 @@ public interface Matcher {
public boolean match(NetworkTopology cluster, Node left, Node right) { public boolean match(NetworkTopology cluster, Node left, Node right) {
return cluster.isOnSameNodeGroup(left, right); return cluster.isOnSameNodeGroup(left, right);
} }
@Override
public String toString() {
return "SAME_NODE_GROUP";
}
}; };
/** Match datanodes in the same rack. */ /** Match datanodes in the same rack. */
@ -39,6 +44,11 @@ public interface Matcher {
public boolean match(NetworkTopology cluster, Node left, Node right) { public boolean match(NetworkTopology cluster, Node left, Node right) {
return cluster.isOnSameRack(left, right); return cluster.isOnSameRack(left, right);
} }
@Override
public String toString() {
return "SAME_RACK";
}
}; };
/** Match any datanode with any other datanode. */ /** Match any datanode with any other datanode. */
@ -47,5 +57,10 @@ public interface Matcher {
public boolean match(NetworkTopology cluster, Node left, Node right) { public boolean match(NetworkTopology cluster, Node left, Node right) {
return left != right; return left != right;
} }
@Override
public String toString() {
return "ANY_OTHER";
}
}; };
} }

View File

@ -40,7 +40,7 @@ public class MovedBlocks<L> {
public static class Locations<L> { public static class Locations<L> {
private final Block block; // the block private final Block block; // the block
/** The locations of the replicas of the block. */ /** The locations of the replicas of the block. */
private final List<L> locations = new ArrayList<L>(3); protected final List<L> locations = new ArrayList<L>(3);
public Locations(Block block) { public Locations(Block block) {
this.block = block; this.block = block;

View File

@ -18,17 +18,22 @@
package org.apache.hadoop.hdfs.server.balancer; package org.apache.hadoop.hdfs.server.balancer;
import java.io.Closeable; import java.io.Closeable;
import java.io.DataOutputStream;
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; import java.io.OutputStream;
import java.net.InetAddress; import java.net.InetAddress;
import java.net.URI; import java.net.URI;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsServerDefaults; import org.apache.hadoop.fs.FsServerDefaults;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -45,6 +50,8 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import com.google.common.annotations.VisibleForTesting;
/** /**
* The class provides utilities for accessing a NameNode. * The class provides utilities for accessing a NameNode.
*/ */
@ -53,6 +60,41 @@ public class NameNodeConnector implements Closeable {
private static final Log LOG = LogFactory.getLog(NameNodeConnector.class); private static final Log LOG = LogFactory.getLog(NameNodeConnector.class);
private static final int MAX_NOT_CHANGED_ITERATIONS = 5; private static final int MAX_NOT_CHANGED_ITERATIONS = 5;
private static boolean write2IdFile = true;
/** Create {@link NameNodeConnector} for the given namenodes. */
public static List<NameNodeConnector> newNameNodeConnectors(
Collection<URI> namenodes, String name, Path idPath, Configuration conf)
throws IOException {
final List<NameNodeConnector> connectors = new ArrayList<NameNodeConnector>(
namenodes.size());
for (URI uri : namenodes) {
NameNodeConnector nnc = new NameNodeConnector(name, uri, idPath,
null, conf);
nnc.getKeyManager().startBlockKeyUpdater();
connectors.add(nnc);
}
return connectors;
}
public static List<NameNodeConnector> newNameNodeConnectors(
Map<URI, List<Path>> namenodes, String name, Path idPath,
Configuration conf) throws IOException {
final List<NameNodeConnector> connectors = new ArrayList<NameNodeConnector>(
namenodes.size());
for (Map.Entry<URI, List<Path>> entry : namenodes.entrySet()) {
NameNodeConnector nnc = new NameNodeConnector(name, entry.getKey(),
idPath, entry.getValue(), conf);
nnc.getKeyManager().startBlockKeyUpdater();
connectors.add(nnc);
}
return connectors;
}
@VisibleForTesting
public static void setWrite2IdFile(boolean write2IdFile) {
NameNodeConnector.write2IdFile = write2IdFile;
}
private final URI nameNodeUri; private final URI nameNodeUri;
private final String blockpoolID; private final String blockpoolID;
@ -62,16 +104,20 @@ public class NameNodeConnector implements Closeable {
private final KeyManager keyManager; private final KeyManager keyManager;
final AtomicBoolean fallbackToSimpleAuth = new AtomicBoolean(false); final AtomicBoolean fallbackToSimpleAuth = new AtomicBoolean(false);
private final FileSystem fs; private final DistributedFileSystem fs;
private final Path idPath; private final Path idPath;
private final OutputStream out; private final OutputStream out;
private final List<Path> targetPaths;
private int notChangedIterations = 0; private int notChangedIterations = 0;
public NameNodeConnector(String name, URI nameNodeUri, Path idPath, public NameNodeConnector(String name, URI nameNodeUri, Path idPath,
Configuration conf) throws IOException { List<Path> targetPaths, Configuration conf)
throws IOException {
this.nameNodeUri = nameNodeUri; this.nameNodeUri = nameNodeUri;
this.idPath = idPath; this.idPath = idPath;
this.targetPaths = targetPaths == null || targetPaths.isEmpty() ? Arrays
.asList(new Path("/")) : targetPaths;
this.namenode = NameNodeProxies.createProxy(conf, nameNodeUri, this.namenode = NameNodeProxies.createProxy(conf, nameNodeUri,
NamenodeProtocol.class).getProxy(); NamenodeProtocol.class).getProxy();
@ -85,13 +131,18 @@ public class NameNodeConnector implements Closeable {
final FsServerDefaults defaults = fs.getServerDefaults(new Path("/")); final FsServerDefaults defaults = fs.getServerDefaults(new Path("/"));
this.keyManager = new KeyManager(blockpoolID, namenode, this.keyManager = new KeyManager(blockpoolID, namenode,
defaults.getEncryptDataTransfer(), conf); defaults.getEncryptDataTransfer(), conf);
// Exit if there is another one running. // if it is for test, we do not create the id file
out = checkAndMarkRunning(); out = checkAndMarkRunning();
if (out == null) { if (out == null) {
// Exit if there is another one running.
throw new IOException("Another " + name + " is running."); throw new IOException("Another " + name + " is running.");
} }
} }
public DistributedFileSystem getDistributedFileSystem() {
return fs;
}
/** @return the block pool ID */ /** @return the block pool ID */
public String getBlockpoolID() { public String getBlockpoolID() {
return blockpoolID; return blockpoolID;
@ -114,6 +165,11 @@ public class NameNodeConnector implements Closeable {
return keyManager; return keyManager;
} }
/** @return the list of paths to scan/migrate */
public List<Path> getTargetPaths() {
return targetPaths;
}
/** Should the instance continue running? */ /** Should the instance continue running? */
public boolean shouldContinue(long dispatchBlockMoveBytes) { public boolean shouldContinue(long dispatchBlockMoveBytes) {
if (dispatchBlockMoveBytes > 0) { if (dispatchBlockMoveBytes > 0) {
@ -147,9 +203,11 @@ public class NameNodeConnector implements Closeable {
*/ */
private OutputStream checkAndMarkRunning() throws IOException { private OutputStream checkAndMarkRunning() throws IOException {
try { try {
final DataOutputStream out = fs.create(idPath); final FSDataOutputStream out = fs.create(idPath);
if (write2IdFile) {
out.writeBytes(InetAddress.getLocalHost().getHostName()); out.writeBytes(InetAddress.getLocalHost().getHostName());
out.flush(); out.hflush();
}
return out; return out;
} catch(RemoteException e) { } catch(RemoteException e) {
if(AlreadyBeingCreatedException.class.getName().equals(e.getClassName())){ if(AlreadyBeingCreatedException.class.getName().equals(e.getClassName())){

View File

@ -60,6 +60,11 @@ public interface BlockCollection {
*/ */
public short getBlockReplication(); public short getBlockReplication();
/**
* @return the storage policy ID.
*/
public byte getStoragePolicyID();
/** /**
* Get the name of the collection. * Get the name of the collection.
*/ */

View File

@ -42,6 +42,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HAUtil; import org.apache.hadoop.hdfs.HAUtil;
@ -254,6 +255,7 @@ public class BlockManager {
/** for block replicas placement */ /** for block replicas placement */
private BlockPlacementPolicy blockplacement; private BlockPlacementPolicy blockplacement;
private final BlockStoragePolicy.Suite storagePolicySuite;
/** Check whether name system is running before terminating */ /** Check whether name system is running before terminating */
private boolean checkNSRunning = true; private boolean checkNSRunning = true;
@ -276,6 +278,7 @@ public class BlockManager {
blockplacement = BlockPlacementPolicy.getInstance( blockplacement = BlockPlacementPolicy.getInstance(
conf, stats, datanodeManager.getNetworkTopology(), conf, stats, datanodeManager.getNetworkTopology(),
datanodeManager.getHost2DatanodeMap()); datanodeManager.getHost2DatanodeMap());
storagePolicySuite = BlockStoragePolicy.readBlockStorageSuite(conf);
pendingReplications = new PendingReplicationBlocks(conf.getInt( pendingReplications = new PendingReplicationBlocks(conf.getInt(
DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_DEFAULT) * 1000L); DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_DEFAULT) * 1000L);
@ -395,6 +398,10 @@ public class BlockManager {
} }
} }
public BlockStoragePolicy getStoragePolicy(final String policyName) {
return storagePolicySuite.getPolicy(policyName);
}
public void setBlockPoolId(String blockPoolId) { public void setBlockPoolId(String blockPoolId) {
if (isBlockTokenEnabled()) { if (isBlockTokenEnabled()) {
blockTokenSecretManager.setBlockPoolId(blockPoolId); blockTokenSecretManager.setBlockPoolId(blockPoolId);
@ -445,7 +452,7 @@ public class BlockManager {
return datanodeManager; return datanodeManager;
} }
/** @return the BlockPlacementPolicy */ @VisibleForTesting
public BlockPlacementPolicy getBlockPlacementPolicy() { public BlockPlacementPolicy getBlockPlacementPolicy() {
return blockplacement; return blockplacement;
} }
@ -1366,7 +1373,7 @@ public class BlockManager {
// choose replication targets: NOT HOLDING THE GLOBAL LOCK // choose replication targets: NOT HOLDING THE GLOBAL LOCK
// It is costly to extract the filename for which chooseTargets is called, // It is costly to extract the filename for which chooseTargets is called,
// so for now we pass in the block collection itself. // so for now we pass in the block collection itself.
rw.chooseTargets(blockplacement, excludedNodes); rw.chooseTargets(blockplacement, storagePolicySuite, excludedNodes);
} }
namesystem.writeLock(); namesystem.writeLock();
@ -1470,24 +1477,48 @@ public class BlockManager {
return scheduledWork; return scheduledWork;
} }
/** Choose target for WebHDFS redirection. */
public DatanodeStorageInfo[] chooseTarget4WebHDFS(String src,
DatanodeDescriptor clientnode, Set<Node> excludes, long blocksize) {
return blockplacement.chooseTarget(src, 1, clientnode,
Collections.<DatanodeStorageInfo>emptyList(), false, excludes,
blocksize, storagePolicySuite.getDefaultPolicy());
}
/** Choose target for getting additional datanodes for an existing pipeline. */
public DatanodeStorageInfo[] chooseTarget4AdditionalDatanode(String src,
int numAdditionalNodes,
DatanodeDescriptor clientnode,
List<DatanodeStorageInfo> chosen,
Set<Node> excludes,
long blocksize,
byte storagePolicyID) {
final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID);
return blockplacement.chooseTarget(src, numAdditionalNodes, clientnode,
chosen, true, excludes, blocksize, storagePolicy);
}
/** /**
* Choose target datanodes according to the replication policy. * Choose target datanodes for creating a new block.
* *
* @throws IOException * @throws IOException
* if the number of targets < minimum replication. * if the number of targets < minimum replication.
* @see BlockPlacementPolicy#chooseTarget(String, int, Node, * @see BlockPlacementPolicy#chooseTarget(String, int, Node,
* List, boolean, Set, long, StorageType) * Set, long, List, BlockStoragePolicy)
*/ */
public DatanodeStorageInfo[] chooseTarget(final String src, public DatanodeStorageInfo[] chooseTarget4NewBlock(final String src,
final int numOfReplicas, final DatanodeDescriptor client, final int numOfReplicas, final DatanodeDescriptor client,
final Set<Node> excludedNodes, final Set<Node> excludedNodes,
final long blocksize, List<String> favoredNodes) throws IOException { final long blocksize,
final List<String> favoredNodes,
final byte storagePolicyID) throws IOException {
List<DatanodeDescriptor> favoredDatanodeDescriptors = List<DatanodeDescriptor> favoredDatanodeDescriptors =
getDatanodeDescriptors(favoredNodes); getDatanodeDescriptors(favoredNodes);
final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID);
final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src, final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src,
numOfReplicas, client, excludedNodes, blocksize, numOfReplicas, client, excludedNodes, blocksize,
// TODO: get storage type from file favoredDatanodeDescriptors, storagePolicy);
favoredDatanodeDescriptors, StorageType.DEFAULT);
if (targets.length < minReplication) { if (targets.length < minReplication) {
throw new IOException("File " + src + " could only be replicated to " throw new IOException("File " + src + " could only be replicated to "
+ targets.length + " nodes instead of minReplication (=" + targets.length + " nodes instead of minReplication (="
@ -2719,6 +2750,10 @@ public class BlockManager {
assert namesystem.hasWriteLock(); assert namesystem.hasWriteLock();
// first form a rack to datanodes map and // first form a rack to datanodes map and
BlockCollection bc = getBlockCollection(b); BlockCollection bc = getBlockCollection(b);
final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(bc.getStoragePolicyID());
final List<StorageType> excessTypes = storagePolicy.chooseExcess(
replication, DatanodeStorageInfo.toStorageTypes(nonExcess));
final Map<String, List<DatanodeStorageInfo>> rackMap final Map<String, List<DatanodeStorageInfo>> rackMap
= new HashMap<String, List<DatanodeStorageInfo>>(); = new HashMap<String, List<DatanodeStorageInfo>>();
@ -2739,16 +2774,13 @@ public class BlockManager {
final DatanodeStorageInfo addedNodeStorage final DatanodeStorageInfo addedNodeStorage
= DatanodeStorageInfo.getDatanodeStorageInfo(nonExcess, addedNode); = DatanodeStorageInfo.getDatanodeStorageInfo(nonExcess, addedNode);
while (nonExcess.size() - replication > 0) { while (nonExcess.size() - replication > 0) {
// check if we can delete delNodeHint
final DatanodeStorageInfo cur; final DatanodeStorageInfo cur;
if (firstOne && delNodeHintStorage != null if (useDelHint(firstOne, delNodeHintStorage, addedNodeStorage,
&& (moreThanOne.contains(delNodeHintStorage) moreThanOne, excessTypes)) {
|| (addedNodeStorage != null
&& !moreThanOne.contains(addedNodeStorage)))) {
cur = delNodeHintStorage; cur = delNodeHintStorage;
} else { // regular excessive replica removal } else { // regular excessive replica removal
cur = replicator.chooseReplicaToDelete(bc, b, replication, cur = replicator.chooseReplicaToDelete(bc, b, replication,
moreThanOne, exactlyOne); moreThanOne, exactlyOne, excessTypes);
} }
firstOne = false; firstOne = false;
@ -2774,6 +2806,27 @@ public class BlockManager {
} }
} }
/** Check if we can use delHint */
static boolean useDelHint(boolean isFirst, DatanodeStorageInfo delHint,
DatanodeStorageInfo added, List<DatanodeStorageInfo> moreThan1Racks,
List<StorageType> excessTypes) {
if (!isFirst) {
return false; // only consider delHint for the first case
} else if (delHint == null) {
return false; // no delHint
} else if (!excessTypes.contains(delHint.getStorageType())) {
return false; // delHint storage type is not an excess type
} else {
// check if removing delHint reduces the number of racks
if (moreThan1Racks.contains(delHint)) {
return true; // delHint and some other nodes are under the same rack
} else if (added != null && !moreThan1Racks.contains(added)) {
return true; // the added node adds a new rack
}
return false; // removing delHint reduces the number of racks;
}
}
private void addToExcessReplicate(DatanodeInfo dn, Block block) { private void addToExcessReplicate(DatanodeInfo dn, Block block) {
assert namesystem.hasWriteLock(); assert namesystem.hasWriteLock();
LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(dn.getDatanodeUuid()); LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(dn.getDatanodeUuid());
@ -2880,7 +2933,7 @@ public class BlockManager {
// Decrement number of blocks scheduled to this datanode. // Decrement number of blocks scheduled to this datanode.
// for a retry request (of DatanodeProtocol#blockReceivedAndDeleted with // for a retry request (of DatanodeProtocol#blockReceivedAndDeleted with
// RECEIVED_BLOCK), we currently also decrease the approximate number. // RECEIVED_BLOCK), we currently also decrease the approximate number.
node.decrementBlocksScheduled(); node.decrementBlocksScheduled(storageInfo.getStorageType());
// get the deletion hint node // get the deletion hint node
DatanodeDescriptor delHintNode = null; DatanodeDescriptor delHintNode = null;
@ -3549,10 +3602,12 @@ public class BlockManager {
} }
private void chooseTargets(BlockPlacementPolicy blockplacement, private void chooseTargets(BlockPlacementPolicy blockplacement,
BlockStoragePolicy.Suite storagePolicySuite,
Set<Node> excludedNodes) { Set<Node> excludedNodes) {
targets = blockplacement.chooseTarget(bc.getName(), targets = blockplacement.chooseTarget(bc.getName(),
additionalReplRequired, srcNode, liveReplicaStorages, false, additionalReplRequired, srcNode, liveReplicaStorages, false,
excludedNodes, block.getNumBytes(), StorageType.DEFAULT); excludedNodes, block.getNumBytes(),
storagePolicySuite.getPolicy(bc.getStoragePolicyID()));
} }
} }

View File

@ -27,6 +27,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
@ -75,7 +76,7 @@ public abstract class BlockPlacementPolicy {
boolean returnChosenNodes, boolean returnChosenNodes,
Set<Node> excludedNodes, Set<Node> excludedNodes,
long blocksize, long blocksize,
StorageType storageType); BlockStoragePolicy storagePolicy);
/** /**
* Same as {@link #chooseTarget(String, int, Node, Set, long, List, StorageType)} * Same as {@link #chooseTarget(String, int, Node, Set, long, List, StorageType)}
@ -89,14 +90,14 @@ public abstract class BlockPlacementPolicy {
Set<Node> excludedNodes, Set<Node> excludedNodes,
long blocksize, long blocksize,
List<DatanodeDescriptor> favoredNodes, List<DatanodeDescriptor> favoredNodes,
StorageType storageType) { BlockStoragePolicy storagePolicy) {
// This class does not provide the functionality of placing // This class does not provide the functionality of placing
// a block in favored datanodes. The implementations of this class // a block in favored datanodes. The implementations of this class
// are expected to provide this functionality // are expected to provide this functionality
return chooseTarget(src, numOfReplicas, writer, return chooseTarget(src, numOfReplicas, writer,
new ArrayList<DatanodeStorageInfo>(numOfReplicas), false, new ArrayList<DatanodeStorageInfo>(numOfReplicas), false,
excludedNodes, blocksize, storageType); excludedNodes, blocksize, storagePolicy);
} }
/** /**
@ -118,18 +119,21 @@ public abstract class BlockPlacementPolicy {
* @param srcBC block collection of file to which block-to-be-deleted belongs * @param srcBC block collection of file to which block-to-be-deleted belongs
* @param block The block to be deleted * @param block The block to be deleted
* @param replicationFactor The required number of replicas for this block * @param replicationFactor The required number of replicas for this block
* @param existingReplicas The replica locations of this block that are present * @param moreThanOne The replica locations of this block that are present
on at least two unique racks. * on more than one unique racks.
* @param moreExistingReplicas Replica locations of this block that are not * @param exactlyOne Replica locations of this block that are present
listed in the previous parameter. * on exactly one unique racks.
* @param excessTypes The excess {@link StorageType}s according to the
* {@link BlockStoragePolicy}.
* @return the replica that is the best candidate for deletion * @return the replica that is the best candidate for deletion
*/ */
abstract public DatanodeStorageInfo chooseReplicaToDelete( abstract public DatanodeStorageInfo chooseReplicaToDelete(
BlockCollection srcBC, BlockCollection srcBC,
Block block, Block block,
short replicationFactor, short replicationFactor,
Collection<DatanodeStorageInfo> existingReplicas, Collection<DatanodeStorageInfo> moreThanOne,
Collection<DatanodeStorageInfo> moreExistingReplicas); Collection<DatanodeStorageInfo> exactlyOne,
List<StorageType> excessTypes);
/** /**
* Used to setup a BlockPlacementPolicy object. This should be defined by * Used to setup a BlockPlacementPolicy object. This should be defined by

View File

@ -19,15 +19,11 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
import static org.apache.hadoop.util.Time.now; import static org.apache.hadoop.util.Time.now;
import java.util.ArrayList; import java.util.*;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.StorageType;
@ -80,12 +76,6 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
*/ */
protected int tolerateHeartbeatMultiplier; protected int tolerateHeartbeatMultiplier;
protected BlockPlacementPolicyDefault(Configuration conf, FSClusterStats stats,
NetworkTopology clusterMap,
Host2NodesMap host2datanodeMap) {
initialize(conf, stats, clusterMap, host2datanodeMap);
}
protected BlockPlacementPolicyDefault() { protected BlockPlacementPolicyDefault() {
} }
@ -117,9 +107,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
boolean returnChosenNodes, boolean returnChosenNodes,
Set<Node> excludedNodes, Set<Node> excludedNodes,
long blocksize, long blocksize,
StorageType storageType) { final BlockStoragePolicy storagePolicy) {
return chooseTarget(numOfReplicas, writer, chosenNodes, returnChosenNodes, return chooseTarget(numOfReplicas, writer, chosenNodes, returnChosenNodes,
excludedNodes, blocksize, storageType); excludedNodes, blocksize, storagePolicy);
} }
@Override @Override
@ -129,17 +119,21 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
Set<Node> excludedNodes, Set<Node> excludedNodes,
long blocksize, long blocksize,
List<DatanodeDescriptor> favoredNodes, List<DatanodeDescriptor> favoredNodes,
StorageType storageType) { BlockStoragePolicy storagePolicy) {
try { try {
if (favoredNodes == null || favoredNodes.size() == 0) { if (favoredNodes == null || favoredNodes.size() == 0) {
// Favored nodes not specified, fall back to regular block placement. // Favored nodes not specified, fall back to regular block placement.
return chooseTarget(src, numOfReplicas, writer, return chooseTarget(src, numOfReplicas, writer,
new ArrayList<DatanodeStorageInfo>(numOfReplicas), false, new ArrayList<DatanodeStorageInfo>(numOfReplicas), false,
excludedNodes, blocksize, storageType); excludedNodes, blocksize, storagePolicy);
} }
Set<Node> favoriteAndExcludedNodes = excludedNodes == null ? Set<Node> favoriteAndExcludedNodes = excludedNodes == null ?
new HashSet<Node>() : new HashSet<Node>(excludedNodes); new HashSet<Node>() : new HashSet<Node>(excludedNodes);
final List<StorageType> requiredStorageTypes = storagePolicy
.chooseStorageTypes((short)numOfReplicas);
final EnumMap<StorageType, Integer> storageTypes =
getRequiredStorageTypes(requiredStorageTypes);
// Choose favored nodes // Choose favored nodes
List<DatanodeStorageInfo> results = new ArrayList<DatanodeStorageInfo>(); List<DatanodeStorageInfo> results = new ArrayList<DatanodeStorageInfo>();
@ -152,7 +146,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
final DatanodeStorageInfo target = chooseLocalStorage(favoredNode, final DatanodeStorageInfo target = chooseLocalStorage(favoredNode,
favoriteAndExcludedNodes, blocksize, favoriteAndExcludedNodes, blocksize,
getMaxNodesPerRack(results.size(), numOfReplicas)[1], getMaxNodesPerRack(results.size(), numOfReplicas)[1],
results, avoidStaleNodes, storageType, false); results, avoidStaleNodes, storageTypes, false);
if (target == null) { if (target == null) {
LOG.warn("Could not find a target for file " + src LOG.warn("Could not find a target for file " + src
+ " with favored node " + favoredNode); + " with favored node " + favoredNode);
@ -166,7 +160,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
numOfReplicas -= results.size(); numOfReplicas -= results.size();
DatanodeStorageInfo[] remainingTargets = DatanodeStorageInfo[] remainingTargets =
chooseTarget(src, numOfReplicas, writer, results, chooseTarget(src, numOfReplicas, writer, results,
false, favoriteAndExcludedNodes, blocksize, storageType); false, favoriteAndExcludedNodes, blocksize, storagePolicy);
for (int i = 0; i < remainingTargets.length; i++) { for (int i = 0; i < remainingTargets.length; i++) {
results.add(remainingTargets[i]); results.add(remainingTargets[i]);
} }
@ -174,10 +168,14 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
return getPipeline(writer, return getPipeline(writer,
results.toArray(new DatanodeStorageInfo[results.size()])); results.toArray(new DatanodeStorageInfo[results.size()]));
} catch (NotEnoughReplicasException nr) { } catch (NotEnoughReplicasException nr) {
if (LOG.isDebugEnabled()) {
LOG.debug("Failed to choose with favored nodes (=" + favoredNodes
+ "), disregard favored nodes hint and retry.", nr);
}
// Fall back to regular block placement disregarding favored nodes hint // Fall back to regular block placement disregarding favored nodes hint
return chooseTarget(src, numOfReplicas, writer, return chooseTarget(src, numOfReplicas, writer,
new ArrayList<DatanodeStorageInfo>(numOfReplicas), false, new ArrayList<DatanodeStorageInfo>(numOfReplicas), false,
excludedNodes, blocksize, storageType); excludedNodes, blocksize, storagePolicy);
} }
} }
@ -188,7 +186,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
boolean returnChosenNodes, boolean returnChosenNodes,
Set<Node> excludedNodes, Set<Node> excludedNodes,
long blocksize, long blocksize,
StorageType storageType) { final BlockStoragePolicy storagePolicy) {
if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) { if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
return DatanodeStorageInfo.EMPTY_ARRAY; return DatanodeStorageInfo.EMPTY_ARRAY;
} }
@ -213,8 +211,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
boolean avoidStaleNodes = (stats != null boolean avoidStaleNodes = (stats != null
&& stats.isAvoidingStaleDataNodesForWrite()); && stats.isAvoidingStaleDataNodesForWrite());
Node localNode = chooseTarget(numOfReplicas, writer, final Node localNode = chooseTarget(numOfReplicas, writer, excludedNodes,
excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType); blocksize, maxNodesPerRack, results, avoidStaleNodes, storagePolicy,
EnumSet.noneOf(StorageType.class), results.isEmpty());
if (!returnChosenNodes) { if (!returnChosenNodes) {
results.removeAll(chosenStorage); results.removeAll(chosenStorage);
} }
@ -235,6 +234,21 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
return new int[] {numOfReplicas, maxNodesPerRack}; return new int[] {numOfReplicas, maxNodesPerRack};
} }
private EnumMap<StorageType, Integer> getRequiredStorageTypes(
List<StorageType> types) {
EnumMap<StorageType, Integer> map = new EnumMap<StorageType,
Integer>(StorageType.class);
for (StorageType type : types) {
if (!map.containsKey(type)) {
map.put(type, 1);
} else {
int num = map.get(type);
map.put(type, num + 1);
}
}
return map;
}
/** /**
* choose <i>numOfReplicas</i> from all data nodes * choose <i>numOfReplicas</i> from all data nodes
* @param numOfReplicas additional number of replicas wanted * @param numOfReplicas additional number of replicas wanted
@ -248,30 +262,48 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
*/ */
private Node chooseTarget(int numOfReplicas, private Node chooseTarget(int numOfReplicas,
Node writer, Node writer,
Set<Node> excludedNodes, final Set<Node> excludedNodes,
long blocksize, final long blocksize,
int maxNodesPerRack, final int maxNodesPerRack,
List<DatanodeStorageInfo> results, final List<DatanodeStorageInfo> results,
final boolean avoidStaleNodes, final boolean avoidStaleNodes,
StorageType storageType) { final BlockStoragePolicy storagePolicy,
final EnumSet<StorageType> unavailableStorages,
final boolean newBlock) {
if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) { if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
return writer; return writer;
} }
int totalReplicasExpected = numOfReplicas + results.size(); final int numOfResults = results.size();
final int totalReplicasExpected = numOfReplicas + numOfResults;
int numOfResults = results.size();
boolean newBlock = (numOfResults==0);
if ((writer == null || !(writer instanceof DatanodeDescriptor)) && !newBlock) { if ((writer == null || !(writer instanceof DatanodeDescriptor)) && !newBlock) {
writer = results.get(0).getDatanodeDescriptor(); writer = results.get(0).getDatanodeDescriptor();
} }
// Keep a copy of original excludedNodes // Keep a copy of original excludedNodes
final Set<Node> oldExcludedNodes = avoidStaleNodes ? final Set<Node> oldExcludedNodes = new HashSet<Node>(excludedNodes);
new HashSet<Node>(excludedNodes) : null;
// choose storage types; use fallbacks for unavailable storages
final List<StorageType> requiredStorageTypes = storagePolicy
.chooseStorageTypes((short) totalReplicasExpected,
DatanodeStorageInfo.toStorageTypes(results),
unavailableStorages, newBlock);
final EnumMap<StorageType, Integer> storageTypes =
getRequiredStorageTypes(requiredStorageTypes);
if (LOG.isTraceEnabled()) {
LOG.trace("storageTypes=" + storageTypes);
}
try { try {
if ((numOfReplicas = requiredStorageTypes.size()) == 0) {
throw new NotEnoughReplicasException(
"All required storage types are unavailable: "
+ " unavailableStorages=" + unavailableStorages
+ ", storagePolicy=" + storagePolicy);
}
if (numOfResults == 0) { if (numOfResults == 0) {
writer = chooseLocalStorage(writer, excludedNodes, blocksize, writer = chooseLocalStorage(writer, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType, true) maxNodesPerRack, results, avoidStaleNodes, storageTypes, true)
.getDatanodeDescriptor(); .getDatanodeDescriptor();
if (--numOfReplicas == 0) { if (--numOfReplicas == 0) {
return writer; return writer;
@ -280,7 +312,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
final DatanodeDescriptor dn0 = results.get(0).getDatanodeDescriptor(); final DatanodeDescriptor dn0 = results.get(0).getDatanodeDescriptor();
if (numOfResults <= 1) { if (numOfResults <= 1) {
chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack, chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack,
results, avoidStaleNodes, storageType); results, avoidStaleNodes, storageTypes);
if (--numOfReplicas == 0) { if (--numOfReplicas == 0) {
return writer; return writer;
} }
@ -289,24 +321,28 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
final DatanodeDescriptor dn1 = results.get(1).getDatanodeDescriptor(); final DatanodeDescriptor dn1 = results.get(1).getDatanodeDescriptor();
if (clusterMap.isOnSameRack(dn0, dn1)) { if (clusterMap.isOnSameRack(dn0, dn1)) {
chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack, chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack,
results, avoidStaleNodes, storageType); results, avoidStaleNodes, storageTypes);
} else if (newBlock){ } else if (newBlock){
chooseLocalRack(dn1, excludedNodes, blocksize, maxNodesPerRack, chooseLocalRack(dn1, excludedNodes, blocksize, maxNodesPerRack,
results, avoidStaleNodes, storageType); results, avoidStaleNodes, storageTypes);
} else { } else {
chooseLocalRack(writer, excludedNodes, blocksize, maxNodesPerRack, chooseLocalRack(writer, excludedNodes, blocksize, maxNodesPerRack,
results, avoidStaleNodes, storageType); results, avoidStaleNodes, storageTypes);
} }
if (--numOfReplicas == 0) { if (--numOfReplicas == 0) {
return writer; return writer;
} }
} }
chooseRandom(numOfReplicas, NodeBase.ROOT, excludedNodes, blocksize, chooseRandom(numOfReplicas, NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType); maxNodesPerRack, results, avoidStaleNodes, storageTypes);
} catch (NotEnoughReplicasException e) { } catch (NotEnoughReplicasException e) {
final String message = "Failed to place enough replicas, still in need of " final String message = "Failed to place enough replicas, still in need of "
+ (totalReplicasExpected - results.size()) + " to reach " + (totalReplicasExpected - results.size()) + " to reach "
+ totalReplicasExpected + "."; + totalReplicasExpected
+ " (unavailableStorages=" + unavailableStorages
+ ", storagePolicy=" + storagePolicy
+ ", newBlock=" + newBlock + ")";
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
LOG.trace(message, e); LOG.trace(message, e);
} else { } else {
@ -327,7 +363,28 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
// if the NotEnoughReplicasException was thrown in chooseRandom(). // if the NotEnoughReplicasException was thrown in chooseRandom().
numOfReplicas = totalReplicasExpected - results.size(); numOfReplicas = totalReplicasExpected - results.size();
return chooseTarget(numOfReplicas, writer, oldExcludedNodes, blocksize, return chooseTarget(numOfReplicas, writer, oldExcludedNodes, blocksize,
maxNodesPerRack, results, false, storageType); maxNodesPerRack, results, false, storagePolicy, unavailableStorages,
newBlock);
}
boolean retry = false;
// simply add all the remaining types into unavailableStorages and give
// another try. No best effort is guaranteed here.
for (StorageType type : storageTypes.keySet()) {
if (!unavailableStorages.contains(type)) {
unavailableStorages.add(type);
retry = true;
}
}
if (retry) {
for (DatanodeStorageInfo resultStorage : results) {
addToExcludedNodes(resultStorage.getDatanodeDescriptor(),
oldExcludedNodes);
}
numOfReplicas = totalReplicasExpected - results.size();
return chooseTarget(numOfReplicas, writer, oldExcludedNodes, blocksize,
maxNodesPerRack, results, false, storagePolicy, unavailableStorages,
newBlock);
} }
} }
return writer; return writer;
@ -340,39 +397,46 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
* @return the chosen storage * @return the chosen storage
*/ */
protected DatanodeStorageInfo chooseLocalStorage(Node localMachine, protected DatanodeStorageInfo chooseLocalStorage(Node localMachine,
Set<Node> excludedNodes, Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
long blocksize, List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
int maxNodesPerRack, EnumMap<StorageType, Integer> storageTypes, boolean fallbackToLocalRack)
List<DatanodeStorageInfo> results,
boolean avoidStaleNodes,
StorageType storageType,
boolean fallbackToLocalRack)
throws NotEnoughReplicasException { throws NotEnoughReplicasException {
// if no local machine, randomly choose one node // if no local machine, randomly choose one node
if (localMachine == null) { if (localMachine == null) {
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType); maxNodesPerRack, results, avoidStaleNodes, storageTypes);
} }
if (preferLocalNode && localMachine instanceof DatanodeDescriptor) { if (preferLocalNode && localMachine instanceof DatanodeDescriptor) {
DatanodeDescriptor localDatanode = (DatanodeDescriptor) localMachine; DatanodeDescriptor localDatanode = (DatanodeDescriptor) localMachine;
// otherwise try local machine first // otherwise try local machine first
if (excludedNodes.add(localMachine)) { // was not in the excluded list if (excludedNodes.add(localMachine)) { // was not in the excluded list
for(DatanodeStorageInfo localStorage : DFSUtil.shuffle( for (Iterator<Map.Entry<StorageType, Integer>> iter = storageTypes
.entrySet().iterator(); iter.hasNext(); ) {
Map.Entry<StorageType, Integer> entry = iter.next();
for (DatanodeStorageInfo localStorage : DFSUtil.shuffle(
localDatanode.getStorageInfos())) { localDatanode.getStorageInfos())) {
StorageType type = entry.getKey();
if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize, if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
maxNodesPerRack, false, results, avoidStaleNodes, storageType) >= 0) { maxNodesPerRack, false, results, avoidStaleNodes, type) >= 0) {
int num = entry.getValue();
if (num == 1) {
iter.remove();
} else {
entry.setValue(num - 1);
}
return localStorage; return localStorage;
} }
} }
} }
} }
}
if (!fallbackToLocalRack) { if (!fallbackToLocalRack) {
return null; return null;
} }
// try a node on local rack // try a node on local rack
return chooseLocalRack(localMachine, excludedNodes, blocksize, return chooseLocalRack(localMachine, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType); maxNodesPerRack, results, avoidStaleNodes, storageTypes);
} }
/** /**
@ -400,42 +464,63 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
int maxNodesPerRack, int maxNodesPerRack,
List<DatanodeStorageInfo> results, List<DatanodeStorageInfo> results,
boolean avoidStaleNodes, boolean avoidStaleNodes,
StorageType storageType) EnumMap<StorageType, Integer> storageTypes)
throws NotEnoughReplicasException { throws NotEnoughReplicasException {
// no local machine, so choose a random machine // no local machine, so choose a random machine
if (localMachine == null) { if (localMachine == null) {
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType); maxNodesPerRack, results, avoidStaleNodes, storageTypes);
} }
final String localRack = localMachine.getNetworkLocation();
// choose one from the local rack
try { try {
return chooseRandom(localMachine.getNetworkLocation(), excludedNodes, // choose one from the local rack
blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType); return chooseRandom(localRack, excludedNodes,
} catch (NotEnoughReplicasException e1) { blocksize, maxNodesPerRack, results, avoidStaleNodes, storageTypes);
// find the second replica } catch (NotEnoughReplicasException e) {
DatanodeDescriptor newLocal=null; // find the next replica and retry with its rack
for(DatanodeStorageInfo resultStorage : results) { for(DatanodeStorageInfo resultStorage : results) {
DatanodeDescriptor nextNode = resultStorage.getDatanodeDescriptor(); DatanodeDescriptor nextNode = resultStorage.getDatanodeDescriptor();
if (nextNode != localMachine) { if (nextNode != localMachine) {
newLocal = nextNode; if (LOG.isDebugEnabled()) {
break; LOG.debug("Failed to choose from local rack (location = " + localRack
+ "), retry with the rack of the next replica (location = "
+ nextNode.getNetworkLocation() + ")", e);
}
return chooseFromNextRack(nextNode, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageTypes);
} }
} }
if (newLocal != null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Failed to choose from local rack (location = " + localRack
+ "); the second replica is not found, retry choosing ramdomly", e);
}
//the second replica is not found, randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageTypes);
}
}
private DatanodeStorageInfo chooseFromNextRack(Node next,
Set<Node> excludedNodes,
long blocksize,
int maxNodesPerRack,
List<DatanodeStorageInfo> results,
boolean avoidStaleNodes,
EnumMap<StorageType, Integer> storageTypes) throws NotEnoughReplicasException {
final String nextRack = next.getNetworkLocation();
try { try {
return chooseRandom(newLocal.getNetworkLocation(), excludedNodes, return chooseRandom(nextRack, excludedNodes, blocksize, maxNodesPerRack,
blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType); results, avoidStaleNodes, storageTypes);
} catch(NotEnoughReplicasException e2) { } catch(NotEnoughReplicasException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Failed to choose from the next rack (location = " + nextRack
+ "), retry choosing ramdomly", e);
}
//otherwise randomly choose one from the network //otherwise randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType); maxNodesPerRack, results, avoidStaleNodes, storageTypes);
}
} else {
//otherwise randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType);
}
} }
} }
@ -453,18 +538,22 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
int maxReplicasPerRack, int maxReplicasPerRack,
List<DatanodeStorageInfo> results, List<DatanodeStorageInfo> results,
boolean avoidStaleNodes, boolean avoidStaleNodes,
StorageType storageType) EnumMap<StorageType, Integer> storageTypes)
throws NotEnoughReplicasException { throws NotEnoughReplicasException {
int oldNumOfReplicas = results.size(); int oldNumOfReplicas = results.size();
// randomly choose one node from remote racks // randomly choose one node from remote racks
try { try {
chooseRandom(numOfReplicas, "~" + localMachine.getNetworkLocation(), chooseRandom(numOfReplicas, "~" + localMachine.getNetworkLocation(),
excludedNodes, blocksize, maxReplicasPerRack, results, excludedNodes, blocksize, maxReplicasPerRack, results,
avoidStaleNodes, storageType); avoidStaleNodes, storageTypes);
} catch (NotEnoughReplicasException e) { } catch (NotEnoughReplicasException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Failed to choose remote rack (location = ~"
+ localMachine.getNetworkLocation() + "), fallback to local rack", e);
}
chooseRandom(numOfReplicas-(results.size()-oldNumOfReplicas), chooseRandom(numOfReplicas-(results.size()-oldNumOfReplicas),
localMachine.getNetworkLocation(), excludedNodes, blocksize, localMachine.getNetworkLocation(), excludedNodes, blocksize,
maxReplicasPerRack, results, avoidStaleNodes, storageType); maxReplicasPerRack, results, avoidStaleNodes, storageTypes);
} }
} }
@ -478,10 +567,10 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
int maxNodesPerRack, int maxNodesPerRack,
List<DatanodeStorageInfo> results, List<DatanodeStorageInfo> results,
boolean avoidStaleNodes, boolean avoidStaleNodes,
StorageType storageType) EnumMap<StorageType, Integer> storageTypes)
throws NotEnoughReplicasException { throws NotEnoughReplicasException {
return chooseRandom(1, scope, excludedNodes, blocksize, maxNodesPerRack, return chooseRandom(1, scope, excludedNodes, blocksize, maxNodesPerRack,
results, avoidStaleNodes, storageType); results, avoidStaleNodes, storageTypes);
} }
/** /**
@ -495,7 +584,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
int maxNodesPerRack, int maxNodesPerRack,
List<DatanodeStorageInfo> results, List<DatanodeStorageInfo> results,
boolean avoidStaleNodes, boolean avoidStaleNodes,
StorageType storageType) EnumMap<StorageType, Integer> storageTypes)
throws NotEnoughReplicasException { throws NotEnoughReplicasException {
int numOfAvailableNodes = clusterMap.countNumOfAvailableNodes( int numOfAvailableNodes = clusterMap.countNumOfAvailableNodes(
@ -512,24 +601,43 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
DatanodeDescriptor chosenNode = DatanodeDescriptor chosenNode =
(DatanodeDescriptor)clusterMap.chooseRandom(scope); (DatanodeDescriptor)clusterMap.chooseRandom(scope);
if (excludedNodes.add(chosenNode)) { //was not in the excluded list if (excludedNodes.add(chosenNode)) { //was not in the excluded list
if (LOG.isDebugEnabled()) {
builder.append("\nNode ").append(NodeBase.getPath(chosenNode)).append(" [");
}
numOfAvailableNodes--; numOfAvailableNodes--;
final DatanodeStorageInfo[] storages = DFSUtil.shuffle( final DatanodeStorageInfo[] storages = DFSUtil.shuffle(
chosenNode.getStorageInfos()); chosenNode.getStorageInfos());
int i; int i = 0;
for(i = 0; i < storages.length; i++) { boolean search = true;
for (Iterator<Map.Entry<StorageType, Integer>> iter = storageTypes
.entrySet().iterator(); search && iter.hasNext(); ) {
Map.Entry<StorageType, Integer> entry = iter.next();
for (i = 0; i < storages.length; i++) {
StorageType type = entry.getKey();
final int newExcludedNodes = addIfIsGoodTarget(storages[i], final int newExcludedNodes = addIfIsGoodTarget(storages[i],
excludedNodes, blocksize, maxNodesPerRack, considerLoad, results, excludedNodes, blocksize, maxNodesPerRack, considerLoad, results,
avoidStaleNodes, storageType); avoidStaleNodes, type);
if (newExcludedNodes >= 0) { if (newExcludedNodes >= 0) {
numOfReplicas--; numOfReplicas--;
if (firstChosen == null) { if (firstChosen == null) {
firstChosen = storages[i]; firstChosen = storages[i];
} }
numOfAvailableNodes -= newExcludedNodes; numOfAvailableNodes -= newExcludedNodes;
int num = entry.getValue();
if (num == 1) {
iter.remove();
} else {
entry.setValue(num - 1);
}
search = false;
break; break;
} }
} }
}
if (LOG.isDebugEnabled()) {
builder.append("\n]");
}
// If no candidate storage was found on this DN then set badTarget. // If no candidate storage was found on this DN then set badTarget.
badTarget = (i == storages.length); badTarget = (i == storages.length);
@ -540,9 +648,11 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
String detail = enableDebugLogging; String detail = enableDebugLogging;
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
if (badTarget && builder != null) { if (badTarget && builder != null) {
detail = builder.append("]").toString(); detail = builder.toString();
builder.setLength(0); builder.setLength(0);
} else detail = ""; } else {
detail = "";
}
} }
throw new NotEnoughReplicasException(detail); throw new NotEnoughReplicasException(detail);
} }
@ -576,14 +686,10 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
private static void logNodeIsNotChosen(DatanodeStorageInfo storage, String reason) { private static void logNodeIsNotChosen(DatanodeStorageInfo storage, String reason) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
// build the error message for later use. // build the error message for later use.
debugLoggingBuilder.get() debugLoggingBuilder.get()
.append(node).append(": ") .append("\n Storage ").append(storage)
.append("Storage ").append(storage) .append(" is not chosen since ").append(reason).append(".");
.append("at node ").append(NodeBase.getPath(node))
.append(" is not chosen because ")
.append(reason);
} }
} }
@ -608,11 +714,10 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
boolean considerLoad, boolean considerLoad,
List<DatanodeStorageInfo> results, List<DatanodeStorageInfo> results,
boolean avoidStaleNodes, boolean avoidStaleNodes,
StorageType storageType) { StorageType requiredStorageType) {
if (storage.getStorageType() != storageType) { if (storage.getStorageType() != requiredStorageType) {
logNodeIsNotChosen(storage, logNodeIsNotChosen(storage, "storage types do not match,"
"storage types do not match, where the expected storage type is " + " where the required storage type is " + requiredStorageType);
+ storageType);
return false; return false;
} }
if (storage.getState() == State.READ_ONLY_SHARED) { if (storage.getState() == State.READ_ONLY_SHARED) {
@ -634,9 +739,14 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
} }
final long requiredSize = blockSize * HdfsConstants.MIN_BLOCKS_FOR_WRITE; final long requiredSize = blockSize * HdfsConstants.MIN_BLOCKS_FOR_WRITE;
final long scheduledSize = blockSize * node.getBlocksScheduled(); final long scheduledSize = blockSize * node.getBlocksScheduled(storage.getStorageType());
if (requiredSize > storage.getRemaining() - scheduledSize) { final long remaining = node.getRemaining(storage.getStorageType());
logNodeIsNotChosen(storage, "the node does not have enough space "); if (requiredSize > remaining - scheduledSize) {
logNodeIsNotChosen(storage, "the node does not have enough "
+ storage.getStorageType() + " space"
+ " (required=" + requiredSize
+ ", scheduled=" + scheduledSize
+ ", remaining=" + remaining + ")");
return false; return false;
} }
@ -645,8 +755,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
final double maxLoad = 2.0 * stats.getInServiceXceiverAverage(); final double maxLoad = 2.0 * stats.getInServiceXceiverAverage();
final int nodeLoad = node.getXceiverCount(); final int nodeLoad = node.getXceiverCount();
if (nodeLoad > maxLoad) { if (nodeLoad > maxLoad) {
logNodeIsNotChosen(storage, logNodeIsNotChosen(storage, "the node is too busy (load: " + nodeLoad
"the node is too busy (load:"+nodeLoad+" > "+maxLoad+") "); + " > " + maxLoad + ") ");
return false; return false;
} }
} }
@ -732,7 +842,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
public DatanodeStorageInfo chooseReplicaToDelete(BlockCollection bc, public DatanodeStorageInfo chooseReplicaToDelete(BlockCollection bc,
Block block, short replicationFactor, Block block, short replicationFactor,
Collection<DatanodeStorageInfo> first, Collection<DatanodeStorageInfo> first,
Collection<DatanodeStorageInfo> second) { Collection<DatanodeStorageInfo> second,
final List<StorageType> excessTypes) {
long oldestHeartbeat = long oldestHeartbeat =
now() - heartbeatInterval * tolerateHeartbeatMultiplier; now() - heartbeatInterval * tolerateHeartbeatMultiplier;
DatanodeStorageInfo oldestHeartbeatStorage = null; DatanodeStorageInfo oldestHeartbeatStorage = null;
@ -742,6 +853,10 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
// Pick the node with the oldest heartbeat or with the least free space, // Pick the node with the oldest heartbeat or with the least free space,
// if all hearbeats are within the tolerable heartbeat interval // if all hearbeats are within the tolerable heartbeat interval
for(DatanodeStorageInfo storage : pickupReplicaSet(first, second)) { for(DatanodeStorageInfo storage : pickupReplicaSet(first, second)) {
if (!excessTypes.contains(storage.getStorageType())) {
continue;
}
final DatanodeDescriptor node = storage.getDatanodeDescriptor(); final DatanodeDescriptor node = storage.getDatanodeDescriptor();
long free = node.getRemaining(); long free = node.getRemaining();
long lastHeartbeat = node.getLastUpdate(); long lastHeartbeat = node.getLastUpdate();
@ -755,8 +870,16 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
} }
} }
return oldestHeartbeatStorage != null? oldestHeartbeatStorage final DatanodeStorageInfo storage;
: minSpaceStorage; if (oldestHeartbeatStorage != null) {
storage = oldestHeartbeatStorage;
} else if (minSpaceStorage != null) {
storage = minSpaceStorage;
} else {
return null;
}
excessTypes.remove(storage.getStorageType());
return storage;
} }
/** /**

View File

@ -17,12 +17,7 @@
*/ */
package org.apache.hadoop.hdfs.server.blockmanagement; package org.apache.hadoop.hdfs.server.blockmanagement;
import java.util.ArrayList; import java.util.*;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
@ -69,31 +64,42 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
protected DatanodeStorageInfo chooseLocalStorage(Node localMachine, protected DatanodeStorageInfo chooseLocalStorage(Node localMachine,
Set<Node> excludedNodes, long blocksize, int maxNodesPerRack, Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
List<DatanodeStorageInfo> results, boolean avoidStaleNodes, List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
StorageType storageType, boolean fallbackToLocalRack EnumMap<StorageType, Integer> storageTypes, boolean fallbackToLocalRack)
) throws NotEnoughReplicasException { throws NotEnoughReplicasException {
// if no local machine, randomly choose one node // if no local machine, randomly choose one node
if (localMachine == null) if (localMachine == null)
return chooseRandom(NodeBase.ROOT, excludedNodes, return chooseRandom(NodeBase.ROOT, excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType); blocksize, maxNodesPerRack, results, avoidStaleNodes, storageTypes);
// otherwise try local machine first // otherwise try local machine first
if (localMachine instanceof DatanodeDescriptor) { if (localMachine instanceof DatanodeDescriptor) {
DatanodeDescriptor localDataNode = (DatanodeDescriptor)localMachine; DatanodeDescriptor localDataNode = (DatanodeDescriptor)localMachine;
if (excludedNodes.add(localMachine)) { // was not in the excluded list if (excludedNodes.add(localMachine)) { // was not in the excluded list
for(DatanodeStorageInfo localStorage : DFSUtil.shuffle( for (Iterator<Map.Entry<StorageType, Integer>> iter = storageTypes
.entrySet().iterator(); iter.hasNext(); ) {
Map.Entry<StorageType, Integer> entry = iter.next();
for (DatanodeStorageInfo localStorage : DFSUtil.shuffle(
localDataNode.getStorageInfos())) { localDataNode.getStorageInfos())) {
StorageType type = entry.getKey();
if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize, if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
maxNodesPerRack, false, results, avoidStaleNodes, storageType) >= 0) { maxNodesPerRack, false, results, avoidStaleNodes, type) >= 0) {
int num = entry.getValue();
if (num == 1) {
iter.remove();
} else {
entry.setValue(num - 1);
}
return localStorage; return localStorage;
} }
} }
} }
} }
}
// try a node on local node group // try a node on local node group
DatanodeStorageInfo chosenStorage = chooseLocalNodeGroup( DatanodeStorageInfo chosenStorage = chooseLocalNodeGroup(
(NetworkTopologyWithNodeGroup)clusterMap, localMachine, excludedNodes, (NetworkTopologyWithNodeGroup)clusterMap, localMachine, excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType); blocksize, maxNodesPerRack, results, avoidStaleNodes, storageTypes);
if (chosenStorage != null) { if (chosenStorage != null) {
return chosenStorage; return chosenStorage;
} }
@ -103,7 +109,7 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
} }
// try a node on local rack // try a node on local rack
return chooseLocalRack(localMachine, excludedNodes, return chooseLocalRack(localMachine, excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType); blocksize, maxNodesPerRack, results, avoidStaleNodes, storageTypes);
} }
/** @return the node of the second replica */ /** @return the node of the second replica */
@ -123,18 +129,19 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
protected DatanodeStorageInfo chooseLocalRack(Node localMachine, protected DatanodeStorageInfo chooseLocalRack(Node localMachine,
Set<Node> excludedNodes, long blocksize, int maxNodesPerRack, Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
List<DatanodeStorageInfo> results, boolean avoidStaleNodes, List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
StorageType storageType) throws NotEnoughReplicasException { EnumMap<StorageType, Integer> storageTypes) throws
NotEnoughReplicasException {
// no local machine, so choose a random machine // no local machine, so choose a random machine
if (localMachine == null) { if (localMachine == null) {
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType); maxNodesPerRack, results, avoidStaleNodes, storageTypes);
} }
// choose one from the local rack, but off-nodegroup // choose one from the local rack, but off-nodegroup
try { try {
final String scope = NetworkTopology.getFirstHalf(localMachine.getNetworkLocation()); final String scope = NetworkTopology.getFirstHalf(localMachine.getNetworkLocation());
return chooseRandom(scope, excludedNodes, blocksize, maxNodesPerRack, return chooseRandom(scope, excludedNodes, blocksize, maxNodesPerRack,
results, avoidStaleNodes, storageType); results, avoidStaleNodes, storageTypes);
} catch (NotEnoughReplicasException e1) { } catch (NotEnoughReplicasException e1) {
// find the second replica // find the second replica
final DatanodeDescriptor newLocal = secondNode(localMachine, results); final DatanodeDescriptor newLocal = secondNode(localMachine, results);
@ -142,16 +149,17 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
try { try {
return chooseRandom( return chooseRandom(
clusterMap.getRack(newLocal.getNetworkLocation()), excludedNodes, clusterMap.getRack(newLocal.getNetworkLocation()), excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType); blocksize, maxNodesPerRack, results, avoidStaleNodes,
storageTypes);
} catch(NotEnoughReplicasException e2) { } catch(NotEnoughReplicasException e2) {
//otherwise randomly choose one from the network //otherwise randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType); maxNodesPerRack, results, avoidStaleNodes, storageTypes);
} }
} else { } else {
//otherwise randomly choose one from the network //otherwise randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType); maxNodesPerRack, results, avoidStaleNodes, storageTypes);
} }
} }
} }
@ -163,7 +171,7 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
protected void chooseRemoteRack(int numOfReplicas, protected void chooseRemoteRack(int numOfReplicas,
DatanodeDescriptor localMachine, Set<Node> excludedNodes, DatanodeDescriptor localMachine, Set<Node> excludedNodes,
long blocksize, int maxReplicasPerRack, List<DatanodeStorageInfo> results, long blocksize, int maxReplicasPerRack, List<DatanodeStorageInfo> results,
boolean avoidStaleNodes, StorageType storageType) boolean avoidStaleNodes, EnumMap<StorageType, Integer> storageTypes)
throws NotEnoughReplicasException { throws NotEnoughReplicasException {
int oldNumOfReplicas = results.size(); int oldNumOfReplicas = results.size();
@ -172,12 +180,12 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
try { try {
// randomly choose from remote racks // randomly choose from remote racks
chooseRandom(numOfReplicas, "~" + rackLocation, excludedNodes, blocksize, chooseRandom(numOfReplicas, "~" + rackLocation, excludedNodes, blocksize,
maxReplicasPerRack, results, avoidStaleNodes, storageType); maxReplicasPerRack, results, avoidStaleNodes, storageTypes);
} catch (NotEnoughReplicasException e) { } catch (NotEnoughReplicasException e) {
// fall back to the local rack // fall back to the local rack
chooseRandom(numOfReplicas - (results.size() - oldNumOfReplicas), chooseRandom(numOfReplicas - (results.size() - oldNumOfReplicas),
rackLocation, excludedNodes, blocksize, rackLocation, excludedNodes, blocksize,
maxReplicasPerRack, results, avoidStaleNodes, storageType); maxReplicasPerRack, results, avoidStaleNodes, storageTypes);
} }
} }
@ -191,11 +199,12 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
NetworkTopologyWithNodeGroup clusterMap, Node localMachine, NetworkTopologyWithNodeGroup clusterMap, Node localMachine,
Set<Node> excludedNodes, long blocksize, int maxNodesPerRack, Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
List<DatanodeStorageInfo> results, boolean avoidStaleNodes, List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
StorageType storageType) throws NotEnoughReplicasException { EnumMap<StorageType, Integer> storageTypes) throws
NotEnoughReplicasException {
// no local machine, so choose a random machine // no local machine, so choose a random machine
if (localMachine == null) { if (localMachine == null) {
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType); maxNodesPerRack, results, avoidStaleNodes, storageTypes);
} }
// choose one from the local node group // choose one from the local node group
@ -203,7 +212,7 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
return chooseRandom( return chooseRandom(
clusterMap.getNodeGroup(localMachine.getNetworkLocation()), clusterMap.getNodeGroup(localMachine.getNetworkLocation()),
excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes, excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes,
storageType); storageTypes);
} catch (NotEnoughReplicasException e1) { } catch (NotEnoughReplicasException e1) {
final DatanodeDescriptor newLocal = secondNode(localMachine, results); final DatanodeDescriptor newLocal = secondNode(localMachine, results);
if (newLocal != null) { if (newLocal != null) {
@ -211,16 +220,16 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
return chooseRandom( return chooseRandom(
clusterMap.getNodeGroup(newLocal.getNetworkLocation()), clusterMap.getNodeGroup(newLocal.getNetworkLocation()),
excludedNodes, blocksize, maxNodesPerRack, results, excludedNodes, blocksize, maxNodesPerRack, results,
avoidStaleNodes, storageType); avoidStaleNodes, storageTypes);
} catch(NotEnoughReplicasException e2) { } catch(NotEnoughReplicasException e2) {
//otherwise randomly choose one from the network //otherwise randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType); maxNodesPerRack, results, avoidStaleNodes, storageTypes);
} }
} else { } else {
//otherwise randomly choose one from the network //otherwise randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize, return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType); maxNodesPerRack, results, avoidStaleNodes, storageTypes);
} }
} }
} }

View File

@ -28,16 +28,19 @@ import java.util.Map;
import java.util.Queue; import java.util.Queue;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.server.namenode.CachedBlock; import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.hdfs.util.EnumCounters;
import org.apache.hadoop.hdfs.util.LightWeightHashSet; import org.apache.hadoop.hdfs.util.LightWeightHashSet;
import org.apache.hadoop.util.IntrusiveCollection; import org.apache.hadoop.util.IntrusiveCollection;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
@ -204,8 +207,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
* in case of errors (e.g. datanode does not report if an error occurs * in case of errors (e.g. datanode does not report if an error occurs
* while writing the block). * while writing the block).
*/ */
private int currApproxBlocksScheduled = 0; private EnumCounters<StorageType> currApproxBlocksScheduled
private int prevApproxBlocksScheduled = 0; = new EnumCounters<StorageType>(StorageType.class);
private EnumCounters<StorageType> prevApproxBlocksScheduled
= new EnumCounters<StorageType>(StorageType.class);
private long lastBlocksScheduledRollTime = 0; private long lastBlocksScheduledRollTime = 0;
private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min
private int volumeFailures = 0; private int volumeFailures = 0;
@ -476,25 +481,48 @@ public class DatanodeDescriptor extends DatanodeInfo {
} }
} }
/**
* @return Approximate number of blocks currently scheduled to be written
*/
public long getRemaining(StorageType t) {
long remaining = 0;
for(DatanodeStorageInfo s : getStorageInfos()) {
if (s.getStorageType() == t) {
remaining += s.getRemaining();
}
}
return remaining;
}
/**
* @return Approximate number of blocks currently scheduled to be written
* to the given storage type of this datanode.
*/
public int getBlocksScheduled(StorageType t) {
return (int)(currApproxBlocksScheduled.get(t)
+ prevApproxBlocksScheduled.get(t));
}
/** /**
* @return Approximate number of blocks currently scheduled to be written * @return Approximate number of blocks currently scheduled to be written
* to this datanode. * to this datanode.
*/ */
public int getBlocksScheduled() { public int getBlocksScheduled() {
return currApproxBlocksScheduled + prevApproxBlocksScheduled; return (int)(currApproxBlocksScheduled.sum()
+ prevApproxBlocksScheduled.sum());
} }
/** Increment the number of blocks scheduled. */ /** Increment the number of blocks scheduled. */
void incrementBlocksScheduled() { void incrementBlocksScheduled(StorageType t) {
currApproxBlocksScheduled++; currApproxBlocksScheduled.add(t, 1);;
} }
/** Decrement the number of blocks scheduled. */ /** Decrement the number of blocks scheduled. */
void decrementBlocksScheduled() { void decrementBlocksScheduled(StorageType t) {
if (prevApproxBlocksScheduled > 0) { if (prevApproxBlocksScheduled.get(t) > 0) {
prevApproxBlocksScheduled--; prevApproxBlocksScheduled.subtract(t, 1);
} else if (currApproxBlocksScheduled > 0) { } else if (currApproxBlocksScheduled.get(t) > 0) {
currApproxBlocksScheduled--; currApproxBlocksScheduled.subtract(t, 1);
} }
// its ok if both counters are zero. // its ok if both counters are zero.
} }
@ -502,8 +530,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
/** Adjusts curr and prev number of blocks scheduled every few minutes. */ /** Adjusts curr and prev number of blocks scheduled every few minutes. */
private void rollBlocksScheduled(long now) { private void rollBlocksScheduled(long now) {
if (now - lastBlocksScheduledRollTime > BLOCKS_SCHEDULED_ROLL_INTERVAL) { if (now - lastBlocksScheduledRollTime > BLOCKS_SCHEDULED_ROLL_INTERVAL) {
prevApproxBlocksScheduled = currApproxBlocksScheduled; prevApproxBlocksScheduled.set(currApproxBlocksScheduled);
currApproxBlocksScheduled = 0; currApproxBlocksScheduled.reset();
lastBlocksScheduledRollTime = now; lastBlocksScheduledRollTime = now;
} }
} }

View File

@ -109,7 +109,7 @@ public class DatanodeStorageInfo {
private long capacity; private long capacity;
private long dfsUsed; private long dfsUsed;
private long remaining; private volatile long remaining;
private long blockPoolUsed; private long blockPoolUsed;
private volatile BlockInfo blockList = null; private volatile BlockInfo blockList = null;
@ -283,7 +283,7 @@ public class DatanodeStorageInfo {
/** Increment the number of blocks scheduled for each given storage */ /** Increment the number of blocks scheduled for each given storage */
public static void incrementBlocksScheduled(DatanodeStorageInfo... storages) { public static void incrementBlocksScheduled(DatanodeStorageInfo... storages) {
for (DatanodeStorageInfo s : storages) { for (DatanodeStorageInfo s : storages) {
s.getDatanodeDescriptor().incrementBlocksScheduled(); s.getDatanodeDescriptor().incrementBlocksScheduled(s.getStorageType());
} }
} }
@ -314,6 +314,26 @@ public class DatanodeStorageInfo {
false, capacity, dfsUsed, remaining, blockPoolUsed); false, capacity, dfsUsed, remaining, blockPoolUsed);
} }
static Iterable<StorageType> toStorageTypes(
final Iterable<DatanodeStorageInfo> infos) {
return new Iterable<StorageType>() {
@Override
public Iterator<StorageType> iterator() {
return new Iterator<StorageType>() {
final Iterator<DatanodeStorageInfo> i = infos.iterator();
@Override
public boolean hasNext() {return i.hasNext();}
@Override
public StorageType next() {return i.next().getStorageType();}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
};
}
};
}
/** @return the first {@link DatanodeStorageInfo} corresponding to /** @return the first {@link DatanodeStorageInfo} corresponding to
* the given datanode * the given datanode
*/ */

View File

@ -1930,7 +1930,9 @@ public class DataNode extends ReconfigurableBase
+ b + " (numBytes=" + b.getNumBytes() + ")" + b + " (numBytes=" + b.getNumBytes() + ")"
+ ", stage=" + stage + ", stage=" + stage
+ ", clientname=" + clientname + ", clientname=" + clientname
+ ", targets=" + Arrays.asList(targets)); + ", targets=" + Arrays.asList(targets)
+ ", target storage types=" + (targetStorageTypes == null ? "[]" :
Arrays.asList(targetStorageTypes)));
} }
this.targets = targets; this.targets = targets;
this.targetStorageTypes = targetStorageTypes; this.targetStorageTypes = targetStorageTypes;

View File

@ -0,0 +1,644 @@
/**
* 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.mover;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.commons.cli.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.*;
import org.apache.hadoop.hdfs.protocol.*;
import org.apache.hadoop.hdfs.server.balancer.Dispatcher;
import org.apache.hadoop.hdfs.server.balancer.Dispatcher.*;
import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DDatanode.StorageGroup;
import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
import org.apache.hadoop.hdfs.server.balancer.Matcher;
import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import java.io.BufferedReader;
import java.io.FileReader;
import java.io.IOException;
import java.net.URI;
import java.text.DateFormat;
import java.util.*;
@InterfaceAudience.Private
public class Mover {
static final Log LOG = LogFactory.getLog(Mover.class);
static final Path MOVER_ID_PATH = new Path("/system/mover.id");
private static class StorageMap {
private final StorageGroupMap<Source> sources
= new StorageGroupMap<Source>();
private final StorageGroupMap<StorageGroup> targets
= new StorageGroupMap<StorageGroup>();
private final EnumMap<StorageType, List<StorageGroup>> targetStorageTypeMap
= new EnumMap<StorageType, List<StorageGroup>>(StorageType.class);
private StorageMap() {
for(StorageType t : StorageType.asList()) {
targetStorageTypeMap.put(t, new LinkedList<StorageGroup>());
}
}
private void add(Source source, StorageGroup target) {
sources.put(source);
targets.put(target);
getTargetStorages(target.getStorageType()).add(target);
}
private Source getSource(MLocation ml) {
return get(sources, ml);
}
private StorageGroup getTarget(MLocation ml) {
return get(targets, ml);
}
private static <G extends StorageGroup> G get(StorageGroupMap<G> map, MLocation ml) {
return map.get(ml.datanode.getDatanodeUuid(), ml.storageType);
}
private List<StorageGroup> getTargetStorages(StorageType t) {
return targetStorageTypeMap.get(t);
}
}
private final Dispatcher dispatcher;
private final StorageMap storages;
private final List<Path> targetPaths;
private final BlockStoragePolicy.Suite blockStoragePolicies;
Mover(NameNodeConnector nnc, Configuration conf) {
final long movedWinWidth = conf.getLong(
DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_KEY,
DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_DEFAULT);
final int moverThreads = conf.getInt(
DFSConfigKeys.DFS_MOVER_MOVERTHREADS_KEY,
DFSConfigKeys.DFS_MOVER_MOVERTHREADS_DEFAULT);
final int maxConcurrentMovesPerNode = conf.getInt(
DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY,
DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT);
this.dispatcher = new Dispatcher(nnc, Collections.<String> emptySet(),
Collections.<String> emptySet(), movedWinWidth, moverThreads, 0,
maxConcurrentMovesPerNode, conf);
this.storages = new StorageMap();
this.blockStoragePolicies = BlockStoragePolicy.readBlockStorageSuite(conf);
this.targetPaths = nnc.getTargetPaths();
}
void init() throws IOException {
final List<DatanodeStorageReport> reports = dispatcher.init();
for(DatanodeStorageReport r : reports) {
final DDatanode dn = dispatcher.newDatanode(r.getDatanodeInfo());
for(StorageType t : StorageType.asList()) {
final long maxRemaining = getMaxRemaining(r, t);
if (maxRemaining > 0L) {
final Source source = dn.addSource(t, Long.MAX_VALUE, dispatcher);
final StorageGroup target = dn.addTarget(t, maxRemaining);
storages.add(source, target);
}
}
}
}
private ExitStatus run() {
try {
init();
boolean hasRemaining = new Processor().processNamespace();
return hasRemaining ? ExitStatus.IN_PROGRESS : ExitStatus.SUCCESS;
} catch (IllegalArgumentException e) {
System.out.println(e + ". Exiting ...");
return ExitStatus.ILLEGAL_ARGUMENTS;
} catch (IOException e) {
System.out.println(e + ". Exiting ...");
return ExitStatus.IO_EXCEPTION;
} finally {
dispatcher.shutdownNow();
}
}
DBlock newDBlock(Block block, List<MLocation> locations) {
final DBlock db = new DBlock(block);
for(MLocation ml : locations) {
db.addLocation(storages.getTarget(ml));
}
return db;
}
private static long getMaxRemaining(DatanodeStorageReport report, StorageType t) {
long max = 0L;
for(StorageReport r : report.getStorageReports()) {
if (r.getStorage().getStorageType() == t) {
if (r.getRemaining() > max) {
max = r.getRemaining();
}
}
}
return max;
}
/**
* convert a snapshot path to non-snapshot path. E.g.,
* /foo/.snapshot/snapshot-name/bar --> /foo/bar
*/
private static String convertSnapshotPath(String[] pathComponents) {
StringBuilder sb = new StringBuilder(Path.SEPARATOR);
for (int i = 0; i < pathComponents.length; i++) {
if (pathComponents[i].equals(HdfsConstants.DOT_SNAPSHOT_DIR)) {
i++;
} else {
sb.append(pathComponents[i]);
}
}
return sb.toString();
}
class Processor {
private final DFSClient dfs;
private final List<String> snapshottableDirs = new ArrayList<String>();
Processor() {
dfs = dispatcher.getDistributedFileSystem().getClient();
}
private void getSnapshottableDirs() {
SnapshottableDirectoryStatus[] dirs = null;
try {
dirs = dfs.getSnapshottableDirListing();
} catch (IOException e) {
LOG.warn("Failed to get snapshottable directories."
+ " Ignore and continue.", e);
}
if (dirs != null) {
for (SnapshottableDirectoryStatus dir : dirs) {
snapshottableDirs.add(dir.getFullPath().toString());
}
}
}
/**
* @return true if the given path is a snapshot path and the corresponding
* INode is still in the current fsdirectory.
*/
private boolean isSnapshotPathInCurrent(String path) throws IOException {
// if the parent path contains "/.snapshot/", this is a snapshot path
if (path.contains(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR)) {
String[] pathComponents = INode.getPathNames(path);
if (HdfsConstants.DOT_SNAPSHOT_DIR
.equals(pathComponents[pathComponents.length - 2])) {
// this is a path for a specific snapshot (e.g., /foo/.snapshot/s1)
return false;
}
String nonSnapshotPath = convertSnapshotPath(pathComponents);
return dfs.getFileInfo(nonSnapshotPath) != null;
} else {
return false;
}
}
/**
* @return whether there is still remaining migration work for the next
* round
*/
private boolean processNamespace() {
getSnapshottableDirs();
boolean hasRemaining = true;
try {
for (Path target : targetPaths) {
hasRemaining = processDirRecursively("", dfs.getFileInfo(target
.toUri().getPath()));
}
} catch (IOException e) {
LOG.warn("Failed to get root directory status. Ignore and continue.", e);
}
// wait for pending move to finish and retry the failed migration
hasRemaining |= Dispatcher.waitForMoveCompletion(storages.targets.values());
return hasRemaining;
}
/**
* @return whether there is still remaing migration work for the next
* round
*/
private boolean processChildrenList(String fullPath) {
boolean hasRemaining = false;
for (byte[] lastReturnedName = HdfsFileStatus.EMPTY_NAME;;) {
final DirectoryListing children;
try {
children = dfs.listPaths(fullPath, lastReturnedName, true);
} catch(IOException e) {
LOG.warn("Failed to list directory " + fullPath
+ ". Ignore the directory and continue.", e);
return hasRemaining;
}
if (children == null) {
return hasRemaining;
}
for (HdfsFileStatus child : children.getPartialListing()) {
hasRemaining |= processDirRecursively(fullPath, child);
}
if (children.hasMore()) {
lastReturnedName = children.getLastName();
} else {
return hasRemaining;
}
}
}
/** @return whether the migration requires next round */
private boolean processDirRecursively(String parent,
HdfsFileStatus status) {
String fullPath = status.getFullName(parent);
boolean hasRemaining = false;
if (status.isDir()) {
if (!fullPath.endsWith(Path.SEPARATOR)) {
fullPath = fullPath + Path.SEPARATOR;
}
hasRemaining = processChildrenList(fullPath);
// process snapshots if this is a snapshottable directory
if (snapshottableDirs.contains(fullPath)) {
final String dirSnapshot = fullPath + HdfsConstants.DOT_SNAPSHOT_DIR;
hasRemaining |= processChildrenList(dirSnapshot);
}
} else if (!status.isSymlink()) { // file
try {
if (!isSnapshotPathInCurrent(fullPath)) {
// the full path is a snapshot path but it is also included in the
// current directory tree, thus ignore it.
hasRemaining = processFile((HdfsLocatedFileStatus)status);
}
} catch (IOException e) {
LOG.warn("Failed to check the status of " + parent
+ ". Ignore it and continue.", e);
return false;
}
}
return hasRemaining;
}
/** @return true if it is necessary to run another round of migration */
private boolean processFile(HdfsLocatedFileStatus status) {
final BlockStoragePolicy policy = blockStoragePolicies.getPolicy(
status.getStoragePolicy());
final List<StorageType> types = policy.chooseStorageTypes(
status.getReplication());
final LocatedBlocks locatedBlocks = status.getBlockLocations();
boolean hasRemaining = false;
final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete();
List<LocatedBlock> lbs = locatedBlocks.getLocatedBlocks();
for(int i = 0; i < lbs.size(); i++) {
if (i == lbs.size() - 1 && !lastBlkComplete) {
// last block is incomplete, skip it
continue;
}
LocatedBlock lb = lbs.get(i);
final StorageTypeDiff diff = new StorageTypeDiff(types,
lb.getStorageTypes());
if (!diff.removeOverlap()) {
if (scheduleMoves4Block(diff, lb)) {
hasRemaining |= (diff.existing.size() > 1 &&
diff.expected.size() > 1);
}
}
}
return hasRemaining;
}
boolean scheduleMoves4Block(StorageTypeDiff diff, LocatedBlock lb) {
final List<MLocation> locations = MLocation.toLocations(lb);
Collections.shuffle(locations);
final DBlock db = newDBlock(lb.getBlock().getLocalBlock(), locations);
for (final StorageType t : diff.existing) {
for (final MLocation ml : locations) {
final Source source = storages.getSource(ml);
if (ml.storageType == t) {
// try to schedule one replica move.
if (scheduleMoveReplica(db, source, diff.expected)) {
return true;
}
}
}
}
return false;
}
@VisibleForTesting
boolean scheduleMoveReplica(DBlock db, MLocation ml,
List<StorageType> targetTypes) {
return scheduleMoveReplica(db, storages.getSource(ml), targetTypes);
}
boolean scheduleMoveReplica(DBlock db, Source source,
List<StorageType> targetTypes) {
if (dispatcher.getCluster().isNodeGroupAware()) {
if (chooseTarget(db, source, targetTypes, Matcher.SAME_NODE_GROUP)) {
return true;
}
}
// Then, match nodes on the same rack
if (chooseTarget(db, source, targetTypes, Matcher.SAME_RACK)) {
return true;
}
// At last, match all remaining nodes
return chooseTarget(db, source, targetTypes, Matcher.ANY_OTHER);
}
boolean chooseTarget(DBlock db, Source source,
List<StorageType> targetTypes, Matcher matcher) {
final NetworkTopology cluster = dispatcher.getCluster();
for (StorageType t : targetTypes) {
for(StorageGroup target : storages.getTargetStorages(t)) {
if (matcher.match(cluster, source.getDatanodeInfo(),
target.getDatanodeInfo())) {
final PendingMove pm = source.addPendingMove(db, target);
if (pm != null) {
dispatcher.executePendingMove(pm);
return true;
}
}
}
}
return false;
}
}
static class MLocation {
final DatanodeInfo datanode;
final StorageType storageType;
final long size;
MLocation(DatanodeInfo datanode, StorageType storageType, long size) {
this.datanode = datanode;
this.storageType = storageType;
this.size = size;
}
static List<MLocation> toLocations(LocatedBlock lb) {
final DatanodeInfo[] datanodeInfos = lb.getLocations();
final StorageType[] storageTypes = lb.getStorageTypes();
final long size = lb.getBlockSize();
final List<MLocation> locations = new LinkedList<MLocation>();
for(int i = 0; i < datanodeInfos.length; i++) {
locations.add(new MLocation(datanodeInfos[i], storageTypes[i], size));
}
return locations;
}
}
@VisibleForTesting
static class StorageTypeDiff {
final List<StorageType> expected;
final List<StorageType> existing;
StorageTypeDiff(List<StorageType> expected, StorageType[] existing) {
this.expected = new LinkedList<StorageType>(expected);
this.existing = new LinkedList<StorageType>(Arrays.asList(existing));
}
/**
* Remove the overlap between the expected types and the existing types.
* @return if the existing types or the expected types is empty after
* removing the overlap.
*/
boolean removeOverlap() {
for(Iterator<StorageType> i = existing.iterator(); i.hasNext(); ) {
final StorageType t = i.next();
if (expected.remove(t)) {
i.remove();
}
}
return expected.isEmpty() || existing.isEmpty();
}
@Override
public String toString() {
return getClass().getSimpleName() + "{expected=" + expected
+ ", existing=" + existing + "}";
}
}
static int run(Map<URI, List<Path>> namenodes, Configuration conf)
throws IOException, InterruptedException {
final long sleeptime =
conf.getLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT) * 2000 +
conf.getLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000;
LOG.info("namenodes = " + namenodes);
List<NameNodeConnector> connectors = Collections.emptyList();
try {
connectors = NameNodeConnector.newNameNodeConnectors(namenodes,
Mover.class.getSimpleName(), MOVER_ID_PATH, conf);
while (connectors.size() > 0) {
Collections.shuffle(connectors);
Iterator<NameNodeConnector> iter = connectors.iterator();
while (iter.hasNext()) {
NameNodeConnector nnc = iter.next();
final Mover m = new Mover(nnc, conf);
final ExitStatus r = m.run();
if (r == ExitStatus.SUCCESS) {
IOUtils.cleanup(LOG, nnc);
iter.remove();
} else if (r != ExitStatus.IN_PROGRESS) {
// must be an error statue, return
return r.getExitCode();
}
}
Thread.sleep(sleeptime);
}
return ExitStatus.SUCCESS.getExitCode();
} finally {
for (NameNodeConnector nnc : connectors) {
IOUtils.cleanup(LOG, nnc);
}
}
}
static class Cli extends Configured implements Tool {
private static final String USAGE = "Usage: java "
+ Mover.class.getSimpleName() + " [-p <files/dirs> | -f <local file>]"
+ "\n\t-p <files/dirs>\ta space separated list of HDFS files/dirs to migrate."
+ "\n\t-f <local file>\ta local file containing a list of HDFS files/dirs to migrate.";
private static Options buildCliOptions() {
Options opts = new Options();
Option file = OptionBuilder.withArgName("pathsFile").hasArg()
.withDescription("a local file containing files/dirs to migrate")
.create("f");
Option paths = OptionBuilder.withArgName("paths").hasArgs()
.withDescription("specify space separated files/dirs to migrate")
.create("p");
OptionGroup group = new OptionGroup();
group.addOption(file);
group.addOption(paths);
opts.addOptionGroup(group);
return opts;
}
private static String[] readPathFile(String file) throws IOException {
List<String> list = Lists.newArrayList();
BufferedReader reader = new BufferedReader(new FileReader(file));
try {
String line;
while ((line = reader.readLine()) != null) {
if (!line.trim().isEmpty()) {
list.add(line);
}
}
} finally {
IOUtils.cleanup(LOG, reader);
}
return list.toArray(new String[list.size()]);
}
private static Map<URI, List<Path>> getNameNodePaths(CommandLine line,
Configuration conf) throws Exception {
Map<URI, List<Path>> map = Maps.newHashMap();
String[] paths = null;
if (line.hasOption("f")) {
paths = readPathFile(line.getOptionValue("f"));
} else if (line.hasOption("p")) {
paths = line.getOptionValues("p");
}
Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
if (paths == null || paths.length == 0) {
for (URI namenode : namenodes) {
map.put(namenode, null);
}
return map;
}
final URI singleNs = namenodes.size() == 1 ?
namenodes.iterator().next() : null;
for (String path : paths) {
Path target = new Path(path);
if (!target.isUriPathAbsolute()) {
throw new IllegalArgumentException("The path " + target
+ " is not absolute");
}
URI targetUri = target.toUri();
if ((targetUri.getAuthority() == null || targetUri.getScheme() ==
null) && singleNs == null) {
// each path must contains both scheme and authority information
// unless there is only one name service specified in the
// configuration
throw new IllegalArgumentException("The path " + target
+ " does not contain scheme and authority thus cannot identify"
+ " its name service");
}
URI key = singleNs;
if (singleNs == null) {
key = new URI(targetUri.getScheme(), targetUri.getAuthority(),
null, null, null);
if (!namenodes.contains(key)) {
throw new IllegalArgumentException("Cannot resolve the path " +
target + ". The namenode services specified in the " +
"configuration: " + namenodes);
}
}
List<Path> targets = map.get(key);
if (targets == null) {
targets = Lists.newArrayList();
map.put(key, targets);
}
targets.add(Path.getPathWithoutSchemeAndAuthority(target));
}
return map;
}
@VisibleForTesting
static Map<URI, List<Path>> getNameNodePathsToMove(Configuration conf,
String... args) throws Exception {
final Options opts = buildCliOptions();
CommandLineParser parser = new GnuParser();
CommandLine commandLine = parser.parse(opts, args, true);
return getNameNodePaths(commandLine, conf);
}
@Override
public int run(String[] args) throws Exception {
final long startTime = Time.monotonicNow();
final Configuration conf = getConf();
try {
final Map<URI, List<Path>> map = getNameNodePathsToMove(conf, args);
return Mover.run(map, conf);
} catch (IOException e) {
System.out.println(e + ". Exiting ...");
return ExitStatus.IO_EXCEPTION.getExitCode();
} catch (InterruptedException e) {
System.out.println(e + ". Exiting ...");
return ExitStatus.INTERRUPTED.getExitCode();
} catch (ParseException e) {
System.out.println(e + ". Exiting ...");
return ExitStatus.ILLEGAL_ARGUMENTS.getExitCode();
} catch (IllegalArgumentException e) {
System.out.println(e + ". Exiting ...");
return ExitStatus.ILLEGAL_ARGUMENTS.getExitCode();
} finally {
System.out.format("%-24s ", DateFormat.getDateTimeInstance().format(new Date()));
System.out.println("Mover took " + StringUtils.formatTime(Time.monotonicNow()-startTime));
}
}
}
/**
* Run a Mover in command line.
*
* @param args Command line arguments
*/
public static void main(String[] args) {
if (DFSUtil.parseHelpArgument(args, Cli.USAGE, System.out, true)) {
System.exit(0);
}
try {
System.exit(ToolRunner.run(new HdfsConfiguration(), new Cli(), args));
} catch (Throwable e) {
LOG.error("Exiting " + Mover.class.getSimpleName()
+ " due to an exception", e);
System.exit(-1);
}
}
}

View File

@ -53,6 +53,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem;
@ -274,6 +275,12 @@ public class FSDirectory implements Closeable {
skipQuotaCheck = true; skipQuotaCheck = true;
} }
private static INodeFile newINodeFile(long id, PermissionStatus permissions,
long mtime, long atime, short replication, long preferredBlockSize) {
return new INodeFile(id, null, permissions, mtime, atime,
BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize, (byte)0);
}
/** /**
* Add the given filename to the fs. * Add the given filename to the fs.
* @throws FileAlreadyExistsException * @throws FileAlreadyExistsException
@ -288,9 +295,8 @@ public class FSDirectory implements Closeable {
UnresolvedLinkException, SnapshotAccessControlException, AclException { UnresolvedLinkException, SnapshotAccessControlException, AclException {
long modTime = now(); long modTime = now();
INodeFile newNode = new INodeFile(namesystem.allocateNewInodeId(), null, INodeFile newNode = newINodeFile(namesystem.allocateNewInodeId(),
permissions, modTime, modTime, BlockInfo.EMPTY_ARRAY, replication, permissions, modTime, modTime, replication, preferredBlockSize);
preferredBlockSize);
newNode.toUnderConstruction(clientName, clientMachine); newNode.toUnderConstruction(clientName, clientMachine);
boolean added = false; boolean added = false;
@ -326,14 +332,13 @@ public class FSDirectory implements Closeable {
final INodeFile newNode; final INodeFile newNode;
assert hasWriteLock(); assert hasWriteLock();
if (underConstruction) { if (underConstruction) {
newNode = new INodeFile(id, null, permissions, modificationTime, newNode = newINodeFile(id, permissions, modificationTime,
modificationTime, BlockInfo.EMPTY_ARRAY, replication, modificationTime, replication, preferredBlockSize);
preferredBlockSize);
newNode.toUnderConstruction(clientName, clientMachine); newNode.toUnderConstruction(clientName, clientMachine);
} else { } else {
newNode = new INodeFile(id, null, permissions, modificationTime, atime, newNode = newINodeFile(id, permissions, modificationTime, atime,
BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize); replication, preferredBlockSize);
} }
try { try {
@ -1000,6 +1005,44 @@ public class FSDirectory implements Closeable {
return file.getBlocks(); return file.getBlocks();
} }
/** Set block storage policy for a directory */
void setStoragePolicy(String src, byte policyId)
throws IOException {
writeLock();
try {
unprotectedSetStoragePolicy(src, policyId);
} finally {
writeUnlock();
}
}
void unprotectedSetStoragePolicy(String src, byte policyId)
throws IOException {
assert hasWriteLock();
final INodesInPath iip = getINodesInPath4Write(src, true);
final INode inode = iip.getLastINode();
if (inode == null) {
throw new FileNotFoundException("File/Directory does not exist: " + src);
}
final int snapshotId = iip.getLatestSnapshotId();
if (inode.isFile()) {
inode.asFile().setStoragePolicyID(policyId, snapshotId);
} else if (inode.isDirectory()) {
setDirStoragePolicy(inode.asDirectory(), policyId, snapshotId);
} else {
throw new FileNotFoundException(src + " is not a file or directory");
}
}
private void setDirStoragePolicy(INodeDirectory inode, byte policyId,
int latestSnapshotId) throws IOException {
List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
XAttr xAttr = BlockStoragePolicy.buildXAttr(policyId);
List<XAttr> newXAttrs = setINodeXAttrs(existingXAttrs, Arrays.asList(xAttr),
EnumSet.of(XAttrSetFlag.CREATE, XAttrSetFlag.REPLACE));
XAttrStorage.updateINodeXAttrs(inode, newXAttrs, latestSnapshotId);
}
/** /**
* @param path the file path * @param path the file path
* @return the block size of the file. * @return the block size of the file.
@ -1331,6 +1374,11 @@ public class FSDirectory implements Closeable {
} }
} }
private byte getStoragePolicyID(byte inodePolicy, byte parentPolicy) {
return inodePolicy != BlockStoragePolicy.ID_UNSPECIFIED ? inodePolicy :
parentPolicy;
}
/** /**
* Get a partial listing of the indicated directory * Get a partial listing of the indicated directory
* *
@ -1345,7 +1393,8 @@ public class FSDirectory implements Closeable {
* @return a partial listing starting after startAfter * @return a partial listing starting after startAfter
*/ */
DirectoryListing getListing(String src, byte[] startAfter, DirectoryListing getListing(String src, byte[] startAfter,
boolean needLocation) throws UnresolvedLinkException, IOException { boolean needLocation, boolean isSuperUser)
throws UnresolvedLinkException, IOException {
String srcs = normalizePath(src); String srcs = normalizePath(src);
final boolean isRawPath = isReservedRawName(src); final boolean isRawPath = isReservedRawName(src);
@ -1354,18 +1403,19 @@ public class FSDirectory implements Closeable {
if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) { if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
return getSnapshotsListing(srcs, startAfter); return getSnapshotsListing(srcs, startAfter);
} }
final INodesInPath inodesInPath = getINodesInPath(srcs, true); final INodesInPath inodesInPath = getLastINodeInPath(srcs);
final INode[] inodes = inodesInPath.getINodes();
final int snapshot = inodesInPath.getPathSnapshotId(); final int snapshot = inodesInPath.getPathSnapshotId();
final INode targetNode = inodes[inodes.length - 1]; final INode targetNode = inodesInPath.getLastINode();
if (targetNode == null) if (targetNode == null)
return null; return null;
byte parentStoragePolicy = isSuperUser ?
targetNode.getStoragePolicyID() : BlockStoragePolicy.ID_UNSPECIFIED;
if (!targetNode.isDirectory()) { if (!targetNode.isDirectory()) {
return new DirectoryListing( return new DirectoryListing(
new HdfsFileStatus[]{createFileStatus(HdfsFileStatus.EMPTY_NAME, new HdfsFileStatus[]{createFileStatus(HdfsFileStatus.EMPTY_NAME,
targetNode, needLocation, snapshot, isRawPath, targetNode, needLocation, parentStoragePolicy, snapshot,
inodesInPath)}, 0); isRawPath, inodesInPath)}, 0);
} }
final INodeDirectory dirInode = targetNode.asDirectory(); final INodeDirectory dirInode = targetNode.asDirectory();
@ -1378,8 +1428,11 @@ public class FSDirectory implements Closeable {
HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing]; HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
for (int i=0; i<numOfListing && locationBudget>0; i++) { for (int i=0; i<numOfListing && locationBudget>0; i++) {
INode cur = contents.get(startChild+i); INode cur = contents.get(startChild+i);
listing[i] = createFileStatus(cur.getLocalNameBytes(), cur, byte curPolicy = isSuperUser && !cur.isSymlink()?
needLocation, snapshot, isRawPath, inodesInPath); cur.getLocalStoragePolicyID(): BlockStoragePolicy.ID_UNSPECIFIED;
listing[i] = createFileStatus(cur.getLocalNameBytes(), cur, needLocation,
getStoragePolicyID(curPolicy, parentStoragePolicy), snapshot,
isRawPath, inodesInPath);
listingCnt++; listingCnt++;
if (needLocation) { if (needLocation) {
// Once we hit lsLimit locations, stop. // Once we hit lsLimit locations, stop.
@ -1430,7 +1483,8 @@ public class FSDirectory implements Closeable {
for (int i = 0; i < numOfListing; i++) { for (int i = 0; i < numOfListing; i++) {
Root sRoot = snapshots.get(i + skipSize).getRoot(); Root sRoot = snapshots.get(i + skipSize).getRoot();
listing[i] = createFileStatus(sRoot.getLocalNameBytes(), sRoot, listing[i] = createFileStatus(sRoot.getLocalNameBytes(), sRoot,
Snapshot.CURRENT_STATE_ID, false, null); BlockStoragePolicy.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
false, null);
} }
return new DirectoryListing( return new DirectoryListing(
listing, snapshots.size() - skipSize - numOfListing); listing, snapshots.size() - skipSize - numOfListing);
@ -1440,10 +1494,12 @@ public class FSDirectory implements Closeable {
* @param src The string representation of the path to the file * @param src The string representation of the path to the file
* @param resolveLink whether to throw UnresolvedLinkException * @param resolveLink whether to throw UnresolvedLinkException
* @param isRawPath true if a /.reserved/raw pathname was passed by the user * @param isRawPath true if a /.reserved/raw pathname was passed by the user
* @param includeStoragePolicy whether to include storage policy
* @return object containing information regarding the file * @return object containing information regarding the file
* or null if file not found * or null if file not found
*/ */
HdfsFileStatus getFileInfo(String src, boolean resolveLink, boolean isRawPath) HdfsFileStatus getFileInfo(String src, boolean resolveLink,
boolean isRawPath, boolean includeStoragePolicy)
throws IOException { throws IOException {
String srcs = normalizePath(src); String srcs = normalizePath(src);
readLock(); readLock();
@ -1454,9 +1510,10 @@ public class FSDirectory implements Closeable {
final INodesInPath inodesInPath = getINodesInPath(srcs, resolveLink); final INodesInPath inodesInPath = getINodesInPath(srcs, resolveLink);
final INode[] inodes = inodesInPath.getINodes(); final INode[] inodes = inodesInPath.getINodes();
final INode i = inodes[inodes.length - 1]; final INode i = inodes[inodes.length - 1];
byte policyId = includeStoragePolicy && i != null && !i.isSymlink() ?
return i == null? null: createFileStatus(HdfsFileStatus.EMPTY_NAME, i, i.getStoragePolicyID() : BlockStoragePolicy.ID_UNSPECIFIED;
inodesInPath.getPathSnapshotId(), isRawPath, inodesInPath); return i == null ? null : createFileStatus(HdfsFileStatus.EMPTY_NAME, i,
policyId, inodesInPath.getPathSnapshotId(), isRawPath, inodesInPath);
} finally { } finally {
readUnlock(); readUnlock();
} }
@ -1473,7 +1530,7 @@ public class FSDirectory implements Closeable {
throws UnresolvedLinkException { throws UnresolvedLinkException {
if (getINode4DotSnapshot(src) != null) { if (getINode4DotSnapshot(src) != null) {
return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null, return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
HdfsFileStatus.EMPTY_NAME, -1L, 0, null); HdfsFileStatus.EMPTY_NAME, -1L, 0, null, BlockStoragePolicy.ID_UNSPECIFIED);
} }
return null; return null;
} }
@ -2312,19 +2369,22 @@ public class FSDirectory implements Closeable {
* @throws IOException if any error occurs * @throws IOException if any error occurs
*/ */
private HdfsFileStatus createFileStatus(byte[] path, INode node, private HdfsFileStatus createFileStatus(byte[] path, INode node,
boolean needLocation, int snapshot, boolean isRawPath, boolean needLocation, byte storagePolicy, int snapshot,
INodesInPath iip) boolean isRawPath, INodesInPath iip)
throws IOException { throws IOException {
if (needLocation) { if (needLocation) {
return createLocatedFileStatus(path, node, snapshot, isRawPath, iip); return createLocatedFileStatus(path, node, storagePolicy, snapshot,
isRawPath, iip);
} else { } else {
return createFileStatus(path, node, snapshot, isRawPath, iip); return createFileStatus(path, node, storagePolicy, snapshot,
isRawPath, iip);
} }
} }
/** /**
* Create FileStatus by file INode * Create FileStatus by file INode
*/ */
HdfsFileStatus createFileStatus(byte[] path, INode node, HdfsFileStatus createFileStatus(byte[] path, INode node, byte storagePolicy,
int snapshot, boolean isRawPath, INodesInPath iip) throws IOException { int snapshot, boolean isRawPath, INodesInPath iip) throws IOException {
long size = 0; // length is zero for directories long size = 0; // length is zero for directories
short replication = 0; short replication = 0;
@ -2362,14 +2422,15 @@ public class FSDirectory implements Closeable {
path, path,
node.getId(), node.getId(),
childrenNum, childrenNum,
feInfo); feInfo,
storagePolicy);
} }
/** /**
* Create FileStatus with location info by file INode * Create FileStatus with location info by file INode
*/ */
private HdfsLocatedFileStatus createLocatedFileStatus(byte[] path, private HdfsLocatedFileStatus createLocatedFileStatus(byte[] path, INode node,
INode node, int snapshot, boolean isRawPath, byte storagePolicy, int snapshot, boolean isRawPath,
INodesInPath iip) throws IOException { INodesInPath iip) throws IOException {
assert hasReadLock(); assert hasReadLock();
long size = 0; // length is zero for directories long size = 0; // length is zero for directories
@ -2411,7 +2472,7 @@ public class FSDirectory implements Closeable {
getPermissionForFileStatus(node, snapshot, isEncrypted), getPermissionForFileStatus(node, snapshot, isEncrypted),
node.getUserName(snapshot), node.getGroupName(snapshot), node.getUserName(snapshot), node.getGroupName(snapshot),
node.isSymlink() ? node.asSymlink().getSymlink() : null, path, node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
node.getId(), loc, childrenNum, feInfo); node.getId(), loc, childrenNum, feInfo, storagePolicy);
// Set caching information for the located blocks. // Set caching information for the located blocks.
if (loc != null) { if (loc != null) {
CacheManager cacheManager = namesystem.getCacheManager(); CacheManager cacheManager = namesystem.getCacheManager();

View File

@ -83,6 +83,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetStoragePolicyOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetXAttrOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetXAttrOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
@ -830,6 +831,15 @@ public class FSEditLog implements LogsPurgeable {
logEdit(op); logEdit(op);
} }
/**
* Add set storage policy id record to edit log
*/
void logSetStoragePolicy(String src, byte policyId) {
SetStoragePolicyOp op = SetStoragePolicyOp.getInstance(cache.get())
.setPath(src).setPolicyId(policyId);
logEdit(op);
}
/** Add set namespace quota record to edit log /** Add set namespace quota record to edit log
* *
* @param src the string representation of the path to a directory * @param src the string representation of the path to a directory

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.fs.XAttrSetFlag;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@ -79,6 +80,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetStoragePolicyOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetXAttrOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetXAttrOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveXAttrOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveXAttrOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
@ -371,7 +373,8 @@ public class FSEditLogLoader {
// add the op into retry cache if necessary // add the op into retry cache if necessary
if (toAddRetryCache) { if (toAddRetryCache) {
HdfsFileStatus stat = fsNamesys.dir.createFileStatus( HdfsFileStatus stat = fsNamesys.dir.createFileStatus(
HdfsFileStatus.EMPTY_NAME, newFile, Snapshot.CURRENT_STATE_ID, HdfsFileStatus.EMPTY_NAME, newFile,
BlockStoragePolicy.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
false, iip); false, iip);
fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId, fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
addCloseOp.rpcCallId, stat); addCloseOp.rpcCallId, stat);
@ -838,6 +841,13 @@ public class FSEditLogLoader {
} }
break; break;
} }
case OP_SET_STORAGE_POLICY: {
SetStoragePolicyOp setStoragePolicyOp = (SetStoragePolicyOp) op;
fsDir.unprotectedSetStoragePolicy(
renameReservedPathsOnUpgrade(setStoragePolicyOp.path, logVersion),
setStoragePolicyOp.policyId);
break;
}
default: default:
throw new IOException("Invalid operation read " + op.opCode); throw new IOException("Invalid operation read " + op.opCode);
} }

View File

@ -61,6 +61,7 @@ import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SYMLINK
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_TIMES; import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_TIMES;
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_UPDATE_BLOCKS; import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_UPDATE_BLOCKS;
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_UPDATE_MASTER_KEY; import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_UPDATE_MASTER_KEY;
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_STORAGE_POLICY;
import java.io.DataInput; import java.io.DataInput;
import java.io.DataInputStream; import java.io.DataInputStream;
@ -195,6 +196,7 @@ public abstract class FSEditLogOp {
OP_ROLLING_UPGRADE_FINALIZE, "finalize")); OP_ROLLING_UPGRADE_FINALIZE, "finalize"));
inst.put(OP_SET_XATTR, new SetXAttrOp()); inst.put(OP_SET_XATTR, new SetXAttrOp());
inst.put(OP_REMOVE_XATTR, new RemoveXAttrOp()); inst.put(OP_REMOVE_XATTR, new RemoveXAttrOp());
inst.put(OP_SET_STORAGE_POLICY, new SetStoragePolicyOp());
} }
public FSEditLogOp get(FSEditLogOpCodes opcode) { public FSEditLogOp get(FSEditLogOpCodes opcode) {
@ -3800,6 +3802,71 @@ public abstract class FSEditLogOp {
} }
} }
/** {@literal @Idempotent} for {@link ClientProtocol#setStoragePolicy} */
static class SetStoragePolicyOp extends FSEditLogOp {
String path;
byte policyId;
private SetStoragePolicyOp() {
super(OP_SET_STORAGE_POLICY);
}
static SetStoragePolicyOp getInstance(OpInstanceCache cache) {
return (SetStoragePolicyOp) cache.get(OP_SET_STORAGE_POLICY);
}
SetStoragePolicyOp setPath(String path) {
this.path = path;
return this;
}
SetStoragePolicyOp setPolicyId(byte policyId) {
this.policyId = policyId;
return this;
}
@Override
public void writeFields(DataOutputStream out) throws IOException {
FSImageSerialization.writeString(path, out);
out.writeByte(policyId);
}
@Override
void readFields(DataInputStream in, int logVersion)
throws IOException {
this.path = FSImageSerialization.readString(in);
this.policyId = in.readByte();
}
@Override
public String toString() {
StringBuilder builder = new StringBuilder();
builder.append("SetStoragePolicyOp [path=");
builder.append(path);
builder.append(", policyId=");
builder.append(policyId);
builder.append(", opCode=");
builder.append(opCode);
builder.append(", txid=");
builder.append(txid);
builder.append("]");
return builder.toString();
}
@Override
protected void toXml(ContentHandler contentHandler) throws SAXException {
XMLUtils.addSaxString(contentHandler, "PATH", path);
XMLUtils.addSaxString(contentHandler, "POLICYID",
Byte.valueOf(policyId).toString());
}
@Override
void fromXml(Stanza st) throws InvalidXmlException {
this.path = st.getValue("PATH");
this.policyId = Byte.valueOf(st.getValue("POLICYID"));
}
}
/** /**
* Class for writing editlog ops * Class for writing editlog ops
*/ */

View File

@ -72,6 +72,7 @@ public enum FSEditLogOpCodes {
OP_ROLLING_UPGRADE_FINALIZE ((byte) 42), OP_ROLLING_UPGRADE_FINALIZE ((byte) 42),
OP_SET_XATTR ((byte) 43), OP_SET_XATTR ((byte) 43),
OP_REMOVE_XATTR ((byte) 44), OP_REMOVE_XATTR ((byte) 44),
OP_SET_STORAGE_POLICY ((byte) 45),
// Note that the current range of the valid OP code is 0~127 // Note that the current range of the valid OP code is 0~127
OP_INVALID ((byte) -1); OP_INVALID ((byte) -1);

View File

@ -786,7 +786,7 @@ public class FSImageFormat {
counter.increment(); counter.increment();
} }
final INodeFile file = new INodeFile(inodeId, localName, permissions, final INodeFile file = new INodeFile(inodeId, localName, permissions,
modificationTime, atime, blocks, replication, blockSize); modificationTime, atime, blocks, replication, blockSize, (byte)0);
if (underConstruction) { if (underConstruction) {
file.toUnderConstruction(clientName, clientMachine); file.toUnderConstruction(clientName, clientMachine);
} }
@ -890,7 +890,7 @@ public class FSImageFormat {
final long preferredBlockSize = in.readLong(); final long preferredBlockSize = in.readLong();
return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime, return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime,
accessTime, replication, preferredBlockSize, null); accessTime, replication, preferredBlockSize, (byte)0, null);
} }
public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in) public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in)

View File

@ -289,7 +289,8 @@ public final class FSImageFormatPBINode {
final INodeFile file = new INodeFile(n.getId(), final INodeFile file = new INodeFile(n.getId(),
n.getName().toByteArray(), permissions, f.getModificationTime(), n.getName().toByteArray(), permissions, f.getModificationTime(),
f.getAccessTime(), blocks, replication, f.getPreferredBlockSize()); f.getAccessTime(), blocks, replication, f.getPreferredBlockSize(),
(byte)f.getStoragePolicyID());
if (f.hasAcl()) { if (f.hasAcl()) {
file.addAclFeature(new AclFeature(loadAclEntries(f.getAcl(), file.addAclFeature(new AclFeature(loadAclEntries(f.getAcl(),
@ -398,7 +399,8 @@ public final class FSImageFormatPBINode {
.setModificationTime(file.getModificationTime()) .setModificationTime(file.getModificationTime())
.setPermission(buildPermissionStatus(file, state.getStringMap())) .setPermission(buildPermissionStatus(file, state.getStringMap()))
.setPreferredBlockSize(file.getPreferredBlockSize()) .setPreferredBlockSize(file.getPreferredBlockSize())
.setReplication(file.getFileReplication()); .setReplication(file.getFileReplication())
.setStoragePolicyID(file.getLocalStoragePolicyID());
AclFeature f = file.getAclFeature(); AclFeature f = file.getAclFeature();
if (f != null) { if (f != null) {

View File

@ -149,7 +149,7 @@ public class FSImageSerialization {
assert numLocs == 0 : "Unexpected block locations"; assert numLocs == 0 : "Unexpected block locations";
INodeFile file = new INodeFile(inodeId, name, perm, modificationTime, INodeFile file = new INodeFile(inodeId, name, perm, modificationTime,
modificationTime, blocks, blockReplication, preferredBlockSize); modificationTime, blocks, blockReplication, preferredBlockSize, (byte)0);
file.toUnderConstruction(clientName, clientMachine); file.toUnderConstruction(clientName, clientMachine);
return file; return file;
} }

View File

@ -166,6 +166,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.ServiceFailedException; import org.apache.hadoop.ha.ServiceFailedException;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HAUtil; import org.apache.hadoop.hdfs.HAUtil;
@ -325,7 +326,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
private HdfsFileStatus getAuditFileInfo(String path, boolean resolveSymlink) private HdfsFileStatus getAuditFileInfo(String path, boolean resolveSymlink)
throws IOException { throws IOException {
return (isAuditEnabled() && isExternalInvocation()) return (isAuditEnabled() && isExternalInvocation())
? dir.getFileInfo(path, resolveSymlink, false) : null; ? dir.getFileInfo(path, resolveSymlink, false, false) : null;
} }
private void logAuditEvent(boolean succeeded, String cmd, String src) private void logAuditEvent(boolean succeeded, String cmd, String src)
@ -2258,6 +2259,52 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
return isFile; return isFile;
} }
/**
* Set the storage policy for a file or a directory.
*
* @param src file/directory path
* @param policyName storage policy name
*/
void setStoragePolicy(String src, final String policyName)
throws IOException {
try {
setStoragePolicyInt(src, policyName);
} catch (AccessControlException e) {
logAuditEvent(false, "setStoragePolicy", src);
throw e;
}
}
private void setStoragePolicyInt(String src, final String policyName)
throws IOException {
checkSuperuserPrivilege();
checkOperation(OperationCategory.WRITE);
byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
waitForLoadingFSImage();
HdfsFileStatus fileStat;
writeLock();
try {
checkOperation(OperationCategory.WRITE);
checkNameNodeSafeMode("Cannot set storage policy for " + src);
src = FSDirectory.resolvePath(src, pathComponents, dir);
// get the corresponding policy and make sure the policy name is valid
BlockStoragePolicy policy = blockManager.getStoragePolicy(policyName);
if (policy == null) {
throw new HadoopIllegalArgumentException(
"Cannot find a block policy with the name " + policyName);
}
dir.setStoragePolicy(src, policy.getId());
getEditLog().logSetStoragePolicy(src, policy.getId());
fileStat = getAuditFileInfo(src, false);
} finally {
writeUnlock();
}
getEditLog().logSync();
logAuditEvent(true, "setStoragePolicy", src, null, fileStat);
}
long getPreferredBlockSize(String filename) long getPreferredBlockSize(String filename)
throws IOException, UnresolvedLinkException { throws IOException, UnresolvedLinkException {
FSPermissionChecker pc = getPermissionChecker(); FSPermissionChecker pc = getPermissionChecker();
@ -2495,7 +2542,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
clientMachine, create, overwrite, createParent, replication, clientMachine, create, overwrite, createParent, replication,
blockSize, suite, edek, logRetryCache); blockSize, suite, edek, logRetryCache);
stat = dir.getFileInfo(src, false, stat = dir.getFileInfo(src, false,
FSDirectory.isReservedRawName(srcArg)); FSDirectory.isReservedRawName(srcArg), false);
} catch (StandbyException se) { } catch (StandbyException se) {
skipSync = true; skipSync = true;
throw se; throw se;
@ -2967,8 +3014,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
throws LeaseExpiredException, NotReplicatedYetException, throws LeaseExpiredException, NotReplicatedYetException,
QuotaExceededException, SafeModeException, UnresolvedLinkException, QuotaExceededException, SafeModeException, UnresolvedLinkException,
IOException { IOException {
long blockSize; final long blockSize;
int replication; final int replication;
final byte storagePolicyID;
DatanodeDescriptor clientNode = null; DatanodeDescriptor clientNode = null;
if(NameNode.stateChangeLog.isDebugEnabled()) { if(NameNode.stateChangeLog.isDebugEnabled()) {
@ -3003,13 +3051,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
clientNode = blockManager.getDatanodeManager().getDatanodeByHost( clientNode = blockManager.getDatanodeManager().getDatanodeByHost(
pendingFile.getFileUnderConstructionFeature().getClientMachine()); pendingFile.getFileUnderConstructionFeature().getClientMachine());
replication = pendingFile.getFileReplication(); replication = pendingFile.getFileReplication();
storagePolicyID = pendingFile.getStoragePolicyID();
} finally { } finally {
readUnlock(); readUnlock();
} }
// choose targets for the new block to be allocated. // choose targets for the new block to be allocated.
final DatanodeStorageInfo targets[] = getBlockManager().chooseTarget( final DatanodeStorageInfo targets[] = getBlockManager().chooseTarget4NewBlock(
src, replication, clientNode, excludedNodes, blockSize, favoredNodes); src, replication, clientNode, excludedNodes, blockSize, favoredNodes,
storagePolicyID);
// Part II. // Part II.
// Allocate a new block, add it to the INode and the BlocksMap. // Allocate a new block, add it to the INode and the BlocksMap.
@ -3197,6 +3247,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
final DatanodeDescriptor clientnode; final DatanodeDescriptor clientnode;
final long preferredblocksize; final long preferredblocksize;
final byte storagePolicyID;
final List<DatanodeStorageInfo> chosen; final List<DatanodeStorageInfo> chosen;
checkOperation(OperationCategory.READ); checkOperation(OperationCategory.READ);
byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
@ -3223,6 +3274,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
.getClientMachine(); .getClientMachine();
clientnode = blockManager.getDatanodeManager().getDatanodeByHost(clientMachine); clientnode = blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
preferredblocksize = file.getPreferredBlockSize(); preferredblocksize = file.getPreferredBlockSize();
storagePolicyID = file.getStoragePolicyID();
//find datanode storages //find datanode storages
final DatanodeManager dm = blockManager.getDatanodeManager(); final DatanodeManager dm = blockManager.getDatanodeManager();
@ -3232,10 +3284,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
} }
// choose new datanodes. // choose new datanodes.
final DatanodeStorageInfo[] targets = blockManager.getBlockPlacementPolicy( final DatanodeStorageInfo[] targets = blockManager.chooseTarget4AdditionalDatanode(
).chooseTarget(src, numAdditionalNodes, clientnode, chosen, true, src, numAdditionalNodes, clientnode, chosen,
// TODO: get storage type from the file excludes, preferredblocksize, storagePolicyID);
excludes, preferredblocksize, StorageType.DEFAULT);
final LocatedBlock lb = new LocatedBlock(blk, targets); final LocatedBlock lb = new LocatedBlock(blk, targets);
blockManager.setBlockToken(lb, AccessMode.COPY); blockManager.setBlockToken(lb, AccessMode.COPY);
return lb; return lb;
@ -3922,12 +3973,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
try { try {
checkOperation(OperationCategory.READ); checkOperation(OperationCategory.READ);
src = resolvePath(src, pathComponents); src = resolvePath(src, pathComponents);
boolean isSuperUser = true;
if (isPermissionEnabled) { if (isPermissionEnabled) {
checkPermission(pc, src, false, null, null, null, null, false, checkPermission(pc, src, false, null, null, null, null, false,
resolveLink); resolveLink);
isSuperUser = pc.isSuperUser();
} }
stat = dir.getFileInfo(src, resolveLink, stat = dir.getFileInfo(src, resolveLink,
FSDirectory.isReservedRawName(srcArg)); FSDirectory.isReservedRawName(srcArg), isSuperUser);
} catch (AccessControlException e) { } catch (AccessControlException e) {
logAuditEvent(false, "getfileinfo", srcArg); logAuditEvent(false, "getfileinfo", srcArg);
throw e; throw e;
@ -4156,7 +4209,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
/** /**
* Get the content summary for a specific file/dir. * Get the content summary for a specific file/dir.
* *
* @param src The string representation of the path to the file * @param srcArg The string representation of the path to the file
* *
* @throws AccessControlException if access is denied * @throws AccessControlException if access is denied
* @throws UnresolvedLinkException if a symlink is encountered. * @throws UnresolvedLinkException if a symlink is encountered.
@ -4733,15 +4786,17 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
} }
} }
boolean isSuperUser = true;
if (isPermissionEnabled) { if (isPermissionEnabled) {
if (dir.isDir(src)) { if (dir.isDir(src)) {
checkPathAccess(pc, src, FsAction.READ_EXECUTE); checkPathAccess(pc, src, FsAction.READ_EXECUTE);
} else { } else {
checkTraverse(pc, src); checkTraverse(pc, src);
} }
isSuperUser = pc.isSuperUser();
} }
logAuditEvent(true, "listStatus", srcArg); logAuditEvent(true, "listStatus", srcArg);
dl = dir.getListing(src, startAfter, needLocation); dl = dir.getListing(src, startAfter, needLocation, isSuperUser);
} finally { } finally {
readUnlock(); readUnlock();
} }
@ -4891,12 +4946,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
/** /**
* Add the given symbolic link to the fs. Record it in the edits log. * Add the given symbolic link to the fs. Record it in the edits log.
* @param path
* @param target
* @param dirPerms
* @param createParent
* @param logRetryCache
* @param dir
*/ */
private INodeSymlink addSymlink(String path, String target, private INodeSymlink addSymlink(String path, String target,
PermissionStatus dirPerms, PermissionStatus dirPerms,

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
@ -684,6 +685,20 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
return this; return this;
} }
/**
* @return the latest block storage policy id of the INode. Specifically,
* if a storage policy is directly specified on the INode then return the ID
* of that policy. Otherwise follow the latest parental path and return the
* ID of the first specified storage policy.
*/
public abstract byte getStoragePolicyID();
/**
* @return the storage policy directly specified on the INode. Return
* {@link BlockStoragePolicy#ID_UNSPECIFIED} if no policy has
* been specified.
*/
public abstract byte getLocalStoragePolicyID();
/** /**
* Breaks {@code path} into components. * Breaks {@code path} into components.
@ -711,7 +726,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
* @throws AssertionError if the given path is invalid. * @throws AssertionError if the given path is invalid.
* @return array of path components. * @return array of path components.
*/ */
static String[] getPathNames(String path) { public static String[] getPathNames(String path) {
if (path == null || !path.startsWith(Path.SEPARATOR)) { if (path == null || !path.startsWith(Path.SEPARATOR)) {
throw new AssertionError("Absolute path required"); throw new AssertionError("Absolute path required");
} }

View File

@ -26,7 +26,9 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import org.apache.hadoop.fs.PathIsNotDirectoryException; import org.apache.hadoop.fs.PathIsNotDirectoryException;
import org.apache.hadoop.fs.XAttr;
import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.protocol.SnapshotException; import org.apache.hadoop.hdfs.protocol.SnapshotException;
@ -40,6 +42,7 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
/** /**
* Directory INode class. * Directory INode class.
@ -103,6 +106,30 @@ public class INodeDirectory extends INodeWithAdditionalFields
return this; return this;
} }
@Override
public byte getLocalStoragePolicyID() {
XAttrFeature f = getXAttrFeature();
ImmutableList<XAttr> xattrs = f == null ? ImmutableList.<XAttr> of() : f
.getXAttrs();
for (XAttr xattr : xattrs) {
if (BlockStoragePolicy.isStoragePolicyXAttr(xattr)) {
return (xattr.getValue())[0];
}
}
return BlockStoragePolicy.ID_UNSPECIFIED;
}
@Override
public byte getStoragePolicyID() {
byte id = getLocalStoragePolicyID();
if (id != BlockStoragePolicy.ID_UNSPECIFIED) {
return id;
}
// if it is unspecified, check its parent
return getParent() != null ? getParent().getStoragePolicyID() :
BlockStoragePolicy.ID_UNSPECIFIED;
}
void setQuota(long nsQuota, long dsQuota) { void setQuota(long nsQuota, long dsQuota) {
DirectoryWithQuotaFeature quota = getDirectoryWithQuotaFeature(); DirectoryWithQuotaFeature quota = getDirectoryWithQuotaFeature();
if (quota != null) { if (quota != null) {

View File

@ -18,10 +18,12 @@
package org.apache.hadoop.hdfs.server.namenode; package org.apache.hadoop.hdfs.server.namenode;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.XAttr;
import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.server.namenode.XAttrFeature; import org.apache.hadoop.hdfs.BlockStoragePolicy;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
/** /**
* The attributes of an inode. * The attributes of an inode.

View File

@ -28,6 +28,7 @@ import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
@ -71,10 +72,14 @@ public class INodeFile extends INodeWithAdditionalFields
return inode.asFile(); return inode.asFile();
} }
/** Format: [16 bits for replication][48 bits for PreferredBlockSize] */ /**
* Bit format:
* [4-bit storagePolicyID][12-bit replication][48-bit preferredBlockSize]
*/
static enum HeaderFormat { static enum HeaderFormat {
PREFERRED_BLOCK_SIZE(null, 48, 1), PREFERRED_BLOCK_SIZE(null, 48, 1),
REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 16, 1); REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 1),
STORAGE_POLICY_ID(REPLICATION.BITS, BlockStoragePolicy.ID_BIT_LENGTH, 0);
private final LongBitFormat BITS; private final LongBitFormat BITS;
@ -90,10 +95,16 @@ public class INodeFile extends INodeWithAdditionalFields
return PREFERRED_BLOCK_SIZE.BITS.retrieve(header); return PREFERRED_BLOCK_SIZE.BITS.retrieve(header);
} }
static long toLong(long preferredBlockSize, short replication) { static byte getStoragePolicyID(long header) {
return (byte)STORAGE_POLICY_ID.BITS.retrieve(header);
}
static long toLong(long preferredBlockSize, short replication,
byte storagePolicyID) {
long h = 0; long h = 0;
h = PREFERRED_BLOCK_SIZE.BITS.combine(preferredBlockSize, h); h = PREFERRED_BLOCK_SIZE.BITS.combine(preferredBlockSize, h);
h = REPLICATION.BITS.combine(replication, h); h = REPLICATION.BITS.combine(replication, h);
h = STORAGE_POLICY_ID.BITS.combine(storagePolicyID, h);
return h; return h;
} }
} }
@ -104,9 +115,10 @@ public class INodeFile extends INodeWithAdditionalFields
INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime, INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
long atime, BlockInfo[] blklist, short replication, long atime, BlockInfo[] blklist, short replication,
long preferredBlockSize) { long preferredBlockSize, byte storagePolicyID) {
super(id, name, permissions, mtime, atime); super(id, name, permissions, mtime, atime);
header = HeaderFormat.toLong(preferredBlockSize, replication); header = HeaderFormat.toLong(preferredBlockSize, replication,
storagePolicyID);
this.blocks = blklist; this.blocks = blklist;
} }
@ -160,7 +172,6 @@ public class INodeFile extends INodeWithAdditionalFields
return getFileUnderConstructionFeature() != null; return getFileUnderConstructionFeature() != null;
} }
/** Convert this file to an {@link INodeFileUnderConstruction}. */
INodeFile toUnderConstruction(String clientName, String clientMachine) { INodeFile toUnderConstruction(String clientName, String clientMachine) {
Preconditions.checkState(!isUnderConstruction(), Preconditions.checkState(!isUnderConstruction(),
"file is already under construction"); "file is already under construction");
@ -355,6 +366,32 @@ public class INodeFile extends INodeWithAdditionalFields
return HeaderFormat.getPreferredBlockSize(header); return HeaderFormat.getPreferredBlockSize(header);
} }
@Override
public byte getLocalStoragePolicyID() {
return HeaderFormat.getStoragePolicyID(header);
}
@Override
public byte getStoragePolicyID() {
byte id = getLocalStoragePolicyID();
if (id == BlockStoragePolicy.ID_UNSPECIFIED) {
return this.getParent() != null ?
this.getParent().getStoragePolicyID() : id;
}
return id;
}
private void setStoragePolicyID(byte storagePolicyId) {
header = HeaderFormat.STORAGE_POLICY_ID.BITS.combine(storagePolicyId,
header);
}
public final void setStoragePolicyID(byte storagePolicyId,
int latestSnapshotId) throws QuotaExceededException {
recordModification(latestSnapshotId);
setStoragePolicyID(storagePolicyId);
}
@Override @Override
public long getHeaderLong() { public long getHeaderLong() {
return header; return header;

View File

@ -38,6 +38,8 @@ public interface INodeFileAttributes extends INodeAttributes {
public boolean metadataEquals(INodeFileAttributes other); public boolean metadataEquals(INodeFileAttributes other);
public byte getLocalStoragePolicyID();
/** A copy of the inode file attributes */ /** A copy of the inode file attributes */
public static class SnapshotCopy extends INodeAttributes.SnapshotCopy public static class SnapshotCopy extends INodeAttributes.SnapshotCopy
implements INodeFileAttributes { implements INodeFileAttributes {
@ -45,10 +47,11 @@ public interface INodeFileAttributes extends INodeAttributes {
public SnapshotCopy(byte[] name, PermissionStatus permissions, public SnapshotCopy(byte[] name, PermissionStatus permissions,
AclFeature aclFeature, long modificationTime, long accessTime, AclFeature aclFeature, long modificationTime, long accessTime,
short replication, long preferredBlockSize, XAttrFeature xAttrsFeature) { short replication, long preferredBlockSize, byte storagePolicyID,
XAttrFeature xAttrsFeature) {
super(name, permissions, aclFeature, modificationTime, accessTime, super(name, permissions, aclFeature, modificationTime, accessTime,
xAttrsFeature); xAttrsFeature);
header = HeaderFormat.toLong(preferredBlockSize, replication); header = HeaderFormat.toLong(preferredBlockSize, replication, storagePolicyID);
} }
public SnapshotCopy(INodeFile file) { public SnapshotCopy(INodeFile file) {
@ -66,6 +69,11 @@ public interface INodeFileAttributes extends INodeAttributes {
return HeaderFormat.getPreferredBlockSize(header); return HeaderFormat.getPreferredBlockSize(header);
} }
@Override
public byte getLocalStoragePolicyID() {
return HeaderFormat.getStoragePolicyID(header);
}
@Override @Override
public long getHeaderLong() { public long getHeaderLong() {
return header; return header;

View File

@ -22,6 +22,7 @@ import java.util.List;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.namenode.Quota.Counts; import org.apache.hadoop.hdfs.server.namenode.Quota.Counts;
import org.apache.hadoop.util.GSet; import org.apache.hadoop.util.GSet;
@ -121,6 +122,16 @@ public class INodeMap {
boolean countDiffChange) throws QuotaExceededException { boolean countDiffChange) throws QuotaExceededException {
return null; return null;
} }
@Override
public byte getStoragePolicyID(){
return BlockStoragePolicy.ID_UNSPECIFIED;
}
@Override
public byte getLocalStoragePolicyID() {
return BlockStoragePolicy.ID_UNSPECIFIED;
}
}; };
return map.get(inode); return map.get(inode);

View File

@ -285,6 +285,16 @@ public abstract class INodeReference extends INode {
referred.setAccessTime(accessTime); referred.setAccessTime(accessTime);
} }
@Override
public final byte getStoragePolicyID() {
return referred.getStoragePolicyID();
}
@Override
public final byte getLocalStoragePolicyID() {
return referred.getLocalStoragePolicyID();
}
@Override @Override
final void recordModification(int latestSnapshotId) final void recordModification(int latestSnapshotId)
throws QuotaExceededException { throws QuotaExceededException {

View File

@ -145,4 +145,16 @@ public class INodeSymlink extends INodeWithAdditionalFields {
public void addXAttrFeature(XAttrFeature f) { public void addXAttrFeature(XAttrFeature f) {
throw new UnsupportedOperationException("XAttrs are not supported on symlinks"); throw new UnsupportedOperationException("XAttrs are not supported on symlinks");
} }
@Override
public byte getStoragePolicyID() {
throw new UnsupportedOperationException(
"Storage policy are not supported on symlinks");
}
@Override
public byte getLocalStoragePolicyID() {
throw new UnsupportedOperationException(
"Storage policy are not supported on symlinks");
}
} }

View File

@ -68,7 +68,8 @@ public class NameNodeLayoutVersion {
XATTRS(-57, "Extended attributes"), XATTRS(-57, "Extended attributes"),
CREATE_OVERWRITE(-58, "Use single editlog record for " + CREATE_OVERWRITE(-58, "Use single editlog record for " +
"creating file with overwrite"), "creating file with overwrite"),
XATTRS_NAMESPACE_EXT(-59, "Increase number of xattr namespaces"); XATTRS_NAMESPACE_EXT(-59, "Increase number of xattr namespaces"),
BLOCK_STORAGE_POLICY(-60, "Block Storage policy");
private final FeatureInfo info; private final FeatureInfo info;

View File

@ -581,6 +581,12 @@ class NameNodeRpcServer implements NamenodeProtocols {
return namesystem.setReplication(src, replication); return namesystem.setReplication(src, replication);
} }
@Override
public void setStoragePolicy(String src, String policyName)
throws IOException {
namesystem.setStoragePolicy(src, policyName);
}
@Override // ClientProtocol @Override // ClientProtocol
public void setPermission(String src, FsPermission permissions) public void setPermission(String src, FsPermission permissions)
throws IOException { throws IOException {

View File

@ -220,7 +220,8 @@ public class FSImageFormatPBSnapshot {
copy = new INodeFileAttributes.SnapshotCopy(pbf.getName() copy = new INodeFileAttributes.SnapshotCopy(pbf.getName()
.toByteArray(), permission, acl, fileInPb.getModificationTime(), .toByteArray(), permission, acl, fileInPb.getModificationTime(),
fileInPb.getAccessTime(), (short) fileInPb.getReplication(), fileInPb.getAccessTime(), (short) fileInPb.getReplication(),
fileInPb.getPreferredBlockSize(), xAttrs); fileInPb.getPreferredBlockSize(),
(byte)fileInPb.getStoragePolicyID(), xAttrs);
} }
FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null, FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,

View File

@ -26,7 +26,6 @@ import java.net.InetAddress;
import java.net.URI; import java.net.URI;
import java.net.URISyntaxException; import java.net.URISyntaxException;
import java.security.PrivilegedExceptionAction; import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
@ -223,11 +222,8 @@ public class NamenodeWebHdfsMethods {
final DatanodeDescriptor clientNode = bm.getDatanodeManager( final DatanodeDescriptor clientNode = bm.getDatanodeManager(
).getDatanodeByHost(getRemoteAddress()); ).getDatanodeByHost(getRemoteAddress());
if (clientNode != null) { if (clientNode != null) {
final DatanodeStorageInfo[] storages = bm.getBlockPlacementPolicy() final DatanodeStorageInfo[] storages = bm.chooseTarget4WebHDFS(
.chooseTarget(path, 1, clientNode, path, clientNode, excludes, blocksize);
new ArrayList<DatanodeStorageInfo>(), false, excludes, blocksize,
// TODO: get storage type from the file
StorageType.DEFAULT);
if (storages.length > 0) { if (storages.length > 0) {
return storages[0].getDatanodeDescriptor(); return storages[0].getDatanodeDescriptor();
} }

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hdfs.tools; package org.apache.hadoop.hdfs.tools;
import java.io.File; import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.io.PrintStream; import java.io.PrintStream;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
@ -48,6 +49,7 @@ import org.apache.hadoop.fs.FsStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.shell.Command; import org.apache.hadoop.fs.shell.Command;
import org.apache.hadoop.fs.shell.CommandFormat; import org.apache.hadoop.fs.shell.CommandFormat;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem;
@ -63,23 +65,24 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction; import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo; import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
import org.apache.hadoop.hdfs.protocol.SnapshotException; import org.apache.hadoop.hdfs.protocol.SnapshotException;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.TransferFsImage; import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RefreshCallQueueProtocol;
import org.apache.hadoop.ipc.GenericRefreshProtocol; import org.apache.hadoop.ipc.GenericRefreshProtocol;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RefreshCallQueueProtocol;
import org.apache.hadoop.ipc.RefreshResponse; import org.apache.hadoop.ipc.RefreshResponse;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.protocolPB.GenericRefreshProtocolClientSideTranslatorPB;
import org.apache.hadoop.ipc.protocolPB.GenericRefreshProtocolPB;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.RefreshUserMappingsProtocol; import org.apache.hadoop.security.RefreshUserMappingsProtocol;
import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol; import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
import org.apache.hadoop.ipc.protocolPB.GenericRefreshProtocolClientSideTranslatorPB;
import org.apache.hadoop.ipc.protocolPB.GenericRefreshProtocolPB;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
@ -390,6 +393,8 @@ public class DFSAdmin extends FsShell {
"\t[-shutdownDatanode <datanode_host:ipc_port> [upgrade]]\n" + "\t[-shutdownDatanode <datanode_host:ipc_port> [upgrade]]\n" +
"\t[-getDatanodeInfo <datanode_host:ipc_port>]\n" + "\t[-getDatanodeInfo <datanode_host:ipc_port>]\n" +
"\t[-metasave filename]\n" + "\t[-metasave filename]\n" +
"\t[-setStoragePolicy path policyName]\n" +
"\t[-getStoragePolicy path]\n" +
"\t[-help [cmd]]\n"; "\t[-help [cmd]]\n";
/** /**
@ -595,6 +600,32 @@ public class DFSAdmin extends FsShell {
return inSafeMode; return inSafeMode;
} }
public int setStoragePolicy(String[] argv) throws IOException {
DistributedFileSystem dfs = getDFS();
dfs.setStoragePolicy(new Path(argv[1]), argv[2]);
System.out.println("Set storage policy " + argv[2] + " on " + argv[1]);
return 0;
}
public int getStoragePolicy(String[] argv) throws IOException {
DistributedFileSystem dfs = getDFS();
HdfsFileStatus status = dfs.getClient().getFileInfo(argv[1]);
if (status == null) {
throw new FileNotFoundException("File/Directory does not exist: "
+ argv[1]);
}
byte storagePolicyId = status.getStoragePolicy();
BlockStoragePolicy.Suite suite = BlockStoragePolicy
.readBlockStorageSuite(getConf());
BlockStoragePolicy policy = suite.getPolicy(storagePolicyId);
if (policy != null) {
System.out.println("The storage policy of " + argv[1] + ":\n" + policy);
return 0;
} else {
throw new IOException("Cannot identify the storage policy for " + argv[1]);
}
}
/** /**
* Allow snapshot on a directory. * Allow snapshot on a directory.
* Usage: java DFSAdmin -allowSnapshot snapshotDir * Usage: java DFSAdmin -allowSnapshot snapshotDir
@ -942,6 +973,12 @@ public class DFSAdmin extends FsShell {
+ "\tGet the information about the given datanode. This command can\n" + "\tGet the information about the given datanode. This command can\n"
+ "\tbe used for checking if a datanode is alive.\n"; + "\tbe used for checking if a datanode is alive.\n";
String setStoragePolicy = "-setStoragePolicy path policyName\n"
+ "\tSet the storage policy for a file/directory.\n";
String getStoragePolicy = "-getStoragePolicy path\n"
+ "\tGet the storage policy for a file/directory.\n";
String help = "-help [cmd]: \tDisplays help for the given command or all commands if none\n" + String help = "-help [cmd]: \tDisplays help for the given command or all commands if none\n" +
"\t\tis specified.\n"; "\t\tis specified.\n";
@ -1001,6 +1038,10 @@ public class DFSAdmin extends FsShell {
System.out.println(shutdownDatanode); System.out.println(shutdownDatanode);
} else if ("getDatanodeInfo".equalsIgnoreCase(cmd)) { } else if ("getDatanodeInfo".equalsIgnoreCase(cmd)) {
System.out.println(getDatanodeInfo); System.out.println(getDatanodeInfo);
} else if ("setStoragePolicy".equalsIgnoreCase(cmd)) {
System.out.println(setStoragePolicy);
} else if ("getStoragePolicy".equalsIgnoreCase(cmd)) {
System.out.println(getStoragePolicy);
} else if ("help".equals(cmd)) { } else if ("help".equals(cmd)) {
System.out.println(help); System.out.println(help);
} else { } else {
@ -1033,6 +1074,8 @@ public class DFSAdmin extends FsShell {
System.out.println(disallowSnapshot); System.out.println(disallowSnapshot);
System.out.println(shutdownDatanode); System.out.println(shutdownDatanode);
System.out.println(getDatanodeInfo); System.out.println(getDatanodeInfo);
System.out.println(setStoragePolicy);
System.out.println(getStoragePolicy);
System.out.println(help); System.out.println(help);
System.out.println(); System.out.println();
ToolRunner.printGenericCommandUsage(System.out); ToolRunner.printGenericCommandUsage(System.out);
@ -1461,6 +1504,12 @@ public class DFSAdmin extends FsShell {
} else if ("-safemode".equals(cmd)) { } else if ("-safemode".equals(cmd)) {
System.err.println("Usage: hdfs dfsadmin" System.err.println("Usage: hdfs dfsadmin"
+ " [-safemode enter | leave | get | wait]"); + " [-safemode enter | leave | get | wait]");
} else if ("-setStoragePolicy".equals(cmd)) {
System.err.println("Usage: java DFSAdmin"
+ " [-setStoragePolicy path policyName]");
} else if ("-getStoragePolicy".equals(cmd)) {
System.err.println("Usage: java DFSAdmin"
+ " [-getStoragePolicy path]");
} else if ("-allowSnapshot".equalsIgnoreCase(cmd)) { } else if ("-allowSnapshot".equalsIgnoreCase(cmd)) {
System.err.println("Usage: hdfs dfsadmin" System.err.println("Usage: hdfs dfsadmin"
+ " [-allowSnapshot <snapshotDir>]"); + " [-allowSnapshot <snapshotDir>]");
@ -1677,6 +1726,16 @@ public class DFSAdmin extends FsShell {
printUsage(cmd); printUsage(cmd);
return exitCode; return exitCode;
} }
} else if ("-setStoragePolicy".equals(cmd)) {
if (argv.length != 3) {
printUsage(cmd);
return exitCode;
}
} else if ("-getStoragePolicy".equals(cmd)) {
if (argv.length != 2) {
printUsage(cmd);
return exitCode;
}
} }
// initialize DFSAdmin // initialize DFSAdmin
@ -1750,6 +1809,10 @@ public class DFSAdmin extends FsShell {
exitCode = getDatanodeInfo(argv, i); exitCode = getDatanodeInfo(argv, i);
} else if ("-reconfig".equals(cmd)) { } else if ("-reconfig".equals(cmd)) {
exitCode = reconfig(argv, i); exitCode = reconfig(argv, i);
} else if ("-setStoragePolicy".equals(cmd)) {
exitCode = setStoragePolicy(argv);
} else if ("-getStoragePolicy".equals(cmd)) {
exitCode = getStoragePolicy(argv);
} else if ("-help".equals(cmd)) { } else if ("-help".equals(cmd)) {
if (i < argv.length) { if (i < argv.length) {
printHelp(argv[i]); printHelp(argv[i]);

View File

@ -106,6 +106,15 @@ public class EnumCounters<E extends Enum<E>> {
} }
} }
/** @return the sum of all counters. */
public final long sum() {
long sum = 0;
for(int i = 0; i < counters.length; i++) {
sum += counters[i];
}
return sum;
}
@Override @Override
public boolean equals(Object obj) { public boolean equals(Object obj) {
if (obj == this) { if (obj == this) {

View File

@ -21,6 +21,7 @@ import org.apache.hadoop.fs.*;
import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.XAttrHelper; import org.apache.hadoop.hdfs.XAttrHelper;
import org.apache.hadoop.hdfs.protocol.*; import org.apache.hadoop.hdfs.protocol.*;
@ -230,6 +231,7 @@ public class JsonUtil {
m.put("replication", status.getReplication()); m.put("replication", status.getReplication());
m.put("fileId", status.getFileId()); m.put("fileId", status.getFileId());
m.put("childrenNum", status.getChildrenNum()); m.put("childrenNum", status.getChildrenNum());
m.put("storagePolicy", status.getStoragePolicy());
return includeType ? toJsonString(FileStatus.class, m): JSON.toString(m); return includeType ? toJsonString(FileStatus.class, m): JSON.toString(m);
} }
@ -260,10 +262,12 @@ public class JsonUtil {
Long childrenNumLong = (Long) m.get("childrenNum"); Long childrenNumLong = (Long) m.get("childrenNum");
final int childrenNum = (childrenNumLong == null) ? -1 final int childrenNum = (childrenNumLong == null) ? -1
: childrenNumLong.intValue(); : childrenNumLong.intValue();
final byte storagePolicy = m.containsKey("storagePolicy") ?
(byte) (long) (Long) m.get("storagePolicy") :
BlockStoragePolicy.ID_UNSPECIFIED;
return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication, return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication,
blockSize, mTime, aTime, permission, owner, group, blockSize, mTime, aTime, permission, owner, group, symlink,
symlink, DFSUtil.string2Bytes(localName), fileId, childrenNum, DFSUtil.string2Bytes(localName), fileId, childrenNum, null, storagePolicy);
null);
} }
/** Convert an ExtendedBlock to a Json map. */ /** Convert an ExtendedBlock to a Json map. */

View File

@ -100,6 +100,14 @@ message SetReplicationResponseProto {
required bool result = 1; required bool result = 1;
} }
message SetStoragePolicyRequestProto {
required string src = 1;
required string policyName = 2;
}
message SetStoragePolicyResponseProto { // void response
}
message SetPermissionRequestProto { message SetPermissionRequestProto {
required string src = 1; required string src = 1;
required FsPermissionProto permission = 2; required FsPermissionProto permission = 2;
@ -689,6 +697,8 @@ service ClientNamenodeProtocol {
rpc append(AppendRequestProto) returns(AppendResponseProto); rpc append(AppendRequestProto) returns(AppendResponseProto);
rpc setReplication(SetReplicationRequestProto) rpc setReplication(SetReplicationRequestProto)
returns(SetReplicationResponseProto); returns(SetReplicationResponseProto);
rpc setStoragePolicy(SetStoragePolicyRequestProto)
returns(SetStoragePolicyResponseProto);
rpc setPermission(SetPermissionRequestProto) rpc setPermission(SetPermissionRequestProto)
returns(SetPermissionResponseProto); returns(SetPermissionResponseProto);
rpc setOwner(SetOwnerRequestProto) returns(SetOwnerResponseProto); rpc setOwner(SetOwnerRequestProto) returns(SetOwnerResponseProto);

View File

@ -138,6 +138,7 @@ message INodeSection {
optional FileUnderConstructionFeature fileUC = 7; optional FileUnderConstructionFeature fileUC = 7;
optional AclFeatureProto acl = 8; optional AclFeatureProto acl = 8;
optional XAttrFeatureProto xAttrs = 9; optional XAttrFeatureProto xAttrs = 9;
optional uint32 storagePolicyID = 10;
} }
message INodeDirectory { message INodeDirectory {

View File

@ -158,6 +158,7 @@ message FsPermissionProto {
enum StorageTypeProto { enum StorageTypeProto {
DISK = 1; DISK = 1;
SSD = 2; SSD = 2;
ARCHIVE = 3;
} }
/** /**
@ -283,6 +284,8 @@ message HdfsFileStatusProto {
// Optional field for file encryption // Optional field for file encryption
optional FileEncryptionInfoProto fileEncryptionInfo = 15; optional FileEncryptionInfoProto fileEncryptionInfo = 15;
optional uint32 storagePolicy = 16 [default = 0]; // block storage policy id
} }
/** /**

View File

@ -0,0 +1,118 @@
<?xml version="1.0"?>
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<!--
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.
-->
<!-- Do not modify this file directly. Instead, copy entries that you wish -->
<!-- to modify from this file into blockStoragePolicy-site.xml and change -->
<!-- there. If blockStoragePolicy-site.xml does not exist, create it. -->
<configuration>
<property>
<name>dfs.block.storage.policies</name>
<value>HOT:12, WARM:8, COLD:4</value>
<description>
A list of block storage policy names and IDs. The syntax is
NAME_1:ID_1, NAME_2:ID_2, ..., NAME_n:ID_n
where ID is an integer in the range [1,15] and NAME is case insensitive.
The first element is the default policy. Empty list is not allowed.
</description>
</property>
<!-- Block Storage Policy HOT:12 -->
<property>
<name>dfs.block.storage.policy.12</name>
<value>DISK</value>
<description>
A list of storage types for storing the block replicas such as
STORAGE_TYPE_1, STORAGE_TYPE_2, ..., STORAGE_TYPE_n
When creating a block, the i-th replica is stored using i-th storage type
for i less than or equal to n, and
the j-th replica is stored using n-th storage type for j greater than n.
Empty list is not allowed.
Examples:
DISK : all replicas stored using DISK.
DISK, ARCHIVE : the first replica is stored using DISK and all the
remaining replicas are stored using ARCHIVE.
</description>
</property>
<property>
<name>dfs.block.storage.policy.creation-fallback.12</name>
<value></value>
<description>
A list of storage types for creation fallback storage.
STORAGE_TYPE_1, STORAGE_TYPE_2, ..., STORAGE_TYPE_n
When creating a block, if a particular storage type specified in the policy
is unavailable, the fallback STORAGE_TYPE_1 is used. Further, if
STORAGE_TYPE_i is also unavailable, the fallback STORAGE_TYPE_(i+1) is used.
In case that all fallback storages are unavailabe, the block will be created
with number of replicas less than the specified replication factor.
An empty list indicates that there is no fallback storage.
</description>
</property>
<property>
<name>dfs.block.storage.policy.replication-fallback.12</name>
<value>ARCHIVE</value>
<description>
Similar to dfs.block.storage.policy.creation-fallback.x but for replication.
</description>
</property>
<!-- Block Storage Policy WARM:8 -->
<property>
<name>dfs.block.storage.policy.8</name>
<value>DISK, ARCHIVE</value>
</property>
<property>
<name>dfs.block.storage.policy.creation-fallback.8</name>
<value>DISK, ARCHIVE</value>
</property>
<property>
<name>dfs.block.storage.policy.replication-fallback.8</name>
<value>DISK, ARCHIVE</value>
</property>
<!-- Block Storage Policy COLD:4 -->
<property>
<name>dfs.block.storage.policy.4</name>
<value>ARCHIVE</value>
</property>
<property>
<name>dfs.block.storage.policy.creation-fallback.4</name>
<value></value>
</property>
<property>
<name>dfs.block.storage.policy.replication-fallback.4</name>
<value></value>
</property>
</configuration>

View File

@ -22,7 +22,8 @@
<!-- wish to modify from this file into hdfs-site.xml and change them --> <!-- wish to modify from this file into hdfs-site.xml and change them -->
<!-- there. If hdfs-site.xml does not already exist, create it. --> <!-- there. If hdfs-site.xml does not already exist, create it. -->
<configuration> <configuration xmlns:xi="http://www.w3.org/2001/XInclude">
<xi:include href="blockStoragePolicy-default.xml" />
<property> <property>
<name>hadoop.hdfs.configuration.version</name> <name>hadoop.hdfs.configuration.version</name>

View File

@ -0,0 +1,302 @@
~~ Licensed 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. See accompanying LICENSE file.
---
HDFS Archival Storage
---
---
${maven.build.timestamp}
HDFS Archival Storage
%{toc|section=1|fromDepth=0}
* {Introduction}
<Archival Storage> is a solution to decouple growing storage capacity from compute capacity.
Nodes with higher density and less expensive storage with low compute power are becoming available
and can be used as cold storage in the clusters.
Based on policy the data from hot can be moved to the cold.
Adding more nodes to the cold storage can grow the storage independent of the compute capacity
in the cluster.
* {Storage Types and Storage Policies}
** {Storage Types: DISK, SSD and ARCHIVE}
The first phase of
{{{https://issues.apache.org/jira/browse/HDFS-2832}Heterogeneous Storage (HDFS-2832)}}
changed datanode storage model from a single storage,
which may correspond to multiple physical storage medias,
to a collection of storages with each storage corresponding to a physical storage media.
It also added the notion of storage types, DISK and SSD,
where DISK is the default storage type.
A new storage type <ARCHIVE>,
which has high storage density (petabyte of storage) but little compute power,
is added for supporting archival storage.
** {Storage Policies: Hot, Warm and Cold}
A new concept of storage policies is introduced in order to allow files to be stored
in different storage types according to the storage policy.
We have the following storage policies:
* <<Hot>> - for both storage and compute.
The data that is popular and still being used for processing will stay in this policy.
When a block is hot, all replicas are stored in DISK.
* <<Cold>> - only for storage with limited compute.
The data that is no longer being used, or data that needs to be archived is moved
from hot storage to cold storage.
When a block is cold, all replicas are stored in ARCHIVE.
* <<Warm>> - partially hot and partially cold.
When a block is warm, some of its replicas are stored in DISK
and the remaining replicas are stored in ARCHIVE.
[]
More formally, a storage policy consists of the following fields:
[[1]] Policy ID
[[2]] Policy name
[[3]] A list of storage types for block placement
[[4]] A list of fallback storage types for file creation
[[5]] A list of fallback storage types for replication
[]
When there is enough space,
block replicas are stored according to the storage type list specified in #3.
When some of the storage types in list #3 are running out of space,
the fallback storage type lists specified in #4 and #5 are used
to replace the out-of-space storage types for file creation and replication, respectively.
The following is a typical storage policy table.
*--------+---------------+-------------------------+-----------------------+-----------------------+
| <<Policy>> | <<Policy>>| <<Block Placement>> | <<Fallback storages>> | <<Fallback storages>> |
| <<ID>> | <<Name>> | <<(n\ replicas)>> | <<for creation>> | <<for replication>> |
*--------+---------------+-------------------------+-----------------------+-----------------------+
| 12 | Hot (default) | DISK: <n> | \<none\> | ARCHIVE |
*--------+---------------+-------------------------+-----------------------+-----------------------+
| 8 | Warm | DISK: 1, ARCHIVE: <n>-1 | ARCHIVE, DISK | ARCHIVE, DISK |
*--------+---------------+-------------------------+-----------------------+-----------------------+
| 4 | Cold | ARCHIVE: <n> | \<none\> | \<none\> |
*--------+---------------+-------------------------+-----------------------+-----------------------+
Note that cluster administrators may change the storage policy table
according to the characteristic of the cluster.
For example, in order to prevent losing archival data,
administrators may want to use DISK as fallback storage for replication in the Cold policy.
A drawback of such setting is that the DISK storages could be filled up with archival data.
As a result, the entire cluster may become full and cannot serve hot data anymore.
** {Configurations}
*** {Setting The List of All Storage Policies}
* <<dfs.block.storage.policies>>
- a list of block storage policy names and IDs.
The syntax is
NAME_1:ID_1, NAME_2:ID_2, ..., NAME_<n>:ID_<n>
where ID is an integer in the closed range [1,15] and NAME is case insensitive.
The first element is the <default policy>. Empty list is not allowed.
The default value is shown below.
+------------------------------------------+
<property>
<name>dfs.block.storage.policies</name>
<value>HOT:12, WARM:8, COLD:4</value>
</property>
+------------------------------------------+
[]
*** {Setting Storage Policy Details}
The following configuration properties are for setting the details of each storage policy,
where <<<\<ID\>>>> is the actual policy ID.
* <<dfs.block.storage.policy.\<ID\>>>
- a list of storage types for storing the block replicas.
The syntax is
STORAGE_TYPE_1, STORAGE_TYPE_2, ..., STORAGE_TYPE_<n>
When creating a block, the <i>-th replica is stored using <i>-th storage type
for <i> less than or equal to <n>, and
the <j>-th replica is stored using <n>-th storage type for <j> greater than <n>.
Empty list is not allowed.
Examples:
+------------------------------------------+
DISK : all replicas stored using DISK.
DISK, ARCHIVE : the first replica is stored using DISK and all the
remaining replicas are stored using ARCHIVE.
+------------------------------------------+
* <<dfs.block.storage.policy.creation-fallback.\<ID\>>>
- a list of storage types for creation fallback storage.
The syntax is
STORAGE_TYPE_1, STORAGE_TYPE_2, ..., STORAGE_TYPE_n
When creating a block, if a particular storage type specified in the policy
is unavailable, the fallback STORAGE_TYPE_1 is used. Further, if
STORAGE_TYPE_<i> is also unavailable, the fallback STORAGE_TYPE_<(i+1)> is used.
In case all fallback storages are unavailable, the block will be created
with number of replicas less than the specified replication factor.
An empty list indicates that there is no fallback storage.
* <<dfs.block.storage.policy.replication-fallback.\<ID\>>>
- a list of storage types for replication fallback storage.
The usage of this configuration property is similar to
<<<dfs.block.storage.policy.creation-fallback.\<ID\>>>>
except that it takes effect on replication but not block creation.
[]
The following are the default configuration values for Hot, Warm and Cold storage policies.
* Block Storage Policy <<HOT:12>>
+------------------------------------------+
<property>
<name>dfs.block.storage.policy.12</name>
<value>DISK</value>
</property>
<property>
<name>dfs.block.storage.policy.creation-fallback.12</name>
<value></value>
</property>
<property>
<name>dfs.block.storage.policy.replication-fallback.12</name>
<value>ARCHIVE</value>
</property>
+------------------------------------------+
* Block Storage Policy <<WARM:8>>
+------------------------------------------+
<property>
<name>dfs.block.storage.policy.8</name>
<value>DISK, ARCHIVE</value>
</property>
<property>
<name>dfs.block.storage.policy.creation-fallback.8</name>
<value>DISK, ARCHIVE</value>
</property>
<property>
<name>dfs.block.storage.policy.replication-fallback.8</name>
<value>DISK, ARCHIVE</value>
</property>
+------------------------------------------+
* Block Storage Policy <<COLD:4>>
+------------------------------------------+
<property>
<name>dfs.block.storage.policy.4</name>
<value>ARCHIVE</value>
</property>
<property>
<name>dfs.block.storage.policy.creation-fallback.4</name>
<value></value>
</property>
<property>
<name>dfs.block.storage.policy.replication-fallback.4</name>
<value></value>
</property>
+------------------------------------------+
[]
* {Mover - A New Data Migration Tool}
A new data migration tool is added for archiving data.
The tool is similar to Balancer.
It periodically scans the files in HDFS to check if the block placement satisfies the storage policy.
For the blocks violating the storage policy,
it moves the replicas to a different storage type
in order to fulfill the storage policy requirement.
* Command:
+------------------------------------------+
hdfs mover [-p <files/dirs> | -f <local file name>]
+------------------------------------------+
* Arguments:
*-------------------------+--------------------------------------------------------+
| <<<-p \<files/dirs\>>>> | Specify a space separated list of HDFS files/dirs to migrate.
*-------------------------+--------------------------------------------------------+
| <<<-f \<local file\>>>> | Specify a local file containing a list of HDFS files/dirs to migrate.
*-------------------------+--------------------------------------------------------+
Note that, when both -p and -f options are omitted, the default path is the root directory.
[]
* {<<<DFSAdmin>>> Commands}
** {Set Storage Policy}
Set a storage policy to a file or a directory.
* Command:
+------------------------------------------+
hdfs dfsadmin -setStoragePolicy <path> <policyName>
+------------------------------------------+
* Arguments:
*----------------------+-----------------------------------------------------+
| <<<\<path\>>>> | The path referring to either a directory or a file. |
*----------------------+-----------------------------------------------------+
| <<<\<policyName\>>>> | The name of the storage policy. |
*----------------------+-----------------------------------------------------+
[]
** {Get Storage Policy}
Get the storage policy of a file or a directory.
* Command:
+------------------------------------------+
hdfs dfsadmin -getStoragePolicy <path>
+------------------------------------------+
* Arguments:
*----------------------+-----------------------------------------------------+
| <<<\<path\>>>> | The path referring to either a directory or a file. |
*----------------------+-----------------------------------------------------+
[]

View File

@ -143,18 +143,19 @@ HDFS Commands Guide
*-----------------+-----------------------------------------------------------+ *-----------------+-----------------------------------------------------------+
| -regular | Normal datanode startup (default). | -regular | Normal datanode startup (default).
*-----------------+-----------------------------------------------------------+ *-----------------+-----------------------------------------------------------+
| -rollback | Rollsback the datanode to the previous version. This should | -rollback | Rollback the datanode to the previous version. This should
| | be used after stopping the datanode and distributing the | | be used after stopping the datanode and distributing the
| | old hadoop version. | | old hadoop version.
*-----------------+-----------------------------------------------------------+ *-----------------+-----------------------------------------------------------+
| -rollingupgrade rollback | Rollsback a rolling upgrade operation. | -rollingupgrade rollback | Rollback a rolling upgrade operation.
*-----------------+-----------------------------------------------------------+ *-----------------+-----------------------------------------------------------+
** <<<dfsadmin>>> ** <<<dfsadmin>>>
Runs a HDFS dfsadmin client. Runs a HDFS dfsadmin client.
Usage: <<<hdfs dfsadmin [GENERIC_OPTIONS] +------------------------------------------+
Usage: hdfs dfsadmin [GENERIC_OPTIONS]
[-report [-live] [-dead] [-decommissioning]] [-report [-live] [-dead] [-decommissioning]]
[-safemode enter | leave | get | wait] [-safemode enter | leave | get | wait]
[-saveNamespace] [-saveNamespace]
@ -165,6 +166,8 @@ HDFS Commands Guide
[-clrQuota <dirname>...<dirname>] [-clrQuota <dirname>...<dirname>]
[-setSpaceQuota <quota> <dirname>...<dirname>] [-setSpaceQuota <quota> <dirname>...<dirname>]
[-clrSpaceQuota <dirname>...<dirname>] [-clrSpaceQuota <dirname>...<dirname>]
[-setStoragePolicy <path> <policyName>]
[-getStoragePolicy <path>]
[-finalizeUpgrade] [-finalizeUpgrade]
[-rollingUpgrade [<query>|<prepare>|<finalize>]] [-rollingUpgrade [<query>|<prepare>|<finalize>]]
[-metasave filename] [-metasave filename]
@ -182,7 +185,8 @@ HDFS Commands Guide
[-fetchImage <local directory>] [-fetchImage <local directory>]
[-shutdownDatanode <datanode_host:ipc_port> [upgrade]] [-shutdownDatanode <datanode_host:ipc_port> [upgrade]]
[-getDatanodeInfo <datanode_host:ipc_port>] [-getDatanodeInfo <datanode_host:ipc_port>]
[-help [cmd]]>>> [-help [cmd]]
+------------------------------------------+
*-----------------+-----------------------------------------------------------+ *-----------------+-----------------------------------------------------------+
|| COMMAND_OPTION || Description || COMMAND_OPTION || Description
@ -232,6 +236,10 @@ HDFS Commands Guide
| {{{../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands}HDFS Quotas Guide}} | {{{../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands}HDFS Quotas Guide}}
| for the detail. | for the detail.
*-----------------+-----------------------------------------------------------+ *-----------------+-----------------------------------------------------------+
| -setStoragePolicy \<path\> \<policyName\> | Set a storage policy to a file or a directory.
*-----------------+-----------------------------------------------------------+
| -getStoragePolicy \<path\> | Get the storage policy of a file or a directory.
*-----------------+-----------------------------------------------------------+
| -finalizeUpgrade| Finalize upgrade of HDFS. Datanodes delete their previous | -finalizeUpgrade| Finalize upgrade of HDFS. Datanodes delete their previous
| version working directories, followed by Namenode doing the | version working directories, followed by Namenode doing the
| same. This completes the upgrade process. | same. This completes the upgrade process.
@ -246,7 +254,7 @@ HDFS Commands Guide
| <filename> will contain one line for each of the following\ | <filename> will contain one line for each of the following\
| 1. Datanodes heart beating with Namenode\ | 1. Datanodes heart beating with Namenode\
| 2. Blocks waiting to be replicated\ | 2. Blocks waiting to be replicated\
| 3. Blocks currrently being replicated\ | 3. Blocks currently being replicated\
| 4. Blocks waiting to be deleted | 4. Blocks waiting to be deleted
*-----------------+-----------------------------------------------------------+ *-----------------+-----------------------------------------------------------+
| -refreshServiceAcl | Reload the service-level authorization policy file. | -refreshServiceAcl | Reload the service-level authorization policy file.
@ -308,12 +316,30 @@ HDFS Commands Guide
| is specified. | is specified.
*-----------------+-----------------------------------------------------------+ *-----------------+-----------------------------------------------------------+
** <<<mover>>>
Runs the data migration utility.
See {{{./ArchivalStorage.html#Mover_-_A_New_Data_Migration_Tool}Mover}} for more details.
Usage: <<<hdfs mover [-p <files/dirs> | -f <local file name>]>>>
*--------------------+--------------------------------------------------------+
|| COMMAND_OPTION || Description
*--------------------+--------------------------------------------------------+
| -p \<files/dirs\> | Specify a space separated list of HDFS files/dirs to migrate.
*--------------------+--------------------------------------------------------+
| -f \<local file\> | Specify a local file containing a list of HDFS files/dirs to migrate.
*--------------------+--------------------------------------------------------+
Note that, when both -p and -f options are omitted, the default path is the root directory.
** <<<namenode>>> ** <<<namenode>>>
Runs the namenode. More info about the upgrade, rollback and finalize is at Runs the namenode. More info about the upgrade, rollback and finalize is at
{{{./HdfsUserGuide.html#Upgrade_and_Rollback}Upgrade Rollback}}. {{{./HdfsUserGuide.html#Upgrade_and_Rollback}Upgrade Rollback}}.
Usage: <<<hdfs namenode [-backup] | +------------------------------------------+
Usage: hdfs namenode [-backup] |
[-checkpoint] | [-checkpoint] |
[-format [-clusterid cid ] [-force] [-nonInteractive] ] | [-format [-clusterid cid ] [-force] [-nonInteractive] ] |
[-upgrade [-clusterid cid] [-renameReserved<k-v pairs>] ] | [-upgrade [-clusterid cid] [-renameReserved<k-v pairs>] ] |
@ -325,7 +351,8 @@ HDFS Commands Guide
[-initializeSharedEdits] | [-initializeSharedEdits] |
[-bootstrapStandby] | [-bootstrapStandby] |
[-recover [-force] ] | [-recover [-force] ] |
[-metadataVersion ]>>> [-metadataVersion ]
+------------------------------------------+
*--------------------+--------------------------------------------------------+ *--------------------+--------------------------------------------------------+
|| COMMAND_OPTION || Description || COMMAND_OPTION || Description
@ -347,7 +374,7 @@ HDFS Commands Guide
| -upgradeOnly [-clusterid cid] [-renameReserved\<k-v pairs\>] | Upgrade the | -upgradeOnly [-clusterid cid] [-renameReserved\<k-v pairs\>] | Upgrade the
| specified NameNode and then shutdown it. | specified NameNode and then shutdown it.
*--------------------+--------------------------------------------------------+ *--------------------+--------------------------------------------------------+
| -rollback | Rollsback the NameNode to the previous version. This | -rollback | Rollback the NameNode to the previous version. This
| should be used after stopping the cluster and | should be used after stopping the cluster and
| distributing the old Hadoop version. | distributing the old Hadoop version.
*--------------------+--------------------------------------------------------+ *--------------------+--------------------------------------------------------+

View File

@ -24,6 +24,8 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -66,6 +68,7 @@ import org.apache.hadoop.hdfs.server.namenode.ha
.ConfiguredFailoverProxyProvider; .ConfiguredFailoverProxyProvider;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.tools.DFSAdmin;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
@ -75,6 +78,8 @@ import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.util.VersionInfo;
import org.junit.Assume; import org.junit.Assume;
@ -88,8 +93,7 @@ import java.security.PrivilegedExceptionAction;
import java.util.*; import java.util.*;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
@ -893,6 +897,37 @@ public class DFSTestUtil {
.join(nameservices)); .join(nameservices));
} }
public static void setFederatedHAConfiguration(MiniDFSCluster cluster,
Configuration conf) {
Map<String, List<String>> nameservices = Maps.newHashMap();
for (NameNodeInfo info : cluster.getNameNodeInfos()) {
Preconditions.checkState(info.nameserviceId != null);
List<String> nns = nameservices.get(info.nameserviceId);
if (nns == null) {
nns = Lists.newArrayList();
nameservices.put(info.nameserviceId, nns);
}
nns.add(info.nnId);
conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
info.nameserviceId, info.nnId),
DFSUtil.createUri(HdfsConstants.HDFS_URI_SCHEME,
info.nameNode.getNameNodeAddress()).toString());
conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
info.nameserviceId, info.nnId),
DFSUtil.createUri(HdfsConstants.HDFS_URI_SCHEME,
info.nameNode.getNameNodeAddress()).toString());
}
for (Map.Entry<String, List<String>> entry : nameservices.entrySet()) {
conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX,
entry.getKey()), Joiner.on(",").join(entry.getValue()));
conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + entry
.getKey(), ConfiguredFailoverProxyProvider.class.getName());
}
conf.set(DFSConfigKeys.DFS_NAMESERVICES, Joiner.on(",")
.join(nameservices.keySet()));
}
private static DatanodeID getDatanodeID(String ipAddr) { private static DatanodeID getDatanodeID(String ipAddr) {
return new DatanodeID(ipAddr, "localhost", return new DatanodeID(ipAddr, "localhost",
UUID.randomUUID().toString(), UUID.randomUUID().toString(),
@ -978,6 +1013,11 @@ public class DFSTestUtil {
public static DatanodeStorageInfo[] createDatanodeStorageInfos( public static DatanodeStorageInfo[] createDatanodeStorageInfos(
int n, String[] racks, String[] hostnames) { int n, String[] racks, String[] hostnames) {
return createDatanodeStorageInfos(n, racks, hostnames, null);
}
public static DatanodeStorageInfo[] createDatanodeStorageInfos(
int n, String[] racks, String[] hostnames, StorageType[] types) {
DatanodeStorageInfo[] storages = new DatanodeStorageInfo[n]; DatanodeStorageInfo[] storages = new DatanodeStorageInfo[n];
for(int i = storages.length; i > 0; ) { for(int i = storages.length; i > 0; ) {
final String storageID = "s" + i; final String storageID = "s" + i;
@ -985,16 +1025,30 @@ public class DFSTestUtil {
i--; i--;
final String rack = (racks!=null && i < racks.length)? racks[i]: "defaultRack"; final String rack = (racks!=null && i < racks.length)? racks[i]: "defaultRack";
final String hostname = (hostnames!=null && i < hostnames.length)? hostnames[i]: "host"; final String hostname = (hostnames!=null && i < hostnames.length)? hostnames[i]: "host";
storages[i] = createDatanodeStorageInfo(storageID, ip, rack, hostname); final StorageType type = (types != null && i < types.length) ? types[i]
: StorageType.DEFAULT;
storages[i] = createDatanodeStorageInfo(storageID, ip, rack, hostname,
type);
} }
return storages; return storages;
} }
public static DatanodeStorageInfo createDatanodeStorageInfo( public static DatanodeStorageInfo createDatanodeStorageInfo(
String storageID, String ip, String rack, String hostname) { String storageID, String ip, String rack, String hostname) {
final DatanodeStorage storage = new DatanodeStorage(storageID); return createDatanodeStorageInfo(storageID, ip, rack, hostname,
final DatanodeDescriptor dn = BlockManagerTestUtil.getDatanodeDescriptor(ip, rack, storage, hostname); StorageType.DEFAULT);
}
public static DatanodeStorageInfo createDatanodeStorageInfo(
String storageID, String ip, String rack, String hostname,
StorageType type) {
final DatanodeStorage storage = new DatanodeStorage(storageID,
DatanodeStorage.State.NORMAL, type);
final DatanodeDescriptor dn = BlockManagerTestUtil.getDatanodeDescriptor(
ip, rack, storage, hostname);
return BlockManagerTestUtil.newDatanodeStorageInfo(dn, storage); return BlockManagerTestUtil.newDatanodeStorageInfo(dn, storage);
} }
public static DatanodeDescriptor[] toDatanodeDescriptor( public static DatanodeDescriptor[] toDatanodeDescriptor(
DatanodeStorageInfo[] storages) { DatanodeStorageInfo[] storages) {
DatanodeDescriptor[] datanodes = new DatanodeDescriptor[storages.length]; DatanodeDescriptor[] datanodes = new DatanodeDescriptor[storages.length];
@ -1081,6 +1135,8 @@ public class DFSTestUtil {
FSDataOutputStream s = filesystem.create(pathFileCreate); FSDataOutputStream s = filesystem.create(pathFileCreate);
// OP_CLOSE 9 // OP_CLOSE 9
s.close(); s.close();
// OP_SET_STORAGE_POLICY 45
filesystem.setStoragePolicy(pathFileCreate, "HOT");
// OP_RENAME_OLD 1 // OP_RENAME_OLD 1
final Path pathFileMoved = new Path("/file_moved"); final Path pathFileMoved = new Path("/file_moved");
filesystem.rename(pathFileCreate, pathFileMoved); filesystem.rename(pathFileCreate, pathFileMoved);
@ -1441,6 +1497,57 @@ public class DFSTestUtil {
return expectedPrimary.getDatanodeDescriptor(); return expectedPrimary.getDatanodeDescriptor();
} }
public static void toolRun(Tool tool, String cmd, int retcode, String contain)
throws Exception {
String [] cmds = StringUtils.split(cmd, ' ');
System.out.flush();
System.err.flush();
PrintStream origOut = System.out;
PrintStream origErr = System.err;
String output = null;
int ret = 0;
try {
ByteArrayOutputStream bs = new ByteArrayOutputStream(1024);
PrintStream out = new PrintStream(bs);
System.setOut(out);
System.setErr(out);
ret = tool.run(cmds);
System.out.flush();
System.err.flush();
out.close();
output = bs.toString();
} finally {
System.setOut(origOut);
System.setErr(origErr);
}
System.out.println("Output for command: " + cmd + " retcode: " + ret);
if (output != null) {
System.out.println(output);
}
assertEquals(retcode, ret);
if (contain != null) {
assertTrue("The real output is: " + output + ".\n It should contain: "
+ contain, output.contains(contain));
}
}
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 {
DFSAdmin admin = new DFSAdmin(new Configuration(conf));
toolRun(admin, cmd, retcode, contain);
}
public static void FsShellRun(String cmd, Configuration conf)
throws Exception {
FsShellRun(cmd, 0, null, conf);
}
public static void addDataNodeLayoutVersion(final int lv, final String description) public static void addDataNodeLayoutVersion(final int lv, final String description)
throws NoSuchFieldException, IllegalAccessException { throws NoSuchFieldException, IllegalAccessException {
Preconditions.checkState(lv < DataNodeLayoutVersion.CURRENT_LAYOUT_VERSION); Preconditions.checkState(lv < DataNodeLayoutVersion.CURRENT_LAYOUT_VERSION);

View File

@ -1477,19 +1477,21 @@ public class MiniDFSCluster {
secureResources, dn.getIpcPort())); secureResources, dn.getIpcPort()));
dns[i - curDatanodesNum] = dn; dns[i - curDatanodesNum] = dn;
} }
curDatanodesNum += numDataNodes;
this.numDataNodes += numDataNodes; this.numDataNodes += numDataNodes;
waitActive(); waitActive();
if (storageCapacities != null) { if (storageCapacities != null) {
for (int i = curDatanodesNum; i < curDatanodesNum+numDataNodes; ++i) { for (int i = curDatanodesNum; i < curDatanodesNum+numDataNodes; ++i) {
List<? extends FsVolumeSpi> volumes = dns[i].getFSDataset().getVolumes(); final int index = i - curDatanodesNum;
assert storageCapacities[i].length == storagesPerDatanode; List<? extends FsVolumeSpi> volumes = dns[index].getFSDataset().getVolumes();
assert storageCapacities[index].length == storagesPerDatanode;
assert volumes.size() == storagesPerDatanode; assert volumes.size() == storagesPerDatanode;
for (int j = 0; j < volumes.size(); ++j) { for (int j = 0; j < volumes.size(); ++j) {
FsVolumeImpl volume = (FsVolumeImpl) volumes.get(j); FsVolumeImpl volume = (FsVolumeImpl) volumes.get(j);
volume.setCapacityForTesting(storageCapacities[i][j]); LOG.info("setCapacityForTesting " + storageCapacities[index][j]
+ " for [" + volume.getStorageType() + "]" + volume.getStorageID());
volume.setCapacityForTesting(storageCapacities[index][j]);
} }
} }
} }

View File

@ -255,12 +255,12 @@ public class TestDFSClientRetries {
Mockito.doReturn( Mockito.doReturn(
new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
(short) 777), "owner", "group", new byte[0], new byte[0], (short) 777), "owner", "group", new byte[0], new byte[0],
1010, 0, null)).when(mockNN).getFileInfo(anyString()); 1010, 0, null, (byte) 0)).when(mockNN).getFileInfo(anyString());
Mockito.doReturn( Mockito.doReturn(
new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
(short) 777), "owner", "group", new byte[0], new byte[0], (short) 777), "owner", "group", new byte[0], new byte[0],
1010, 0, null)) 1010, 0, null, (byte) 0))
.when(mockNN) .when(mockNN)
.create(anyString(), (FsPermission) anyObject(), anyString(), .create(anyString(), (FsPermission) anyObject(), anyString(),
(EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(), (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),

View File

@ -64,7 +64,7 @@ public class TestDFSInotifyEventInputStream {
*/ */
@Test @Test
public void testOpcodeCount() { public void testOpcodeCount() {
Assert.assertTrue(FSEditLogOpCodes.values().length == 46); Assert.assertTrue(FSEditLogOpCodes.values().length == 47);
} }

View File

@ -17,12 +17,14 @@
*/ */
package org.apache.hadoop.hdfs; package org.apache.hadoop.hdfs;
import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Matchers.anyList; import static org.mockito.Matchers.anyList;
import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.anyString;
import static org.mockito.Matchers.anyShort; import static org.mockito.Matchers.anyShort;
import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.anyLong;
import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Matchers.anyObject; import static org.mockito.Matchers.anyObject;
import static org.mockito.Matchers.anyShort;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.spy; import static org.mockito.Mockito.spy;
@ -341,12 +343,12 @@ public class TestLease {
Mockito.doReturn( Mockito.doReturn(
new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
(short) 777), "owner", "group", new byte[0], new byte[0], (short) 777), "owner", "group", new byte[0], new byte[0],
1010, 0, null)).when(mcp).getFileInfo(anyString()); 1010, 0, null, (byte) 0)).when(mcp).getFileInfo(anyString());
Mockito Mockito
.doReturn( .doReturn(
new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
(short) 777), "owner", "group", new byte[0], new byte[0], (short) 777), "owner", "group", new byte[0], new byte[0],
1010, 0, null)) 1010, 0, null, (byte) 0))
.when(mcp) .when(mcp)
.create(anyString(), (FsPermission) anyObject(), anyString(), .create(anyString(), (FsPermission) anyObject(), anyString(),
(EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(), (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),

View File

@ -18,22 +18,11 @@
package org.apache.hadoop.hdfs; package org.apache.hadoop.hdfs;
import static org.junit.Assert.*;
import java.io.ByteArrayOutputStream;
import java.io.IOException; import java.io.IOException;
import java.io.PrintStream;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FsShell;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.tools.DFSAdmin;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
@ -89,136 +78,87 @@ public class TestSnapshotCommands {
} }
} }
private void toolRun(Tool tool, String cmd, int retcode, String contain)
throws Exception {
String [] cmds = StringUtils.split(cmd, ' ');
System.out.flush();
System.err.flush();
PrintStream origOut = System.out;
PrintStream origErr = System.err;
String output = null;
int ret = 0;
try {
ByteArrayOutputStream bs = new ByteArrayOutputStream(1024);
PrintStream out = new PrintStream(bs);
System.setOut(out);
System.setErr(out);
ret = tool.run(cmds);
System.out.flush();
System.err.flush();
out.close();
output = bs.toString();
} finally {
System.setOut(origOut);
System.setErr(origErr);
}
System.out.println("Output for command: " + cmd + " retcode: " + ret);
if (output != null) {
System.out.println(output);
}
assertEquals(retcode, ret);
if (contain != null) {
assertTrue(output.contains(contain));
}
}
private void FsShellRun(String cmd, int retcode, String contain)
throws Exception {
FsShell shell = new FsShell(new Configuration(conf));
toolRun(shell, cmd, retcode, contain);
}
private void DFSAdminRun(String cmd, int retcode, String contain)
throws Exception {
DFSAdmin admin = new DFSAdmin(new Configuration(conf));
toolRun(admin, cmd, retcode, contain);
}
private void FsShellRun(String cmd) throws Exception {
FsShellRun(cmd, 0, null);
}
@Test @Test
public void testAllowSnapshot() throws Exception { public void testAllowSnapshot() throws Exception {
// Idempotent test // Idempotent test
DFSAdminRun("-allowSnapshot /sub1", 0, "Allowing snaphot on /sub1 succeeded"); DFSTestUtil.DFSAdminRun("-allowSnapshot /sub1", 0, "Allowing snaphot on /sub1 succeeded", conf);
// allow normal dir success // allow normal dir success
FsShellRun("-mkdir /sub2"); DFSTestUtil.FsShellRun("-mkdir /sub2", conf);
DFSAdminRun("-allowSnapshot /sub2", 0, "Allowing snaphot on /sub2 succeeded"); DFSTestUtil.DFSAdminRun("-allowSnapshot /sub2", 0, "Allowing snaphot on /sub2 succeeded", conf);
// allow non-exists dir failed // allow non-exists dir failed
DFSAdminRun("-allowSnapshot /sub3", -1, null); DFSTestUtil.DFSAdminRun("-allowSnapshot /sub3", -1, null, conf);
} }
@Test @Test
public void testCreateSnapshot() throws Exception { public void testCreateSnapshot() throws Exception {
// test createSnapshot // test createSnapshot
FsShellRun("-createSnapshot /sub1 sn0", 0, "Created snapshot /sub1/.snapshot/sn0"); DFSTestUtil.FsShellRun("-createSnapshot /sub1 sn0", 0, "Created snapshot /sub1/.snapshot/sn0", conf);
FsShellRun("-createSnapshot /sub1 sn0", 1, "there is already a snapshot with the same name \"sn0\""); DFSTestUtil.FsShellRun("-createSnapshot /sub1 sn0", 1, "there is already a snapshot with the same name \"sn0\"", conf);
FsShellRun("-rmr /sub1/sub1sub2"); DFSTestUtil.FsShellRun("-rmr /sub1/sub1sub2", conf);
FsShellRun("-mkdir /sub1/sub1sub3"); DFSTestUtil.FsShellRun("-mkdir /sub1/sub1sub3", conf);
FsShellRun("-createSnapshot /sub1 sn1", 0, "Created snapshot /sub1/.snapshot/sn1"); DFSTestUtil.FsShellRun("-createSnapshot /sub1 sn1", 0, "Created snapshot /sub1/.snapshot/sn1", conf);
// check snapshot contents // check snapshot contents
FsShellRun("-ls /sub1", 0, "/sub1/sub1sub1"); DFSTestUtil.FsShellRun("-ls /sub1", 0, "/sub1/sub1sub1", conf);
FsShellRun("-ls /sub1", 0, "/sub1/sub1sub3"); DFSTestUtil.FsShellRun("-ls /sub1", 0, "/sub1/sub1sub3", conf);
FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn0"); DFSTestUtil.FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn0", conf);
FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn1"); DFSTestUtil.FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn1", conf);
FsShellRun("-ls /sub1/.snapshot/sn0", 0, "/sub1/.snapshot/sn0/sub1sub1"); DFSTestUtil.FsShellRun("-ls /sub1/.snapshot/sn0", 0, "/sub1/.snapshot/sn0/sub1sub1", conf);
FsShellRun("-ls /sub1/.snapshot/sn0", 0, "/sub1/.snapshot/sn0/sub1sub2"); DFSTestUtil.FsShellRun("-ls /sub1/.snapshot/sn0", 0, "/sub1/.snapshot/sn0/sub1sub2", conf);
FsShellRun("-ls /sub1/.snapshot/sn1", 0, "/sub1/.snapshot/sn1/sub1sub1"); DFSTestUtil.FsShellRun("-ls /sub1/.snapshot/sn1", 0, "/sub1/.snapshot/sn1/sub1sub1", conf);
FsShellRun("-ls /sub1/.snapshot/sn1", 0, "/sub1/.snapshot/sn1/sub1sub3"); DFSTestUtil.FsShellRun("-ls /sub1/.snapshot/sn1", 0, "/sub1/.snapshot/sn1/sub1sub3", conf);
} }
@Test @Test
public void testMkdirUsingReservedName() throws Exception { public void testMkdirUsingReservedName() throws Exception {
// test can not create dir with reserved name: .snapshot // test can not create dir with reserved name: .snapshot
FsShellRun("-ls /"); DFSTestUtil.FsShellRun("-ls /", conf);
FsShellRun("-mkdir /.snapshot", 1, "File exists"); DFSTestUtil.FsShellRun("-mkdir /.snapshot", 1, "File exists", conf);
FsShellRun("-mkdir /sub1/.snapshot", 1, "File exists"); DFSTestUtil.FsShellRun("-mkdir /sub1/.snapshot", 1, "File exists", conf);
// mkdir -p ignore reserved name check if dir already exists // mkdir -p ignore reserved name check if dir already exists
FsShellRun("-mkdir -p /sub1/.snapshot"); DFSTestUtil.FsShellRun("-mkdir -p /sub1/.snapshot", conf);
FsShellRun("-mkdir -p /sub1/sub1sub1/.snapshot", 1, "mkdir: \".snapshot\" is a reserved name."); DFSTestUtil.FsShellRun("-mkdir -p /sub1/sub1sub1/.snapshot", 1, "mkdir: \".snapshot\" is a reserved name.", conf);
} }
@Test @Test
public void testRenameSnapshot() throws Exception { public void testRenameSnapshot() throws Exception {
FsShellRun("-createSnapshot /sub1 sn.orig"); DFSTestUtil.FsShellRun("-createSnapshot /sub1 sn.orig", conf);
FsShellRun("-renameSnapshot /sub1 sn.orig sn.rename"); DFSTestUtil.FsShellRun("-renameSnapshot /sub1 sn.orig sn.rename", conf);
FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn.rename"); DFSTestUtil.FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn.rename", conf);
FsShellRun("-ls /sub1/.snapshot/sn.rename", 0, "/sub1/.snapshot/sn.rename/sub1sub1"); DFSTestUtil.FsShellRun("-ls /sub1/.snapshot/sn.rename", 0, "/sub1/.snapshot/sn.rename/sub1sub1", conf);
FsShellRun("-ls /sub1/.snapshot/sn.rename", 0, "/sub1/.snapshot/sn.rename/sub1sub2"); DFSTestUtil.FsShellRun("-ls /sub1/.snapshot/sn.rename", 0, "/sub1/.snapshot/sn.rename/sub1sub2", conf);
//try renaming from a non-existing snapshot //try renaming from a non-existing snapshot
FsShellRun("-renameSnapshot /sub1 sn.nonexist sn.rename", 1, DFSTestUtil.FsShellRun("-renameSnapshot /sub1 sn.nonexist sn.rename", 1,
"renameSnapshot: The snapshot sn.nonexist does not exist for directory /sub1"); "renameSnapshot: The snapshot sn.nonexist does not exist for directory /sub1", conf);
//try renaming to existing snapshots //try renaming to existing snapshots
FsShellRun("-createSnapshot /sub1 sn.new"); DFSTestUtil.FsShellRun("-createSnapshot /sub1 sn.new", conf);
FsShellRun("-renameSnapshot /sub1 sn.new sn.rename", 1, DFSTestUtil.FsShellRun("-renameSnapshot /sub1 sn.new sn.rename", 1,
"renameSnapshot: The snapshot sn.rename already exists for directory /sub1"); "renameSnapshot: The snapshot sn.rename already exists for directory /sub1", conf);
FsShellRun("-renameSnapshot /sub1 sn.rename sn.new", 1, DFSTestUtil.FsShellRun("-renameSnapshot /sub1 sn.rename sn.new", 1,
"renameSnapshot: The snapshot sn.new already exists for directory /sub1"); "renameSnapshot: The snapshot sn.new already exists for directory /sub1", conf);
} }
@Test @Test
public void testDeleteSnapshot() throws Exception { public void testDeleteSnapshot() throws Exception {
FsShellRun("-createSnapshot /sub1 sn1"); DFSTestUtil.FsShellRun("-createSnapshot /sub1 sn1", conf);
FsShellRun("-deleteSnapshot /sub1 sn1"); DFSTestUtil.FsShellRun("-deleteSnapshot /sub1 sn1", conf);
FsShellRun("-deleteSnapshot /sub1 sn1", 1, DFSTestUtil.FsShellRun("-deleteSnapshot /sub1 sn1", 1,
"deleteSnapshot: Cannot delete snapshot sn1 from path /sub1: the snapshot does not exist."); "deleteSnapshot: Cannot delete snapshot sn1 from path /sub1: the snapshot does not exist.", conf);
} }
@Test @Test
public void testDisallowSnapshot() throws Exception { public void testDisallowSnapshot() throws Exception {
FsShellRun("-createSnapshot /sub1 sn1"); DFSTestUtil.FsShellRun("-createSnapshot /sub1 sn1", conf);
// cannot delete snapshotable dir // cannot delete snapshotable dir
FsShellRun("-rmr /sub1", 1, "The directory /sub1 cannot be deleted since /sub1 is snapshottable and already has snapshots"); DFSTestUtil.FsShellRun("-rmr /sub1", 1, "The directory /sub1 cannot be deleted since /sub1 is snapshottable and already has snapshots", conf);
DFSAdminRun("-disallowSnapshot /sub1", -1, DFSTestUtil.DFSAdminRun("-disallowSnapshot /sub1", -1,
"disallowSnapshot: The directory /sub1 has snapshot(s). Please redo the operation after removing all the snapshots."); "disallowSnapshot: The directory /sub1 has snapshot(s). Please redo the operation after removing all the snapshots.", conf);
FsShellRun("-deleteSnapshot /sub1 sn1"); DFSTestUtil.FsShellRun("-deleteSnapshot /sub1 sn1", conf);
DFSAdminRun("-disallowSnapshot /sub1", 0, "Disallowing snaphot on /sub1 succeeded"); DFSTestUtil.DFSAdminRun("-disallowSnapshot /sub1", 0, "Disallowing snaphot on /sub1 succeeded", conf);
// Idempotent test // Idempotent test
DFSAdminRun("-disallowSnapshot /sub1", 0, "Disallowing snaphot on /sub1 succeeded"); DFSTestUtil.DFSAdminRun("-disallowSnapshot /sub1", 0, "Disallowing snaphot on /sub1 succeeded", conf);
// now it can be deleted // now it can be deleted
FsShellRun("-rmr /sub1"); DFSTestUtil.FsShellRun("-rmr /sub1", conf);
} }
} }

View File

@ -0,0 +1,81 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
/**
* Test storage policy related DFSAdmin commands
*/
public class TestStoragePolicyCommands {
private static final short REPL = 1;
private static final int SIZE = 128;
private static Configuration conf;
private static MiniDFSCluster cluster;
private static DistributedFileSystem fs;
@Before
public void clusterSetUp() throws IOException {
conf = new HdfsConfiguration();
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL).build();
cluster.waitActive();
fs = cluster.getFileSystem();
}
@After
public void clusterShutdown() throws IOException{
if(fs != null){
fs.close();
}
if(cluster != null){
cluster.shutdown();
}
}
@Test
public void testSetAndGetStoragePolicy() throws Exception {
final Path foo = new Path("/foo");
final Path bar = new Path(foo, "bar");
DFSTestUtil.createFile(fs, bar, SIZE, REPL, 0);
DFSTestUtil.DFSAdminRun("-setStoragePolicy /foo WARM", 0,
"Set storage policy WARM on " + foo.toString(), conf);
DFSTestUtil.DFSAdminRun("-setStoragePolicy /foo/bar COLD", 0,
"Set storage policy COLD on " + bar.toString(), conf);
DFSTestUtil.DFSAdminRun("-setStoragePolicy /fooz WARM", -1,
"File/Directory does not exist: /fooz", conf);
final BlockStoragePolicy.Suite suite = BlockStoragePolicy
.readBlockStorageSuite(conf);
final BlockStoragePolicy warm = suite.getPolicy("WARM");
final BlockStoragePolicy cold = suite.getPolicy("COLD");
DFSTestUtil.DFSAdminRun("-getStoragePolicy /foo", 0,
"The storage policy of " + foo.toString() + ":\n" + warm, conf);
DFSTestUtil.DFSAdminRun("-getStoragePolicy /foo/bar", 0,
"The storage policy of " + bar.toString() + ":\n" + cold, conf);
DFSTestUtil.DFSAdminRun("-getStoragePolicy /fooz", -1,
"File/Directory does not exist: /fooz", conf);
}
}

View File

@ -89,7 +89,14 @@ public class TestBalancer {
private static final Random r = new Random(); private static final Random r = new Random();
static { static {
initTestSetup();
}
public static void initTestSetup() {
Dispatcher.setBlockMoveWaitTime(1000L) ; Dispatcher.setBlockMoveWaitTime(1000L) ;
// do not create id file since it occupies the disk space
NameNodeConnector.setWrite2IdFile(false);
} }
static void initConf(Configuration conf) { static void initConf(Configuration conf) {

View File

@ -44,7 +44,7 @@ public class TestBalancerWithHANameNodes {
ClientProtocol client; ClientProtocol client;
static { static {
Dispatcher.setBlockMoveWaitTime(1000L); TestBalancer.initTestSetup();
} }
/** /**

View File

@ -73,7 +73,7 @@ public class TestBalancerWithMultipleNameNodes {
private static final Random RANDOM = new Random(); private static final Random RANDOM = new Random();
static { static {
Dispatcher.setBlockMoveWaitTime(1000L) ; TestBalancer.initTestSetup();
} }
/** Common objects used in various methods. */ /** Common objects used in various methods. */

View File

@ -75,7 +75,7 @@ public class TestBalancerWithNodeGroup {
static final int DEFAULT_BLOCK_SIZE = 100; static final int DEFAULT_BLOCK_SIZE = 100;
static { static {
Dispatcher.setBlockMoveWaitTime(1000L) ; TestBalancer.initTestSetup();
} }
static Configuration createConf() { static Configuration createConf() {

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
@ -52,6 +53,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.net.NetworkTopology;
import org.junit.Assert;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -607,7 +609,6 @@ public class TestBlockManager {
assertEquals(1, ds.getBlockReportCount()); assertEquals(1, ds.getBlockReportCount());
} }
/** /**
* Tests that a namenode doesn't choose a datanode with full disks to * Tests that a namenode doesn't choose a datanode with full disks to
* store blocks. * store blocks.
@ -654,5 +655,20 @@ public class TestBlockManager {
cluster.shutdown(); cluster.shutdown();
} }
} }
}
@Test
public void testUseDelHint() {
DatanodeStorageInfo delHint = new DatanodeStorageInfo(
DFSTestUtil.getLocalDatanodeDescriptor(), new DatanodeStorage("id"));
List<DatanodeStorageInfo> moreThan1Racks = Arrays.asList(delHint);
List<StorageType> excessTypes = new ArrayList<StorageType>();
excessTypes.add(StorageType.DEFAULT);
Assert.assertTrue(BlockManager.useDelHint(true, delHint, null,
moreThan1Racks, excessTypes));
excessTypes.remove(0);
excessTypes.add(StorageType.SSD);
Assert.assertFalse(BlockManager.useDelHint(true, delHint, null,
moreThan1Racks, excessTypes));
}
}

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any; import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
@ -45,10 +46,15 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.LogVerificationAppender; import org.apache.hadoop.hdfs.LogVerificationAppender;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager.StatefulBlockInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.Namesystem; import org.apache.hadoop.hdfs.server.namenode.Namesystem;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@ -222,7 +228,7 @@ public class TestReplicationPolicy {
List<DatanodeStorageInfo> chosenNodes, List<DatanodeStorageInfo> chosenNodes,
Set<Node> excludedNodes) { Set<Node> excludedNodes) {
return replicator.chooseTarget(filename, numOfReplicas, writer, chosenNodes, return replicator.chooseTarget(filename, numOfReplicas, writer, chosenNodes,
false, excludedNodes, BLOCK_SIZE, StorageType.DEFAULT); false, excludedNodes, BLOCK_SIZE, TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
} }
/** /**
@ -289,7 +295,7 @@ public class TestReplicationPolicy {
excludedNodes.add(dataNodes[1]); excludedNodes.add(dataNodes[1]);
chosenNodes.add(storages[2]); chosenNodes.add(storages[2]);
targets = replicator.chooseTarget(filename, 1, dataNodes[0], chosenNodes, true, targets = replicator.chooseTarget(filename, 1, dataNodes[0], chosenNodes, true,
excludedNodes, BLOCK_SIZE, StorageType.DEFAULT); excludedNodes, BLOCK_SIZE, TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
System.out.println("targets=" + Arrays.asList(targets)); System.out.println("targets=" + Arrays.asList(targets));
assertEquals(2, targets.length); assertEquals(2, targets.length);
//make sure that the chosen node is in the target. //make sure that the chosen node is in the target.
@ -624,7 +630,7 @@ public class TestReplicationPolicy {
.getNamesystem().getBlockManager().getBlockPlacementPolicy(); .getNamesystem().getBlockManager().getBlockPlacementPolicy();
DatanodeStorageInfo[] targets = replicator.chooseTarget(filename, 3, DatanodeStorageInfo[] targets = replicator.chooseTarget(filename, 3,
staleNodeInfo, new ArrayList<DatanodeStorageInfo>(), false, null, staleNodeInfo, new ArrayList<DatanodeStorageInfo>(), false, null,
BLOCK_SIZE, StorageType.DEFAULT); BLOCK_SIZE, TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
assertEquals(targets.length, 3); assertEquals(targets.length, 3);
assertFalse(isOnSameRack(targets[0], staleNodeInfo)); assertFalse(isOnSameRack(targets[0], staleNodeInfo));
@ -650,7 +656,7 @@ public class TestReplicationPolicy {
// Call chooseTarget // Call chooseTarget
targets = replicator.chooseTarget(filename, 3, staleNodeInfo, targets = replicator.chooseTarget(filename, 3, staleNodeInfo,
new ArrayList<DatanodeStorageInfo>(), false, null, BLOCK_SIZE, new ArrayList<DatanodeStorageInfo>(), false, null, BLOCK_SIZE,
StorageType.DEFAULT); TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
assertEquals(targets.length, 3); assertEquals(targets.length, 3);
assertTrue(isOnSameRack(targets[0], staleNodeInfo)); assertTrue(isOnSameRack(targets[0], staleNodeInfo));
@ -927,8 +933,16 @@ public class TestReplicationPolicy {
// replica nodes, while storages[2] and dataNodes[5] are in second set. // replica nodes, while storages[2] and dataNodes[5] are in second set.
assertEquals(2, first.size()); assertEquals(2, first.size());
assertEquals(2, second.size()); assertEquals(2, second.size());
List<StorageType> excessTypes = new ArrayList<StorageType>();
{
// test returning null
excessTypes.add(StorageType.SSD);
assertNull(replicator.chooseReplicaToDelete(
null, null, (short)3, first, second, excessTypes));
}
excessTypes.add(StorageType.DEFAULT);
DatanodeStorageInfo chosen = replicator.chooseReplicaToDelete( DatanodeStorageInfo chosen = replicator.chooseReplicaToDelete(
null, null, (short)3, first, second); null, null, (short)3, first, second, excessTypes);
// Within first set, storages[1] with less free space // Within first set, storages[1] with less free space
assertEquals(chosen, storages[1]); assertEquals(chosen, storages[1]);
@ -936,8 +950,9 @@ public class TestReplicationPolicy {
assertEquals(0, first.size()); assertEquals(0, first.size());
assertEquals(3, second.size()); assertEquals(3, second.size());
// Within second set, storages[5] with less free space // Within second set, storages[5] with less free space
excessTypes.add(StorageType.DEFAULT);
chosen = replicator.chooseReplicaToDelete( chosen = replicator.chooseReplicaToDelete(
null, null, (short)2, first, second); null, null, (short)2, first, second, excessTypes);
assertEquals(chosen, storages[5]); assertEquals(chosen, storages[5]);
} }

View File

@ -33,13 +33,12 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@ -147,7 +146,7 @@ public class TestReplicationPolicyConsiderLoad {
DatanodeStorageInfo[] targets = namenode.getNamesystem().getBlockManager() DatanodeStorageInfo[] targets = namenode.getNamesystem().getBlockManager()
.getBlockPlacementPolicy().chooseTarget("testFile.txt", 3, .getBlockPlacementPolicy().chooseTarget("testFile.txt", 3,
dataNodes[0], new ArrayList<DatanodeStorageInfo>(), false, null, dataNodes[0], new ArrayList<DatanodeStorageInfo>(), false, null,
1024, StorageType.DEFAULT); 1024, TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
assertEquals(3, targets.length); assertEquals(3, targets.length);
Set<DatanodeStorageInfo> targetSet = new HashSet<DatanodeStorageInfo>( Set<DatanodeStorageInfo> targetSet = new HashSet<DatanodeStorageInfo>(

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.net.NetworkTopology;
@ -258,7 +259,7 @@ public class TestReplicationPolicyWithNodeGroup {
List<DatanodeStorageInfo> chosenNodes, List<DatanodeStorageInfo> chosenNodes,
Set<Node> excludedNodes) { Set<Node> excludedNodes) {
return replicator.chooseTarget(filename, numOfReplicas, writer, chosenNodes, return replicator.chooseTarget(filename, numOfReplicas, writer, chosenNodes,
false, excludedNodes, BLOCK_SIZE, StorageType.DEFAULT); false, excludedNodes, BLOCK_SIZE, TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
} }
/** /**
@ -340,7 +341,7 @@ public class TestReplicationPolicyWithNodeGroup {
Set<Node> excludedNodes = new HashSet<Node>(); Set<Node> excludedNodes = new HashSet<Node>();
excludedNodes.add(dataNodes[1]); excludedNodes.add(dataNodes[1]);
targets = repl.chooseTarget(filename, 4, dataNodes[0], chosenNodes, false, targets = repl.chooseTarget(filename, 4, dataNodes[0], chosenNodes, false,
excludedNodes, BLOCK_SIZE, StorageType.DEFAULT); excludedNodes, BLOCK_SIZE, TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
assertEquals(targets.length, 4); assertEquals(targets.length, 4);
assertEquals(storages[0], targets[0]); assertEquals(storages[0], targets[0]);
@ -358,7 +359,7 @@ public class TestReplicationPolicyWithNodeGroup {
excludedNodes.add(dataNodes[1]); excludedNodes.add(dataNodes[1]);
chosenNodes.add(storages[2]); chosenNodes.add(storages[2]);
targets = repl.chooseTarget(filename, 1, dataNodes[0], chosenNodes, true, targets = repl.chooseTarget(filename, 1, dataNodes[0], chosenNodes, true,
excludedNodes, BLOCK_SIZE, StorageType.DEFAULT); excludedNodes, BLOCK_SIZE, TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
System.out.println("targets=" + Arrays.asList(targets)); System.out.println("targets=" + Arrays.asList(targets));
assertEquals(2, targets.length); assertEquals(2, targets.length);
//make sure that the chosen node is in the target. //make sure that the chosen node is in the target.
@ -612,8 +613,10 @@ public class TestReplicationPolicyWithNodeGroup {
replicaList, rackMap, first, second); replicaList, rackMap, first, second);
assertEquals(3, first.size()); assertEquals(3, first.size());
assertEquals(1, second.size()); assertEquals(1, second.size());
List<StorageType> excessTypes = new ArrayList<StorageType>();
excessTypes.add(StorageType.DEFAULT);
DatanodeStorageInfo chosen = replicator.chooseReplicaToDelete( DatanodeStorageInfo chosen = replicator.chooseReplicaToDelete(
null, null, (short)3, first, second); null, null, (short)3, first, second, excessTypes);
// Within first set {dataNodes[0], dataNodes[1], dataNodes[2]}, // Within first set {dataNodes[0], dataNodes[1], dataNodes[2]},
// dataNodes[0] and dataNodes[1] are in the same nodegroup, // dataNodes[0] and dataNodes[1] are in the same nodegroup,
// but dataNodes[1] is chosen as less free space // but dataNodes[1] is chosen as less free space
@ -624,16 +627,18 @@ public class TestReplicationPolicyWithNodeGroup {
assertEquals(1, second.size()); assertEquals(1, second.size());
// Within first set {dataNodes[0], dataNodes[2]}, dataNodes[2] is chosen // Within first set {dataNodes[0], dataNodes[2]}, dataNodes[2] is chosen
// as less free space // as less free space
excessTypes.add(StorageType.DEFAULT);
chosen = replicator.chooseReplicaToDelete( chosen = replicator.chooseReplicaToDelete(
null, null, (short)2, first, second); null, null, (short)2, first, second, excessTypes);
assertEquals(chosen, storages[2]); assertEquals(chosen, storages[2]);
replicator.adjustSetsWithChosenReplica(rackMap, first, second, chosen); replicator.adjustSetsWithChosenReplica(rackMap, first, second, chosen);
assertEquals(0, first.size()); assertEquals(0, first.size());
assertEquals(2, second.size()); assertEquals(2, second.size());
// Within second set, dataNodes[5] with less free space // Within second set, dataNodes[5] with less free space
excessTypes.add(StorageType.DEFAULT);
chosen = replicator.chooseReplicaToDelete( chosen = replicator.chooseReplicaToDelete(
null, null, (short)1, first, second); null, null, (short)1, first, second, excessTypes);
assertEquals(chosen, storages[5]); assertEquals(chosen, storages[5]);
} }

View File

@ -0,0 +1,222 @@
/**
* 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.mover;
import java.io.IOException;
import java.net.URI;
import java.util.*;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.*;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DBlock;
import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
import org.apache.hadoop.hdfs.server.mover.Mover.MLocation;
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.GenericOptionsParser;
import org.junit.Assert;
import org.junit.Test;
public class TestMover {
static Mover newMover(Configuration conf) throws IOException {
final Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
Assert.assertEquals(1, namenodes.size());
final List<NameNodeConnector> nncs = NameNodeConnector.newNameNodeConnectors(
namenodes, Mover.class.getSimpleName(), Mover.MOVER_ID_PATH, conf);
return new Mover(nncs.get(0), conf);
}
@Test
public void testScheduleSameBlock() throws IOException {
final Configuration conf = new HdfsConfiguration();
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(4).build();
try {
cluster.waitActive();
final DistributedFileSystem dfs = cluster.getFileSystem();
final String file = "/testScheduleSameBlock/file";
{
final FSDataOutputStream out = dfs.create(new Path(file));
out.writeChars("testScheduleSameBlock");
out.close();
}
final Mover mover = newMover(conf);
mover.init();
final Mover.Processor processor = mover.new Processor();
final LocatedBlock lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
final List<MLocation> locations = MLocation.toLocations(lb);
final MLocation ml = locations.get(0);
final DBlock db = mover.newDBlock(lb.getBlock().getLocalBlock(), locations);
final List<StorageType> storageTypes = new ArrayList<StorageType>(
Arrays.asList(StorageType.DEFAULT, StorageType.DEFAULT));
Assert.assertTrue(processor.scheduleMoveReplica(db, ml, storageTypes));
Assert.assertFalse(processor.scheduleMoveReplica(db, ml, storageTypes));
} finally {
cluster.shutdown();
}
}
private void checkMovePaths(List<Path> actual, Path... expected) {
Assert.assertEquals(expected.length, actual.size());
for (Path p : expected) {
Assert.assertTrue(actual.contains(p));
}
}
/**
* Test Mover Cli by specifying a list of files/directories using option "-p".
* There is only one namenode (and hence name service) specified in the conf.
*/
@Test
public void testMoverCli() throws Exception {
final MiniDFSCluster cluster = new MiniDFSCluster
.Builder(new HdfsConfiguration()).numDataNodes(0).build();
try {
final Configuration conf = cluster.getConfiguration(0);
try {
Mover.Cli.getNameNodePathsToMove(conf, "-p", "/foo", "bar");
Assert.fail("Expected exception for illegal path bar");
} catch (IllegalArgumentException e) {
GenericTestUtils.assertExceptionContains("bar is not absolute", e);
}
Map<URI, List<Path>> movePaths = Mover.Cli.getNameNodePathsToMove(conf);
Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
Assert.assertEquals(1, namenodes.size());
Assert.assertEquals(1, movePaths.size());
URI nn = namenodes.iterator().next();
Assert.assertTrue(movePaths.containsKey(nn));
Assert.assertNull(movePaths.get(nn));
movePaths = Mover.Cli.getNameNodePathsToMove(conf, "-p", "/foo", "/bar");
namenodes = DFSUtil.getNsServiceRpcUris(conf);
Assert.assertEquals(1, movePaths.size());
nn = namenodes.iterator().next();
Assert.assertTrue(movePaths.containsKey(nn));
checkMovePaths(movePaths.get(nn), new Path("/foo"), new Path("/bar"));
} finally {
cluster.shutdown();
}
}
@Test
public void testMoverCliWithHAConf() throws Exception {
final Configuration conf = new HdfsConfiguration();
final MiniDFSCluster cluster = new MiniDFSCluster
.Builder(new HdfsConfiguration())
.nnTopology(MiniDFSNNTopology.simpleHATopology())
.numDataNodes(0).build();
HATestUtil.setFailoverConfigurations(cluster, conf, "MyCluster");
try {
Map<URI, List<Path>> movePaths = Mover.Cli.getNameNodePathsToMove(conf,
"-p", "/foo", "/bar");
Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
Assert.assertEquals(1, namenodes.size());
Assert.assertEquals(1, movePaths.size());
URI nn = namenodes.iterator().next();
Assert.assertEquals(new URI("hdfs://MyCluster"), nn);
Assert.assertTrue(movePaths.containsKey(nn));
checkMovePaths(movePaths.get(nn), new Path("/foo"), new Path("/bar"));
} finally {
cluster.shutdown();
}
}
@Test
public void testMoverCliWithFederation() throws Exception {
final MiniDFSCluster cluster = new MiniDFSCluster
.Builder(new HdfsConfiguration())
.nnTopology(MiniDFSNNTopology.simpleFederatedTopology(3))
.numDataNodes(0).build();
final Configuration conf = new HdfsConfiguration();
DFSTestUtil.setFederatedConfiguration(cluster, conf);
try {
Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
Assert.assertEquals(3, namenodes.size());
try {
Mover.Cli.getNameNodePathsToMove(conf, "-p", "/foo");
Assert.fail("Expect exception for missing authority information");
} catch (IllegalArgumentException e) {
GenericTestUtils.assertExceptionContains(
"does not contain scheme and authority", e);
}
try {
Mover.Cli.getNameNodePathsToMove(conf, "-p", "hdfs:///foo");
Assert.fail("Expect exception for missing authority information");
} catch (IllegalArgumentException e) {
GenericTestUtils.assertExceptionContains(
"does not contain scheme and authority", e);
}
try {
Mover.Cli.getNameNodePathsToMove(conf, "-p", "wrong-hdfs://ns1/foo");
Assert.fail("Expect exception for wrong scheme");
} catch (IllegalArgumentException e) {
GenericTestUtils.assertExceptionContains("Cannot resolve the path", e);
}
Iterator<URI> iter = namenodes.iterator();
URI nn1 = iter.next();
URI nn2 = iter.next();
Map<URI, List<Path>> movePaths = Mover.Cli.getNameNodePathsToMove(conf,
"-p", nn1 + "/foo", nn1 + "/bar", nn2 + "/foo/bar");
Assert.assertEquals(2, movePaths.size());
checkMovePaths(movePaths.get(nn1), new Path("/foo"), new Path("/bar"));
checkMovePaths(movePaths.get(nn2), new Path("/foo/bar"));
} finally {
cluster.shutdown();
}
}
@Test
public void testMoverCliWithFederationHA() throws Exception {
final MiniDFSCluster cluster = new MiniDFSCluster
.Builder(new HdfsConfiguration())
.nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(3))
.numDataNodes(0).build();
final Configuration conf = new HdfsConfiguration();
DFSTestUtil.setFederatedHAConfiguration(cluster, conf);
try {
Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
Assert.assertEquals(3, namenodes.size());
Iterator<URI> iter = namenodes.iterator();
URI nn1 = iter.next();
URI nn2 = iter.next();
URI nn3 = iter.next();
Map<URI, List<Path>> movePaths = Mover.Cli.getNameNodePathsToMove(conf,
"-p", nn1 + "/foo", nn1 + "/bar", nn2 + "/foo/bar", nn3 + "/foobar");
Assert.assertEquals(3, movePaths.size());
checkMovePaths(movePaths.get(nn1), new Path("/foo"), new Path("/bar"));
checkMovePaths(movePaths.get(nn2), new Path("/foo/bar"));
checkMovePaths(movePaths.get(nn3), new Path("/foobar"));
} finally {
cluster.shutdown();
}
}
}

View File

@ -0,0 +1,766 @@
/**
* 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.mover;
import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
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.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
import org.apache.hadoop.hdfs.server.balancer.Dispatcher;
import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
import org.apache.hadoop.io.IOUtils;
import org.apache.log4j.Level;
import org.junit.Assert;
import org.junit.Test;
import com.google.common.base.Preconditions;
import com.google.common.collect.Maps;
/**
* Test the data migration tool (for Archival Storage)
*/
public class TestStorageMover {
static final Log LOG = LogFactory.getLog(TestStorageMover.class);
static {
((Log4JLogger)LogFactory.getLog(BlockPlacementPolicy.class)
).getLogger().setLevel(Level.ALL);
((Log4JLogger)LogFactory.getLog(Dispatcher.class)
).getLogger().setLevel(Level.ALL);
((Log4JLogger)LogFactory.getLog(DataTransferProtocol.class)).getLogger()
.setLevel(Level.ALL);
}
private static final int BLOCK_SIZE = 1024;
private static final short REPL = 3;
private static final int NUM_DATANODES = 6;
private static final Configuration DEFAULT_CONF = new HdfsConfiguration();
private static final BlockStoragePolicy.Suite DEFAULT_POLICIES;
private static final BlockStoragePolicy HOT;
private static final BlockStoragePolicy WARM;
private static final BlockStoragePolicy COLD;
static {
DEFAULT_CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
DEFAULT_CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
DEFAULT_CONF.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
2L);
DEFAULT_CONF.setLong(DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_KEY, 2000L);
DEFAULT_POLICIES = BlockStoragePolicy.readBlockStorageSuite(DEFAULT_CONF);
HOT = DEFAULT_POLICIES.getPolicy("HOT");
WARM = DEFAULT_POLICIES.getPolicy("WARM");
COLD = DEFAULT_POLICIES.getPolicy("COLD");
TestBalancer.initTestSetup();
Dispatcher.setDelayAfterErrors(1000L);
}
/**
* This scheme defines files/directories and their block storage policies. It
* also defines snapshots.
*/
static class NamespaceScheme {
final List<Path> dirs;
final List<Path> files;
final long fileSize;
final Map<Path, List<String>> snapshotMap;
final Map<Path, BlockStoragePolicy> policyMap;
NamespaceScheme(List<Path> dirs, List<Path> files, long fileSize,
Map<Path,List<String>> snapshotMap,
Map<Path, BlockStoragePolicy> policyMap) {
this.dirs = dirs == null? Collections.<Path>emptyList(): dirs;
this.files = files == null? Collections.<Path>emptyList(): files;
this.fileSize = fileSize;
this.snapshotMap = snapshotMap == null ?
Collections.<Path, List<String>>emptyMap() : snapshotMap;
this.policyMap = policyMap;
}
/**
* Create files/directories/snapshots.
*/
void prepare(DistributedFileSystem dfs, short repl) throws Exception {
for (Path d : dirs) {
dfs.mkdirs(d);
}
for (Path file : files) {
DFSTestUtil.createFile(dfs, file, fileSize, repl, 0L);
}
for (Map.Entry<Path, List<String>> entry : snapshotMap.entrySet()) {
for (String snapshot : entry.getValue()) {
SnapshotTestHelper.createSnapshot(dfs, entry.getKey(), snapshot);
}
}
}
/**
* Set storage policies according to the corresponding scheme.
*/
void setStoragePolicy(DistributedFileSystem dfs) throws Exception {
for (Map.Entry<Path, BlockStoragePolicy> entry : policyMap.entrySet()) {
dfs.setStoragePolicy(entry.getKey(), entry.getValue().getName());
}
}
}
/**
* This scheme defines DataNodes and their storage, including storage types
* and remaining capacities.
*/
static class ClusterScheme {
final Configuration conf;
final int numDataNodes;
final short repl;
final StorageType[][] storageTypes;
final long[][] storageCapacities;
ClusterScheme() {
this(DEFAULT_CONF, NUM_DATANODES, REPL,
genStorageTypes(NUM_DATANODES), null);
}
ClusterScheme(Configuration conf, int numDataNodes, short repl,
StorageType[][] types, long[][] capacities) {
Preconditions.checkArgument(types == null || types.length == numDataNodes);
Preconditions.checkArgument(capacities == null || capacities.length ==
numDataNodes);
this.conf = conf;
this.numDataNodes = numDataNodes;
this.repl = repl;
this.storageTypes = types;
this.storageCapacities = capacities;
}
}
class MigrationTest {
private final ClusterScheme clusterScheme;
private final NamespaceScheme nsScheme;
private final Configuration conf;
private MiniDFSCluster cluster;
private DistributedFileSystem dfs;
private final BlockStoragePolicy.Suite policies;
MigrationTest(ClusterScheme cScheme, NamespaceScheme nsScheme) {
this.clusterScheme = cScheme;
this.nsScheme = nsScheme;
this.conf = clusterScheme.conf;
this.policies = BlockStoragePolicy.readBlockStorageSuite(conf);
}
/**
* Set up the cluster and start NameNode and DataNodes according to the
* corresponding scheme.
*/
void setupCluster() throws Exception {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(clusterScheme
.numDataNodes).storageTypes(clusterScheme.storageTypes)
.storageCapacities(clusterScheme.storageCapacities).build();
cluster.waitActive();
dfs = cluster.getFileSystem();
}
private void runBasicTest(boolean shutdown) throws Exception {
setupCluster();
try {
prepareNamespace();
verify(true);
setStoragePolicy();
migrate();
verify(true);
} finally {
if (shutdown) {
shutdownCluster();
}
}
}
void shutdownCluster() throws Exception {
IOUtils.cleanup(null, dfs);
if (cluster != null) {
cluster.shutdown();
}
}
/**
* Create files/directories and set their storage policies according to the
* corresponding scheme.
*/
void prepareNamespace() throws Exception {
nsScheme.prepare(dfs, clusterScheme.repl);
}
void setStoragePolicy() throws Exception {
nsScheme.setStoragePolicy(dfs);
}
/**
* Run the migration tool.
*/
void migrate() throws Exception {
runMover();
Thread.sleep(5000); // let the NN finish deletion
}
/**
* Verify block locations after running the migration tool.
*/
void verify(boolean verifyAll) throws Exception {
for (DataNode dn : cluster.getDataNodes()) {
DataNodeTestUtils.triggerBlockReport(dn);
}
if (verifyAll) {
verifyNamespace();
} else {
// TODO verify according to the given path list
}
}
private void runMover() throws Exception {
Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
Map<URI, List<Path>> nnMap = Maps.newHashMap();
for (URI nn : namenodes) {
nnMap.put(nn, null);
}
int result = Mover.run(nnMap, conf);
Assert.assertEquals(ExitStatus.SUCCESS.getExitCode(), result);
}
private void verifyNamespace() throws Exception {
HdfsFileStatus status = dfs.getClient().getFileInfo("/");
verifyRecursively(null, status);
}
private void verifyRecursively(final Path parent,
final HdfsFileStatus status) throws Exception {
if (status.isDir()) {
Path fullPath = parent == null ?
new Path("/") : status.getFullPath(parent);
DirectoryListing children = dfs.getClient().listPaths(
fullPath.toString(), HdfsFileStatus.EMPTY_NAME, true);
for (HdfsFileStatus child : children.getPartialListing()) {
verifyRecursively(fullPath, child);
}
} else if (!status.isSymlink()) { // is file
verifyFile(parent, status, null);
}
}
void verifyFile(final Path file, final Byte expectedPolicyId)
throws Exception {
final Path parent = file.getParent();
DirectoryListing children = dfs.getClient().listPaths(
parent.toString(), HdfsFileStatus.EMPTY_NAME, true);
for (HdfsFileStatus child : children.getPartialListing()) {
if (child.getLocalName().equals(file.getName())) {
verifyFile(parent, child, expectedPolicyId);
return;
}
}
Assert.fail("File " + file + " not found.");
}
private void verifyFile(final Path parent, final HdfsFileStatus status,
final Byte expectedPolicyId) throws Exception {
HdfsLocatedFileStatus fileStatus = (HdfsLocatedFileStatus) status;
byte policyId = fileStatus.getStoragePolicy();
BlockStoragePolicy policy = policies.getPolicy(policyId);
if (expectedPolicyId != null) {
Assert.assertEquals((byte)expectedPolicyId, policy.getId());
}
final List<StorageType> types = policy.chooseStorageTypes(
status.getReplication());
for(LocatedBlock lb : fileStatus.getBlockLocations().getLocatedBlocks()) {
final Mover.StorageTypeDiff diff = new Mover.StorageTypeDiff(types,
lb.getStorageTypes());
Assert.assertTrue(fileStatus.getFullName(parent.toString())
+ " with policy " + policy + " has non-empty overlap: " + diff
+ ", the corresponding block is " + lb.getBlock().getLocalBlock(),
diff.removeOverlap());
}
}
Replication getReplication(Path file) throws IOException {
return getOrVerifyReplication(file, null);
}
Replication verifyReplication(Path file, int expectedDiskCount,
int expectedArchiveCount) throws IOException {
final Replication r = new Replication();
r.disk = expectedDiskCount;
r.archive = expectedArchiveCount;
return getOrVerifyReplication(file, r);
}
private Replication getOrVerifyReplication(Path file, Replication expected)
throws IOException {
final List<LocatedBlock> lbs = dfs.getClient().getLocatedBlocks(
file.toString(), 0).getLocatedBlocks();
Assert.assertEquals(1, lbs.size());
LocatedBlock lb = lbs.get(0);
StringBuilder types = new StringBuilder();
final Replication r = new Replication();
for(StorageType t : lb.getStorageTypes()) {
types.append(t).append(", ");
if (t == StorageType.DISK) {
r.disk++;
} else if (t == StorageType.ARCHIVE) {
r.archive++;
} else {
Assert.fail("Unexpected storage type " + t);
}
}
if (expected != null) {
final String s = "file = " + file + "\n types = [" + types + "]";
Assert.assertEquals(s, expected, r);
}
return r;
}
}
static class Replication {
int disk;
int archive;
@Override
public int hashCode() {
return disk ^ archive;
}
@Override
public boolean equals(Object obj) {
if (obj == this) {
return true;
} else if (obj == null || !(obj instanceof Replication)) {
return false;
}
final Replication that = (Replication)obj;
return this.disk == that.disk && this.archive == that.archive;
}
@Override
public String toString() {
return "[disk=" + disk + ", archive=" + archive + "]";
}
}
private static StorageType[][] genStorageTypes(int numDataNodes) {
return genStorageTypes(numDataNodes, 0, 0);
}
private static StorageType[][] genStorageTypes(int numDataNodes,
int numAllDisk, int numAllArchive) {
StorageType[][] types = new StorageType[numDataNodes][];
int i = 0;
for (; i < numAllDisk; i++) {
types[i] = new StorageType[]{StorageType.DISK, StorageType.DISK};
}
for (; i < numAllDisk + numAllArchive; i++) {
types[i] = new StorageType[]{StorageType.ARCHIVE, StorageType.ARCHIVE};
}
for (; i < types.length; i++) {
types[i] = new StorageType[]{StorageType.DISK, StorageType.ARCHIVE};
}
return types;
}
private static long[][] genCapacities(int nDatanodes, int numAllDisk,
int numAllArchive, long diskCapacity, long archiveCapacity) {
final long[][] capacities = new long[nDatanodes][];
int i = 0;
for (; i < numAllDisk; i++) {
capacities[i] = new long[]{diskCapacity, diskCapacity};
}
for (; i < numAllDisk + numAllArchive; i++) {
capacities[i] = new long[]{archiveCapacity, archiveCapacity};
}
for(; i < capacities.length; i++) {
capacities[i] = new long[]{diskCapacity, archiveCapacity};
}
return capacities;
}
private static class PathPolicyMap {
final Map<Path, BlockStoragePolicy> map = Maps.newHashMap();
final Path hot = new Path("/hot");
final Path warm = new Path("/warm");
final Path cold = new Path("/cold");
final List<Path> files;
PathPolicyMap(int filesPerDir){
map.put(hot, HOT);
map.put(warm, WARM);
map.put(cold, COLD);
files = new ArrayList<Path>();
for(Path dir : map.keySet()) {
for(int i = 0; i < filesPerDir; i++) {
files.add(new Path(dir, "file" + i));
}
}
}
NamespaceScheme newNamespaceScheme() {
return new NamespaceScheme(Arrays.asList(hot, warm, cold),
files, BLOCK_SIZE/2, null, map);
}
/**
* Move hot files to warm and cold, warm files to hot and cold,
* and cold files to hot and warm.
*/
void moveAround(DistributedFileSystem dfs) throws Exception {
for(Path srcDir : map.keySet()) {
int i = 0;
for(Path dstDir : map.keySet()) {
if (!srcDir.equals(dstDir)) {
final Path src = new Path(srcDir, "file" + i++);
final Path dst = new Path(dstDir, srcDir.getName() + "2" + dstDir.getName());
LOG.info("rename " + src + " to " + dst);
dfs.rename(src, dst);
}
}
}
}
}
/**
* A normal case for Mover: move a file into archival storage
*/
@Test
public void testMigrateFileToArchival() throws Exception {
LOG.info("testMigrateFileToArchival");
final Path foo = new Path("/foo");
Map<Path, BlockStoragePolicy> policyMap = Maps.newHashMap();
policyMap.put(foo, COLD);
NamespaceScheme nsScheme = new NamespaceScheme(null, Arrays.asList(foo),
2*BLOCK_SIZE, null, policyMap);
ClusterScheme clusterScheme = new ClusterScheme(DEFAULT_CONF,
NUM_DATANODES, REPL, genStorageTypes(NUM_DATANODES), null);
new MigrationTest(clusterScheme, nsScheme).runBasicTest(true);
}
/**
* Print a big banner in the test log to make debug easier.
*/
static void banner(String string) {
LOG.info("\n\n\n\n================================================\n" +
string + "\n" +
"==================================================\n\n");
}
/**
* Move an open file into archival storage
*/
@Test
public void testMigrateOpenFileToArchival() throws Exception {
LOG.info("testMigrateOpenFileToArchival");
final Path fooDir = new Path("/foo");
Map<Path, BlockStoragePolicy> policyMap = Maps.newHashMap();
policyMap.put(fooDir, COLD);
NamespaceScheme nsScheme = new NamespaceScheme(Arrays.asList(fooDir), null,
BLOCK_SIZE, null, policyMap);
ClusterScheme clusterScheme = new ClusterScheme(DEFAULT_CONF,
NUM_DATANODES, REPL, genStorageTypes(NUM_DATANODES), null);
MigrationTest test = new MigrationTest(clusterScheme, nsScheme);
test.setupCluster();
// create an open file
banner("writing to file /foo/bar");
final Path barFile = new Path(fooDir, "bar");
DFSTestUtil.createFile(test.dfs, barFile, BLOCK_SIZE, (short) 1, 0L);
FSDataOutputStream out = test.dfs.append(barFile);
out.writeBytes("hello, ");
((DFSOutputStream) out.getWrappedStream()).hsync();
try {
banner("start data migration");
test.setStoragePolicy(); // set /foo to COLD
test.migrate();
// make sure the under construction block has not been migrated
LocatedBlocks lbs = test.dfs.getClient().getLocatedBlocks(
barFile.toString(), BLOCK_SIZE);
LOG.info("Locations: " + lbs);
List<LocatedBlock> blks = lbs.getLocatedBlocks();
Assert.assertEquals(1, blks.size());
Assert.assertEquals(1, blks.get(0).getLocations().length);
banner("finish the migration, continue writing");
// make sure the writing can continue
out.writeBytes("world!");
((DFSOutputStream) out.getWrappedStream()).hsync();
IOUtils.cleanup(LOG, out);
lbs = test.dfs.getClient().getLocatedBlocks(
barFile.toString(), BLOCK_SIZE);
LOG.info("Locations: " + lbs);
blks = lbs.getLocatedBlocks();
Assert.assertEquals(1, blks.size());
Assert.assertEquals(1, blks.get(0).getLocations().length);
banner("finish writing, starting reading");
// check the content of /foo/bar
FSDataInputStream in = test.dfs.open(barFile);
byte[] buf = new byte[13];
// read from offset 1024
in.readFully(BLOCK_SIZE, buf, 0, buf.length);
IOUtils.cleanup(LOG, in);
Assert.assertEquals("hello, world!", new String(buf));
} finally {
test.shutdownCluster();
}
}
/**
* Test directories with Hot, Warm and Cold polices.
*/
@Test
public void testHotWarmColdDirs() throws Exception {
LOG.info("testHotWarmColdDirs");
PathPolicyMap pathPolicyMap = new PathPolicyMap(3);
NamespaceScheme nsScheme = pathPolicyMap.newNamespaceScheme();
ClusterScheme clusterScheme = new ClusterScheme();
MigrationTest test = new MigrationTest(clusterScheme, nsScheme);
try {
test.runBasicTest(false);
pathPolicyMap.moveAround(test.dfs);
test.migrate();
test.verify(true);
} finally {
test.shutdownCluster();
}
}
private void waitForAllReplicas(int expectedReplicaNum, Path file,
DistributedFileSystem dfs) throws Exception {
for (int i = 0; i < 5; i++) {
LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(file.toString(), 0,
BLOCK_SIZE);
LocatedBlock lb = lbs.get(0);
if (lb.getLocations().length >= expectedReplicaNum) {
return;
} else {
Thread.sleep(1000);
}
}
}
/**
* Test DISK is running out of spaces.
*/
@Test
public void testNoSpaceDisk() throws Exception {
LOG.info("testNoSpaceDisk");
final PathPolicyMap pathPolicyMap = new PathPolicyMap(0);
final NamespaceScheme nsScheme = pathPolicyMap.newNamespaceScheme();
final long diskCapacity = (6 + HdfsConstants.MIN_BLOCKS_FOR_WRITE)
* BLOCK_SIZE;
final long archiveCapacity = 100 * BLOCK_SIZE;
final long[][] capacities = genCapacities(NUM_DATANODES, 1, 1,
diskCapacity, archiveCapacity);
Configuration conf = new Configuration(DEFAULT_CONF);
final ClusterScheme clusterScheme = new ClusterScheme(conf,
NUM_DATANODES, REPL, genStorageTypes(NUM_DATANODES, 1, 1), capacities);
final MigrationTest test = new MigrationTest(clusterScheme, nsScheme);
try {
test.runBasicTest(false);
// create hot files with replication 3 until not more spaces.
final short replication = 3;
{
int hotFileCount = 0;
try {
for (; ; hotFileCount++) {
final Path p = new Path(pathPolicyMap.hot, "file" + hotFileCount);
DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
waitForAllReplicas(replication, p, test.dfs);
}
} catch (IOException e) {
LOG.info("Expected: hotFileCount=" + hotFileCount, e);
}
Assert.assertTrue(hotFileCount >= 1);
}
// create hot files with replication 1 to use up all remaining spaces.
{
int hotFileCount_r1 = 0;
try {
for (; ; hotFileCount_r1++) {
final Path p = new Path(pathPolicyMap.hot, "file_r1_" + hotFileCount_r1);
DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, (short) 1, 0L);
waitForAllReplicas(1, p, test.dfs);
}
} catch (IOException e) {
LOG.info("Expected: hotFileCount_r1=" + hotFileCount_r1, e);
}
}
{ // test increasing replication. Since DISK is full,
// new replicas should be stored in ARCHIVE as a fallback storage.
final Path file0 = new Path(pathPolicyMap.hot, "file0");
final Replication r = test.getReplication(file0);
final short newReplication = (short) 5;
test.dfs.setReplication(file0, newReplication);
Thread.sleep(10000);
test.verifyReplication(file0, r.disk, newReplication - r.disk);
}
{ // test creating a cold file and then increase replication
final Path p = new Path(pathPolicyMap.cold, "foo");
DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
test.verifyReplication(p, 0, replication);
final short newReplication = 5;
test.dfs.setReplication(p, newReplication);
Thread.sleep(10000);
test.verifyReplication(p, 0, newReplication);
}
{ //test move a hot file to warm
final Path file1 = new Path(pathPolicyMap.hot, "file1");
test.dfs.rename(file1, pathPolicyMap.warm);
test.migrate();
test.verifyFile(new Path(pathPolicyMap.warm, "file1"), WARM.getId());
}
} finally {
test.shutdownCluster();
}
}
/**
* Test ARCHIVE is running out of spaces.
*/
@Test
public void testNoSpaceArchive() throws Exception {
LOG.info("testNoSpaceArchive");
final PathPolicyMap pathPolicyMap = new PathPolicyMap(0);
final NamespaceScheme nsScheme = pathPolicyMap.newNamespaceScheme();
final long diskCapacity = 100 * BLOCK_SIZE;
final long archiveCapacity = (6 + HdfsConstants.MIN_BLOCKS_FOR_WRITE)
* BLOCK_SIZE;
final long[][] capacities = genCapacities(NUM_DATANODES, 1, 1,
diskCapacity, archiveCapacity);
final ClusterScheme clusterScheme = new ClusterScheme(DEFAULT_CONF,
NUM_DATANODES, REPL, genStorageTypes(NUM_DATANODES, 1, 1), capacities);
final MigrationTest test = new MigrationTest(clusterScheme, nsScheme);
try {
test.runBasicTest(false);
// create cold files with replication 3 until not more spaces.
final short replication = 3;
{
int coldFileCount = 0;
try {
for (; ; coldFileCount++) {
final Path p = new Path(pathPolicyMap.cold, "file" + coldFileCount);
DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
waitForAllReplicas(replication, p, test.dfs);
}
} catch (IOException e) {
LOG.info("Expected: coldFileCount=" + coldFileCount, e);
}
Assert.assertTrue(coldFileCount >= 1);
}
// create cold files with replication 1 to use up all remaining spaces.
{
int coldFileCount_r1 = 0;
try {
for (; ; coldFileCount_r1++) {
final Path p = new Path(pathPolicyMap.cold, "file_r1_" + coldFileCount_r1);
DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, (short) 1, 0L);
waitForAllReplicas(1, p, test.dfs);
}
} catch (IOException e) {
LOG.info("Expected: coldFileCount_r1=" + coldFileCount_r1, e);
}
}
{ // test increasing replication but new replicas cannot be created
// since no more ARCHIVE space.
final Path file0 = new Path(pathPolicyMap.cold, "file0");
final Replication r = test.getReplication(file0);
LOG.info("XXX " + file0 + ": replication=" + r);
Assert.assertEquals(0, r.disk);
final short newReplication = (short) 5;
test.dfs.setReplication(file0, newReplication);
Thread.sleep(10000);
test.verifyReplication(file0, 0, r.archive);
}
{ // test creating a hot file
final Path p = new Path(pathPolicyMap.hot, "foo");
DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, (short) 3, 0L);
}
{ //test move a cold file to warm
final Path file1 = new Path(pathPolicyMap.cold, "file1");
test.dfs.rename(file1, pathPolicyMap.warm);
test.migrate();
test.verify(true);
}
} finally {
test.shutdownCluster();
}
}
}

View File

@ -82,7 +82,7 @@ public class CreateEditsLog {
} }
final INodeFile inode = new INodeFile(inodeId.nextValue(), null, final INodeFile inode = new INodeFile(inodeId.nextValue(), null,
p, 0L, 0L, blocks, replication, blockSize); p, 0L, 0L, blocks, replication, blockSize, (byte)0);
inode.toUnderConstruction("", ""); inode.toUnderConstruction("", "");
// Append path to filename with information about blockIDs // Append path to filename with information about blockIDs
@ -97,7 +97,7 @@ public class CreateEditsLog {
editLog.logMkDir(currentDir, dirInode); editLog.logMkDir(currentDir, dirInode);
} }
INodeFile fileUc = new INodeFile(inodeId.nextValue(), null, INodeFile fileUc = new INodeFile(inodeId.nextValue(), null,
p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize); p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize, (byte)0);
fileUc.toUnderConstruction("", ""); fileUc.toUnderConstruction("", "");
editLog.logOpenFile(filePath, fileUc, false, false); editLog.logOpenFile(filePath, fileUc, false, false);
editLog.logCloseFile(filePath, inode); editLog.logCloseFile(filePath, inode);

View File

@ -120,9 +120,9 @@ public class TestAddBlockRetry {
} }
return ret; return ret;
} }
}).when(spyBM).chooseTarget(Mockito.anyString(), Mockito.anyInt(), }).when(spyBM).chooseTarget4NewBlock(Mockito.anyString(), Mockito.anyInt(),
Mockito.<DatanodeDescriptor>any(), Mockito.<HashSet<Node>>any(), Mockito.<DatanodeDescriptor>any(), Mockito.<HashSet<Node>>any(),
Mockito.anyLong(), Mockito.<List<String>>any()); Mockito.anyLong(), Mockito.<List<String>>any(), Mockito.anyByte());
// create file // create file
nn.create(src, FsPermission.getFileDefault(), nn.create(src, FsPermission.getFileDefault(),

View File

@ -29,13 +29,13 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.BlockStoragePolicy;
import org.apache.hadoop.hdfs.AppendTestUtil; import org.apache.hadoop.hdfs.AppendTestUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB; import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
@ -121,10 +121,10 @@ public class TestDeleteRace {
boolean returnChosenNodes, boolean returnChosenNodes,
Set<Node> excludedNodes, Set<Node> excludedNodes,
long blocksize, long blocksize,
StorageType storageType) { final BlockStoragePolicy storagePolicy) {
DatanodeStorageInfo[] results = super.chooseTarget(srcPath, DatanodeStorageInfo[] results = super.chooseTarget(srcPath,
numOfReplicas, writer, chosenNodes, returnChosenNodes, excludedNodes, numOfReplicas, writer, chosenNodes, returnChosenNodes, excludedNodes,
blocksize, storageType); blocksize, storagePolicy);
try { try {
Thread.sleep(3000); Thread.sleep(3000);
} catch (InterruptedException e) {} } catch (InterruptedException e) {}

View File

@ -194,7 +194,7 @@ public class TestEditLog {
for (int i = 0; i < numTransactions; i++) { for (int i = 0; i < numTransactions; i++) {
INodeFile inode = new INodeFile(namesystem.allocateNewInodeId(), null, INodeFile inode = new INodeFile(namesystem.allocateNewInodeId(), null,
p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize); p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize, (byte)0);
inode.toUnderConstruction("", ""); inode.toUnderConstruction("", "");
editLog.logOpenFile("/filename" + (startIndex + i), inode, false, false); editLog.logOpenFile("/filename" + (startIndex + i), inode, false, false);

View File

@ -17,32 +17,41 @@
*/ */
package org.apache.hadoop.hdfs.server.namenode; package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.fs.permission.AclEntryScope.*; import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
import static org.apache.hadoop.fs.permission.AclEntryType.*; import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT;
import static org.apache.hadoop.fs.permission.FsAction.*; import static org.apache.hadoop.fs.permission.AclEntryType.GROUP;
import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.*; import static org.apache.hadoop.fs.permission.AclEntryType.MASK;
import static org.junit.Assert.*; import static org.apache.hadoop.fs.permission.AclEntryType.OTHER;
import static org.apache.hadoop.fs.permission.AclEntryType.USER;
import static org.apache.hadoop.fs.permission.FsAction.ALL;
import static org.apache.hadoop.fs.permission.FsAction.EXECUTE;
import static org.apache.hadoop.fs.permission.FsAction.NONE;
import static org.apache.hadoop.fs.permission.FsAction.READ;
import static org.apache.hadoop.fs.permission.FsAction.READ_EXECUTE;
import static org.apache.hadoop.fs.permission.FsAction.READ_WRITE;
import static org.apache.hadoop.fs.permission.FsAction.WRITE;
import static org.apache.hadoop.fs.permission.FsAction.WRITE_EXECUTE;
import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.junit.Before;
import org.junit.Test;
import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclEntryScope;
import org.apache.hadoop.fs.permission.AclEntryType;
import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot; import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.junit.Before;
import org.junit.Test;
import org.mockito.invocation.InvocationOnMock; import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer; import org.mockito.stubbing.Answer;
import static org.mockito.Mockito.*;
/** /**
* Unit tests covering FSPermissionChecker. All tests in this suite have been * Unit tests covering FSPermissionChecker. All tests in this suite have been
* cross-validated against Linux setfacl/getfacl to check for consistency of the * cross-validated against Linux setfacl/getfacl to check for consistency of the
@ -423,7 +432,7 @@ public class TestFSPermissionChecker {
FsPermission.createImmutable(perm)); FsPermission.createImmutable(perm));
INodeFile inodeFile = new INodeFile(INodeId.GRANDFATHER_INODE_ID, INodeFile inodeFile = new INodeFile(INodeId.GRANDFATHER_INODE_ID,
name.getBytes("UTF-8"), permStatus, 0L, 0L, null, REPLICATION, name.getBytes("UTF-8"), permStatus, 0L, 0L, null, REPLICATION,
PREFERRED_BLOCK_SIZE); PREFERRED_BLOCK_SIZE, (byte)0);
parent.addChild(inodeFile); parent.addChild(inodeFile);
return inodeFile; return inodeFile;
} }

View File

@ -1015,10 +1015,11 @@ public class TestFsck {
path = DFSUtil.string2Bytes(pathString); path = DFSUtil.string2Bytes(pathString);
long fileId = 312321L; long fileId = 312321L;
int numChildren = 1; int numChildren = 1;
byte storagePolicy = 0;
HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication, HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication,
blockSize, modTime, accessTime, perms, owner, group, symlink, path, blockSize, modTime, accessTime, perms, owner, group, symlink, path,
fileId, numChildren, null); fileId, numChildren, null, storagePolicy);
Result res = new Result(conf); Result res = new Result(conf);
try { try {

View File

@ -78,15 +78,39 @@ public class TestINodeFile {
static final short BLOCKBITS = 48; static final short BLOCKBITS = 48;
static final long BLKSIZE_MAXVALUE = ~(0xffffL << BLOCKBITS); static final long BLKSIZE_MAXVALUE = ~(0xffffL << BLOCKBITS);
private final PermissionStatus perm = new PermissionStatus( private static final PermissionStatus perm = new PermissionStatus(
"userName", null, FsPermission.getDefault()); "userName", null, FsPermission.getDefault());
private short replication; private short replication;
private long preferredBlockSize = 1024; private long preferredBlockSize = 1024;
INodeFile createINodeFile(short replication, long preferredBlockSize) { INodeFile createINodeFile(short replication, long preferredBlockSize) {
return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
null, replication, preferredBlockSize); null, replication, preferredBlockSize, (byte)0);
} }
private static INodeFile createINodeFile(byte storagePolicyID) {
return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
null, (short)3, 1024L, storagePolicyID);
}
@Test
public void testStoragePolicyID () {
for(byte i = 0; i < 16; i++) {
final INodeFile f = createINodeFile(i);
assertEquals(i, f.getStoragePolicyID());
}
}
@Test(expected=IllegalArgumentException.class)
public void testStoragePolicyIdBelowLowerBound () throws IllegalArgumentException {
createINodeFile((byte)-1);
}
@Test(expected=IllegalArgumentException.class)
public void testStoragePolicyIdAboveUpperBound () throws IllegalArgumentException {
createINodeFile((byte)16);
}
/** /**
* Test for the Replication value. Sets a value and checks if it was set * Test for the Replication value. Sets a value and checks if it was set
* correct. * correct.
@ -262,7 +286,7 @@ public class TestINodeFile {
INodeFile[] iNodes = new INodeFile[nCount]; INodeFile[] iNodes = new INodeFile[nCount];
for (int i = 0; i < nCount; i++) { for (int i = 0; i < nCount; i++) {
iNodes[i] = new INodeFile(i, null, perm, 0L, 0L, null, replication, iNodes[i] = new INodeFile(i, null, perm, 0L, 0L, null, replication,
preferredBlockSize); preferredBlockSize, (byte)0);
iNodes[i].setLocalName(DFSUtil.string2Bytes(fileNamePrefix + i)); iNodes[i].setLocalName(DFSUtil.string2Bytes(fileNamePrefix + i));
BlockInfo newblock = new BlockInfo(replication); BlockInfo newblock = new BlockInfo(replication);
iNodes[i].addBlock(newblock); iNodes[i].addBlock(newblock);
@ -319,7 +343,8 @@ public class TestINodeFile {
{//cast from INodeFileUnderConstruction {//cast from INodeFileUnderConstruction
final INode from = new INodeFile( final INode from = new INodeFile(
INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication, 1024L); INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication,
1024L, (byte)0);
from.asFile().toUnderConstruction("client", "machine"); from.asFile().toUnderConstruction("client", "machine");
//cast to INodeFile, should success //cast to INodeFile, should success
@ -1043,7 +1068,7 @@ public class TestINodeFile {
public void testFileUnderConstruction() { public void testFileUnderConstruction() {
replication = 3; replication = 3;
final INodeFile file = new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, final INodeFile file = new INodeFile(INodeId.GRANDFATHER_INODE_ID, null,
perm, 0L, 0L, null, replication, 1024L); perm, 0L, 0L, null, replication, 1024L, (byte)0);
assertFalse(file.isUnderConstruction()); assertFalse(file.isUnderConstruction());
final String clientName = "client"; final String clientName = "client";

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology; import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB; import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
@ -588,7 +589,8 @@ public class TestDNFencing {
public DatanodeStorageInfo chooseReplicaToDelete(BlockCollection inode, public DatanodeStorageInfo chooseReplicaToDelete(BlockCollection inode,
Block block, short replicationFactor, Block block, short replicationFactor,
Collection<DatanodeStorageInfo> first, Collection<DatanodeStorageInfo> first,
Collection<DatanodeStorageInfo> second) { Collection<DatanodeStorageInfo> second,
List<StorageType> excessTypes) {
Collection<DatanodeStorageInfo> chooseFrom = !first.isEmpty() ? first : second; Collection<DatanodeStorageInfo> chooseFrom = !first.isEmpty() ? first : second;

View File

@ -64,7 +64,7 @@ public class TestJsonUtil {
final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26, final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26,
now, now + 10, new FsPermission((short) 0644), "user", "group", now, now + 10, new FsPermission((short) 0644), "user", "group",
DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"), DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"),
INodeId.GRANDFATHER_INODE_ID, 0, null); INodeId.GRANDFATHER_INODE_ID, 0, null, (byte) 0);
final FileStatus fstatus = toFileStatus(status, parent); final FileStatus fstatus = toFileStatus(status, parent);
System.out.println("status = " + status); System.out.println("status = " + status);
System.out.println("fstatus = " + fstatus); System.out.println("fstatus = " + fstatus);

View File

@ -93,6 +93,7 @@
<item name="Extended Attributes" href="hadoop-project-dist/hadoop-hdfs/ExtendedAttributes.html"/> <item name="Extended Attributes" href="hadoop-project-dist/hadoop-hdfs/ExtendedAttributes.html"/>
<item name="Transparent Encryption" href="hadoop-project-dist/hadoop-hdfs/TransparentEncryption.html"/> <item name="Transparent Encryption" href="hadoop-project-dist/hadoop-hdfs/TransparentEncryption.html"/>
<item name="HDFS Support for Multihoming" href="hadoop-project-dist/hadoop-hdfs/HdfsMultihoming.html"/> <item name="HDFS Support for Multihoming" href="hadoop-project-dist/hadoop-hdfs/HdfsMultihoming.html"/>
<item name="Archival Storage" href="hadoop-project-dist/hadoop-hdfs/ArchivalStorage.html"/>
</menu> </menu>
<menu name="MapReduce" inherit="top"> <menu name="MapReduce" inherit="top">