From 6d12cd8d609dec26d44cece9937c35b7d72a3cd1 Mon Sep 17 00:00:00 2001 From: Haohui Mai Date: Fri, 28 Aug 2015 14:10:40 -0700 Subject: [PATCH 01/18] HDFS-8938. Extract BlockToMarkCorrupt and ReplicationWork as standalone classes from BlockManager. Contributed by Mingliang Liu. --- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 + .../server/blockmanagement/BlockManager.java | 368 +++++++----------- .../blockmanagement/BlockToMarkCorrupt.java | 87 +++++ .../blockmanagement/ReplicationWork.java | 87 +++++ 4 files changed, 319 insertions(+), 226 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 67a6a6e6ec..c6acfc83ed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -855,6 +855,9 @@ Release 2.8.0 - UNRELEASED HDFS-8865. Improve quota initialization performance. (kihwal) + HDFS-8938. Extract BlockToMarkCorrupt and ReplicationWork as standalone + classes from BlockManager. (Mingliang Liu via wheat9) + OPTIMIZATIONS HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index 95933d203a..8f7bb55b09 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -1181,24 +1181,24 @@ public class BlockManager implements BlockStatsMXBean { DatanodeStorageInfo storageInfo, DatanodeDescriptor node) throws IOException { - if (b.corrupted.isDeleted()) { + if (b.getCorrupted().isDeleted()) { blockLog.debug("BLOCK markBlockAsCorrupt: {} cannot be marked as" + " corrupt as it does not belong to any file", b); - addToInvalidates(b.corrupted, node); + addToInvalidates(b.getCorrupted(), node); return; } - short expectedReplicas = b.corrupted.getReplication(); + short expectedReplicas = b.getCorrupted().getReplication(); // Add replica to the data-node if it is not already there if (storageInfo != null) { - storageInfo.addBlock(b.stored); + storageInfo.addBlock(b.getStored()); } // Add this replica to corruptReplicas Map - corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason, - b.reasonCode); + corruptReplicas.addToCorruptReplicasMap(b.getCorrupted(), node, + b.getReason(), b.getReasonCode()); - NumberReplicas numberOfReplicas = countNodes(b.stored); + NumberReplicas numberOfReplicas = countNodes(b.getStored()); boolean hasEnoughLiveReplicas = numberOfReplicas.liveReplicas() >= expectedReplicas; boolean minReplicationSatisfied = @@ -1207,7 +1207,7 @@ public class BlockManager implements BlockStatsMXBean { (numberOfReplicas.liveReplicas() + numberOfReplicas.corruptReplicas()) > expectedReplicas; boolean corruptedDuringWrite = minReplicationSatisfied && - (b.stored.getGenerationStamp() > b.corrupted.getGenerationStamp()); + b.isCorruptedDuringWrite(); // case 1: have enough number of live replicas // case 2: corrupted replicas + live replicas > Replication factor // case 3: Block is marked corrupt due to failure while writing. In this @@ -1220,7 +1220,7 @@ public class BlockManager implements BlockStatsMXBean { invalidateBlock(b, node); } else if (namesystem.isPopulatingReplQueues()) { // add the block to neededReplication - updateNeededReplications(b.stored, -1, 0); + updateNeededReplications(b.getStored(), -1, 0); } } @@ -1239,18 +1239,18 @@ public class BlockManager implements BlockStatsMXBean { } // Check how many copies we have of the block - NumberReplicas nr = countNodes(b.stored); + NumberReplicas nr = countNodes(b.getStored()); if (nr.replicasOnStaleNodes() > 0) { blockLog.debug("BLOCK* invalidateBlocks: postponing " + "invalidation of {} on {} because {} replica(s) are located on " + "nodes with potentially out-of-date block reports", b, dn, nr.replicasOnStaleNodes()); - postponeBlock(b.corrupted); + postponeBlock(b.getCorrupted()); return false; } else if (nr.liveReplicas() >= 1) { // If we have at least one copy on a live node, then we can delete it. - addToInvalidates(b.corrupted, dn); - removeStoredBlock(b.stored, node); + addToInvalidates(b.getCorrupted(), dn); + removeStoredBlock(b.getStored(), node); blockLog.debug("BLOCK* invalidateBlocks: {} on {} listed for deletion.", b, dn); return true; @@ -1338,71 +1338,18 @@ public class BlockManager implements BlockStatsMXBean { */ @VisibleForTesting int computeReplicationWorkForBlocks(List> blocksToReplicate) { - int requiredReplication, numEffectiveReplicas; - List containingNodes; - DatanodeDescriptor srcNode; - BlockCollection bc = null; - int additionalReplRequired; - int scheduledWork = 0; - List work = new LinkedList(); + final List work = new LinkedList<>(); namesystem.writeLock(); try { synchronized (neededReplications) { for (int priority = 0; priority < blocksToReplicate.size(); priority++) { for (BlockInfo block : blocksToReplicate.get(priority)) { - // block should belong to a file - bc = getBlockCollection(block); - // abandoned block or block reopened for append - if (bc == null - || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) { - // remove from neededReplications - neededReplications.remove(block, priority); - continue; + ReplicationWork rw = scheduleReplication(block, priority); + if (rw != null) { + work.add(rw); } - - requiredReplication = getExpectedReplicaNum(block); - - // get a source data-node - containingNodes = new ArrayList(); - List liveReplicaNodes = new ArrayList(); - NumberReplicas numReplicas = new NumberReplicas(); - srcNode = chooseSourceDatanode( - block, containingNodes, liveReplicaNodes, numReplicas, - priority); - if(srcNode == null) { // block can not be replicated from any node - LOG.debug("Block " + block + " cannot be repl from any node"); - continue; - } - - // liveReplicaNodes can include READ_ONLY_SHARED replicas which are - // not included in the numReplicas.liveReplicas() count - assert liveReplicaNodes.size() >= numReplicas.liveReplicas(); - - // do not schedule more if enough replicas is already pending - numEffectiveReplicas = numReplicas.liveReplicas() + - pendingReplications.getNumReplicas(block); - - if (numEffectiveReplicas >= requiredReplication) { - if ( (pendingReplications.getNumReplicas(block) > 0) || - (blockHasEnoughRacks(block)) ) { - neededReplications.remove(block, priority); // remove from neededReplications - blockLog.debug("BLOCK* Removing {} from neededReplications as" + - " it has enough replicas", block); - continue; - } - } - - if (numReplicas.liveReplicas() < requiredReplication) { - additionalReplRequired = requiredReplication - - numEffectiveReplicas; - } else { - additionalReplRequired = 1; // Needed on a new rack - } - work.add(new ReplicationWork(block, bc, srcNode, - containingNodes, liveReplicaNodes, additionalReplRequired, - priority)); } } } @@ -1410,12 +1357,12 @@ public class BlockManager implements BlockStatsMXBean { namesystem.writeUnlock(); } - final Set excludedNodes = new HashSet(); + final Set excludedNodes = new HashSet<>(); for(ReplicationWork rw : work){ // Exclude all of the containing nodes from being targets. // This list includes decommissioning or corrupt nodes. excludedNodes.clear(); - for (DatanodeDescriptor dn : rw.containingNodes) { + for (DatanodeDescriptor dn : rw.getContainingNodes()) { excludedNodes.add(dn); } @@ -1428,67 +1375,15 @@ public class BlockManager implements BlockStatsMXBean { namesystem.writeLock(); try { for(ReplicationWork rw : work){ - final DatanodeStorageInfo[] targets = rw.targets; + final DatanodeStorageInfo[] targets = rw.getTargets(); if(targets == null || targets.length == 0){ - rw.targets = null; + rw.resetTargets(); continue; } synchronized (neededReplications) { - BlockInfo block = rw.block; - int priority = rw.priority; - // Recheck since global lock was released - // block should belong to a file - bc = getBlockCollection(block); - // abandoned block or block reopened for append - if(bc == null || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) { - neededReplications.remove(block, priority); // remove from neededReplications - rw.targets = null; - continue; - } - requiredReplication = getExpectedReplicaNum(block); - - // do not schedule more if enough replicas is already pending - NumberReplicas numReplicas = countNodes(block); - numEffectiveReplicas = numReplicas.liveReplicas() + - pendingReplications.getNumReplicas(block); - - if (numEffectiveReplicas >= requiredReplication) { - if ( (pendingReplications.getNumReplicas(block) > 0) || - (blockHasEnoughRacks(block)) ) { - neededReplications.remove(block, priority); // remove from neededReplications - rw.targets = null; - blockLog.debug("BLOCK* Removing {} from neededReplications as" + - " it has enough replicas", block); - continue; - } - } - - if ( (numReplicas.liveReplicas() >= requiredReplication) && - (!blockHasEnoughRacks(block)) ) { - if (rw.srcNode.getNetworkLocation().equals( - targets[0].getDatanodeDescriptor().getNetworkLocation())) { - //No use continuing, unless a new rack in this case - continue; - } - } - - // Add block to the to be replicated list - rw.srcNode.addBlockToBeReplicated(block, targets); - scheduledWork++; - DatanodeStorageInfo.incrementBlocksScheduled(targets); - - // Move the block-replication into a "pending" state. - // The reason we use 'pending' is so we can retry - // replications that fail after an appropriate amount of time. - pendingReplications.increment(block, - DatanodeStorageInfo.toDatanodeDescriptors(targets)); - blockLog.debug("BLOCK* block {} is moved from neededReplications to " - + "pendingReplications", block); - - // remove from neededReplications - if(numEffectiveReplicas + targets.length >= requiredReplication) { - neededReplications.remove(block, priority); // remove from neededReplications + if (validateReplicationWork(rw)) { + scheduledWork++; } } } @@ -1499,15 +1394,15 @@ public class BlockManager implements BlockStatsMXBean { if (blockLog.isInfoEnabled()) { // log which blocks have been scheduled for replication for(ReplicationWork rw : work){ - DatanodeStorageInfo[] targets = rw.targets; + DatanodeStorageInfo[] targets = rw.getTargets(); if (targets != null && targets.length != 0) { StringBuilder targetList = new StringBuilder("datanode(s)"); - for (int k = 0; k < targets.length; k++) { + for (DatanodeStorageInfo target : targets) { targetList.append(' '); - targetList.append(targets[k].getDatanodeDescriptor()); + targetList.append(target.getDatanodeDescriptor()); } - blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.srcNode, - rw.block, targetList); + blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.getSrcNode(), + rw.getBlock(), targetList); } } } @@ -1519,6 +1414,118 @@ public class BlockManager implements BlockStatsMXBean { return scheduledWork; } + boolean hasEnoughEffectiveReplicas(BlockInfo block, + NumberReplicas numReplicas, int pendingReplicaNum, int required) { + int numEffectiveReplicas = numReplicas.liveReplicas() + pendingReplicaNum; + return (numEffectiveReplicas >= required) && + (pendingReplicaNum > 0 || blockHasEnoughRacks(block)); + } + + private ReplicationWork scheduleReplication(BlockInfo block, int priority) { + // block should belong to a file + BlockCollection bc = getBlockCollection(block); + // abandoned block or block reopened for append + if (bc == null + || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) { + // remove from neededReplications + neededReplications.remove(block, priority); + return null; + } + + short requiredReplication = getExpectedReplicaNum(block); + + // get a source data-node + List containingNodes = new ArrayList<>(); + List liveReplicaNodes = new ArrayList<>(); + NumberReplicas numReplicas = new NumberReplicas(); + DatanodeDescriptor srcNode = chooseSourceDatanode(block, containingNodes, + liveReplicaNodes, numReplicas, priority); + if (srcNode == null) { // block can not be replicated from any node + LOG.debug("Block " + block + " cannot be repl from any node"); + return null; + } + + // liveReplicaNodes can include READ_ONLY_SHARED replicas which are + // not included in the numReplicas.liveReplicas() count + assert liveReplicaNodes.size() >= numReplicas.liveReplicas(); + + int pendingNum = pendingReplications.getNumReplicas(block); + if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum, + requiredReplication)) { + neededReplications.remove(block, priority); + blockLog.debug("BLOCK* Removing {} from neededReplications as" + + " it has enough replicas", block); + return null; + } + + final int additionalReplRequired; + if (numReplicas.liveReplicas() < requiredReplication) { + additionalReplRequired = requiredReplication - numReplicas.liveReplicas() + - pendingNum; + } else { + additionalReplRequired = 1; // Needed on a new rack + } + return new ReplicationWork(block, bc, srcNode, containingNodes, + liveReplicaNodes, additionalReplRequired, priority); + } + + private boolean validateReplicationWork(ReplicationWork rw) { + BlockInfo block = rw.getBlock(); + int priority = rw.getPriority(); + // Recheck since global lock was released + // block should belong to a file + BlockCollection bc = getBlockCollection(block); + // abandoned block or block reopened for append + if (bc == null + || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) { + neededReplications.remove(block, priority); + rw.resetTargets(); + return false; + } + + // do not schedule more if enough replicas is already pending + final short requiredReplication = getExpectedReplicaNum(block); + NumberReplicas numReplicas = countNodes(block); + final int pendingNum = pendingReplications.getNumReplicas(block); + if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum, + requiredReplication)) { + neededReplications.remove(block, priority); + rw.resetTargets(); + blockLog.debug("BLOCK* Removing {} from neededReplications as" + + " it has enough replicas", block); + return false; + } + + DatanodeStorageInfo[] targets = rw.getTargets(); + if ( (numReplicas.liveReplicas() >= requiredReplication) && + (!blockHasEnoughRacks(block)) ) { + if (rw.getSrcNode().getNetworkLocation().equals( + targets[0].getDatanodeDescriptor().getNetworkLocation())) { + //No use continuing, unless a new rack in this case + return false; + } + } + + // Add block to the to be replicated list + rw.getSrcNode().addBlockToBeReplicated(block, targets); + DatanodeStorageInfo.incrementBlocksScheduled(targets); + + // Move the block-replication into a "pending" state. + // The reason we use 'pending' is so we can retry + // replications that fail after an appropriate amount of time. + pendingReplications.increment(block, + DatanodeStorageInfo.toDatanodeDescriptors(targets)); + blockLog.debug("BLOCK* block {} is moved from neededReplications to " + + "pendingReplications", block); + + int numEffectiveReplicas = numReplicas.liveReplicas() + pendingNum; + // remove from neededReplications + if(numEffectiveReplicas + targets.length >= requiredReplication) { + neededReplications.remove(block, priority); + } + return true; + } + /** Choose target for WebHDFS redirection. */ public DatanodeStorageInfo[] chooseTarget4WebHDFS(String src, DatanodeDescriptor clientnode, Set excludes, long blocksize) { @@ -1765,52 +1772,6 @@ public class BlockManager implements BlockStatsMXBean { this.reportedState = reportedState; } } - - /** - * BlockToMarkCorrupt is used to build the "toCorrupt" list, which is a - * list of blocks that should be considered corrupt due to a block report. - */ - private static class BlockToMarkCorrupt { - /** The corrupted block in a datanode. */ - final BlockInfo corrupted; - /** The corresponding block stored in the BlockManager. */ - final BlockInfo stored; - /** The reason to mark corrupt. */ - final String reason; - /** The reason code to be stored */ - final Reason reasonCode; - - BlockToMarkCorrupt(BlockInfo corrupted, - BlockInfo stored, String reason, - Reason reasonCode) { - Preconditions.checkNotNull(corrupted, "corrupted is null"); - Preconditions.checkNotNull(stored, "stored is null"); - - this.corrupted = corrupted; - this.stored = stored; - this.reason = reason; - this.reasonCode = reasonCode; - } - - BlockToMarkCorrupt(BlockInfo stored, String reason, - Reason reasonCode) { - this(stored, stored, reason, reasonCode); - } - - BlockToMarkCorrupt(BlockInfo stored, long gs, String reason, - Reason reasonCode) { - this(new BlockInfoContiguous((BlockInfoContiguous)stored), stored, - reason, reasonCode); - //the corrupted block in datanode has a different generation stamp - corrupted.setGenerationStamp(gs); - } - - @Override - public String toString() { - return corrupted + "(" - + (corrupted == stored? "same as stored": "stored=" + stored) + ")"; - } - } /** * The given storage is reporting all its blocks. @@ -3797,51 +3758,6 @@ public class BlockManager implements BlockStatsMXBean { null); } - private static class ReplicationWork { - - private final BlockInfo block; - private final BlockCollection bc; - - private final DatanodeDescriptor srcNode; - private final List containingNodes; - private final List liveReplicaStorages; - private final int additionalReplRequired; - - private DatanodeStorageInfo targets[]; - private final int priority; - - public ReplicationWork(BlockInfo block, - BlockCollection bc, - DatanodeDescriptor srcNode, - List containingNodes, - List liveReplicaStorages, - int additionalReplRequired, - int priority) { - this.block = block; - this.bc = bc; - this.srcNode = srcNode; - this.srcNode.incrementPendingReplicationWithoutTargets(); - this.containingNodes = containingNodes; - this.liveReplicaStorages = liveReplicaStorages; - this.additionalReplRequired = additionalReplRequired; - this.priority = priority; - this.targets = null; - } - - private void chooseTargets(BlockPlacementPolicy blockplacement, - BlockStoragePolicySuite storagePolicySuite, - Set excludedNodes) { - try { - targets = blockplacement.chooseTarget(bc.getName(), - additionalReplRequired, srcNode, liveReplicaStorages, false, - excludedNodes, block.getNumBytes(), - storagePolicySuite.getPolicy(bc.getStoragePolicyID())); - } finally { - srcNode.decrementPendingReplicationWithoutTargets(); - } - } - } - /** * A simple result enum for the result of * {@link BlockManager#processMisReplicatedBlock(BlockInfo)}. @@ -3855,9 +3771,9 @@ public class BlockManager implements BlockStatsMXBean { OVER_REPLICATED, /** A decision can't currently be made about this block. */ POSTPONE, - /** The block is under construction, so should be ignored */ + /** The block is under construction, so should be ignored. */ UNDER_CONSTRUCTION, - /** The block is properly replicated */ + /** The block is properly replicated. */ OK } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java new file mode 100644 index 0000000000..3842e562ac --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java @@ -0,0 +1,87 @@ +/** + * 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.blockmanagement; + +import static org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason; + +import com.google.common.base.Preconditions; + +/** + * BlockToMarkCorrupt is used to build the "toCorrupt" list, which is a + * list of blocks that should be considered corrupt due to a block report. + */ +class BlockToMarkCorrupt { + /** The corrupted block in a datanode. */ + private final BlockInfo corrupted; + /** The corresponding block stored in the BlockManager. */ + private final BlockInfo stored; + /** The reason to mark corrupt. */ + private final String reason; + /** The reason code to be stored */ + private final CorruptReplicasMap.Reason reasonCode; + + BlockToMarkCorrupt(BlockInfo corrupted, BlockInfo stored, String reason, + CorruptReplicasMap.Reason reasonCode) { + Preconditions.checkNotNull(corrupted, "corrupted is null"); + Preconditions.checkNotNull(stored, "stored is null"); + + this.corrupted = corrupted; + this.stored = stored; + this.reason = reason; + this.reasonCode = reasonCode; + } + + BlockToMarkCorrupt(BlockInfo stored, String reason, + CorruptReplicasMap.Reason reasonCode) { + this(stored, stored, reason, reasonCode); + } + + BlockToMarkCorrupt(BlockInfo stored, long gs, String reason, + CorruptReplicasMap.Reason reasonCode) { + this(new BlockInfoContiguous((BlockInfoContiguous)stored), stored, + reason, reasonCode); + //the corrupted block in datanode has a different generation stamp + corrupted.setGenerationStamp(gs); + } + + public boolean isCorruptedDuringWrite() { + return stored.getGenerationStamp() > corrupted.getGenerationStamp(); + } + + public BlockInfo getCorrupted() { + return corrupted; + } + + public BlockInfo getStored() { + return stored; + } + + public String getReason() { + return reason; + } + + public Reason getReasonCode() { + return reasonCode; + } + + @Override + public String toString() { + return corrupted + "(" + + (corrupted == stored ? "same as stored": "stored=" + stored) + ")"; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java new file mode 100644 index 0000000000..f8a6dad1d1 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java @@ -0,0 +1,87 @@ +/** + * 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.blockmanagement; + +import org.apache.hadoop.net.Node; + +import java.util.Collections; +import java.util.List; +import java.util.Set; + +class ReplicationWork { + private final BlockInfo block; + private final BlockCollection bc; + private final DatanodeDescriptor srcNode; + private final int additionalReplRequired; + private final int priority; + private final List containingNodes; + private final List liveReplicaStorages; + private DatanodeStorageInfo[] targets; + + public ReplicationWork(BlockInfo block, BlockCollection bc, + DatanodeDescriptor srcNode, List containingNodes, + List liveReplicaStorages, int additionalReplRequired, + int priority) { + this.block = block; + this.bc = bc; + this.srcNode = srcNode; + this.srcNode.incrementPendingReplicationWithoutTargets(); + this.containingNodes = containingNodes; + this.liveReplicaStorages = liveReplicaStorages; + this.additionalReplRequired = additionalReplRequired; + this.priority = priority; + this.targets = null; + } + + void chooseTargets(BlockPlacementPolicy blockplacement, + BlockStoragePolicySuite storagePolicySuite, + Set excludedNodes) { + try { + targets = blockplacement.chooseTarget(bc.getName(), + additionalReplRequired, srcNode, liveReplicaStorages, false, + excludedNodes, block.getNumBytes(), + storagePolicySuite.getPolicy(bc.getStoragePolicyID())); + } finally { + srcNode.decrementPendingReplicationWithoutTargets(); + } + } + + DatanodeStorageInfo[] getTargets() { + return targets; + } + + void resetTargets() { + this.targets = null; + } + + List getContainingNodes() { + return Collections.unmodifiableList(containingNodes); + } + + public int getPriority() { + return priority; + } + + public BlockInfo getBlock() { + return block; + } + + public DatanodeDescriptor getSrcNode() { + return srcNode; + } +} From b94b56806d3d6e04984e229b479f7ac15b62bbfa Mon Sep 17 00:00:00 2001 From: Colin Patrick Mccabe Date: Fri, 28 Aug 2015 14:13:23 -0700 Subject: [PATCH 02/18] HDFS-8950. NameNode refresh doesn't remove DataNodes that are no longer in the allowed list (Daniel Templeton) --- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 + .../blockmanagement/DatanodeManager.java | 9 +- .../blockmanagement/HostFileManager.java | 19 ++++ .../apache/hadoop/hdfs/TestDecommission.java | 15 +-- .../blockmanagement/TestDatanodeManager.java | 103 +++++++++++++++++- .../blockmanagement/TestHostFileManager.java | 7 +- 6 files changed, 139 insertions(+), 17 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index c6acfc83ed..9f77e8510d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -1251,6 +1251,9 @@ Release 2.8.0 - UNRELEASED HDFS-8963. Fix incorrect sign extension of xattr length in HDFS-8900. (Colin Patrick McCabe via yliu) + HDFS-8950. NameNode refresh doesn't remove DataNodes that are no longer in + the allowed list (Daniel Templeton) + Release 2.7.2 - UNRELEASED INCOMPATIBLE CHANGES diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index d1900f4b92..95ec6485d1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -1279,11 +1279,14 @@ public class DatanodeManager { for (DatanodeDescriptor dn : datanodeMap.values()) { final boolean isDead = isDatanodeDead(dn); final boolean isDecommissioning = dn.isDecommissionInProgress(); - if ((listLiveNodes && !isDead) || + + if (((listLiveNodes && !isDead) || (listDeadNodes && isDead) || - (listDecommissioningNodes && isDecommissioning)) { - nodes.add(dn); + (listDecommissioningNodes && isDecommissioning)) && + hostFileManager.isIncluded(dn)) { + nodes.add(dn); } + foundNodes.add(HostFileManager.resolvedAddressFromDatanodeID(dn)); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java index 0b8d6c5bc1..e05ef9a404 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java @@ -126,9 +126,28 @@ class HostFileManager { return !includes.isEmpty(); } + /** + * Read the includes and excludes lists from the named files. Any previous + * includes and excludes lists are discarded. + * @param includeFile the path to the new includes list + * @param excludeFile the path to the new excludes list + * @throws IOException thrown if there is a problem reading one of the files + */ void refresh(String includeFile, String excludeFile) throws IOException { HostSet newIncludes = readFile("included", includeFile); HostSet newExcludes = readFile("excluded", excludeFile); + + refresh(newIncludes, newExcludes); + } + + /** + * Set the includes and excludes lists by the new HostSet instances. The + * old instances are discarded. + * @param newIncludes the new includes list + * @param newExcludes the new excludes list + */ + @VisibleForTesting + void refresh(HostSet newIncludes, HostSet newExcludes) { synchronized (this) { includes = newIncludes; excludes = newExcludes; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java index 413a3cfbb4..7c30361ee2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java @@ -813,16 +813,13 @@ public class TestDecommission { } assertEquals("Number of live nodes should be 0", 0, info.length); - // Test that non-live and bogus hostnames are considered "dead". - // The dead report should have an entry for (1) the DN that is - // now considered dead because it is no longer allowed to connect - // and (2) the bogus entry in the hosts file (these entries are - // always added last) + // Test that bogus hostnames are considered "dead". + // The dead report should have an entry for the bogus entry in the hosts + // file. The original datanode is excluded from the report because it + // is no longer in the included list. info = client.datanodeReport(DatanodeReportType.DEAD); - assertEquals("There should be 2 dead nodes", 2, info.length); - DatanodeID id = cluster.getDataNodes().get(0).getDatanodeId(); - assertEquals(id.getHostName(), info[0].getHostName()); - assertEquals(bogusIp, info[1].getHostName()); + assertEquals("There should be 1 dead node", 1, info.length); + assertEquals(bogusIp, info[0].getHostName()); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java index 39bd5d13aa..b55a716e69 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java @@ -19,11 +19,13 @@ package org.apache.hadoop.hdfs.server.blockmanagement; import java.io.IOException; +import java.net.InetSocketAddress; import java.net.URISyntaxException; import java.net.URL; import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -38,17 +40,23 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.DFSConfigKeys; +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.LocatedBlock; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.net.DNSToSwitchMapping; import org.apache.hadoop.util.Shell; import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; +import org.mockito.internal.util.reflection.Whitebox; import static org.hamcrest.core.Is.is; import static org.junit.Assert.*; @@ -68,6 +76,15 @@ public class TestDatanodeManager { return dm; } + /** + * Create an InetSocketAddress for a host:port string + * @param host a host identifier in host:port format + * @return a corresponding InetSocketAddress object + */ + private static InetSocketAddress entry(String host) { + return HostFileManager.parseEntry("dummy", "dummy", host); + } + /** * This test sends a random sequence of node registrations and node removals * to the DatanodeManager (of nodes with different IDs and versions), and @@ -352,5 +369,89 @@ public class TestDatanodeManager { assertThat(sortedLocs[sortedLocs.length - 2].getAdminState(), is(DatanodeInfo.AdminStates.DECOMMISSIONED)); } -} + /** + * Test whether removing a host from the includes list without adding it to + * the excludes list will exclude it from data node reports. + */ + @Test + public void testRemoveIncludedNode() throws IOException { + FSNamesystem fsn = Mockito.mock(FSNamesystem.class); + + // Set the write lock so that the DatanodeManager can start + Mockito.when(fsn.hasWriteLock()).thenReturn(true); + + DatanodeManager dm = mockDatanodeManager(fsn, new Configuration()); + HostFileManager hm = new HostFileManager(); + HostFileManager.HostSet noNodes = new HostFileManager.HostSet(); + HostFileManager.HostSet oneNode = new HostFileManager.HostSet(); + HostFileManager.HostSet twoNodes = new HostFileManager.HostSet(); + DatanodeRegistration dr1 = new DatanodeRegistration( + new DatanodeID("127.0.0.1", "127.0.0.1", "someStorageID-123", + 12345, 12345, 12345, 12345), + new StorageInfo(HdfsServerConstants.NodeType.DATA_NODE), + new ExportedBlockKeys(), "test"); + DatanodeRegistration dr2 = new DatanodeRegistration( + new DatanodeID("127.0.0.1", "127.0.0.1", "someStorageID-234", + 23456, 23456, 23456, 23456), + new StorageInfo(HdfsServerConstants.NodeType.DATA_NODE), + new ExportedBlockKeys(), "test"); + + twoNodes.add(entry("127.0.0.1:12345")); + twoNodes.add(entry("127.0.0.1:23456")); + oneNode.add(entry("127.0.0.1:23456")); + + hm.refresh(twoNodes, noNodes); + Whitebox.setInternalState(dm, "hostFileManager", hm); + + // Register two data nodes to simulate them coming up. + // We need to add two nodes, because if we have only one node, removing it + // will cause the includes list to be empty, which means all hosts will be + // allowed. + dm.registerDatanode(dr1); + dm.registerDatanode(dr2); + + // Make sure that both nodes are reported + List both = + dm.getDatanodeListForReport(HdfsConstants.DatanodeReportType.ALL); + + // Sort the list so that we know which one is which + Collections.sort(both); + + Assert.assertEquals("Incorrect number of hosts reported", + 2, both.size()); + Assert.assertEquals("Unexpected host or host in unexpected position", + "127.0.0.1:12345", both.get(0).getInfoAddr()); + Assert.assertEquals("Unexpected host or host in unexpected position", + "127.0.0.1:23456", both.get(1).getInfoAddr()); + + // Remove one node from includes, but do not add it to excludes. + hm.refresh(oneNode, noNodes); + + // Make sure that only one node is still reported + List onlyOne = + dm.getDatanodeListForReport(HdfsConstants.DatanodeReportType.ALL); + + Assert.assertEquals("Incorrect number of hosts reported", + 1, onlyOne.size()); + Assert.assertEquals("Unexpected host reported", + "127.0.0.1:23456", onlyOne.get(0).getInfoAddr()); + + // Remove all nodes from includes + hm.refresh(noNodes, noNodes); + + // Check that both nodes are reported again + List bothAgain = + dm.getDatanodeListForReport(HdfsConstants.DatanodeReportType.ALL); + + // Sort the list so that we know which one is which + Collections.sort(bothAgain); + + Assert.assertEquals("Incorrect number of hosts reported", + 2, bothAgain.size()); + Assert.assertEquals("Unexpected host or host in unexpected position", + "127.0.0.1:12345", bothAgain.get(0).getInfoAddr()); + Assert.assertEquals("Unexpected host or host in unexpected position", + "127.0.0.1:23456", bothAgain.get(1).getInfoAddr()); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java index 733446ceb1..c65b580439 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java @@ -104,23 +104,22 @@ public class TestHostFileManager { BlockManager bm = mock(BlockManager.class); FSNamesystem fsn = mock(FSNamesystem.class); Configuration conf = new Configuration(); - HostFileManager hm = mock(HostFileManager.class); + HostFileManager hm = new HostFileManager(); HostFileManager.HostSet includedNodes = new HostFileManager.HostSet(); HostFileManager.HostSet excludedNodes = new HostFileManager.HostSet(); includedNodes.add(entry("127.0.0.1:12345")); includedNodes.add(entry("localhost:12345")); includedNodes.add(entry("127.0.0.1:12345")); - includedNodes.add(entry("127.0.0.2")); + excludedNodes.add(entry("127.0.0.1:12346")); excludedNodes.add(entry("127.0.30.1:12346")); Assert.assertEquals(2, includedNodes.size()); Assert.assertEquals(2, excludedNodes.size()); - doReturn(includedNodes).when(hm).getIncludes(); - doReturn(excludedNodes).when(hm).getExcludes(); + hm.refresh(includedNodes, excludedNodes); DatanodeManager dm = new DatanodeManager(bm, fsn, conf); Whitebox.setInternalState(dm, "hostFileManager", hm); From e2c9b288b223b9fd82dc12018936e13128413492 Mon Sep 17 00:00:00 2001 From: Haohui Mai Date: Fri, 28 Aug 2015 14:20:55 -0700 Subject: [PATCH 03/18] HDFS-8925. Move BlockReaderLocal to hdfs-client. Contributed by Mingliang Liu. --- .../org/apache/hadoop/hdfs/BlockReader.java | 0 .../apache/hadoop/hdfs/BlockReaderLocal.java | 10 +-- .../hadoop/hdfs/BlockReaderLocalLegacy.java | 15 ++-- .../apache/hadoop/hdfs/BlockReaderUtil.java | 0 .../org/apache/hadoop/hdfs/ClientContext.java | 7 +- .../org/apache/hadoop/hdfs/DFSUtilClient.java | 68 +++++++++++++++++++ .../hadoop/hdfs/ExternalBlockReader.java | 0 .../apache/hadoop/hdfs/KeyProviderCache.java | 15 ++-- .../org/apache/hadoop/hdfs/PeerCache.java | 15 ++-- .../hdfs/client/BlockReportOptions.java | 0 .../hdfs/client/HdfsClientConfigKeys.java | 5 ++ .../hdfs/protocol/BlockLocalPathInfo.java | 0 .../hdfs/protocol/ClientDatanodeProtocol.java | 4 +- .../InvalidEncryptionKeyException.java | 0 .../protocolPB/ClientDatanodeProtocolPB.java | 4 +- .../ClientDatanodeProtocolTranslatorPB.java | 12 ++-- .../hdfs/protocolPB/PBHelperClient.java | 13 ++++ .../token/block/BlockTokenSelector.java | 0 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 + .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 14 ++-- .../apache/hadoop/hdfs/DFSInputStream.java | 2 +- .../java/org/apache/hadoop/hdfs/DFSUtil.java | 67 +----------------- .../hdfs/protocol/datatransfer/Receiver.java | 15 ++-- ...atanodeProtocolServerSideTranslatorPB.java | 6 +- ...amenodeProtocolServerSideTranslatorPB.java | 14 ++-- ...atanodeProtocolServerSideTranslatorPB.java | 2 +- ...atanodeProtocolServerSideTranslatorPB.java | 2 +- .../hadoop/hdfs/protocolPB/PBHelper.java | 26 +------ .../hadoop/hdfs/server/datanode/DNConf.java | 4 +- .../apache/hadoop/hdfs/tools/DFSAdmin.java | 3 +- .../hadoop/hdfs/TestBlockReaderLocal.java | 30 ++++---- .../hdfs/TestBlockReaderLocalLegacy.java | 2 +- .../hadoop/hdfs/TestDFSClientRetries.java | 2 +- .../hadoop/hdfs/protocolPB/TestPBHelper.java | 4 +- .../security/token/block/TestBlockToken.java | 10 +-- .../TestShortCircuitLocalRead.java | 4 +- 36 files changed, 197 insertions(+), 181 deletions(-) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/BlockReader.java (100%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java (98%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java (98%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/BlockReaderUtil.java (100%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/ClientContext.java (97%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java (100%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java (89%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/PeerCache.java (95%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/client/BlockReportOptions.java (100%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java (100%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java (97%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/InvalidEncryptionKeyException.java (100%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java (91%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java (97%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSelector.java (100%) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java similarity index 100% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java similarity index 98% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java index d913f3a283..2a0e21be1d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java @@ -22,11 +22,10 @@ import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.util.EnumSet; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader; @@ -42,6 +41,9 @@ import org.apache.htrace.TraceScope; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * BlockReaderLocal enables local short circuited reads. If the DFS client is on * the same machine as the datanode, then the client can read files directly @@ -60,7 +62,7 @@ import com.google.common.base.Preconditions; */ @InterfaceAudience.Private class BlockReaderLocal implements BlockReader { - static final Log LOG = LogFactory.getLog(BlockReaderLocal.class); + static final Logger LOG = LoggerFactory.getLogger(BlockReaderLocal.class); private static final DirectBufferPool bufferPool = new DirectBufferPool(); @@ -88,7 +90,7 @@ class BlockReaderLocal implements BlockReader { public Builder setCachingStrategy(CachingStrategy cachingStrategy) { long readahead = cachingStrategy.getReadahead() != null ? cachingStrategy.getReadahead() : - DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT; + HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT; this.maxReadahead = (int)Math.min(Integer.MAX_VALUE, readahead); return this; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java similarity index 98% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java index c16ffdf2e0..eea3f067cf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java @@ -29,8 +29,6 @@ import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; -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.ReadOption; @@ -45,6 +43,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader; import org.apache.hadoop.hdfs.shortcircuit.ClientMmap; +import org.apache.hadoop.hdfs.util.IOUtilsClient; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.security.UserGroupInformation; @@ -55,6 +54,9 @@ import org.apache.htrace.Sampler; import org.apache.htrace.Trace; import org.apache.htrace.TraceScope; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * BlockReaderLocalLegacy enables local short circuited reads. If the DFS client is on * the same machine as the datanode, then the client can read files directly @@ -79,7 +81,8 @@ import org.apache.htrace.TraceScope; */ @InterfaceAudience.Private class BlockReaderLocalLegacy implements BlockReader { - private static final Log LOG = LogFactory.getLog(BlockReaderLocalLegacy.class); + private static final Logger LOG = LoggerFactory.getLogger( + BlockReaderLocalLegacy.class); //Stores the cache and proxy for a local datanode. private static class LocalDatanodeInfo { @@ -112,7 +115,7 @@ class BlockReaderLocalLegacy implements BlockReader { proxy = ugi.doAs(new PrivilegedExceptionAction() { @Override public ClientDatanodeProtocol run() throws Exception { - return DFSUtil.createClientDatanodeProtocolProxy(node, conf, + return DFSUtilClient.createClientDatanodeProtocolProxy(node, conf, socketTimeout, connectToDnViaHostname); } }); @@ -244,7 +247,7 @@ class BlockReaderLocalLegacy implements BlockReader { } catch (IOException e) { // remove from cache localDatanodeInfo.removeBlockLocalPathInfo(blk); - DFSClient.LOG.warn("BlockReaderLocalLegacy: Removing " + blk + LOG.warn("BlockReaderLocalLegacy: Removing " + blk + " from cache because local file " + pathinfo.getBlockPath() + " could not be opened."); throw e; @@ -689,7 +692,7 @@ class BlockReaderLocalLegacy implements BlockReader { @Override public synchronized void close() throws IOException { - IOUtils.cleanup(LOG, dataIn, checksumIn); + IOUtilsClient.cleanup(LOG, dataIn, checksumIn); if (slowReadBuff != null) { bufferPool.returnBuffer(slowReadBuff); slowReadBuff = null; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderUtil.java similarity index 100% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderUtil.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderUtil.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ClientContext.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java similarity index 97% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ClientContext.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java index bf114631cf..3836979826 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ClientContext.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java @@ -19,8 +19,6 @@ package org.apache.hadoop.hdfs; import java.util.HashMap; -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.hdfs.client.HdfsClientConfigKeys; @@ -32,6 +30,9 @@ import org.apache.hadoop.hdfs.util.ByteArrayManager; import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * ClientContext contains context information for a client. * @@ -40,7 +41,7 @@ import com.google.common.annotations.VisibleForTesting; */ @InterfaceAudience.Private public class ClientContext { - private static final Log LOG = LogFactory.getLog(ClientContext.class); + private static final Logger LOG = LoggerFactory.getLogger(ClientContext.class); /** * Global map of context names to caches contexts. diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java index 3d0acb0792..a89f556744 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java @@ -22,22 +22,32 @@ import com.google.common.collect.Maps; import com.google.common.primitives.SignedBytes; import org.apache.commons.io.Charsets; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.crypto.key.KeyProvider; +import org.apache.hadoop.crypto.key.KeyProviderFactory; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +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.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB; import org.apache.hadoop.hdfs.web.WebHdfsConstants; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NodeBase; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.net.SocketFactory; +import java.io.IOException; import java.io.UnsupportedEncodingException; import java.net.InetAddress; import java.net.InetSocketAddress; +import java.net.URI; +import java.net.URISyntaxException; import java.text.SimpleDateFormat; import java.util.Collection; import java.util.Collections; @@ -455,4 +465,62 @@ public class DFSUtilClient { localAddrMap.put(addr.getHostAddress(), local); return local; } + + /** Create a {@link ClientDatanodeProtocol} proxy */ + public static ClientDatanodeProtocol createClientDatanodeProtocolProxy( + DatanodeID datanodeid, Configuration conf, int socketTimeout, + boolean connectToDnViaHostname, LocatedBlock locatedBlock) throws IOException { + return new ClientDatanodeProtocolTranslatorPB(datanodeid, conf, socketTimeout, + connectToDnViaHostname, locatedBlock); + } + + /** Create {@link ClientDatanodeProtocol} proxy using kerberos ticket */ + public static ClientDatanodeProtocol createClientDatanodeProtocolProxy( + DatanodeID datanodeid, Configuration conf, int socketTimeout, + boolean connectToDnViaHostname) throws IOException { + return new ClientDatanodeProtocolTranslatorPB( + datanodeid, conf, socketTimeout, connectToDnViaHostname); + } + + /** Create a {@link ClientDatanodeProtocol} proxy */ + public static ClientDatanodeProtocol createClientDatanodeProtocolProxy( + InetSocketAddress addr, UserGroupInformation ticket, Configuration conf, + SocketFactory factory) throws IOException { + return new ClientDatanodeProtocolTranslatorPB(addr, ticket, conf, factory); + } + + /** + * Creates a new KeyProvider from the given Configuration. + * + * @param conf Configuration + * @return new KeyProvider, or null if no provider was found. + * @throws IOException if the KeyProvider is improperly specified in + * the Configuration + */ + public static KeyProvider createKeyProvider( + final Configuration conf) throws IOException { + final String providerUriStr = + conf.getTrimmed(HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, ""); + // No provider set in conf + if (providerUriStr.isEmpty()) { + return null; + } + final URI providerUri; + try { + providerUri = new URI(providerUriStr); + } catch (URISyntaxException e) { + throw new IOException(e); + } + KeyProvider keyProvider = KeyProviderFactory.get(providerUri, conf); + if (keyProvider == null) { + throw new IOException("Could not instantiate KeyProvider from " + + HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI + " setting of '" + + providerUriStr + "'"); + } + if (keyProvider.isTransient()) { + throw new IOException("KeyProvider " + keyProvider.toString() + + " was found but it is a transient provider."); + } + return keyProvider; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java similarity index 100% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java similarity index 89% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java index a2b6c7e9a9..05492e018f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java @@ -21,14 +21,12 @@ import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -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.crypto.key.KeyProvider; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import com.google.common.annotations.VisibleForTesting; import com.google.common.cache.Cache; @@ -36,10 +34,13 @@ import com.google.common.cache.CacheBuilder; import com.google.common.cache.RemovalListener; import com.google.common.cache.RemovalNotification; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + @InterfaceAudience.Private public class KeyProviderCache { - public static final Log LOG = LogFactory.getLog(KeyProviderCache.class); + public static final Logger LOG = LoggerFactory.getLogger(KeyProviderCache.class); private final Cache cache; @@ -72,7 +73,7 @@ public class KeyProviderCache { return cache.get(kpURI, new Callable() { @Override public KeyProvider call() throws Exception { - return DFSUtil.createKeyProvider(conf); + return DFSUtilClient.createKeyProvider(conf); } }); } catch (Exception e) { @@ -83,11 +84,11 @@ public class KeyProviderCache { private URI createKeyProviderURI(Configuration conf) { final String providerUriStr = - conf.getTrimmed(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, ""); + conf.getTrimmed(HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, ""); // No provider set in conf if (providerUriStr.isEmpty()) { LOG.error("Could not find uri with key [" - + DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI + + HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI + "] to create a keyProvider !!"); return null; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/PeerCache.java similarity index 95% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/PeerCache.java index 08b0468bfa..55aa741ecf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/PeerCache.java @@ -27,15 +27,16 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.LinkedListMultimap; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.protocol.DatanodeID; +import org.apache.hadoop.hdfs.util.IOUtilsClient; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A cache of input stream sockets to Data Node. @@ -44,7 +45,7 @@ import org.apache.hadoop.util.Time; @InterfaceAudience.Private @VisibleForTesting public class PeerCache { - private static final Log LOG = LogFactory.getLog(PeerCache.class); + private static final Logger LOG = LoggerFactory.getLogger(PeerCache.class); private static class Key { final DatanodeID dnID; @@ -188,7 +189,7 @@ public class PeerCache { if (peer.isClosed()) return; if (capacity <= 0) { // Cache disabled. - IOUtils.cleanup(LOG, peer); + IOUtilsClient.cleanup(LOG, peer); return; } putInternal(dnId, peer); @@ -222,7 +223,7 @@ public class PeerCache { expiryPeriod) { break; } - IOUtils.cleanup(LOG, entry.getValue().getPeer()); + IOUtilsClient.cleanup(LOG, entry.getValue().getPeer()); iter.remove(); } } @@ -241,7 +242,7 @@ public class PeerCache { "capacity: " + capacity); } Entry entry = iter.next(); - IOUtils.cleanup(LOG, entry.getValue().getPeer()); + IOUtilsClient.cleanup(LOG, entry.getValue().getPeer()); iter.remove(); } @@ -269,7 +270,7 @@ public class PeerCache { @VisibleForTesting synchronized void clear() { for (Value value : multimap.values()) { - IOUtils.cleanup(LOG, value.getPeer()); + IOUtilsClient.cleanup(LOG, value.getPeer()); } multimap.clear(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/BlockReportOptions.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/BlockReportOptions.java similarity index 100% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/BlockReportOptions.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/BlockReportOptions.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java index 3b492ae58c..7b1e4384b5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java @@ -113,6 +113,11 @@ public interface HdfsClientConfigKeys { "dfs.datanode.hdfs-blocks-metadata.enabled"; boolean DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT = false; + String DFS_DATANODE_KERBEROS_PRINCIPAL_KEY = "dfs.datanode.kerberos.principal"; + String DFS_DATANODE_READAHEAD_BYTES_KEY = "dfs.datanode.readahead.bytes"; + long DFS_DATANODE_READAHEAD_BYTES_DEFAULT = 4 * 1024 * 1024; // 4MB + String DFS_ENCRYPTION_KEY_PROVIDER_URI = "dfs.encryption.key.provider.uri"; + String REPLICA_ACCESSOR_BUILDER_CLASSES_KEY = PREFIX + "replica.accessor.builder.classes"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java similarity index 100% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java similarity index 97% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java index da8f4ab858..3374868278 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java @@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.ReconfigurationTaskStatus; import org.apache.hadoop.hdfs.client.BlockReportOptions; -import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.BlockTokenSelector; import org.apache.hadoop.security.KerberosInfo; @@ -36,7 +36,7 @@ import org.apache.hadoop.security.token.TokenInfo; @InterfaceAudience.Private @InterfaceStability.Evolving @KerberosInfo( - serverPrincipal = DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY) + serverPrincipal = HdfsClientConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY) @TokenInfo(BlockTokenSelector.class) public interface ClientDatanodeProtocol { /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/InvalidEncryptionKeyException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/InvalidEncryptionKeyException.java similarity index 100% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/InvalidEncryptionKeyException.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/InvalidEncryptionKeyException.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java similarity index 91% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java index 21073eb2e3..7e3f66b548 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hdfs.protocolPB; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService; import org.apache.hadoop.hdfs.security.token.block.BlockTokenSelector; import org.apache.hadoop.ipc.ProtocolInfo; @@ -26,7 +26,7 @@ import org.apache.hadoop.security.KerberosInfo; import org.apache.hadoop.security.token.TokenInfo; @KerberosInfo( - serverPrincipal = DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY) + serverPrincipal = HdfsClientConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY) @TokenInfo(BlockTokenSelector.class) @ProtocolInfo(protocolName = "org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol", diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java similarity index 97% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java index 7b427fa58e..311fcea3e8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java @@ -27,8 +27,6 @@ import javax.net.SocketFactory; import com.google.common.base.Optional; import com.google.common.collect.Maps; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -70,6 +68,8 @@ import org.apache.hadoop.security.token.Token; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * This class is the client side translator to translate the requests made on @@ -81,8 +81,8 @@ import com.google.protobuf.ServiceException; public class ClientDatanodeProtocolTranslatorPB implements ProtocolMetaInterface, ClientDatanodeProtocol, ProtocolTranslator, Closeable { - public static final Log LOG = LogFactory - .getLog(ClientDatanodeProtocolTranslatorPB.class); + public static final Logger LOG = LoggerFactory + .getLogger(ClientDatanodeProtocolTranslatorPB.class); /** RpcController is not used and hence is set to null */ private final static RpcController NULL_CONTROLLER = null; @@ -219,7 +219,7 @@ public class ClientDatanodeProtocolTranslatorPB implements } catch (ServiceException e) { throw ProtobufHelper.getRemoteException(e); } - return new BlockLocalPathInfo(PBHelper.convert(resp.getBlock()), + return new BlockLocalPathInfo(PBHelperClient.convert(resp.getBlock()), resp.getLocalPath(), resp.getLocalMetaPath()); } @@ -251,7 +251,7 @@ public class ClientDatanodeProtocolTranslatorPB implements GetDatanodeInfoResponseProto response; try { response = rpcProxy.getDatanodeInfo(NULL_CONTROLLER, VOID_GET_DATANODE_INFO); - return PBHelper.convert(response.getLocalInfo()); + return PBHelperClient.convert(response.getLocalInfo()); } catch (ServiceException e) { throw ProtobufHelper.getRemoteException(e); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java index edf658ab4c..d921507487 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java @@ -23,12 +23,14 @@ import com.google.protobuf.CodedInputStream; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeLocalInfoProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto; import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId; @@ -185,6 +187,17 @@ public class PBHelperClient { return pinnings; } + public static ExtendedBlock convert(ExtendedBlockProto eb) { + if (eb == null) return null; + return new ExtendedBlock( eb.getPoolId(), eb.getBlockId(), eb.getNumBytes(), + eb.getGenerationStamp()); + } + + public static DatanodeLocalInfo convert(DatanodeLocalInfoProto proto) { + return new DatanodeLocalInfo(proto.getSoftwareVersion(), + proto.getConfigVersion(), proto.getUptime()); + } + static public DatanodeInfoProto convertDatanodeInfo(DatanodeInfo di) { if (di == null) return null; return convert(di); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSelector.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSelector.java similarity index 100% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSelector.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSelector.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 9f77e8510d..a561909210 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -858,6 +858,9 @@ Release 2.8.0 - UNRELEASED HDFS-8938. Extract BlockToMarkCorrupt and ReplicationWork as standalone classes from BlockManager. (Mingliang Liu via wheat9) + HDFS-8925. Move BlockReaderLocal to hdfs-client. + (Mingliang Liu via wheat9) + OPTIMIZATIONS HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 57a5aed64d..6420b55e11 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -80,8 +80,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final long DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_DEFAULT = 1024*1024; public static final String DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY = "dfs.datanode.balance.max.concurrent.moves"; public static final int DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT = 5; - public static final String DFS_DATANODE_READAHEAD_BYTES_KEY = "dfs.datanode.readahead.bytes"; - public static final long DFS_DATANODE_READAHEAD_BYTES_DEFAULT = 4 * 1024 * 1024; // 4MB + @Deprecated + public static final String DFS_DATANODE_READAHEAD_BYTES_KEY = + HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_KEY; + @Deprecated + public static final long DFS_DATANODE_READAHEAD_BYTES_DEFAULT = + HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT; public static final String DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_KEY = "dfs.datanode.drop.cache.behind.writes"; public static final boolean DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_DEFAULT = false; public static final String DFS_DATANODE_SYNC_BEHIND_WRITES_KEY = "dfs.datanode.sync.behind.writes"; @@ -505,7 +509,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final String DFS_WEB_UGI_KEY = "dfs.web.ugi"; public static final String DFS_NAMENODE_STARTUP_KEY = "dfs.namenode.startup"; public static final String DFS_DATANODE_KEYTAB_FILE_KEY = "dfs.datanode.keytab.file"; - public static final String DFS_DATANODE_KERBEROS_PRINCIPAL_KEY = "dfs.datanode.kerberos.principal"; + public static final String DFS_DATANODE_KERBEROS_PRINCIPAL_KEY = + HdfsClientConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY; @Deprecated public static final String DFS_DATANODE_USER_NAME_KEY = DFS_DATANODE_KERBEROS_PRINCIPAL_KEY; public static final String DFS_DATANODE_SHARED_FILE_DESCRIPTOR_PATHS = "dfs.datanode.shared.file.descriptor.paths"; @@ -604,7 +609,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final String DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY = "dfs.data.transfer.saslproperties.resolver.class"; public static final int DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES_DEFAULT = 100; public static final String DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES = "dfs.namenode.list.encryption.zones.num.responses"; - public static final String DFS_ENCRYPTION_KEY_PROVIDER_URI = "dfs.encryption.key.provider.uri"; + public static final String DFS_ENCRYPTION_KEY_PROVIDER_URI = + HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI; // Journal-node related configs. These are read on the JN side. public static final String DFS_JOURNALNODE_EDITS_DIR_KEY = "dfs.journalnode.edits.dir"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java index 7f3722f2f9..139a27cfd2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java @@ -364,7 +364,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead, ClientDatanodeProtocol cdp = null; try { - cdp = DFSUtil.createClientDatanodeProtocolProxy(datanode, + cdp = DFSUtilClient.createClientDatanodeProtocolProxy(datanode, dfsClient.getConfiguration(), conf.getSocketTimeout(), conf.isConnectToDnViaHostname(), locatedblock); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java index cae56c04da..5c8a7008a6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java @@ -53,8 +53,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ThreadLocalRandom; -import javax.net.SocketFactory; - import com.google.common.collect.Sets; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; @@ -69,16 +67,11 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.crypto.key.KeyProvider; import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension; -import org.apache.hadoop.crypto.key.KeyProviderFactory; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -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.HdfsConstants; -import org.apache.hadoop.hdfs.protocol.LocatedBlock; -import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.http.HttpConfig; @@ -932,29 +925,6 @@ public class DFSUtil { public static int roundBytesToGB(long bytes) { return Math.round((float)bytes/ 1024 / 1024 / 1024); } - - /** Create a {@link ClientDatanodeProtocol} proxy */ - public static ClientDatanodeProtocol createClientDatanodeProtocolProxy( - DatanodeID datanodeid, Configuration conf, int socketTimeout, - boolean connectToDnViaHostname, LocatedBlock locatedBlock) throws IOException { - return new ClientDatanodeProtocolTranslatorPB(datanodeid, conf, socketTimeout, - connectToDnViaHostname, locatedBlock); - } - - /** Create {@link ClientDatanodeProtocol} proxy using kerberos ticket */ - public static ClientDatanodeProtocol createClientDatanodeProtocolProxy( - DatanodeID datanodeid, Configuration conf, int socketTimeout, - boolean connectToDnViaHostname) throws IOException { - return new ClientDatanodeProtocolTranslatorPB( - datanodeid, conf, socketTimeout, connectToDnViaHostname); - } - - /** Create a {@link ClientDatanodeProtocol} proxy */ - public static ClientDatanodeProtocol createClientDatanodeProtocolProxy( - InetSocketAddress addr, UserGroupInformation ticket, Configuration conf, - SocketFactory factory) throws IOException { - return new ClientDatanodeProtocolTranslatorPB(addr, ticket, conf, factory); - } /** * Get nameservice Id for the {@link NameNode} based on namenode RPC address @@ -1449,41 +1419,6 @@ public class DFSUtil { } } - /** - * Creates a new KeyProvider from the given Configuration. - * - * @param conf Configuration - * @return new KeyProvider, or null if no provider was found. - * @throws IOException if the KeyProvider is improperly specified in - * the Configuration - */ - public static KeyProvider createKeyProvider( - final Configuration conf) throws IOException { - final String providerUriStr = - conf.getTrimmed(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, ""); - // No provider set in conf - if (providerUriStr.isEmpty()) { - return null; - } - final URI providerUri; - try { - providerUri = new URI(providerUriStr); - } catch (URISyntaxException e) { - throw new IOException(e); - } - KeyProvider keyProvider = KeyProviderFactory.get(providerUri, conf); - if (keyProvider == null) { - throw new IOException("Could not instantiate KeyProvider from " + - DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI + " setting of '" + - providerUriStr +"'"); - } - if (keyProvider.isTransient()) { - throw new IOException("KeyProvider " + keyProvider.toString() - + " was found but it is a transient provider."); - } - return keyProvider; - } - /** * Creates a new KeyProviderCryptoExtension by wrapping the * KeyProvider specified in the given Configuration. @@ -1495,7 +1430,7 @@ public class DFSUtil { */ public static KeyProviderCryptoExtension createKeyProviderCryptoExtension( final Configuration conf) throws IOException { - KeyProvider keyProvider = createKeyProvider(conf); + KeyProvider keyProvider = DFSUtilClient.createKeyProvider(conf); if (keyProvider == null) { return null; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java index 694f5212b9..85da414dcc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto; -import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCustomProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto; @@ -115,7 +114,7 @@ public abstract class Receiver implements DataTransferProtocol { TraceScope traceScope = continueTraceSpan(proto.getHeader(), proto.getClass().getSimpleName()); try { - readBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), + readBlock(PBHelperClient.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), proto.getOffset(), @@ -136,7 +135,7 @@ public abstract class Receiver implements DataTransferProtocol { TraceScope traceScope = continueTraceSpan(proto.getHeader(), proto.getClass().getSimpleName()); try { - writeBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), + writeBlock(PBHelperClient.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelperClient.convertStorageType(proto.getStorageType()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), @@ -167,7 +166,7 @@ public abstract class Receiver implements DataTransferProtocol { TraceScope traceScope = continueTraceSpan(proto.getHeader(), proto.getClass().getSimpleName()); try { - transferBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), + transferBlock(PBHelperClient.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), targets, @@ -186,7 +185,7 @@ public abstract class Receiver implements DataTransferProtocol { TraceScope traceScope = continueTraceSpan(proto.getHeader(), proto.getClass().getSimpleName()); try { - requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()), + requestShortCircuitFds(PBHelperClient.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken()), slotId, proto.getMaxVersion(), proto.getSupportsReceiptVerification()); @@ -228,7 +227,7 @@ public abstract class Receiver implements DataTransferProtocol { TraceScope traceScope = continueTraceSpan(proto.getHeader(), proto.getClass().getSimpleName()); try { - replaceBlock(PBHelper.convert(proto.getHeader().getBlock()), + replaceBlock(PBHelperClient.convert(proto.getHeader().getBlock()), PBHelperClient.convertStorageType(proto.getStorageType()), PBHelper.convert(proto.getHeader().getToken()), proto.getDelHint(), @@ -244,7 +243,7 @@ public abstract class Receiver implements DataTransferProtocol { TraceScope traceScope = continueTraceSpan(proto.getHeader(), proto.getClass().getSimpleName()); try { - copyBlock(PBHelper.convert(proto.getHeader().getBlock()), + copyBlock(PBHelperClient.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken())); } finally { if (traceScope != null) traceScope.close(); @@ -257,7 +256,7 @@ public abstract class Receiver implements DataTransferProtocol { TraceScope traceScope = continueTraceSpan(proto.getHeader(), proto.getClass().getSimpleName()); try { - blockChecksum(PBHelper.convert(proto.getHeader().getBlock()), + blockChecksum(PBHelperClient.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken())); } finally { if (traceScope != null) traceScope.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java index 3886007c30..5efcf67984 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java @@ -18,8 +18,6 @@ package org.apache.hadoop.hdfs.protocolPB; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import java.util.Map; import com.google.common.base.Optional; @@ -86,7 +84,7 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements throws ServiceException { long len; try { - len = impl.getReplicaVisibleLength(PBHelper.convert(request.getBlock())); + len = impl.getReplicaVisibleLength(PBHelperClient.convert(request.getBlock())); } catch (IOException e) { throw new ServiceException(e); } @@ -123,7 +121,7 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements throws ServiceException { BlockLocalPathInfo resp; try { - resp = impl.getBlockLocalPathInfo(PBHelper.convert(request.getBlock()), PBHelper.convert(request.getToken())); + resp = impl.getBlockLocalPathInfo(PBHelperClient.convert(request.getBlock()), PBHelper.convert(request.getToken())); } catch (IOException e) { throw new ServiceException(e); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java index beaa903e47..9f26ca3daf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java @@ -474,7 +474,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements public AbandonBlockResponseProto abandonBlock(RpcController controller, AbandonBlockRequestProto req) throws ServiceException { try { - server.abandonBlock(PBHelper.convert(req.getB()), req.getFileId(), + server.abandonBlock(PBHelperClient.convert(req.getB()), req.getFileId(), req.getSrc(), req.getHolder()); } catch (IOException e) { throw new ServiceException(e); @@ -492,7 +492,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements LocatedBlock result = server.addBlock( req.getSrc(), req.getClientName(), - req.hasPrevious() ? PBHelper.convert(req.getPrevious()) : null, + req.hasPrevious() ? PBHelperClient.convert(req.getPrevious()) : null, (excl == null || excl.size() == 0) ? null : PBHelper.convert(excl .toArray(new DatanodeInfoProto[excl.size()])), req.getFileId(), (favor == null || favor.size() == 0) ? null : favor @@ -513,7 +513,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements List existingStorageIDsList = req.getExistingStorageUuidsList(); List excludesList = req.getExcludesList(); LocatedBlock result = server.getAdditionalDatanode(req.getSrc(), - req.getFileId(), PBHelper.convert(req.getBlk()), + req.getFileId(), PBHelperClient.convert(req.getBlk()), PBHelper.convert(existingList.toArray( new DatanodeInfoProto[existingList.size()])), existingStorageIDsList.toArray( @@ -535,7 +535,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements try { boolean result = server.complete(req.getSrc(), req.getClientName(), - req.hasLast() ? PBHelper.convert(req.getLast()) : null, + req.hasLast() ? PBHelperClient.convert(req.getLast()) : null, req.hasFileId() ? req.getFileId() : HdfsConstants.GRANDFATHER_INODE_ID); return CompleteResponseProto.newBuilder().setResult(result).build(); } catch (IOException e) { @@ -954,7 +954,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements throws ServiceException { try { LocatedBlockProto result = PBHelper.convert(server - .updateBlockForPipeline(PBHelper.convert(req.getBlock()), + .updateBlockForPipeline(PBHelperClient.convert(req.getBlock()), req.getClientName())); return UpdateBlockForPipelineResponseProto.newBuilder().setBlock(result) .build(); @@ -970,8 +970,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements List newNodes = req.getNewNodesList(); List newStorageIDs = req.getStorageIDsList(); server.updatePipeline(req.getClientName(), - PBHelper.convert(req.getOldBlock()), - PBHelper.convert(req.getNewBlock()), + PBHelperClient.convert(req.getOldBlock()), + PBHelperClient.convert(req.getNewBlock()), PBHelper.convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()])), newStorageIDs.toArray(new String[newStorageIDs.size()])); return VOID_UPDATEPIPELINE_RESPONSE; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java index e133ec7923..5964e151ec 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java @@ -281,7 +281,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements final List sidprotos = request.getNewTargetStoragesList(); final String[] storageIDs = sidprotos.toArray(new String[sidprotos.size()]); try { - impl.commitBlockSynchronization(PBHelper.convert(request.getBlock()), + impl.commitBlockSynchronization(PBHelperClient.convert(request.getBlock()), request.getNewGenStamp(), request.getNewLength(), request.getCloseFile(), request.getDeleteBlock(), dns, storageIDs); } catch (IOException e) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java index ba0a8fc24a..fb67e36e5b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java @@ -76,7 +76,7 @@ public class InterDatanodeProtocolServerSideTranslatorPB implements final String storageID; try { storageID = impl.updateReplicaUnderRecovery( - PBHelper.convert(request.getBlock()), request.getRecoveryId(), + PBHelperClient.convert(request.getBlock()), request.getRecoveryId(), request.getNewBlockId(), request.getNewLength()); } catch (IOException e) { throw new ServiceException(e); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java index 887accfe39..6f16d8327f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java @@ -23,9 +23,7 @@ import static org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CipherSuiteProto; import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CryptoProtocolVersionProto; -import java.io.EOFException; import java.io.IOException; -import java.io.InputStream; import java.util.ArrayList; import java.util.Arrays; import java.util.EnumSet; @@ -110,7 +108,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsS import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeActionProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeInfoProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto; -import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto; @@ -146,7 +143,6 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto.StorageState; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto; -import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsServerDefaultsProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto; @@ -218,20 +214,16 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog; import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest; import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary; -import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId; import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId; -import org.apache.hadoop.hdfs.util.ExactSizeInputStream; import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.util.DataChecksum; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.primitives.Shorts; import com.google.protobuf.ByteString; -import com.google.protobuf.CodedInputStream; /** * Utilities for converting protobuf classes to and from implementation classes @@ -574,13 +566,7 @@ public class PBHelper { return new NamenodeCommand(cmd.getAction()); } } - - public static ExtendedBlock convert(ExtendedBlockProto eb) { - if (eb == null) return null; - return new ExtendedBlock( eb.getPoolId(), eb.getBlockId(), eb.getNumBytes(), - eb.getGenerationStamp()); - } - + public static RecoveringBlockProto convert(RecoveringBlock b) { if (b == null) { return null; @@ -594,7 +580,7 @@ public class PBHelper { } public static RecoveringBlock convert(RecoveringBlockProto b) { - ExtendedBlock block = convert(b.getBlock().getB()); + ExtendedBlock block = PBHelperClient.convert(b.getBlock().getB()); DatanodeInfo[] locs = convert(b.getBlock().getLocsList()); return (b.hasTruncateBlock()) ? new RecoveringBlock(block, locs, PBHelper.convert(b.getTruncateBlock())) : @@ -740,7 +726,7 @@ public class PBHelper { } } - LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets, + LocatedBlock lb = new LocatedBlock(PBHelperClient.convert(proto.getB()), targets, storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(), cachedLocs.toArray(new DatanodeInfo[0])); lb.setBlockToken(PBHelper.convert(proto.getBlockToken())); @@ -2119,12 +2105,6 @@ public class PBHelper { return builder.build(); } - public static DatanodeLocalInfo convert(DatanodeLocalInfoProto proto) { - return new DatanodeLocalInfo(proto.getSoftwareVersion(), - proto.getConfigVersion(), proto.getUptime()); - } - - private static AclEntryScopeProto convert(AclEntryScope v) { return AclEntryScopeProto.valueOf(v.ordinal()); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java index 66fd5671dd..9c25f5eb35 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java @@ -125,8 +125,8 @@ public class DNConf { DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT); readaheadLength = conf.getLong( - DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_KEY, - DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_KEY, + HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); dropCacheBehindWrites = conf.getBoolean( DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_KEY, DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_DEFAULT); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java index 298d55e6b1..2e64c4468d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java @@ -51,6 +51,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.shell.Command; import org.apache.hadoop.fs.shell.CommandFormat; import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.DFSUtilClient; import org.apache.hadoop.hdfs.HAUtilClient; import org.apache.hadoop.hdfs.client.BlockReportOptions; import org.apache.hadoop.hdfs.DFSConfigKeys; @@ -1967,7 +1968,7 @@ public class DFSAdmin extends FsShell { // Create the client ClientDatanodeProtocol dnProtocol = - DFSUtil.createClientDatanodeProtocolProxy(datanodeAddr, getUGI(), conf, + DFSUtilClient.createClientDatanodeProtocolProxy(datanodeAddr, getUGI(), conf, NetUtils.getSocketFactory(conf, ClientDatanodeProtocol.class)); return dnProtocol; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java index aad670a671..1c4394897c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java @@ -247,7 +247,7 @@ public class TestBlockReaderLocal { @Test public void testBlockReaderSimpleReads() throws IOException { runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true, - DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } @Test @@ -259,7 +259,7 @@ public class TestBlockReaderLocal { @Test public void testBlockReaderSimpleReadsNoChecksum() throws IOException { runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), false, - DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } @Test @@ -297,14 +297,14 @@ public class TestBlockReaderLocal { @Test public void testBlockReaderLocalArrayReads2() throws IOException { runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(), - true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } @Test public void testBlockReaderLocalArrayReads2NoChecksum() throws IOException { runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(), - false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } @Test @@ -341,7 +341,7 @@ public class TestBlockReaderLocal { public void testBlockReaderLocalByteBufferReads() throws IOException { runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferReads(), - true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } @Test @@ -349,7 +349,7 @@ public class TestBlockReaderLocal { throws IOException { runBlockReaderLocalTest( new TestBlockReaderLocalByteBufferReads(), - false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } @Test @@ -473,7 +473,7 @@ public class TestBlockReaderLocal { public void testBlockReaderLocalReadCorruptStart() throws IOException { runBlockReaderLocalTest(new TestBlockReaderLocalReadCorruptStart(), true, - DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } private static class TestBlockReaderLocalReadCorrupt @@ -524,14 +524,14 @@ public class TestBlockReaderLocal { public void testBlockReaderLocalReadCorrupt() throws IOException { runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), true, - DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } @Test public void testBlockReaderLocalReadCorruptNoChecksum() throws IOException { runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), false, - DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } @Test @@ -576,14 +576,14 @@ public class TestBlockReaderLocal { public void testBlockReaderLocalWithMlockChanges() throws IOException { runBlockReaderLocalTest(new TestBlockReaderLocalWithMlockChanges(), - true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } @Test public void testBlockReaderLocalWithMlockChangesNoChecksum() throws IOException { runBlockReaderLocalTest(new TestBlockReaderLocalWithMlockChanges(), - false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } @Test @@ -649,14 +649,14 @@ public class TestBlockReaderLocal { public void testBlockReaderLocalOnFileWithoutChecksum() throws IOException { runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(), - true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } @Test public void testBlockReaderLocalOnFileWithoutChecksumNoChecksum() throws IOException { runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(), - false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } @Test @@ -677,14 +677,14 @@ public class TestBlockReaderLocal { public void testBlockReaderLocalReadZeroBytes() throws IOException { runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(), - true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } @Test public void testBlockReaderLocalReadZeroBytesNoChecksum() throws IOException { runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(), - false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); + false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); } @Test diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java index 3deca17048..af28bd314f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java @@ -182,7 +182,7 @@ public class TestBlockReaderLocalLegacy { { final LocatedBlock lb = cluster.getNameNode().getRpcServer() .getBlockLocations(path.toString(), 0, 1).get(0); - proxy = DFSUtil.createClientDatanodeProtocolProxy( + proxy = DFSUtilClient.createClientDatanodeProtocolProxy( lb.getLocations()[0], conf, 60000, false); token = lb.getBlockToken(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java index 5082525eff..6863ebe94a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java @@ -860,7 +860,7 @@ public class TestDFSClientRetries { ClientDatanodeProtocol proxy = null; try { - proxy = DFSUtil.createClientDatanodeProtocolProxy( + proxy = DFSUtilClient.createClientDatanodeProtocolProxy( fakeDnId, conf, 500, false, fakeBlock); proxy.getReplicaVisibleLength(new ExtendedBlock("bpid", 1)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java index f25fb1b0aa..3d2e8b945d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java @@ -333,12 +333,12 @@ public class TestPBHelper { public void testConvertExtendedBlock() { ExtendedBlock b = getExtendedBlock(); ExtendedBlockProto bProto = PBHelperClient.convert(b); - ExtendedBlock b1 = PBHelper.convert(bProto); + ExtendedBlock b1 = PBHelperClient.convert(bProto); assertEquals(b, b1); b.setBlockId(-1); bProto = PBHelperClient.convert(b); - b1 = PBHelper.convert(bProto); + b1 = PBHelperClient.convert(bProto); assertEquals(b, b1); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java index b2038720db..ab42498185 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java @@ -43,7 +43,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSTestUtil; -import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.DFSUtilClient; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.Block; @@ -57,7 +57,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.Client import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto; import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolPB; -import org.apache.hadoop.hdfs.protocolPB.PBHelper; +import org.apache.hadoop.hdfs.protocolPB.PBHelperClient; import org.apache.hadoop.io.TestWritable; import org.apache.hadoop.ipc.Client; import org.apache.hadoop.ipc.ProtobufRpcEngine; @@ -138,7 +138,7 @@ public class TestBlockToken { BlockTokenIdentifier id = (BlockTokenIdentifier) tokenId; LOG.info("Got: " + id.toString()); assertTrue("Received BlockTokenIdentifier is wrong", ident.equals(id)); - sm.checkAccess(id, null, PBHelper.convert(req.getBlock()), + sm.checkAccess(id, null, PBHelperClient.convert(req.getBlock()), BlockTokenIdentifier.AccessMode.WRITE); result = id.getBlockId(); } @@ -259,7 +259,7 @@ public class TestBlockToken { ClientDatanodeProtocol proxy = null; try { - proxy = DFSUtil.createClientDatanodeProtocolProxy(addr, ticket, conf, + proxy = DFSUtilClient.createClientDatanodeProtocolProxy(addr, ticket, conf, NetUtils.getDefaultSocketFactory(conf)); assertEquals(block3.getBlockId(), proxy.getReplicaVisibleLength(block3)); } finally { @@ -313,7 +313,7 @@ public class TestBlockToken { try { long endTime = Time.now() + 3000; while (Time.now() < endTime) { - proxy = DFSUtil.createClientDatanodeProtocolProxy(fakeDnId, conf, 1000, + proxy = DFSUtilClient.createClientDatanodeProtocolProxy(fakeDnId, conf, 1000, false, fakeBlock); assertEquals(block3.getBlockId(), proxy.getReplicaVisibleLength(block3)); if (proxy != null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java index 0ee433f391..b6b8a490bf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java @@ -41,7 +41,7 @@ import org.apache.hadoop.hdfs.ClientContext; import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSTestUtil; -import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.DFSUtilClient; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -367,7 +367,7 @@ public class TestShortCircuitLocalRead { Token token = lb.get(0).getBlockToken(); final DatanodeInfo dnInfo = lb.get(0).getLocations()[0]; ClientDatanodeProtocol proxy = - DFSUtil.createClientDatanodeProtocolProxy(dnInfo, conf, 60000, false); + DFSUtilClient.createClientDatanodeProtocolProxy(dnInfo, conf, 60000, false); try { proxy.getBlockLocalPathInfo(blk, token); Assert.fail("The call should have failed as this user " From bdbe53c676dd4ff135ea2f64d3b9193fe43d7c8e Mon Sep 17 00:00:00 2001 From: Arpit Agarwal Date: Sat, 29 Aug 2015 09:51:55 -0700 Subject: [PATCH 04/18] HDFS-8983. NameNode support for protected directories. (Contributed by Arpit Agarwal) --- .../fs/CommonConfigurationKeysPublic.java | 7 + .../src/main/resources/core-default.xml | 10 + hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 2 + .../hdfs/server/namenode/FSDirDeleteOp.java | 40 ++ .../hdfs/server/namenode/FSDirectory.java | 63 +++ .../namenode/TestProtectedDirectories.java | 373 ++++++++++++++++++ 6 files changed, 495 insertions(+) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProtectedDirectories.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java index 24d648fb10..f3bc2e14f3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java @@ -85,6 +85,13 @@ public class CommonConfigurationKeysPublic { /** Default value for FS_TRASH_CHECKPOINT_INTERVAL_KEY */ public static final long FS_TRASH_CHECKPOINT_INTERVAL_DEFAULT = 0; + /** + * Directories that cannot be removed unless empty, even by an + * administrator. + */ + public static final String FS_PROTECTED_DIRECTORIES = + "fs.protected.directories"; + // TBD: Code is still using hardcoded values (e.g. "fs.automatic.close") // instead of constant (e.g. FS_AUTOMATIC_CLOSE_KEY) // diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index d02f0acc35..cef32d3f89 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -564,6 +564,16 @@ for ldap providers in the same way as above does. + + fs.protected.directories + + A comma-separated list of directories which cannot + be deleted even by the superuser unless they are empty. This + setting can be used to guard important system directories + against accidental deletion due to administrator error. + + + fs.AbstractFileSystem.file.impl org.apache.hadoop.fs.local.LocalFs diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index a561909210..6f46ea5453 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -861,6 +861,8 @@ Release 2.8.0 - UNRELEASED HDFS-8925. Move BlockReaderLocal to hdfs-client. (Mingliang Liu via wheat9) + HDFS-8983. NameNode support for protected directories. (Arpit Agarwal) + OPTIMIZATIONS HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java index b0e9a5c087..51d643a8fc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java @@ -17,15 +17,19 @@ */ package org.apache.hadoop.hdfs.server.namenode; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException; +import org.apache.hadoop.fs.UnresolvedLinkException; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; import org.apache.hadoop.hdfs.server.namenode.INode.ReclaimContext; +import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.util.ChunkedArrayList; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.SortedSet; import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID; import static org.apache.hadoop.util.Time.now; @@ -103,6 +107,9 @@ class FSDirDeleteOp { fsd.checkPermission(pc, iip, false, null, FsAction.WRITE, null, FsAction.ALL, true); } + if (recursive && fsd.isNonEmptyDirectory(iip)) { + checkProtectedDescendants(fsd, fsd.normalizePath(src)); + } return deleteInternal(fsn, src, iip, logRetryCache); } @@ -262,4 +269,37 @@ class FSDirDeleteOp { } return true; } + + /** + * Throw if the given directory has any non-empty protected descendants + * (including itself). + * + * @param src directory whose descendants are to be checked. The caller + * must ensure src is not terminated with {@link Path#SEPARATOR}. + * @throws AccessControlException if a non-empty protected descendant + * was found. + */ + private static void checkProtectedDescendants(FSDirectory fsd, String src) + throws AccessControlException, UnresolvedLinkException { + final SortedSet protectedDirs = fsd.getProtectedDirectories(); + + // Is src protected? Caller has already checked it is non-empty. + if (protectedDirs.contains(src)) { + throw new AccessControlException( + "Cannot delete non-empty protected directory " + src); + } + + // Are any descendants of src protected? + // The subSet call returns only the descendants of src since + // {@link Path#SEPARATOR} is "/" and '0' is the next ASCII + // character after '/'. + for (String descendant : + protectedDirs.subSet(src + Path.SEPARATOR, src + "0")) { + if (fsd.isNonEmptyDirectory(fsd.getINodesInPath4Write( + descendant, false))) { + throw new AccessControlException( + "Cannot delete non-empty protected subdirectory " + descendant); + } + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java index 409174ae9f..734d3c0e58 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java @@ -70,9 +70,12 @@ import java.util.Collection; import java.util.EnumSet; import java.util.List; import java.util.Map; +import java.util.SortedSet; +import java.util.TreeSet; import java.util.concurrent.locks.ReentrantReadWriteLock; import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries; +import static org.apache.hadoop.fs.CommonConfigurationKeys.FS_PROTECTED_DIRECTORIES; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_QUOTA_BY_STORAGETYPE_ENABLED_DEFAULT; @@ -137,6 +140,13 @@ public class FSDirectory implements Closeable { private final int inodeXAttrsLimit; //inode xattrs max limit + // A set of directories that have been protected using the + // dfs.namenode.protected.directories setting. These directories cannot + // be deleted unless they are empty. + // + // Each entry in this set must be a normalized path. + private final SortedSet protectedDirectories; + // lock to protect the directory and BlockMap private final ReentrantReadWriteLock dirLock; @@ -278,6 +288,8 @@ public class FSDirectory implements Closeable { DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY, DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_DEFAULT); + this.protectedDirectories = parseProtectedDirectories(conf); + Preconditions.checkArgument(this.inodeXAttrsLimit >= 0, "Cannot set a negative limit on the number of xattrs per inode (%s).", DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY); @@ -305,6 +317,25 @@ public class FSDirectory implements Closeable { return namesystem; } + /** + * Parse configuration setting dfs.namenode.protected.directories to + * retrieve the set of protected directories. + * + * @param conf + * @return a TreeSet + */ + @VisibleForTesting + static SortedSet parseProtectedDirectories(Configuration conf) { + // Normalize each input path to guard against administrator error. + return new TreeSet<>(normalizePaths( + conf.getTrimmedStringCollection(FS_PROTECTED_DIRECTORIES), + FS_PROTECTED_DIRECTORIES)); + } + + SortedSet getProtectedDirectories() { + return protectedDirectories; + } + BlockManager getBlockManager() { return getFSNamesystem().getBlockManager(); } @@ -905,6 +936,38 @@ public class FSDirectory implements Closeable { && INodeReference.tryRemoveReference(last) > 0) ? 0 : 1; } + /** + * Return a new collection of normalized paths from the given input + * collection. The input collection is unmodified. + * + * Reserved paths, relative paths and paths with scheme are ignored. + * + * @param paths collection whose contents are to be normalized. + * @return collection with all input paths normalized. + */ + static Collection normalizePaths(Collection paths, + String errorString) { + if (paths.isEmpty()) { + return paths; + } + final Collection normalized = new ArrayList<>(paths.size()); + for (String dir : paths) { + if (isReservedName(dir)) { + LOG.error("{} ignoring reserved path {}", errorString, dir); + } else { + final Path path = new Path(dir); + if (!path.isAbsolute()) { + LOG.error("{} ignoring relative path {}", errorString, dir); + } else if (path.toUri().getScheme() != null) { + LOG.error("{} ignoring path {} with scheme", errorString, dir); + } else { + normalized.add(path.toString()); + } + } + } + return normalized; + } + static String normalizePath(String src) { if (src.length() > 1 && src.endsWith("/")) { src = src.substring(0, src.length() - 1); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProtectedDirectories.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProtectedDirectories.java new file mode 100644 index 0000000000..be7b686675 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProtectedDirectories.java @@ -0,0 +1,373 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdfs.server.namenode; + +import com.google.common.base.Joiner; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.security.AccessControlException; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.*; + +import static org.hamcrest.core.Is.is; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; + +/** + * Verify that the dfs.namenode.protected.directories setting is respected. + */ +public class TestProtectedDirectories { + static final Logger LOG = LoggerFactory.getLogger( + TestProtectedDirectories.class); + + @Rule + public Timeout timeout = new Timeout(300000); + + /** + * Start a namenode-only 'cluster' which is configured to protect + * the given list of directories. + * @param conf + * @param protectedDirs + * @param unProtectedDirs + * @return + * @throws IOException + */ + public MiniDFSCluster setupTestCase(Configuration conf, + Collection protectedDirs, + Collection unProtectedDirs) + throws Throwable { + // Initialize the configuration. + conf.set( + CommonConfigurationKeys.FS_PROTECTED_DIRECTORIES, + Joiner.on(",").skipNulls().join(protectedDirs)); + + // Start the cluster. + MiniDFSCluster cluster = + new MiniDFSCluster.Builder(conf).numDataNodes(0).build(); + + // Create all the directories. + try { + cluster.waitActive(); + FileSystem fs = cluster.getFileSystem(); + for (Path path : Iterables.concat(protectedDirs, unProtectedDirs)) { + fs.mkdirs(path); + } + return cluster; + } catch (Throwable t) { + cluster.shutdown(); + throw t; + } + } + + /** + * Initialize a collection of file system layouts that will be used + * as the test matrix. + * + * @return + */ + private Collection createTestMatrix() { + Collection matrix = new ArrayList(); + + // single empty unprotected dir. + matrix.add(TestMatrixEntry.get() + .addUnprotectedDir("/1", true)); + + // Single empty protected dir. + matrix.add(TestMatrixEntry.get() + .addProtectedDir("/1", true)); + + // Nested unprotected dirs. + matrix.add(TestMatrixEntry.get() + .addUnprotectedDir("/1", true) + .addUnprotectedDir("/1/2", true) + .addUnprotectedDir("/1/2/3", true) + .addUnprotectedDir("/1/2/3/4", true)); + + // Non-empty protected dir. + matrix.add(TestMatrixEntry.get() + .addProtectedDir("/1", false) + .addUnprotectedDir("/1/2", true)); + + // Protected empty child of unprotected parent. + matrix.add(TestMatrixEntry.get() + .addProtectedDir("/1/2", true) + .addUnprotectedDir("/1/2", true)); + + // Protected empty child of protected parent. + // We should not be able to delete the parent. + matrix.add(TestMatrixEntry.get() + .addProtectedDir("/1", false) + .addProtectedDir("/1/2", true)); + + // One of each, non-nested. + matrix.add(TestMatrixEntry.get() + .addProtectedDir("/1", true) + .addUnprotectedDir("/a", true)); + + // Protected non-empty child of unprotected parent. + // Neither should be deletable. + matrix.add(TestMatrixEntry.get() + .addProtectedDir("/1/2", false) + .addUnprotectedDir("/1/2/3", true) + .addUnprotectedDir("/1", false)); + + // Protected non-empty child has unprotected siblings. + matrix.add(TestMatrixEntry.get() + .addProtectedDir("/1/2.2", false) + .addUnprotectedDir("/1/2.2/3", true) + .addUnprotectedDir("/1/2.1", true) + .addUnprotectedDir("/1/2.3", true) + .addUnprotectedDir("/1", false)); + + // Deeply nested protected child. + matrix.add(TestMatrixEntry.get() + .addProtectedDir("/1/2/3/4/5", false) + .addUnprotectedDir("/1/2/3/4/5/6", true) + .addUnprotectedDir("/1", false) + .addUnprotectedDir("/1/2", false) + .addUnprotectedDir("/1/2/3", false) + .addUnprotectedDir("/1/2/3/4", false)); + + // Disjoint trees. + matrix.add(TestMatrixEntry.get() + .addProtectedDir("/1/2", false) + .addProtectedDir("/a/b", false) + .addUnprotectedDir("/1/2/3", true) + .addUnprotectedDir("/a/b/c", true)); + + // The following tests exercise special cases in the path prefix + // checks and handling of trailing separators. + + // A disjoint non-empty protected dir has the same string prefix as the + // directory we are trying to delete. + matrix.add(TestMatrixEntry.get() + .addProtectedDir("/a1", false) + .addUnprotectedDir("/a1/a2", true) + .addUnprotectedDir("/a", true)); + + // The directory we are trying to delete has a non-empty protected + // child and we try to delete it with a trailing separator. + matrix.add(TestMatrixEntry.get() + .addProtectedDir("/a/b", false) + .addUnprotectedDir("/a/b/c", true) + .addUnprotectedDir("/a/", false)); + + // The directory we are trying to delete has an empty protected + // child and we try to delete it with a trailing separator. + matrix.add(TestMatrixEntry.get() + .addProtectedDir("/a/b", true) + .addUnprotectedDir("/a/", true)); + + return matrix; + } + + @Test + public void testAll() throws Throwable { + for (TestMatrixEntry testMatrixEntry : createTestMatrix()) { + Configuration conf = new HdfsConfiguration(); + MiniDFSCluster cluster = setupTestCase( + conf, testMatrixEntry.getProtectedPaths(), + testMatrixEntry.getUnprotectedPaths()); + + try { + LOG.info("Running {}", testMatrixEntry); + FileSystem fs = cluster.getFileSystem(); + for (Path path : testMatrixEntry.getAllPathsToBeDeleted()) { + final long countBefore = cluster.getNamesystem().getFilesTotal(); + assertThat( + testMatrixEntry + ": Testing whether " + path + " can be deleted", + deletePath(fs, path), + is(testMatrixEntry.canPathBeDeleted(path))); + final long countAfter = cluster.getNamesystem().getFilesTotal(); + + if (!testMatrixEntry.canPathBeDeleted(path)) { + assertThat( + "Either all paths should be deleted or none", + countAfter, is(countBefore)); + } + } + } finally { + cluster.shutdown(); + } + } + } + + /** + * Verify that configured paths are normalized by removing + * redundant separators. + */ + @Test + public void testProtectedDirNormalization1() { + Configuration conf = new HdfsConfiguration(); + conf.set( + CommonConfigurationKeys.FS_PROTECTED_DIRECTORIES, + "/foo//bar"); + Collection paths = FSDirectory.parseProtectedDirectories(conf); + assertThat(paths.size(), is(1)); + assertThat(paths.iterator().next(), is("/foo/bar")); + } + + /** + * Verify that configured paths are normalized by removing + * trailing separators. + */ + @Test + public void testProtectedDirNormalization2() { + Configuration conf = new HdfsConfiguration(); + conf.set( + CommonConfigurationKeys.FS_PROTECTED_DIRECTORIES, + "/a/b/,/c,/d/e/f/"); + Collection paths = FSDirectory.parseProtectedDirectories(conf); + + for (String path : paths) { + assertFalse(path.endsWith("/")); + } + } + + /** + * Verify that configured paths are canonicalized. + */ + @Test + public void testProtectedDirIsCanonicalized() { + Configuration conf = new HdfsConfiguration(); + conf.set( + CommonConfigurationKeys.FS_PROTECTED_DIRECTORIES, + "/foo/../bar/"); + Collection paths = FSDirectory.parseProtectedDirectories(conf); + assertThat(paths.size(), is(1)); + assertThat(paths.iterator().next(), is("/bar")); + } + + /** + * Verify that the root directory in the configuration is correctly handled. + */ + @Test + public void testProtectedRootDirectory() { + Configuration conf = new HdfsConfiguration(); + conf.set( + CommonConfigurationKeys.FS_PROTECTED_DIRECTORIES, "/"); + Collection paths = FSDirectory.parseProtectedDirectories(conf); + assertThat(paths.size(), is(1)); + assertThat(paths.iterator().next(), is("/")); + } + + /** + * Verify that invalid paths in the configuration are filtered out. + * (Path with scheme, reserved path). + */ + @Test + public void testBadPathsInConfig() { + Configuration conf = new HdfsConfiguration(); + conf.set( + CommonConfigurationKeys.FS_PROTECTED_DIRECTORIES, + "hdfs://foo/,/.reserved/foo"); + Collection paths = FSDirectory.parseProtectedDirectories(conf); + assertThat("Unexpected directories " + paths, + paths.size(), is(0)); + } + + /** + * Return true if the path was successfully deleted. False if it + * failed with AccessControlException. Any other exceptions are + * propagated to the caller. + * + * @param fs + * @param path + * @return + */ + private boolean deletePath(FileSystem fs, Path path) throws IOException { + try { + fs.delete(path, true); + return true; + } catch (AccessControlException ace) { + return false; + } + } + + private static class TestMatrixEntry { + // true if the path can be deleted. + final Map protectedPaths = Maps.newHashMap(); + final Map unProtectedPaths = Maps.newHashMap(); + + private TestMatrixEntry() { + } + + public static TestMatrixEntry get() { + return new TestMatrixEntry(); + } + + public Collection getProtectedPaths() { + return protectedPaths.keySet(); + } + + public Collection getUnprotectedPaths() { + return unProtectedPaths.keySet(); + } + + /** + * Get all paths to be deleted in sorted order. + * @return sorted collection of paths to be deleted. + */ + @SuppressWarnings("unchecked") // Path implements Comparable incorrectly + public Iterable getAllPathsToBeDeleted() { + // Sorting ensures deletion of parents is attempted first. + ArrayList combined = new ArrayList<>(); + combined.addAll(protectedPaths.keySet()); + combined.addAll(unProtectedPaths.keySet()); + Collections.sort(combined); + return combined; + } + + public boolean canPathBeDeleted(Path path) { + return protectedPaths.containsKey(path) ? + protectedPaths.get(path) : unProtectedPaths.get(path); + } + + + public TestMatrixEntry addProtectedDir(String dir, boolean canBeDeleted) { + protectedPaths.put(new Path(dir), canBeDeleted); + return this; + } + + public TestMatrixEntry addUnprotectedDir(String dir, boolean canBeDeleted) { + unProtectedPaths.put(new Path(dir), canBeDeleted); + return this; + } + + @Override + public String toString() { + return "TestMatrixEntry - ProtectedPaths=[" + + Joiner.on(", ").join(protectedPaths.keySet()) + + "]; UnprotectedPaths=[" + + Joiner.on(", ").join(unProtectedPaths.keySet()) + "]"; + } + } +} From 6ab2d19f5c010ab1d318214916ba95daa91a4dbf Mon Sep 17 00:00:00 2001 From: Lei Xu Date: Sat, 29 Aug 2015 09:57:27 -0700 Subject: [PATCH 05/18] HADOOP-12346. Increase some default timeouts / retries for S3a connector. (Sean Mackrory via Lei (Eddy) Xu) --- .../hadoop-common/src/main/resources/core-default.xml | 4 ++-- .../src/main/java/org/apache/hadoop/fs/s3a/Constants.java | 4 ++-- .../hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index cef32d3f89..b813aa982b 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -793,7 +793,7 @@ for ldap providers in the same way as above does. fs.s3a.attempts.maximum - 10 + 20 How many times we should retry commands on transient errors. @@ -805,7 +805,7 @@ for ldap providers in the same way as above does. fs.s3a.connection.timeout - 50000 + 200000 Socket connection timeout in milliseconds. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index fe8dd77b7b..fa81d93522 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -45,7 +45,7 @@ public class Constants { // number of times we should retry errors public static final String MAX_ERROR_RETRIES = "fs.s3a.attempts.maximum"; - public static final int DEFAULT_MAX_ERROR_RETRIES = 10; + public static final int DEFAULT_MAX_ERROR_RETRIES = 20; // seconds until we give up trying to establish a connection to s3 public static final String ESTABLISH_TIMEOUT = "fs.s3a.connection.establish.timeout"; @@ -53,7 +53,7 @@ public class Constants { // seconds until we give up on a connection to s3 public static final String SOCKET_TIMEOUT = "fs.s3a.connection.timeout"; - public static final int DEFAULT_SOCKET_TIMEOUT = 50000; + public static final int DEFAULT_SOCKET_TIMEOUT = 200000; // number of records to get while paging through a directory listing public static final String MAX_PAGING_KEYS = "fs.s3a.paging.maximum"; diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index 5d45e0ab64..6df15e68cf 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -206,7 +206,7 @@ If you do any of these: change your credentials immediately! fs.s3a.attempts.maximum - 10 + 20 How many times we should retry commands on transient errors. @@ -218,7 +218,7 @@ If you do any of these: change your credentials immediately! fs.s3a.connection.timeout - 50000 + 200000 Socket connection timeout in milliseconds. From 837fb75e8e03b2f016bcea2f4605106a5022491c Mon Sep 17 00:00:00 2001 From: Jakob Homan Date: Sat, 29 Aug 2015 18:37:05 -0700 Subject: [PATCH 06/18] HDFS-8155. Support OAuth2 in WebHDFS. --- .../hadoop-hdfs-client/pom.xml | 5 + .../hdfs/client/HdfsClientConfigKeys.java | 8 + .../hadoop/hdfs/web/URLConnectionFactory.java | 30 ++- .../hadoop/hdfs/web/WebHdfsFileSystem.java | 15 +- .../hdfs/web/oauth2/AccessTokenProvider.java | 66 ++++++ .../hdfs/web/oauth2/AccessTokenTimer.java | 103 +++++++++ ...onfCredentialBasedAccessTokenProvider.java | 62 +++++ ...fRefreshTokenBasedAccessTokenProvider.java | 146 ++++++++++++ .../CredentialBasedAccessTokenProvider.java | 135 +++++++++++ .../oauth2/OAuth2ConnectionConfigurator.java | 79 +++++++ .../hdfs/web/oauth2/OAuth2Constants.java | 46 ++++ .../apache/hadoop/hdfs/web/oauth2/Utils.java | 63 +++++ .../hadoop/hdfs/web/oauth2/package-info.java | 26 +++ hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 2 + hadoop-hdfs-project/hadoop-hdfs/pom.xml | 6 + .../hadoop-hdfs/src/site/markdown/WebHDFS.md | 25 ++ .../hadoop/hdfs/web/TestWebHDFSOAuth2.java | 216 ++++++++++++++++++ .../hdfs/web/oauth2/TestAccessTokenTimer.java | 63 +++++ ...ientCredentialTimeBasedTokenRefresher.java | 138 +++++++++++ ...stRefreshTokenTimeBasedTokenRefresher.java | 138 +++++++++++ 20 files changed, 1369 insertions(+), 3 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenProvider.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenTimer.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfCredentialBasedAccessTokenProvider.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2ConnectionConfigurator.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2Constants.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/Utils.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSOAuth2.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestAccessTokenTimer.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestClientCredentialTimeBasedTokenRefresher.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestRefreshTokenTimeBasedTokenRefresher.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml index aeaa98058c..68bd289e79 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml @@ -31,6 +31,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> jar + + com.squareup.okhttp + okhttp + 2.4.0 + org.apache.hadoop hadoop-common diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java index 7b1e4384b5..96bc8d3064 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java @@ -36,6 +36,14 @@ public interface HdfsClientConfigKeys { String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT = "^(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?)*$"; + String DFS_WEBHDFS_OAUTH_ENABLED_KEY = "dfs.webhdfs.oauth2.enabled"; + boolean DFS_WEBHDFS_OAUTH_ENABLED_DEFAULT = false; + + String OAUTH_CLIENT_ID_KEY = "dfs.webhdfs.oauth2.client.id"; + String OAUTH_REFRESH_URL_KEY = "dfs.webhdfs.oauth2.refresh.url"; + + String ACCESS_TOKEN_PROVIDER_KEY = "dfs.webhdfs.oauth2.access.token.provider"; + String PREFIX = "dfs.client."; String DFS_NAMESERVICES = "dfs.nameservices"; int DFS_NAMENODE_HTTP_PORT_DEFAULT = 50070; diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java index a5e02f234d..4c2324152a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java @@ -31,6 +31,7 @@ import javax.net.ssl.SSLSocketFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.web.oauth2.OAuth2ConnectionConfigurator; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authentication.client.AuthenticatedURL; import org.apache.hadoop.security.authentication.client.AuthenticationException; @@ -77,15 +78,42 @@ public class URLConnectionFactory { * try to load SSL certificates when it is specified. */ public static URLConnectionFactory newDefaultURLConnectionFactory(Configuration conf) { + ConnectionConfigurator conn = getSSLConnectionConfiguration(conf); + + return new URLConnectionFactory(conn); + } + + private static ConnectionConfigurator + getSSLConnectionConfiguration(Configuration conf) { ConnectionConfigurator conn = null; try { conn = newSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf); } catch (Exception e) { LOG.debug( - "Cannot load customized ssl related configuration. Fallback to system-generic settings.", + "Cannot load customized ssl related configuration. Fallback to" + + " system-generic settings.", e); conn = DEFAULT_TIMEOUT_CONN_CONFIGURATOR; } + + return conn; + } + + /** + * Construct a new URLConnectionFactory that supports OAut-based connections. + * It will also try to load the SSL configuration when they are specified. + */ + public static URLConnectionFactory + newOAuth2URLConnectionFactory(Configuration conf) throws IOException { + ConnectionConfigurator conn = null; + try { + ConnectionConfigurator sslConnConfigurator + = newSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf); + + conn = new OAuth2ConnectionConfigurator(conf, sslConnConfigurator); + } catch (Exception e) { + throw new IOException("Unable to load OAuth2 connection factory.", e); + } return new URLConnectionFactory(conn); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java index ee5238acf9..a75e78f98e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java @@ -149,8 +149,19 @@ public class WebHdfsFileSystem extends FileSystem HdfsClientConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY, HdfsClientConfigKeys.DFS_WEBHDFS_USER_PATTERN_DEFAULT)); - connectionFactory = URLConnectionFactory - .newDefaultURLConnectionFactory(conf); + boolean isOAuth = conf.getBoolean( + HdfsClientConfigKeys.DFS_WEBHDFS_OAUTH_ENABLED_KEY, + HdfsClientConfigKeys.DFS_WEBHDFS_OAUTH_ENABLED_DEFAULT); + + if(isOAuth) { + LOG.info("Enabling OAuth2 in WebHDFS"); + connectionFactory = URLConnectionFactory + .newOAuth2URLConnectionFactory(conf); + } else { + LOG.info("Not enabling OAuth2 in WebHDFS"); + connectionFactory = URLConnectionFactory + .newDefaultURLConnectionFactory(conf); + } ugi = UserGroupInformation.getCurrentUser(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenProvider.java new file mode 100644 index 0000000000..99e153d7b3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenProvider.java @@ -0,0 +1,66 @@ +/* + * 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.web.oauth2; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; + +import java.io.IOException; + +/** + * Provide an OAuth2 access token to be used to authenticate http calls in + * WebHDFS. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public abstract class AccessTokenProvider implements Configurable { + private Configuration conf; + + /** + * Obtain the access token that should be added to http connection's header. + * Will be called for each connection, so implementations should be + * performant. Implementations are responsible for any refreshing of + * the token. + * + * @return Access token to be added to connection header. + */ + abstract String getAccessToken() throws IOException; + + /** + * Return the conf. + * + * @return the conf. + */ + @Override + public Configuration getConf() { + return conf; + } + + /** + * Set the conf. + * + * @param configuration New configuration. + */ + @Override + public void setConf(Configuration configuration) { + this.conf = configuration; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenTimer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenTimer.java new file mode 100644 index 0000000000..aa05dd6db3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenTimer.java @@ -0,0 +1,103 @@ +/* + * 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.web.oauth2; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.util.Timer; + +/** + * Access tokens generally expire. This timer helps keep track of that. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class AccessTokenTimer { + public static final long EXPIRE_BUFFER_MS = 30 * 1000L; + + private final Timer timer; + + /** + * When the current access token will expire in milliseconds since + * epoch. + */ + private long nextRefreshMSSinceEpoch; + + public AccessTokenTimer() { + this(new Timer()); + } + + /** + * + * @param timer Timer instance for unit testing + */ + public AccessTokenTimer(Timer timer) { + this.timer = timer; + this.nextRefreshMSSinceEpoch = 0; + } + + /** + * Set when the access token will expire as reported by the oauth server, + * ie in seconds from now. + * @param expiresIn Access time expiration as reported by OAuth server + */ + public void setExpiresIn(String expiresIn) { + this.nextRefreshMSSinceEpoch = convertExpiresIn(timer, expiresIn); + } + + /** + * Set when the access token will expire in milliseconds from epoch, + * as required by the WebHDFS configuration. This is a bit hacky and lame. + * + * @param expiresInMSSinceEpoch Access time expiration in ms since epoch. + */ + public void setExpiresInMSSinceEpoch(String expiresInMSSinceEpoch){ + this.nextRefreshMSSinceEpoch = Long.parseLong(expiresInMSSinceEpoch); + } + + /** + * Get next time we should refresh the token. + * + * @return Next time since epoch we'll need to refresh the token. + */ + public long getNextRefreshMSSinceEpoch() { + return nextRefreshMSSinceEpoch; + } + + /** + * Return true if the current token has expired or will expire within the + * EXPIRE_BUFFER_MS (to give ample wiggle room for the call to be made to + * the server). + */ + public boolean shouldRefresh() { + long lowerLimit = nextRefreshMSSinceEpoch - EXPIRE_BUFFER_MS; + long currTime = timer.now(); + return currTime > lowerLimit; + } + + /** + * The expires_in param from OAuth is in seconds-from-now. Convert to + * milliseconds-from-epoch + */ + static Long convertExpiresIn(Timer timer, String expiresInSecs) { + long expiresSecs = Long.parseLong(expiresInSecs); + long expiresMs = expiresSecs * 1000; + return timer.now() + expiresMs; + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfCredentialBasedAccessTokenProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfCredentialBasedAccessTokenProvider.java new file mode 100644 index 0000000000..b56dbde1bf --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfCredentialBasedAccessTokenProvider.java @@ -0,0 +1,62 @@ +/* + * 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.web.oauth2; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.Timer; + +import static org.apache.hadoop.hdfs.web.oauth2.Utils.notNull; + +/** + * Obtain an access token via a a credential (provided through the + * Configuration) using the + * + * Client Credentials Grant workflow. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class ConfCredentialBasedAccessTokenProvider + extends CredentialBasedAccessTokenProvider { + private String credential; + + public ConfCredentialBasedAccessTokenProvider() { + } + + public ConfCredentialBasedAccessTokenProvider(Timer timer) { + super(timer); + } + + @Override + public void setConf(Configuration conf) { + super.setConf(conf); + credential = notNull(conf, OAUTH_CREDENTIAL_KEY); + } + + @Override + public String getCredential() { + if(credential == null) { + throw new IllegalArgumentException("Credential has not been " + + "provided in configuration"); + } + + return credential; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java new file mode 100644 index 0000000000..1e80451fb9 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java @@ -0,0 +1,146 @@ +/* + * 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.web.oauth2; + +import com.squareup.okhttp.OkHttpClient; +import com.squareup.okhttp.Request; +import com.squareup.okhttp.RequestBody; +import com.squareup.okhttp.Response; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.web.URLConnectionFactory; +import org.apache.hadoop.util.Timer; +import org.apache.http.HttpStatus; +import org.codehaus.jackson.map.ObjectMapper; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY; +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.ACCESS_TOKEN; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_ID; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.EXPIRES_IN; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.GRANT_TYPE; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.REFRESH_TOKEN; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.URLENCODED; +import static org.apache.hadoop.hdfs.web.oauth2.Utils.notNull; + +/** + * Supply a access token obtained via a refresh token (provided through the + * Configuration using the second half of the + * + * Authorization Code Grant workflow. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class ConfRefreshTokenBasedAccessTokenProvider + extends AccessTokenProvider { + + public static final String OAUTH_REFRESH_TOKEN_KEY + = "dfs.webhdfs.oauth2.refresh.token"; + public static final String OAUTH_REFRESH_TOKEN_EXPIRES_KEY + = "dfs.webhdfs.oauth2.refresh.token.expires.ms.since.epoch"; + + private AccessTokenTimer accessTokenTimer; + + private String accessToken; + + private String refreshToken; + + private String clientId; + + private String refreshURL; + + + public ConfRefreshTokenBasedAccessTokenProvider() { + this.accessTokenTimer = new AccessTokenTimer(); + } + + public ConfRefreshTokenBasedAccessTokenProvider(Timer timer) { + this.accessTokenTimer = new AccessTokenTimer(timer); + } + + @Override + public void setConf(Configuration conf) { + super.setConf(conf); + refreshToken = notNull(conf, (OAUTH_REFRESH_TOKEN_KEY)); + + accessTokenTimer.setExpiresInMSSinceEpoch( + notNull(conf, OAUTH_REFRESH_TOKEN_EXPIRES_KEY)); + + clientId = notNull(conf, OAUTH_CLIENT_ID_KEY); + refreshURL = notNull(conf, OAUTH_REFRESH_URL_KEY); + + } + + @Override + public synchronized String getAccessToken() throws IOException { + if(accessTokenTimer.shouldRefresh()) { + refresh(); + } + + return accessToken; + } + + void refresh() throws IOException { + try { + OkHttpClient client = new OkHttpClient(); + client.setConnectTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT, + TimeUnit.MILLISECONDS); + client.setReadTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT, + TimeUnit.MILLISECONDS); + + String bodyString = Utils.postBody(GRANT_TYPE, REFRESH_TOKEN, + REFRESH_TOKEN, refreshToken, + CLIENT_ID, clientId); + + RequestBody body = RequestBody.create(URLENCODED, bodyString); + + Request request = new Request.Builder() + .url(refreshURL) + .post(body) + .build(); + Response responseBody = client.newCall(request).execute(); + + if (responseBody.code() != HttpStatus.SC_OK) { + throw new IllegalArgumentException("Received invalid http response: " + + responseBody.code() + ", text = " + responseBody.toString()); + } + + ObjectMapper mapper = new ObjectMapper(); + Map response = mapper.reader(Map.class) + .readValue(responseBody.body().string()); + + + String newExpiresIn = response.get(EXPIRES_IN).toString(); + accessTokenTimer.setExpiresIn(newExpiresIn); + + accessToken = response.get(ACCESS_TOKEN).toString(); + } catch (Exception e) { + throw new IOException("Exception while refreshing access token", e); + } + } + + public String getRefreshToken() { + return refreshToken; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java new file mode 100644 index 0000000000..c058e05b93 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java @@ -0,0 +1,135 @@ +/* + * 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.web.oauth2; + +import com.squareup.okhttp.OkHttpClient; +import com.squareup.okhttp.Request; +import com.squareup.okhttp.RequestBody; +import com.squareup.okhttp.Response; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.web.URLConnectionFactory; +import org.apache.hadoop.util.Timer; +import org.apache.http.HttpStatus; +import org.codehaus.jackson.map.ObjectMapper; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY; +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.ACCESS_TOKEN; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_CREDENTIALS; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_ID; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_SECRET; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.EXPIRES_IN; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.GRANT_TYPE; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.URLENCODED; +import static org.apache.hadoop.hdfs.web.oauth2.Utils.notNull; + +/** + * Obtain an access token via the credential-based OAuth2 workflow. This + * abstract class requires only that implementations provide the credential, + * which the class then uses to obtain a refresh token. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public abstract class CredentialBasedAccessTokenProvider + extends AccessTokenProvider { + public static final String OAUTH_CREDENTIAL_KEY + = "dfs.webhdfs.oauth2.credential"; + + private AccessTokenTimer timer; + + private String clientId; + + private String refreshURL; + + private String accessToken; + + private boolean initialCredentialObtained = false; + + CredentialBasedAccessTokenProvider() { + this.timer = new AccessTokenTimer(); + } + + CredentialBasedAccessTokenProvider(Timer timer) { + this.timer = new AccessTokenTimer(timer); + } + + abstract String getCredential(); + + @Override + public void setConf(Configuration conf) { + super.setConf(conf); + clientId = notNull(conf, OAUTH_CLIENT_ID_KEY); + refreshURL = notNull(conf, OAUTH_REFRESH_URL_KEY); + } + + @Override + public synchronized String getAccessToken() throws IOException { + if(timer.shouldRefresh() || !initialCredentialObtained) { + refresh(); + initialCredentialObtained = true; + } + + return accessToken; + } + + void refresh() throws IOException { + try { + OkHttpClient client = new OkHttpClient(); + client.setConnectTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT, + TimeUnit.MILLISECONDS); + client.setReadTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT, + TimeUnit.MILLISECONDS); + + String bodyString = Utils.postBody(CLIENT_SECRET, getCredential(), + GRANT_TYPE, CLIENT_CREDENTIALS, + CLIENT_ID, clientId); + + RequestBody body = RequestBody.create(URLENCODED, bodyString); + + Request request = new Request.Builder() + .url(refreshURL) + .post(body) + .build(); + Response responseBody = client.newCall(request).execute(); + + if (responseBody.code() != HttpStatus.SC_OK) { + throw new IllegalArgumentException("Received invalid http response: " + + responseBody.code() + ", text = " + responseBody.toString()); + } + + ObjectMapper mapper = new ObjectMapper(); + Map response = mapper.reader(Map.class) + .readValue(responseBody.body().string()); + + String newExpiresIn = response.get(EXPIRES_IN).toString(); + timer.setExpiresIn(newExpiresIn); + + accessToken = response.get(ACCESS_TOKEN).toString(); + + } catch (Exception e) { + throw new IOException("Unable to obtain access token from credential", e); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2ConnectionConfigurator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2ConnectionConfigurator.java new file mode 100644 index 0000000000..f334b2431f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2ConnectionConfigurator.java @@ -0,0 +1,79 @@ +/* + * 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.web.oauth2; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.authentication.client.ConnectionConfigurator; +import org.apache.hadoop.util.ReflectionUtils; + +import java.io.IOException; +import java.net.HttpURLConnection; + +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.ACCESS_TOKEN_PROVIDER_KEY; +import static org.apache.hadoop.hdfs.web.oauth2.Utils.notNull; + +/** + * Configure a connection to use OAuth2 authentication. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class OAuth2ConnectionConfigurator implements ConnectionConfigurator { + + public static final String HEADER = "Bearer "; + + private final AccessTokenProvider accessTokenProvider; + + private ConnectionConfigurator sslConfigurator = null; + + public OAuth2ConnectionConfigurator(Configuration conf) { + this(conf, null); + } + + @SuppressWarnings("unchecked") + public OAuth2ConnectionConfigurator(Configuration conf, + ConnectionConfigurator sslConfigurator) { + this.sslConfigurator = sslConfigurator; + + notNull(conf, ACCESS_TOKEN_PROVIDER_KEY); + + Class accessTokenProviderClass = conf.getClass(ACCESS_TOKEN_PROVIDER_KEY, + ConfCredentialBasedAccessTokenProvider.class, + AccessTokenProvider.class); + + accessTokenProvider = (AccessTokenProvider) ReflectionUtils + .newInstance(accessTokenProviderClass, conf); + accessTokenProvider.setConf(conf); + } + + @Override + public HttpURLConnection configure(HttpURLConnection conn) + throws IOException { + if(sslConfigurator != null) { + sslConfigurator.configure(conn); + } + + String accessToken = accessTokenProvider.getAccessToken(); + + conn.setRequestProperty("AUTHORIZATION", HEADER + accessToken); + + return conn; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2Constants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2Constants.java new file mode 100644 index 0000000000..190a1f5b6a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2Constants.java @@ -0,0 +1,46 @@ +/* + * 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.web.oauth2; + +import com.squareup.okhttp.MediaType; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Sundry constants relating to OAuth2 within WebHDFS. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class OAuth2Constants { + private OAuth2Constants() { /** Private constructor. **/ } + + public static final MediaType URLENCODED + = MediaType.parse("application/x-www-form-urlencoded; charset=utf-8"); + + /* Constants for OAuth protocol */ + public static final String ACCESS_TOKEN = "access_token"; + public static final String BEARER = "bearer"; + public static final String CLIENT_CREDENTIALS = "client_credentials"; + public static final String CLIENT_ID = "client_id"; + public static final String CLIENT_SECRET = "client_secret"; + public static final String EXPIRES_IN = "expires_in"; + public static final String GRANT_TYPE = "grant_type"; + public static final String REFRESH_TOKEN = "refresh_token"; + public static final String TOKEN_TYPE = "token_type"; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/Utils.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/Utils.java new file mode 100644 index 0000000000..939798db16 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/Utils.java @@ -0,0 +1,63 @@ +/* + * 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.web.oauth2; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; + +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; + +@InterfaceAudience.Private +@InterfaceStability.Evolving +final class Utils { + private Utils() { /* Private constructor */ } + + public static String notNull(Configuration conf, String key) { + String value = conf.get(key); + + if(value == null) { + throw new IllegalArgumentException("No value for " + key + + " found in conf file."); + } + + return value; + } + + public static String postBody(String ... kv) + throws UnsupportedEncodingException { + if(kv.length % 2 != 0) { + throw new IllegalArgumentException("Arguments must be key value pairs"); + } + StringBuilder sb = new StringBuilder(); + int i = 0; + + while(i < kv.length) { + if(i > 0) { + sb.append("&"); + } + sb.append(URLEncoder.encode(kv[i++], "UTF-8")); + sb.append("="); + sb.append(URLEncoder.encode(kv[i++], "UTF-8")); + } + + return sb.toString(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java new file mode 100644 index 0000000000..aeb581fd1e --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * OAuth2-based WebHDFS authentication. + */ +@InterfaceAudience.Public +package org.apache.hadoop.hdfs.web.oauth2; + +import org.apache.hadoop.classification.InterfaceAudience; \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 6f46ea5453..3382f81a20 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -357,6 +357,8 @@ Release 2.8.0 - UNRELEASED HDFS-8131. Implement a space balanced block placement policy (Liu Shaohui via kihwal) + HDFS-8155. Support OAuth2 in WebHDFS. (jghoman) + IMPROVEMENTS HDFS-2390. dfsadmin -setBalancerBandwidth does not validate -ve value diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml index 1a29ad3d8c..d0c2dc7d59 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml @@ -213,6 +213,12 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> leveldbjni-all 1.8 + + org.mock-server + mockserver-netty + 3.9.2 + test + org.bouncycastle diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md index 20b9d731ca..d0a0fe08c7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md @@ -221,6 +221,31 @@ Below are examples using the `curl` command tool. See also: [Authentication for Hadoop HTTP web-consoles](../hadoop-common/HttpAuthentication.html) +Additionally, WebHDFS supports OAuth2 on the client side. The Namenode and Datanodes do not currently support clients using OAuth2 but other backends that implement the WebHDFS REST interface may. + +WebHDFS supports two type of OAuth2 code grants (user-provided refresh and access token or user provided credential) by default and provides a pluggable mechanism for implementing other OAuth2 authentications per the [OAuth2 RFC](https://tools.ietf.org/html/rfc6749), or custom authentications. When using either of the provided code grant mechanisms, the WebHDFS client will refresh the access token as necessary. + +OAuth2 should only be enabled for clients not running with Kerberos SPENGO. + +| OAuth2 code grant mechanism | Description | Value of `dfs.webhdfs.oauth2.access.token.provider` that implements code grant | +|:---- |:---- |:----| +| Authorization Code Grant | The user provides an initial access token and refresh token, which are then used to authenticate WebHDFS requests and obtain replacement access tokens, respectively. | org.apache.hadoop.hdfs.web.oauth2.ConfRefreshTokenBasedAccessTokenProvider | +| Client Credentials Grant | The user provides a credential which is used to obtain access tokens, which are then used to authenticate WebHDFS requests. | org.apache.hadoop.hdfs.web.oauth2.ConfCredentialBasedAccessTokenProvider | + + +The following properties control OAuth2 authentication. + +| OAuth2 related property | Description | +|:---- |:---- | +| `dfs.webhdfs.oauth2.enabled` | Boolean to enable/disable OAuth2 authentication | +| `dfs.webhdfs.oauth2.access.token.provider` | Class name of an implementation of `org.apache.hadoop.hdfs.web.oauth.AccessTokenProvider.` Two are provided with the code, as described above, or the user may specify a user-provided implementation. The default value for this configuration key is the `ConfCredentialBasedAccessTokenProvider` implementation. | +| `dfs.webhdfs.oauth2.client.id` | Client id used to obtain access token with either credential or refresh token | +| `dfs.webhdfs.oauth2.refresh.url` | URL against which to post for obtaining bearer token with either credential or refresh token | +| `dfs.webhdfs.oauth2.access.token` | (required if using ConfRefreshTokenBasedAccessTokenProvider) Initial access token with which to authenticate | +| `dfs.webhdfs.oauth2.refresh.token` | (required if using ConfRefreshTokenBasedAccessTokenProvider) Initial refresh token to use to obtain new access tokens | +| `dfs.webhdfs.oauth2.refresh.token.expires.ms.since.epoch` | (required if using ConfRefreshTokenBasedAccessTokenProvider) Access token expiration measured in milliseconds since Jan 1, 1970. *Note this is a different value than provided by OAuth providers and has been munged as described in interface to be suitable for a client application* | +| `dfs.webhdfs.oauth2.credential` | (required if using ConfCredentialBasedAccessTokenProvider). Credential used to obtain initial and subsequent access tokens. | + Proxy Users ----------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSOAuth2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSOAuth2.java new file mode 100644 index 0000000000..e2f6230ef2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSOAuth2.java @@ -0,0 +1,216 @@ +/* + * 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.web; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +import org.apache.hadoop.hdfs.web.oauth2.ConfCredentialBasedAccessTokenProvider; +import org.apache.hadoop.hdfs.web.oauth2.CredentialBasedAccessTokenProvider; +import org.apache.hadoop.hdfs.web.oauth2.OAuth2ConnectionConfigurator; +import org.apache.http.HttpStatus; +import org.codehaus.jackson.map.ObjectMapper; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockserver.client.server.MockServerClient; +import org.mockserver.integration.ClientAndServer; +import org.mockserver.model.Header; +import org.mockserver.model.HttpRequest; +import org.mockserver.model.HttpResponse; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Map; +import java.util.TreeMap; + +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.ACCESS_TOKEN_PROVIDER_KEY; +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY; +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.ACCESS_TOKEN; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.EXPIRES_IN; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.TOKEN_TYPE; +import static org.junit.Assert.assertEquals; +import static org.mockserver.integration.ClientAndServer.startClientAndServer; +import static org.mockserver.matchers.Times.exactly; +import static org.mockserver.model.HttpRequest.request; +import static org.mockserver.model.HttpResponse.response; + +public class TestWebHDFSOAuth2 { + public static final Log LOG = LogFactory.getLog(TestWebHDFSOAuth2.class); + + private ClientAndServer mockWebHDFS; + private ClientAndServer mockOAuthServer; + + public final static int WEBHDFS_PORT = 7552; + public final static int OAUTH_PORT = 7553; + + public final static Header CONTENT_TYPE_APPLICATION_JSON = new Header("Content-Type", "application/json"); + + public final static String AUTH_TOKEN = "0123456789abcdef"; + public final static Header AUTH_TOKEN_HEADER = new Header("AUTHORIZATION", OAuth2ConnectionConfigurator.HEADER + AUTH_TOKEN); + + @Before + public void startMockOAuthServer() { + mockOAuthServer = startClientAndServer(OAUTH_PORT); + } + @Before + public void startMockWebHDFSServer() { + System.setProperty("hadoop.home.dir", System.getProperty("user.dir")); + + mockWebHDFS = startClientAndServer(WEBHDFS_PORT); + } + + @Test + public void listStatusReturnsAsExpected() throws URISyntaxException, IOException { + MockServerClient mockWebHDFSServerClient = new MockServerClient("localhost", WEBHDFS_PORT); + MockServerClient mockOAuthServerClient = new MockServerClient("localhost", OAUTH_PORT); + + HttpRequest oauthServerRequest = getOAuthServerMockRequest(mockOAuthServerClient); + + HttpRequest fileSystemRequest = request() + .withMethod("GET") + .withPath(WebHdfsFileSystem.PATH_PREFIX + "/test1/test2") + .withHeader(AUTH_TOKEN_HEADER); + + try { + mockWebHDFSServerClient.when(fileSystemRequest, + exactly(1) + ) + .respond( + response() + .withStatusCode(HttpStatus.SC_OK) + .withHeaders( + CONTENT_TYPE_APPLICATION_JSON + ) + .withBody("{\n" + + " \"FileStatuses\":\n" + + " {\n" + + " \"FileStatus\":\n" + + " [\n" + + " {\n" + + " \"accessTime\" : 1320171722771,\n" + + " \"blockSize\" : 33554432,\n" + + " \"group\" : \"supergroup\",\n" + + " \"length\" : 24930,\n" + + " \"modificationTime\": 1320171722771,\n" + + " \"owner\" : \"webuser\",\n" + + " \"pathSuffix\" : \"a.patch\",\n" + + " \"permission\" : \"644\",\n" + + " \"replication\" : 1,\n" + + " \"type\" : \"FILE\"\n" + + " },\n" + + " {\n" + + " \"accessTime\" : 0,\n" + + " \"blockSize\" : 0,\n" + + " \"group\" : \"supergroup\",\n" + + " \"length\" : 0,\n" + + " \"modificationTime\": 1320895981256,\n" + + " \"owner\" : \"szetszwo\",\n" + + " \"pathSuffix\" : \"bar\",\n" + + " \"permission\" : \"711\",\n" + + " \"replication\" : 0,\n" + + " \"type\" : \"DIRECTORY\"\n" + + " }\n" + + " ]\n" + + " }\n" + + "}\n") + ); + + FileSystem fs = new WebHdfsFileSystem(); + Configuration conf = getConfiguration(); + conf.set(OAUTH_REFRESH_URL_KEY, "http://localhost:" + OAUTH_PORT + "/refresh"); + conf.set(CredentialBasedAccessTokenProvider.OAUTH_CREDENTIAL_KEY, "credential"); + + URI uri = new URI("webhdfs://localhost:" + WEBHDFS_PORT); + fs.initialize(uri, conf); + + FileStatus[] ls = fs.listStatus(new Path("/test1/test2")); + + mockOAuthServer.verify(oauthServerRequest); + mockWebHDFSServerClient.verify(fileSystemRequest); + + assertEquals(2, ls.length); + assertEquals("a.patch", ls[0].getPath().getName()); + assertEquals("bar", ls[1].getPath().getName()); + + fs.close(); + } finally { + mockWebHDFSServerClient.clear(fileSystemRequest); + mockOAuthServerClient.clear(oauthServerRequest); + } + } + + private HttpRequest getOAuthServerMockRequest(MockServerClient mockServerClient) throws IOException { + HttpRequest expectedRequest = request() + .withMethod("POST") + .withPath("/refresh") + .withBody("client_secret=credential&grant_type=client_credentials&client_id=MY_CLIENTID"); + + Map map = new TreeMap<>(); + + map.put(EXPIRES_IN, "0987654321"); + map.put(TOKEN_TYPE, "bearer"); + map.put(ACCESS_TOKEN, AUTH_TOKEN); + + ObjectMapper mapper = new ObjectMapper(); + + HttpResponse resp = response() + .withStatusCode(HttpStatus.SC_OK) + .withHeaders( + CONTENT_TYPE_APPLICATION_JSON + ) + .withBody(mapper.writeValueAsString(map)); + + mockServerClient + .when(expectedRequest, exactly(1)) + .respond(resp); + + return expectedRequest; + } + + public Configuration getConfiguration() { + Configuration conf = new Configuration(); + + // Configs for OAuth2 + conf.setBoolean(HdfsClientConfigKeys.DFS_WEBHDFS_OAUTH_ENABLED_KEY, true); + conf.set(OAUTH_CLIENT_ID_KEY, "MY_CLIENTID"); + + conf.set(ACCESS_TOKEN_PROVIDER_KEY, + ConfCredentialBasedAccessTokenProvider.class.getName()); + + return conf; + + } + + @After + public void stopMockWebHDFSServer() { + mockWebHDFS.stop(); + } + + @After + public void stopMockOAuthServer() { + mockOAuthServer.stop(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestAccessTokenTimer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestAccessTokenTimer.java new file mode 100644 index 0000000000..c387b1ebce --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestAccessTokenTimer.java @@ -0,0 +1,63 @@ +/* + * 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.web.oauth2; + +import org.apache.hadoop.util.Timer; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class TestAccessTokenTimer { + @Test + public void expireConversionWorks() { + Timer mockTimer = mock(Timer.class); + when(mockTimer.now()) + .thenReturn(5l); + + AccessTokenTimer timer = new AccessTokenTimer(mockTimer); + + timer.setExpiresIn("3"); + assertEquals(3005, timer.getNextRefreshMSSinceEpoch()); + + assertTrue(timer.shouldRefresh()); + } + + @Test + public void shouldRefreshIsCorrect() { + Timer mockTimer = mock(Timer.class); + when(mockTimer.now()) + .thenReturn(500l) + .thenReturn(1000000l + 500l); + + AccessTokenTimer timer = new AccessTokenTimer(mockTimer); + + timer.setExpiresInMSSinceEpoch("1000000"); + + assertFalse(timer.shouldRefresh()); + assertTrue(timer.shouldRefresh()); + + verify(mockTimer, times(2)).now(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestClientCredentialTimeBasedTokenRefresher.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestClientCredentialTimeBasedTokenRefresher.java new file mode 100644 index 0000000000..c259b30aae --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestClientCredentialTimeBasedTokenRefresher.java @@ -0,0 +1,138 @@ +/* + * 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.web.oauth2; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.Timer; +import org.apache.http.HttpStatus; +import org.codehaus.jackson.map.ObjectMapper; +import org.junit.Test; +import org.mockserver.client.server.MockServerClient; +import org.mockserver.integration.ClientAndServer; +import org.mockserver.model.Header; +import org.mockserver.model.HttpRequest; +import org.mockserver.model.HttpResponse; +import org.mockserver.model.Parameter; +import org.mockserver.model.ParameterBody; + +import java.io.IOException; +import java.util.Map; +import java.util.TreeMap; + +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.ACCESS_TOKEN_PROVIDER_KEY; +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY; +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.ACCESS_TOKEN; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_CREDENTIALS; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_ID; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_SECRET; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.EXPIRES_IN; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.GRANT_TYPE; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.TOKEN_TYPE; +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockserver.integration.ClientAndServer.startClientAndServer; +import static org.mockserver.matchers.Times.exactly; +import static org.mockserver.model.HttpRequest.request; +import static org.mockserver.model.HttpResponse.response; + +public class TestClientCredentialTimeBasedTokenRefresher { + public final static Header CONTENT_TYPE_APPLICATION_JSON + = new Header("Content-Type", "application/json"); + + public final static String CLIENT_ID_FOR_TESTING = "joebob"; + + public Configuration buildConf(String credential, String tokenExpires, + String clientId, String refreshURL) { + // Configurations are simple enough that it's not worth mocking them out. + Configuration conf = new Configuration(); + conf.set(CredentialBasedAccessTokenProvider.OAUTH_CREDENTIAL_KEY, + credential); + conf.set(ACCESS_TOKEN_PROVIDER_KEY, + ConfCredentialBasedAccessTokenProvider.class.getName()); + conf.set(OAUTH_CLIENT_ID_KEY, clientId); + conf.set(OAUTH_REFRESH_URL_KEY, refreshURL); + return conf; + } + + @Test + public void refreshUrlIsCorrect() throws IOException { + final int PORT = 7552; + final String REFRESH_ADDRESS = "http://localhost:" + PORT + "/refresh"; + + long tokenExpires = 0; + + Configuration conf = buildConf("myreallycoolcredential", + Long.toString(tokenExpires), + CLIENT_ID_FOR_TESTING, + REFRESH_ADDRESS); + + Timer mockTimer = mock(Timer.class); + when(mockTimer.now()).thenReturn(tokenExpires + 1000l); + + AccessTokenProvider credProvider = + new ConfCredentialBasedAccessTokenProvider(mockTimer); + credProvider.setConf(conf); + + // Build mock server to receive refresh request + ClientAndServer mockServer = startClientAndServer(PORT); + + HttpRequest expectedRequest = request() + .withMethod("POST") + .withPath("/refresh") + .withBody( + // Note, OkHttp does not sort the param values, so we need to do + // it ourselves via the ordering provided to ParameterBody... + ParameterBody.params( + Parameter.param(CLIENT_SECRET, "myreallycoolcredential"), + Parameter.param(GRANT_TYPE, CLIENT_CREDENTIALS), + Parameter.param(CLIENT_ID, CLIENT_ID_FOR_TESTING) + )); + + MockServerClient mockServerClient = new MockServerClient("localhost", PORT); + + // https://tools.ietf.org/html/rfc6749#section-5.1 + Map map = new TreeMap<>(); + + map.put(EXPIRES_IN, "0987654321"); + map.put(TOKEN_TYPE, "bearer"); + map.put(ACCESS_TOKEN, "new access token"); + + ObjectMapper mapper = new ObjectMapper(); + + HttpResponse resp = response() + .withStatusCode(HttpStatus.SC_OK) + .withHeaders( + CONTENT_TYPE_APPLICATION_JSON + ) + .withBody(mapper.writeValueAsString(map)); + + mockServerClient + .when(expectedRequest, exactly(1)) + .respond(resp); + + assertEquals("new access token", credProvider.getAccessToken()); + + mockServerClient.verify(expectedRequest); + + mockServerClient.clear(expectedRequest); + mockServer.stop(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestRefreshTokenTimeBasedTokenRefresher.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestRefreshTokenTimeBasedTokenRefresher.java new file mode 100644 index 0000000000..889ad0e9f5 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestRefreshTokenTimeBasedTokenRefresher.java @@ -0,0 +1,138 @@ +/* + * 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.web.oauth2; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.Timer; +import org.apache.http.HttpStatus; +import org.codehaus.jackson.map.ObjectMapper; +import org.junit.Test; +import org.mockserver.client.server.MockServerClient; +import org.mockserver.integration.ClientAndServer; +import org.mockserver.model.Header; +import org.mockserver.model.HttpRequest; +import org.mockserver.model.HttpResponse; +import org.mockserver.model.Parameter; +import org.mockserver.model.ParameterBody; + +import java.io.IOException; +import java.util.Map; +import java.util.TreeMap; + +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY; +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY; +import static org.apache.hadoop.hdfs.web.oauth2.ConfRefreshTokenBasedAccessTokenProvider.OAUTH_REFRESH_TOKEN_EXPIRES_KEY; +import static org.apache.hadoop.hdfs.web.oauth2.ConfRefreshTokenBasedAccessTokenProvider.OAUTH_REFRESH_TOKEN_KEY; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.ACCESS_TOKEN; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.BEARER; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_ID; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.EXPIRES_IN; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.GRANT_TYPE; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.REFRESH_TOKEN; +import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.TOKEN_TYPE; +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockserver.integration.ClientAndServer.startClientAndServer; +import static org.mockserver.matchers.Times.exactly; +import static org.mockserver.model.HttpRequest.request; +import static org.mockserver.model.HttpResponse.response; + +public class TestRefreshTokenTimeBasedTokenRefresher { + + public final static Header CONTENT_TYPE_APPLICATION_JSON + = new Header("Content-Type", "application/json"); + + public Configuration buildConf(String refreshToken, String tokenExpires, + String clientId, String refreshURL) { + // Configurations are simple enough that it's not worth mocking them out. + Configuration conf = new Configuration(); + conf.set(OAUTH_REFRESH_TOKEN_KEY, refreshToken); + conf.set(OAUTH_REFRESH_TOKEN_EXPIRES_KEY, tokenExpires); + conf.set(OAUTH_CLIENT_ID_KEY, clientId); + conf.set(OAUTH_REFRESH_URL_KEY, refreshURL); + + return conf; + } + + @Test + public void refreshUrlIsCorrect() throws IOException { + final int PORT = 7552; + final String REFRESH_ADDRESS = "http://localhost:" + PORT + "/refresh"; + + long tokenExpires = 0; + + Configuration conf = buildConf("refresh token key", + Long.toString(tokenExpires), + "joebob", + REFRESH_ADDRESS); + + Timer mockTimer = mock(Timer.class); + when(mockTimer.now()).thenReturn(tokenExpires + 1000l); + + AccessTokenProvider tokenProvider = + new ConfRefreshTokenBasedAccessTokenProvider(mockTimer); + tokenProvider.setConf(conf); + + // Build mock server to receive refresh request + + ClientAndServer mockServer = startClientAndServer(PORT); + + HttpRequest expectedRequest = request() + .withMethod("POST") + .withPath("/refresh") + // Note, OkHttp does not sort the param values, so we need to + // do it ourselves via the ordering provided to ParameterBody... + .withBody( + ParameterBody.params( + Parameter.param(CLIENT_ID, "joebob"), + Parameter.param(GRANT_TYPE, REFRESH_TOKEN), + Parameter.param(REFRESH_TOKEN, "refresh token key"))); + + MockServerClient mockServerClient = new MockServerClient("localhost", PORT); + + // https://tools.ietf.org/html/rfc6749#section-5.1 + Map map = new TreeMap<>(); + + map.put(EXPIRES_IN, "0987654321"); + map.put(TOKEN_TYPE, BEARER); + map.put(ACCESS_TOKEN, "new access token"); + + ObjectMapper mapper = new ObjectMapper(); + + HttpResponse resp = response() + .withStatusCode(HttpStatus.SC_OK) + .withHeaders( + CONTENT_TYPE_APPLICATION_JSON + ) + .withBody(mapper.writeValueAsString(map)); + + mockServerClient + .when(expectedRequest, exactly(1)) + .respond(resp); + + assertEquals("new access token", tokenProvider.getAccessToken()); + + mockServerClient.verify(expectedRequest); + + mockServerClient.clear(expectedRequest); + mockServer.stop(); + } + +} From cf831565e8344523e1bd0eaf686ed56a2b48b920 Mon Sep 17 00:00:00 2001 From: Vinod Kumar Vavilapalli Date: Sun, 30 Aug 2015 20:01:47 -0700 Subject: [PATCH 07/18] YARN-2945. Fixing the CHANGES.txt to have the right JIRA number. --- hadoop-yarn-project/CHANGES.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index 0b733a4a40..4201b4f59e 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -1453,7 +1453,7 @@ Release 2.7.0 - 2015-04-20 YARN-2914. [YARN-1492] Potential race condition in Singleton implementation of SharedCacheUploaderMetrics, CleanerMetrics, ClientSCMMetrics. (Varun Saxena via kasha) - YARN-2964. FSLeafQueue#assignContainer - document the reason for using both write and + YARN-2945. FSLeafQueue#assignContainer - document the reason for using both write and read locks. (Tsuyoshi Ozawa via kasha) YARN-2944. InMemorySCMStore can not be instantiated with ReflectionUtils#newInstance. From caa04de149030691b7bc952b534c6128db217ed2 Mon Sep 17 00:00:00 2001 From: Jing Zhao Date: Mon, 31 Aug 2015 11:48:09 -0700 Subject: [PATCH 08/18] HDFS-8980. Remove unnecessary block replacement in INodeFile. Contributed by Jing Zhao. --- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 2 + .../server/blockmanagement/BlockInfo.java | 19 +----- .../blockmanagement/BlockInfoContiguous.java | 15 ----- .../server/blockmanagement/BlockManager.java | 58 +++++++------------ .../server/blockmanagement/BlocksMap.java | 16 ----- .../hdfs/server/namenode/FSEditLogLoader.java | 8 +-- 6 files changed, 29 insertions(+), 89 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 3382f81a20..7b5979eae0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -865,6 +865,8 @@ Release 2.8.0 - UNRELEASED HDFS-8983. NameNode support for protected directories. (Arpit Agarwal) + HDFS-8980. Remove unnecessary block replacement in INodeFile. (jing9) + OPTIMIZATIONS HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java index 706cbcdb3b..810784dbe2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java @@ -36,7 +36,7 @@ import static org.apache.hadoop.hdfs.server.namenode.INodeId.INVALID_INODE_ID; * the block are stored. */ @InterfaceAudience.Private -public abstract class BlockInfo extends Block +public abstract class BlockInfo extends Block implements LightWeightGSet.LinkedElement { public static final BlockInfo[] EMPTY_ARRAY = {}; @@ -206,12 +206,6 @@ public abstract class BlockInfo extends Block */ abstract boolean removeStorage(DatanodeStorageInfo storage); - /** - * Replace the current BlockInfo with the new one in corresponding - * DatanodeStorageInfo's linked list - */ - abstract void replaceBlock(BlockInfo newBlock); - /** * Find specified DatanodeStorageInfo. * @return DatanodeStorageInfo or null if not found. @@ -372,19 +366,12 @@ public abstract class BlockInfo extends Block } /** - * Convert an under construction block to a complete block. - * - * @return BlockInfo - a complete block. - * @throws IOException if the state of the block - * (the generation stamp and the length) has not been committed by - * the client or it does not have at least a minimal number of replicas - * reported from data-nodes. + * Convert an under construction block to complete. */ - BlockInfo convertToCompleteBlock() throws IOException { + void convertToCompleteBlock() { assert getBlockUCState() != BlockUCState.COMPLETE : "Trying to convert a COMPLETE block"; uc = null; - return this; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java index 42934c3abb..94fb222068 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java @@ -104,19 +104,4 @@ public class BlockInfoContiguous extends BlockInfo { } return 0; } - - @Override - void replaceBlock(BlockInfo newBlock) { - assert newBlock instanceof BlockInfoContiguous; - for (int i = this.numNodes() - 1; i >= 0; i--) { - final DatanodeStorageInfo storage = this.getStorageInfo(i); - final boolean removed = storage.removeBlock(this); - assert removed : "currentBlock not found."; - - final DatanodeStorageInfo.AddBlockResult result = storage.addBlock( - newBlock); - assert result == DatanodeStorageInfo.AddBlockResult.ADDED : - "newBlock already exists."; - } - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index 8f7bb55b09..1346ab388a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -648,37 +648,34 @@ public class BlockManager implements BlockStatsMXBean { return false; // already completed (e.g. by syncBlock) final boolean b = commitBlock(lastBlock, commitBlock); - if(countNodes(lastBlock).liveReplicas() >= minReplication) - completeBlock(bc, bc.numBlocks()-1, false); + if (countNodes(lastBlock).liveReplicas() >= minReplication) { + completeBlock(lastBlock, false); + } return b; } /** * Convert a specified block of the file to a complete block. - * @param bc file - * @param blkIndex block index in the file * @throws IOException if the block does not have at least a minimal number * of replicas reported from data-nodes. */ - private BlockInfo completeBlock(final BlockCollection bc, - final int blkIndex, boolean force) throws IOException { - if(blkIndex < 0) - return null; - BlockInfo curBlock = bc.getBlocks()[blkIndex]; - if(curBlock.isComplete()) - return curBlock; + private void completeBlock(BlockInfo curBlock, boolean force) + throws IOException { + if (curBlock.isComplete()) { + return; + } int numNodes = curBlock.numNodes(); - if (!force && numNodes < minReplication) - throw new IOException("Cannot complete block: " + - "block does not satisfy minimal replication requirement."); - if(!force && curBlock.getBlockUCState() != BlockUCState.COMMITTED) + if (!force && numNodes < minReplication) { + throw new IOException("Cannot complete block: " + + "block does not satisfy minimal replication requirement."); + } + if (!force && curBlock.getBlockUCState() != BlockUCState.COMMITTED) { throw new IOException( "Cannot complete block: block has not been COMMITTED by the client"); - BlockInfo completeBlock = curBlock.convertToCompleteBlock(); - // replace penultimate block in file - bc.setBlock(blkIndex, completeBlock); - + } + + curBlock.convertToCompleteBlock(); // Since safe-mode only counts complete blocks, and we now have // one more complete block, we need to adjust the total up, and // also count it as safe, if we have at least the minimum replica @@ -688,33 +685,18 @@ public class BlockManager implements BlockStatsMXBean { namesystem.adjustSafeModeBlockTotals(0, 1); namesystem.incrementSafeBlockCount( Math.min(numNodes, minReplication)); - - // replace block in the blocksMap - return blocksMap.replaceBlock(completeBlock); } - private BlockInfo completeBlock(final BlockCollection bc, - final BlockInfo block, boolean force) throws IOException { - BlockInfo[] fileBlocks = bc.getBlocks(); - for(int idx = 0; idx < fileBlocks.length; idx++) - if(fileBlocks[idx] == block) { - return completeBlock(bc, idx, force); - } - return block; - } - /** * Force the given block in the given file to be marked as complete, * regardless of whether enough replicas are present. This is necessary * when tailing edit logs as a Standby. */ - public BlockInfo forceCompleteBlock(final BlockCollection bc, - final BlockInfo block) throws IOException { + public void forceCompleteBlock(final BlockInfo block) throws IOException { block.commitBlock(block); - return completeBlock(bc, block, true); + completeBlock(block, true); } - /** * Convert the last block of the file to an under construction block.

* The block is converted only if the file has blocks and the last one @@ -2503,7 +2485,7 @@ public class BlockManager implements BlockStatsMXBean { int numCurrentReplica = countLiveNodes(storedBlock); if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED && numCurrentReplica >= minReplication) { - completeBlock(getBlockCollection(storedBlock), storedBlock, false); + completeBlock(storedBlock, false); } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) { // check whether safe replication is reached for the block // only complete blocks are counted towards that. @@ -2577,7 +2559,7 @@ public class BlockManager implements BlockStatsMXBean { if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED && numLiveReplicas >= minReplication) { - storedBlock = completeBlock(bc, storedBlock, false); + completeBlock(storedBlock, false); } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) { // check whether safe replication is reached for the block // only complete blocks are counted towards that diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java index 33c68f3946..9189c3261a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java @@ -209,20 +209,4 @@ class BlocksMap { int getCapacity() { return capacity; } - - /** - * Replace a block in the block map by a new block. - * The new block and the old one have the same key. - * @param newBlock - block for replacement - * @return new block - */ - BlockInfo replaceBlock(BlockInfo newBlock) { - BlockInfo currentBlock = blocks.get(newBlock); - assert currentBlock != null : "the block if not in blocksMap"; - // replace block in data-node lists - currentBlock.replaceBlock(newBlock); - // replace block in the map itself - blocks.put(newBlock); - return newBlock; - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java index f22762c7bc..fc0bb78e0c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java @@ -506,7 +506,7 @@ public class FSEditLogLoader { } INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path); // add the new block to the INodeFile - addNewBlock(fsDir, addBlockOp, oldFile); + addNewBlock(addBlockOp, oldFile); break; } case OP_SET_REPLICATION: { @@ -940,7 +940,7 @@ public class FSEditLogLoader { /** * Add a new block into the given INodeFile */ - private void addNewBlock(FSDirectory fsDir, AddBlockOp op, INodeFile file) + private void addNewBlock(AddBlockOp op, INodeFile file) throws IOException { BlockInfo[] oldBlocks = file.getBlocks(); Block pBlock = op.getPenultimateBlock(); @@ -960,7 +960,7 @@ public class FSEditLogLoader { oldLastBlock.setNumBytes(pBlock.getNumBytes()); if (!oldLastBlock.isComplete()) { - fsNamesys.getBlockManager().forceCompleteBlock(file, oldLastBlock); + fsNamesys.getBlockManager().forceCompleteBlock(oldLastBlock); fsNamesys.getBlockManager().processQueuedMessagesForBlock(pBlock); } } else { // the penultimate block is null @@ -1013,7 +1013,7 @@ public class FSEditLogLoader { if (!oldBlock.isComplete() && (!isLastBlock || op.shouldCompleteLastBlock())) { changeMade = true; - fsNamesys.getBlockManager().forceCompleteBlock(file, oldBlock); + fsNamesys.getBlockManager().forceCompleteBlock(oldBlock); } if (changeMade) { // The state or gen-stamp of the block has changed. So, we may be From 826ae1c26d31f87d88efc920b271bec7eec2e17a Mon Sep 17 00:00:00 2001 From: Haohui Mai Date: Mon, 31 Aug 2015 13:54:14 -0700 Subject: [PATCH 09/18] HDFS-8990. Move RemoteBlockReader to hdfs-client module. Contributed by Mingliang Liu. --- .../org/apache/hadoop/hdfs/RemoteBlockReader.java | 6 +++++- .../apache/hadoop/hdfs/RemoteBlockReader2.java | 15 +++++++++------ .../hdfs/protocol/datatransfer/PacketHeader.java | 0 .../protocol/datatransfer/PacketReceiver.java | 6 +++--- .../hadoop/hdfs/util/ByteBufferOutputStream.java | 0 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++ .../java/org/apache/hadoop/hdfs/DFSClient.java | 1 - .../hadoop/hdfs/TestClientBlockVerification.java | 4 ++-- 8 files changed, 22 insertions(+), 13 deletions(-) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java (99%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java (98%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java (100%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java (98%) rename hadoop-hdfs-project/{hadoop-hdfs => hadoop-hdfs-client}/src/main/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java (100%) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java similarity index 99% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java index 015e154974..7509da52f8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java @@ -50,6 +50,8 @@ import org.apache.hadoop.util.DataChecksum; import org.apache.htrace.Sampler; import org.apache.htrace.Trace; import org.apache.htrace.TraceScope; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -60,6 +62,8 @@ import org.apache.htrace.TraceScope; @InterfaceAudience.Private @Deprecated public class RemoteBlockReader extends FSInputChecker implements BlockReader { + static final Logger LOG = LoggerFactory.getLogger(FSInputChecker.class); + private final Peer peer; private final DatanodeID datanodeID; private final DataInputStream in; @@ -488,7 +492,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader { public int available() throws IOException { // An optimistic estimate of how much data is available // to us without doing network I/O. - return DFSClient.TCP_WINDOW_SIZE; + return RemoteBlockReader2.TCP_WINDOW_SIZE; } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java similarity index 98% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java index 2a77cb6a84..5541e6d7d2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java @@ -28,8 +28,6 @@ import java.nio.channels.ReadableByteChannel; import java.util.EnumSet; import java.util.UUID; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.hdfs.net.Peer; @@ -56,6 +54,9 @@ import org.apache.htrace.TraceScope; import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * This is a wrapper around connection to datanode * and understands checksum, offset etc. @@ -85,16 +86,18 @@ import com.google.common.annotations.VisibleForTesting; @InterfaceAudience.Private public class RemoteBlockReader2 implements BlockReader { - static final Log LOG = LogFactory.getLog(RemoteBlockReader2.class); - + static final Logger LOG = LoggerFactory.getLogger(RemoteBlockReader2.class); + static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB; + final private Peer peer; final private DatanodeID datanodeID; final private PeerCache peerCache; final private long blockId; private final ReadableByteChannel in; + private DataChecksum checksum; - private final PacketReceiver packetReceiver = new PacketReceiver(true); + private ByteBuffer curDataSlice = null; /** offset in block of the last chunk received */ @@ -457,7 +460,7 @@ public class RemoteBlockReader2 implements BlockReader { public int available() throws IOException { // An optimistic estimate of how much data is available // to us without doing network I/O. - return DFSClient.TCP_WINDOW_SIZE; + return TCP_WINDOW_SIZE; } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java similarity index 100% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java similarity index 98% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java index 3045a13b20..c4093b1bd6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java @@ -24,14 +24,14 @@ import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.util.DirectBufferPool; import org.apache.hadoop.io.IOUtils; import com.google.common.base.Preconditions; import com.google.common.primitives.Ints; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Class to handle reading packets one-at-a-time from the wire. @@ -47,7 +47,7 @@ public class PacketReceiver implements Closeable { */ private static final int MAX_PACKET_SIZE = 16 * 1024 * 1024; - static final Log LOG = LogFactory.getLog(PacketReceiver.class); + static final Logger LOG = LoggerFactory.getLogger(PacketReceiver.class); private static final DirectBufferPool bufferPool = new DirectBufferPool(); private final boolean useDirectBuffers; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java similarity index 100% rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 7b5979eae0..ef8fac58fc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -867,6 +867,9 @@ Release 2.8.0 - UNRELEASED HDFS-8980. Remove unnecessary block replacement in INodeFile. (jing9) + HDFS-8990. Move RemoteBlockReader to hdfs-client module. + (Mingliang via wheat9) + OPTIMIZATIONS HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 3c49ef7977..268a5b9cd7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -203,7 +203,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, DataEncryptionKeyFactory { public static final Log LOG = LogFactory.getLog(DFSClient.class); public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour - static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB private final Configuration conf; private final DfsClientConf dfsClientConf; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java index 8dd3d6fd38..5ff343ab0c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java @@ -24,10 +24,10 @@ import static org.mockito.Mockito.verify; import java.util.List; -import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; +import org.apache.hadoop.test.GenericTestUtils; import org.apache.log4j.Level; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -41,7 +41,7 @@ public class TestClientBlockVerification { static LocatedBlock testBlock = null; static { - ((Log4JLogger)RemoteBlockReader2.LOG).getLogger().setLevel(Level.ALL); + GenericTestUtils.setLogLevel(RemoteBlockReader2.LOG, Level.ALL); } @BeforeClass public static void setupCluster() throws Exception { From a3fd2ccc869dfc1f04d1cf0a8678d4d90a43a80f Mon Sep 17 00:00:00 2001 From: Jian He Date: Mon, 31 Aug 2015 17:33:24 -0700 Subject: [PATCH 10/18] YARN-4092. Fixed UI redirection to print useful messages when both RMs are in standby mode. Contributed by Xuan Gong --- hadoop-yarn-project/CHANGES.txt | 3 + .../hadoop/yarn/client/TestRMFailover.java | 27 ++++++ .../hadoop/yarn/webapp/YarnWebParams.java | 1 + .../webapp/RMWebAppFilter.java | 90 ++++++++++++++++++- 4 files changed, 117 insertions(+), 4 deletions(-) diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index 4201b4f59e..19c1082b71 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -422,6 +422,9 @@ Release 2.8.0 - UNRELEASED YARN-1556. NPE getting application report with a null appId. (Weiwei Yang via junping_du) + YARN-4092. Fixed UI redirection to print useful messages when both RMs are + in standby mode. (Xuan Gong via jianhe) + OPTIMIZATIONS YARN-3339. TestDockerContainerExecutor should pull a single image and not diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java index 0d03fd461f..cbc220aa06 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java @@ -27,6 +27,7 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.net.HttpURLConnection; import java.net.URL; + import javax.servlet.http.HttpServletResponse; import org.apache.commons.logging.Log; @@ -45,6 +46,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.AdminService; import org.apache.hadoop.yarn.server.resourcemanager.HATestUtil; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.webproxy.WebAppProxyServer; +import org.apache.hadoop.yarn.webapp.YarnWebParams; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -265,6 +267,7 @@ public class TestRMFailover extends ClientBaseWithFixes { getAdminService(0).transitionToActive(req); String rm1Url = "http://0.0.0.0:18088"; String rm2Url = "http://0.0.0.0:28088"; + String redirectURL = getRedirectURL(rm2Url); // if uri is null, RMWebAppFilter will append a slash at the trail of the redirection url assertEquals(redirectURL,rm1Url+"/"); @@ -304,6 +307,17 @@ public class TestRMFailover extends ClientBaseWithFixes { redirectURL = getRedirectURL(rm2Url + "/proxy/" + fakeAppId); assertNull(redirectURL); + + // transit the active RM to standby + // Both of RMs are in standby mode + getAdminService(0).transitionToStandby(req); + // RM2 is expected to send the httpRequest to itself. + // The Header Field: Refresh is expected to be set. + redirectURL = getRefreshURL(rm2Url); + assertTrue(redirectURL != null + && redirectURL.contains(YarnWebParams.NEXT_REFRESH_INTERVAL) + && redirectURL.contains(rm2Url)); + } // set up http connection with the given url and get the redirection url from the response @@ -323,4 +337,17 @@ public class TestRMFailover extends ClientBaseWithFixes { return redirectUrl; } + static String getRefreshURL(String url) { + String redirectUrl = null; + try { + HttpURLConnection conn = (HttpURLConnection) new URL(url).openConnection(); + // do not automatically follow the redirection + // otherwise we get too many redirections exception + conn.setInstanceFollowRedirects(false); + redirectUrl = conn.getHeaderField("Refresh"); + } catch (Exception e) { + // throw new RuntimeException(e); + } + return redirectUrl; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java index 679e1d6d5d..37926496fe 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java @@ -40,4 +40,5 @@ public interface YarnWebParams { String NODE_STATE = "node.state"; String NODE_LABEL = "node.label"; String WEB_UI_TYPE = "web.ui.type"; + String NEXT_REFRESH_INTERVAL = "next.fresh.interval"; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java index 500f17abc0..a8f793a5a8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java @@ -20,6 +20,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp; import java.io.IOException; import java.io.PrintWriter; +import java.net.InetSocketAddress; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Random; import java.util.Set; import javax.inject.Inject; @@ -29,8 +33,11 @@ import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.http.HtmlQuoting; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils; +import org.apache.hadoop.yarn.webapp.YarnWebParams; import com.google.common.collect.Sets; import com.google.inject.Injector; @@ -48,11 +55,26 @@ public class RMWebAppFilter extends GuiceContainer { // define a set of URIs which do not need to do redirection private static final Set NON_REDIRECTED_URIS = Sets.newHashSet( "/conf", "/stacks", "/logLevel", "/logs"); + private String path; + private static final int BASIC_SLEEP_TIME = 5; + private static final int MAX_SLEEP_TIME = 5 * 60; @Inject - public RMWebAppFilter(Injector injector) { + public RMWebAppFilter(Injector injector, Configuration conf) { super(injector); this.injector=injector; + InetSocketAddress sock = YarnConfiguration.useHttps(conf) + ? conf.getSocketAddr(YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS, + YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_ADDRESS, + YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT) + : conf.getSocketAddr(YarnConfiguration.RM_WEBAPP_ADDRESS, + YarnConfiguration.DEFAULT_RM_WEBAPP_ADDRESS, + YarnConfiguration.DEFAULT_RM_WEBAPP_PORT); + + path = sock.getHostName() + ":" + Integer.toString(sock.getPort()); + path = YarnConfiguration.useHttps(conf) + ? "https://" + path + : "http://" + path; } @Override @@ -69,9 +91,11 @@ public class RMWebAppFilter extends GuiceContainer { rmWebApp.checkIfStandbyRM(); if (rmWebApp.isStandby() && shouldRedirect(rmWebApp, uri)) { - String redirectPath = rmWebApp.getRedirectPath() + uri; + + String redirectPath = rmWebApp.getRedirectPath(); if (redirectPath != null && !redirectPath.isEmpty()) { + redirectPath += uri; String redirectMsg = "This is standby RM. The redirect url is: " + redirectPath; PrintWriter out = response.getWriter(); @@ -79,11 +103,40 @@ public class RMWebAppFilter extends GuiceContainer { response.setHeader("Location", redirectPath); response.setStatus(HttpServletResponse.SC_TEMPORARY_REDIRECT); return; + } else { + boolean doRetry = true; + String retryIntervalStr = + request.getParameter(YarnWebParams.NEXT_REFRESH_INTERVAL); + int retryInterval = 0; + if (retryIntervalStr != null) { + try { + retryInterval = Integer.parseInt(retryIntervalStr.trim()); + } catch (NumberFormatException ex) { + doRetry = false; + } + } + int next = calculateExponentialTime(retryInterval); + + String redirectUrl = + appendOrReplaceParamter(path + uri, + YarnWebParams.NEXT_REFRESH_INTERVAL + "=" + (retryInterval + 1)); + if (redirectUrl == null || next > MAX_SLEEP_TIME) { + doRetry = false; + } + String redirectMsg = + doRetry ? "Can not find any active RM. Will retry in next " + next + + " seconds." : "There is no active RM right now."; + PrintWriter out = response.getWriter(); + out.println(redirectMsg); + if (doRetry) { + response.setHeader("Refresh", next + ";url=" + redirectUrl); + response.setStatus(HttpServletResponse.SC_TEMPORARY_REDIRECT); + } } + return; } super.doFilter(request, response, chain); - } private boolean shouldRedirect(RMWebApp rmWebApp, String uri) { @@ -92,4 +145,33 @@ public class RMWebAppFilter extends GuiceContainer { && !uri.startsWith(ProxyUriUtils.PROXY_BASE) && !NON_REDIRECTED_URIS.contains(uri); } -} + + private String appendOrReplaceParamter(String uri, String newQuery) { + if (uri.contains(YarnWebParams.NEXT_REFRESH_INTERVAL + "=")) { + return uri.replaceAll(YarnWebParams.NEXT_REFRESH_INTERVAL + "=[^&]+", + newQuery); + } + try { + URI oldUri = new URI(uri); + String appendQuery = oldUri.getQuery(); + if (appendQuery == null) { + appendQuery = newQuery; + } else { + appendQuery += "&" + newQuery; + } + + URI newUri = + new URI(oldUri.getScheme(), oldUri.getAuthority(), oldUri.getPath(), + appendQuery, oldUri.getFragment()); + + return newUri.toString(); + } catch (URISyntaxException e) { + return null; + } + } + + private static int calculateExponentialTime(int retries) { + long baseTime = BASIC_SLEEP_TIME * (1L << retries); + return (int) (baseTime * ((new Random()).nextDouble() + 0.5)); + } +} \ No newline at end of file From 4eaa7fd3eae4412ac0b964c617b1bbb17a39d8be Mon Sep 17 00:00:00 2001 From: Jian He Date: Mon, 31 Aug 2015 17:43:36 -0700 Subject: [PATCH 11/18] Move YARN-4092 to 2.7.2 --- hadoop-yarn-project/CHANGES.txt | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index 19c1082b71..80cf793784 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -422,9 +422,6 @@ Release 2.8.0 - UNRELEASED YARN-1556. NPE getting application report with a null appId. (Weiwei Yang via junping_du) - YARN-4092. Fixed UI redirection to print useful messages when both RMs are - in standby mode. (Xuan Gong via jianhe) - OPTIMIZATIONS YARN-3339. TestDockerContainerExecutor should pull a single image and not @@ -824,6 +821,8 @@ Release 2.7.2 - UNRELEASED YARN-3978. Configurably turn off the saving of container info in Generic AHS (Eric Payne via jeagles) + YARN-4092. Fixed UI redirection to print useful messages when both RMs are + in standby mode. (Xuan Gong via jianhe) OPTIMIZATIONS From 8fa41d9dd4b923bf4141f019414a1a8b079124c6 Mon Sep 17 00:00:00 2001 From: yliu Date: Tue, 1 Sep 2015 08:52:50 +0800 Subject: [PATCH 12/18] HDFS-8946. Improve choosing datanode storage for block placement. (yliu) --- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 2 + .../BlockPlacementPolicyDefault.java | 147 +++++------------- .../blockmanagement/DatanodeDescriptor.java | 36 +++-- .../TestReplicationPolicy.java | 26 +++- 4 files changed, 93 insertions(+), 118 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index ef8fac58fc..6584c84d15 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -870,6 +870,8 @@ Release 2.8.0 - UNRELEASED HDFS-8990. Move RemoteBlockReader to hdfs-client module. (Mingliang via wheat9) + HDFS-8946. Improve choosing datanode storage for block placement. (yliu) + OPTIMIZATIONS HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java index 6d7a765605..f761150efb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java @@ -26,12 +26,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.DFSConfigKeys; -import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.LocatedBlock; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; -import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State; import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.net.Node; import org.apache.hadoop.net.NodeBase; @@ -458,19 +455,18 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { for (Iterator> iter = storageTypes .entrySet().iterator(); iter.hasNext(); ) { Map.Entry entry = iter.next(); - for (DatanodeStorageInfo localStorage : DFSUtil.shuffle( - localDatanode.getStorageInfos())) { - StorageType type = entry.getKey(); - if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize, - results, type) >= 0) { - int num = entry.getValue(); - if (num == 1) { - iter.remove(); - } else { - entry.setValue(num - 1); - } - return localStorage; + DatanodeStorageInfo localStorage = chooseStorage4Block( + localDatanode, blocksize, results, entry.getKey()); + if (localStorage != null) { + // add node and related nodes to excludedNode + addToExcludedNodes(localDatanode, excludedNodes); + int num = entry.getValue(); + if (num == 1) { + iter.remove(); + } else { + entry.setValue(num - 1); } + return localStorage; } } } @@ -651,7 +647,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { boolean avoidStaleNodes, EnumMap storageTypes) throws NotEnoughReplicasException { - + int numOfAvailableNodes = clusterMap.countNumOfAvailableNodes( scope, excludedNodes); StringBuilder builder = null; @@ -669,49 +665,39 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { builder.append("\nNode ").append(NodeBase.getPath(chosenNode)).append(" ["); } numOfAvailableNodes--; - if (!isGoodDatanode(chosenNode, maxNodesPerRack, considerLoad, + DatanodeStorageInfo storage = null; + if (isGoodDatanode(chosenNode, maxNodesPerRack, considerLoad, results, avoidStaleNodes)) { - if (LOG.isDebugEnabled()) { - builder.append("\n]"); - } - badTarget = true; - continue; - } - - final DatanodeStorageInfo[] storages = DFSUtil.shuffle( - chosenNode.getStorageInfos()); - int i = 0; - boolean search = true; - for (Iterator> iter = storageTypes - .entrySet().iterator(); search && iter.hasNext(); ) { - Map.Entry entry = iter.next(); - for (i = 0; i < storages.length; i++) { - StorageType type = entry.getKey(); - final int newExcludedNodes = addIfIsGoodTarget(storages[i], - excludedNodes, blocksize, results, type); - if (newExcludedNodes >= 0) { + for (Iterator> iter = storageTypes + .entrySet().iterator(); iter.hasNext(); ) { + Map.Entry entry = iter.next(); + storage = chooseStorage4Block( + chosenNode, blocksize, results, entry.getKey()); + if (storage != null) { numOfReplicas--; if (firstChosen == null) { - firstChosen = storages[i]; + firstChosen = storage; } - numOfAvailableNodes -= newExcludedNodes; + // add node and related nodes to excludedNode + numOfAvailableNodes -= + addToExcludedNodes(chosenNode, excludedNodes); int num = entry.getValue(); if (num == 1) { iter.remove(); } else { entry.setValue(num - 1); } - search = false; break; } } } + if (LOG.isDebugEnabled()) { builder.append("\n]"); } // If no candidate storage was found on this DN then set badTarget. - badTarget = (i == storages.length); + badTarget = (storage == null); } } @@ -740,32 +726,27 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { } /** - * If the given storage is a good target, add it to the result list and - * update the set of excluded nodes. - * @return -1 if the given is not a good target; - * otherwise, return the number of nodes added to excludedNodes set. + * Choose a good storage of given storage type from datanode, and add it to + * the result list. + * + * @param dnd datanode descriptor + * @param blockSize requested block size + * @param results the result storages + * @param storageType requested storage type + * @return the chosen datanode storage */ - int addIfIsGoodTarget(DatanodeStorageInfo storage, - Set excludedNodes, + DatanodeStorageInfo chooseStorage4Block(DatanodeDescriptor dnd, long blockSize, List results, StorageType storageType) { - if (isGoodTarget(storage, blockSize, results, storageType)) { + DatanodeStorageInfo storage = + dnd.chooseStorage4Block(storageType, blockSize); + if (storage != null) { results.add(storage); - // add node and related nodes to excludedNode - return addToExcludedNodes(storage.getDatanodeDescriptor(), excludedNodes); - } else { - return -1; - } - } - - private static void logNodeIsNotChosen(DatanodeStorageInfo storage, String reason) { - if (LOG.isDebugEnabled()) { - // build the error message for later use. - debugLoggingBuilder.get() - .append("\n Storage ").append(storage) - .append(" is not chosen since ").append(reason).append("."); + } else { + logNodeIsNotChosen(dnd, "no good storage to place the block "); } + return storage; } private static void logNodeIsNotChosen(DatanodeDescriptor node, @@ -836,52 +817,6 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy { return true; } - /** - * Determine if a storage is a good target. - * - * @param storage The target storage - * @param blockSize Size of block - * @param results A list containing currently chosen nodes. Used to check if - * too many nodes has been chosen in the target rack. - * @return Return true if node has enough space. - */ - private boolean isGoodTarget(DatanodeStorageInfo storage, - long blockSize, - List results, - StorageType requiredStorageType) { - if (storage.getStorageType() != requiredStorageType) { - logNodeIsNotChosen(storage, "storage types do not match," - + " where the required storage type is " + requiredStorageType); - return false; - } - if (storage.getState() == State.READ_ONLY_SHARED) { - logNodeIsNotChosen(storage, "storage is read-only"); - return false; - } - - if (storage.getState() == State.FAILED) { - logNodeIsNotChosen(storage, "storage has failed"); - return false; - } - - DatanodeDescriptor node = storage.getDatanodeDescriptor(); - - final long requiredSize = blockSize * HdfsServerConstants.MIN_BLOCKS_FOR_WRITE; - final long scheduledSize = blockSize * node.getBlocksScheduled(storage.getStorageType()); - final long remaining = node.getRemaining(storage.getStorageType(), - requiredSize); - if (requiredSize > remaining - scheduledSize) { - logNodeIsNotChosen(storage, "the node does not have enough " - + storage.getStorageType() + " space" - + " (required=" + requiredSize - + ", scheduled=" + scheduledSize - + ", remaining=" + remaining + ")"); - return false; - } - - return true; - } - /** * Return a pipeline of nodes. * The pipeline is formed finding a shortest path that diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java index 7e3c59b817..0b398c51ca 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java @@ -31,14 +31,15 @@ import java.util.Queue; import java.util.Set; import com.google.common.annotations.VisibleForTesting; - import com.google.common.collect.ImmutableList; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.namenode.CachedBlock; import org.apache.hadoop.hdfs.server.protocol.BlockReportContext; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; @@ -663,26 +664,39 @@ public class DatanodeDescriptor extends DatanodeInfo { } /** - * Return the sum of remaining spaces of the specified type. If the remaining - * space of a storage is less than minSize, it won't be counted toward the - * sum. + * Find whether the datanode contains good storage of given type to + * place block of size blockSize. * - * @param t The storage type. If null, the type is ignored. - * @param minSize The minimum free space required. - * @return the sum of remaining spaces that are bigger than minSize. + *

Currently datanode only cares about the storage type, in this + * method, the first storage of given type we see is returned. + * + * @param t requested storage type + * @param blockSize requested block size + * @return */ - public long getRemaining(StorageType t, long minSize) { + public DatanodeStorageInfo chooseStorage4Block(StorageType t, + long blockSize) { + final long requiredSize = + blockSize * HdfsServerConstants.MIN_BLOCKS_FOR_WRITE; + final long scheduledSize = blockSize * getBlocksScheduled(t); long remaining = 0; + DatanodeStorageInfo storage = null; for (DatanodeStorageInfo s : getStorageInfos()) { if (s.getState() == State.NORMAL && - (t == null || s.getStorageType() == t)) { + s.getStorageType() == t) { + if (storage == null) { + storage = s; + } long r = s.getRemaining(); - if (r >= minSize) { + if (r >= requiredSize) { remaining += r; } } } - return remaining; + if (requiredSize > remaining - scheduledSize) { + return null; + } + return storage; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java index cec33fef7c..27d647c84d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java @@ -181,7 +181,7 @@ public class TestReplicationPolicy { * considered. */ @Test - public void testChooseNodeWithMultipleStorages() throws Exception { + public void testChooseNodeWithMultipleStorages1() throws Exception { updateHeartbeatWithUsage(dataNodes[5], 2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, (2*HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE)/3, 0L, @@ -200,6 +200,30 @@ public class TestReplicationPolicy { resetHeartbeatForStorages(); } + /** + * Test whether all storages on the datanode are considered while + * choosing target to place block. + */ + @Test + public void testChooseNodeWithMultipleStorages2() throws Exception { + updateHeartbeatWithUsage(dataNodes[5], + 2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, + (2*HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE)/3, 0L, + 0L, 0L, 0, 0); + + updateHeartbeatForExtraStorage( + 2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, + HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L); + + DatanodeStorageInfo[] targets; + targets = chooseTarget (1, dataNodes[5], + new ArrayList(), null); + assertEquals(1, targets.length); + assertEquals(dataNodes[5], targets[0].getDatanodeDescriptor()); + + resetHeartbeatForStorages(); + } + /** * In this testcase, client is dataNodes[0]. So the 1st replica should be * placed on dataNodes[0], the 2nd replica should be placed on From 24f6a7c9563757234f53ca23e12f9c9208b53082 Mon Sep 17 00:00:00 2001 From: Colin Patrick Mccabe Date: Mon, 31 Aug 2015 17:31:29 -0700 Subject: [PATCH 13/18] HDFS-8965. Harden edit log reading code against out of memory errors (cmccabe) --- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 2 + .../BookKeeperEditLogInputStream.java | 2 +- .../hadoop/hdfs/protocol/LayoutVersion.java | 2 +- .../namenode/EditLogBackupInputStream.java | 2 +- .../namenode/EditLogFileInputStream.java | 2 +- .../hdfs/server/namenode/FSEditLogOp.java | 354 +++++++++++++----- .../hdfs/server/namenode/TestEditLog.java | 2 +- .../namenode/TestEditLogFileInputStream.java | 80 ++++ 8 files changed, 341 insertions(+), 105 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 6584c84d15..57ddcb2943 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -872,6 +872,8 @@ Release 2.8.0 - UNRELEASED HDFS-8946. Improve choosing datanode storage for block placement. (yliu) + HDFS-8965. Harden edit log reading code against out of memory errors (cmccabe) + OPTIMIZATIONS HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java index e2098ddee1..86da80728b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java @@ -83,7 +83,7 @@ class BookKeeperEditLogInputStream extends EditLogInputStream { tracker = new FSEditLogLoader.PositionTrackingInputStream(bin); DataInputStream in = new DataInputStream(tracker); - reader = new FSEditLogOp.Reader(in, tracker, logVersion); + reader = FSEditLogOp.Reader.create(in, tracker, logVersion); } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java index c893744434..1750790894 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java @@ -87,7 +87,7 @@ public class LayoutVersion { FSIMAGE_COMPRESSION(-25, "Support for fsimage compression"), FSIMAGE_CHECKSUM(-26, "Support checksum for fsimage"), REMOVE_REL13_DISK_LAYOUT_SUPPORT(-27, "Remove support for 0.13 disk layout"), - EDITS_CHESKUM(-28, "Support checksum for editlog"), + EDITS_CHECKSUM(-28, "Support checksum for editlog"), UNUSED(-29, "Skipped version"), FSIMAGE_NAME_OPTIMIZATION(-30, "Store only last part of path in fsimage"), RESERVED_REL20_203(-31, -19, "Reserved for release 0.20.203", true, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java index 689caccdc7..81d285a036 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java @@ -119,7 +119,7 @@ class EditLogBackupInputStream extends EditLogInputStream { this.version = version; - reader = new FSEditLogOp.Reader(in, tracker, version); + reader = FSEditLogOp.Reader.create(in, tracker, version); } void clear() throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java index 3e21c24289..73a162ebb1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java @@ -157,7 +157,7 @@ public class EditLogFileInputStream extends EditLogInputStream { "flags from log"); } } - reader = new FSEditLogOp.Reader(dataIn, tracker, logVersion); + reader = FSEditLogOp.Reader.create(dataIn, tracker, logVersion); reader.setMaxOpSize(maxOpSize); state = State.OPEN; } finally { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java index ab36f17af0..125e1cf55a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java @@ -4518,42 +4518,46 @@ public abstract class FSEditLogOp { /** * Class for reading editlog ops from a stream */ - public static class Reader { - private final DataInputStream in; - private final StreamLimiter limiter; - private final int logVersion; - private final Checksum checksum; - private final OpInstanceCache cache; - private int maxOpSize; - private final boolean supportEditLogLength; + public abstract static class Reader { + final DataInputStream in; + final StreamLimiter limiter; + final OpInstanceCache cache; + final byte[] temp = new byte[4096]; + final int logVersion; + int maxOpSize; + + public static Reader create(DataInputStream in, StreamLimiter limiter, + int logVersion) { + if (logVersion < NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION) { + // Use the LengthPrefixedReader on edit logs which are newer than what + // we can parse. (Newer layout versions are represented by smaller + // negative integers, for historical reasons.) Even though we can't + // parse the Ops contained in them, we should still be able to call + // scanOp on them. This is important for the JournalNode during rolling + // upgrade. + return new LengthPrefixedReader(in, limiter, logVersion); + } else if (NameNodeLayoutVersion.supports( + NameNodeLayoutVersion.Feature.EDITLOG_LENGTH, logVersion)) { + return new LengthPrefixedReader(in, limiter, logVersion); + } else if (NameNodeLayoutVersion.supports( + LayoutVersion.Feature.EDITS_CHECKSUM, logVersion)) { + Checksum checksum = DataChecksum.newCrc32(); + return new ChecksummedReader(checksum, in, limiter, logVersion); + } else { + return new LegacyReader(in, limiter, logVersion); + } + } /** * Construct the reader - * @param in The stream to read from. - * @param logVersion The version of the data coming from the stream. + * @param in The stream to read from. + * @param limiter The limiter for this stream. + * @param logVersion The version of the data coming from the stream. */ - public Reader(DataInputStream in, StreamLimiter limiter, int logVersion) { - this.logVersion = logVersion; - if (NameNodeLayoutVersion.supports( - LayoutVersion.Feature.EDITS_CHESKUM, logVersion)) { - this.checksum = DataChecksum.newCrc32(); - } else { - this.checksum = null; - } - // It is possible that the logVersion is actually a future layoutversion - // during the rolling upgrade (e.g., the NN gets upgraded first). We - // assume future layout will also support length of editlog op. - this.supportEditLogLength = NameNodeLayoutVersion.supports( - NameNodeLayoutVersion.Feature.EDITLOG_LENGTH, logVersion) - || logVersion < NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION; - - if (this.checksum != null) { - this.in = new DataInputStream( - new CheckedInputStream(in, this.checksum)); - } else { - this.in = in; - } + Reader(DataInputStream in, StreamLimiter limiter, int logVersion) { + this.in = in; this.limiter = limiter; + this.logVersion = logVersion; this.cache = new OpInstanceCache(); this.maxOpSize = DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT; } @@ -4606,26 +4610,25 @@ public abstract class FSEditLogOp { } } - private void verifyTerminator() throws IOException { + void verifyTerminator() throws IOException { /** The end of the edit log should contain only 0x00 or 0xff bytes. * If it contains other bytes, the log itself may be corrupt. * It is important to check this; if we don't, a stray OP_INVALID byte * could make us stop reading the edit log halfway through, and we'd never * know that we had lost data. */ - byte[] buf = new byte[4096]; limiter.clearLimit(); int numRead = -1, idx = 0; while (true) { try { numRead = -1; idx = 0; - numRead = in.read(buf); + numRead = in.read(temp); if (numRead == -1) { return; } while (idx < numRead) { - if ((buf[idx] != (byte)0) && (buf[idx] != (byte)-1)) { + if ((temp[idx] != (byte)0) && (temp[idx] != (byte)-1)) { throw new IOException("Read extra bytes after " + "the terminator!"); } @@ -4638,7 +4641,7 @@ public abstract class FSEditLogOp { if (numRead != -1) { in.reset(); IOUtils.skipFully(in, idx); - in.mark(buf.length + 1); + in.mark(temp.length + 1); IOUtils.skipFully(in, 1); } } @@ -4653,14 +4656,164 @@ public abstract class FSEditLogOp { * If an exception is thrown, the stream's mark will be set to the first * problematic byte. This usually means the beginning of the opcode. */ - private FSEditLogOp decodeOp() throws IOException { + public abstract FSEditLogOp decodeOp() throws IOException; + + /** + * Similar to decodeOp(), but we only retrieve the transaction ID of the + * Op rather than reading it. If the edit log format supports length + * prefixing, this can be much faster than full decoding. + * + * @return the last txid of the segment, or INVALID_TXID on EOF. + */ + public abstract long scanOp() throws IOException; + } + + /** + * Reads edit logs which are prefixed with a length. These edit logs also + * include a checksum and transaction ID. + */ + private static class LengthPrefixedReader extends Reader { + /** + * The minimum length of a length-prefixed Op. + * + * The minimum Op has: + * 1-byte opcode + * 4-byte length + * 8-byte txid + * 0-byte body + * 4-byte checksum + */ + private static final int MIN_OP_LENGTH = 17; + + /** + * The op id length. + * + * Not included in the stored length. + */ + private static final int OP_ID_LENGTH = 1; + + /** + * The checksum length. + * + * Not included in the stored length. + */ + private static final int CHECKSUM_LENGTH = 4; + + private final Checksum checksum; + + LengthPrefixedReader(DataInputStream in, StreamLimiter limiter, + int logVersion) { + super(in, limiter, logVersion); + this.checksum = DataChecksum.newCrc32(); + } + + @Override + public FSEditLogOp decodeOp() throws IOException { + long txid = decodeOpFrame(); + if (txid == HdfsServerConstants.INVALID_TXID) { + return null; + } + in.reset(); + in.mark(maxOpSize); + FSEditLogOpCodes opCode = FSEditLogOpCodes.fromByte(in.readByte()); + FSEditLogOp op = cache.get(opCode); + if (op == null) { + throw new IOException("Read invalid opcode " + opCode); + } + op.setTransactionId(txid); + IOUtils.skipFully(in, 4 + 8); // skip length and txid + op.readFields(in, logVersion); + // skip over the checksum, which we validated above. + IOUtils.skipFully(in, CHECKSUM_LENGTH); + return op; + } + + @Override + public long scanOp() throws IOException { + return decodeOpFrame(); + } + + /** + * Decode the opcode "frame". This includes reading the opcode and + * transaction ID, and validating the checksum and length. It does not + * include reading the opcode-specific fields. + * The input stream will be advanced to the end of the op at the end of this + * function. + * + * @return An op with the txid set, but none of the other fields + * filled in, or null if we hit EOF. + */ + private long decodeOpFrame() throws IOException { limiter.setLimit(maxOpSize); in.mark(maxOpSize); - - if (checksum != null) { - checksum.reset(); + byte opCodeByte; + try { + opCodeByte = in.readByte(); + } catch (EOFException eof) { + // EOF at an opcode boundary is expected. + return HdfsServerConstants.INVALID_TXID; } + if (opCodeByte == FSEditLogOpCodes.OP_INVALID.getOpCode()) { + verifyTerminator(); + return HdfsServerConstants.INVALID_TXID; + } + // Here, we verify that the Op size makes sense and that the + // data matches its checksum before attempting to construct an Op. + // This is important because otherwise we may encounter an + // OutOfMemoryException which could bring down the NameNode or + // JournalNode when reading garbage data. + int opLength = in.readInt() + OP_ID_LENGTH + CHECKSUM_LENGTH; + if (opLength > maxOpSize) { + throw new IOException("Op " + (int)opCodeByte + " has size " + + opLength + ", but maxOpSize = " + maxOpSize); + } else if (opLength < MIN_OP_LENGTH) { + throw new IOException("Op " + (int)opCodeByte + " has size " + + opLength + ", but the minimum op size is " + MIN_OP_LENGTH); + } + long txid = in.readLong(); + // Verify checksum + in.reset(); + in.mark(maxOpSize); + checksum.reset(); + for (int rem = opLength - CHECKSUM_LENGTH; rem > 0;) { + int toRead = Math.min(temp.length, rem); + IOUtils.readFully(in, temp, 0, toRead); + checksum.update(temp, 0, toRead); + rem -= toRead; + } + int expectedChecksum = in.readInt(); + int calculatedChecksum = (int)checksum.getValue(); + if (expectedChecksum != calculatedChecksum) { + throw new ChecksumException( + "Transaction is corrupt. Calculated checksum is " + + calculatedChecksum + " but read checksum " + + expectedChecksum, txid); + } + return txid; + } + } + /** + * Read edit logs which have a checksum and a transaction ID, but not a + * length. + */ + private static class ChecksummedReader extends Reader { + private final Checksum checksum; + + ChecksummedReader(Checksum checksum, DataInputStream in, + StreamLimiter limiter, int logVersion) { + super(new DataInputStream( + new CheckedInputStream(in, checksum)), limiter, logVersion); + this.checksum = checksum; + } + + @Override + public FSEditLogOp decodeOp() throws IOException { + limiter.setLimit(maxOpSize); + in.mark(maxOpSize); + // Reset the checksum. Since we are using a CheckedInputStream, each + // subsequent read from the stream will update the checksum. + checksum.reset(); byte opCodeByte; try { opCodeByte = in.readByte(); @@ -4668,88 +4821,89 @@ public abstract class FSEditLogOp { // EOF at an opcode boundary is expected. return null; } - FSEditLogOpCodes opCode = FSEditLogOpCodes.fromByte(opCodeByte); if (opCode == OP_INVALID) { verifyTerminator(); return null; } - FSEditLogOp op = cache.get(opCode); if (op == null) { throw new IOException("Read invalid opcode " + opCode); } - - if (supportEditLogLength) { - in.readInt(); + op.setTransactionId(in.readLong()); + op.readFields(in, logVersion); + // Verify checksum + int calculatedChecksum = (int)checksum.getValue(); + int expectedChecksum = in.readInt(); + if (expectedChecksum != calculatedChecksum) { + throw new ChecksumException( + "Transaction is corrupt. Calculated checksum is " + + calculatedChecksum + " but read checksum " + + expectedChecksum, op.txid); } + return op; + } + @Override + public long scanOp() throws IOException { + // Edit logs of this age don't have any length prefix, so we just have + // to read the entire Op. + FSEditLogOp op = decodeOp(); + return op == null ? + HdfsServerConstants.INVALID_TXID : op.getTransactionId(); + } + } + + /** + * Read older edit logs which may or may not have transaction IDs and other + * features. This code is used during upgrades and to allow HDFS INotify to + * read older edit log files. + */ + private static class LegacyReader extends Reader { + LegacyReader(DataInputStream in, + StreamLimiter limiter, int logVersion) { + super(in, limiter, logVersion); + } + + @Override + public FSEditLogOp decodeOp() throws IOException { + limiter.setLimit(maxOpSize); + in.mark(maxOpSize); + byte opCodeByte; + try { + opCodeByte = in.readByte(); + } catch (EOFException eof) { + // EOF at an opcode boundary is expected. + return null; + } + FSEditLogOpCodes opCode = FSEditLogOpCodes.fromByte(opCodeByte); + if (opCode == OP_INVALID) { + verifyTerminator(); + return null; + } + FSEditLogOp op = cache.get(opCode); + if (op == null) { + throw new IOException("Read invalid opcode " + opCode); + } if (NameNodeLayoutVersion.supports( - LayoutVersion.Feature.STORED_TXIDS, logVersion)) { - // Read the txid + LayoutVersion.Feature.STORED_TXIDS, logVersion)) { op.setTransactionId(in.readLong()); } else { op.setTransactionId(HdfsServerConstants.INVALID_TXID); } - op.readFields(in, logVersion); - - validateChecksum(in, checksum, op.txid); return op; } - /** - * Similar with decodeOp(), but instead of doing the real decoding, we skip - * the content of the op if the length of the editlog is supported. - * @return the last txid of the segment, or INVALID_TXID on exception - */ + @Override public long scanOp() throws IOException { - if (supportEditLogLength) { - limiter.setLimit(maxOpSize); - in.mark(maxOpSize); - - final byte opCodeByte; - try { - opCodeByte = in.readByte(); // op code - } catch (EOFException e) { - return HdfsServerConstants.INVALID_TXID; - } - - FSEditLogOpCodes opCode = FSEditLogOpCodes.fromByte(opCodeByte); - if (opCode == OP_INVALID) { - verifyTerminator(); - return HdfsServerConstants.INVALID_TXID; - } - - int length = in.readInt(); // read the length of the op - long txid = in.readLong(); // read the txid - - // skip the remaining content - IOUtils.skipFully(in, length - 8); - // TODO: do we want to verify checksum for JN? For now we don't. - return txid; - } else { - FSEditLogOp op = decodeOp(); - return op == null ? HdfsServerConstants.INVALID_TXID : op.getTransactionId(); - } - } - - /** - * Validate a transaction's checksum - */ - private void validateChecksum(DataInputStream in, - Checksum checksum, - long txid) - throws IOException { - if (checksum != null) { - int calculatedChecksum = (int)checksum.getValue(); - int readChecksum = in.readInt(); // read in checksum - if (readChecksum != calculatedChecksum) { - throw new ChecksumException( - "Transaction is corrupt. Calculated checksum is " + - calculatedChecksum + " but read checksum " + readChecksum, txid); - } + if (!NameNodeLayoutVersion.supports( + LayoutVersion.Feature.STORED_TXIDS, logVersion)) { + throw new IOException("Can't scan a pre-transactional edit log."); } + FSEditLogOp op = decodeOp(); + return op == null ? + HdfsServerConstants.INVALID_TXID : op.getTransactionId(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java index 68d008f567..e59dec400e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java @@ -875,7 +875,7 @@ public class TestEditLog { tracker = new FSEditLogLoader.PositionTrackingInputStream(in); in = new DataInputStream(tracker); - reader = new FSEditLogOp.Reader(in, tracker, version); + reader = FSEditLogOp.Reader.create(in, tracker, version); } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java index c0eb890536..aecdc789c4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java @@ -25,19 +25,35 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; import java.net.HttpURLConnection; import java.net.URL; import java.util.EnumMap; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.util.Holder; import org.apache.hadoop.hdfs.web.URLConnectionFactory; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.test.PathUtils; +import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; public class TestEditLogFileInputStream { + private static final Log LOG = + LogFactory.getLog(TestEditLogFileInputStream.class); private static final byte[] FAKE_LOG_DATA = TestEditLog.HADOOP20_SOME_EDITS; + private final static File TEST_DIR = PathUtils + .getTestDir(TestEditLogFileInputStream.class); + @Test public void testReadURL() throws Exception { HttpURLConnection conn = mock(HttpURLConnection.class); @@ -63,4 +79,68 @@ public class TestEditLogFileInputStream { assertEquals(FAKE_LOG_DATA.length, elis.length()); elis.close(); } + + /** + * Regression test for HDFS-8965 which verifies that + * FSEditLogFileInputStream#scanOp verifies Op checksums. + */ + @Test(timeout=60000) + public void testScanCorruptEditLog() throws Exception { + Configuration conf = new Configuration(); + File editLog = new File(System.getProperty( + "test.build.data", "/tmp"), "testCorruptEditLog"); + + LOG.debug("Creating test edit log file: " + editLog); + EditLogFileOutputStream elos = new EditLogFileOutputStream(conf, + editLog.getAbsoluteFile(), 8192); + elos.create(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION); + FSEditLogOp.OpInstanceCache cache = new FSEditLogOp.OpInstanceCache(); + FSEditLogOp.MkdirOp mkdirOp = FSEditLogOp.MkdirOp.getInstance(cache); + mkdirOp.reset(); + mkdirOp.setRpcCallId(123); + mkdirOp.setTransactionId(1); + mkdirOp.setInodeId(789L); + mkdirOp.setPath("/mydir"); + PermissionStatus perms = PermissionStatus.createImmutable( + "myuser", "mygroup", FsPermission.createImmutable((short)0777)); + mkdirOp.setPermissionStatus(perms); + elos.write(mkdirOp); + mkdirOp.reset(); + mkdirOp.setRpcCallId(456); + mkdirOp.setTransactionId(2); + mkdirOp.setInodeId(123L); + mkdirOp.setPath("/mydir2"); + perms = PermissionStatus.createImmutable( + "myuser", "mygroup", FsPermission.createImmutable((short)0666)); + mkdirOp.setPermissionStatus(perms); + elos.write(mkdirOp); + elos.setReadyToFlush(); + elos.flushAndSync(false); + elos.close(); + long fileLen = editLog.length(); + + LOG.debug("Corrupting last 4 bytes of edit log file " + editLog + + ", whose length is " + fileLen); + RandomAccessFile rwf = new RandomAccessFile(editLog, "rw"); + rwf.seek(fileLen - 4); + int b = rwf.readInt(); + rwf.seek(fileLen - 4); + rwf.writeInt(b + 1); + rwf.close(); + + EditLogFileInputStream elis = new EditLogFileInputStream(editLog); + Assert.assertEquals(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION, + elis.getVersion(true)); + Assert.assertEquals(1, elis.scanNextOp()); + LOG.debug("Read transaction 1 from " + editLog); + try { + elis.scanNextOp(); + Assert.fail("Expected scanNextOp to fail when op checksum was corrupt."); + } catch (IOException e) { + LOG.debug("Caught expected checksum error when reading corrupt " + + "transaction 2", e); + GenericTestUtils.assertExceptionContains("Transaction is corrupt.", e); + } + elis.close(); + } } From 7ad3556ed38560585579172aa68356f37b2288c8 Mon Sep 17 00:00:00 2001 From: Andrew Wang Date: Mon, 31 Aug 2015 18:17:14 -0700 Subject: [PATCH 14/18] HADOOP-12368. Mark ViewFileSystemBaseTest and ViewFsBaseTest as abstract. --- hadoop-common-project/hadoop-common/CHANGES.txt | 3 +++ .../org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java | 2 +- .../test/java/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java | 2 +- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index 95eb677278..0f52d22089 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -756,6 +756,9 @@ Release 2.8.0 - UNRELEASED HADOOP-12325. RPC Metrics : Add the ability track and log slow RPCs. (Anu Engineer via xyao) + HADOOP-12368. Mark ViewFileSystemBaseTest and ViewFsBaseTest as abstract. + (wang) + OPTIMIZATIONS HADOOP-11785. Reduce the number of listStatus operation in distcp diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java index 7fad990fb8..ea4d9b1548 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java @@ -77,7 +77,7 @@ import org.junit.Test; *

*/ -public class ViewFileSystemBaseTest { +abstract public class ViewFileSystemBaseTest { FileSystem fsView; // the view file system - the mounts are here FileSystem fsTarget; // the target file system - the mount will point here Path targetTestRoot; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java index d8ab53911c..ceebb26cb6 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java @@ -76,7 +76,7 @@ import org.mockito.Mockito; * @AfterClass public static void ClusterShutdownAtEnd() *

*/ -public class ViewFsBaseTest { +abstract public class ViewFsBaseTest { FileContext fcView; // the view file system - the mounts are here FileContext fcTarget; // the target file system - the mount will point here Path targetTestRoot; From f4d96be6c637ff54903615cff04b365e25bb3229 Mon Sep 17 00:00:00 2001 From: yliu Date: Tue, 1 Sep 2015 16:20:56 +0800 Subject: [PATCH 15/18] HADOOP-12367. Move TestFileUtil's test resources to resources folder. (wang via yliu) --- .../hadoop-common/CHANGES.txt | 3 +++ hadoop-common-project/hadoop-common/pom.xml | 19 +----------------- .../hadoop/fs => resources}/test-untar.tar | Bin .../hadoop/fs => resources}/test-untar.tgz | Bin 4 files changed, 4 insertions(+), 18 deletions(-) rename hadoop-common-project/hadoop-common/src/test/{java/org/apache/hadoop/fs => resources}/test-untar.tar (100%) rename hadoop-common-project/hadoop-common/src/test/{java/org/apache/hadoop/fs => resources}/test-untar.tgz (100%) diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index 0f52d22089..14e6fda709 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -759,6 +759,9 @@ Release 2.8.0 - UNRELEASED HADOOP-12368. Mark ViewFileSystemBaseTest and ViewFsBaseTest as abstract. (wang) + HADOOP-12367. Move TestFileUtil's test resources to resources folder. + (wang via yliu) + OPTIMIZATIONS HADOOP-11785. Reduce the number of listStatus operation in distcp diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml index 282735dd40..3ae09a0f57 100644 --- a/hadoop-common-project/hadoop-common/pom.xml +++ b/hadoop-common-project/hadoop-common/pom.xml @@ -453,23 +453,6 @@ - - copy-test-tarballs - process-test-resources - - run - - - - - - - - - - - - pre-site @@ -505,7 +488,7 @@ src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4hc.h src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4hc.c src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4hc_encoder.h - src/test/java/org/apache/hadoop/fs/test-untar.tgz + src/test/resources/test-untar.tgz src/test/resources/test.har/_SUCCESS src/test/resources/test.har/_index src/test/resources/test.har/_masterindex diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/test-untar.tar b/hadoop-common-project/hadoop-common/src/test/resources/test-untar.tar similarity index 100% rename from hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/test-untar.tar rename to hadoop-common-project/hadoop-common/src/test/resources/test-untar.tar diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/test-untar.tgz b/hadoop-common-project/hadoop-common/src/test/resources/test-untar.tgz similarity index 100% rename from hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/test-untar.tgz rename to hadoop-common-project/hadoop-common/src/test/resources/test-untar.tgz From bf669b6d9f8ba165e30b8823218d625a49958925 Mon Sep 17 00:00:00 2001 From: Varun Vasudev Date: Tue, 1 Sep 2015 14:19:11 +0530 Subject: [PATCH 16/18] YARN-4082. Container shouldn't be killed when node's label updated. Contributed by Wangda Tan. --- hadoop-yarn-project/CHANGES.txt | 3 + .../scheduler/capacity/AbstractCSQueue.java | 27 ++ .../scheduler/capacity/CSQueue.java | 26 ++ .../scheduler/capacity/CapacityScheduler.java | 40 +-- .../scheduler/capacity/LeafQueue.java | 16 ++ .../common/fica/FiCaSchedulerApp.java | 9 + .../TestCapacitySchedulerNodeLabelUpdate.java | 255 +++++++++++++++--- 7 files changed, 317 insertions(+), 59 deletions(-) diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index 80cf793784..999654dab5 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -804,6 +804,9 @@ Release 2.8.0 - UNRELEASED YARN-3896. RMNode transitioned from RUNNING to REBOOTED because its response id has not been reset synchronously. (Jun Gong via rohithsharmaks) + YARN-4082. Container shouldn't be killed when node's label updated. + (Wangda Tan via vvasudev) + Release 2.7.2 - UNRELEASED INCOMPATIBLE CHANGES diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java index 792c25c332..0ae4d1a85e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java @@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsMana import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; @@ -543,6 +544,32 @@ public abstract class AbstractCSQueue implements CSQueue { } } + @Override + public void incUsedResource(String nodeLabel, Resource resourceToInc, + SchedulerApplicationAttempt application) { + if (nodeLabel == null) { + nodeLabel = RMNodeLabelsManager.NO_LABEL; + } + // ResourceUsage has its own lock, no addition lock needs here. + queueUsage.incUsed(nodeLabel, resourceToInc); + if (null != parent) { + parent.incUsedResource(nodeLabel, resourceToInc, null); + } + } + + @Override + public void decUsedResource(String nodeLabel, Resource resourceToDec, + SchedulerApplicationAttempt application) { + if (nodeLabel == null) { + nodeLabel = RMNodeLabelsManager.NO_LABEL; + } + // ResourceUsage has its own lock, no addition lock needs here. + queueUsage.decUsed(nodeLabel, resourceToDec); + if (null != parent) { + parent.decUsedResource(nodeLabel, resourceToDec, null); + } + } + /** * Return if the queue has pending resource on given nodePartition and * schedulingMode. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java index b06a646cec..9855dd4882 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java @@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; @@ -287,4 +288,29 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue { * @return resourceUsage */ public ResourceUsage getQueueResourceUsage(); + + /** + * When partition of node updated, we will update queue's resource usage if it + * has container(s) running on that. + */ + public void incUsedResource(String nodePartition, Resource resourceToInc, + SchedulerApplicationAttempt application); + + /** + * When partition of node updated, we will update queue's resource usage if it + * has container(s) running on that. + */ + public void decUsedResource(String nodePartition, Resource resourceToDec, + SchedulerApplicationAttempt application); + + /** + * When an outstanding resource is fulfilled or canceled, calling this will + * decrease pending resource in a queue. + * + * @param nodeLabel + * asked by application + * @param resourceToDec + * new resource asked + */ + public void decPendingResource(String nodeLabel, Resource resourceToDec); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java index cff1fe5515..b5ccbd900c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java @@ -1040,12 +1040,6 @@ public class CapacityScheduler extends /** * Process node labels update on a node. - * - * TODO: Currently capacity scheduler will kill containers on a node when - * labels on the node changed. It is a simply solution to ensure guaranteed - * capacity on labels of queues. When YARN-2498 completed, we can let - * preemption policy to decide if such containers need to be killed or just - * keep them running. */ private synchronized void updateLabelsOnNode(NodeId nodeId, Set newLabels) { @@ -1060,17 +1054,31 @@ public class CapacityScheduler extends return; } - // Kill running containers since label is changed + // Get new partition, we have only one partition per node + String newPartition; + if (newLabels.isEmpty()) { + newPartition = RMNodeLabelsManager.NO_LABEL; + } else { + newPartition = newLabels.iterator().next(); + } + + // old partition as well + String oldPartition = node.getPartition(); + + // Update resources of these containers for (RMContainer rmContainer : node.getRunningContainers()) { - ContainerId containerId = rmContainer.getContainerId(); - completedContainer(rmContainer, - ContainerStatus.newInstance(containerId, - ContainerState.COMPLETE, - String.format( - "Container=%s killed since labels on the node=%s changed", - containerId.toString(), nodeId.toString()), - ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), - RMContainerEventType.KILL); + FiCaSchedulerApp application = + getApplicationAttempt(rmContainer.getApplicationAttemptId()); + if (null != application) { + application.nodePartitionUpdated(rmContainer, oldPartition, + newPartition); + } else { + LOG.warn("There's something wrong, some RMContainers running on" + + " a node, but we cannot find SchedulerApplicationAttempt for it. Node=" + + node.getNodeID() + " applicationAttemptId=" + + rmContainer.getApplicationAttemptId()); + continue; + } } // Unreserve container on this node diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java index ff1baff2ee..658eae10f8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java @@ -1262,6 +1262,22 @@ public class LeafQueue extends AbstractCSQueue { } } + @Override + public void incUsedResource(String nodeLabel, Resource resourceToInc, + SchedulerApplicationAttempt application) { + getUser(application.getUser()).getResourceUsage().incUsed(nodeLabel, + resourceToInc); + super.incUsedResource(nodeLabel, resourceToInc, application); + } + + @Override + public void decUsedResource(String nodeLabel, Resource resourceToDec, + SchedulerApplicationAttempt application) { + getUser(application.getUser()).getResourceUsage().decUsed(nodeLabel, + resourceToDec); + super.decUsedResource(nodeLabel, resourceToDec, application); + } + @VisibleForTesting public static class User { ResourceUsage userResourceUsage = new ResourceUsage(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java index 74d77f59b1..300cba9760 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java @@ -443,4 +443,13 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { schedulingMode, currentResourceLimits, reservedContainer); } } + + public void nodePartitionUpdated(RMContainer rmContainer, String oldPartition, + String newPartition) { + Resource containerResource = rmContainer.getAllocatedResource(); + this.attemptResourceUsage.decUsed(oldPartition, containerResource); + this.attemptResourceUsage.incUsed(newPartition, containerResource); + getCSLeafQueue().decUsedResource(oldPartition, containerResource, this); + getCSLeafQueue().incUsedResource(newPartition, containerResource, this); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java index 0a701d8c9d..94af4e0bff 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java @@ -19,22 +19,29 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import java.util.ArrayList; +import java.util.HashSet; +import java.util.Map; import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.MockAM; import org.apache.hadoop.yarn.server.resourcemanager.MockNM; import org.apache.hadoop.yarn.server.resourcemanager.MockRM; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeLabelsUpdateSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceInfo; import org.junit.Assert; import org.junit.Before; @@ -97,8 +104,18 @@ public class TestCapacitySchedulerNodeLabelUpdate { .getMemory()); } + private void checkUserUsedResource(MockRM rm, String queueName, + String userName, String partition, int memory) { + CapacityScheduler scheduler = (CapacityScheduler) rm.getResourceScheduler(); + LeafQueue queue = (LeafQueue) scheduler.getQueue(queueName); + LeafQueue.User user = queue.getUser(userName); + Assert.assertEquals(memory, + user.getResourceUsage().getUsed(partition).getMemory()); + } + @Test(timeout = 60000) - public void testResourceUsage() throws Exception { + public void testRequestContainerAfterNodePartitionUpdated() + throws Exception { // set node -> label mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y", "z")); @@ -160,7 +177,8 @@ public class TestCapacitySchedulerNodeLabelUpdate { } @Test (timeout = 60000) - public void testNodeUpdate() throws Exception { + public void testResourceUsageWhenNodeUpdatesPartition() + throws Exception { // set node -> label mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y", "z")); @@ -183,8 +201,9 @@ public class TestCapacitySchedulerNodeLabelUpdate { MockNM nm1 = rm.registerNode("h1:1234", 8000); MockNM nm2 = rm.registerNode("h2:1234", 8000); MockNM nm3 = rm.registerNode("h3:1234", 8000); - - ContainerId containerId; + + ContainerId containerId1; + ContainerId containerId2; // launch an app to queue a1 (label = x), and check all container will // be allocated in h1 @@ -193,9 +212,9 @@ public class TestCapacitySchedulerNodeLabelUpdate { // request a container. am1.allocate("*", GB, 1, new ArrayList(), "x"); - containerId = - ContainerId.newContainerId(am1.getApplicationAttemptId(), 2); - Assert.assertTrue(rm.waitForState(nm1, containerId, + containerId1 = ContainerId.newContainerId(am1.getApplicationAttemptId(), 1); + containerId2 = ContainerId.newContainerId(am1.getApplicationAttemptId(), 2); + Assert.assertTrue(rm.waitForState(nm1, containerId2, RMContainerState.ALLOCATED, 10 * 1000)); // check used resource: @@ -203,55 +222,205 @@ public class TestCapacitySchedulerNodeLabelUpdate { checkUsedResource(rm, "a", 1024, "x"); checkUsedResource(rm, "a", 1024); - // change h1's label to z, container should be killed - mgr.replaceLabelsOnNode(ImmutableMap.of(NodeId.newInstance("h1", 0), - toSet("z"))); - Assert.assertTrue(rm.waitForState(nm1, containerId, - RMContainerState.KILLED, 10 * 1000)); + CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); + FiCaSchedulerApp app = cs.getApplicationAttempt(am1.getApplicationAttemptId()); - // check used resource: - // queue-a used x=0G, ""=1G ("" not changed) + // change h1's label to z + cs.handle(new NodeLabelsUpdateSchedulerEvent(ImmutableMap.of(nm1.getNodeId(), + toSet("z")))); checkUsedResource(rm, "a", 0, "x"); + checkUsedResource(rm, "a", 1024, "z"); checkUsedResource(rm, "a", 1024); + checkUsedResource(rm, "root", 0, "x"); + checkUsedResource(rm, "root", 1024, "z"); + checkUsedResource(rm, "root", 1024); + checkUserUsedResource(rm, "a", "user", "x", 0); + checkUserUsedResource(rm, "a", "user", "z", 1024); + Assert.assertEquals(0, + app.getAppAttemptResourceUsage().getUsed("x").getMemory()); + Assert.assertEquals(1024, + app.getAppAttemptResourceUsage().getUsed("z").getMemory()); - // request a container with label = y - am1.allocate("*", GB, 1, new ArrayList(), "y"); + // change h1's label to y + cs.handle(new NodeLabelsUpdateSchedulerEvent(ImmutableMap.of(nm1.getNodeId(), + toSet("y")))); + checkUsedResource(rm, "a", 0, "x"); + checkUsedResource(rm, "a", 1024, "y"); + checkUsedResource(rm, "a", 0, "z"); + checkUsedResource(rm, "a", 1024); + checkUsedResource(rm, "root", 0, "x"); + checkUsedResource(rm, "root", 1024, "y"); + checkUsedResource(rm, "root", 0, "z"); + checkUsedResource(rm, "root", 1024); + checkUserUsedResource(rm, "a", "user", "x", 0); + checkUserUsedResource(rm, "a", "user", "y", 1024); + checkUserUsedResource(rm, "a", "user", "z", 0); + Assert.assertEquals(0, + app.getAppAttemptResourceUsage().getUsed("x").getMemory()); + Assert.assertEquals(1024, + app.getAppAttemptResourceUsage().getUsed("y").getMemory()); + Assert.assertEquals(0, + app.getAppAttemptResourceUsage().getUsed("z").getMemory()); + + // change h1's label to no label + Set emptyLabels = new HashSet<>(); + Map> map = ImmutableMap.of(nm1.getNodeId(), + emptyLabels); + cs.handle(new NodeLabelsUpdateSchedulerEvent(map)); + checkUsedResource(rm, "a", 0, "x"); + checkUsedResource(rm, "a", 0, "y"); + checkUsedResource(rm, "a", 0, "z"); + checkUsedResource(rm, "a", 2048); + checkUsedResource(rm, "root", 0, "x"); + checkUsedResource(rm, "root", 0, "y"); + checkUsedResource(rm, "root", 0, "z"); + checkUsedResource(rm, "root", 2048); + checkUserUsedResource(rm, "a", "user", "x", 0); + checkUserUsedResource(rm, "a", "user", "y", 0); + checkUserUsedResource(rm, "a", "user", "z", 0); + checkUserUsedResource(rm, "a", "user", "", 2048); + Assert.assertEquals(0, + app.getAppAttemptResourceUsage().getUsed("x").getMemory()); + Assert.assertEquals(0, + app.getAppAttemptResourceUsage().getUsed("y").getMemory()); + Assert.assertEquals(0, + app.getAppAttemptResourceUsage().getUsed("z").getMemory()); + Assert.assertEquals(2048, + app.getAppAttemptResourceUsage().getUsed("").getMemory()); + + // Finish the two containers, we should see used resource becomes 0 + cs.completedContainer(cs.getRMContainer(containerId2), + ContainerStatus.newInstance(containerId2, ContainerState.COMPLETE, "", + ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), + RMContainerEventType.KILL); + cs.completedContainer(cs.getRMContainer(containerId1), + ContainerStatus.newInstance(containerId1, ContainerState.COMPLETE, "", + ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), + RMContainerEventType.KILL); + + checkUsedResource(rm, "a", 0, "x"); + checkUsedResource(rm, "a", 0, "y"); + checkUsedResource(rm, "a", 0, "z"); + checkUsedResource(rm, "a", 0); + checkUsedResource(rm, "root", 0, "x"); + checkUsedResource(rm, "root", 0, "y"); + checkUsedResource(rm, "root", 0, "z"); + checkUsedResource(rm, "root", 0); + checkUserUsedResource(rm, "a", "user", "x", 0); + checkUserUsedResource(rm, "a", "user", "y", 0); + checkUserUsedResource(rm, "a", "user", "z", 0); + checkUserUsedResource(rm, "a", "user", "", 0); + + rm.close(); + } + + + @Test (timeout = 60000) + public void testComplexResourceUsageWhenNodeUpdatesPartition() + throws Exception { + /* + * This test is similar to testResourceUsageWhenNodeUpdatesPartition, this + * will include multiple applications, multiple users and multiple + * containers running on a single node, size of each container is 1G + * + * Node 1 + * ------ + * App1-container3 + * App2-container2 + * App2-Container3 + * + * Node 2 + * ------ + * App2-container1 + * App1-container1 + * App1-container2 + */ + // set node -> label + mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y", "z")); + + // set mapping: + // h1 -> x + // h2 -> y + mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"))); + + // inject node label manager + MockRM rm = new MockRM(getConfigurationWithQueueLabels(conf)) { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + + rm.getRMContext().setNodeLabelManager(mgr); + rm.start(); + MockNM nm1 = rm.registerNode("h1:1234", 80000); + MockNM nm2 = rm.registerNode("h2:1234", 80000); + + // app1 + RMApp app1 = rm.submitApp(GB, "app", "u1", null, "a"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2); + + // c2 on n1, c3 on n2 + am1.allocate("*", GB, 1, new ArrayList(), "x"); + ContainerId containerId = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 2); + Assert.assertTrue(rm.waitForState(nm1, containerId, + RMContainerState.ALLOCATED, 10 * 1000)); + am1.allocate("*", GB, 1, new ArrayList()); containerId = ContainerId.newContainerId(am1.getApplicationAttemptId(), 3); Assert.assertTrue(rm.waitForState(nm2, containerId, RMContainerState.ALLOCATED, 10 * 1000)); - // check used resource: - // queue-a used y=1G, ""=1G - checkUsedResource(rm, "a", 1024, "y"); - checkUsedResource(rm, "a", 1024); - - // change h2's label to no label, container should be killed - mgr.replaceLabelsOnNode(ImmutableMap.of(NodeId.newInstance("h2", 0), - CommonNodeLabelsManager.EMPTY_STRING_SET)); - Assert.assertTrue(rm.waitForState(nm1, containerId, - RMContainerState.KILLED, 10 * 1000)); - - // check used resource: - // queue-a used x=0G, y=0G, ""=1G ("" not changed) - checkUsedResource(rm, "a", 0, "x"); - checkUsedResource(rm, "a", 0, "y"); - checkUsedResource(rm, "a", 1024); - + // app2 + RMApp app2 = rm.submitApp(GB, "app", "u2", null, "a"); + MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2); + + // c2/c3 on n1 + am2.allocate("*", GB, 2, new ArrayList(), "x"); containerId = - ContainerId.newContainerId(am1.getApplicationAttemptId(), 1); - - // change h3's label to z, AM container should be killed - mgr.replaceLabelsOnNode(ImmutableMap.of(NodeId.newInstance("h3", 0), - toSet("z"))); + ContainerId.newContainerId(am2.getApplicationAttemptId(), 3); Assert.assertTrue(rm.waitForState(nm1, containerId, - RMContainerState.KILLED, 10 * 1000)); + RMContainerState.ALLOCATED, 10 * 1000)); // check used resource: - // queue-a used x=0G, y=0G, ""=1G ("" not changed) + // queue-a used x=1G, ""=1G + checkUsedResource(rm, "a", 3 * GB, "x"); + checkUsedResource(rm, "a", 3 * GB); + + CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); + FiCaSchedulerApp application1 = + cs.getApplicationAttempt(am1.getApplicationAttemptId()); + FiCaSchedulerApp application2 = + cs.getApplicationAttempt(am2.getApplicationAttemptId()); + + // change h1's label to z + cs.handle(new NodeLabelsUpdateSchedulerEvent(ImmutableMap.of(nm1.getNodeId(), + toSet("z")))); checkUsedResource(rm, "a", 0, "x"); - checkUsedResource(rm, "a", 0, "y"); - checkUsedResource(rm, "a", 0); + checkUsedResource(rm, "a", 3 * GB, "z"); + checkUsedResource(rm, "a", 3 * GB); + checkUsedResource(rm, "root", 0, "x"); + checkUsedResource(rm, "root", 3 * GB, "z"); + checkUsedResource(rm, "root", 3 * GB); + checkUserUsedResource(rm, "a", "u1", "x", 0 * GB); + checkUserUsedResource(rm, "a", "u1", "z", 1 * GB); + checkUserUsedResource(rm, "a", "u1", "", 2 * GB); + checkUserUsedResource(rm, "a", "u2", "x", 0 * GB); + checkUserUsedResource(rm, "a", "u2", "z", 2 * GB); + checkUserUsedResource(rm, "a", "u2", "", 1 * GB); + Assert.assertEquals(0, + application1.getAppAttemptResourceUsage().getUsed("x").getMemory()); + Assert.assertEquals(1 * GB, + application1.getAppAttemptResourceUsage().getUsed("z").getMemory()); + Assert.assertEquals(2 * GB, + application1.getAppAttemptResourceUsage().getUsed("").getMemory()); + Assert.assertEquals(0, + application2.getAppAttemptResourceUsage().getUsed("x").getMemory()); + Assert.assertEquals(2 * GB, + application2.getAppAttemptResourceUsage().getUsed("z").getMemory()); + Assert.assertEquals(1 * GB, + application2.getAppAttemptResourceUsage().getUsed("").getMemory()); rm.close(); } From faa38e1aa49907254bf981662a8aeb5dc52e75e3 Mon Sep 17 00:00:00 2001 From: Tsuyoshi Ozawa Date: Tue, 1 Sep 2015 17:54:14 +0900 Subject: [PATCH 17/18] YARN-2801. Add documentation for node labels feature. Contributed by Wangda Tan and Naganarasimha G R. --- hadoop-project/src/site/site.xml | 1 + hadoop-yarn-project/CHANGES.txt | 3 + .../src/site/markdown/NodeLabel.md | 140 ++++++++++++++++++ 3 files changed, 144 insertions(+) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml index ee0dfcdc16..272ea7bbf3 100644 --- a/hadoop-project/src/site/site.xml +++ b/hadoop-project/src/site/site.xml @@ -121,6 +121,7 @@ + diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index 999654dab5..e2b13078f4 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -827,6 +827,9 @@ Release 2.7.2 - UNRELEASED YARN-4092. Fixed UI redirection to print useful messages when both RMs are in standby mode. (Xuan Gong via jianhe) + YARN-2801. Add documentation for node labels feature. (Wangda Tan and Naganarasimha + G R via ozawa) + OPTIMIZATIONS BUG FIXES diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md new file mode 100644 index 0000000000..87019cddba --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md @@ -0,0 +1,140 @@ + + +YARN Node Labels +=============== + +# Overview +Node label is a way to group nodes with similar characteristics and applications can specify where to run. + +Now we only support node partition, which is: + +* One node can have only one node partition, so a cluster is partitioned to several disjoint sub-clusters by node partitions. By default, nodes belong to DEFAULT partition (partition="") +* User need to configure how much resources of each partition can be used by different queues. For more detail, please refer next section. +* There are two kinds of node partitions: + * Exclusive: containers will be allocated to nodes with exactly match node partition. (e.g. asking partition=“x” will be allocated to node with partition=“x”, asking DEFAULT partition will be allocated to DEFAULT partition nodes). + * Non-exclusive: if a partition is non-exclusive, it shares idle resource to container requesting DEFAULT partition. + +User can specify set of node labels which can be accessed by each queue, one application can only use subset of node labels that can be accessed by the queue which contains the application. + +# Features +The ```Node Labels``` supports the following features for now: + +* Partition cluster - each node can be assigned one label, so the cluster will be divided to several smaller disjoint partitions. +* ACL of node-labels on queues - user can set accessible node labels on each queue so only some nodes can only be accessed by specific queues. +* Specify percentage of resource of a partition which can be accessed by a queue - user can set percentage like: queue A can access 30% of resources on nodes with label=hbase. Such percentage setting will be consistent with existing resource manager +* Specify required Node Label in resource request, it will only be allocated when node has the same label. If no node label requirement specified, such Resource Request will only be allocated on nodes belong to DEFAULT partition. +* Operability + * Node labels and node labels mapping can be recovered across RM restart + * Update node labels - admin can update labels on nodes and labels on queues + when RM is running + +# Configuration +## Setting up ```ResourceManager``` to enable ```Node Labels```: + +Setup following properties in ```yarn-site.xml``` + +Property | Value +--- | ---- +yarn.node-labels.fs-store.root-dir | hdfs://namenode:port/path/to/store/node-labels/ +yarn.node-labels.enabled | true + +Notes: + +* Make sure ```yarn.node-labels.fs-store.root-dir``` is created and ```ResourceManager``` has permission to access it. (Typically from “yarn” user) +* If user want to store node label to local file system of RM (instead of HDFS), paths like `file:///home/yarn/node-label` can be used + +### Add/modify node labels list and node-to-labels mapping to YARN +* Add cluster node labels list: + * Executing ```yarn rmadmin -addToClusterNodeLabels "label_1(exclusive=true/false),label_2(exclusive=true/false)"``` to add node label. + * If user don’t specify “(exclusive=…)”, execlusive will be ```true``` by default. + * Run ```yarn cluster --list-node-labels``` to check added node labels are visible in the cluster. + +* Add labels to nodes + * Executing ```yarn rmadmin -replaceLabelsOnNode “node1[:port]=label1 node2=label2”```. Added label1 to node1, label2 to node2. If user don’t specify port, it added the label to all ```NodeManagers``` running on the node. + +## Configuration of Schedulers for node labels +### Capacity Scheduler Configuration +Property | Value +----- | ------ +yarn.scheduler.capacity.``.capacity | Set the percentage of the queue can access to nodes belong to DEFAULT partition. The sum of DEFAULT capacities for direct children under each parent, must be equal to 100. +yarn.scheduler.capacity.``.accessible-node-labels | Admin need specify labels can be accessible by each queue, split by comma, like “hbase,storm” means queue can access label hbase and storm. All queues can access to nodes without label, user don’t have to specify that. If user don’t specify this field, it will inherit from its parent. If user want to explicitly specify a queue can only access nodes without labels, just put a space as the value. +yarn.scheduler.capacity.``.accessible-node-labels.`