|
@@ -48,7 +48,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTO
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY;
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY;
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STORAGEID_KEY;
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STORAGEID_KEY;
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY;
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY;
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICES;
|
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTPS_ENABLE_KEY;
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTPS_ENABLE_KEY;
|
|
|
|
|
|
import java.io.BufferedOutputStream;
|
|
import java.io.BufferedOutputStream;
|
|
@@ -71,12 +70,10 @@ import java.util.AbstractList;
|
|
import java.util.ArrayList;
|
|
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.EnumSet;
|
|
import java.util.EnumSet;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
-import java.util.Map.Entry;
|
|
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
@@ -91,7 +88,6 @@ import org.apache.hadoop.fs.LocalFileSystem;
|
|
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.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
-import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
|
import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
|
|
import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
|
|
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
|
|
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
@@ -169,7 +165,6 @@ import org.apache.hadoop.util.VersionInfo;
|
|
import org.mortbay.util.ajax.JSON;
|
|
import org.mortbay.util.ajax.JSON;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
-import com.google.common.collect.Lists;
|
|
|
|
import com.google.common.collect.Sets;
|
|
import com.google.common.collect.Sets;
|
|
|
|
|
|
|
|
|
|
@@ -236,163 +231,6 @@ public class DataNode extends Configured
|
|
return NetUtils.createSocketAddr(target);
|
|
return NetUtils.createSocketAddr(target);
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Manages he BPOfferService objects for the data node.
|
|
|
|
- * Creation, removal, starting, stopping, shutdown on BPOfferService
|
|
|
|
- * objects must be done via APIs in this class.
|
|
|
|
- */
|
|
|
|
- @InterfaceAudience.Private
|
|
|
|
- class BlockPoolManager {
|
|
|
|
- private final Map<String, BPOfferService> bpMapping;
|
|
|
|
- private final List<BPOfferService> offerServices;
|
|
|
|
-
|
|
|
|
- //This lock is used only to ensure exclusion of refreshNamenodes
|
|
|
|
- private final Object refreshNamenodesLock = new Object();
|
|
|
|
-
|
|
|
|
- BlockPoolManager(Configuration conf)
|
|
|
|
- throws IOException {
|
|
|
|
- bpMapping = new HashMap<String, BPOfferService>();
|
|
|
|
- offerServices = new ArrayList<BPOfferService>();
|
|
|
|
-
|
|
|
|
- Map<String, Map<String, InetSocketAddress>> map =
|
|
|
|
- DFSUtil.getNNServiceRpcAddresses(conf);
|
|
|
|
- for (Entry<String, Map<String, InetSocketAddress>> entry :
|
|
|
|
- map.entrySet()) {
|
|
|
|
- List<InetSocketAddress> nnList = Lists.newArrayList(entry.getValue().values());
|
|
|
|
- BPOfferService bpos = new BPOfferService(nnList, DataNode.this);
|
|
|
|
- offerServices.add(bpos);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- synchronized void addBlockPool(BPOfferService bpos) {
|
|
|
|
- Preconditions.checkArgument(offerServices.contains(bpos),
|
|
|
|
- "Unknown BPOS: %s", bpos);
|
|
|
|
- if (bpos.getBlockPoolId() == null) {
|
|
|
|
- throw new IllegalArgumentException("Null blockpool id");
|
|
|
|
- }
|
|
|
|
- LOG.info("===> registering in bpmapping: " + bpos);
|
|
|
|
- bpMapping.put(bpos.getBlockPoolId(), bpos);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Returns the array of BPOfferService objects.
|
|
|
|
- * Caution: The BPOfferService returned could be shutdown any time.
|
|
|
|
- */
|
|
|
|
- synchronized BPOfferService[] getAllNamenodeThreads() {
|
|
|
|
- BPOfferService[] bposArray = new BPOfferService[offerServices.size()];
|
|
|
|
- return offerServices.toArray(bposArray);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- synchronized BPOfferService get(String bpid) {
|
|
|
|
- return bpMapping.get(bpid);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // TODO(HA) would be good to kill this
|
|
|
|
- synchronized BPOfferService get(InetSocketAddress addr) {
|
|
|
|
- for (BPOfferService bpos : offerServices) {
|
|
|
|
- if (bpos.containsNN(addr)) {
|
|
|
|
- return bpos;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- synchronized void remove(BPOfferService t) {
|
|
|
|
- offerServices.remove(t);
|
|
|
|
- bpMapping.remove(t.getBlockPoolId());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- void shutDownAll() throws InterruptedException {
|
|
|
|
- BPOfferService[] bposArray = this.getAllNamenodeThreads();
|
|
|
|
-
|
|
|
|
- for (BPOfferService bpos : bposArray) {
|
|
|
|
- bpos.stop(); //interrupts the threads
|
|
|
|
- }
|
|
|
|
- //now join
|
|
|
|
- for (BPOfferService bpos : bposArray) {
|
|
|
|
- bpos.join();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- synchronized void startAll() throws IOException {
|
|
|
|
- try {
|
|
|
|
- UserGroupInformation.getLoginUser().doAs(
|
|
|
|
- new PrivilegedExceptionAction<Object>() {
|
|
|
|
- public Object run() throws Exception {
|
|
|
|
- for (BPOfferService bpos : offerServices) {
|
|
|
|
- bpos.start();
|
|
|
|
- }
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
- });
|
|
|
|
- } catch (InterruptedException ex) {
|
|
|
|
- IOException ioe = new IOException();
|
|
|
|
- ioe.initCause(ex.getCause());
|
|
|
|
- throw ioe;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- void joinAll() {
|
|
|
|
- for (BPOfferService bpos: this.getAllNamenodeThreads()) {
|
|
|
|
- bpos.join();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- void refreshNamenodes(Configuration conf)
|
|
|
|
- throws IOException {
|
|
|
|
- throw new UnsupportedOperationException("TODO(HA)");
|
|
|
|
-/*
|
|
|
|
- * TODO(HA)
|
|
|
|
-
|
|
|
|
- LOG.info("Refresh request received for nameservices: "
|
|
|
|
- + conf.get(DFS_FEDERATION_NAMESERVICES));
|
|
|
|
-
|
|
|
|
- // TODO(HA): need to update this for multiple NNs per nameservice
|
|
|
|
- // For now, just list all of the NNs into this set
|
|
|
|
- Map<String, Map<String, InetSocketAddress>> newAddressMap =
|
|
|
|
- DFSUtil.getNNServiceRpcAddresses(conf);
|
|
|
|
- Set<InetSocketAddress> newAddresses = Sets.newHashSet();
|
|
|
|
- for (ConfiguredNNAddress cnn : DFSUtil.flattenAddressMap(newAddressMap)) {
|
|
|
|
- newAddresses.add(cnn.getAddress());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- List<BPOfferService> toShutdown = new ArrayList<BPOfferService>();
|
|
|
|
- List<InetSocketAddress> toStart = new ArrayList<InetSocketAddress>();
|
|
|
|
- synchronized (refreshNamenodesLock) {
|
|
|
|
- synchronized (this) {
|
|
|
|
- for (InetSocketAddress nnaddr : offerServices.keySet()) {
|
|
|
|
- if (!(newAddresses.contains(nnaddr))) {
|
|
|
|
- toShutdown.add(offerServices.get(nnaddr));
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- for (InetSocketAddress nnaddr : newAddresses) {
|
|
|
|
- if (!(offerServices.containsKey(nnaddr))) {
|
|
|
|
- toStart.add(nnaddr);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- for (InetSocketAddress nnaddr : toStart) {
|
|
|
|
- BPOfferService bpos = new BPOfferService(nnaddr, DataNode.this);
|
|
|
|
- offerServices.put(bpos.getNNSocketAddress(), bpos);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- for (BPOfferService bpos : toShutdown) {
|
|
|
|
- bpos.stop();
|
|
|
|
- bpos.join();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // stoping the BPOSes causes them to call remove() on their own when they
|
|
|
|
- // clean up.
|
|
|
|
-
|
|
|
|
- // Now start the threads that are not already running.
|
|
|
|
- startAll();
|
|
|
|
- }
|
|
|
|
- */
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
volatile boolean shouldRun = true;
|
|
volatile boolean shouldRun = true;
|
|
private BlockPoolManager blockPoolManager;
|
|
private BlockPoolManager blockPoolManager;
|
|
public volatile FSDatasetInterface data = null;
|
|
public volatile FSDatasetInterface data = null;
|
|
@@ -779,7 +617,8 @@ public class DataNode extends Configured
|
|
|
|
|
|
metrics = DataNodeMetrics.create(conf, getMachineName());
|
|
metrics = DataNodeMetrics.create(conf, getMachineName());
|
|
|
|
|
|
- blockPoolManager = new BlockPoolManager(conf);
|
|
|
|
|
|
+ blockPoolManager = new BlockPoolManager(this);
|
|
|
|
+ blockPoolManager.refreshNamenodes(conf);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|