|
@@ -18,10 +18,40 @@
|
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
-import com.google.common.base.Joiner;
|
|
|
-import com.google.common.base.Preconditions;
|
|
|
-import com.google.protobuf.BlockingService;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
|
|
|
+import static org.apache.hadoop.util.ExitUtil.terminate;
|
|
|
+
|
|
|
+import java.io.BufferedOutputStream;
|
|
|
+import java.io.ByteArrayInputStream;
|
|
|
+import java.io.DataInputStream;
|
|
|
+import java.io.DataOutputStream;
|
|
|
+import java.io.FileInputStream;
|
|
|
+import java.io.IOException;
|
|
|
+import java.io.InputStream;
|
|
|
+import java.io.OutputStream;
|
|
|
+import java.io.PrintStream;
|
|
|
+import java.net.InetSocketAddress;
|
|
|
+import java.net.Socket;
|
|
|
+import java.net.SocketException;
|
|
|
+import java.net.SocketTimeoutException;
|
|
|
+import java.net.URI;
|
|
|
+import java.net.UnknownHostException;
|
|
|
+import java.nio.channels.ClosedByInterruptException;
|
|
|
+import java.nio.channels.SocketChannel;
|
|
|
+import java.security.PrivilegedExceptionAction;
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.Arrays;
|
|
|
+import java.util.Collection;
|
|
|
+import java.util.EnumSet;
|
|
|
+import java.util.HashMap;
|
|
|
+import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.Set;
|
|
|
+import java.util.UUID;
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
+
|
|
|
+import javax.management.ObjectName;
|
|
|
+
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
@@ -38,15 +68,37 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.net.DomainPeerServer;
|
|
|
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
|
|
-import org.apache.hadoop.hdfs.protocol.*;
|
|
|
-import org.apache.hadoop.hdfs.protocol.datatransfer.*;
|
|
|
+import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
|
|
|
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
+import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
|
|
|
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
|
|
|
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
|
|
|
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
|
|
|
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
|
|
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InterDatanodeProtocolService;
|
|
|
-import org.apache.hadoop.hdfs.protocolPB.*;
|
|
|
-import org.apache.hadoop.hdfs.security.token.block.*;
|
|
|
+import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolPB;
|
|
|
+import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolServerSideTranslatorPB;
|
|
|
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
|
|
|
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolPB;
|
|
|
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolServerSideTranslatorPB;
|
|
|
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolTranslatorPB;
|
|
|
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
|
|
+import org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager;
|
|
|
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
|
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
|
|
|
+import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
|
|
|
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
|
@@ -59,8 +111,12 @@ import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.web.resources.DatanodeWebHdfsMethods;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.StreamFile;
|
|
|
-import org.apache.hadoop.hdfs.server.protocol.*;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
|
|
|
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
|
|
|
import org.apache.hadoop.hdfs.web.resources.Param;
|
|
|
import org.apache.hadoop.http.HttpServer;
|
|
@@ -82,23 +138,21 @@ import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
|
|
|
import org.apache.hadoop.security.authorize.AccessControlList;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
|
-import org.apache.hadoop.util.*;
|
|
|
+import org.apache.hadoop.util.Daemon;
|
|
|
+import org.apache.hadoop.util.DiskChecker;
|
|
|
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
|
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
|
|
|
+import org.apache.hadoop.util.GenericOptionsParser;
|
|
|
+import org.apache.hadoop.util.JvmPauseMonitor;
|
|
|
+import org.apache.hadoop.util.ServicePlugin;
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
|
+import org.apache.hadoop.util.VersionInfo;
|
|
|
import org.mortbay.util.ajax.JSON;
|
|
|
|
|
|
-import java.io.*;
|
|
|
-import java.net.*;
|
|
|
-import java.nio.channels.ClosedByInterruptException;
|
|
|
-import java.nio.channels.SocketChannel;
|
|
|
-import java.security.PrivilegedExceptionAction;
|
|
|
-import java.util.*;
|
|
|
-import java.util.concurrent.atomic.AtomicInteger;
|
|
|
-
|
|
|
-import javax.management.ObjectName;
|
|
|
-
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
|
|
|
-import static org.apache.hadoop.util.ExitUtil.terminate;
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
+import com.google.common.base.Joiner;
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
+import com.google.protobuf.BlockingService;
|
|
|
|
|
|
/**********************************************************
|
|
|
* DataNode is a class (and program) that stores a set of
|
|
@@ -204,7 +258,7 @@ public class DataNode extends Configured
|
|
|
private JvmPauseMonitor pauseMonitor;
|
|
|
|
|
|
private SecureResources secureResources = null;
|
|
|
- private AbstractList<StorageLocation> dataDirs;
|
|
|
+ private List<StorageLocation> dataDirs;
|
|
|
private Configuration conf;
|
|
|
private final long maxNumberOfBlocksToLog;
|
|
|
|
|
@@ -219,7 +273,7 @@ public class DataNode extends Configured
|
|
|
* and a namenode proxy
|
|
|
*/
|
|
|
DataNode(final Configuration conf,
|
|
|
- final AbstractList<StorageLocation> dataDirs,
|
|
|
+ final List<StorageLocation> dataDirs,
|
|
|
final SecureResources resources) throws IOException {
|
|
|
super(conf);
|
|
|
this.maxNumberOfBlocksToLog = conf.getLong(DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
|
|
@@ -640,7 +694,7 @@ public class DataNode extends Configured
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
void startDataNode(Configuration conf,
|
|
|
- AbstractList<StorageLocation> dataDirs,
|
|
|
+ List<StorageLocation> dataDirs,
|
|
|
// DatanodeProtocol namenode,
|
|
|
SecureResources resources
|
|
|
) throws IOException {
|
|
@@ -915,7 +969,7 @@ public class DataNode extends Configured
|
|
|
/**
|
|
|
* NB: The datanode can perform data transfer on the streaming
|
|
|
* address however clients are given the IPC IP address for data
|
|
|
- * transfer, and that may be a different address.
|
|
|
+ * transfer, and that may be a different address.
|
|
|
*
|
|
|
* @return socket address for data transfer
|
|
|
*/
|
|
@@ -1660,21 +1714,19 @@ public class DataNode extends Configured
|
|
|
return makeInstance(dataLocations, conf, resources);
|
|
|
}
|
|
|
|
|
|
- static Collection<StorageLocation> parseStorageLocations(
|
|
|
- Collection<String> rawLocations) {
|
|
|
+ public static List<StorageLocation> getStorageLocations(Configuration conf) {
|
|
|
+ Collection<String> rawLocations =
|
|
|
+ conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY);
|
|
|
List<StorageLocation> locations =
|
|
|
new ArrayList<StorageLocation>(rawLocations.size());
|
|
|
|
|
|
for(String locationString : rawLocations) {
|
|
|
- StorageLocation location;
|
|
|
+ final StorageLocation location;
|
|
|
try {
|
|
|
location = StorageLocation.parse(locationString);
|
|
|
} catch (IOException ioe) {
|
|
|
- LOG.error("Failed to parse storage location " + locationString);
|
|
|
- continue;
|
|
|
- } catch (IllegalArgumentException iae) {
|
|
|
- LOG.error(iae.toString());
|
|
|
- continue;
|
|
|
+ throw new IllegalArgumentException("Failed to parse conf property "
|
|
|
+ + DFS_DATANODE_DATA_DIR_KEY + ": " + locationString, ioe);
|
|
|
}
|
|
|
|
|
|
locations.add(location);
|
|
@@ -1683,12 +1735,6 @@ public class DataNode extends Configured
|
|
|
return locations;
|
|
|
}
|
|
|
|
|
|
- public static Collection<StorageLocation> getStorageLocations(
|
|
|
- Configuration conf) {
|
|
|
- return parseStorageLocations(
|
|
|
- conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY));
|
|
|
- }
|
|
|
-
|
|
|
/** Instantiate & Start a single datanode daemon and wait for it to finish.
|
|
|
* If this thread is specifically interrupted, it will stop waiting.
|
|
|
*/
|
|
@@ -1760,7 +1806,7 @@ public class DataNode extends Configured
|
|
|
DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT));
|
|
|
DataNodeDiskChecker dataNodeDiskChecker =
|
|
|
new DataNodeDiskChecker(permission);
|
|
|
- ArrayList<StorageLocation> locations =
|
|
|
+ List<StorageLocation> locations =
|
|
|
checkStorageLocations(dataDirs, localFS, dataNodeDiskChecker);
|
|
|
DefaultMetricsSystem.initialize("DataNode");
|
|
|
|
|
@@ -1769,20 +1815,21 @@ public class DataNode extends Configured
|
|
|
}
|
|
|
|
|
|
// DataNode ctor expects AbstractList instead of List or Collection...
|
|
|
- static ArrayList<StorageLocation> checkStorageLocations(
|
|
|
+ static List<StorageLocation> checkStorageLocations(
|
|
|
Collection<StorageLocation> dataDirs,
|
|
|
LocalFileSystem localFS, DataNodeDiskChecker dataNodeDiskChecker)
|
|
|
throws IOException {
|
|
|
ArrayList<StorageLocation> locations = new ArrayList<StorageLocation>();
|
|
|
StringBuilder invalidDirs = new StringBuilder();
|
|
|
for (StorageLocation location : dataDirs) {
|
|
|
+ final URI uri = location.getUri();
|
|
|
try {
|
|
|
- dataNodeDiskChecker.checkDir(localFS, new Path(location.getUri()));
|
|
|
+ dataNodeDiskChecker.checkDir(localFS, new Path(uri));
|
|
|
locations.add(location);
|
|
|
} catch (IOException ioe) {
|
|
|
LOG.warn("Invalid " + DFS_DATANODE_DATA_DIR_KEY + " "
|
|
|
+ location.getFile() + " : ", ioe);
|
|
|
- invalidDirs.append("\"").append(location.getUri().getPath()).append("\" ");
|
|
|
+ invalidDirs.append("\"").append(uri.getPath()).append("\" ");
|
|
|
}
|
|
|
}
|
|
|
if (locations.size() == 0) {
|