001 /**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements. See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership. The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License. You may obtain a copy of the License at
009 *
010 * http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018 package org.apache.hadoop.hdfs.server.balancer;
019
020 import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
021
022 import java.io.BufferedInputStream;
023 import java.io.BufferedOutputStream;
024 import java.io.DataInputStream;
025 import java.io.DataOutputStream;
026 import java.io.IOException;
027 import java.io.InputStream;
028 import java.io.OutputStream;
029 import java.net.Socket;
030 import java.util.ArrayList;
031 import java.util.Arrays;
032 import java.util.Collection;
033 import java.util.EnumMap;
034 import java.util.HashMap;
035 import java.util.HashSet;
036 import java.util.Iterator;
037 import java.util.List;
038 import java.util.Map;
039 import java.util.Set;
040 import java.util.concurrent.ExecutionException;
041 import java.util.concurrent.ExecutorService;
042 import java.util.concurrent.Executors;
043 import java.util.concurrent.Future;
044
045 import org.apache.commons.logging.Log;
046 import org.apache.commons.logging.LogFactory;
047 import org.apache.hadoop.classification.InterfaceAudience;
048 import org.apache.hadoop.conf.Configuration;
049 import org.apache.hadoop.fs.CommonConfigurationKeys;
050 import org.apache.hadoop.hdfs.DFSConfigKeys;
051 import org.apache.hadoop.hdfs.DFSUtil;
052 import org.apache.hadoop.hdfs.DistributedFileSystem;
053 import org.apache.hadoop.hdfs.StorageType;
054 import org.apache.hadoop.hdfs.protocol.Block;
055 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
056 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
057 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
058 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
059 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
060 import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
061 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil;
062 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
063 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
064 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
065 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
066 import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DDatanode.StorageGroup;
067 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
068 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
069 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
070 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
071 import org.apache.hadoop.io.IOUtils;
072 import org.apache.hadoop.net.NetUtils;
073 import org.apache.hadoop.net.NetworkTopology;
074 import org.apache.hadoop.security.token.Token;
075 import org.apache.hadoop.util.HostsFileReader;
076 import org.apache.hadoop.util.StringUtils;
077 import org.apache.hadoop.util.Time;
078
079 import com.google.common.annotations.VisibleForTesting;
080 import com.google.common.base.Preconditions;
081
082 /** Dispatching block replica moves between datanodes. */
083 @InterfaceAudience.Private
084 public class Dispatcher {
085 static final Log LOG = LogFactory.getLog(Dispatcher.class);
086
087 private static final long GB = 1L << 30; // 1GB
088 private static final long MAX_BLOCKS_SIZE_TO_FETCH = 2 * GB;
089
090 private static final int MAX_NO_PENDING_MOVE_ITERATIONS = 5;
091 /**
092 * the period of time to delay the usage of a DataNode after hitting
093 * errors when using it for migrating data
094 */
095 private static long delayAfterErrors = 10 * 1000;
096
097 private final NameNodeConnector nnc;
098 private final SaslDataTransferClient saslClient;
099
100 /** Set of datanodes to be excluded. */
101 private final Set<String> excludedNodes;
102 /** Restrict to the following nodes. */
103 private final Set<String> includedNodes;
104
105 private final Collection<Source> sources = new HashSet<Source>();
106 private final Collection<StorageGroup> targets = new HashSet<StorageGroup>();
107
108 private final GlobalBlockMap globalBlocks = new GlobalBlockMap();
109 private final MovedBlocks<StorageGroup> movedBlocks;
110
111 /** Map (datanodeUuid,storageType -> StorageGroup) */
112 private final StorageGroupMap<StorageGroup> storageGroupMap
113 = new StorageGroupMap<StorageGroup>();
114
115 private NetworkTopology cluster;
116
117 private final ExecutorService moveExecutor;
118 private final ExecutorService dispatchExecutor;
119
120 /** The maximum number of concurrent blocks moves at a datanode */
121 private final int maxConcurrentMovesPerNode;
122
123 private static class GlobalBlockMap {
124 private final Map<Block, DBlock> map = new HashMap<Block, DBlock>();
125
126 /**
127 * Get the block from the map;
128 * if the block is not found, create a new block and put it in the map.
129 */
130 private DBlock get(Block b) {
131 DBlock block = map.get(b);
132 if (block == null) {
133 block = new DBlock(b);
134 map.put(b, block);
135 }
136 return block;
137 }
138
139 /** Remove all blocks except for the moved blocks. */
140 private void removeAllButRetain(MovedBlocks<StorageGroup> movedBlocks) {
141 for (Iterator<Block> i = map.keySet().iterator(); i.hasNext();) {
142 if (!movedBlocks.contains(i.next())) {
143 i.remove();
144 }
145 }
146 }
147 }
148
149 public static class StorageGroupMap<G extends StorageGroup> {
150 private static String toKey(String datanodeUuid, StorageType storageType) {
151 return datanodeUuid + ":" + storageType;
152 }
153
154 private final Map<String, G> map = new HashMap<String, G>();
155
156 public G get(String datanodeUuid, StorageType storageType) {
157 return map.get(toKey(datanodeUuid, storageType));
158 }
159
160 public void put(G g) {
161 final String key = toKey(g.getDatanodeInfo().getDatanodeUuid(), g.storageType);
162 final StorageGroup existing = map.put(key, g);
163 Preconditions.checkState(existing == null);
164 }
165
166 int size() {
167 return map.size();
168 }
169
170 void clear() {
171 map.clear();
172 }
173
174 public Collection<G> values() {
175 return map.values();
176 }
177 }
178
179 /** This class keeps track of a scheduled block move */
180 public class PendingMove {
181 private DBlock block;
182 private Source source;
183 private DDatanode proxySource;
184 private StorageGroup target;
185
186 private PendingMove(Source source, StorageGroup target) {
187 this.source = source;
188 this.target = target;
189 }
190
191 @Override
192 public String toString() {
193 final Block b = block != null ? block.getBlock() : null;
194 String bStr = b != null ? (b + " with size=" + b.getNumBytes() + " ")
195 : " ";
196 return bStr + "from " + source.getDisplayName() + " to " + target
197 .getDisplayName() + " through " + (proxySource != null ? proxySource
198 .datanode : "");
199 }
200
201 /**
202 * Choose a block & a proxy source for this pendingMove whose source &
203 * target have already been chosen.
204 *
205 * @return true if a block and its proxy are chosen; false otherwise
206 */
207 private boolean chooseBlockAndProxy() {
208 // source and target must have the same storage type
209 final StorageType t = source.getStorageType();
210 // iterate all source's blocks until find a good one
211 for (Iterator<DBlock> i = source.getBlockIterator(); i.hasNext();) {
212 if (markMovedIfGoodBlock(i.next(), t)) {
213 i.remove();
214 return true;
215 }
216 }
217 return false;
218 }
219
220 /**
221 * @return true if the given block is good for the tentative move.
222 */
223 private boolean markMovedIfGoodBlock(DBlock block, StorageType targetStorageType) {
224 synchronized (block) {
225 synchronized (movedBlocks) {
226 if (isGoodBlockCandidate(source, target, targetStorageType, block)) {
227 this.block = block;
228 if (chooseProxySource()) {
229 movedBlocks.put(block);
230 if (LOG.isDebugEnabled()) {
231 LOG.debug("Decided to move " + this);
232 }
233 return true;
234 }
235 }
236 }
237 }
238 return false;
239 }
240
241 /**
242 * Choose a proxy source.
243 *
244 * @return true if a proxy is found; otherwise false
245 */
246 private boolean chooseProxySource() {
247 final DatanodeInfo targetDN = target.getDatanodeInfo();
248 // if node group is supported, first try add nodes in the same node group
249 if (cluster.isNodeGroupAware()) {
250 for (StorageGroup loc : block.getLocations()) {
251 if (cluster.isOnSameNodeGroup(loc.getDatanodeInfo(), targetDN)
252 && addTo(loc)) {
253 return true;
254 }
255 }
256 }
257 // check if there is replica which is on the same rack with the target
258 for (StorageGroup loc : block.getLocations()) {
259 if (cluster.isOnSameRack(loc.getDatanodeInfo(), targetDN) && addTo(loc)) {
260 return true;
261 }
262 }
263 // find out a non-busy replica
264 for (StorageGroup loc : block.getLocations()) {
265 if (addTo(loc)) {
266 return true;
267 }
268 }
269 return false;
270 }
271
272 /** add to a proxy source for specific block movement */
273 private boolean addTo(StorageGroup g) {
274 final DDatanode dn = g.getDDatanode();
275 if (dn.addPendingBlock(this)) {
276 proxySource = dn;
277 return true;
278 }
279 return false;
280 }
281
282 /** Dispatch the move to the proxy source & wait for the response. */
283 private void dispatch() {
284 if (LOG.isDebugEnabled()) {
285 LOG.debug("Start moving " + this);
286 }
287
288 Socket sock = new Socket();
289 DataOutputStream out = null;
290 DataInputStream in = null;
291 try {
292 sock.connect(
293 NetUtils.createSocketAddr(target.getDatanodeInfo().getXferAddr()),
294 HdfsServerConstants.READ_TIMEOUT);
295
296 sock.setKeepAlive(true);
297
298 OutputStream unbufOut = sock.getOutputStream();
299 InputStream unbufIn = sock.getInputStream();
300 ExtendedBlock eb = new ExtendedBlock(nnc.getBlockpoolID(),
301 block.getBlock());
302 final KeyManager km = nnc.getKeyManager();
303 Token<BlockTokenIdentifier> accessToken = km.getAccessToken(eb);
304 IOStreamPair saslStreams = saslClient.socketSend(sock, unbufOut,
305 unbufIn, km, accessToken, target.getDatanodeInfo());
306 unbufOut = saslStreams.out;
307 unbufIn = saslStreams.in;
308 out = new DataOutputStream(new BufferedOutputStream(unbufOut,
309 HdfsConstants.IO_FILE_BUFFER_SIZE));
310 in = new DataInputStream(new BufferedInputStream(unbufIn,
311 HdfsConstants.IO_FILE_BUFFER_SIZE));
312
313 sendRequest(out, eb, accessToken);
314 receiveResponse(in);
315 nnc.getBytesMoved().addAndGet(block.getNumBytes());
316 LOG.info("Successfully moved " + this);
317 } catch (IOException e) {
318 LOG.warn("Failed to move " + this + ": " + e.getMessage());
319 target.getDDatanode().setHasFailure();
320 // Proxy or target may have some issues, delay before using these nodes
321 // further in order to avoid a potential storm of "threads quota
322 // exceeded" warnings when the dispatcher gets out of sync with work
323 // going on in datanodes.
324 proxySource.activateDelay(delayAfterErrors);
325 target.getDDatanode().activateDelay(delayAfterErrors);
326 } finally {
327 IOUtils.closeStream(out);
328 IOUtils.closeStream(in);
329 IOUtils.closeSocket(sock);
330
331 proxySource.removePendingBlock(this);
332 target.getDDatanode().removePendingBlock(this);
333
334 synchronized (this) {
335 reset();
336 }
337 synchronized (Dispatcher.this) {
338 Dispatcher.this.notifyAll();
339 }
340 }
341 }
342
343 /** Send a block replace request to the output stream */
344 private void sendRequest(DataOutputStream out, ExtendedBlock eb,
345 Token<BlockTokenIdentifier> accessToken) throws IOException {
346 new Sender(out).replaceBlock(eb, target.storageType, accessToken,
347 source.getDatanodeInfo().getDatanodeUuid(), proxySource.datanode);
348 }
349
350 /** Receive a block copy response from the input stream */
351 private void receiveResponse(DataInputStream in) throws IOException {
352 BlockOpResponseProto response =
353 BlockOpResponseProto.parseFrom(vintPrefixed(in));
354 while (response.getStatus() == Status.IN_PROGRESS) {
355 // read intermediate responses
356 response = BlockOpResponseProto.parseFrom(vintPrefixed(in));
357 }
358 if (response.getStatus() != Status.SUCCESS) {
359 if (response.getStatus() == Status.ERROR_ACCESS_TOKEN) {
360 throw new IOException("block move failed due to access token error");
361 }
362 throw new IOException("block move is failed: " + response.getMessage());
363 }
364 }
365
366 /** reset the object */
367 private void reset() {
368 block = null;
369 source = null;
370 proxySource = null;
371 target = null;
372 }
373 }
374
375 /** A class for keeping track of block locations in the dispatcher. */
376 public static class DBlock extends MovedBlocks.Locations<StorageGroup> {
377 public DBlock(Block block) {
378 super(block);
379 }
380
381 @Override
382 public synchronized boolean isLocatedOn(StorageGroup loc) {
383 // currently we only check if replicas are located on the same DataNodes
384 // since we do not have the capability to store two replicas in the same
385 // DataNode even though they are on two different storage types
386 for (StorageGroup existing : locations) {
387 if (existing.getDatanodeInfo().equals(loc.getDatanodeInfo())) {
388 return true;
389 }
390 }
391 return false;
392 }
393 }
394
395 /** The class represents a desired move. */
396 static class Task {
397 private final StorageGroup target;
398 private long size; // bytes scheduled to move
399
400 Task(StorageGroup target, long size) {
401 this.target = target;
402 this.size = size;
403 }
404
405 long getSize() {
406 return size;
407 }
408 }
409
410 /** A class that keeps track of a datanode. */
411 public static class DDatanode {
412
413 /** A group of storages in a datanode with the same storage type. */
414 public class StorageGroup {
415 final StorageType storageType;
416 final long maxSize2Move;
417 private long scheduledSize = 0L;
418
419 private StorageGroup(StorageType storageType, long maxSize2Move) {
420 this.storageType = storageType;
421 this.maxSize2Move = maxSize2Move;
422 }
423
424 public StorageType getStorageType() {
425 return storageType;
426 }
427
428 private DDatanode getDDatanode() {
429 return DDatanode.this;
430 }
431
432 public DatanodeInfo getDatanodeInfo() {
433 return DDatanode.this.datanode;
434 }
435
436 /** Decide if still need to move more bytes */
437 boolean hasSpaceForScheduling() {
438 return hasSpaceForScheduling(0L);
439 }
440
441 synchronized boolean hasSpaceForScheduling(long size) {
442 return availableSizeToMove() > size;
443 }
444
445 /** @return the total number of bytes that need to be moved */
446 synchronized long availableSizeToMove() {
447 return maxSize2Move - scheduledSize;
448 }
449
450 /** increment scheduled size */
451 public synchronized void incScheduledSize(long size) {
452 scheduledSize += size;
453 }
454
455 /** @return scheduled size */
456 synchronized long getScheduledSize() {
457 return scheduledSize;
458 }
459
460 /** Reset scheduled size to zero. */
461 synchronized void resetScheduledSize() {
462 scheduledSize = 0L;
463 }
464
465 private PendingMove addPendingMove(DBlock block, final PendingMove pm) {
466 if (getDDatanode().addPendingBlock(pm)) {
467 if (pm.markMovedIfGoodBlock(block, getStorageType())) {
468 incScheduledSize(pm.block.getNumBytes());
469 return pm;
470 } else {
471 getDDatanode().removePendingBlock(pm);
472 }
473 }
474 return null;
475 }
476
477 /** @return the name for display */
478 String getDisplayName() {
479 return datanode + ":" + storageType;
480 }
481
482 @Override
483 public String toString() {
484 return getDisplayName();
485 }
486 }
487
488 final DatanodeInfo datanode;
489 private final EnumMap<StorageType, Source> sourceMap
490 = new EnumMap<StorageType, Source>(StorageType.class);
491 private final EnumMap<StorageType, StorageGroup> targetMap
492 = new EnumMap<StorageType, StorageGroup>(StorageType.class);
493 protected long delayUntil = 0L;
494 /** blocks being moved but not confirmed yet */
495 private final List<PendingMove> pendings;
496 private volatile boolean hasFailure = false;
497 private final int maxConcurrentMoves;
498
499 @Override
500 public String toString() {
501 return getClass().getSimpleName() + ":" + datanode;
502 }
503
504 private DDatanode(DatanodeInfo datanode, int maxConcurrentMoves) {
505 this.datanode = datanode;
506 this.maxConcurrentMoves = maxConcurrentMoves;
507 this.pendings = new ArrayList<PendingMove>(maxConcurrentMoves);
508 }
509
510 public DatanodeInfo getDatanodeInfo() {
511 return datanode;
512 }
513
514 private static <G extends StorageGroup> void put(StorageType storageType,
515 G g, EnumMap<StorageType, G> map) {
516 final StorageGroup existing = map.put(storageType, g);
517 Preconditions.checkState(existing == null);
518 }
519
520 public StorageGroup addTarget(StorageType storageType, long maxSize2Move) {
521 final StorageGroup g = new StorageGroup(storageType, maxSize2Move);
522 put(storageType, g, targetMap);
523 return g;
524 }
525
526 public Source addSource(StorageType storageType, long maxSize2Move, Dispatcher d) {
527 final Source s = d.new Source(storageType, maxSize2Move, this);
528 put(storageType, s, sourceMap);
529 return s;
530 }
531
532 synchronized private void activateDelay(long delta) {
533 delayUntil = Time.monotonicNow() + delta;
534 }
535
536 synchronized private boolean isDelayActive() {
537 if (delayUntil == 0 || Time.monotonicNow() > delayUntil) {
538 delayUntil = 0;
539 return false;
540 }
541 return true;
542 }
543
544 /** Check if the node can schedule more blocks to move */
545 synchronized boolean isPendingQNotFull() {
546 return pendings.size() < maxConcurrentMoves;
547 }
548
549 /** Check if all the dispatched moves are done */
550 synchronized boolean isPendingQEmpty() {
551 return pendings.isEmpty();
552 }
553
554 /** Add a scheduled block move to the node */
555 synchronized boolean addPendingBlock(PendingMove pendingBlock) {
556 if (!isDelayActive() && isPendingQNotFull()) {
557 return pendings.add(pendingBlock);
558 }
559 return false;
560 }
561
562 /** Remove a scheduled block move from the node */
563 synchronized boolean removePendingBlock(PendingMove pendingBlock) {
564 return pendings.remove(pendingBlock);
565 }
566
567 void setHasFailure() {
568 this.hasFailure = true;
569 }
570 }
571
572 /** A node that can be the sources of a block move */
573 public class Source extends DDatanode.StorageGroup {
574
575 private final List<Task> tasks = new ArrayList<Task>(2);
576 private long blocksToReceive = 0L;
577 /**
578 * Source blocks point to the objects in {@link Dispatcher#globalBlocks}
579 * because we want to keep one copy of a block and be aware that the
580 * locations are changing over time.
581 */
582 private final List<DBlock> srcBlocks = new ArrayList<DBlock>();
583
584 private Source(StorageType storageType, long maxSize2Move, DDatanode dn) {
585 dn.super(storageType, maxSize2Move);
586 }
587
588 /** Add a task */
589 void addTask(Task task) {
590 Preconditions.checkState(task.target != this,
591 "Source and target are the same storage group " + getDisplayName());
592 incScheduledSize(task.size);
593 tasks.add(task);
594 }
595
596 /** @return an iterator to this source's blocks */
597 Iterator<DBlock> getBlockIterator() {
598 return srcBlocks.iterator();
599 }
600
601 /**
602 * Fetch new blocks of this source from namenode and update this source's
603 * block list & {@link Dispatcher#globalBlocks}.
604 *
605 * @return the total size of the received blocks in the number of bytes.
606 */
607 private long getBlockList() throws IOException {
608 final long size = Math.min(MAX_BLOCKS_SIZE_TO_FETCH, blocksToReceive);
609 final BlocksWithLocations newBlocks = nnc.getBlocks(getDatanodeInfo(), size);
610
611 long bytesReceived = 0;
612 for (BlockWithLocations blk : newBlocks.getBlocks()) {
613 bytesReceived += blk.getBlock().getNumBytes();
614 synchronized (globalBlocks) {
615 final DBlock block = globalBlocks.get(blk.getBlock());
616 synchronized (block) {
617 block.clearLocations();
618
619 // update locations
620 final String[] datanodeUuids = blk.getDatanodeUuids();
621 final StorageType[] storageTypes = blk.getStorageTypes();
622 for (int i = 0; i < datanodeUuids.length; i++) {
623 final StorageGroup g = storageGroupMap.get(
624 datanodeUuids[i], storageTypes[i]);
625 if (g != null) { // not unknown
626 block.addLocation(g);
627 }
628 }
629 }
630 if (!srcBlocks.contains(block) && isGoodBlockCandidate(block)) {
631 // filter bad candidates
632 srcBlocks.add(block);
633 }
634 }
635 }
636 return bytesReceived;
637 }
638
639 /** Decide if the given block is a good candidate to move or not */
640 private boolean isGoodBlockCandidate(DBlock block) {
641 // source and target must have the same storage type
642 final StorageType sourceStorageType = getStorageType();
643 for (Task t : tasks) {
644 if (Dispatcher.this.isGoodBlockCandidate(this, t.target,
645 sourceStorageType, block)) {
646 return true;
647 }
648 }
649 return false;
650 }
651
652 /**
653 * Choose a move for the source. The block's source, target, and proxy
654 * are determined too. When choosing proxy and target, source &
655 * target throttling has been considered. They are chosen only when they
656 * have the capacity to support this block move. The block should be
657 * dispatched immediately after this method is returned.
658 *
659 * @return a move that's good for the source to dispatch immediately.
660 */
661 private PendingMove chooseNextMove() {
662 for (Iterator<Task> i = tasks.iterator(); i.hasNext();) {
663 final Task task = i.next();
664 final DDatanode target = task.target.getDDatanode();
665 final PendingMove pendingBlock = new PendingMove(this, task.target);
666 if (target.addPendingBlock(pendingBlock)) {
667 // target is not busy, so do a tentative block allocation
668 if (pendingBlock.chooseBlockAndProxy()) {
669 long blockSize = pendingBlock.block.getNumBytes();
670 incScheduledSize(-blockSize);
671 task.size -= blockSize;
672 if (task.size == 0) {
673 i.remove();
674 }
675 return pendingBlock;
676 } else {
677 // cancel the tentative move
678 target.removePendingBlock(pendingBlock);
679 }
680 }
681 }
682 return null;
683 }
684
685 /** Add a pending move */
686 public PendingMove addPendingMove(DBlock block, StorageGroup target) {
687 return target.addPendingMove(block, new PendingMove(this, target));
688 }
689
690 /** Iterate all source's blocks to remove moved ones */
691 private void removeMovedBlocks() {
692 for (Iterator<DBlock> i = getBlockIterator(); i.hasNext();) {
693 if (movedBlocks.contains(i.next().getBlock())) {
694 i.remove();
695 }
696 }
697 }
698
699 private static final int SOURCE_BLOCKS_MIN_SIZE = 5;
700
701 /** @return if should fetch more blocks from namenode */
702 private boolean shouldFetchMoreBlocks() {
703 return srcBlocks.size() < SOURCE_BLOCKS_MIN_SIZE && blocksToReceive > 0;
704 }
705
706 private static final long MAX_ITERATION_TIME = 20 * 60 * 1000L; // 20 mins
707
708 /**
709 * This method iteratively does the following: it first selects a block to
710 * move, then sends a request to the proxy source to start the block move
711 * when the source's block list falls below a threshold, it asks the
712 * namenode for more blocks. It terminates when it has dispatch enough block
713 * move tasks or it has received enough blocks from the namenode, or the
714 * elapsed time of the iteration has exceeded the max time limit.
715 */
716 private void dispatchBlocks() {
717 final long startTime = Time.monotonicNow();
718 this.blocksToReceive = 2 * getScheduledSize();
719 boolean isTimeUp = false;
720 int noPendingMoveIteration = 0;
721 while (!isTimeUp && getScheduledSize() > 0
722 && (!srcBlocks.isEmpty() || blocksToReceive > 0)) {
723 final PendingMove p = chooseNextMove();
724 if (p != null) {
725 // Reset no pending move counter
726 noPendingMoveIteration=0;
727 executePendingMove(p);
728 continue;
729 }
730
731 // Since we cannot schedule any block to move,
732 // remove any moved blocks from the source block list and
733 removeMovedBlocks(); // filter already moved blocks
734 // check if we should fetch more blocks from the namenode
735 if (shouldFetchMoreBlocks()) {
736 // fetch new blocks
737 try {
738 blocksToReceive -= getBlockList();
739 continue;
740 } catch (IOException e) {
741 LOG.warn("Exception while getting block list", e);
742 return;
743 }
744 } else {
745 // source node cannot find a pending block to move, iteration +1
746 noPendingMoveIteration++;
747 // in case no blocks can be moved for source node's task,
748 // jump out of while-loop after 5 iterations.
749 if (noPendingMoveIteration >= MAX_NO_PENDING_MOVE_ITERATIONS) {
750 resetScheduledSize();
751 }
752 }
753
754 // check if time is up or not
755 if (Time.monotonicNow() - startTime > MAX_ITERATION_TIME) {
756 isTimeUp = true;
757 continue;
758 }
759
760 // Now we can not schedule any block to move and there are
761 // no new blocks added to the source block list, so we wait.
762 try {
763 synchronized (Dispatcher.this) {
764 Dispatcher.this.wait(1000); // wait for targets/sources to be idle
765 }
766 } catch (InterruptedException ignored) {
767 }
768 }
769 }
770 }
771
772 public Dispatcher(NameNodeConnector nnc, Set<String> includedNodes,
773 Set<String> excludedNodes, long movedWinWidth, int moverThreads,
774 int dispatcherThreads, int maxConcurrentMovesPerNode, Configuration conf) {
775 this.nnc = nnc;
776 this.excludedNodes = excludedNodes;
777 this.includedNodes = includedNodes;
778 this.movedBlocks = new MovedBlocks<StorageGroup>(movedWinWidth);
779
780 this.cluster = NetworkTopology.getInstance(conf);
781
782 this.moveExecutor = Executors.newFixedThreadPool(moverThreads);
783 this.dispatchExecutor = dispatcherThreads == 0? null
784 : Executors.newFixedThreadPool(dispatcherThreads);
785 this.maxConcurrentMovesPerNode = maxConcurrentMovesPerNode;
786
787 this.saslClient = new SaslDataTransferClient(conf,
788 DataTransferSaslUtil.getSaslPropertiesResolver(conf),
789 TrustedChannelResolver.getInstance(conf), nnc.fallbackToSimpleAuth);
790 }
791
792 public DistributedFileSystem getDistributedFileSystem() {
793 return nnc.getDistributedFileSystem();
794 }
795
796 public StorageGroupMap<StorageGroup> getStorageGroupMap() {
797 return storageGroupMap;
798 }
799
800 public NetworkTopology getCluster() {
801 return cluster;
802 }
803
804 long getBytesMoved() {
805 return nnc.getBytesMoved().get();
806 }
807
808 long bytesToMove() {
809 Preconditions.checkState(
810 storageGroupMap.size() >= sources.size() + targets.size(),
811 "Mismatched number of storage groups (" + storageGroupMap.size()
812 + " < " + sources.size() + " sources + " + targets.size()
813 + " targets)");
814
815 long b = 0L;
816 for (Source src : sources) {
817 b += src.getScheduledSize();
818 }
819 return b;
820 }
821
822 void add(Source source, StorageGroup target) {
823 sources.add(source);
824 targets.add(target);
825 }
826
827 private boolean shouldIgnore(DatanodeInfo dn) {
828 // ignore decommissioned nodes
829 final boolean decommissioned = dn.isDecommissioned();
830 // ignore decommissioning nodes
831 final boolean decommissioning = dn.isDecommissionInProgress();
832 // ignore nodes in exclude list
833 final boolean excluded = Util.isExcluded(excludedNodes, dn);
834 // ignore nodes not in the include list (if include list is not empty)
835 final boolean notIncluded = !Util.isIncluded(includedNodes, dn);
836
837 if (decommissioned || decommissioning || excluded || notIncluded) {
838 if (LOG.isTraceEnabled()) {
839 LOG.trace("Excluding datanode " + dn + ": " + decommissioned + ", "
840 + decommissioning + ", " + excluded + ", " + notIncluded);
841 }
842 return true;
843 }
844 return false;
845 }
846
847 /** Get live datanode storage reports and then build the network topology. */
848 public List<DatanodeStorageReport> init() throws IOException {
849 final DatanodeStorageReport[] reports = nnc.getLiveDatanodeStorageReport();
850 final List<DatanodeStorageReport> trimmed = new ArrayList<DatanodeStorageReport>();
851 // create network topology and classify utilization collections:
852 // over-utilized, above-average, below-average and under-utilized.
853 for (DatanodeStorageReport r : DFSUtil.shuffle(reports)) {
854 final DatanodeInfo datanode = r.getDatanodeInfo();
855 if (shouldIgnore(datanode)) {
856 continue;
857 }
858 trimmed.add(r);
859 cluster.add(datanode);
860 }
861 return trimmed;
862 }
863
864 public DDatanode newDatanode(DatanodeInfo datanode) {
865 return new DDatanode(datanode, maxConcurrentMovesPerNode);
866 }
867
868 public void executePendingMove(final PendingMove p) {
869 // move the block
870 moveExecutor.execute(new Runnable() {
871 @Override
872 public void run() {
873 p.dispatch();
874 }
875 });
876 }
877
878 public boolean dispatchAndCheckContinue() throws InterruptedException {
879 return nnc.shouldContinue(dispatchBlockMoves());
880 }
881
882 /**
883 * Dispatch block moves for each source. The thread selects blocks to move &
884 * sends request to proxy source to initiate block move. The process is flow
885 * controlled. Block selection is blocked if there are too many un-confirmed
886 * block moves.
887 *
888 * @return the total number of bytes successfully moved in this iteration.
889 */
890 private long dispatchBlockMoves() throws InterruptedException {
891 final long bytesLastMoved = getBytesMoved();
892 final Future<?>[] futures = new Future<?>[sources.size()];
893
894 final Iterator<Source> i = sources.iterator();
895 for (int j = 0; j < futures.length; j++) {
896 final Source s = i.next();
897 futures[j] = dispatchExecutor.submit(new Runnable() {
898 @Override
899 public void run() {
900 s.dispatchBlocks();
901 }
902 });
903 }
904
905 // wait for all dispatcher threads to finish
906 for (Future<?> future : futures) {
907 try {
908 future.get();
909 } catch (ExecutionException e) {
910 LOG.warn("Dispatcher thread failed", e.getCause());
911 }
912 }
913
914 // wait for all block moving to be done
915 waitForMoveCompletion(targets);
916
917 return getBytesMoved() - bytesLastMoved;
918 }
919
920 /** The sleeping period before checking if block move is completed again */
921 static private long blockMoveWaitTime = 30000L;
922
923 /**
924 * Wait for all block move confirmations.
925 * @return true if there is failed move execution
926 */
927 public static boolean waitForMoveCompletion(
928 Iterable<? extends StorageGroup> targets) {
929 boolean hasFailure = false;
930 for(;;) {
931 boolean empty = true;
932 for (StorageGroup t : targets) {
933 if (!t.getDDatanode().isPendingQEmpty()) {
934 empty = false;
935 break;
936 } else {
937 hasFailure |= t.getDDatanode().hasFailure;
938 }
939 }
940 if (empty) {
941 return hasFailure; // all pending queues are empty
942 }
943 try {
944 Thread.sleep(blockMoveWaitTime);
945 } catch (InterruptedException ignored) {
946 }
947 }
948 }
949
950 /**
951 * Decide if the block is a good candidate to be moved from source to target.
952 * A block is a good candidate if
953 * 1. the block is not in the process of being moved/has not been moved;
954 * 2. the block does not have a replica on the target;
955 * 3. doing the move does not reduce the number of racks that the block has
956 */
957 private boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target,
958 StorageType targetStorageType, DBlock block) {
959 if (target.storageType != targetStorageType) {
960 return false;
961 }
962 // check if the block is moved or not
963 if (movedBlocks.contains(block.getBlock())) {
964 return false;
965 }
966 if (block.isLocatedOn(target)) {
967 return false;
968 }
969 if (cluster.isNodeGroupAware()
970 && isOnSameNodeGroupWithReplicas(source, target, block)) {
971 return false;
972 }
973 if (reduceNumOfRacks(source, target, block)) {
974 return false;
975 }
976 return true;
977 }
978
979 /**
980 * Determine whether moving the given block replica from source to target
981 * would reduce the number of racks of the block replicas.
982 */
983 private boolean reduceNumOfRacks(StorageGroup source, StorageGroup target,
984 DBlock block) {
985 final DatanodeInfo sourceDn = source.getDatanodeInfo();
986 if (cluster.isOnSameRack(sourceDn, target.getDatanodeInfo())) {
987 // source and target are on the same rack
988 return false;
989 }
990 boolean notOnSameRack = true;
991 synchronized (block) {
992 for (StorageGroup loc : block.getLocations()) {
993 if (cluster.isOnSameRack(loc.getDatanodeInfo(), target.getDatanodeInfo())) {
994 notOnSameRack = false;
995 break;
996 }
997 }
998 }
999 if (notOnSameRack) {
1000 // target is not on the same rack as any replica
1001 return false;
1002 }
1003 for (StorageGroup g : block.getLocations()) {
1004 if (g != source && cluster.isOnSameRack(g.getDatanodeInfo(), sourceDn)) {
1005 // source is on the same rack of another replica
1006 return false;
1007 }
1008 }
1009 return true;
1010 }
1011
1012 /**
1013 * Check if there are any replica (other than source) on the same node group
1014 * with target. If true, then target is not a good candidate for placing
1015 * specific replica as we don't want 2 replicas under the same nodegroup.
1016 *
1017 * @return true if there are any replica (other than source) on the same node
1018 * group with target
1019 */
1020 private boolean isOnSameNodeGroupWithReplicas(StorageGroup source,
1021 StorageGroup target, DBlock block) {
1022 final DatanodeInfo targetDn = target.getDatanodeInfo();
1023 for (StorageGroup g : block.getLocations()) {
1024 if (g != source && cluster.isOnSameNodeGroup(g.getDatanodeInfo(), targetDn)) {
1025 return true;
1026 }
1027 }
1028 return false;
1029 }
1030
1031 /** Reset all fields in order to prepare for the next iteration */
1032 void reset(Configuration conf) {
1033 cluster = NetworkTopology.getInstance(conf);
1034 storageGroupMap.clear();
1035 sources.clear();
1036 targets.clear();
1037 globalBlocks.removeAllButRetain(movedBlocks);
1038 movedBlocks.cleanup();
1039 }
1040
1041 /** set the sleeping period for block move completion check */
1042 @VisibleForTesting
1043 public static void setBlockMoveWaitTime(long time) {
1044 blockMoveWaitTime = time;
1045 }
1046
1047 @VisibleForTesting
1048 public static void setDelayAfterErrors(long time) {
1049 delayAfterErrors = time;
1050 }
1051
1052 /** shutdown thread pools */
1053 public void shutdownNow() {
1054 if (dispatchExecutor != null) {
1055 dispatchExecutor.shutdownNow();
1056 }
1057 moveExecutor.shutdownNow();
1058 }
1059
1060 static class Util {
1061 /** @return true if data node is part of the excludedNodes. */
1062 static boolean isExcluded(Set<String> excludedNodes, DatanodeInfo dn) {
1063 return isIn(excludedNodes, dn);
1064 }
1065
1066 /**
1067 * @return true if includedNodes is empty or data node is part of the
1068 * includedNodes.
1069 */
1070 static boolean isIncluded(Set<String> includedNodes, DatanodeInfo dn) {
1071 return (includedNodes.isEmpty() || isIn(includedNodes, dn));
1072 }
1073
1074 /**
1075 * Match is checked using host name , ip address with and without port
1076 * number.
1077 *
1078 * @return true if the datanode's transfer address matches the set of nodes.
1079 */
1080 private static boolean isIn(Set<String> datanodes, DatanodeInfo dn) {
1081 return isIn(datanodes, dn.getPeerHostName(), dn.getXferPort())
1082 || isIn(datanodes, dn.getIpAddr(), dn.getXferPort())
1083 || isIn(datanodes, dn.getHostName(), dn.getXferPort());
1084 }
1085
1086 /** @return true if nodes contains host or host:port */
1087 private static boolean isIn(Set<String> nodes, String host, int port) {
1088 if (host == null) {
1089 return false;
1090 }
1091 return (nodes.contains(host) || nodes.contains(host + ":" + port));
1092 }
1093
1094 /**
1095 * Parse a comma separated string to obtain set of host names
1096 *
1097 * @return set of host names
1098 */
1099 static Set<String> parseHostList(String string) {
1100 String[] addrs = StringUtils.getTrimmedStrings(string);
1101 return new HashSet<String>(Arrays.asList(addrs));
1102 }
1103
1104 /**
1105 * Read set of host names from a file
1106 *
1107 * @return set of host names
1108 */
1109 static Set<String> getHostListFromFile(String fileName, String type) {
1110 Set<String> nodes = new HashSet<String>();
1111 try {
1112 HostsFileReader.readFileToSet(type, fileName, nodes);
1113 return StringUtils.getTrimmedStrings(nodes);
1114 } catch (IOException e) {
1115 throw new IllegalArgumentException(
1116 "Failed to read host list from file: " + fileName);
1117 }
1118 }
1119 }
1120 }