From cf1b3711cb6f2d5ca7fb13a5a0fafaaa12ee8095 Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Thu, 26 Jan 2023 19:45:54 +0100 Subject: [PATCH 01/50] YARN-10965. Centralize queue resource calculation based on CapacityVectors. Contributed by Andras Gyori --- .../yarn/util/resource/ResourceUtils.java | 22 + .../AbsoluteResourceCapacityCalculator.java | 135 +++++ .../scheduler/capacity/AbstractCSQueue.java | 37 +- .../scheduler/capacity/AbstractLeafQueue.java | 50 +- .../AbstractQueueCapacityCalculator.java | 109 ++++ .../scheduler/capacity/CSQueue.java | 39 +- .../capacity/CSQueueUsageTracker.java | 1 + .../capacity/CalculationContext.java | 72 +++ .../CapacitySchedulerConfiguration.java | 57 +- ...CapacitySchedulerQueueCapacityHandler.java | 221 ++++++++ .../CapacitySchedulerQueueManager.java | 6 + .../DefaultQueueResourceRoundingStrategy.java | 48 ++ .../scheduler/capacity/ParentQueue.java | 180 +++--- .../PercentageQueueCapacityCalculator.java | 72 +++ .../capacity/QueueCapacityUpdateContext.java | 76 +++ .../capacity/QueueCapacityVector.java | 78 ++- .../QueueResourceRoundingStrategy.java | 36 ++ .../capacity/QueueUpdateWarning.java | 78 +++ .../capacity/ResourceCalculationDriver.java | 336 +++++++++++ .../scheduler/capacity/ResourceVector.java | 41 +- .../capacity/RootCalculationDriver.java | 64 +++ .../capacity/RootQueueCapacityCalculator.java | 59 ++ .../WeightQueueCapacityCalculator.java | 103 ++++ .../conf/QueueCapacityConfigParser.java | 32 +- .../nodelabels/NullRMNodeLabelsManager.java | 22 + ...citySchedulerQueueCalculationTestBase.java | 131 +++++ .../capacity/QueueAssertionBuilder.java | 210 +++++++ .../TestMixedQueueResourceCalculation.java | 536 ++++++++++++++++++ .../capacity/TestQueueCapacityVector.java | 36 +- .../capacity/TestResourceVector.java | 6 +- .../TestUniformQueueResourceCalculation.java | 191 +++++++ .../conf/TestQueueCapacityConfigParser.java | 96 ++-- 32 files changed, 2949 insertions(+), 231 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbsoluteResourceCapacityCalculator.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractQueueCapacityCalculator.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CalculationContext.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueCapacityHandler.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/DefaultQueueResourceRoundingStrategy.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PercentageQueueCapacityCalculator.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueCapacityUpdateContext.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueResourceRoundingStrategy.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueUpdateWarning.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ResourceCalculationDriver.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/RootCalculationDriver.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/RootQueueCapacityCalculator.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/WeightQueueCapacityCalculator.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueCalculationTestBase.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueAssertionBuilder.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestMixedQueueResourceCalculation.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUniformQueueResourceCalculation.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java index 1433c24e9e1..cde38219fcc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java @@ -818,6 +818,28 @@ public class ResourceUtils { return res; } + public static Resource multiplyFloor(Resource resource, double multiplier) { + Resource newResource = Resource.newInstance(0, 0); + + for (ResourceInformation resourceInformation : resource.getResources()) { + newResource.setResourceValue(resourceInformation.getName(), + (long) Math.floor(resourceInformation.getValue() * multiplier)); + } + + return newResource; + } + + public static Resource multiplyRound(Resource resource, double multiplier) { + Resource newResource = Resource.newInstance(0, 0); + + for (ResourceInformation resourceInformation : resource.getResources()) { + newResource.setResourceValue(resourceInformation.getName(), + Math.round(resourceInformation.getValue() * multiplier)); + } + + return newResource; + } + @InterfaceAudience.Private @InterfaceStability.Unstable public static Resource createResourceFromString( 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java new file mode 100644 index 00000000000..33b45741079 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbsoluteResourceCapacityCalculator.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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; +import org.apache.hadoop.yarn.util.UnitsConversionUtil; + +import java.util.Map; + +import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueUpdateWarning.QueueUpdateWarningType.BRANCH_DOWNSCALED; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ResourceCalculationDriver.MB_UNIT; + +public class AbsoluteResourceCapacityCalculator extends AbstractQueueCapacityCalculator { + + @Override + public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCalculationDriver) { + setNormalizedResourceRatio(resourceCalculationDriver); + } + + @Override + public double calculateMinimumResource( + ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, + String label) { + String resourceName = context.getResourceName(); + double normalizedRatio = resourceCalculationDriver.getNormalizedResourceRatios().getOrDefault( + label, ResourceVector.of(1)).getValue(resourceName); + double remainingResourceRatio = resourceCalculationDriver.getRemainingRatioOfResource( + label, resourceName); + + return normalizedRatio * remainingResourceRatio * context.getCurrentMinimumCapacityEntry( + label).getResourceValue(); + } + + @Override + public double calculateMaximumResource( + ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, + String label) { + return context.getCurrentMaximumCapacityEntry(label).getResourceValue(); + } + + @Override + public void updateCapacitiesAfterCalculation( + ResourceCalculationDriver resourceCalculationDriver, CSQueue queue, String label) { + CapacitySchedulerQueueCapacityHandler.setQueueCapacities( + resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label), queue, label); + } + + @Override + public ResourceUnitCapacityType getCapacityType() { + return ResourceUnitCapacityType.ABSOLUTE; + } + + /** + * Calculates the normalized resource ratio of a parent queue, under which children are defined + * with absolute capacity type. If the effective resource of the parent is less, than the + * aggregated configured absolute resource of its children, the resource ratio will be less, + * than 1. + * + * @param calculationDriver the driver, which contains the parent queue that will form the base + * of the normalization calculation + */ + public static void setNormalizedResourceRatio(ResourceCalculationDriver calculationDriver) { + CSQueue queue = calculationDriver.getQueue(); + + for (String label : queue.getConfiguredNodeLabels()) { + // ManagedParents assign zero capacity to queues in case of overutilization, downscaling is + // turned off for their children + if (queue instanceof ManagedParentQueue) { + return; + } + + for (String resourceName : queue.getConfiguredCapacityVector(label).getResourceNames()) { + long childrenConfiguredResource = 0; + long effectiveMinResource = queue.getQueueResourceQuotas().getEffectiveMinResource( + label).getResourceValue(resourceName); + + // Total configured min resources of direct children of the queue + for (CSQueue childQueue : queue.getChildQueues()) { + if (!childQueue.getConfiguredNodeLabels().contains(label)) { + continue; + } + QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); + if (capacityVector.isResourceOfType(resourceName, ResourceUnitCapacityType.ABSOLUTE)) { + childrenConfiguredResource += capacityVector.getResource(resourceName) + .getResourceValue(); + } + } + // If no children is using ABSOLUTE capacity type, normalization is not needed + if (childrenConfiguredResource == 0) { + continue; + } + // Factor to scale down effective resource: When cluster has sufficient + // resources, effective_min_resources will be same as configured + // min_resources. + float numeratorForMinRatio = childrenConfiguredResource; + if (effectiveMinResource < childrenConfiguredResource) { + numeratorForMinRatio = queue.getQueueResourceQuotas().getEffectiveMinResource(label) + .getResourceValue(resourceName); + calculationDriver.getUpdateContext().addUpdateWarning(BRANCH_DOWNSCALED.ofQueue( + queue.getQueuePath())); + } + + String unit = resourceName.equals(MEMORY_URI) ? MB_UNIT : ""; + long convertedValue = UnitsConversionUtil.convert(unit, calculationDriver.getUpdateContext() + .getUpdatedClusterResource(label).getResourceInformation(resourceName).getUnits(), + childrenConfiguredResource); + + if (convertedValue != 0) { + Map normalizedResourceRatios = + calculationDriver.getNormalizedResourceRatios(); + normalizedResourceRatios.putIfAbsent(label, ResourceVector.newInstance()); + normalizedResourceRatios.get(label).setValue(resourceName, numeratorForMinRatio / + convertedValue); + } + } + } + } +} 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 7a3ec4f6f4e..f9304cc9604 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 @@ -115,6 +115,7 @@ public abstract class AbstractCSQueue implements CSQueue { CapacityConfigType.NONE; protected Map configuredCapacityVectors; + protected Map configuredMaxCapacityVectors; private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); @@ -379,7 +380,10 @@ public abstract class AbstractCSQueue implements CSQueue { this.configuredCapacityVectors = configuration .parseConfiguredResourceVector(queuePath.getFullPath(), this.queueNodeLabelsSettings.getConfiguredNodeLabels()); - + this.configuredMaxCapacityVectors = configuration + .parseConfiguredMaximumCapacityVector(queuePath.getFullPath(), + this.queueNodeLabelsSettings.getConfiguredNodeLabels(), + QueueCapacityVector.newInstance()); // Update metrics CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, this, labelManager, null); @@ -533,7 +537,8 @@ public abstract class AbstractCSQueue implements CSQueue { private void validateAbsoluteVsPercentageCapacityConfig( CapacityConfigType localType) { if (!queuePath.isRoot() - && !this.capacityConfigType.equals(localType)) { + && !this.capacityConfigType.equals(localType) && + queueContext.getConfiguration().isLegacyQueueMode()) { throw new IllegalArgumentException("Queue '" + getQueuePath() + "' should use either percentage based capacity" + " configuration or absolute resource."); @@ -572,11 +577,25 @@ public abstract class AbstractCSQueue implements CSQueue { } @Override - public QueueCapacityVector getConfiguredCapacityVector( - String label) { + public QueueCapacityVector getConfiguredCapacityVector(String label) { return configuredCapacityVectors.get(label); } + @Override + public QueueCapacityVector getConfiguredMaxCapacityVector(String label) { + return configuredMaxCapacityVectors.get(label); + } + + @Override + public void setConfiguredMinCapacityVector(String label, QueueCapacityVector minCapacityVector) { + configuredCapacityVectors.put(label, minCapacityVector); + } + + @Override + public void setConfiguredMaxCapacityVector(String label, QueueCapacityVector maxCapacityVector) { + configuredMaxCapacityVectors.put(label, maxCapacityVector); + } + protected QueueInfo getQueueInfo() { // Deliberately doesn't use lock here, because this method will be invoked // from schedulerApplicationAttempt, to avoid deadlock, sacrifice @@ -691,6 +710,11 @@ public abstract class AbstractCSQueue implements CSQueue { return readLock; } + @Override + public ReentrantReadWriteLock.WriteLock getWriteLock() { + return writeLock; + } + private Resource getCurrentLimitResource(String nodePartition, Resource clusterResource, ResourceLimits currentResourceLimits, SchedulingMode schedulingMode) { @@ -827,6 +851,11 @@ public abstract class AbstractCSQueue implements CSQueue { } + @Override + public Set getConfiguredNodeLabels() { + return queueNodeLabelsSettings.getConfiguredNodeLabels(); + } + private static String ensurePartition(String partition) { return Optional.ofNullable(partition).orElse(NO_LABEL); } 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/AbstractLeafQueue.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/AbstractLeafQueue.java index 08fedb578ca..72ea63a2fc5 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/AbstractLeafQueue.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/AbstractLeafQueue.java @@ -88,6 +88,9 @@ import org.slf4j.LoggerFactory; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getACLsForFlexibleAutoCreatedLeafQueue; +import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NO_LABEL; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.PERCENTAGE; + public class AbstractLeafQueue extends AbstractCSQueue { private static final Logger LOG = LoggerFactory.getLogger(AbstractLeafQueue.class); @@ -164,7 +167,7 @@ public class AbstractLeafQueue extends AbstractCSQueue { resourceCalculator); // One time initialization is enough since it is static ordering policy - this.pendingOrderingPolicy = new FifoOrderingPolicyForPendingApps(); + this.pendingOrderingPolicy = new FifoOrderingPolicyForPendingApps<>(); } @SuppressWarnings("checkstyle:nowhitespaceafter") @@ -1936,6 +1939,49 @@ public class AbstractLeafQueue extends AbstractCSQueue { currentResourceLimits.getLimit())); } + @Override + public void refreshAfterResourceCalculation(Resource clusterResource, + ResourceLimits resourceLimits) { + lastClusterResource = clusterResource; + // Update maximum applications for the queue and for users + updateMaximumApplications(); + + updateCurrentResourceLimits(resourceLimits, clusterResource); + + // Update headroom info based on new cluster resource value + // absoluteMaxCapacity now, will be replaced with absoluteMaxAvailCapacity + // during allocation + setQueueResourceLimitsInfo(clusterResource); + + // Update user consumedRatios + recalculateQueueUsageRatio(clusterResource, null); + + // Update metrics + CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, + this, labelManager, null); + // Update configured capacity/max-capacity for default partition only + CSQueueUtils.updateConfiguredCapacityMetrics(resourceCalculator, + labelManager.getResourceByLabel(null, clusterResource), + NO_LABEL, this); + + // queue metrics are updated, more resource may be available + // activate the pending applications if possible + activateApplications(); + + // In case of any resource change, invalidate recalculateULCount to clear + // the computed user-limit. + usersManager.userLimitNeedsRecompute(); + + // Update application properties + for (FiCaSchedulerApp application : orderingPolicy + .getSchedulableEntities()) { + computeUserLimitAndSetHeadroom(application, clusterResource, + NO_LABEL, + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY, null); + + } + } + @Override public void updateClusterResource(Resource clusterResource, ResourceLimits currentResourceLimits) { @@ -2225,10 +2271,12 @@ public class AbstractLeafQueue extends AbstractCSQueue { } public void setCapacity(float capacity) { + configuredCapacityVectors.put(NO_LABEL, QueueCapacityVector.of(capacity * 100, PERCENTAGE)); queueCapacities.setCapacity(capacity); } public void setCapacity(String nodeLabel, float capacity) { + configuredCapacityVectors.put(nodeLabel, QueueCapacityVector.of(capacity * 100, PERCENTAGE)); queueCapacities.setCapacity(nodeLabel, capacity); } 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java new file mode 100644 index 00000000000..8b48da88ff8 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractQueueCapacityCalculator.java @@ -0,0 +1,109 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; +import java.util.Set; + +/** + * A strategy class to encapsulate queue capacity setup and resource calculation + * logic. + */ +public abstract class AbstractQueueCapacityCalculator { + + /** + * Sets the metrics and statistics after effective resource values calculation. + * + * @param queue the queue on which the calculations are based + * @param resourceCalculationDriver driver that contains the intermediate calculation results for + * a queue branch + * @param label node label + */ + public abstract void updateCapacitiesAfterCalculation( + ResourceCalculationDriver resourceCalculationDriver, CSQueue queue, String label); + + + /** + * Returns the capacity type the calculator could handle. + * + * @return capacity type + */ + public abstract ResourceUnitCapacityType getCapacityType(); + + /** + * Calculates the minimum effective resource. + * + * @param resourceCalculationDriver driver that contains the intermediate calculation results for + * a queue branch + * @param context the units evaluated in the current iteration phase + * @param label node label + * @return minimum effective resource + */ + public abstract double calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, + CalculationContext context, + String label); + + /** + * Calculates the maximum effective resource. + * + * @param resourceCalculationDriver driver that contains the intermediate calculation results for + * a queue branch + * @param context the units evaluated in the current iteration phase + * @param label node label + * @return minimum effective resource + */ + public abstract double calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, + CalculationContext context, + String label); + + /** + * Executes all logic that must be called prior to the effective resource value calculations. + * + * @param resourceCalculationDriver driver that contains the parent queue on which the + * prerequisite calculation should be made + */ + public abstract void calculateResourcePrerequisites( + ResourceCalculationDriver resourceCalculationDriver); + + /** + * Returns all resource names that are defined for the capacity type that is + * handled by the calculator. + * + * @param queue queue for which the capacity vector is defined + * @param label node label + * @return resource names + */ + protected Set getResourceNames(CSQueue queue, String label) { + return getResourceNames(queue, label, getCapacityType()); + } + + /** + * Returns all resource names that are defined for a capacity type. + * + * @param queue queue for which the capacity vector is defined + * @param label node label + * @param capacityType capacity type for which the resource names are defined + * @return resource names + */ + protected Set getResourceNames(CSQueue queue, String label, + ResourceUnitCapacityType capacityType) { + return queue.getConfiguredCapacityVector(label) + .getResourceNamesByCapacityType(capacityType); + } +} 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 e2aeaab4180..91dab98ce76 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 @@ -121,7 +121,7 @@ public interface CSQueue extends SchedulerQueue { * cumulative capacity in the cluster */ public float getAbsoluteCapacity(); - + /** * Get the configured maximum-capacity of the queue. * @return the configured maximum-capacity of the queue @@ -169,7 +169,7 @@ public interface CSQueue extends SchedulerQueue { * @return max-parallel-applications */ public int getMaxParallelApps(); - + /** * Get child queues * @return child queues @@ -270,6 +270,9 @@ public interface CSQueue extends SchedulerQueue { public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource) throws IOException; + public void refreshAfterResourceCalculation( + Resource clusterResource, ResourceLimits resourceLimits); + /** * Update the cluster resource for queues as we add/remove nodes * @param clusterResource the current cluster resource @@ -388,6 +391,12 @@ public interface CSQueue extends SchedulerQueue { */ public ReentrantReadWriteLock.ReadLock getReadLock(); + /** + * Get writeLock associated with the Queue. + * @return writeLock of corresponding queue. + */ + ReentrantReadWriteLock.WriteLock getWriteLock(); + /** * Validate submitApplication api so that moveApplication do a pre-check. * @param applicationId Application ID @@ -433,13 +442,37 @@ public interface CSQueue extends SchedulerQueue { Resource getEffectiveCapacity(String label); /** - * Get configured capacity resource vector parsed from the capacity config + * Get configured capacity vector parsed from the capacity config * of the queue. * @param label node label (partition) * @return capacity resource vector */ QueueCapacityVector getConfiguredCapacityVector(String label); + /** + * Get configured maximum capacity vector parsed from the capacity config + * of the queue. + * @param label node label (partition) + * @return capacity resource vector + */ + QueueCapacityVector getConfiguredMaxCapacityVector(String label); + + /** + * Sets the configured minimum capacity vector to a specific value. + * @param label node label (partition) + * @param minCapacityVector capacity vector + */ + void setConfiguredMinCapacityVector(String label, QueueCapacityVector minCapacityVector); + + /** + * Sets the configured maximum capacity vector to a specific value. + * @param label node label (partition) + * @param maxCapacityVector capacity vector + */ + void setConfiguredMaxCapacityVector(String label, QueueCapacityVector maxCapacityVector); + + Set getConfiguredNodeLabels(); + /** * Get effective capacity of queue. If min/max resource is configured, * preference will be given to absolute configuration over normal capacity. 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/CSQueueUsageTracker.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/CSQueueUsageTracker.java index 0f18e944e9a..dd6b9b17ac0 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/CSQueueUsageTracker.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/CSQueueUsageTracker.java @@ -75,4 +75,5 @@ public class CSQueueUsageTracker { public QueueResourceQuotas getQueueResourceQuotas() { return queueResourceQuotas; } + } 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/CalculationContext.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/CalculationContext.java new file mode 100644 index 00000000000..7ec85e19b1f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CalculationContext.java @@ -0,0 +1,72 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; + +/** + * A storage class that wraps arguments used in a resource calculation iteration. + */ +public class CalculationContext { + private final String resourceName; + private final ResourceUnitCapacityType capacityType; + private final CSQueue queue; + + public CalculationContext(String resourceName, ResourceUnitCapacityType capacityType, + CSQueue queue) { + this.resourceName = resourceName; + this.capacityType = capacityType; + this.queue = queue; + } + + public String getResourceName() { + return resourceName; + } + + public ResourceUnitCapacityType getCapacityType() { + return capacityType; + } + + public CSQueue getQueue() { + return queue; + } + + /** + * A shorthand to return the minimum capacity vector entry for the currently evaluated child and + * resource name. + * + * @param label node label + * @return capacity vector entry + */ + public QueueCapacityVectorEntry getCurrentMinimumCapacityEntry(String label) { + return queue.getConfiguredCapacityVector(label).getResource(resourceName); + } + + /** + * A shorthand to return the maximum capacity vector entry for the currently evaluated child and + * resource name. + * + * @param label node label + * @return capacity vector entry + */ + public QueueCapacityVectorEntry getCurrentMaximumCapacityEntry(String label) { + return queue.getConfiguredMaxCapacityVector(label).getResource(resourceName); + } +} 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java index 51616da14b6..757120e1621 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java @@ -423,6 +423,8 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur private static final QueueCapacityConfigParser queueCapacityConfigParser = new QueueCapacityConfigParser(); + private static final String LEGACY_QUEUE_MODE_ENABLED = PREFIX + "legacy-queue-mode.enabled"; + public static final boolean DEFAULT_LEGACY_QUEUE_MODE = true; private ConfigurationProperties configurationProperties; @@ -572,8 +574,10 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur String configuredCapacity = get(getQueuePrefix(queue.getFullPath()) + CAPACITY); boolean absoluteResourceConfigured = (configuredCapacity != null) && RESOURCE_PATTERN.matcher(configuredCapacity).find(); + boolean isCapacityVectorFormat = queueCapacityConfigParser + .isCapacityVectorFormat(configuredCapacity); if (absoluteResourceConfigured || configuredWeightAsCapacity( - configuredCapacity)) { + configuredCapacity) || isCapacityVectorFormat) { // Return capacity in percentage as 0 for non-root queues and 100 for // root.From AbstractCSQueue, absolute resource will be parsed and // updated. Once nodes are added/removed in cluster, capacity in @@ -623,7 +627,8 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur public float getNonLabeledQueueMaximumCapacity(QueuePath queue) { String configuredCapacity = get(getQueuePrefix(queue.getFullPath()) + MAXIMUM_CAPACITY); boolean matcher = (configuredCapacity != null) - && RESOURCE_PATTERN.matcher(configuredCapacity).find(); + && RESOURCE_PATTERN.matcher(configuredCapacity).find() + || queueCapacityConfigParser.isCapacityVectorFormat(configuredCapacity); if (matcher) { // Return capacity in percentage as 0 for non-root queues and 100 for // root.From AbstractCSQueue, absolute resource will be parsed and @@ -819,6 +824,16 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur return Collections.unmodifiableSet(set); } + public void setCapacityVector(String queuePath, String label, String capacityVector) { + String capacityPropertyName = getNodeLabelPrefix(queuePath, label) + CAPACITY; + set(capacityPropertyName, capacityVector); + } + + public void setMaximumCapacityVector(String queuePath, String label, String capacityVector) { + String capacityPropertyName = getNodeLabelPrefix(queuePath, label) + MAXIMUM_CAPACITY; + set(capacityPropertyName, capacityVector); + } + private boolean configuredWeightAsCapacity(String configureValue) { if (configureValue == null) { return false; @@ -843,7 +858,7 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur (configuredCapacity != null) && RESOURCE_PATTERN.matcher( configuredCapacity).find(); if (absoluteResourceConfigured || configuredWeightAsCapacity( - configuredCapacity)) { + configuredCapacity) || queueCapacityConfigParser.isCapacityVectorFormat(configuredCapacity)) { // Return capacity in percentage as 0 for non-root queues and 100 for // root.From AbstractCSQueue, absolute resource, and weight will be parsed // and updated separately. Once nodes are added/removed in cluster, @@ -2701,7 +2716,28 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur String queuePath, Set labels) { Map queueResourceVectors = new HashMap<>(); for (String label : labels) { - queueResourceVectors.put(label, queueCapacityConfigParser.parse(this, queuePath, label)); + String propertyName = CapacitySchedulerConfiguration.getNodeLabelPrefix( + queuePath, label) + CapacitySchedulerConfiguration.CAPACITY; + String capacityString = get(propertyName); + queueResourceVectors.put(label, queueCapacityConfigParser.parse(capacityString, queuePath)); + } + + return queueResourceVectors; + } + + public Map parseConfiguredMaximumCapacityVector( + String queuePath, Set labels, QueueCapacityVector defaultVector) { + Map queueResourceVectors = new HashMap<>(); + for (String label : labels) { + String propertyName = CapacitySchedulerConfiguration.getNodeLabelPrefix( + queuePath, label) + CapacitySchedulerConfiguration.MAXIMUM_CAPACITY; + String capacityString = get(propertyName); + QueueCapacityVector capacityVector = queueCapacityConfigParser.parse(capacityString, + queuePath); + if (capacityVector.isEmpty()) { + capacityVector = defaultVector; + } + queueResourceVectors.put(label, capacityVector); } return queueResourceVectors; @@ -2806,6 +2842,11 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur } String units = getUnits(splits[1]); + + if (!UnitsConversionUtil.KNOWN_UNITS.contains(units)) { + return; + } + Long resourceValue = Long .valueOf(splits[1].substring(0, splits[1].length() - units.length())); @@ -2888,6 +2929,14 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur return normalizePolicyName(policyClassName.trim()); } + public boolean isLegacyQueueMode() { + return getBoolean(LEGACY_QUEUE_MODE_ENABLED, DEFAULT_LEGACY_QUEUE_MODE); + } + + public void setLegacyQueueModeEnabled(boolean value) { + setBoolean(LEGACY_QUEUE_MODE_ENABLED, value); + } + public boolean getMultiNodePlacementEnabled() { return getBoolean(MULTI_NODE_PLACEMENT_ENABLED, DEFAULT_MULTI_NODE_PLACEMENT_ENABLED); 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java new file mode 100644 index 00000000000..f197ccf6be2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueCapacityHandler.java @@ -0,0 +1,221 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; +import org.apache.hadoop.yarn.util.resource.ResourceCalculator; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.Map; +import java.util.Set; + +import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; +import static org.apache.hadoop.yarn.api.records.ResourceInformation.VCORES_URI; +import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NO_LABEL; + +/** + * Controls how capacity and resource values are set and calculated for a queue. + * Effective minimum and maximum resource values are set for each label and resource separately. + */ +public class CapacitySchedulerQueueCapacityHandler { + + private static final Logger LOG = + LoggerFactory.getLogger(CapacitySchedulerQueueCapacityHandler.class); + + private final Map + calculators; + private final AbstractQueueCapacityCalculator rootCalculator = + new RootQueueCapacityCalculator(); + private final RMNodeLabelsManager labelsManager; + private final Collection definedResources = new LinkedHashSet<>(); + + public CapacitySchedulerQueueCapacityHandler(RMNodeLabelsManager labelsManager) { + this.calculators = new HashMap<>(); + this.labelsManager = labelsManager; + + this.calculators.put(ResourceUnitCapacityType.ABSOLUTE, + new AbsoluteResourceCapacityCalculator()); + this.calculators.put(ResourceUnitCapacityType.PERCENTAGE, + new PercentageQueueCapacityCalculator()); + this.calculators.put(ResourceUnitCapacityType.WEIGHT, + new WeightQueueCapacityCalculator()); + + loadResourceNames(); + } + + /** + * Updates the resource and metrics values of all children under a specific queue. + * These values are calculated at runtime. + * + * @param clusterResource resource of the cluster + * @param queue parent queue whose children will be updated + * @return update context that contains information about the update phase + */ + public QueueCapacityUpdateContext updateChildren(Resource clusterResource, CSQueue queue) { + ResourceLimits resourceLimits = new ResourceLimits(clusterResource); + QueueCapacityUpdateContext updateContext = + new QueueCapacityUpdateContext(clusterResource, labelsManager); + + update(queue, updateContext, resourceLimits); + return updateContext; + } + + /** + * Updates the resource and metrics value of the root queue. Root queue always has percentage + * capacity type and is assigned the cluster resource as its minimum and maximum effective + * resource. + * @param rootQueue root queue + * @param clusterResource cluster resource + */ + public void updateRoot(CSQueue rootQueue, Resource clusterResource) { + ResourceLimits resourceLimits = new ResourceLimits(clusterResource); + QueueCapacityUpdateContext updateContext = + new QueueCapacityUpdateContext(clusterResource, labelsManager); + + RootCalculationDriver rootCalculationDriver = new RootCalculationDriver(rootQueue, + updateContext, + rootCalculator, definedResources); + rootCalculationDriver.calculateResources(); + rootQueue.refreshAfterResourceCalculation(updateContext.getUpdatedClusterResource(), + resourceLimits); + } + + private void update( + CSQueue queue, QueueCapacityUpdateContext updateContext, ResourceLimits resourceLimits) { + if (queue == null || CollectionUtils.isEmpty(queue.getChildQueues())) { + return; + } + + ResourceCalculationDriver resourceCalculationDriver = new ResourceCalculationDriver( + queue, updateContext, calculators, definedResources); + resourceCalculationDriver.calculateResources(); + + updateChildrenAfterCalculation(resourceCalculationDriver, resourceLimits); + } + + private void updateChildrenAfterCalculation( + ResourceCalculationDriver resourceCalculationDriver, ResourceLimits resourceLimits) { + ParentQueue parentQueue = (ParentQueue) resourceCalculationDriver.getQueue(); + for (CSQueue childQueue : parentQueue.getChildQueues()) { + updateQueueCapacities(resourceCalculationDriver, childQueue); + + ResourceLimits childLimit = parentQueue.getResourceLimitsOfChild(childQueue, + resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(), + resourceLimits, NO_LABEL, false); + childQueue.refreshAfterResourceCalculation(resourceCalculationDriver.getUpdateContext() + .getUpdatedClusterResource(), childLimit); + + update(childQueue, resourceCalculationDriver.getUpdateContext(), childLimit); + } + } + + /** + * Updates the capacity values of the currently evaluated child. + * @param queue queue on which the capacities are set + */ + private void updateQueueCapacities( + ResourceCalculationDriver resourceCalculationDriver, CSQueue queue) { + queue.getWriteLock().lock(); + try { + for (String label : queue.getConfiguredNodeLabels()) { + QueueCapacityVector capacityVector = queue.getConfiguredCapacityVector(label); + if (capacityVector.isMixedCapacityVector()) { + // Post update capacities based on the calculated effective resource values + setQueueCapacities(resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource( + label), queue, label); + } else { + // Update capacities according to the legacy logic + for (ResourceUnitCapacityType capacityType : + queue.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { + AbstractQueueCapacityCalculator calculator = calculators.get(capacityType); + calculator.updateCapacitiesAfterCalculation(resourceCalculationDriver, queue, label); + } + } + } + } finally { + queue.getWriteLock().unlock(); + } + } + + /** + * Sets capacity and absolute capacity values of a queue based on minimum and + * maximum effective resources. + * + * @param clusterResource overall cluster resource + * @param queue child queue for which the capacities are set + * @param label node label + */ + public static void setQueueCapacities(Resource clusterResource, CSQueue queue, String label) { + if (!(queue instanceof AbstractCSQueue)) { + return; + } + + AbstractCSQueue csQueue = (AbstractCSQueue) queue; + ResourceCalculator resourceCalculator = csQueue.resourceCalculator; + + CSQueue parent = queue.getParent(); + if (parent == null) { + return; + } + // Update capacity with a double calculated from the parent's minResources + // and the recently changed queue minResources. + // capacity = effectiveMinResource / {parent's effectiveMinResource} + float result = resourceCalculator.divide(clusterResource, + queue.getQueueResourceQuotas().getEffectiveMinResource(label), + parent.getQueueResourceQuotas().getEffectiveMinResource(label)); + queue.getQueueCapacities().setCapacity(label, + Float.isInfinite(result) ? 0 : result); + + // Update maxCapacity with a double calculated from the parent's maxResources + // and the recently changed queue maxResources. + // maxCapacity = effectiveMaxResource / parent's effectiveMaxResource + result = resourceCalculator.divide(clusterResource, + queue.getQueueResourceQuotas().getEffectiveMaxResource(label), + parent.getQueueResourceQuotas().getEffectiveMaxResource(label)); + queue.getQueueCapacities().setMaximumCapacity(label, + Float.isInfinite(result) ? 0 : result); + + csQueue.updateAbsoluteCapacities(); + } + + private void loadResourceNames() { + Set resources = new HashSet<>(ResourceUtils.getResourceTypes().keySet()); + if (resources.contains(MEMORY_URI)) { + resources.remove(MEMORY_URI); + definedResources.add(MEMORY_URI); + } + + if (resources.contains(VCORES_URI)) { + resources.remove(VCORES_URI); + definedResources.add(VCORES_URI); + } + + definedResources.addAll(resources); + } +} \ No newline at end of file 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/CapacitySchedulerQueueManager.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/CapacitySchedulerQueueManager.java index ba6849cb780..d8108c0f007 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/CapacitySchedulerQueueManager.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/CapacitySchedulerQueueManager.java @@ -81,6 +81,7 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager< private CSQueue root; private final RMNodeLabelsManager labelManager; private AppPriorityACLsManager appPriorityACLManager; + private CapacitySchedulerQueueCapacityHandler queueCapacityHandler; private QueueStateManager queueStateManager; @@ -100,6 +101,7 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager< this.queueStateManager = new QueueStateManager<>(); this.appPriorityACLManager = appPriorityACLManager; this.configuredNodeLabels = new ConfiguredNodeLabels(); + this.queueCapacityHandler = new CapacitySchedulerQueueCapacityHandler(labelManager); } @Override @@ -413,6 +415,10 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager< return this.queueStateManager; } + public CapacitySchedulerQueueCapacityHandler getQueueCapacityHandler() { + return queueCapacityHandler; + } + /** * Removes an {@code AutoCreatedLeafQueue} from the manager collection and * from its parent children collection. 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/DefaultQueueResourceRoundingStrategy.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/DefaultQueueResourceRoundingStrategy.java new file mode 100644 index 00000000000..3a0254cdc53 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/DefaultQueueResourceRoundingStrategy.java @@ -0,0 +1,48 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; + +/** + * The default rounding strategy for resource calculation. Uses floor for all types except WEIGHT, + * which is always the last type to consider, therefore it is safe to round up. + */ +public class DefaultQueueResourceRoundingStrategy implements QueueResourceRoundingStrategy { + private final ResourceUnitCapacityType lastCapacityType; + + public DefaultQueueResourceRoundingStrategy( + ResourceUnitCapacityType[] capacityTypePrecedence) { + if (capacityTypePrecedence.length == 0) { + throw new IllegalArgumentException("Capacity type precedence collection is empty"); + } + + lastCapacityType = capacityTypePrecedence[capacityTypePrecedence.length - 1]; + } + + @Override + public double getRoundedResource(double resourceValue, QueueCapacityVectorEntry capacityVectorEntry) { + if (capacityVectorEntry.getVectorResourceType().equals(lastCapacityType)) { + return Math.round(resourceValue); + } else { + return Math.floor(resourceValue); + } + } +} 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/ParentQueue.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/ParentQueue.java index 0949d512a79..a816b91034c 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/ParentQueue.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/ParentQueue.java @@ -302,94 +302,97 @@ public class ParentQueue extends AbstractCSQueue { void setChildQueues(Collection childQueues) throws IOException { writeLock.lock(); try { - QueueCapacityType childrenCapacityType = - getCapacityConfigurationTypeForQueues(childQueues); - QueueCapacityType parentCapacityType = - getCapacityConfigurationTypeForQueues(ImmutableList.of(this)); + boolean isLegacyQueueMode = queueContext.getConfiguration().isLegacyQueueMode(); + if (isLegacyQueueMode) { + QueueCapacityType childrenCapacityType = + getCapacityConfigurationTypeForQueues(childQueues); + QueueCapacityType parentCapacityType = + getCapacityConfigurationTypeForQueues(ImmutableList.of(this)); - if (childrenCapacityType == QueueCapacityType.ABSOLUTE_RESOURCE - || parentCapacityType == QueueCapacityType.ABSOLUTE_RESOURCE) { - // We don't allow any mixed absolute + {weight, percentage} between - // children and parent - if (childrenCapacityType != parentCapacityType && !this.getQueuePath() - .equals(CapacitySchedulerConfiguration.ROOT)) { - throw new IOException("Parent=" + this.getQueuePath() - + ": When absolute minResource is used, we must make sure both " - + "parent and child all use absolute minResource"); - } - - // Ensure that for each parent queue: parent.min-resource >= - // Σ(child.min-resource). - for (String nodeLabel : queueCapacities.getExistingNodeLabels()) { - Resource minRes = Resources.createResource(0, 0); - for (CSQueue queue : childQueues) { - // Accumulate all min/max resource configured for all child queues. - Resources.addTo(minRes, queue.getQueueResourceQuotas() - .getConfiguredMinResource(nodeLabel)); - } - Resource resourceByLabel = labelManager.getResourceByLabel(nodeLabel, - queueContext.getClusterResource()); - Resource parentMinResource = - usageTracker.getQueueResourceQuotas().getConfiguredMinResource(nodeLabel); - if (!parentMinResource.equals(Resources.none()) && Resources.lessThan( - resourceCalculator, resourceByLabel, parentMinResource, minRes)) { - throw new IOException( - "Parent Queues" + " capacity: " + parentMinResource - + " is less than" + " to its children:" + minRes - + " for queue:" + getQueueName()); - } - } - } - - // When child uses percent - if (childrenCapacityType == QueueCapacityType.PERCENT) { - float childrenPctSum = 0; - // check label capacities - for (String nodeLabel : queueCapacities.getExistingNodeLabels()) { - // check children's labels - childrenPctSum = 0; - for (CSQueue queue : childQueues) { - childrenPctSum += queue.getQueueCapacities().getCapacity(nodeLabel); + if (childrenCapacityType == QueueCapacityType.ABSOLUTE_RESOURCE + || parentCapacityType == QueueCapacityType.ABSOLUTE_RESOURCE) { + // We don't allow any mixed absolute + {weight, percentage} between + // children and parent + if (childrenCapacityType != parentCapacityType && !this.getQueuePath() + .equals(CapacitySchedulerConfiguration.ROOT)) { + throw new IOException("Parent=" + this.getQueuePath() + + ": When absolute minResource is used, we must make sure both " + + "parent and child all use absolute minResource"); } - if (Math.abs(1 - childrenPctSum) > PRECISION) { - // When children's percent sum != 100% - if (Math.abs(childrenPctSum) > PRECISION) { - // It is wrong when percent sum != {0, 1} + // Ensure that for each parent queue: parent.min-resource >= + // Σ(child.min-resource). + for (String nodeLabel : queueCapacities.getExistingNodeLabels()) { + Resource minRes = Resources.createResource(0, 0); + for (CSQueue queue : childQueues) { + // Accumulate all min/max resource configured for all child queues. + Resources.addTo(minRes, queue.getQueueResourceQuotas() + .getConfiguredMinResource(nodeLabel)); + } + Resource resourceByLabel = labelManager.getResourceByLabel(nodeLabel, + queueContext.getClusterResource()); + Resource parentMinResource = + usageTracker.getQueueResourceQuotas().getConfiguredMinResource(nodeLabel); + if (!parentMinResource.equals(Resources.none()) && Resources.lessThan( + resourceCalculator, resourceByLabel, parentMinResource, minRes)) { throw new IOException( - "Illegal capacity sum of " + childrenPctSum - + " for children of queue " + getQueueName() + " for label=" - + nodeLabel + ". It should be either 0 or 1.0"); - } else{ - // We also allow children's percent sum = 0 under the following - // conditions - // - Parent uses weight mode - // - Parent uses percent mode, and parent has - // (capacity=0 OR allowZero) - if (parentCapacityType == QueueCapacityType.PERCENT) { - if ((Math.abs(queueCapacities.getCapacity(nodeLabel)) - > PRECISION) && (!allowZeroCapacitySum)) { - throw new IOException( - "Illegal capacity sum of " + childrenPctSum - + " for children of queue " + getQueueName() - + " for label=" + nodeLabel - + ". It is set to 0, but parent percent != 0, and " - + "doesn't allow children capacity to set to 0"); + "Parent Queues" + " capacity: " + parentMinResource + + " is less than" + " to its children:" + minRes + + " for queue:" + getQueueName()); + } + } + } + + // When child uses percent + if (childrenCapacityType == QueueCapacityType.PERCENT) { + float childrenPctSum = 0; + // check label capacities + for (String nodeLabel : queueCapacities.getExistingNodeLabels()) { + // check children's labels + childrenPctSum = 0; + for (CSQueue queue : childQueues) { + childrenPctSum += queue.getQueueCapacities().getCapacity(nodeLabel); + } + + if (Math.abs(1 - childrenPctSum) > PRECISION) { + // When children's percent sum != 100% + if (Math.abs(childrenPctSum) > PRECISION) { + // It is wrong when percent sum != {0, 1} + throw new IOException( + "Illegal" + " capacity sum of " + childrenPctSum + + " for children of queue " + getQueueName() + " for label=" + + nodeLabel + ". It should be either 0 or 1.0"); + } else { + // We also allow children's percent sum = 0 under the following + // conditions + // - Parent uses weight mode + // - Parent uses percent mode, and parent has + // (capacity=0 OR allowZero) + if (parentCapacityType == QueueCapacityType.PERCENT) { + if ((Math.abs(queueCapacities.getCapacity(nodeLabel)) + > PRECISION) && (!allowZeroCapacitySum)) { + throw new IOException( + "Illegal" + " capacity sum of " + childrenPctSum + + " for children of queue " + getQueueName() + + " for label=" + nodeLabel + + ". It is set to 0, but parent percent != 0, and " + + "doesn't allow children capacity to set to 0"); + } } } - } - } else { - // Even if child pct sum == 1.0, we will make sure parent has - // positive percent. - if (parentCapacityType == QueueCapacityType.PERCENT && Math.abs( - queueCapacities.getCapacity(nodeLabel)) <= 0f - && !allowZeroCapacitySum) { - throw new IOException( - "Illegal capacity sum of " + childrenPctSum - + " for children of queue " + getQueueName() + " for label=" - + nodeLabel + ". queue=" + getQueueName() - + " has zero capacity, but child" - + "queues have positive capacities"); + } else { + // Even if child pct sum == 1.0, we will make sure parent has + // positive percent. + if (parentCapacityType == QueueCapacityType.PERCENT && Math.abs( + queueCapacities.getCapacity(nodeLabel)) <= 0f + && !allowZeroCapacitySum) { + throw new IOException( + "Illegal" + " capacity sum of " + childrenPctSum + + " for children of queue " + getQueueName() + " for label=" + + nodeLabel + ". queue=" + getQueueName() + + " has zero capacity, but child" + + "queues have positive capacities"); + } } } } @@ -1057,7 +1060,7 @@ public class ParentQueue extends AbstractCSQueue { return accept; } - private ResourceLimits getResourceLimitsOfChild(CSQueue child, + public ResourceLimits getResourceLimitsOfChild(CSQueue child, Resource clusterResource, ResourceLimits parentLimits, String nodePartition, boolean netLimit) { // Set resource-limit of a given child, child.limit = @@ -1208,6 +1211,17 @@ public class ParentQueue extends AbstractCSQueue { } } + @Override + public void refreshAfterResourceCalculation(Resource clusterResource, + ResourceLimits resourceLimits) { + CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, + this, labelManager, null); + // Update configured capacity/max-capacity for default partition only + CSQueueUtils.updateConfiguredCapacityMetrics(resourceCalculator, + labelManager.getResourceByLabel(null, clusterResource), + RMNodeLabelsManager.NO_LABEL, this); + } + @Override public void updateClusterResource(Resource clusterResource, ResourceLimits resourceLimits) { 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java new file mode 100644 index 00000000000..6a73459aaf4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PercentageQueueCapacityCalculator.java @@ -0,0 +1,72 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; + +public class PercentageQueueCapacityCalculator extends AbstractQueueCapacityCalculator { + + @Override + public double calculateMinimumResource( + ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, + String label) { + String resourceName = context.getResourceName(); + + double parentAbsoluteCapacity = resourceCalculationDriver.getParentAbsoluteMinCapacity(label, + resourceName); + double remainingPerEffectiveResourceRatio = + resourceCalculationDriver.getRemainingRatioOfResource(label, resourceName); + double absoluteCapacity = parentAbsoluteCapacity * remainingPerEffectiveResourceRatio + * context.getCurrentMinimumCapacityEntry(label).getResourceValue() / 100; + + return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label) + .getResourceValue(resourceName) * absoluteCapacity; + } + + @Override + public double calculateMaximumResource( + ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, + String label) { + String resourceName = context.getResourceName(); + + double parentAbsoluteMaxCapacity = + resourceCalculationDriver.getParentAbsoluteMaxCapacity(label, resourceName); + double absoluteMaxCapacity = parentAbsoluteMaxCapacity + * context.getCurrentMaximumCapacityEntry(label).getResourceValue() / 100; + + return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label) + .getResourceValue(resourceName) * absoluteMaxCapacity; + } + + @Override + public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCalculationDriver) { + + } + + @Override + public void updateCapacitiesAfterCalculation(ResourceCalculationDriver resourceCalculationDriver, + CSQueue queue, String label) { + ((AbstractCSQueue) queue).updateAbsoluteCapacities(); + } + + @Override + public ResourceUnitCapacityType getCapacityType() { + return ResourceUnitCapacityType.PERCENTAGE; + } +} 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/QueueCapacityUpdateContext.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/QueueCapacityUpdateContext.java new file mode 100644 index 00000000000..4eb270be515 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueCapacityUpdateContext.java @@ -0,0 +1,76 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; + +import java.util.ArrayList; +import java.util.List; + +/** + * A storage that encapsulates intermediate calculation values throughout a + * full queue capacity update phase. + */ +public class QueueCapacityUpdateContext { + private final Resource updatedClusterResource; + private final RMNodeLabelsManager labelsManager; + + private final List warnings = new ArrayList(); + + public QueueCapacityUpdateContext(Resource updatedClusterResource, + RMNodeLabelsManager labelsManager) { + this.updatedClusterResource = updatedClusterResource; + this.labelsManager = labelsManager; + } + + /** + * Returns the overall cluster resource available for the update phase. + * + * @param label node label + * @return cluster resource + */ + public Resource getUpdatedClusterResource(String label) { + return labelsManager.getResourceByLabel(label, updatedClusterResource); + } + + /** + * Returns the overall cluster resource available for the update phase of empty label. + * @return cluster resource + */ + public Resource getUpdatedClusterResource() { + return updatedClusterResource; + } + + /** + * Adds an update warning to the context. + * @param warning warning during update phase + */ + public void addUpdateWarning(QueueUpdateWarning warning) { + warnings.add(warning); + } + + /** + * Returns all update warnings occurred in this update phase. + * @return update warnings + */ + public List getUpdateWarnings() { + return warnings; + } +} 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/QueueCapacityVector.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/QueueCapacityVector.java index 9f6e0e264a3..bcce996b279 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/QueueCapacityVector.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/QueueCapacityVector.java @@ -39,9 +39,9 @@ public class QueueCapacityVector implements private static final String VALUE_DELIMITER = "="; private final ResourceVector resource; - private final Map capacityTypes + private final Map capacityTypes = new HashMap<>(); - private final Map> capacityTypePerResource + private final Map> capacityTypePerResource = new HashMap<>(); public QueueCapacityVector() { @@ -61,9 +61,9 @@ public class QueueCapacityVector implements public static QueueCapacityVector newInstance() { QueueCapacityVector newCapacityVector = new QueueCapacityVector(ResourceVector.newInstance()); - for (Map.Entry resourceEntry : newCapacityVector.resource) { + for (Map.Entry resourceEntry : newCapacityVector.resource) { newCapacityVector.storeResourceType(resourceEntry.getKey(), - QueueCapacityType.ABSOLUTE); + ResourceUnitCapacityType.ABSOLUTE); } return newCapacityVector; @@ -78,10 +78,10 @@ public class QueueCapacityVector implements * @return uniform capacity vector */ public static QueueCapacityVector of( - float value, QueueCapacityType capacityType) { + double value, ResourceUnitCapacityType capacityType) { QueueCapacityVector newCapacityVector = new QueueCapacityVector(ResourceVector.of(value)); - for (Map.Entry resourceEntry : newCapacityVector.resource) { + for (Map.Entry resourceEntry : newCapacityVector.resource) { newCapacityVector.storeResourceType(resourceEntry.getKey(), capacityType); } @@ -109,8 +109,8 @@ public class QueueCapacityVector implements * @param value value of the resource * @param capacityType type of the resource */ - public void setResource(String resourceName, float value, - QueueCapacityType capacityType) { + public void setResource(String resourceName, double value, + ResourceUnitCapacityType capacityType) { // Necessary due to backward compatibility (memory = memory-mb) String convertedResourceName = resourceName; if (resourceName.equals("memory")) { @@ -125,10 +125,14 @@ public class QueueCapacityVector implements * * @return value of memory resource */ - public float getMemory() { + public double getMemory() { return resource.getValue(ResourceInformation.MEMORY_URI); } + public boolean isEmpty() { + return resource.isEmpty() && capacityTypePerResource.isEmpty() && capacityTypes.isEmpty(); + } + /** * Returns the name of all resources that are defined in the given capacity * type. @@ -137,13 +141,20 @@ public class QueueCapacityVector implements * @return all resource names for the given capacity type */ public Set getResourceNamesByCapacityType( - QueueCapacityType capacityType) { - return capacityTypePerResource.getOrDefault(capacityType, - Collections.emptySet()); + ResourceUnitCapacityType capacityType) { + return new HashSet<>(capacityTypePerResource.getOrDefault(capacityType, + Collections.emptySet())); } + /** + * Checks whether a resource unit is defined as a specific type. + * + * @param resourceName resource unit name + * @param capacityType capacity type + * @return true, if resource unit is defined as the given type + */ public boolean isResourceOfType( - String resourceName, QueueCapacityType capacityType) { + String resourceName, ResourceUnitCapacityType capacityType) { return capacityTypes.containsKey(resourceName) && capacityTypes.get(resourceName).equals(capacityType); } @@ -151,7 +162,7 @@ public class QueueCapacityVector implements @Override public Iterator iterator() { return new Iterator() { - private final Iterator> resources = + private final Iterator> resources = resource.iterator(); private int i = 0; @@ -162,7 +173,7 @@ public class QueueCapacityVector implements @Override public QueueCapacityVectorEntry next() { - Map.Entry resourceInformation = resources.next(); + Map.Entry resourceInformation = resources.next(); i++; return new QueueCapacityVectorEntry( capacityTypes.get(resourceInformation.getKey()), @@ -172,16 +183,29 @@ public class QueueCapacityVector implements } /** - * Returns a set of all capacity type defined for this vector. + * Returns a set of all capacity types defined for this vector. * * @return capacity types */ - public Set getDefinedCapacityTypes() { + public Set getDefinedCapacityTypes() { return capacityTypePerResource.keySet(); } + /** + * Checks whether the vector is a mixed capacity vector (more than one capacity type is used, + * therefore it is not uniform). + * @return true, if the vector is mixed + */ + public boolean isMixedCapacityVector() { + return getDefinedCapacityTypes().size() > 1; + } + + public Set getResourceNames() { + return resource.getResourceNames(); + } + private void storeResourceType( - String resourceName, QueueCapacityType resourceType) { + String resourceName, ResourceUnitCapacityType resourceType) { if (capacityTypes.get(resourceName) != null && !capacityTypes.get(resourceName).equals(resourceType)) { capacityTypePerResource.get(capacityTypes.get(resourceName)) @@ -199,7 +223,7 @@ public class QueueCapacityVector implements stringVector.append(START_PARENTHESES); int resourceCount = 0; - for (Map.Entry resourceEntry : resource) { + for (Map.Entry resourceEntry : resource) { resourceCount++; stringVector.append(resourceEntry.getKey()) .append(VALUE_DELIMITER) @@ -218,11 +242,11 @@ public class QueueCapacityVector implements /** * Represents a capacity type associated with its syntax postfix. */ - public enum QueueCapacityType { + public enum ResourceUnitCapacityType { PERCENTAGE("%"), ABSOLUTE(""), WEIGHT("w"); private final String postfix; - QueueCapacityType(String postfix) { + ResourceUnitCapacityType(String postfix) { this.postfix = postfix; } @@ -232,22 +256,22 @@ public class QueueCapacityVector implements } public static class QueueCapacityVectorEntry { - private final QueueCapacityType vectorResourceType; - private final float resourceValue; + private final ResourceUnitCapacityType vectorResourceType; + private final double resourceValue; private final String resourceName; - public QueueCapacityVectorEntry(QueueCapacityType vectorResourceType, - String resourceName, float resourceValue) { + public QueueCapacityVectorEntry(ResourceUnitCapacityType vectorResourceType, + String resourceName, double resourceValue) { this.vectorResourceType = vectorResourceType; this.resourceValue = resourceValue; this.resourceName = resourceName; } - public QueueCapacityType getVectorResourceType() { + public ResourceUnitCapacityType getVectorResourceType() { return vectorResourceType; } - public float getResourceValue() { + public double getResourceValue() { return resourceValue; } 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/QueueResourceRoundingStrategy.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/QueueResourceRoundingStrategy.java new file mode 100644 index 00000000000..ef753316e84 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueResourceRoundingStrategy.java @@ -0,0 +1,36 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; + +/** + * Represents an approach on how to convert a calculated resource from floating point to a whole + * number. + */ +public interface QueueResourceRoundingStrategy { + + /** + * Returns a whole number converted from the calculated resource value. + * @param resourceValue calculated resource value + * @param capacityVectorEntry configured capacity entry + * @return rounded resource value + */ + double getRoundedResource(double resourceValue, QueueCapacityVectorEntry capacityVectorEntry); +} 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/QueueUpdateWarning.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/QueueUpdateWarning.java new file mode 100644 index 00000000000..43c345b1bc3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueUpdateWarning.java @@ -0,0 +1,78 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +/** + * Represents a warning event that occurred during a queue capacity update phase. + */ +public class QueueUpdateWarning { + private final String queue; + private final QueueUpdateWarningType warningType; + private String info = ""; + + public QueueUpdateWarning(QueueUpdateWarningType queueUpdateWarningType, String queue) { + this.warningType = queueUpdateWarningType; + this.queue = queue; + } + + public enum QueueUpdateWarningType { + BRANCH_UNDERUTILIZED("Remaining resource found in branch under parent queue '%s'. %s"), + QUEUE_OVERUTILIZED("Queue '%s' is configured to use more resources than what is available " + + "under its parent. %s"), + QUEUE_ZERO_RESOURCE("Queue '%s' is assigned zero resource. %s"), + BRANCH_DOWNSCALED("Child queues with absolute configured capacity under parent queue '%s' are" + + " downscaled due to insufficient cluster resource. %s"), + QUEUE_EXCEEDS_MAX_RESOURCE("Queue '%s' exceeds its maximum available resources. %s"), + QUEUE_MAX_RESOURCE_EXCEEDS_PARENT("Maximum resources of queue '%s' are greater than its " + + "parent's. %s"); + + private final String template; + + QueueUpdateWarningType(String template) { + this.template = template; + } + + public QueueUpdateWarning ofQueue(String queue) { + return new QueueUpdateWarning(this, queue); + } + + public String getTemplate() { + return template; + } + } + + public QueueUpdateWarning withInfo(String info) { + this.info = info; + + return this; + } + + public String getQueue() { + return queue; + } + + public QueueUpdateWarningType getWarningType() { + return warningType; + } + + @Override + public String toString() { + return String.format(warningType.getTemplate(), queue, info); + } +} 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java new file mode 100644 index 00000000000..5993042c0e5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ResourceCalculationDriver.java @@ -0,0 +1,336 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueUpdateWarning.QueueUpdateWarningType; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; + +/** + * Drives the main logic of resource calculation for all children under a queue. Acts as a + * bookkeeper of disposable update information that is used by all children under the common parent. + */ +public class ResourceCalculationDriver { + private static final ResourceUnitCapacityType[] CALCULATOR_PRECEDENCE = + new ResourceUnitCapacityType[] { + ResourceUnitCapacityType.ABSOLUTE, + ResourceUnitCapacityType.PERCENTAGE, + ResourceUnitCapacityType.WEIGHT}; + static final String MB_UNIT = "Mi"; + + protected final QueueResourceRoundingStrategy roundingStrategy = + new DefaultQueueResourceRoundingStrategy(CALCULATOR_PRECEDENCE); + protected final CSQueue queue; + protected final QueueCapacityUpdateContext updateContext; + protected final Map calculators; + protected final Collection definedResources; + + protected final Map overallRemainingResourcePerLabel = new HashMap<>(); + protected final Map batchRemainingResourcePerLabel = new HashMap<>(); + // Used by ABSOLUTE capacity types + protected final Map normalizedResourceRatioPerLabel = new HashMap<>(); + // Used by WEIGHT capacity types + protected final Map> sumWeightsPerLabel = new HashMap<>(); + protected Map usedResourceByCurrentCalculatorPerLabel = new HashMap<>(); + + public ResourceCalculationDriver( + CSQueue queue, QueueCapacityUpdateContext updateContext, + Map calculators, + Collection definedResources) { + this.queue = queue; + this.updateContext = updateContext; + this.calculators = calculators; + this.definedResources = definedResources; + } + + + /** + * Returns the parent that is driving the calculation. + * + * @return a common parent queue + */ + public CSQueue getQueue() { + return queue; + } + + /** + * Returns all the children defined under the driver parent queue. + * + * @return child queues + */ + public Collection getChildQueues() { + return queue.getChildQueues(); + } + + /** + * Returns the context that is used throughout the whole update phase. + * + * @return update context + */ + public QueueCapacityUpdateContext getUpdateContext() { + return updateContext; + } + + /** + * Increments the aggregated weight. + * + * @param label node label + * @param resourceName resource unit name + * @param value weight value + */ + public void incrementWeight(String label, String resourceName, double value) { + sumWeightsPerLabel.putIfAbsent(label, new HashMap<>()); + sumWeightsPerLabel.get(label).put(resourceName, + sumWeightsPerLabel.get(label).getOrDefault(resourceName, 0d) + value); + } + + /** + * Returns the aggregated children weights. + * + * @param label node label + * @param resourceName resource unit name + * @return aggregated weights of children + */ + public double getSumWeightsByResource(String label, String resourceName) { + return sumWeightsPerLabel.get(label).get(resourceName); + } + + /** + * Returns the ratio of the summary of children absolute configured resources and the parent's + * effective minimum resource. + * + * @return normalized resource ratio for all labels + */ + public Map getNormalizedResourceRatios() { + return normalizedResourceRatioPerLabel; + } + + /** + * Returns the remaining resource ratio under the parent queue. The remaining resource is only + * decremented after a capacity type is fully evaluated. + * + * @param label node label + * @param resourceName name of resource unit + * @return resource ratio + */ + public double getRemainingRatioOfResource(String label, String resourceName) { + return batchRemainingResourcePerLabel.get(label).getValue(resourceName) + / queue.getEffectiveCapacity(label).getResourceValue(resourceName); + } + + /** + * Returns the ratio of the parent queue's effective minimum resource relative to the full cluster + * resource. + * + * @param label node label + * @param resourceName name of resource unit + * @return absolute minimum capacity + */ + public double getParentAbsoluteMinCapacity(String label, String resourceName) { + return (double) queue.getEffectiveCapacity(label).getResourceValue(resourceName) + / getUpdateContext().getUpdatedClusterResource(label).getResourceValue(resourceName); + } + + /** + * Returns the ratio of the parent queue's effective maximum resource relative to the full cluster + * resource. + * + * @param label node label + * @param resourceName name of resource unit + * @return absolute maximum capacity + */ + public double getParentAbsoluteMaxCapacity(String label, String resourceName) { + return (double) queue.getEffectiveMaxCapacity(label).getResourceValue(resourceName) + / getUpdateContext().getUpdatedClusterResource(label).getResourceValue(resourceName); + } + + /** + * Returns the remaining resources of a parent that is still available for its + * children. Decremented only after the calculator is finished its work on the corresponding + * resources. + * + * @param label node label + * @return remaining resources + */ + public ResourceVector getBatchRemainingResource(String label) { + batchRemainingResourcePerLabel.putIfAbsent(label, ResourceVector.newInstance()); + return batchRemainingResourcePerLabel.get(label); + } + + /** + * Calculates and sets the minimum and maximum effective resources for all children under the + * parent queue with which this driver was initialized. + */ + public void calculateResources() { + // Reset both remaining resource storage to the parent's available resource + for (String label : queue.getConfiguredNodeLabels()) { + overallRemainingResourcePerLabel.put(label, + ResourceVector.of(queue.getEffectiveCapacity(label))); + batchRemainingResourcePerLabel.put(label, + ResourceVector.of(queue.getEffectiveCapacity(label))); + } + + for (AbstractQueueCapacityCalculator capacityCalculator : calculators.values()) { + capacityCalculator.calculateResourcePrerequisites(this); + } + + for (String resourceName : definedResources) { + for (ResourceUnitCapacityType capacityType : CALCULATOR_PRECEDENCE) { + for (CSQueue childQueue : getChildQueues()) { + CalculationContext context = new CalculationContext(resourceName, capacityType, + childQueue); + calculateResourceOnChild(context); + } + + // Flush aggregated used resource by labels at the end of a calculator phase + for (Map.Entry entry : usedResourceByCurrentCalculatorPerLabel.entrySet()) { + batchRemainingResourcePerLabel.get(entry.getKey()).decrement(resourceName, + entry.getValue()); + } + + usedResourceByCurrentCalculatorPerLabel = new HashMap<>(); + } + } + + validateRemainingResource(); + } + + private void calculateResourceOnChild(CalculationContext context) { + context.getQueue().getWriteLock().lock(); + try { + for (String label : context.getQueue().getConfiguredNodeLabels()) { + if (!context.getQueue().getConfiguredCapacityVector(label).isResourceOfType( + context.getResourceName(), context.getCapacityType())) { + continue; + } + double usedResourceByChild = setChildResources(context, label); + double aggregatedUsedResource = usedResourceByCurrentCalculatorPerLabel.getOrDefault(label, + 0d); + double resourceUsedByLabel = aggregatedUsedResource + usedResourceByChild; + + overallRemainingResourcePerLabel.get(label).decrement(context.getResourceName(), + usedResourceByChild); + usedResourceByCurrentCalculatorPerLabel.put(label, resourceUsedByLabel); + } + } finally { + context.getQueue().getWriteLock().unlock(); + } + } + + private double setChildResources(CalculationContext context, String label) { + QueueCapacityVectorEntry capacityVectorEntry = context.getQueue().getConfiguredCapacityVector( + label).getResource(context.getResourceName()); + QueueCapacityVectorEntry maximumCapacityVectorEntry = context.getQueue() + .getConfiguredMaxCapacityVector(label).getResource(context.getResourceName()); + AbstractQueueCapacityCalculator maximumCapacityCalculator = calculators.get( + maximumCapacityVectorEntry.getVectorResourceType()); + + double minimumResource = + calculators.get(context.getCapacityType()).calculateMinimumResource(this, context, label); + double maximumResource = maximumCapacityCalculator.calculateMaximumResource(this, context, + label); + + minimumResource = roundingStrategy.getRoundedResource(minimumResource, capacityVectorEntry); + maximumResource = roundingStrategy.getRoundedResource(maximumResource, + maximumCapacityVectorEntry); + Pair resources = validateCalculatedResources(context, label, + new ImmutablePair<>( + minimumResource, maximumResource)); + minimumResource = resources.getLeft(); + maximumResource = resources.getRight(); + + context.getQueue().getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( + context.getResourceName(), (long) minimumResource); + context.getQueue().getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( + context.getResourceName(), (long) maximumResource); + + return minimumResource; + } + + private Pair validateCalculatedResources(CalculationContext context, + String label, Pair calculatedResources) { + double minimumResource = calculatedResources.getLeft(); + long minimumMemoryResource = + context.getQueue().getQueueResourceQuotas().getEffectiveMinResource(label).getMemorySize(); + + double remainingResourceUnderParent = overallRemainingResourcePerLabel.get(label).getValue( + context.getResourceName()); + + long parentMaximumResource = queue.getEffectiveMaxCapacity(label).getResourceValue( + context.getResourceName()); + double maximumResource = calculatedResources.getRight(); + + // Memory is the primary resource, if its zero, all other resource units are zero as well. + if (!context.getResourceName().equals(MEMORY_URI) && minimumMemoryResource == 0) { + minimumResource = 0; + } + + if (maximumResource != 0 && maximumResource > parentMaximumResource) { + updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT + .ofQueue(context.getQueue().getQueuePath())); + } + maximumResource = maximumResource == 0 ? parentMaximumResource : Math.min(maximumResource, + parentMaximumResource); + + if (maximumResource < minimumResource) { + updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_EXCEEDS_MAX_RESOURCE.ofQueue( + context.getQueue().getQueuePath())); + minimumResource = maximumResource; + } + + if (minimumResource > remainingResourceUnderParent) { + // Legacy auto queues are assigned a zero resource if not enough resource is left + if (queue instanceof ManagedParentQueue) { + minimumResource = 0; + } else { + updateContext.addUpdateWarning( + QueueUpdateWarningType.QUEUE_OVERUTILIZED.ofQueue( + context.getQueue().getQueuePath()).withInfo( + "Resource name: " + context.getResourceName() + + " resource value: " + minimumResource)); + minimumResource = remainingResourceUnderParent; + } + } + + if (minimumResource == 0) { + updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_ZERO_RESOURCE.ofQueue( + context.getQueue().getQueuePath()) + .withInfo("Resource name: " + context.getResourceName())); + } + + return new ImmutablePair<>(minimumResource, maximumResource); + } + + private void validateRemainingResource() { + for (String label : queue.getConfiguredNodeLabels()) { + if (!batchRemainingResourcePerLabel.get(label).equals(ResourceVector.newInstance())) { + updateContext.addUpdateWarning(QueueUpdateWarningType.BRANCH_UNDERUTILIZED.ofQueue( + queue.getQueuePath()).withInfo("Label: " + label)); + } + } + } +} 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/ResourceVector.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/ResourceVector.java index 88c09af6b09..8a417b0e66b 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/ResourceVector.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/ResourceVector.java @@ -25,13 +25,13 @@ import org.apache.hadoop.yarn.util.resource.ResourceUtils; import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.Set; /** - * Represents a simple resource floating point value storage - * grouped by resource names. + * Represents a simple resource floating point value grouped by resource names. */ -public class ResourceVector implements Iterable> { - private final Map resourcesByName = new HashMap<>(); +public class ResourceVector implements Iterable> { + private final Map resourcesByName = new HashMap<>(); /** * Creates a new {@code ResourceVector} with all pre-defined resources set to @@ -53,7 +53,7 @@ public class ResourceVector implements Iterable> { * @param value the value to set all resources to * @return uniform resource vector */ - public static ResourceVector of(float value) { + public static ResourceVector of(double value) { ResourceVector emptyResourceVector = new ResourceVector(); for (ResourceInformation resource : ResourceUtils.getResourceTypesArray()) { emptyResourceVector.setValue(resource.getName(), value); @@ -79,34 +79,51 @@ public class ResourceVector implements Iterable> { } /** - * Subtract values for each resource defined in the given resource vector. + * Decrements values for each resource defined in the given resource vector. * @param otherResourceVector rhs resource vector of the subtraction */ - public void subtract(ResourceVector otherResourceVector) { - for (Map.Entry resource : otherResourceVector) { + public void decrement(ResourceVector otherResourceVector) { + for (Map.Entry resource : otherResourceVector) { setValue(resource.getKey(), getValue(resource.getKey()) - resource.getValue()); } } + /** + * Decrements the given resource by the specified value. + * @param resourceName name of the resource + * @param value value to be subtracted from the resource's current value + */ + public void decrement(String resourceName, double value) { + setValue(resourceName, getValue(resourceName) - value); + } + /** * Increments the given resource by the specified value. * @param resourceName name of the resource * @param value value to be added to the resource's current value */ - public void increment(String resourceName, float value) { + public void increment(String resourceName, double value) { setValue(resourceName, getValue(resourceName) + value); } - public Float getValue(String resourceName) { + public double getValue(String resourceName) { return resourcesByName.get(resourceName); } - public void setValue(String resourceName, float value) { + public void setValue(String resourceName, double value) { resourcesByName.put(resourceName, value); } + public boolean isEmpty() { + return resourcesByName.isEmpty(); + } + + public Set getResourceNames() { + return resourcesByName.keySet(); + } + @Override - public Iterator> iterator() { + public Iterator> iterator() { return resourcesByName.entrySet().iterator(); } 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/RootCalculationDriver.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/RootCalculationDriver.java new file mode 100644 index 00000000000..530c5c1086f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/RootCalculationDriver.java @@ -0,0 +1,64 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +import java.util.Collection; +import java.util.Collections; + +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.PERCENTAGE; + +/** + * A special case that contains the resource calculation of the root queue. + */ +public final class RootCalculationDriver extends ResourceCalculationDriver { + private final AbstractQueueCapacityCalculator rootCalculator; + + public RootCalculationDriver(CSQueue rootQueue, QueueCapacityUpdateContext updateContext, + AbstractQueueCapacityCalculator rootCalculator, + Collection definedResources) { + super(rootQueue, updateContext, Collections.emptyMap(), definedResources); + this.rootCalculator = rootCalculator; + } + + @Override + public void calculateResources() { + for (String label : queue.getConfiguredNodeLabels()) { + for (QueueCapacityVector.QueueCapacityVectorEntry capacityVectorEntry : + queue.getConfiguredCapacityVector(label)) { + String resourceName = capacityVectorEntry.getResourceName(); + + CalculationContext context = new CalculationContext(resourceName, PERCENTAGE, queue); + double minimumResource = rootCalculator.calculateMinimumResource(this, context, label); + double maximumResource = rootCalculator.calculateMaximumResource(this, context, label); + long roundedMinResource = (long) roundingStrategy + .getRoundedResource(minimumResource, capacityVectorEntry); + long roundedMaxResource = (long) roundingStrategy + .getRoundedResource(maximumResource, + queue.getConfiguredMaxCapacityVector(label).getResource(resourceName)); + queue.getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( + resourceName, roundedMinResource); + queue.getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( + resourceName, roundedMaxResource); + } + rootCalculator.updateCapacitiesAfterCalculation(this, queue, label); + } + + rootCalculator.calculateResourcePrerequisites(this); + } +} 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java new file mode 100644 index 00000000000..8da1aeab282 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/RootQueueCapacityCalculator.java @@ -0,0 +1,59 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; + +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.PERCENTAGE; + +public class RootQueueCapacityCalculator extends AbstractQueueCapacityCalculator { + + @Override + public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCalculationDriver) { + AbsoluteResourceCapacityCalculator.setNormalizedResourceRatio(resourceCalculationDriver); + } + + @Override + public double calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, + CalculationContext context, String label) { + return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label) + .getResourceValue(context.getResourceName()); + } + + @Override + public double calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, + CalculationContext context, String label) { + return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label) + .getResourceValue(context.getResourceName()); + } + + @Override + public void updateCapacitiesAfterCalculation( + ResourceCalculationDriver resourceCalculationDriver, CSQueue queue, String label) { + queue.getQueueCapacities().setAbsoluteCapacity(label, 1); + if (queue.getQueueCapacities().getWeight(label) == 1) { + queue.getQueueCapacities().setNormalizedWeight(label, 1); + } + } + + @Override + public ResourceUnitCapacityType getCapacityType() { + return PERCENTAGE; + } +} 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java new file mode 100644 index 00000000000..4121a6bf056 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/WeightQueueCapacityCalculator.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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; + +import java.util.Collection; + +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.WEIGHT; + +public class WeightQueueCapacityCalculator extends AbstractQueueCapacityCalculator { + + @Override + public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCalculationDriver) { + // Precalculate the summary of children's weight + for (CSQueue childQueue : resourceCalculationDriver.getChildQueues()) { + for (String label : childQueue.getConfiguredNodeLabels()) { + for (String resourceName : childQueue.getConfiguredCapacityVector(label) + .getResourceNamesByCapacityType(getCapacityType())) { + resourceCalculationDriver.incrementWeight(label, resourceName, childQueue + .getConfiguredCapacityVector(label).getResource(resourceName).getResourceValue()); + } + } + } + } + + @Override + public double calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, + CalculationContext context, + String label) { + String resourceName = context.getResourceName(); + double normalizedWeight = context.getCurrentMinimumCapacityEntry(label).getResourceValue() / + resourceCalculationDriver.getSumWeightsByResource(label, resourceName); + + double remainingResource = resourceCalculationDriver.getBatchRemainingResource(label) + .getValue(resourceName); + + // Due to rounding loss it is better to use all remaining resources if no other resource uses + // weight + if (normalizedWeight == 1) { + return remainingResource; + } + + double remainingResourceRatio = resourceCalculationDriver.getRemainingRatioOfResource( + label, resourceName); + double parentAbsoluteCapacity = resourceCalculationDriver.getParentAbsoluteMinCapacity( + label, resourceName); + double queueAbsoluteCapacity = parentAbsoluteCapacity * remainingResourceRatio + * normalizedWeight; + + return resourceCalculationDriver.getUpdateContext() + .getUpdatedClusterResource(label).getResourceValue(resourceName) * queueAbsoluteCapacity; + } + + @Override + public double calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, + CalculationContext context, + String label) { + throw new IllegalStateException("Resource " + context.getCurrentMinimumCapacityEntry( + label).getResourceName() + + " has " + "WEIGHT maximum capacity type, which is not supported"); + } + + @Override + public ResourceUnitCapacityType getCapacityType() { + return WEIGHT; + } + + @Override + public void updateCapacitiesAfterCalculation( + ResourceCalculationDriver resourceCalculationDriver, CSQueue queue, String label) { + double sumCapacityPerResource = 0f; + + Collection resourceNames = getResourceNames(queue, label); + for (String resourceName : resourceNames) { + double sumBranchWeight = resourceCalculationDriver.getSumWeightsByResource(label, + resourceName); + double capacity = queue.getConfiguredCapacityVector( + label).getResource(resourceName).getResourceValue() / sumBranchWeight; + sumCapacityPerResource += capacity; + } + + queue.getQueueCapacities().setNormalizedWeight(label, + (float) (sumCapacityPerResource / resourceNames.size())); + ((AbstractCSQueue) queue).updateAbsoluteCapacities(); + } +} 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/conf/QueueCapacityConfigParser.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/conf/QueueCapacityConfigParser.java index 28eb33c5536..79786a11b3c 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/conf/QueueCapacityConfigParser.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/conf/QueueCapacityConfigParser.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; import org.apache.hadoop.yarn.util.UnitsConversionUtil; import java.util.ArrayList; @@ -61,22 +61,16 @@ public class QueueCapacityConfigParser { /** * Creates a {@code QueueCapacityVector} parsed from the capacity configuration * property set for a queue. - * @param conf configuration object + * @param capacityString capacity string to parse * @param queuePath queue for which the capacity property is parsed - * @param label node label * @return a parsed capacity vector */ - public QueueCapacityVector parse(CapacitySchedulerConfiguration conf, - String queuePath, String label) { + public QueueCapacityVector parse(String capacityString, String queuePath) { if (queuePath.equals(CapacitySchedulerConfiguration.ROOT)) { - return QueueCapacityVector.of(100f, QueueCapacityType.PERCENTAGE); + return QueueCapacityVector.of(100f, ResourceUnitCapacityType.PERCENTAGE); } - String propertyName = CapacitySchedulerConfiguration.getNodeLabelPrefix( - queuePath, label) + CapacitySchedulerConfiguration.CAPACITY; - String capacityString = conf.get(propertyName); - if (capacityString == null) { return new QueueCapacityVector(); } @@ -101,13 +95,13 @@ public class QueueCapacityConfigParser { * @return a parsed capacity vector */ private QueueCapacityVector uniformParser(Matcher matcher) { - QueueCapacityType capacityType = null; + ResourceUnitCapacityType capacityType = null; String value = matcher.group(1); if (matcher.groupCount() == 2) { String matchedSuffix = matcher.group(2); - for (QueueCapacityType suffix : QueueCapacityType.values()) { + for (ResourceUnitCapacityType suffix : ResourceUnitCapacityType.values()) { // Absolute uniform syntax is not supported - if (suffix.equals(QueueCapacityType.ABSOLUTE)) { + if (suffix.equals(ResourceUnitCapacityType.ABSOLUTE)) { continue; } // when capacity is given in percentage, we do not need % symbol @@ -164,7 +158,7 @@ public class QueueCapacityConfigParser { private void setCapacityVector( QueueCapacityVector resource, String resourceName, String resourceValue) { - QueueCapacityType capacityType = QueueCapacityType.ABSOLUTE; + ResourceUnitCapacityType capacityType = ResourceUnitCapacityType.ABSOLUTE; // Extract suffix from a value e.g. for 6w extract w String suffix = resourceValue.replaceAll(FLOAT_DIGIT_REGEX, ""); @@ -180,7 +174,7 @@ public class QueueCapacityConfigParser { // Convert all incoming units to MB if units is configured. convertedValue = UnitsConversionUtil.convert(suffix, "Mi", (long) parsedResourceValue); } else { - for (QueueCapacityType capacityTypeSuffix : QueueCapacityType.values()) { + for (ResourceUnitCapacityType capacityTypeSuffix : ResourceUnitCapacityType.values()) { if (capacityTypeSuffix.getPostfix().equals(suffix)) { capacityType = capacityTypeSuffix; } @@ -198,8 +192,12 @@ public class QueueCapacityConfigParser { * false otherwise */ public boolean isCapacityVectorFormat(String configuredCapacity) { - return configuredCapacity != null - && RESOURCE_PATTERN.matcher(configuredCapacity).find(); + if (configuredCapacity == null) { + return false; + } + + String formattedCapacityString = configuredCapacity.replaceAll(" ", ""); + return RESOURCE_PATTERN.matcher(formattedCapacityString).find(); } private static class Parser { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java index b8f3fae7da6..10d98455851 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java @@ -27,9 +27,11 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeLabel; +import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; import org.apache.hadoop.yarn.nodelabels.NodeLabelsStore; +import org.apache.hadoop.yarn.nodelabels.RMNodeLabel; public class NullRMNodeLabelsManager extends RMNodeLabelsManager { Map> lastNodeToLabels = null; @@ -98,4 +100,24 @@ public class NullRMNodeLabelsManager extends RMNodeLabelsManager { conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true); super.serviceInit(conf); } + + public void setResourceForLabel(String label, Resource resource) { + if (label.equals(NO_LABEL)) { + noNodeLabel = new FakeLabel(resource); + return; + } + + labelCollections.put(label, new FakeLabel(label, resource)); + } + + private static class FakeLabel extends RMNodeLabel { + + FakeLabel(String label, Resource resource) { + super(label, resource, 1, false); + } + + FakeLabel(Resource resource) { + super(NO_LABEL, resource, 1, false); + } + } } 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/CapacitySchedulerQueueCalculationTestBase.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/CapacitySchedulerQueueCalculationTestBase.java new file mode 100644 index 00000000000..f62945c7a5a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueCalculationTestBase.java @@ -0,0 +1,131 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; +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.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.util.resource.ResourceCalculator; +import org.junit.Before; + +import java.io.IOException; + +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacitySchedulerAutoCreatedQueueBase.GB; + +public class CapacitySchedulerQueueCalculationTestBase { + protected static final String A = "root.a"; + protected static final String A1 = "root.a.a1"; + protected static final String A11 = "root.a.a1.a11"; + protected static final String A12 = "root.a.a1.a12"; + protected static final String A2 = "root.a.a2"; + protected static final String B = "root.b"; + protected static final String B1 = "root.b.b1"; + protected static final String C = "root.c"; + + private static final String CAPACITY_VECTOR_TEMPLATE = "[memory=%s, vcores=%s]"; + + protected ResourceCalculator resourceCalculator; + + protected MockRM mockRM; + protected CapacityScheduler cs; + protected CapacitySchedulerConfiguration csConf; + protected NullRMNodeLabelsManager mgr; + + @Before + public void setUp() throws Exception { + csConf = new CapacitySchedulerConfiguration(); + csConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + + csConf.setQueues("root", new String[]{"a", "b"}); + csConf.setCapacity("root.a", 50f); + csConf.setCapacity("root.b", 50f); + csConf.setQueues("root.a", new String[]{"a1", "a2"}); + csConf.setCapacity("root.a.a1", 100f); + csConf.setQueues("root.a.a1", new String[]{"a11", "a12"}); + csConf.setCapacity("root.a.a1.a11", 50f); + csConf.setCapacity("root.a.a1.a12", 50f); + + mgr = new NullRMNodeLabelsManager(); + mgr.init(csConf); + mockRM = new MockRM(csConf) { + protected RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + cs = (CapacityScheduler) mockRM.getResourceScheduler(); + cs.updatePlacementRules(); + // Policy for new auto created queue's auto deletion when expired + mockRM.start(); + cs.start(); + mockRM.registerNode("h1:1234", 10 * GB); // label = x + resourceCalculator = cs.getResourceCalculator(); + } + protected QueueCapacityUpdateContext update( + QueueAssertionBuilder assertions, Resource clusterResource) + throws IOException { + return update(assertions, clusterResource, clusterResource); + } + + protected QueueCapacityUpdateContext update( + QueueAssertionBuilder assertions, Resource clusterResource, Resource emptyLabelResource) + throws IOException { + cs.reinitialize(csConf, mockRM.getRMContext()); + + CapacitySchedulerQueueCapacityHandler queueController = + new CapacitySchedulerQueueCapacityHandler(mgr); + mgr.setResourceForLabel(CommonNodeLabelsManager.NO_LABEL, emptyLabelResource); + + queueController.updateRoot(cs.getQueue("root"), clusterResource); + QueueCapacityUpdateContext updateContext = + queueController.updateChildren(clusterResource, cs.getQueue("root")); + + assertions.finishAssertion(); + + return updateContext; + } + + protected QueueAssertionBuilder createAssertionBuilder() { + return new QueueAssertionBuilder(cs); + } + + protected static String createCapacityVector(Object memory, Object vcores) { + return String.format(CAPACITY_VECTOR_TEMPLATE, memory, vcores); + } + + protected static String absolute(double value) { + return String.valueOf((long) value); + } + + protected static String weight(float value) { + return value + "w"; + } + + protected static String percentage(float value) { + return value + "%"; + } + + protected static Resource createResource(double memory, double vcores) { + return Resource.newInstance((int) memory, (int) vcores); + } +} 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/QueueAssertionBuilder.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/QueueAssertionBuilder.java new file mode 100644 index 00000000000..1c066719dd0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueAssertionBuilder.java @@ -0,0 +1,210 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas; +import org.junit.Assert; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.BiFunction; +import java.util.function.Supplier; + +import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NO_LABEL; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils.EPSILON; + +/** + * Provides a fluent API to assert resource and capacity attributes of queues. + */ +class QueueAssertionBuilder { + private static final String EFFECTIVE_MAX_RES_INFO = "Effective Maximum Resource"; + private static final BiFunction EFFECTIVE_MAX_RES = + QueueResourceQuotas::getEffectiveMaxResource; + + private static final String EFFECTIVE_MIN_RES_INFO = "Effective Minimum Resource"; + private static final BiFunction EFFECTIVE_MIN_RES = + QueueResourceQuotas::getEffectiveMinResource; + + private static final String CAPACITY_INFO = "Capacity"; + private static final BiFunction CAPACITY = + QueueCapacities::getCapacity; + + private static final String ABS_CAPACITY_INFO = "Absolute Capacity"; + private static final BiFunction ABS_CAPACITY = + QueueCapacities::getAbsoluteCapacity; + + private static final String ASSERTION_ERROR_MESSAGE = + "'%s' of queue '%s' does not match %f for label %s"; + private static final String RESOURCE_ASSERTION_ERROR_MESSAGE = + "'%s' of queue '%s' does not match %s for label %s"; + private final CapacityScheduler cs; + + QueueAssertionBuilder(CapacityScheduler cs) { + this.cs = cs; + } + + public class QueueAssertion { + private final String queuePath; + private final List assertions = new ArrayList<>(); + + QueueAssertion(String queuePath) { + this.queuePath = queuePath; + } + + + public QueueAssertion withQueue(String queuePath) { + return QueueAssertionBuilder.this.withQueue(queuePath); + } + + public QueueAssertionBuilder build() { + return QueueAssertionBuilder.this.build(); + } + + public QueueAssertion assertEffectiveMaxResource(Resource expected) { + ValueAssertion valueAssertion = new ValueAssertion(expected); + valueAssertion.withResourceSupplier(EFFECTIVE_MAX_RES, EFFECTIVE_MAX_RES_INFO); + assertions.add(valueAssertion); + + return this; + } + + public QueueAssertion assertEffectiveMinResource(Resource expected, String label) { + ValueAssertion valueAssertion = new ValueAssertion(expected); + valueAssertion.withResourceSupplier(EFFECTIVE_MIN_RES, EFFECTIVE_MIN_RES_INFO); + assertions.add(valueAssertion); + valueAssertion.label = label; + + return this; + } + + public QueueAssertion assertEffectiveMinResource(Resource expected) { + return assertEffectiveMinResource(expected, NO_LABEL); + } + + public QueueAssertion assertCapacity(double expected) { + ValueAssertion valueAssertion = new ValueAssertion(expected); + valueAssertion.withCapacitySupplier(CAPACITY, CAPACITY_INFO); + assertions.add(valueAssertion); + + return this; + } + + public QueueAssertion assertAbsoluteCapacity(double expected) { + ValueAssertion valueAssertion = new ValueAssertion(expected); + valueAssertion.withCapacitySupplier(ABS_CAPACITY, ABS_CAPACITY_INFO); + assertions.add(valueAssertion); + + return this; + } + + private class ValueAssertion { + private double expectedValue = 0; + private Resource expectedResource = null; + private String assertionType; + private Supplier valueSupplier; + private Supplier resourceSupplier; + private String label = ""; + + ValueAssertion(double expectedValue) { + this.expectedValue = expectedValue; + } + + ValueAssertion(Resource expectedResource) { + this.expectedResource = expectedResource; + } + + public void setLabel(String label) { + this.label = label; + } + + public void withResourceSupplier( + BiFunction assertion, String messageInfo) { + CSQueue queue = cs.getQueue(queuePath); + if (queue == null) { + Assert.fail("Queue " + queuePath + " is not found"); + } + + assertionType = messageInfo; + resourceSupplier = () -> assertion.apply(queue.getQueueResourceQuotas(), label); + } + + public void withCapacitySupplier( + BiFunction assertion, String messageInfo) { + CSQueue queue = cs.getQueue(queuePath); + if (queue == null) { + Assert.fail("Queue " + queuePath + " is not found"); + } + assertionType = messageInfo; + valueSupplier = () -> assertion.apply(queue.getQueueCapacities(), label); + } + } + + } + + private final Map assertions = new LinkedHashMap<>(); + + public QueueAssertionBuilder build() { + return this; + } + + /** + * Creates a new assertion group for a specific queue. + * @param queuePath path of the queue + * @return queue assertion group + */ + public QueueAssertion withQueue(String queuePath) { + assertions.putIfAbsent(queuePath, new QueueAssertion(queuePath)); + return assertions.get(queuePath); + } + + /** + * Executes assertions created for all queues. + */ + public void finishAssertion() { + for (Map.Entry assertionEntry : assertions.entrySet()) { + for (QueueAssertion.ValueAssertion assertion : assertionEntry.getValue().assertions) { + if (assertion.resourceSupplier != null) { + String errorMessage = String.format(RESOURCE_ASSERTION_ERROR_MESSAGE, + assertion.assertionType, assertionEntry.getKey(), + assertion.expectedResource.toString(), assertion.label); + Assert.assertEquals(errorMessage, assertion.expectedResource, + assertion.resourceSupplier.get()); + } else { + String errorMessage = String.format(ASSERTION_ERROR_MESSAGE, + assertion.assertionType, assertionEntry.getKey(), assertion.expectedValue, + assertion.label); + Assert.assertEquals(errorMessage, assertion.expectedValue, + assertion.valueSupplier.get(), EPSILON); + } + } + } + } + + /** + * Returns all queues that have defined assertions. + * @return queue paths + */ + public Set getQueues() { + return assertions.keySet(); + } +} 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/TestMixedQueueResourceCalculation.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/TestMixedQueueResourceCalculation.java new file mode 100644 index 00000000000..e5b7cc964e3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestMixedQueueResourceCalculation.java @@ -0,0 +1,536 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.QueueState; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueUpdateWarning.QueueUpdateWarningType; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collection; +import java.util.Optional; + +import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NO_LABEL; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacitySchedulerAutoCreatedQueueBase.GB; + +public class TestMixedQueueResourceCalculation extends CapacitySchedulerQueueCalculationTestBase { + private static final long MEMORY = 16384; + private static final long VCORES = 16; + private static final String C_VECTOR_WITH_WARNING = createCapacityVector(weight(3), + absolute(VCORES * 0.25)); + private static final String A11_VECTOR_WITH_WARNING = createCapacityVector(weight(1), + absolute(VCORES * 0.25)); + private static final String A1_VECTOR_WITH_WARNING = createCapacityVector(absolute(2048), + absolute(VCORES * 0.25)); + private static final String C_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(weight(3), + absolute(VCORES * 0.25)); + private static final String A1_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(weight(1), + absolute(VCORES * 0.25)); + + private static final Resource A12_EXPECTED_MAX_RESOURCE_MAX_WARNINGS = + createResource(MEMORY * 0.5, VCORES); + private static final Resource A11_EXPECTED_MAX_RESOURCE_MAX_WARNINGS = + createResource(MEMORY * 0.5, 0.1 * VCORES); + private static final Resource A11_EXPECTED_MIN_RESOURCE_MAX_WARNINGS = + createResource(0.5 * 0.5 * MEMORY, 0.1 * VCORES); + private static final Resource A12_EXPECTED_MIN_RESOURCE_MAX_WARNINGS = + createResource(0.5 * 0.5 * MEMORY, 0); + private static final String A11_MAX_VECTOR_MAX_WARNINGS = + createCapacityVector(absolute(MEMORY), percentage(10)); + private static final String A1_MAX_VECTOR_MAX_WARNINGS = + createCapacityVector(absolute(MEMORY * 0.5), + percentage(100)); + + private static final Resource UPDATE_RESOURCE = Resource.newInstance(16384, 16); + private static final Resource ZERO_RESOURCE = Resource.newInstance(0, 0); + + private static final Resource A_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(2486, 9); + private static final Resource A1_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(621, 4); + private static final Resource A11_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(217, 1); + private static final Resource A12_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(403, 3); + private static final Resource A2_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(1865, 5); + private static final Resource B_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(8095, 3); + private static final Resource B1_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(8095, 3); + private static final Resource C_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(5803, 4); + + private static final Resource B_WARNING_RESOURCE = Resource.newInstance(8096, 4); + private static final Resource B1_WARNING_RESOURCE = Resource.newInstance(8096, 3); + private static final Resource A_WARNING_RESOURCE = Resource.newInstance(8288, 12); + private static final Resource A1_WARNING_RESOURCE = Resource.newInstance(2048, 4); + private static final Resource A2_WARNING_RESOURCE = Resource.newInstance(2048, 8); + private static final Resource A12_WARNING_RESOURCE = Resource.newInstance(2048, 4); + + private static final String A_VECTOR_ZERO_RESOURCE = + createCapacityVector(percentage(100), weight(6)); + private static final String B_VECTOR_ZERO_RESOURCE = + createCapacityVector(absolute(MEMORY), absolute(VCORES * 0.5)); + + private static final String A_MAX_VECTOR_DIFFERENT_MIN_MAX = + createCapacityVector(absolute(MEMORY), percentage(80)); + private static final Resource B_EXPECTED_MAX_RESOURCE_DIFFERENT_MIN_MAX = + Resource.newInstance(MEMORY, (int) (VCORES * 0.5)); + private static final Resource A_EXPECTED_MAX_RESOURCE_DIFFERENT_MIN_MAX = + Resource.newInstance(MEMORY, (int) (VCORES * 0.8)); + private static final String B_MAX_VECTOR_DIFFERENT_MIN_MAX = + createCapacityVector(absolute(MEMORY), absolute(VCORES * 0.5)); + private static final String A_MIN_VECTOR_DIFFERENT_MIN_MAX = + createCapacityVector(percentage(50), absolute(VCORES * 0.5)); + private static final String B_MIN_VECTOR_DIFFERENT_MIN_MAX = + createCapacityVector(weight(6), percentage(100)); + private static final String B_INVALID_MAX_VECTOR = + createCapacityVector(absolute(MEMORY), weight(10)); + + private static final String X_LABEL = "x"; + private static final String Y_LABEL = "y"; + private static final String Z_LABEL = "z"; + + private static final String H1_NODE = "h1"; + private static final String H2_NODE = "h2"; + private static final String H3_NODE = "h3"; + private static final String H4_NODE = "h4"; + private static final String H5_NODE = "h5"; + private static final int H1_MEMORY = 60 * GB; + private static final int H1_VCORES = 60; + private static final int H2_MEMORY = 10 * GB; + private static final int H2_VCORES = 25; + private static final int H3_VCORES = 35; + private static final int H3_MEMORY = 10 * GB; + private static final int H4_MEMORY = 10 * GB; + private static final int H4_VCORES = 15; + + private static final String A11_MIN_VECTOR_MAX_WARNINGS = + createCapacityVector(percentage(50), percentage(100)); + private static final String A12_MIN_VECTOR_MAX_WARNINGS = + createCapacityVector(percentage(50), percentage(0)); + + private static final Resource A_EXPECTED_MIN_RESOURCE_NO_LABEL = createResource(2048, 8); + private static final Resource A1_EXPECTED_MIN_RESOURCE_NO_LABEL = createResource(1024, 5); + private static final Resource A2_EXPECTED_MIN_RESOURCE_NO_LABEL = createResource(1024, 2); + private static final Resource B_EXPECTED_MIN_RESOURCE_NO_LABEL = createResource(3072, 8); + private static final Resource A_EXPECTED_MIN_RESOURCE_X_LABEL = createResource(30720, 30); + private static final Resource A1_EXPECTED_MIN_RESOURCE_X_LABEL = createResource(20480, 0); + private static final Resource A2_EXPECTED_MIN_RESOURCE_X_LABEL = createResource(10240, 30); + private static final Resource B_EXPECTED_MIN_RESOURCE_X_LABEL = createResource(30720, 30); + private static final Resource A_EXPECTED_MIN_RESOURCE_Y_LABEL = createResource(8096, 42); + private static final Resource A1_EXPECTED_MIN_RESOURCE_Y_LABEL = createResource(6186, 21); + private static final Resource A2_EXPECTED_MIN_RESOURCE_Y_LABEL = createResource(1910, 21); + private static final Resource B_EXPECTED_MIN_RESOURCE_Y_LABEL = createResource(12384, 18); + private static final Resource A_EXPECTED_MIN_RESOURCE_Z_LABEL = createResource(7168, 11); + private static final Resource A1_EXPECTED_MIN_RESOURCE_Z_LABEL = createResource(6451, 4); + private static final Resource A2_EXPECTED_MIN_RESOURCE_Z_LABEL = createResource(716, 7); + private static final Resource B_EXPECTED_MIN_RESOURCE_Z_LABEL = createResource(3072, 4); + private static final Resource EMPTY_LABEL_RESOURCE = Resource.newInstance(5 * GB, 16); + + private static final String A_VECTOR_NO_LABEL = + createCapacityVector(absolute(2048), percentage(50)); + private static final String A1_VECTOR_NO_LABEL = + createCapacityVector(absolute(1024), percentage(70)); + private static final String A2_VECTOR_NO_LABEL = + createCapacityVector(absolute(1024), percentage(30)); + private static final String B_VECTOR_NO_LABEL = + createCapacityVector(weight(3), percentage(50)); + private static final String A_VECTOR_X_LABEL = + createCapacityVector(percentage(50), weight(3)); + private static final String A1_VECTOR_X_LABEL = + createCapacityVector(absolute(20480), percentage(10)); + private static final String A2_VECTOR_X_LABEL = + createCapacityVector(absolute(10240), absolute(30)); + private static final String B_VECTOR_X_LABEL = + createCapacityVector(percentage(50), percentage(50)); + private static final String A_VECTOR_Y_LABEL = + createCapacityVector(absolute(8096), weight(1)); + private static final String A1_VECTOR_Y_LABEL = + createCapacityVector(absolute(6186), weight(3)); + private static final String A2_VECTOR_Y_LABEL = + createCapacityVector(weight(3), weight(3)); + private static final String B_VECTOR_Y_LABEL = + createCapacityVector(percentage(100), percentage(30)); + private static final String A_VECTOR_Z_LABEL = + createCapacityVector(percentage(70), absolute(11)); + private static final String A1_VECTOR_Z_LABEL = + createCapacityVector(percentage(90), percentage(40)); + private static final String A2_VECTOR_Z_LABEL = + createCapacityVector(percentage(10), weight(4)); + private static final String B_VECTOR_Z_LABEL = + createCapacityVector(percentage(30), absolute(4)); + + private static final String A_VECTOR_NO_REMAINING_RESOURCE = + createCapacityVector(percentage(30), weight(6)); + private static final String A11_VECTOR_NO_REMAINING_RESOURCE = + createCapacityVector(percentage(35), percentage(25)); + private static final String A12_VECTOR_NO_REMAINING_RESOURCE = + createCapacityVector(percentage(65), percentage(75)); + private static final String A2_VECTOR_NO_REMAINING_RESOURCE = + createCapacityVector(weight(3), percentage(100)); + private static final String B_VECTOR_NO_REMAINING_RESOURCE = + createCapacityVector(absolute(8095), percentage(30)); + private static final String B1_VECTOR_NO_REMAINING_RESOURCE = + createCapacityVector(weight(5), absolute(3)); + private static final String A_VECTOR_WITH_WARNINGS = + createCapacityVector(percentage(100), weight(6)); + private static final String A12_VECTOR_WITH_WARNING = + createCapacityVector(percentage(100), percentage(100)); + private static final String A2_VECTOR_WITH_WARNING = + createCapacityVector(absolute(2048), percentage(100)); + private static final String B_VECTOR_WITH_WARNING = + createCapacityVector(absolute(8096), percentage(30)); + private static final String B1_VECTOR_WITH_WARNING = + createCapacityVector(absolute(10256), absolute(3)); + + @Override + public void setUp() throws Exception { + super.setUp(); + csConf.setLegacyQueueModeEnabled(false); + } + + /** + * Tests a complex scenario in which no warning or remaining resource is generated during the + * update phase (except for rounding leftovers, eg. 1 memory or 1 vcores). + * + * -root- + * / \ \ + * A B C + * / \ | + * A1 A2 B1 + * / \ + * A11 A12 + * + * @throws IOException if update is failed + */ + @Test + public void testComplexHierarchyWithoutRemainingResource() throws IOException { + setupQueueHierarchyWithoutRemainingResource(); + + QueueAssertionBuilder assertionBuilder = createAssertionBuilder() + .withQueue(A) + .assertEffectiveMinResource(A_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .withQueue(A1) + .assertEffectiveMinResource(A1_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A1_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .withQueue(A11) + .assertEffectiveMinResource(A11_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A11_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .withQueue(A12) + .assertEffectiveMinResource(A12_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A12_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .withQueue(A2) + .assertEffectiveMinResource(A2_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A2_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .withQueue(B) + .assertEffectiveMinResource(B_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + B_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .withQueue(B1) + .assertEffectiveMinResource(B1_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + B1_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .withQueue(C) + .assertEffectiveMinResource(C_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + C_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .build(); + + update(assertionBuilder, UPDATE_RESOURCE); + } + + /** + * Tests a complex scenario in which several validation warnings are generated during the update + * phase. + * + * -root- + * / \ \ + * A B C + * / \ | + * A1 A2 B1 + * / \ + * A11 A12 + * + * @throws IOException if update is failed + */ + @Test + public void testComplexHierarchyWithWarnings() throws IOException { + setupQueueHierarchyWithWarnings(); + QueueAssertionBuilder assertionBuilder = createAssertionBuilder() + .withQueue(A) + .assertEffectiveMinResource(A_WARNING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A_WARNING_RESOURCE, UPDATE_RESOURCE)) + .withQueue(A1) + .assertEffectiveMinResource(A1_WARNING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A1_WARNING_RESOURCE, UPDATE_RESOURCE)) + .withQueue(A2) + .assertEffectiveMinResource(A2_WARNING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A2_WARNING_RESOURCE, UPDATE_RESOURCE)) + .withQueue(A11) + .assertEffectiveMinResource(ZERO_RESOURCE) + .assertAbsoluteCapacity(0) + .withQueue(A12) + .assertEffectiveMinResource(A12_WARNING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A12_WARNING_RESOURCE, UPDATE_RESOURCE)) + .withQueue(B) + .assertEffectiveMinResource(B_WARNING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + B_WARNING_RESOURCE, UPDATE_RESOURCE)) + .withQueue(B1) + .assertEffectiveMinResource(B1_WARNING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + B1_WARNING_RESOURCE, UPDATE_RESOURCE)) + .withQueue(C) + .assertEffectiveMinResource(ZERO_RESOURCE) + .assertAbsoluteCapacity(0) + .build(); + + QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); + Optional queueCZeroResourceWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, C); + Optional queueARemainingResourceWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.BRANCH_UNDERUTILIZED, A); + Optional queueBDownscalingWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.BRANCH_DOWNSCALED, B); + Optional queueA11ZeroResourceWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, A11); + + Assert.assertTrue(queueCZeroResourceWarning.isPresent()); + Assert.assertTrue(queueARemainingResourceWarning.isPresent()); + Assert.assertTrue(queueBDownscalingWarning.isPresent()); + Assert.assertTrue(queueA11ZeroResourceWarning.isPresent()); + } + + @Test + public void testZeroResourceIfNoMemory() throws IOException { + csConf.setCapacityVector(A, NO_LABEL, A_VECTOR_ZERO_RESOURCE); + csConf.setCapacityVector(B, NO_LABEL, B_VECTOR_ZERO_RESOURCE); + + QueueAssertionBuilder assertionBuilder = createAssertionBuilder() + .withQueue(A) + .assertEffectiveMinResource(ZERO_RESOURCE) + .withQueue(B) + .assertEffectiveMinResource(createResource(MEMORY, VCORES * 0.5)) + .build(); + + QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); + Optional queueAZeroResourceWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, A); + Optional rootUnderUtilizedWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.BRANCH_UNDERUTILIZED, ROOT); + Assert.assertTrue(queueAZeroResourceWarning.isPresent()); + Assert.assertTrue(rootUnderUtilizedWarning.isPresent()); + } + + @Test + public void testDifferentMinimumAndMaximumCapacityTypes() throws IOException { + csConf.setCapacityVector(A, NO_LABEL, A_MIN_VECTOR_DIFFERENT_MIN_MAX); + csConf.setMaximumCapacityVector(A, NO_LABEL, A_MAX_VECTOR_DIFFERENT_MIN_MAX); + csConf.setCapacityVector(B, NO_LABEL, B_MIN_VECTOR_DIFFERENT_MIN_MAX); + csConf.setMaximumCapacityVector(B, NO_LABEL, B_MAX_VECTOR_DIFFERENT_MIN_MAX); + + QueueAssertionBuilder assertionBuilder = createAssertionBuilder() + .withQueue(A) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(UPDATE_RESOURCE, 0.5d)) + .assertEffectiveMaxResource(A_EXPECTED_MAX_RESOURCE_DIFFERENT_MIN_MAX) + .withQueue(B) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(UPDATE_RESOURCE, 0.5d)) + .assertEffectiveMaxResource(B_EXPECTED_MAX_RESOURCE_DIFFERENT_MIN_MAX) + .build(); + + QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); + Assert.assertEquals(0, updateContext.getUpdateWarnings().size()); + + // WEIGHT capacity type for maximum capacity is not supported + csConf.setMaximumCapacityVector(B, NO_LABEL, B_INVALID_MAX_VECTOR); + try { + cs.reinitialize(csConf, mockRM.getRMContext()); + update(assertionBuilder, UPDATE_RESOURCE); + Assert.fail("WEIGHT maximum capacity type is not supported, an error should be thrown when " + + "set up"); + } catch (IllegalStateException ignored) { + } + } + + @Test + public void testMaximumResourceWarnings() throws IOException { + csConf.setMaximumCapacityVector(A1, NO_LABEL, A1_MAX_VECTOR_MAX_WARNINGS); + csConf.setCapacityVector(A11, NO_LABEL, A11_MIN_VECTOR_MAX_WARNINGS); + csConf.setCapacityVector(A12, NO_LABEL, A12_MIN_VECTOR_MAX_WARNINGS); + csConf.setMaximumCapacityVector(A11, NO_LABEL, A11_MAX_VECTOR_MAX_WARNINGS); + + QueueAssertionBuilder assertionBuilder = createAssertionBuilder() + .withQueue(A11) + .assertEffectiveMinResource(A11_EXPECTED_MIN_RESOURCE_MAX_WARNINGS) + .assertEffectiveMaxResource(A11_EXPECTED_MAX_RESOURCE_MAX_WARNINGS) + .withQueue(A12) + .assertEffectiveMinResource(A12_EXPECTED_MIN_RESOURCE_MAX_WARNINGS) + .assertEffectiveMaxResource(A12_EXPECTED_MAX_RESOURCE_MAX_WARNINGS) + .build(); + + QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); + Optional queueA11ExceedsParentMaxResourceWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT, + A11); + Optional queueA11MinExceedsMaxWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_EXCEEDS_MAX_RESOURCE, A11); + Assert.assertTrue(queueA11ExceedsParentMaxResourceWarning.isPresent()); + Assert.assertTrue(queueA11MinExceedsMaxWarning.isPresent()); + } + + @Test + public void testNodeLabels() throws Exception { + setLabeledQueueConfigs(); + + QueueAssertionBuilder assertionBuilder = createAssertionBuilder() + .withQueue(A) + .assertEffectiveMinResource(A_EXPECTED_MIN_RESOURCE_NO_LABEL, NO_LABEL) + .withQueue(A1) + .assertEffectiveMinResource(A1_EXPECTED_MIN_RESOURCE_NO_LABEL, NO_LABEL) + .withQueue(A2) + .assertEffectiveMinResource(A2_EXPECTED_MIN_RESOURCE_NO_LABEL, NO_LABEL) + .withQueue(B) + .assertEffectiveMinResource(B_EXPECTED_MIN_RESOURCE_NO_LABEL, NO_LABEL) + .withQueue(A) + .assertEffectiveMinResource(A_EXPECTED_MIN_RESOURCE_X_LABEL, X_LABEL) + .withQueue(A1) + .assertEffectiveMinResource(A1_EXPECTED_MIN_RESOURCE_X_LABEL, X_LABEL) + .withQueue(A2) + .assertEffectiveMinResource(A2_EXPECTED_MIN_RESOURCE_X_LABEL, X_LABEL) + .withQueue(B) + .assertEffectiveMinResource(B_EXPECTED_MIN_RESOURCE_X_LABEL, X_LABEL) + .withQueue(A) + .assertEffectiveMinResource(A_EXPECTED_MIN_RESOURCE_Y_LABEL, Y_LABEL) + .withQueue(A1) + .assertEffectiveMinResource(A1_EXPECTED_MIN_RESOURCE_Y_LABEL, Y_LABEL) + .withQueue(A2) + .assertEffectiveMinResource(A2_EXPECTED_MIN_RESOURCE_Y_LABEL, Y_LABEL) + .withQueue(B) + .assertEffectiveMinResource(B_EXPECTED_MIN_RESOURCE_Y_LABEL, Y_LABEL) + .withQueue(A) + .assertEffectiveMinResource(A_EXPECTED_MIN_RESOURCE_Z_LABEL, Z_LABEL) + .withQueue(A1) + .assertEffectiveMinResource(A1_EXPECTED_MIN_RESOURCE_Z_LABEL, Z_LABEL) + .withQueue(A2) + .assertEffectiveMinResource(A2_EXPECTED_MIN_RESOURCE_Z_LABEL, Z_LABEL) + .withQueue(B) + .assertEffectiveMinResource(B_EXPECTED_MIN_RESOURCE_Z_LABEL, Z_LABEL) + .build(); + + update(assertionBuilder, UPDATE_RESOURCE, EMPTY_LABEL_RESOURCE); + } + + private void setLabeledQueueConfigs() throws Exception { + mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of(X_LABEL, Y_LABEL, Z_LABEL)); + mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance(H1_NODE, 0), + TestUtils.toSet(X_LABEL), NodeId.newInstance(H2_NODE, 0), + TestUtils.toSet(Y_LABEL), NodeId.newInstance(H3_NODE, 0), + TestUtils.toSet(Y_LABEL), NodeId.newInstance(H4_NODE, 0), + TestUtils.toSet(Z_LABEL), NodeId.newInstance(H5_NODE, 0), + RMNodeLabelsManager.EMPTY_STRING_SET)); + + mockRM.registerNode("h1:1234", H1_MEMORY, H1_VCORES); // label = x + mockRM.registerNode("h2:1234", H2_MEMORY, H2_VCORES); // label = y + mockRM.registerNode("h3:1234", H3_MEMORY, H3_VCORES); // label = y + mockRM.registerNode("h4:1234", H4_MEMORY, H4_VCORES); // label = z + + csConf.setCapacityVector(A, NO_LABEL, A_VECTOR_NO_LABEL); + csConf.setCapacityVector(A1, NO_LABEL, A1_VECTOR_NO_LABEL); + csConf.setCapacityVector(A2, NO_LABEL, A2_VECTOR_NO_LABEL); + csConf.setCapacityVector(B, NO_LABEL, B_VECTOR_NO_LABEL); + + csConf.setCapacityVector(A, X_LABEL, A_VECTOR_X_LABEL); + csConf.setCapacityVector(A1, X_LABEL, A1_VECTOR_X_LABEL); + csConf.setCapacityVector(A2, X_LABEL, A2_VECTOR_X_LABEL); + csConf.setCapacityVector(B, X_LABEL, B_VECTOR_X_LABEL); + + csConf.setCapacityVector(A, Y_LABEL, A_VECTOR_Y_LABEL); + csConf.setCapacityVector(A1, Y_LABEL, A1_VECTOR_Y_LABEL); + csConf.setCapacityVector(A2, Y_LABEL, A2_VECTOR_Y_LABEL); + csConf.setCapacityVector(B, Y_LABEL, B_VECTOR_Y_LABEL); + + csConf.setCapacityVector(A, Z_LABEL, A_VECTOR_Z_LABEL); + csConf.setCapacityVector(A1, Z_LABEL, A1_VECTOR_Z_LABEL); + csConf.setCapacityVector(A2, Z_LABEL, A2_VECTOR_Z_LABEL); + csConf.setCapacityVector(B, Z_LABEL, B_VECTOR_Z_LABEL); + + cs.reinitialize(csConf, mockRM.getRMContext()); + } + + private void setupQueueHierarchyWithoutRemainingResource() throws IOException { + csConf.setState(B, QueueState.STOPPED); + cs.reinitialize(csConf, mockRM.getRMContext()); + setQueues(); + + csConf.setState(B, QueueState.RUNNING); + csConf.setCapacityVector(A, NO_LABEL, A_VECTOR_NO_REMAINING_RESOURCE); + csConf.setCapacityVector(A1, NO_LABEL, A1_VECTOR_NO_REMAINING_RESOURCE); + csConf.setCapacityVector(A11, NO_LABEL, A11_VECTOR_NO_REMAINING_RESOURCE); + csConf.setCapacityVector(A12, NO_LABEL, A12_VECTOR_NO_REMAINING_RESOURCE); + csConf.setCapacityVector(A2, NO_LABEL, A2_VECTOR_NO_REMAINING_RESOURCE); + csConf.setCapacityVector(B, NO_LABEL, B_VECTOR_NO_REMAINING_RESOURCE); + csConf.setCapacityVector(B1, NO_LABEL, B1_VECTOR_NO_REMAINING_RESOURCE); + csConf.setCapacityVector(C, NO_LABEL, C_VECTOR_NO_REMAINING_RESOURCE); + + cs.reinitialize(csConf, mockRM.getRMContext()); + } + + private void setupQueueHierarchyWithWarnings() throws IOException { + csConf.setState(B, QueueState.STOPPED); + cs.reinitialize(csConf, mockRM.getRMContext()); + setQueues(); + + csConf.setState(B, QueueState.RUNNING); + csConf.setCapacityVector(A, NO_LABEL, A_VECTOR_WITH_WARNINGS); + csConf.setCapacityVector(A1, NO_LABEL, A1_VECTOR_WITH_WARNING); + csConf.setCapacityVector(A11, NO_LABEL, A11_VECTOR_WITH_WARNING); + csConf.setCapacityVector(A12, NO_LABEL, A12_VECTOR_WITH_WARNING); + csConf.setCapacityVector(A2, NO_LABEL, A2_VECTOR_WITH_WARNING); + csConf.setCapacityVector(B, NO_LABEL, B_VECTOR_WITH_WARNING); + csConf.setCapacityVector(B1, NO_LABEL, B1_VECTOR_WITH_WARNING); + csConf.setCapacityVector(C, NO_LABEL, C_VECTOR_WITH_WARNING); + + cs.reinitialize(csConf, mockRM.getRMContext()); + } + + private void setQueues() { + csConf.setQueues("root", new String[]{"a", "b", "c"}); + csConf.setQueues(A, new String[]{"a1", "a2"}); + csConf.setQueues(B, new String[]{"b1"}); + } + + private Optional getSpecificWarning( + Collection warnings, QueueUpdateWarningType warningTypeToSelect, + String queue) { + return warnings.stream().filter((w) -> w.getWarningType().equals(warningTypeToSelect) + && w.getQueue().equals(queue)).findFirst(); + } +} 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/TestQueueCapacityVector.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/TestQueueCapacityVector.java index 058e14bfaf2..18eead5d8ec 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/TestQueueCapacityVector.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/TestQueueCapacityVector.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.util.Lists; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.junit.Assert; @@ -50,21 +50,21 @@ public class TestQueueCapacityVector { public void getResourceNamesByCapacityType() { QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); - capacityVector.setResource(MEMORY_URI, 10, QueueCapacityType.PERCENTAGE); - capacityVector.setResource(VCORES_URI, 6, QueueCapacityType.PERCENTAGE); + capacityVector.setResource(MEMORY_URI, 10, ResourceUnitCapacityType.PERCENTAGE); + capacityVector.setResource(VCORES_URI, 6, ResourceUnitCapacityType.PERCENTAGE); // custom is not set, defaults to 0 Assert.assertEquals(1, capacityVector.getResourceNamesByCapacityType( - QueueCapacityType.ABSOLUTE).size()); + ResourceUnitCapacityType.ABSOLUTE).size()); Assert.assertTrue(capacityVector.getResourceNamesByCapacityType( - QueueCapacityType.ABSOLUTE).contains(CUSTOM_RESOURCE)); + ResourceUnitCapacityType.ABSOLUTE).contains(CUSTOM_RESOURCE)); Assert.assertEquals(2, capacityVector.getResourceNamesByCapacityType( - QueueCapacityType.PERCENTAGE).size()); + ResourceUnitCapacityType.PERCENTAGE).size()); Assert.assertTrue(capacityVector.getResourceNamesByCapacityType( - QueueCapacityType.PERCENTAGE).contains(VCORES_URI)); + ResourceUnitCapacityType.PERCENTAGE).contains(VCORES_URI)); Assert.assertTrue(capacityVector.getResourceNamesByCapacityType( - QueueCapacityType.PERCENTAGE).contains(MEMORY_URI)); + ResourceUnitCapacityType.PERCENTAGE).contains(MEMORY_URI)); Assert.assertEquals(10, capacityVector.getResource(MEMORY_URI).getResourceValue(), EPSILON); Assert.assertEquals(6, capacityVector.getResource(VCORES_URI).getResourceValue(), EPSILON); } @@ -73,13 +73,15 @@ public class TestQueueCapacityVector { public void isResourceOfType() { QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); - capacityVector.setResource(MEMORY_URI, 10, QueueCapacityType.WEIGHT); - capacityVector.setResource(VCORES_URI, 6, QueueCapacityType.PERCENTAGE); - capacityVector.setResource(CUSTOM_RESOURCE, 3, QueueCapacityType.ABSOLUTE); + capacityVector.setResource(MEMORY_URI, 10, ResourceUnitCapacityType.WEIGHT); + capacityVector.setResource(VCORES_URI, 6, ResourceUnitCapacityType.PERCENTAGE); + capacityVector.setResource(CUSTOM_RESOURCE, 3, ResourceUnitCapacityType.ABSOLUTE); - Assert.assertTrue(capacityVector.isResourceOfType(MEMORY_URI, QueueCapacityType.WEIGHT)); - Assert.assertTrue(capacityVector.isResourceOfType(VCORES_URI, QueueCapacityType.PERCENTAGE)); - Assert.assertTrue(capacityVector.isResourceOfType(CUSTOM_RESOURCE, QueueCapacityType.ABSOLUTE)); + Assert.assertTrue(capacityVector.isResourceOfType(MEMORY_URI, ResourceUnitCapacityType.WEIGHT)); + Assert.assertTrue(capacityVector.isResourceOfType(VCORES_URI, + ResourceUnitCapacityType.PERCENTAGE)); + Assert.assertTrue(capacityVector.isResourceOfType(CUSTOM_RESOURCE, + ResourceUnitCapacityType.ABSOLUTE)); } @Test @@ -99,9 +101,9 @@ public class TestQueueCapacityVector { public void testToString() { QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); - capacityVector.setResource(MEMORY_URI, 10, QueueCapacityType.WEIGHT); - capacityVector.setResource(VCORES_URI, 6, QueueCapacityType.PERCENTAGE); - capacityVector.setResource(CUSTOM_RESOURCE, 3, QueueCapacityType.ABSOLUTE); + capacityVector.setResource(MEMORY_URI, 10, ResourceUnitCapacityType.WEIGHT); + capacityVector.setResource(VCORES_URI, 6, ResourceUnitCapacityType.PERCENTAGE); + capacityVector.setResource(CUSTOM_RESOURCE, 3, ResourceUnitCapacityType.ABSOLUTE); Assert.assertEquals(MIXED_CAPACITY_VECTOR_STRING, capacityVector.toString()); 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/TestResourceVector.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/TestResourceVector.java index fd6edb1fa5d..c56b37dc990 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/TestResourceVector.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/TestResourceVector.java @@ -68,7 +68,7 @@ public class TestResourceVector { public void testSubtract() { ResourceVector lhsResourceVector = ResourceVector.of(13); ResourceVector rhsResourceVector = ResourceVector.of(5); - lhsResourceVector.subtract(rhsResourceVector); + lhsResourceVector.decrement(rhsResourceVector); Assert.assertEquals(8, lhsResourceVector.getValue(MEMORY_URI), EPSILON); Assert.assertEquals(8, lhsResourceVector.getValue(VCORES_URI), EPSILON); @@ -77,7 +77,7 @@ public class TestResourceVector { ResourceVector negativeResourceVector = ResourceVector.of(-100); // Check whether overflow causes any issues - negativeResourceVector.subtract(ResourceVector.of(Float.MAX_VALUE)); + negativeResourceVector.decrement(ResourceVector.of(Float.MAX_VALUE)); Assert.assertEquals(-Float.MAX_VALUE, negativeResourceVector.getValue(MEMORY_URI), EPSILON); Assert.assertEquals(-Float.MAX_VALUE, negativeResourceVector.getValue(VCORES_URI), EPSILON); Assert.assertEquals(-Float.MAX_VALUE, negativeResourceVector.getValue(CUSTOM_RESOURCE), @@ -111,7 +111,7 @@ public class TestResourceVector { Assert.assertNotEquals(resource, resourceVector); ResourceVector resourceVectorOne = ResourceVector.of(1); - resourceVectorOther.subtract(resourceVectorOne); + resourceVectorOther.decrement(resourceVectorOne); Assert.assertEquals(resourceVectorOther, resourceVector); } 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/TestUniformQueueResourceCalculation.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/TestUniformQueueResourceCalculation.java new file mode 100644 index 00000000000..863baaaaf95 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUniformQueueResourceCalculation.java @@ -0,0 +1,191 @@ +/** + * 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.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; +import org.junit.Test; + +import java.io.IOException; + +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacitySchedulerAutoCreatedQueueBase.GB; + +public class TestUniformQueueResourceCalculation extends CapacitySchedulerQueueCalculationTestBase { + + private static final Resource QUEUE_A_RES = Resource.newInstance(80 * GB, + 10); + private static final Resource QUEUE_B_RES = Resource.newInstance(170 * GB, + 30); + private static final Resource QUEUE_A1_RES = Resource.newInstance(50 * GB, + 4); + private static final Resource QUEUE_A2_RES = Resource.newInstance(30 * GB, + 6); + private static final Resource QUEUE_A11_RES = Resource.newInstance(40 * GB, + 2); + private static final Resource QUEUE_A12_RES = Resource.newInstance(10 * GB, + 2); + private static final Resource UPDATE_RES = Resource.newInstance(250 * GB, 40); + private static final Resource PERCENTAGE_ALL_RES = Resource.newInstance(10 * GB, 20); + + public static final double A_CAPACITY = 0.3; + public static final double B_CAPACITY = 0.7; + public static final double A1_CAPACITY = 0.17; + public static final double A11_CAPACITY = 0.25; + public static final double A12_CAPACITY = 0.75; + public static final double A2_CAPACITY = 0.83; + + public static final float A_WEIGHT = 3; + public static final float B_WEIGHT = 6; + public static final float A1_WEIGHT = 2; + public static final float A11_WEIGHT = 5; + public static final float A12_WEIGHT = 8; + public static final float A2_WEIGHT = 3; + + public static final double A_NORMALIZED_WEIGHT = A_WEIGHT / (A_WEIGHT + B_WEIGHT); + public static final double B_NORMALIZED_WEIGHT = B_WEIGHT / (A_WEIGHT + B_WEIGHT); + public static final double A1_NORMALIZED_WEIGHT = A1_WEIGHT / (A1_WEIGHT + A2_WEIGHT); + public static final double A2_NORMALIZED_WEIGHT = A2_WEIGHT / (A1_WEIGHT + A2_WEIGHT); + public static final double A11_NORMALIZED_WEIGHT = A11_WEIGHT / (A11_WEIGHT + A12_WEIGHT); + public static final double A12_NORMALIZED_WEIGHT = A12_WEIGHT / (A11_WEIGHT + A12_WEIGHT); + + @Test + public void testWeightResourceCalculation() throws IOException { + csConf.setNonLabeledQueueWeight(A, A_WEIGHT); + csConf.setNonLabeledQueueWeight(B, B_WEIGHT); + csConf.setNonLabeledQueueWeight(A1, A1_WEIGHT); + csConf.setNonLabeledQueueWeight(A11, A11_WEIGHT); + csConf.setNonLabeledQueueWeight(A12, A12_WEIGHT); + csConf.setNonLabeledQueueWeight(A2, A2_WEIGHT); + + QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() + .withQueue(A) + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT)) + .assertAbsoluteCapacity(A_NORMALIZED_WEIGHT) + .withQueue(B) + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, B_NORMALIZED_WEIGHT)) + .assertAbsoluteCapacity(B_NORMALIZED_WEIGHT) + .withQueue(A1) + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, + A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT)) + .assertAbsoluteCapacity(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT) + .withQueue(A2) + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, + A_NORMALIZED_WEIGHT * A2_NORMALIZED_WEIGHT)) + .assertAbsoluteCapacity(A_NORMALIZED_WEIGHT * A2_NORMALIZED_WEIGHT) + .withQueue(A11) + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, + A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A11_NORMALIZED_WEIGHT)) + .assertAbsoluteCapacity(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A11_NORMALIZED_WEIGHT) + .withQueue(A12) + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, + A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A12_NORMALIZED_WEIGHT)) + .assertAbsoluteCapacity(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A12_NORMALIZED_WEIGHT) + .build(); + + update(queueAssertionBuilder, UPDATE_RES); + } + + @Test + public void testPercentageResourceCalculation() throws IOException { + csConf.setCapacity(A, (float) (A_CAPACITY * 100)); + csConf.setCapacity(B, (float) (B_CAPACITY * 100)); + csConf.setCapacity(A1, (float) (A1_CAPACITY * 100)); + csConf.setCapacity(A11, (float) (A11_CAPACITY * 100)); + csConf.setCapacity(A12, (float) (A12_CAPACITY * 100)); + csConf.setCapacity(A2, (float) (A2_CAPACITY * 100)); + + QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() + .withQueue(A) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(PERCENTAGE_ALL_RES, A_CAPACITY)) + .assertCapacity(A_CAPACITY) + .assertAbsoluteCapacity(A_CAPACITY) + .withQueue(B) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(PERCENTAGE_ALL_RES, B_CAPACITY)) + .assertCapacity(B_CAPACITY) + .assertAbsoluteCapacity(B_CAPACITY) + .withQueue(A1) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(PERCENTAGE_ALL_RES, + A_CAPACITY * A1_CAPACITY)) + .assertCapacity(A1_CAPACITY) + .assertAbsoluteCapacity(A_CAPACITY * A1_CAPACITY) + .withQueue(A2) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(PERCENTAGE_ALL_RES, + A_CAPACITY * A2_CAPACITY)) + .assertCapacity(A2_CAPACITY) + .assertAbsoluteCapacity(A_CAPACITY * A2_CAPACITY) + .withQueue(A11) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(PERCENTAGE_ALL_RES, + A11_CAPACITY * A_CAPACITY * A1_CAPACITY)) + .assertCapacity(A11_CAPACITY) + .assertAbsoluteCapacity(A11_CAPACITY * A_CAPACITY * A1_CAPACITY) + .withQueue(A12) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(PERCENTAGE_ALL_RES, + A12_CAPACITY * A_CAPACITY * A1_CAPACITY)) + .assertCapacity(A12_CAPACITY) + .assertAbsoluteCapacity(A12_CAPACITY * A_CAPACITY * A1_CAPACITY) + .build(); + + update(queueAssertionBuilder, PERCENTAGE_ALL_RES); + } + + @Test + public void testAbsoluteResourceCalculation() throws IOException { + csConf.setMinimumResourceRequirement("", new QueuePath(A), QUEUE_A_RES); + csConf.setMinimumResourceRequirement("", new QueuePath(B), QUEUE_B_RES); + csConf.setMinimumResourceRequirement("", new QueuePath(A1), QUEUE_A1_RES); + csConf.setMinimumResourceRequirement("", new QueuePath(A2), QUEUE_A2_RES); + csConf.setMinimumResourceRequirement("", new QueuePath(A11), QUEUE_A11_RES); + csConf.setMinimumResourceRequirement("", new QueuePath(A12), QUEUE_A12_RES); + + QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() + .withQueue(A) + .assertEffectiveMinResource(QUEUE_A_RES) + .withQueue(B) + .assertEffectiveMinResource(QUEUE_B_RES) + .withQueue(A1) + .assertEffectiveMinResource(QUEUE_A1_RES) + .withQueue(A2) + .assertEffectiveMinResource(QUEUE_A2_RES) + .withQueue(A11) + .assertEffectiveMinResource(QUEUE_A11_RES) + .withQueue(A12) + .assertEffectiveMinResource(QUEUE_A12_RES) + .build(); + + update(queueAssertionBuilder, UPDATE_RES); + + QueueAssertionBuilder queueAssertionHalfClusterResource = createAssertionBuilder() + .withQueue(A) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(QUEUE_A_RES, 0.5f)) + .withQueue(B) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(QUEUE_B_RES, 0.5f)) + .withQueue(A1) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(QUEUE_A1_RES, 0.5f)) + .withQueue(A2) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(QUEUE_A2_RES, 0.5f)) + .withQueue(A11) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(QUEUE_A11_RES, 0.5f)) + .withQueue(A12) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(QUEUE_A12_RES, 0.5f)) + .build(); + + update(queueAssertionHalfClusterResource, ResourceUtils.multiplyFloor(UPDATE_RES, 0.5f)); + } + +} \ No newline at end of file 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/conf/TestQueueCapacityConfigParser.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/conf/TestQueueCapacityConfigParser.java index 1aba816abd2..4e8f31e1a85 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/conf/TestQueueCapacityConfigParser.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/conf/TestQueueCapacityConfigParser.java @@ -23,7 +23,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.junit.Assert; import org.junit.Test; @@ -33,7 +33,6 @@ import java.util.List; import static org.apache.hadoop.yarn.api.records.ResourceInformation.GPU_URI; import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; import static org.apache.hadoop.yarn.api.records.ResourceInformation.VCORES_URI; -import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NO_LABEL; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestQueueMetricsForCustomResources.GB; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils.EPSILON; @@ -70,47 +69,43 @@ public class TestQueueCapacityConfigParser { @Test public void testPercentageCapacityConfig() { - CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); - conf.setCapacity(QUEUE, PERCENTAGE_VALUE); - - QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(conf, QUEUE, - NO_LABEL); + QueueCapacityVector percentageCapacityVector = + capacityConfigParser.parse(Float.toString(PERCENTAGE_VALUE), QUEUE); QueueCapacityVectorEntry memory = percentageCapacityVector.getResource(MEMORY_URI); QueueCapacityVectorEntry vcore = percentageCapacityVector.getResource(VCORES_URI); - Assert.assertEquals(QueueCapacityType.PERCENTAGE, memory.getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.PERCENTAGE, memory.getVectorResourceType()); Assert.assertEquals(PERCENTAGE_VALUE, memory.getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.PERCENTAGE, vcore.getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.PERCENTAGE, vcore.getVectorResourceType()); Assert.assertEquals(PERCENTAGE_VALUE, vcore.getResourceValue(), EPSILON); - QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(conf, - CapacitySchedulerConfiguration.ROOT, NO_LABEL); + QueueCapacityVector rootCapacityVector = + capacityConfigParser.parse(Float.toString(PERCENTAGE_VALUE), + CapacitySchedulerConfiguration.ROOT); QueueCapacityVectorEntry memoryRoot = rootCapacityVector.getResource(MEMORY_URI); QueueCapacityVectorEntry vcoreRoot = rootCapacityVector.getResource(VCORES_URI); - Assert.assertEquals(QueueCapacityType.PERCENTAGE, memoryRoot.getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.PERCENTAGE, memoryRoot.getVectorResourceType()); Assert.assertEquals(100f, memoryRoot.getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.PERCENTAGE, vcoreRoot.getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.PERCENTAGE, vcoreRoot.getVectorResourceType()); Assert.assertEquals(100f, vcoreRoot.getResourceValue(), EPSILON); } @Test public void testWeightCapacityConfig() { - CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); - conf.setNonLabeledQueueWeight(QUEUE, WEIGHT_VALUE); - - QueueCapacityVector weightCapacityVector = capacityConfigParser.parse(conf, QUEUE, NO_LABEL); + QueueCapacityVector weightCapacityVector = capacityConfigParser.parse(WEIGHT_VALUE + "w", + QUEUE); QueueCapacityVectorEntry memory = weightCapacityVector.getResource(MEMORY_URI); QueueCapacityVectorEntry vcore = weightCapacityVector.getResource(VCORES_URI); - Assert.assertEquals(QueueCapacityType.WEIGHT, memory.getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.WEIGHT, memory.getVectorResourceType()); Assert.assertEquals(WEIGHT_VALUE, memory.getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.WEIGHT, vcore.getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.WEIGHT, vcore.getVectorResourceType()); Assert.assertEquals(WEIGHT_VALUE, vcore.getResourceValue(), EPSILON); } @@ -122,26 +117,26 @@ public class TestQueueCapacityConfigParser { conf.set(YarnConfiguration.RESOURCE_TYPES, RESOURCE_TYPES); ResourceUtils.resetResourceTypes(conf); - QueueCapacityVector absoluteCapacityVector = capacityConfigParser.parse(conf, QUEUE, NO_LABEL); + QueueCapacityVector absoluteCapacityVector = capacityConfigParser.parse(ABSOLUTE_RESOURCE, + QUEUE); - Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(MEMORY_URI) - .getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, + absoluteCapacityVector.getResource(MEMORY_URI).getVectorResourceType()); Assert.assertEquals(12 * GB, absoluteCapacityVector.getResource(MEMORY_URI) .getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(VCORES_URI) - .getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, + absoluteCapacityVector.getResource(VCORES_URI).getVectorResourceType()); Assert.assertEquals(VCORE_ABSOLUTE, absoluteCapacityVector.getResource(VCORES_URI) .getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(GPU_URI) - .getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, + absoluteCapacityVector.getResource(GPU_URI).getVectorResourceType()); Assert.assertEquals(GPU_ABSOLUTE, absoluteCapacityVector.getResource(GPU_URI) .getResourceValue(), EPSILON); - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + - CapacitySchedulerConfiguration.CAPACITY, ABSOLUTE_RESOURCE_MEMORY_VCORE); - QueueCapacityVector withoutGpuVector = capacityConfigParser.parse(conf, QUEUE, NO_LABEL); + QueueCapacityVector withoutGpuVector = capacityConfigParser + .parse(ABSOLUTE_RESOURCE_MEMORY_VCORE, QUEUE); Assert.assertEquals(3, withoutGpuVector.getResourceCount()); Assert.assertEquals(0f, withoutGpuVector.getResource(GPU_URI).getResourceValue(), EPSILON); @@ -150,36 +145,31 @@ public class TestQueueCapacityConfigParser { @Test public void testMixedCapacityConfig() { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, MIXED_RESOURCE); conf.set(YarnConfiguration.RESOURCE_TYPES, RESOURCE_TYPES); ResourceUtils.resetResourceTypes(conf); QueueCapacityVector mixedCapacityVector = - capacityConfigParser.parse(conf, QUEUE, NO_LABEL); + capacityConfigParser.parse(MIXED_RESOURCE, QUEUE); - Assert.assertEquals(QueueCapacityType.ABSOLUTE, + Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, mixedCapacityVector.getResource(MEMORY_URI).getVectorResourceType()); Assert.assertEquals(MEMORY_MIXED, mixedCapacityVector.getResource(MEMORY_URI) .getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.PERCENTAGE, + Assert.assertEquals(ResourceUnitCapacityType.PERCENTAGE, mixedCapacityVector.getResource(VCORES_URI).getVectorResourceType()); Assert.assertEquals(PERCENTAGE_VALUE, mixedCapacityVector.getResource(VCORES_URI).getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.WEIGHT, + Assert.assertEquals(ResourceUnitCapacityType.WEIGHT, mixedCapacityVector.getResource(GPU_URI).getVectorResourceType()); Assert.assertEquals(WEIGHT_VALUE, mixedCapacityVector.getResource(GPU_URI).getResourceValue(), EPSILON); // Test undefined capacity type default value - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, ABSOLUTE_RESOURCE_MEMORY_VCORE); - QueueCapacityVector mixedCapacityVectorWithGpuUndefined = - capacityConfigParser.parse(conf, QUEUE, NO_LABEL); - Assert.assertEquals(QueueCapacityType.ABSOLUTE, + capacityConfigParser.parse(ABSOLUTE_RESOURCE_MEMORY_VCORE, QUEUE); + Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, mixedCapacityVectorWithGpuUndefined.getResource(MEMORY_URI).getVectorResourceType()); Assert.assertEquals(0, mixedCapacityVectorWithGpuUndefined.getResource(GPU_URI) .getResourceValue(), EPSILON); @@ -188,52 +178,38 @@ public class TestQueueCapacityConfigParser { @Test public void testInvalidCapacityConfigs() { - CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); - - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, NONEXISTINGSUFFIX); QueueCapacityVector capacityVectorWithInvalidSuffix = - capacityConfigParser.parse(conf, QUEUE, NO_LABEL); + capacityConfigParser.parse(NONEXISTINGSUFFIX, QUEUE); List entriesWithInvalidSuffix = Lists.newArrayList(capacityVectorWithInvalidSuffix.iterator()); Assert.assertEquals(0, entriesWithInvalidSuffix.size()); - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, INVALID_CAPACITY_FORMAT); QueueCapacityVector invalidDelimiterCapacityVector = - capacityConfigParser.parse(conf, QUEUE, NO_LABEL); + capacityConfigParser.parse(INVALID_CAPACITY_FORMAT, QUEUE); List invalidDelimiterEntries = Lists.newArrayList(invalidDelimiterCapacityVector.iterator()); Assert.assertEquals(0, invalidDelimiterEntries.size()); - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, INVALID_CAPACITY_BRACKET); QueueCapacityVector invalidCapacityVector = - capacityConfigParser.parse(conf, QUEUE, NO_LABEL); + capacityConfigParser.parse(INVALID_CAPACITY_BRACKET, QUEUE); List resources = Lists.newArrayList(invalidCapacityVector.iterator()); Assert.assertEquals(0, resources.size()); - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, EMPTY_BRACKET); QueueCapacityVector emptyBracketCapacityVector = - capacityConfigParser.parse(conf, QUEUE, NO_LABEL); + capacityConfigParser.parse(EMPTY_BRACKET, QUEUE); List emptyEntries = Lists.newArrayList(emptyBracketCapacityVector.iterator()); Assert.assertEquals(0, emptyEntries.size()); - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, ""); QueueCapacityVector emptyCapacity = - capacityConfigParser.parse(conf, QUEUE, NO_LABEL); + capacityConfigParser.parse("", QUEUE); List emptyResources = Lists.newArrayList(emptyCapacity.iterator()); Assert.assertEquals(emptyResources.size(), 0); - conf.unset(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY); QueueCapacityVector nonSetCapacity = - capacityConfigParser.parse(conf, QUEUE, NO_LABEL); + capacityConfigParser.parse(null, QUEUE); List nonSetResources = Lists.newArrayList(nonSetCapacity.iterator()); Assert.assertEquals(nonSetResources.size(), 0); From 468135a4d99ad334a545127bd144c46c23e0ac3b Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Fri, 27 Jan 2023 03:14:05 +0800 Subject: [PATCH 02/50] YARN-11218. [Federation] Add getActivities, getBulkActivities REST APIs for Router. (#5284) --- .../webapp/dao/BulkActivitiesInfo.java | 10 ++ .../yarn/server/router/RouterMetrics.java | 62 +++++++++++ .../webapp/FederationInterceptorREST.java | 100 +++++++++++++++++- .../dao/FederationBulkActivitiesInfo.java | 49 +++++++++ .../yarn/server/router/TestRouterMetrics.java | 66 ++++++++++++ .../MockDefaultRequestInterceptorREST.java | 63 +++++++++++ .../webapp/TestFederationInterceptorREST.java | 86 +++++++++++++++ 7 files changed, 434 insertions(+), 2 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/dao/FederationBulkActivitiesInfo.java 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/dao/BulkActivitiesInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/BulkActivitiesInfo.java index ad360cc6fc2..462003f9464 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/BulkActivitiesInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/BulkActivitiesInfo.java @@ -34,6 +34,8 @@ public class BulkActivitiesInfo { private ArrayList activities = new ArrayList<>(); + private String subClusterId; + public BulkActivitiesInfo() { // JAXB needs this } @@ -49,4 +51,12 @@ public class BulkActivitiesInfo { public void addAll(List activitiesInfoList) { activities.addAll(activitiesInfoList); } + + public String getSubClusterId() { + return subClusterId; + } + + public void setSubClusterId(String subClusterId) { + this.subClusterId = subClusterId; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java index 97d4c424972..033aa076658 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java @@ -135,6 +135,10 @@ public final class RouterMetrics { private MutableGaugeInt numRenewDelegationTokenFailedRetrieved; @Metric("# of renewDelegationToken failed to be retrieved") private MutableGaugeInt numCancelDelegationTokenFailedRetrieved; + @Metric("# of getActivities failed to be retrieved") + private MutableGaugeInt numGetActivitiesFailedRetrieved; + @Metric("# of getBulkActivities failed to be retrieved") + private MutableGaugeInt numGetBulkActivitiesFailedRetrieved; @Metric("# of getSchedulerInfo failed to be retrieved") private MutableGaugeInt numGetSchedulerInfoFailedRetrieved; @Metric("# of refreshSuperUserGroupsConfiguration failed to be retrieved") @@ -237,6 +241,10 @@ public final class RouterMetrics { private MutableRate totalSucceededRenewDelegationTokenRetrieved; @Metric("Total number of successful Retrieved CancelDelegationToken and latency(ms)") private MutableRate totalSucceededCancelDelegationTokenRetrieved; + @Metric("Total number of successful Retrieved GetActivities and latency(ms)") + private MutableRate totalSucceededGetActivitiesRetrieved; + @Metric("Total number of successful Retrieved GetBulkActivities and latency(ms)") + private MutableRate totalSucceededGetBulkActivitiesRetrieved; @Metric("Total number of successful Retrieved RefreshSuperUserGroupsConfig and latency(ms)") private MutableRate totalSucceededRefreshSuperUserGroupsConfigurationRetrieved; @Metric("Total number of successful Retrieved RefreshUserToGroupsMappings and latency(ms)") @@ -295,6 +303,8 @@ public final class RouterMetrics { private MutableQuantiles getDelegationTokenLatency; private MutableQuantiles renewDelegationTokenLatency; private MutableQuantiles cancelDelegationTokenLatency; + private MutableQuantiles getActivitiesLatency; + private MutableQuantiles getBulkActivitiesLatency; private MutableQuantiles getSchedulerInfoRetrievedLatency; private MutableQuantiles refreshSuperUserGroupsConfLatency; private MutableQuantiles refreshUserToGroupsMappingsLatency; @@ -472,6 +482,12 @@ public final class RouterMetrics { cancelDelegationTokenLatency = registry.newQuantiles("cancelDelegationTokenLatency", "latency of cancel delegation token timeouts", "ops", "latency", 10); + getActivitiesLatency = registry.newQuantiles("getActivitiesLatency", + "latency of get activities timeouts", "ops", "latency", 10); + + getBulkActivitiesLatency = registry.newQuantiles("getBulkActivitiesLatency", + "latency of get bulk activities timeouts", "ops", "latency", 10); + getSchedulerInfoRetrievedLatency = registry.newQuantiles("getSchedulerInfoRetrievedLatency", "latency of get scheduler info timeouts", "ops", "latency", 10); @@ -736,6 +752,16 @@ public final class RouterMetrics { return totalSucceededCancelDelegationTokenRetrieved.lastStat().numSamples(); } + @VisibleForTesting + public long getNumSucceededGetActivitiesRetrieved() { + return totalSucceededGetActivitiesRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededGetBulkActivitiesRetrieved() { + return totalSucceededGetBulkActivitiesRetrieved.lastStat().numSamples(); + } + @VisibleForTesting public long getNumSucceededGetSchedulerInfoRetrieved() { return totalSucceededGetSchedulerInfoRetrieved.lastStat().numSamples(); @@ -981,6 +1007,16 @@ public final class RouterMetrics { return totalSucceededCancelDelegationTokenRetrieved.lastStat().mean(); } + @VisibleForTesting + public double getLatencySucceededGetActivitiesRetrieved() { + return totalSucceededGetActivitiesRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededGetBulkActivitiesRetrieved() { + return totalSucceededGetBulkActivitiesRetrieved.lastStat().mean(); + } + @VisibleForTesting public double getLatencySucceededGetSchedulerInfoRetrieved() { return totalSucceededGetSchedulerInfoRetrieved.lastStat().mean(); @@ -1209,6 +1245,14 @@ public final class RouterMetrics { return numCancelDelegationTokenFailedRetrieved.value(); } + public int getActivitiesFailedRetrieved() { + return numGetActivitiesFailedRetrieved.value(); + } + + public int getBulkActivitiesFailedRetrieved(){ + return numGetBulkActivitiesFailedRetrieved.value(); + } + public int getSchedulerInfoFailedRetrieved() { return numGetSchedulerInfoFailedRetrieved.value(); } @@ -1448,6 +1492,16 @@ public final class RouterMetrics { cancelDelegationTokenLatency.add(duration); } + public void succeededGetActivitiesLatencyRetrieved(long duration) { + totalSucceededGetActivitiesRetrieved.add(duration); + getActivitiesLatency.add(duration); + } + + public void succeededGetBulkActivitiesRetrieved(long duration) { + totalSucceededGetBulkActivitiesRetrieved.add(duration); + getBulkActivitiesLatency.add(duration); + } + public void succeededGetSchedulerInfoRetrieved(long duration) { totalSucceededGetSchedulerInfoRetrieved.add(duration); getSchedulerInfoRetrievedLatency.add(duration); @@ -1659,6 +1713,14 @@ public final class RouterMetrics { numCancelDelegationTokenFailedRetrieved.incr(); } + public void incrGetActivitiesFailedRetrieved() { + numGetActivitiesFailedRetrieved.incr(); + } + + public void incrGetBulkActivitiesFailedRetrieved() { + numGetBulkActivitiesFailedRetrieved.incr(); + } + public void incrGetSchedulerInfoFailedRetrieved() { numGetSchedulerInfoFailedRetrieved.incr(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java index 73b0c5f2af8..69dba5b07e6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java @@ -46,6 +46,7 @@ import javax.ws.rs.core.Response.Status; import org.apache.commons.lang3.NotImplementedException; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.impl.prefetch.Validate; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.AuthorizationException; @@ -121,6 +122,7 @@ import org.apache.hadoop.yarn.server.router.clientrm.ClientMethod; import org.apache.hadoop.yarn.server.router.clientrm.RouterClientRMService; import org.apache.hadoop.yarn.server.router.security.RouterDelegationTokenSecretManager; import org.apache.hadoop.yarn.server.router.webapp.cache.RouterAppInfoCacheKey; +import org.apache.hadoop.yarn.server.router.webapp.dao.FederationBulkActivitiesInfo; import org.apache.hadoop.yarn.server.router.webapp.dao.FederationRMQueueAclInfo; import org.apache.hadoop.yarn.server.router.webapp.dao.SubClusterResult; import org.apache.hadoop.yarn.server.router.webapp.dao.FederationSchedulerTypeInfo; @@ -1187,16 +1189,110 @@ public class FederationInterceptorREST extends AbstractRESTRequestInterceptor { throw new NotImplementedException("Code is not implemented"); } + /** + * This method retrieve all the activities in a specific node, and it is + * reachable by using {@link RMWSConsts#SCHEDULER_ACTIVITIES}. + * + * @param hsr the servlet request + * @param nodeId the node we want to retrieve the activities. It is a + * QueryParam. + * @param groupBy the groupBy type by which the activities should be + * aggregated. It is a QueryParam. + * @return all the activities in the specific node + */ @Override public ActivitiesInfo getActivities(HttpServletRequest hsr, String nodeId, String groupBy) { - throw new NotImplementedException("Code is not implemented"); + try { + // Check the parameters to ensure that the parameters are not empty + Validate.checkNotNullAndNotEmpty(nodeId, "nodeId"); + Validate.checkNotNullAndNotEmpty(groupBy, "groupBy"); + + // Query SubClusterInfo according to id, + // if the nodeId cannot get SubClusterInfo, an exception will be thrown directly. + SubClusterInfo subClusterInfo = getNodeSubcluster(nodeId); + + // Call the corresponding subCluster to get ActivitiesInfo. + long startTime = clock.getTime(); + DefaultRequestInterceptorREST interceptor = getOrCreateInterceptorForSubCluster( + subClusterInfo.getSubClusterId(), subClusterInfo.getRMWebServiceAddress()); + final HttpServletRequest hsrCopy = clone(hsr); + ActivitiesInfo activitiesInfo = interceptor.getActivities(hsrCopy, nodeId, groupBy); + if (activitiesInfo != null) { + long stopTime = clock.getTime(); + routerMetrics.succeededGetActivitiesLatencyRetrieved(stopTime - startTime); + return activitiesInfo; + } + } catch (IllegalArgumentException e) { + routerMetrics.incrGetActivitiesFailedRetrieved(); + throw e; + } catch (NotFoundException e) { + routerMetrics.incrGetActivitiesFailedRetrieved(); + throw e; + } + + routerMetrics.incrGetActivitiesFailedRetrieved(); + throw new RuntimeException("getActivities Failed."); } + /** + * This method retrieve the last n activities inside scheduler, and it is + * reachable by using {@link RMWSConsts#SCHEDULER_BULK_ACTIVITIES}. + * + * @param hsr the servlet request + * @param groupBy the groupBy type by which the activities should be + * aggregated. It is a QueryParam. + * @param activitiesCount number of activities + * @return last n activities + */ @Override public BulkActivitiesInfo getBulkActivities(HttpServletRequest hsr, String groupBy, int activitiesCount) throws InterruptedException { - throw new NotImplementedException("Code is not implemented"); + try { + // Step1. Check the parameters to ensure that the parameters are not empty + Validate.checkNotNullAndNotEmpty(groupBy, "groupBy"); + Validate.checkNotNegative(activitiesCount, "activitiesCount"); + + // Step2. Call the interface of subCluster concurrently and get the returned result. + Map subClustersActive = getActiveSubclusters(); + final HttpServletRequest hsrCopy = clone(hsr); + Class[] argsClasses = new Class[]{HttpServletRequest.class, String.class, int.class}; + Object[] args = new Object[]{hsrCopy, groupBy, activitiesCount}; + ClientMethod remoteMethod = new ClientMethod("getBulkActivities", argsClasses, args); + Map appStatisticsMap = invokeConcurrent( + subClustersActive.values(), remoteMethod, BulkActivitiesInfo.class); + + // Step3. Generate Federation objects and set subCluster information. + long startTime = clock.getTime(); + FederationBulkActivitiesInfo fedBulkActivitiesInfo = new FederationBulkActivitiesInfo(); + appStatisticsMap.forEach((subClusterInfo, bulkActivitiesInfo) -> { + SubClusterId subClusterId = subClusterInfo.getSubClusterId(); + bulkActivitiesInfo.setSubClusterId(subClusterId.getId()); + fedBulkActivitiesInfo.getList().add(bulkActivitiesInfo); + }); + long stopTime = clock.getTime(); + routerMetrics.succeededGetBulkActivitiesRetrieved(stopTime - startTime); + return fedBulkActivitiesInfo; + } catch (IllegalArgumentException e) { + routerMetrics.incrGetBulkActivitiesFailedRetrieved(); + throw e; + } catch (NotFoundException e) { + routerMetrics.incrGetBulkActivitiesFailedRetrieved(); + RouterServerUtil.logAndThrowRunTimeException("get all active sub cluster(s) error.", e); + } catch (IOException e) { + routerMetrics.incrGetBulkActivitiesFailedRetrieved(); + RouterServerUtil.logAndThrowRunTimeException(e, + "getBulkActivities by groupBy = %s, activitiesCount = %s with io error.", + groupBy, String.valueOf(activitiesCount)); + } catch (YarnException e) { + routerMetrics.incrGetBulkActivitiesFailedRetrieved(); + RouterServerUtil.logAndThrowRunTimeException(e, + "getBulkActivities by groupBy = %s, activitiesCount = %s with yarn error.", + groupBy, String.valueOf(activitiesCount)); + } + + routerMetrics.incrGetBulkActivitiesFailedRetrieved(); + throw new RuntimeException("getBulkActivities Failed."); } @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/dao/FederationBulkActivitiesInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/dao/FederationBulkActivitiesInfo.java new file mode 100644 index 00000000000..87d11ad0feb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/dao/FederationBulkActivitiesInfo.java @@ -0,0 +1,49 @@ +/** + * 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.yarn.server.router.webapp.dao; + +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.BulkActivitiesInfo; + +import javax.xml.bind.annotation.XmlAccessType; +import javax.xml.bind.annotation.XmlAccessorType; +import javax.xml.bind.annotation.XmlElement; +import javax.xml.bind.annotation.XmlRootElement; +import java.util.ArrayList; + +@XmlRootElement +@XmlAccessorType(XmlAccessType.FIELD) +public class FederationBulkActivitiesInfo extends BulkActivitiesInfo { + + @XmlElement(name = "subCluster") + private ArrayList list = new ArrayList<>(); + + public FederationBulkActivitiesInfo() { + } // JAXB needs this + + public FederationBulkActivitiesInfo(ArrayList list) { + this.list = list; + } + + public ArrayList getList() { + return list; + } + + public void setList(ArrayList list) { + this.list = list; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java index 9d5aeab5c6d..c26df63c954 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java @@ -533,6 +533,16 @@ public class TestRouterMetrics { LOG.info("Mocked: failed renewDelegationToken call"); metrics.incrRenewDelegationTokenFailedRetrieved(); } + + public void getActivitiesFailed() { + LOG.info("Mocked: failed getBulkActivitie call"); + metrics.incrGetActivitiesFailedRetrieved(); + } + + public void getBulkActivitiesFailed() { + LOG.info("Mocked: failed getBulkActivitie call"); + metrics.incrGetBulkActivitiesFailedRetrieved(); + } } // Records successes for all calls @@ -763,6 +773,16 @@ public class TestRouterMetrics { LOG.info("Mocked: successful RenewDelegationToken call with duration {}", duration); metrics.succeededRenewDelegationTokenRetrieved(duration); } + + public void getActivitiesRetrieved(long duration) { + LOG.info("Mocked: successful GetActivities call with duration {}", duration); + metrics.succeededGetActivitiesLatencyRetrieved(duration); + } + + public void getBulkActivitiesRetrieved(long duration) { + LOG.info("Mocked: successful GetBulkActivities call with duration {}", duration); + metrics.succeededGetBulkActivitiesRetrieved(duration); + } } @Test @@ -1597,4 +1617,50 @@ public class TestRouterMetrics { Assert.assertEquals(totalBadBefore + 1, metrics.getRenewDelegationTokenFailedRetrieved()); } + + @Test + public void testGetActivitiesRetrieved() { + long totalGoodBefore = metrics.getNumSucceededGetActivitiesRetrieved(); + goodSubCluster.getActivitiesRetrieved(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededGetActivitiesRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededGetActivitiesRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getActivitiesRetrieved(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededGetActivitiesRetrieved()); + Assert.assertEquals(225, + metrics.getLatencySucceededGetActivitiesRetrieved(), ASSERT_DOUBLE_DELTA); + } + + @Test + public void testGetActivitiesRetrievedFailed() { + long totalBadBefore = metrics.getActivitiesFailedRetrieved(); + badSubCluster.getActivitiesFailed(); + Assert.assertEquals(totalBadBefore + 1, + metrics.getActivitiesFailedRetrieved()); + } + + @Test + public void testGetBulkActivitiesRetrieved() { + long totalGoodBefore = metrics.getNumSucceededGetBulkActivitiesRetrieved(); + goodSubCluster.getBulkActivitiesRetrieved(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededGetBulkActivitiesRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededGetBulkActivitiesRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getBulkActivitiesRetrieved(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededGetBulkActivitiesRetrieved()); + Assert.assertEquals(225, + metrics.getLatencySucceededGetBulkActivitiesRetrieved(), ASSERT_DOUBLE_DELTA); + } + + @Test + public void testGetBulkActivitiesRetrievedFailed() { + long totalBadBefore = metrics.getBulkActivitiesFailedRetrieved(); + badSubCluster.getBulkActivitiesFailed(); + Assert.assertEquals(totalBadBefore + 1, + metrics.getBulkActivitiesFailedRetrieved()); + } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java index ddd15138fdc..2e118d172c1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java @@ -137,6 +137,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationReque import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationRequestsInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateResponseInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteResponseInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.BulkActivitiesInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeAllocationInfo; import org.apache.hadoop.yarn.server.router.RouterServerUtil; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.PartitionInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.RMQueueAclInfo; @@ -1213,6 +1217,65 @@ public class MockDefaultRequestInterceptorREST } @Override + public ActivitiesInfo getActivities(HttpServletRequest hsr, String nodeId, String groupBy) { + if (!EnumUtils.isValidEnum(RMWSConsts.ActivitiesGroupBy.class, groupBy.toUpperCase())) { + String errMessage = "Got invalid groupBy: " + groupBy + ", valid groupBy types: " + + Arrays.asList(RMWSConsts.ActivitiesGroupBy.values()); + throw new IllegalArgumentException(errMessage); + } + + SubClusterId subClusterId = getSubClusterId(); + ActivitiesInfo activitiesInfo = mock(ActivitiesInfo.class); + Mockito.when(activitiesInfo.getNodeId()).thenReturn(nodeId); + Mockito.when(activitiesInfo.getTimestamp()).thenReturn(1673081972L); + Mockito.when(activitiesInfo.getDiagnostic()).thenReturn("Diagnostic:" + subClusterId.getId()); + + List allocationInfos = new ArrayList<>(); + NodeAllocationInfo nodeAllocationInfo = mock(NodeAllocationInfo.class); + Mockito.when(nodeAllocationInfo.getPartition()).thenReturn("p" + subClusterId.getId()); + Mockito.when(nodeAllocationInfo.getFinalAllocationState()).thenReturn("ALLOCATED"); + + allocationInfos.add(nodeAllocationInfo); + Mockito.when(activitiesInfo.getAllocations()).thenReturn(allocationInfos); + return activitiesInfo; + } + + @Override + public BulkActivitiesInfo getBulkActivities(HttpServletRequest hsr, + String groupBy, int activitiesCount) { + + if (activitiesCount <= 0) { + throw new IllegalArgumentException("activitiesCount needs to be greater than 0."); + } + + if (!EnumUtils.isValidEnum(RMWSConsts.ActivitiesGroupBy.class, groupBy.toUpperCase())) { + String errMessage = "Got invalid groupBy: " + groupBy + ", valid groupBy types: " + + Arrays.asList(RMWSConsts.ActivitiesGroupBy.values()); + throw new IllegalArgumentException(errMessage); + } + + BulkActivitiesInfo bulkActivitiesInfo = new BulkActivitiesInfo(); + + for (int i = 0; i < activitiesCount; i++) { + SubClusterId subClusterId = getSubClusterId(); + ActivitiesInfo activitiesInfo = mock(ActivitiesInfo.class); + Mockito.when(activitiesInfo.getNodeId()).thenReturn(subClusterId + "-nodeId-" + i); + Mockito.when(activitiesInfo.getTimestamp()).thenReturn(1673081972L); + Mockito.when(activitiesInfo.getDiagnostic()).thenReturn("Diagnostic:" + subClusterId.getId()); + + List allocationInfos = new ArrayList<>(); + NodeAllocationInfo nodeAllocationInfo = mock(NodeAllocationInfo.class); + Mockito.when(nodeAllocationInfo.getPartition()).thenReturn("p" + subClusterId.getId()); + Mockito.when(nodeAllocationInfo.getFinalAllocationState()).thenReturn("ALLOCATED"); + + allocationInfos.add(nodeAllocationInfo); + Mockito.when(activitiesInfo.getAllocations()).thenReturn(allocationInfos); + bulkActivitiesInfo.getActivities().add(activitiesInfo); + } + + return bulkActivitiesInfo; + } + public SchedulerTypeInfo getSchedulerInfo() { try { ResourceManager resourceManager = CapacitySchedulerTestUtilities.createResourceManager(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java index 070c883615a..edaa1e26e93 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java @@ -34,6 +34,7 @@ import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.security.authorize.AuthorizationException; import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.conf.Configuration; @@ -115,9 +116,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedule import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewReservation; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeAllocationInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.BulkActivitiesInfo; import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo; import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo; import org.apache.hadoop.yarn.server.router.webapp.dao.FederationRMQueueAclInfo; +import org.apache.hadoop.yarn.server.router.webapp.dao.FederationBulkActivitiesInfo; import org.apache.hadoop.yarn.server.router.webapp.dao.FederationSchedulerTypeInfo; import org.apache.hadoop.yarn.util.LRUCacheHashMap; import org.apache.hadoop.yarn.util.MonotonicClock; @@ -1779,4 +1784,85 @@ public class TestFederationInterceptorREST extends BaseRouterWebServicesTest { Assert.assertNotNull(cancelResponse); Assert.assertEquals(response.getStatus(), Status.OK.getStatusCode()); } + + @Test + public void testGetActivitiesNormal() { + ActivitiesInfo activitiesInfo = interceptor.getActivities(null, "1", "DIAGNOSTIC"); + Assert.assertNotNull(activitiesInfo); + + String nodeId = activitiesInfo.getNodeId(); + Assert.assertNotNull(nodeId); + Assert.assertEquals("1", nodeId); + + String diagnostic = activitiesInfo.getDiagnostic(); + Assert.assertNotNull(diagnostic); + Assert.assertTrue(StringUtils.contains(diagnostic, "Diagnostic")); + + long timestamp = activitiesInfo.getTimestamp(); + Assert.assertEquals(1673081972L, timestamp); + + List allocationInfos = activitiesInfo.getAllocations(); + Assert.assertNotNull(allocationInfos); + Assert.assertEquals(1, allocationInfos.size()); + } + + @Test + public void testGetActivitiesError() throws Exception { + // nodeId is empty + LambdaTestUtils.intercept(IllegalArgumentException.class, + "'nodeId' must not be empty.", + () -> interceptor.getActivities(null, "", "DIAGNOSTIC")); + + // groupBy is empty + LambdaTestUtils.intercept(IllegalArgumentException.class, + "'groupBy' must not be empty.", + () -> interceptor.getActivities(null, "1", "")); + + // groupBy value is wrong + LambdaTestUtils.intercept(IllegalArgumentException.class, + "Got invalid groupBy: TEST1, valid groupBy types: [DIAGNOSTIC]", + () -> interceptor.getActivities(null, "1", "TEST1")); + } + + @Test + public void testGetBulkActivitiesNormal() throws InterruptedException { + BulkActivitiesInfo bulkActivitiesInfo = + interceptor.getBulkActivities(null, "DIAGNOSTIC", 5); + Assert.assertNotNull(bulkActivitiesInfo); + + Assert.assertTrue(bulkActivitiesInfo instanceof FederationBulkActivitiesInfo); + + FederationBulkActivitiesInfo federationBulkActivitiesInfo = + FederationBulkActivitiesInfo.class.cast(bulkActivitiesInfo); + Assert.assertNotNull(federationBulkActivitiesInfo); + + List activitiesInfos = federationBulkActivitiesInfo.getList(); + Assert.assertNotNull(activitiesInfos); + Assert.assertEquals(4, activitiesInfos.size()); + + for (BulkActivitiesInfo activitiesInfo : activitiesInfos) { + Assert.assertNotNull(activitiesInfo); + List activitiesInfoList = activitiesInfo.getActivities(); + Assert.assertNotNull(activitiesInfoList); + Assert.assertEquals(5, activitiesInfoList.size()); + } + } + + @Test + public void testGetBulkActivitiesError() throws Exception { + // activitiesCount < 0 + LambdaTestUtils.intercept(IllegalArgumentException.class, + "'activitiesCount' must not be negative.", + () -> interceptor.getBulkActivities(null, "DIAGNOSTIC", -1)); + + // groupBy value is wrong + LambdaTestUtils.intercept(YarnRuntimeException.class, + "Got invalid groupBy: TEST1, valid groupBy types: [DIAGNOSTIC]", + () -> interceptor.getBulkActivities(null, "TEST1", 1)); + + // groupBy is empty + LambdaTestUtils.intercept(IllegalArgumentException.class, + "'groupBy' must not be empty.", + () -> interceptor.getBulkActivities(null, "", 1)); + } } \ No newline at end of file From 970ebaeded13a4978058a7eac52c7097eca8f4db Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 27 Jan 2023 11:50:17 +0000 Subject: [PATCH 03/50] HADOOP-17717. Update wildfly openssl to 1.1.3.Final. (#5310) Contributed by Wei-Chiu Chuang --- LICENSE-binary | 2 +- hadoop-project/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index 876b88f82a7..95ceb0d97c1 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -363,7 +363,7 @@ org.lz4:lz4-java:1.7.1 org.objenesis:objenesis:2.6 org.xerial.snappy:snappy-java:1.0.5 org.yaml:snakeyaml:1.33 -org.wildfly.openssl:wildfly-openssl:1.0.7.Final +org.wildfly.openssl:wildfly-openssl:1.1.3.Final -------------------------------------------------------------------------------- diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index c0801330bc7..33350c929e8 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -208,7 +208,7 @@ 3.9.0 1.5.6 8.8.2 - 1.0.7.Final + 1.1.3.Final 1.0.2 5.4.0 2.4.7 From b677d40ab5ee899a0264b1763d632221cefe44b4 Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Fri, 27 Jan 2023 16:32:50 +0100 Subject: [PATCH 04/50] HADOOP-18602. Remove netty3 dependency --- LICENSE-binary | 1 - hadoop-hdfs-project/hadoop-hdfs/pom.xml | 5 ----- hadoop-project/pom.xml | 8 -------- 3 files changed, 14 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index 95ceb0d97c1..432dc5d28f7 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -260,7 +260,6 @@ io.grpc:grpc-netty:1.26.0 io.grpc:grpc-protobuf:1.26.0 io.grpc:grpc-protobuf-lite:1.26.0 io.grpc:grpc-stub:1.26.0 -io.netty:netty:3.10.6.Final io.netty:netty-all:4.1.77.Final io.netty:netty-buffer:4.1.77.Final io.netty:netty-codec:4.1.77.Final diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml index b51c7154f7b..4e7efab06e9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml @@ -175,11 +175,6 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> slf4j-log4j12 provided - - io.netty - netty - compile - io.netty netty-all diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 33350c929e8..aada4e9b6de 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -140,7 +140,6 @@ 2.2.21 2.9.0 3.2.4 - 3.10.6.Final 4.1.77.Final 1.1.8.2 1.7.1 @@ -1000,13 +999,6 @@ - - - io.netty - netty - ${netty3.version} - - io.netty netty-all From 952d707240cb7dd088820d8b39e705b77fa3b6c4 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Sat, 28 Jan 2023 14:19:19 +0530 Subject: [PATCH 05/50] HADOOP-18604. Add compile platform in the hadoop version output. (#5327). Contributed by Ayush Saxena. Signed-off-by: Chris Nauroth --- .../java/org/apache/hadoop/util/VersionInfo.java | 13 +++++++++++++ .../main/resources/common-version-info.properties | 1 + 2 files changed, 14 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java index ea835023e86..31fe3c6377b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java @@ -93,6 +93,10 @@ public class VersionInfo { return info.getProperty("protocVersion", "Unknown"); } + protected String _getCompilePlatform() { + return info.getProperty("compilePlatform", "Unknown"); + } + private static VersionInfo COMMON_VERSION_INFO = new VersionInfo("common"); /** * Get the Hadoop version. @@ -167,12 +171,21 @@ public class VersionInfo { return COMMON_VERSION_INFO._getProtocVersion(); } + /** + * Returns the OS platform used for the build. + * @return the OS platform + */ + public static String getCompilePlatform() { + return COMMON_VERSION_INFO._getCompilePlatform(); + } + public static void main(String[] args) { LOG.debug("version: "+ getVersion()); System.out.println("Hadoop " + getVersion()); System.out.println("Source code repository " + getUrl() + " -r " + getRevision()); System.out.println("Compiled by " + getUser() + " on " + getDate()); + System.out.println("Compiled on platform " + getCompilePlatform()); System.out.println("Compiled with protoc " + getProtocVersion()); System.out.println("From source with checksum " + getSrcChecksum()); System.out.println("This command was run using " + diff --git a/hadoop-common-project/hadoop-common/src/main/resources/common-version-info.properties b/hadoop-common-project/hadoop-common/src/main/resources/common-version-info.properties index 6f8558b8d4f..0f075c8139a 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/common-version-info.properties +++ b/hadoop-common-project/hadoop-common/src/main/resources/common-version-info.properties @@ -24,3 +24,4 @@ date=${version-info.build.time} url=${version-info.scm.uri} srcChecksum=${version-info.source.md5} protocVersion=${hadoop.protobuf.version} +compilePlatform=${os.detected.classifier} From 9d47108b50fb0cd79ca48e82077e57572d8873e6 Mon Sep 17 00:00:00 2001 From: Wei-Chiu Chuang Date: Mon, 30 Jan 2023 09:17:04 -0800 Subject: [PATCH 06/50] HADOOP-18584. [NFS GW] Fix regression after netty4 migration. (#5252) Reviewed-by: Tsz-Wo Nicholas Sze --- .../org/apache/hadoop/oncrpc/RpcUtil.java | 19 ++++++++++++------- .../org/apache/hadoop/portmap/Portmap.java | 6 ++---- .../apache/hadoop/portmap/TestPortmap.java | 17 +++++++++++++++++ 3 files changed, 31 insertions(+), 11 deletions(-) diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java index edf42444b8b..d814052e43d 100644 --- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java +++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.oncrpc; +import java.net.InetSocketAddress; import java.net.SocketAddress; import java.nio.ByteBuffer; import java.util.List; @@ -26,6 +27,7 @@ import io.netty.buffer.Unpooled; import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.SimpleChannelInboundHandler; import io.netty.channel.socket.DatagramPacket; import io.netty.handler.codec.ByteToMessageDecoder; import org.apache.hadoop.classification.VisibleForTesting; @@ -172,15 +174,18 @@ public final class RpcUtil { */ @ChannelHandler.Sharable private static final class RpcUdpResponseStage extends - ChannelInboundHandlerAdapter { + SimpleChannelInboundHandler { + public RpcUdpResponseStage() { + // do not auto release the RpcResponse message. + super(false); + } @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) - throws Exception { - RpcResponse r = (RpcResponse) msg; - // TODO: check out https://github.com/netty/netty/issues/1282 for - // correct usage - ctx.channel().writeAndFlush(r.data()); + protected void channelRead0(ChannelHandlerContext ctx, + RpcResponse response) throws Exception { + ByteBuf buf = Unpooled.wrappedBuffer(response.data()); + ctx.writeAndFlush(new DatagramPacket( + buf, (InetSocketAddress) response.recipient())); } } } diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/Portmap.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/Portmap.java index 7d1130b40ff..953d74648db 100644 --- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/Portmap.java +++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/Portmap.java @@ -117,15 +117,13 @@ final class Portmap { .childOption(ChannelOption.SO_REUSEADDR, true) .channel(NioServerSocketChannel.class) .childHandler(new ChannelInitializer() { - private final IdleStateHandler idleStateHandler = new IdleStateHandler( - 0, 0, idleTimeMilliSeconds, TimeUnit.MILLISECONDS); - @Override protected void initChannel(SocketChannel ch) throws Exception { ChannelPipeline p = ch.pipeline(); p.addLast(RpcUtil.constructRpcFrameDecoder(), - RpcUtil.STAGE_RPC_MESSAGE_PARSER, idleStateHandler, handler, + RpcUtil.STAGE_RPC_MESSAGE_PARSER, new IdleStateHandler(0, 0, + idleTimeMilliSeconds, TimeUnit.MILLISECONDS), handler, RpcUtil.STAGE_RPC_TCP_RESPONSE); }}); diff --git a/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/portmap/TestPortmap.java b/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/portmap/TestPortmap.java index 84fa71a269d..35ab5cdc3da 100644 --- a/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/portmap/TestPortmap.java +++ b/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/portmap/TestPortmap.java @@ -23,8 +23,10 @@ import java.net.DatagramPacket; import java.net.DatagramSocket; import java.net.InetSocketAddress; import java.net.Socket; +import java.util.Arrays; import java.util.Map; +import org.apache.hadoop.oncrpc.RpcReply; import org.junit.Assert; import org.apache.hadoop.oncrpc.RpcCall; @@ -35,6 +37,8 @@ import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; +import static org.junit.Assert.assertEquals; + public class TestPortmap { private static Portmap pm = new Portmap(); private static final int SHORT_TIMEOUT_MILLISECONDS = 10; @@ -92,6 +96,19 @@ public class TestPortmap { pm.getUdpServerLoAddress()); try { s.send(p); + + // verify that portmap server responds a UDF packet back to the client + byte[] receiveData = new byte[65535]; + DatagramPacket receivePacket = new DatagramPacket(receiveData, + receiveData.length); + s.setSoTimeout(2000); + s.receive(receivePacket); + + // verify that the registration is accepted. + XDR xdr = new XDR(Arrays.copyOfRange(receiveData, 0, + receivePacket.getLength())); + RpcReply reply = RpcReply.read(xdr); + assertEquals(reply.getState(), RpcReply.ReplyState.MSG_ACCEPTED); } finally { s.close(); } From 88c8ac750dc630040a37014dd13391922e625292 Mon Sep 17 00:00:00 2001 From: huhaiyang Date: Tue, 31 Jan 2023 16:46:38 +0800 Subject: [PATCH 07/50] HDFS-16888. BlockManager#maxReplicationStreams, replicationStreamsHardLimit, blocksReplWorkMultiplier and PendingReconstructionBlocks#timeout should be volatile (#5296) Reviewed-by: Tao Li Signed-off-by: Takanobu Asanuma --- .../server/blockmanagement/BlockManager.java | 19 +++++++++++++------ .../PendingReconstructionBlocks.java | 2 +- .../blockmanagement/BlockManagerTestUtil.java | 2 +- .../blockmanagement/TestBlockManager.java | 18 +++++++++--------- .../TestUnderReplicatedBlocks.java | 4 ++-- 5 files changed, 26 insertions(+), 19 deletions(-) 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 3b0bbe847bf..4e5e1234716 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 @@ -395,12 +395,12 @@ public class BlockManager implements BlockStatsMXBean { * The maximum number of outgoing replication streams a given node should have * at one time considering all but the highest priority replications needed. */ - int maxReplicationStreams; + private volatile int maxReplicationStreams; /** * The maximum number of outgoing replication streams a given node should have * at one time. */ - int replicationStreamsHardLimit; + private volatile int replicationStreamsHardLimit; /** Minimum copies needed or else write is disallowed */ public final short minReplication; /** Default number of replicas */ @@ -409,7 +409,7 @@ public class BlockManager implements BlockStatsMXBean { final int maxCorruptFilesReturned; final float blocksInvalidateWorkPct; - private int blocksReplWorkMultiplier; + private volatile int blocksReplWorkMultiplier; // whether or not to issue block encryption keys. final boolean encryptDataTransfer; @@ -1017,12 +1017,19 @@ public class BlockManager implements BlockStatsMXBean { * * @param newVal - Must be a positive non-zero integer. */ - public void setMaxReplicationStreams(int newVal) { - ensurePositiveInt(newVal, - DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY); + @VisibleForTesting + public void setMaxReplicationStreams(int newVal, boolean ensurePositiveInt) { + if (ensurePositiveInt) { + ensurePositiveInt(newVal, + DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY); + } maxReplicationStreams = newVal; } + public void setMaxReplicationStreams(int newVal) { + setMaxReplicationStreams(newVal, true); + } + /** Returns the current setting for maxReplicationStreamsHardLimit, set by * {@code DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY}. * diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReconstructionBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReconstructionBlocks.java index 6c3b4c97bed..553b8218421 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReconstructionBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReconstructionBlocks.java @@ -59,7 +59,7 @@ class PendingReconstructionBlocks { // It might take anywhere between 5 to 10 minutes before // a request is timed out. // - private long timeout = + private volatile long timeout = DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_DEFAULT * 1000; private final static long DEFAULT_RECHECK_INTERVAL = 5 * 60 * 1000; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java index 4fa320ac29e..c25cc88059d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java @@ -44,7 +44,7 @@ public class BlockManagerTestUtil { public static void setNodeReplicationLimit(final BlockManager blockManager, final int limit) { - blockManager.maxReplicationStreams = limit; + blockManager.setMaxReplicationStreams(limit, false); } /** @return the datanode descriptor for the given the given storageID. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java index 04f474f67ef..c8a94e5ad20 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java @@ -677,8 +677,8 @@ public class TestBlockManager { */ @Test public void testHighestPriReplSrcChosenDespiteMaxReplLimit() throws Exception { - bm.maxReplicationStreams = 0; - bm.replicationStreamsHardLimit = 1; + bm.setMaxReplicationStreams(0, false); + bm.setReplicationStreamsHardLimit(1); long blockId = 42; // arbitrary Block aBlock = new Block(blockId, 0, 0); @@ -735,7 +735,7 @@ public class TestBlockManager { @Test public void testChooseSrcDatanodesWithDupEC() throws Exception { - bm.maxReplicationStreams = 4; + bm.setMaxReplicationStreams(4, false); long blockId = -9223372036854775776L; // real ec block id Block aBlock = new Block(blockId, 0, 0); @@ -895,7 +895,7 @@ public class TestBlockManager { assertNotNull(work); // simulate the 2 nodes reach maxReplicationStreams - for(int i = 0; i < bm.maxReplicationStreams; i++){ + for(int i = 0; i < bm.getMaxReplicationStreams(); i++){ ds3.getDatanodeDescriptor().incrementPendingReplicationWithoutTargets(); ds4.getDatanodeDescriptor().incrementPendingReplicationWithoutTargets(); } @@ -939,7 +939,7 @@ public class TestBlockManager { assertNotNull(work); // simulate the 1 node reaches maxReplicationStreams - for(int i = 0; i < bm.maxReplicationStreams; i++){ + for(int i = 0; i < bm.getMaxReplicationStreams(); i++){ ds2.getDatanodeDescriptor().incrementPendingReplicationWithoutTargets(); } @@ -948,7 +948,7 @@ public class TestBlockManager { assertNotNull(work); // simulate the 1 more node reaches maxReplicationStreams - for(int i = 0; i < bm.maxReplicationStreams; i++){ + for(int i = 0; i < bm.getMaxReplicationStreams(); i++){ ds3.getDatanodeDescriptor().incrementPendingReplicationWithoutTargets(); } @@ -997,7 +997,7 @@ public class TestBlockManager { DatanodeDescriptor[] dummyDDArray = new DatanodeDescriptor[]{dummyDD}; DatanodeStorageInfo[] dummyDSArray = new DatanodeStorageInfo[]{ds1}; // Simulate the 2 nodes reach maxReplicationStreams. - for(int i = 0; i < bm.maxReplicationStreams; i++){ //Add some dummy EC reconstruction task. + for(int i = 0; i < bm.getMaxReplicationStreams(); i++){ //Add some dummy EC reconstruction task. ds3.getDatanodeDescriptor().addBlockToBeErasureCoded(dummyBlock, dummyDDArray, dummyDSArray, new byte[0], new byte[0], ecPolicy); ds4.getDatanodeDescriptor().addBlockToBeErasureCoded(dummyBlock, dummyDDArray, @@ -1011,8 +1011,8 @@ public class TestBlockManager { @Test public void testFavorDecomUntilHardLimit() throws Exception { - bm.maxReplicationStreams = 0; - bm.replicationStreamsHardLimit = 1; + bm.setMaxReplicationStreams(0, false); + bm.setReplicationStreamsHardLimit(1); long blockId = 42; // arbitrary Block aBlock = new Block(blockId, 0, 0); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlocks.java index 0487c3f9736..04d2572b392 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlocks.java @@ -159,9 +159,9 @@ public class TestUnderReplicatedBlocks { BlockManagerTestUtil.updateState(bm); assertTrue("The number of blocks to be replicated should be less than " - + "or equal to " + bm.replicationStreamsHardLimit, + + "or equal to " + bm.getReplicationStreamsHardLimit(), secondDn.getNumberOfBlocksToBeReplicated() - <= bm.replicationStreamsHardLimit); + <= bm.getReplicationStreamsHardLimit()); DFSTestUtil.verifyClientStats(conf, cluster); } finally { cluster.shutdown(); From a70f84098fbed9efb0c241384223490209714f65 Mon Sep 17 00:00:00 2001 From: Masatake Iwasaki Date: Tue, 31 Jan 2023 19:44:19 +0900 Subject: [PATCH 08/50] HADOOP-18601. Fix build failure with docs profile. (#5331) Reviewed-by: Steve Loughran --- hadoop-hdfs-project/hadoop-hdfs/pom.xml | 4 ++++ .../hadoop-yarn-server-timelineservice-documentstore/pom.xml | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml index 4e7efab06e9..ab8934f9368 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml @@ -152,6 +152,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> xml-apis xml-apis + + xerces + xercesImpl + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/pom.xml index 17596405caf..56089a42ea8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/pom.xml @@ -122,6 +122,10 @@ io.netty netty-handler-proxy + + xml-apis + xml-apis + From 6d325d9d09cf8595e1e39e39128abd98dea75220 Mon Sep 17 00:00:00 2001 From: Masatake Iwasaki Date: Tue, 31 Jan 2023 19:49:21 +0900 Subject: [PATCH 09/50] HADOOP-18598. maven site generation doesn't include javadocs. (#5319) Reviewed-by: Chris Nauroth (cherry picked from commit 004121f9ccc2d54a6c0a2ae32a41e315fdb9349c) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 80158f8caad..68d890a8b48 100644 --- a/pom.xml +++ b/pom.xml @@ -95,7 +95,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x 2.8.1 - 3.11.0 + 3.9.1 1.5 1.7 2.4 From ad0cff2f973edb927130f4d206fcb1ecdc079e60 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Wed, 1 Feb 2023 10:15:20 -0800 Subject: [PATCH 10/50] HADOOP-18592. Sasl connection failure should log remote address. (#5294) Contributed by Viraj Jasani Signed-off-by: Chris Nauroth Signed-off-by: Steve Loughran Signed-off-by: Mingliang Liu --- .../java/org/apache/hadoop/ipc/Client.java | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java index c43f9224776..c0f90d98bc6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java @@ -704,7 +704,7 @@ public class Client implements AutoCloseable { * handle that, a relogin is attempted. */ private synchronized void handleSaslConnectionFailure( - final int currRetries, final int maxRetries, final Exception ex, + final int currRetries, final int maxRetries, final IOException ex, final Random rand, final UserGroupInformation ugi) throws IOException, InterruptedException { ugi.doAs(new PrivilegedExceptionAction() { @@ -715,10 +715,7 @@ public class Client implements AutoCloseable { disposeSasl(); if (shouldAuthenticateOverKrb()) { if (currRetries < maxRetries) { - if(LOG.isDebugEnabled()) { - LOG.debug("Exception encountered while connecting to " - + "the server : " + ex); - } + LOG.debug("Exception encountered while connecting to the server {}", remoteId, ex); // try re-login if (UserGroupInformation.isLoginKeytabBased()) { UserGroupInformation.getLoginUser().reloginFromKeytab(); @@ -736,7 +733,11 @@ public class Client implements AutoCloseable { + UserGroupInformation.getLoginUser().getUserName() + " to " + remoteId; LOG.warn(msg, ex); - throw (IOException) new IOException(msg).initCause(ex); + throw NetUtils.wrapException(remoteId.getAddress().getHostName(), + remoteId.getAddress().getPort(), + NetUtils.getHostname(), + 0, + ex); } } else { // With RequestHedgingProxyProvider, one rpc call will send multiple @@ -744,11 +745,9 @@ public class Client implements AutoCloseable { // all other requests will be interrupted. It's not a big problem, // and should not print a warning log. if (ex instanceof InterruptedIOException) { - LOG.debug("Exception encountered while connecting to the server", - ex); + LOG.debug("Exception encountered while connecting to the server {}", remoteId, ex); } else { - LOG.warn("Exception encountered while connecting to the server ", - ex); + LOG.warn("Exception encountered while connecting to the server {}", remoteId, ex); } } if (ex instanceof RemoteException) From 22f6d55b71e53fc25f72e18e89a2a647ff8c0dca Mon Sep 17 00:00:00 2001 From: Ankit Saurabh <112006204+sauraank@users.noreply.github.com> Date: Thu, 2 Feb 2023 18:45:21 +0000 Subject: [PATCH 11/50] HADOOP-18246. Reduce lower limit on fs.s3a.prefetch.block.size to 1 byte. (#5120) The minimum value of fs.s3a.prefetch.block.size is now 1 Contributed by Ankit Saurabh --- .../src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java | 3 +-- .../hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md | 1 + .../src/site/markdown/tools/hadoop-aws/prefetching.md | 4 ++++ 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 420a92788c8..cb17b80fb6a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -531,8 +531,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, this.prefetchEnabled = conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT); long prefetchBlockSizeLong = - longBytesOption(conf, PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE, - PREFETCH_BLOCK_DEFAULT_SIZE); + longBytesOption(conf, PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE, 1); if (prefetchBlockSizeLong > (long) Integer.MAX_VALUE) { throw new IOException("S3A prefatch block size exceeds int limit"); } 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 9de259e7982..ae042b16199 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 @@ -1108,6 +1108,7 @@ options are covered in [Testing](./testing.md). 8MB The size of a single prefetched block of data. + Decreasing this will increase the number of prefetches required, and may negatively impact performance. diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md index e966c2dce4c..8bb85008e36 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md @@ -43,6 +43,10 @@ Multiple blocks may be read in parallel. |`fs.s3a.prefetch.block.size` |Size of a block |`8M` | |`fs.s3a.prefetch.block.count` |Number of blocks to prefetch |`8` | +The default size of a block is 8MB, and the minimum allowed block size is 1 byte. +Decreasing block size will increase the number of blocks to be read for a file. +A smaller block size may negatively impact performance as the number of prefetches required will increase. + ### Key Components `S3PrefetchingInputStream` - When prefetching is enabled, S3AFileSystem will return an instance of From bce388fd3f2334cb6331ef781870579db17d3551 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Thu, 2 Feb 2023 17:11:02 -0800 Subject: [PATCH 12/50] HDFS-16902 Add Namenode status to BPServiceActor metrics and improve logging in offerservice (#5334) Reviewed-by: Mingliang Liu Reviewed-by: Shilun Fan Signed-off-by: Tao Li --- .../hdfs/server/datanode/BPServiceActor.java | 15 ++++++++++++++- .../hdfs/server/datanode/ErrorReportAction.java | 9 +++++++++ .../server/datanode/ReportBadBlockAction.java | 10 ++++++++++ .../src/main/webapps/datanode/datanode.html | 2 ++ 4 files changed, 35 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index 844b67ce1a8..f7b09d5fc18 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -202,6 +202,7 @@ class BPServiceActor implements Runnable { Map getActorInfoMap() { final Map info = new HashMap(); info.put("NamenodeAddress", getNameNodeAddress()); + info.put("NamenodeHaState", state != null ? state.toString() : "Unknown"); info.put("BlockPoolID", bpos.getBlockPoolId()); info.put("ActorState", getRunningState()); info.put("LastHeartbeat", @@ -697,6 +698,8 @@ class BPServiceActor implements Runnable { // Every so often, send heartbeat or block-report // final boolean sendHeartbeat = scheduler.isHeartbeatDue(startTime); + LOG.debug("BP offer service run start time: {}, sendHeartbeat: {}", startTime, + sendHeartbeat); HeartbeatResponse resp = null; if (sendHeartbeat) { // @@ -709,6 +712,8 @@ class BPServiceActor implements Runnable { boolean requestBlockReportLease = (fullBlockReportLeaseId == 0) && scheduler.isBlockReportDue(startTime); if (!dn.areHeartbeatsDisabledForTests()) { + LOG.debug("Before sending heartbeat to namenode {}, the state of the namenode known" + + " to datanode so far is {}", this.getNameNodeAddress(), state); resp = sendHeartBeat(requestBlockReportLease); assert resp != null; if (resp.getFullBlockReportLeaseId() != 0) { @@ -733,7 +738,12 @@ class BPServiceActor implements Runnable { // that we should actually process. bpos.updateActorStatesFromHeartbeat( this, resp.getNameNodeHaState()); - state = resp.getNameNodeHaState().getState(); + HAServiceState stateFromResp = resp.getNameNodeHaState().getState(); + if (state != stateFromResp) { + LOG.info("After receiving heartbeat response, updating state of namenode {} to {}", + this.getNameNodeAddress(), stateFromResp); + } + state = stateFromResp; if (state == HAServiceState.ACTIVE) { handleRollingUpgradeStatus(resp); @@ -794,6 +804,7 @@ class BPServiceActor implements Runnable { long sleepTime = Math.min(1000, dnConf.heartBeatInterval); Thread.sleep(sleepTime); } catch (InterruptedException ie) { + LOG.info("BPServiceActor {} is interrupted", this); Thread.currentThread().interrupt(); } } @@ -995,6 +1006,8 @@ class BPServiceActor implements Runnable { while (!duplicateQueue.isEmpty()) { BPServiceActorAction actionItem = duplicateQueue.remove(); try { + LOG.debug("BPServiceActor ( {} ) processing queued messages. Action item: {}", this, + actionItem); actionItem.reportTo(bpNamenode, bpRegistration); } catch (BPServiceActorActionException baae) { LOG.warn(baae.getMessage() + nnAddr , baae); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ErrorReportAction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ErrorReportAction.java index c56c5070a22..34b89024189 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ErrorReportAction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ErrorReportAction.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode; import java.io.IOException; +import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.ipc.RemoteException; @@ -84,4 +85,12 @@ public class ErrorReportAction implements BPServiceActorAction { } return true; } + + @Override + public String toString() { + return new ToStringBuilder(this) + .append("errorCode", errorCode) + .append("errorMessage", errorMessage) + .toString(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java index 1e72a6bc890..ec3884a82fb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode; import java.io.IOException; +import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder; @@ -111,4 +112,13 @@ public class ReportBadBlockAction implements BPServiceActorAction { } return true; } + + @Override + public String toString() { + return new ToStringBuilder(this) + .append("block", block) + .append("storageUuid", storageUuid) + .append("storageType", storageType) + .toString(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/datanode.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/datanode.html index caab81ef686..b491d5a04e3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/datanode.html +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/datanode.html @@ -81,6 +81,7 @@ Namenode Address + Namenode HA State Block Pool ID Actor State Last Heartbeat @@ -91,6 +92,7 @@ {#dn.BPServiceActorInfo} {NamenodeAddress} + {NamenodeHaState} {BlockPoolID} {ActorState} {LastHeartbeat}s From dad73b76c07025264d8daf1d42436f8ed6395995 Mon Sep 17 00:00:00 2001 From: jokercurry <84573424+smallzhongfeng@users.noreply.github.com> Date: Sat, 4 Feb 2023 10:29:19 +0800 Subject: [PATCH 13/50] YARN-11419. Remove redundant exception capture in NMClientAsyncImpl and improve readability in ContainerShellWebSocket, etc (#5309) Co-authored-by: smallzhongfeng <982458633@qq.com> Reviewed-by: Shilun Fan Signed-off-by: Shilun Fan --- .../client/api/ContainerShellWebSocket.java | 4 ++-- .../api/async/impl/NMClientAsyncImpl.java | 20 +++---------------- .../api/impl/SharedCacheClientImpl.java | 8 +------- .../yarn/client/util/YarnClientUtils.java | 2 +- 4 files changed, 7 insertions(+), 27 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/ContainerShellWebSocket.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/ContainerShellWebSocket.java index efcc2ea0aed..66a901fc36a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/ContainerShellWebSocket.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/ContainerShellWebSocket.java @@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.client.api; import java.io.IOException; import java.io.OutputStream; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -62,7 +62,7 @@ public class ContainerShellWebSocket { session.getRemote().flush(); sttySet = true; } - terminal.output().write(message.getBytes(Charset.forName("UTF-8"))); + terminal.output().write(message.getBytes(StandardCharsets.UTF_8)); terminal.output().flush(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java index 3215aa299a4..eb5b9b227fb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java @@ -18,7 +18,6 @@ package org.apache.hadoop.yarn.client.api.async.impl; -import java.io.IOException; import java.nio.ByteBuffer; import java.util.EnumSet; import java.util.HashSet; @@ -51,7 +50,6 @@ import org.apache.hadoop.yarn.client.api.impl.NMClientImpl; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.AbstractEvent; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.ipc.RPCUtil; import org.apache.hadoop.yarn.state.InvalidStateTransitionException; import org.apache.hadoop.yarn.state.MultipleArcTransition; @@ -636,12 +634,8 @@ public class NMClientAsyncImpl extends NMClientAsync { + "Container " + containerId, thr); } return ContainerState.RUNNING; - } catch (YarnException e) { + } catch (Throwable e) { return onExceptionRaised(container, event, e); - } catch (IOException e) { - return onExceptionRaised(container, event, e); - } catch (Throwable t) { - return onExceptionRaised(container, event, t); } } @@ -854,12 +848,8 @@ public class NMClientAsyncImpl extends NMClientAsync { + "Container " + event.getContainerId(), thr); } return ContainerState.DONE; - } catch (YarnException e) { + } catch (Throwable e) { return onExceptionRaised(container, event, e); - } catch (IOException e) { - return onExceptionRaised(container, event, e); - } catch (Throwable t) { - return onExceptionRaised(container, event, t); } } @@ -966,12 +956,8 @@ public class NMClientAsyncImpl extends NMClientAsync { "Unchecked exception is thrown from onContainerStatusReceived" + " for Container " + event.getContainerId(), thr); } - } catch (YarnException e) { + } catch (Throwable e) { onExceptionRaised(containerId, e); - } catch (IOException e) { - onExceptionRaised(containerId, e); - } catch (Throwable t) { - onExceptionRaised(containerId, t); } } else { StatefulContainer container = containers.get(containerId); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/SharedCacheClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/SharedCacheClientImpl.java index cce6ae8df61..103080a8502 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/SharedCacheClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/SharedCacheClientImpl.java @@ -158,14 +158,8 @@ public class SharedCacheClientImpl extends SharedCacheClient { public String getFileChecksum(Path sourceFile) throws IOException { FileSystem fs = sourceFile.getFileSystem(this.conf); - FSDataInputStream in = null; - try { - in = fs.open(sourceFile); + try (FSDataInputStream in = fs.open(sourceFile)) { return this.checksum.computeChecksum(in); - } finally { - if (in != null) { - in.close(); - } } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/util/YarnClientUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/util/YarnClientUtils.java index 041152d7df8..049dbd7962c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/util/YarnClientUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/util/YarnClientUtils.java @@ -145,7 +145,7 @@ public abstract class YarnClientUtils { // Now we only support one property, which is exclusive, so check if // key = exclusive and value = {true/false} - if (key.equals("exclusive") + if ("exclusive".equals(key) && ImmutableSet.of("true", "false").contains(value)) { exclusive = Boolean.parseBoolean(value); } else { From 0ae075a2af68e352aa3a75e9fbf7c777bd09e135 Mon Sep 17 00:00:00 2001 From: sunhao Date: Sun, 5 Feb 2023 09:33:05 +0800 Subject: [PATCH 14/50] HDFS-16848. RBF: Improve StateStoreZooKeeperImpl performance (#5147) --- .../federation/router/RBFConfigKeys.java | 12 ++ .../driver/impl/StateStoreZooKeeperImpl.java | 159 +++++++++++++----- .../src/main/resources/hdfs-rbf-default.xml | 20 +++ .../driver/TestStateStoreDriverBase.java | 2 +- .../store/driver/TestStateStoreZK.java | 69 +++++++- 5 files changed, 207 insertions(+), 55 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java index c598076f636..7e07d7b6549 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java @@ -239,6 +239,18 @@ public class RBFConfigKeys extends CommonConfigurationKeysPublic { public static final long FEDERATION_STORE_ROUTER_EXPIRATION_DELETION_MS_DEFAULT = -1; + // HDFS Router-based federation State Store ZK DRIVER + public static final String FEDERATION_STORE_ZK_DRIVER_PREFIX = + RBFConfigKeys.FEDERATION_STORE_PREFIX + "driver.zk."; + public static final String FEDERATION_STORE_ZK_PARENT_PATH = + FEDERATION_STORE_ZK_DRIVER_PREFIX + "parent-path"; + public static final String FEDERATION_STORE_ZK_PARENT_PATH_DEFAULT = + "/hdfs-federation"; + public static final String FEDERATION_STORE_ZK_ASYNC_MAX_THREADS = + FEDERATION_STORE_ZK_DRIVER_PREFIX + "async.max.threads"; + public static final int FEDERATION_STORE_ZK_ASYNC_MAX_THREADS_DEFAULT = + -1; + // HDFS Router safe mode public static final String DFS_ROUTER_SAFEMODE_ENABLE = FEDERATION_ROUTER_PREFIX + "safemode.enable"; diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreZooKeeperImpl.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreZooKeeperImpl.java index 45442da0ab5..7882c8f8273 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreZooKeeperImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreZooKeeperImpl.java @@ -25,7 +25,16 @@ import static org.apache.hadoop.util.Time.monotonicNow; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.imps.CuratorFrameworkState; import org.apache.hadoop.conf.Configuration; @@ -57,14 +66,9 @@ public class StateStoreZooKeeperImpl extends StateStoreSerializableImpl { private static final Logger LOG = LoggerFactory.getLogger(StateStoreZooKeeperImpl.class); - - /** Configuration keys. */ - public static final String FEDERATION_STORE_ZK_DRIVER_PREFIX = - RBFConfigKeys.FEDERATION_STORE_PREFIX + "driver.zk."; - public static final String FEDERATION_STORE_ZK_PARENT_PATH = - FEDERATION_STORE_ZK_DRIVER_PREFIX + "parent-path"; - public static final String FEDERATION_STORE_ZK_PARENT_PATH_DEFAULT = - "/hdfs-federation"; + /** Service to get/update zk state. */ + private ThreadPoolExecutor executorService; + private boolean enableConcurrent; /** Directory to store the state store data. */ @@ -82,8 +86,22 @@ public class StateStoreZooKeeperImpl extends StateStoreSerializableImpl { Configuration conf = getConf(); baseZNode = conf.get( - FEDERATION_STORE_ZK_PARENT_PATH, - FEDERATION_STORE_ZK_PARENT_PATH_DEFAULT); + RBFConfigKeys.FEDERATION_STORE_ZK_PARENT_PATH, + RBFConfigKeys.FEDERATION_STORE_ZK_PARENT_PATH_DEFAULT); + int numThreads = conf.getInt( + RBFConfigKeys.FEDERATION_STORE_ZK_ASYNC_MAX_THREADS, + RBFConfigKeys.FEDERATION_STORE_ZK_ASYNC_MAX_THREADS_DEFAULT); + enableConcurrent = numThreads > 0; + if (enableConcurrent) { + ThreadFactory threadFactory = new ThreadFactoryBuilder() + .setNameFormat("StateStore ZK Client-%d") + .build(); + this.executorService = new ThreadPoolExecutor(numThreads, numThreads, + 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>(), threadFactory); + LOG.info("Init StateStoreZookeeperImpl by async mode with {} threads.", numThreads); + } else { + LOG.info("Init StateStoreZookeeperImpl by sync mode."); + } try { this.zkManager = new ZKCuratorManager(conf); this.zkManager.start(); @@ -109,8 +127,16 @@ public class StateStoreZooKeeperImpl extends StateStoreSerializableImpl { } } + @VisibleForTesting + public void setEnableConcurrent(boolean enableConcurrent) { + this.enableConcurrent = enableConcurrent; + } + @Override public void close() throws Exception { + if (executorService != null) { + executorService.shutdown(); + } if (zkManager != null) { zkManager.close(); } @@ -136,34 +162,21 @@ public class StateStoreZooKeeperImpl extends StateStoreSerializableImpl { List ret = new ArrayList<>(); String znode = getZNodeForClass(clazz); try { - List children = zkManager.getChildren(znode); - for (String child : children) { - try { - String path = getNodePath(znode, child); - Stat stat = new Stat(); - String data = zkManager.getStringData(path, stat); - boolean corrupted = false; - if (data == null || data.equals("")) { - // All records should have data, otherwise this is corrupted - corrupted = true; - } else { - try { - T record = createRecord(data, stat, clazz); - ret.add(record); - } catch (IOException e) { - LOG.error("Cannot create record type \"{}\" from \"{}\": {}", - clazz.getSimpleName(), data, e.getMessage()); - corrupted = true; - } + List> callables = new ArrayList<>(); + zkManager.getChildren(znode).forEach(c -> callables.add(() -> getRecord(clazz, znode, c))); + if (enableConcurrent) { + List> futures = executorService.invokeAll(callables); + for (Future future : futures) { + if (future.get() != null) { + ret.add(future.get()); } - - if (corrupted) { - LOG.error("Cannot get data for {} at {}, cleaning corrupted data", - child, path); - zkManager.delete(path); + } + } else { + for (Callable callable : callables) { + T record = callable.call(); + if (record != null) { + ret.add(record); } - } catch (Exception e) { - LOG.error("Cannot get data for {}: {}", child, e.getMessage()); } } } catch (Exception e) { @@ -178,6 +191,44 @@ public class StateStoreZooKeeperImpl extends StateStoreSerializableImpl { return new QueryResult(ret, getTime()); } + /** + * Get one data record in the StateStore or delete it if it's corrupted. + * + * @param clazz Record class to evaluate. + * @param znode The ZNode for the class. + * @param child The child for znode to get. + * @return The record to get. + */ + private T getRecord(Class clazz, String znode, String child) { + T record = null; + try { + String path = getNodePath(znode, child); + Stat stat = new Stat(); + String data = zkManager.getStringData(path, stat); + boolean corrupted = false; + if (data == null || data.equals("")) { + // All records should have data, otherwise this is corrupted + corrupted = true; + } else { + try { + record = createRecord(data, stat, clazz); + } catch (IOException e) { + LOG.error("Cannot create record type \"{}\" from \"{}\": {}", + clazz.getSimpleName(), data, e.getMessage()); + corrupted = true; + } + } + + if (corrupted) { + LOG.error("Cannot get data for {} at {}, cleaning corrupted data", child, path); + zkManager.delete(path); + } + } catch (Exception e) { + LOG.error("Cannot get data for {}: {}", child, e.getMessage()); + } + return record; + } + @Override public boolean putAll( List records, boolean update, boolean error) throws IOException { @@ -192,22 +243,40 @@ public class StateStoreZooKeeperImpl extends StateStoreSerializableImpl { String znode = getZNodeForClass(recordClass); long start = monotonicNow(); - boolean status = true; - for (T record : records) { - String primaryKey = getPrimaryKey(record); - String recordZNode = getNodePath(znode, primaryKey); - byte[] data = serialize(record); - if (!writeNode(recordZNode, data, update, error)){ - status = false; + final AtomicBoolean status = new AtomicBoolean(true); + List> callables = new ArrayList<>(); + records.forEach(record -> + callables.add( + () -> { + String primaryKey = getPrimaryKey(record); + String recordZNode = getNodePath(znode, primaryKey); + byte[] data = serialize(record); + if (!writeNode(recordZNode, data, update, error)) { + status.set(false); + } + return null; + } + ) + ); + try { + if (enableConcurrent) { + executorService.invokeAll(callables); + } else { + for(Callable callable : callables) { + callable.call(); + } } + } catch (Exception e) { + LOG.error("Write record failed : {}", e.getMessage(), e); + throw new IOException(e); } long end = monotonicNow(); - if (status) { + if (status.get()) { getMetrics().addWrite(end - start); } else { getMetrics().addFailure(end - start); } - return status; + return status.get(); } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml index 52a1e3a3bd1..b5096cd253d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml @@ -377,6 +377,26 @@ + + dfs.federation.router.store.driver.zk.parent-path + /hdfs-federation + + The parent path of zookeeper for StateStoreZooKeeperImpl. + + + + + dfs.federation.router.store.driver.zk.async.max.threads + -1 + + Max threads number of StateStoreZooKeeperImpl in async mode. + The only class currently being supported: + org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreZooKeeperImpl. + Default value is -1, which means StateStoreZooKeeperImpl is working in sync mode. + Use positive integer value to enable async mode. + + + dfs.federation.router.cache.ttl 1m diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java index 5ad01dce8e7..4eb38b06b12 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java @@ -119,7 +119,7 @@ public class TestStateStoreDriverBase { } @SuppressWarnings("unchecked") - private T generateFakeRecord(Class recordClass) + protected T generateFakeRecord(Class recordClass) throws IllegalArgumentException, IllegalAccessException, IOException { if (recordClass == MembershipState.class) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreZK.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreZK.java index f8be9f0a05b..3ad106697ac 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreZK.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreZK.java @@ -18,12 +18,13 @@ package org.apache.hadoop.hdfs.server.federation.store.driver; import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.getStateStoreConfiguration; -import static org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreZooKeeperImpl.FEDERATION_STORE_ZK_PARENT_PATH; -import static org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreZooKeeperImpl.FEDERATION_STORE_ZK_PARENT_PATH_DEFAULT; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.curator.framework.CuratorFramework; @@ -40,6 +41,7 @@ import org.apache.hadoop.hdfs.server.federation.store.records.DisabledNameservic import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState; import org.apache.hadoop.hdfs.server.federation.store.records.MountTable; import org.apache.hadoop.hdfs.server.federation.store.records.RouterState; +import org.apache.hadoop.util.Time; import org.apache.zookeeper.CreateMode; import org.junit.AfterClass; import org.junit.Before; @@ -73,9 +75,10 @@ public class TestStateStoreZK extends TestStateStoreDriverBase { // Disable auto-repair of connection conf.setLong(RBFConfigKeys.FEDERATION_STORE_CONNECTION_TEST_MS, TimeUnit.HOURS.toMillis(1)); + conf.setInt(RBFConfigKeys.FEDERATION_STORE_ZK_ASYNC_MAX_THREADS, 10); - baseZNode = conf.get(FEDERATION_STORE_ZK_PARENT_PATH, - FEDERATION_STORE_ZK_PARENT_PATH_DEFAULT); + baseZNode = conf.get(RBFConfigKeys.FEDERATION_STORE_ZK_PARENT_PATH, + RBFConfigKeys.FEDERATION_STORE_ZK_PARENT_PATH_DEFAULT); getStateStore(conf); } @@ -91,6 +94,8 @@ public class TestStateStoreZK extends TestStateStoreDriverBase { @Before public void startup() throws IOException { removeAll(getStateStoreDriver()); + StateStoreZooKeeperImpl stateStoreZooKeeper = (StateStoreZooKeeperImpl) getStateStoreDriver(); + stateStoreZooKeeper.setEnableConcurrent(false); } private String generateFakeZNode( @@ -126,33 +131,79 @@ public class TestStateStoreZK extends TestStateStoreDriverBase { assertNull(curatorFramework.checkExists().forPath(znode)); } + @Test + public void testAsyncPerformance() throws Exception { + StateStoreZooKeeperImpl stateStoreDriver = (StateStoreZooKeeperImpl) getStateStoreDriver(); + List insertList = new ArrayList<>(); + for (int i = 0; i < 1000; i++) { + MountTable newRecord = generateFakeRecord(MountTable.class); + insertList.add(newRecord); + } + // Insert Multiple on sync mode + long startSync = Time.now(); + stateStoreDriver.putAll(insertList, true, false); + long endSync = Time.now(); + stateStoreDriver.removeAll(MembershipState.class); + + stateStoreDriver.setEnableConcurrent(true); + // Insert Multiple on async mode + long startAsync = Time.now(); + stateStoreDriver.putAll(insertList, true, false); + long endAsync = Time.now(); + assertTrue((endSync - startSync) > (endAsync - startAsync)); + } + @Test public void testGetNullRecord() throws Exception { - testGetNullRecord(getStateStoreDriver()); + StateStoreZooKeeperImpl stateStoreDriver = (StateStoreZooKeeperImpl) getStateStoreDriver(); + testGetNullRecord(stateStoreDriver); + + // test async mode + stateStoreDriver.setEnableConcurrent(true); + testGetNullRecord(stateStoreDriver); } @Test public void testInsert() throws IllegalArgumentException, IllegalAccessException, IOException { - testInsert(getStateStoreDriver()); + StateStoreZooKeeperImpl stateStoreDriver = (StateStoreZooKeeperImpl) getStateStoreDriver(); + testInsert(stateStoreDriver); + // test async mode + stateStoreDriver.setEnableConcurrent(true); + testInsert(stateStoreDriver); } @Test public void testUpdate() throws IllegalArgumentException, ReflectiveOperationException, IOException, SecurityException { - testPut(getStateStoreDriver()); + StateStoreZooKeeperImpl stateStoreDriver = (StateStoreZooKeeperImpl) getStateStoreDriver(); + testPut(stateStoreDriver); + + // test async mode + stateStoreDriver.setEnableConcurrent(true); + testPut(stateStoreDriver); } @Test public void testDelete() throws IllegalArgumentException, IllegalAccessException, IOException { - testRemove(getStateStoreDriver()); + StateStoreZooKeeperImpl stateStoreDriver = (StateStoreZooKeeperImpl) getStateStoreDriver(); + testRemove(stateStoreDriver); + + // test async mode + stateStoreDriver.setEnableConcurrent(true); + testRemove(stateStoreDriver); } @Test public void testFetchErrors() throws IllegalArgumentException, IllegalAccessException, IOException { - testFetchErrors(getStateStoreDriver()); + StateStoreZooKeeperImpl stateStoreDriver = (StateStoreZooKeeperImpl) getStateStoreDriver(); + testFetchErrors(stateStoreDriver); + + // test async mode + stateStoreDriver.setEnableConcurrent(true); + testFetchErrors(stateStoreDriver); } } \ No newline at end of file From be564f5c20545e9e094806139b08b57fc1e2a961 Mon Sep 17 00:00:00 2001 From: hfutatzhanghb <1036798979@qq.com> Date: Mon, 6 Feb 2023 12:21:28 +0800 Subject: [PATCH 15/50] [HDFS-16903]. Fix javadoc of LightWeightResizableGSet class (#5338) --- .../java/org/apache/hadoop/util/LightWeightResizableGSet.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightResizableGSet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightResizableGSet.java index 051e2680bc3..1383a7fafe7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightResizableGSet.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightResizableGSet.java @@ -33,7 +33,7 @@ import java.util.function.Consumer; * * This class does not support null element. * - * This class is not thread safe. + * This class is thread safe. * * @param Key type for looking up the elements * @param Element type, which must be From aed6fcee5b0f61683e4da9081ed6eead3c5ac6aa Mon Sep 17 00:00:00 2001 From: Steve Vaughan Date: Mon, 6 Feb 2023 13:17:57 -0500 Subject: [PATCH 16/50] HADOOP-18576. Java 11 JavaDoc fails due to missing package comments (#5344) Add JavaDoc comments to package-info.java to avoid errors resulting from the use of Hadoop annotations. Contributed by Steve Vaughan Jr --- .../main/java/org/apache/hadoop/fs/local/package-info.java | 5 +++++ .../main/java/org/apache/hadoop/fs/shell/package-info.java | 4 ++++ .../src/main/java/org/apache/hadoop/http/package-info.java | 4 ++++ .../org/apache/hadoop/io/compress/bzip2/package-info.java | 5 +++++ .../org/apache/hadoop/io/compress/lz4/package-info.java | 7 +++++++ .../org/apache/hadoop/io/compress/snappy/package-info.java | 7 +++++++ .../org/apache/hadoop/io/compress/zlib/package-info.java | 7 +++++++ .../org/apache/hadoop/io/compress/zstd/package-info.java | 7 +++++++ .../java/org/apache/hadoop/io/nativeio/package-info.java | 6 ++++++ .../org/apache/hadoop/security/authorize/package-info.java | 4 ++++ .../java/org/apache/hadoop/security/http/package-info.java | 4 ++++ .../hadoop/security/token/delegation/package-info.java | 4 ++++ .../org/apache/hadoop/security/token/package-info.java | 4 ++++ .../main/java/org/apache/hadoop/service/package-info.java | 4 ++++ .../org/apache/hadoop/util/concurrent/package-info.java | 5 +++-- 15 files changed, 75 insertions(+), 2 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/package-info.java index 48d6644e99b..7d9b829f7d3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/package-info.java @@ -15,6 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/** + * Filesystem implementations that allow Hadoop to read directly from + * the local file system. + */ @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) @InterfaceStability.Unstable package org.apache.hadoop.fs.local; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/package-info.java index 92720bff69b..2f0542aa696 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/package-info.java @@ -15,6 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/** + * Support for the execution of a file system command. + */ @InterfaceAudience.Private @InterfaceStability.Unstable package org.apache.hadoop.fs.shell; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/package-info.java index 32bbbf22307..b1710e0f9cb 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/package-info.java @@ -15,6 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/** + * Support for embedded HTTP services. + */ @InterfaceAudience.LimitedPrivate({"HBase", "HDFS", "MapReduce"}) @InterfaceStability.Unstable package org.apache.hadoop.http; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/package-info.java index 785170eaf62..9973b78e39a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/package-info.java @@ -15,6 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/** + * Implementation of compression/decompression for the BZip2 + * compression algorithm. + */ @InterfaceAudience.Private @InterfaceStability.Unstable package org.apache.hadoop.io.compress.bzip2; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/package-info.java index 11827f17486..438dfdea3e7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/package-info.java @@ -15,6 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/** + * Implementation of compression/decompression for the LZ4 + * compression algorithm. + * + * @see LZ4 + */ @InterfaceAudience.Private @InterfaceStability.Unstable package org.apache.hadoop.io.compress.lz4; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/package-info.java index eedf6550833..320fd026a1d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/package-info.java @@ -15,6 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/** + * Implementation of compression/decompression for the Snappy + * compression algorithm. + * + * @see Snappy + */ @InterfaceAudience.Private @InterfaceStability.Unstable package org.apache.hadoop.io.compress.snappy; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/package-info.java index 33d0a8d7ceb..515eb3498f2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/package-info.java @@ -15,6 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/** + * Implementation of compression/decompression based on the popular + * gzip compressed file format. + * + * @see gzip + */ @InterfaceAudience.Private @InterfaceStability.Unstable package org.apache.hadoop.io.compress.zlib; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zstd/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zstd/package-info.java index 9069070f73a..7214bf8582b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zstd/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zstd/package-info.java @@ -15,6 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/** + * Implementation of compression/decompression based on the zStandard + * compression algorithm. + * + * @see zStandard + */ @InterfaceAudience.Private @InterfaceStability.Unstable package org.apache.hadoop.io.compress.zstd; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/package-info.java index 346f895e650..7e47b3b54af 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/package-info.java @@ -15,6 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/** + * Various native IO-related calls not available in Java. These + * functions should generally be used alongside a fallback to another + * more portable mechanism. + */ @InterfaceAudience.Private @InterfaceStability.Unstable package org.apache.hadoop.io.nativeio; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/package-info.java index e1060e2196d..3c75a2427d8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/package-info.java @@ -15,6 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/** + * Support for service-level authorization. + */ @InterfaceAudience.Public @InterfaceStability.Evolving package org.apache.hadoop.security.authorize; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/http/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/http/package-info.java index 8e9398eb679..a58b3cdcfb9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/http/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/http/package-info.java @@ -15,6 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/** + * Filters for HTTP service security. + */ @InterfaceAudience.Public @InterfaceStability.Evolving package org.apache.hadoop.security.http; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/package-info.java index c85f967ab67..0b3b8c46944 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/package-info.java @@ -15,6 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/** + * ZooKeeper secret manager for TokenIdentifiers and DelegationKeys. + */ @InterfaceAudience.LimitedPrivate({"HBase", "HDFS", "MapReduce"}) @InterfaceStability.Evolving package org.apache.hadoop.security.token.delegation; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/package-info.java index e015056b43e..cdf4e61050d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/package-info.java @@ -15,6 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/** + * Support for delegation tokens. + */ @InterfaceAudience.Public @InterfaceStability.Evolving package org.apache.hadoop.security.token; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/package-info.java index 37164855499..81409382648 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/package-info.java @@ -15,6 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/** + * Support for services. + */ @InterfaceAudience.Public package org.apache.hadoop.service; import org.apache.hadoop.classification.InterfaceAudience; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/package-info.java index 2effb65872e..871005adc0c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/package-info.java @@ -1,5 +1,4 @@ /* - * * * 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 @@ -15,9 +14,11 @@ * 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. - * / */ +/** + * Support for concurrent execution. + */ @InterfaceAudience.Private @InterfaceStability.Unstable package org.apache.hadoop.util.concurrent; From 5f5157ac536388f559361b8df012a7960d19050f Mon Sep 17 00:00:00 2001 From: Steve Vaughan Date: Mon, 6 Feb 2023 13:28:29 -0500 Subject: [PATCH 17/50] HADOOP-18612. Avoid mixing canonical and non-canonical when performing comparisons (#5339) Contributed by Steve Vaughan Jr --- .../src/test/java/org/apache/hadoop/fs/TestFileUtil.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java index c884e223365..94d90b2eb97 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java @@ -1321,16 +1321,16 @@ public class TestFileUtil { if (wildcardPath.equals(classPath)) { // add wildcard matches for (File wildcardMatch: wildcardMatches) { - expectedClassPaths.add(wildcardMatch.toURI().toURL() + expectedClassPaths.add(wildcardMatch.getCanonicalFile().toURI().toURL() .toExternalForm()); } } else { File fileCp = null; if(!new Path(classPath).isAbsolute()) { - fileCp = new File(tmp, classPath); + fileCp = new File(tmp, classPath).getCanonicalFile(); } else { - fileCp = new File(classPath); + fileCp = new File(classPath).getCanonicalFile(); } if (nonExistentSubdir.equals(classPath)) { // expect to maintain trailing path separator if present in input, even @@ -1385,7 +1385,8 @@ public class TestFileUtil { for (Path jar: jars) { URL url = jar.toUri().toURL(); assertTrue("the jar should match either of the jars", - url.equals(jar1.toURI().toURL()) || url.equals(jar2.toURI().toURL())); + url.equals(jar1.getCanonicalFile().toURI().toURL()) || + url.equals(jar2.getCanonicalFile().toURI().toURL())); } } From a6a9fe17e030a809192ffaad9bd42833d6744a24 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Tue, 7 Feb 2023 03:47:07 +0800 Subject: [PATCH 18/50] YARN-3657. Federation maintenance mechanisms (simple CLI and command propagation). (#5348) --- hadoop-yarn-project/hadoop-yarn/bin/yarn | 4 + hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd | 6 ++ .../hadoop/yarn/client/cli/RouterCLI.java | 93 +++++++++++++++++++ .../hadoop/yarn/client/cli/TestRouterCLI.java | 64 +++++++++++++ 4 files changed, 167 insertions(+) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RouterCLI.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRouterCLI.java diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn index f305c2744ef..7fff145c03b 100755 --- a/hadoop-yarn-project/hadoop-yarn/bin/yarn +++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn @@ -50,6 +50,7 @@ function hadoop_usage hadoop_add_subcommand "fs2cs" client "converts Fair Scheduler configuration to Capacity Scheduler (EXPERIMENTAL)" hadoop_add_subcommand "rmadmin" admin "admin tools" hadoop_add_subcommand "router" daemon "run the Router daemon" + hadoop_add_subcommand "routeradmin" admin "router admin tools" hadoop_add_subcommand "schedulerconf" client "Updates scheduler configuration" hadoop_add_subcommand "scmadmin" admin "SharedCacheManager admin tools" hadoop_add_subcommand "sharedcachemanager" daemon "run the SharedCacheManager daemon" @@ -180,6 +181,9 @@ ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_LIB_JARS_DIR}" HADOOP_HEAPSIZE_MAX="${YARN_ROUTER_HEAPSIZE}" fi ;; + routeradmin) + HADOOP_CLASSNAME='org.apache.hadoop.yarn.client.cli.RouterCLI' + ;; schedulerconf) HADOOP_CLASSNAME='org.apache.hadoop.yarn.client.cli.SchedConfCLI' ;; diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd index ed0294c2edf..03c66b09edf 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd +++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd @@ -265,6 +265,11 @@ goto :eof ) goto :eof +:routeradmin + set CLASS=org.apache.hadoop.yarn.client.cli.RouterCLI + set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS% + goto :eof + :nodemanager set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%\nm-config\log4j.properties set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\%YARN_DIR%\timelineservice\* @@ -342,6 +347,7 @@ goto :eof @echo resourcemanager run the ResourceManager @echo nodemanager run a nodemanager on each slave @echo router run the Router daemon + @echo routeradmin router admin tools @echo timelineserver run the timeline server @echo timelinereader run the timeline reader server @echo rmadmin admin tools diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RouterCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RouterCLI.java new file mode 100644 index 00000000000..3aae20ccf96 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RouterCLI.java @@ -0,0 +1,93 @@ +/** + * 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.yarn.client.cli; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; +import org.apache.hadoop.yarn.client.ClientRMProxy; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol; + +import java.io.IOException; + +public class RouterCLI extends Configured implements Tool { + + public RouterCLI() { + super(); + } + + public RouterCLI(Configuration conf) { + super(conf); + } + + private static void printHelp() { + StringBuilder summary = new StringBuilder(); + summary.append("router-admin is the command to execute " + + "YARN Federation administrative commands.\n"); + StringBuilder helpBuilder = new StringBuilder(); + System.out.println(summary); + helpBuilder.append(" -help [cmd]: Displays help for the given command or all commands" + + " if none is specified."); + System.out.println(helpBuilder); + System.out.println(); + ToolRunner.printGenericCommandUsage(System.out); + } + + protected ResourceManagerAdministrationProtocol createAdminProtocol() + throws IOException { + // Get the current configuration + final YarnConfiguration conf = new YarnConfiguration(getConf()); + return ClientRMProxy.createRMProxy(conf, ResourceManagerAdministrationProtocol.class); + } + + private static void buildUsageMsg(StringBuilder builder) { + builder.append("router-admin is only used in Yarn Federation Mode.\n"); + builder.append("Usage: router-admin\n"); + builder.append(" -help" + " [cmd]\n"); + } + + private static void printUsage() { + StringBuilder usageBuilder = new StringBuilder(); + buildUsageMsg(usageBuilder); + System.err.println(usageBuilder); + ToolRunner.printGenericCommandUsage(System.err); + } + + @Override + public int run(String[] args) throws Exception { + YarnConfiguration yarnConf = getConf() == null ? + new YarnConfiguration() : new YarnConfiguration(getConf()); + boolean isFederationEnabled = yarnConf.getBoolean(YarnConfiguration.FEDERATION_ENABLED, + YarnConfiguration.DEFAULT_FEDERATION_ENABLED); + + if (args.length < 1 || !isFederationEnabled) { + printUsage(); + return -1; + } + + String cmd = args[0]; + if ("-help".equals(cmd)) { + printHelp(); + return 0; + } + + return 0; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRouterCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRouterCLI.java new file mode 100644 index 00000000000..a8401f8870e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRouterCLI.java @@ -0,0 +1,64 @@ +/** + * 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.yarn.client.cli; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; + +public class TestRouterCLI { + + private ResourceManagerAdministrationProtocol admin; + private RouterCLI rmAdminCLI; + + @Before + public void setup() throws Exception { + + admin = mock(ResourceManagerAdministrationProtocol.class); + Configuration config = new Configuration(); + config.setBoolean(YarnConfiguration.FEDERATION_ENABLED, true); + + rmAdminCLI = new RouterCLI(config) { + @Override + protected ResourceManagerAdministrationProtocol createAdminProtocol() { + return admin; + } + }; + } + + @Test + public void testHelp() throws Exception { + PrintStream oldOutPrintStream = System.out; + PrintStream oldErrPrintStream = System.err; + ByteArrayOutputStream dataOut = new ByteArrayOutputStream(); + ByteArrayOutputStream dataErr = new ByteArrayOutputStream(); + System.setOut(new PrintStream(dataOut)); + System.setErr(new PrintStream(dataErr)); + + String[] args = {"-help"}; + assertEquals(0, rmAdminCLI.run(args)); + } +} From f02c452cf176e0c70a75207035e410d9f62fb39f Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Mon, 6 Feb 2023 17:54:40 -0800 Subject: [PATCH 19/50] HDFS-16907. Add LastHeartbeatResponseTime for BP service actor (#5349) Reviewed-by: Ayush Saxena Reviewed-by: Shilun Fan Signed-off-by: Tao Li --- .../hdfs/server/datanode/BPServiceActor.java | 15 ++++ .../hadoop/hdfs/server/datanode/DataNode.java | 35 +++++++- .../src/main/webapps/datanode/datanode.html | 4 +- .../apache/hadoop/hdfs/MiniDFSCluster.java | 18 +++++ .../server/datanode/TestDataNodeMXBean.java | 79 +++++++++++++++++++ 5 files changed, 146 insertions(+), 5 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index f7b09d5fc18..35ab6193142 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -207,6 +207,8 @@ class BPServiceActor implements Runnable { info.put("ActorState", getRunningState()); info.put("LastHeartbeat", String.valueOf(getScheduler().getLastHearbeatTime())); + info.put("LastHeartbeatResponseTime", + String.valueOf(getScheduler().getLastHeartbeatResponseTime())); info.put("LastBlockReport", String.valueOf(getScheduler().getLastBlockReportTime())); info.put("maxBlockReportSize", String.valueOf(getMaxBlockReportSize())); @@ -580,6 +582,8 @@ class BPServiceActor implements Runnable { slowPeers, slowDisks); + scheduler.updateLastHeartbeatResponseTime(monotonicNow()); + if (outliersReportDue) { // If the report was due and successfully sent, schedule the next one. scheduler.scheduleNextOutlierReport(); @@ -1202,6 +1206,9 @@ class BPServiceActor implements Runnable { @VisibleForTesting volatile long lastHeartbeatTime = monotonicNow(); + @VisibleForTesting + private volatile long lastHeartbeatResponseTime = -1; + @VisibleForTesting boolean resetBlockReportTime = true; @@ -1250,6 +1257,10 @@ class BPServiceActor implements Runnable { lastHeartbeatTime = heartbeatTime; } + void updateLastHeartbeatResponseTime(long heartbeatTime) { + this.lastHeartbeatResponseTime = heartbeatTime; + } + void updateLastBlockReportTime(long blockReportTime) { lastBlockReportTime = blockReportTime; } @@ -1262,6 +1273,10 @@ class BPServiceActor implements Runnable { return (monotonicNow() - lastHeartbeatTime)/1000; } + private long getLastHeartbeatResponseTime() { + return (monotonicNow() - lastHeartbeatResponseTime) / 1000; + } + long getLastBlockReportTime() { return (monotonicNow() - lastBlockReportTime)/1000; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index c42abda72bc..d8149b6f3e4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -3621,8 +3621,12 @@ public class DataNode extends ReconfigurableBase */ @Override // DataNodeMXBean public String getBPServiceActorInfo() { - final ArrayList> infoArray = - new ArrayList>(); + return JSON.toString(getBPServiceActorInfoMap()); + } + + @VisibleForTesting + public List> getBPServiceActorInfoMap() { + final List> infoArray = new ArrayList<>(); for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) { if (bpos != null) { for (BPServiceActor actor : bpos.getBPServiceActors()) { @@ -3630,7 +3634,7 @@ public class DataNode extends ReconfigurableBase } } } - return JSON.toString(infoArray); + return infoArray; } /** @@ -3825,6 +3829,29 @@ public class DataNode extends ReconfigurableBase * @return true - if the data node is fully started */ public boolean isDatanodeFullyStarted() { + return isDatanodeFullyStarted(false); + } + + /** + * A datanode is considered to be fully started if all the BP threads are + * alive and all the block pools are initialized. If checkConnectionToActiveNamenode is true, + * the datanode is considered to be fully started if it is also heartbeating to + * active namenode in addition to the above-mentioned conditions. + * + * @param checkConnectionToActiveNamenode if true, performs additional check of whether datanode + * is heartbeating to active namenode. + * @return true if the datanode is fully started and also conditionally connected to active + * namenode, false otherwise. + */ + public boolean isDatanodeFullyStarted(boolean checkConnectionToActiveNamenode) { + if (checkConnectionToActiveNamenode) { + for (BPOfferService bp : blockPoolManager.getAllNamenodeThreads()) { + if (!bp.isInitialized() || !bp.isAlive() || bp.getActiveNN() == null) { + return false; + } + } + return true; + } for (BPOfferService bp : blockPoolManager.getAllNamenodeThreads()) { if (!bp.isInitialized() || !bp.isAlive()) { return false; @@ -3832,7 +3859,7 @@ public class DataNode extends ReconfigurableBase } return true; } - + @VisibleForTesting public DatanodeID getDatanodeId() { return id; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/datanode.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/datanode.html index b491d5a04e3..28cba0153c1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/datanode.html +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/datanode.html @@ -84,7 +84,8 @@ Namenode HA State Block Pool ID Actor State - Last Heartbeat + Last Heartbeat Sent + Last Heartbeat Response Last Block Report Last Block Report Size (Max Size) @@ -96,6 +97,7 @@ {BlockPoolID} {ActorState} {LastHeartbeat}s + {LastHeartbeatResponseTime}s {#helper_relative_time value="{LastBlockReport}"/} {maxBlockReportSize|fmt_bytes} ({maxDataLength|fmt_bytes}) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java index dd8bb204382..f5638871d40 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java @@ -2529,6 +2529,24 @@ public class MiniDFSCluster implements AutoCloseable { return restartDataNode(dnprop, false); } + /** + * Wait for the datanode to be fully functional i.e. all the BP service threads are alive, + * all block pools initiated and also connected to active namenode. + * + * @param dn Datanode instance. + * @param timeout Timeout in millis until when we should wait for datanode to be fully + * operational. + * @throws InterruptedException If the thread wait is interrupted. + * @throws TimeoutException If times out while awaiting the fully operational capability of + * datanode. + */ + public void waitDatanodeConnectedToActive(DataNode dn, int timeout) + throws InterruptedException, TimeoutException { + GenericTestUtils.waitFor(() -> dn.isDatanodeFullyStarted(true), + 100, timeout, "Datanode is not connected to active namenode even after " + + timeout + " ms of waiting"); + } + public void waitDatanodeFullyStarted(DataNode dn, int timeout) throws TimeoutException, InterruptedException { GenericTestUtils.waitFor(dn::isDatanodeFullyStarted, 100, timeout, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java index ea43cccbb18..ad4c892b22f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java @@ -38,7 +38,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.MiniDFSNNTopology; import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferTestCase; +import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.test.GenericTestUtils; import org.junit.Assert; @@ -294,4 +296,81 @@ public class TestDataNodeMXBean extends SaslDataTransferTestCase { if (cluster != null) {cluster.shutdown();} } } + + @Test + public void testDataNodeMXBeanLastHeartbeats() throws Exception { + Configuration conf = new Configuration(); + try (MiniDFSCluster cluster = new MiniDFSCluster + .Builder(conf) + .nnTopology(MiniDFSNNTopology.simpleHATopology(2)) + .build()) { + cluster.waitActive(); + cluster.transitionToActive(0); + cluster.transitionToStandby(1); + + DataNode datanode = cluster.getDataNodes().get(0); + + MBeanServer mbs = ManagementFactory.getPlatformMBeanServer(); + ObjectName mxbeanName = new ObjectName( + "Hadoop:service=DataNode,name=DataNodeInfo"); + + // Verify and wait until one of the BP service actor identifies active namenode as active + // and another as standby. + cluster.waitDatanodeConnectedToActive(datanode, 5000); + + // Verify that last heartbeat sent to both namenodes in last 5 sec. + assertLastHeartbeatSentTime(datanode, "LastHeartbeat"); + // Verify that last heartbeat response from both namenodes have been received within + // last 5 sec. + assertLastHeartbeatSentTime(datanode, "LastHeartbeatResponseTime"); + + + NameNode sbNameNode = cluster.getNameNode(1); + + // Stopping standby namenode + sbNameNode.stop(); + + // Verify that last heartbeat response time from one of the namenodes would stay much higher + // after stopping one namenode. + GenericTestUtils.waitFor(() -> { + List> bpServiceActorInfo = datanode.getBPServiceActorInfoMap(); + Map bpServiceActorInfo1 = bpServiceActorInfo.get(0); + Map bpServiceActorInfo2 = bpServiceActorInfo.get(1); + + long lastHeartbeatResponseTime1 = + Long.parseLong(bpServiceActorInfo1.get("LastHeartbeatResponseTime")); + long lastHeartbeatResponseTime2 = + Long.parseLong(bpServiceActorInfo2.get("LastHeartbeatResponseTime")); + + LOG.info("Last heartbeat response from namenode 1: {}", lastHeartbeatResponseTime1); + LOG.info("Last heartbeat response from namenode 2: {}", lastHeartbeatResponseTime2); + + return (lastHeartbeatResponseTime1 < 5L && lastHeartbeatResponseTime2 > 5L) || ( + lastHeartbeatResponseTime1 > 5L && lastHeartbeatResponseTime2 < 5L); + + }, 200, 15000, + "Last heartbeat response should be higher than 5s for at least one namenode"); + + // Verify that last heartbeat sent to both namenodes in last 5 sec even though + // the last heartbeat received from one of the namenodes is greater than 5 sec ago. + assertLastHeartbeatSentTime(datanode, "LastHeartbeat"); + } + } + + private static void assertLastHeartbeatSentTime(DataNode datanode, String lastHeartbeat) { + List> bpServiceActorInfo = datanode.getBPServiceActorInfoMap(); + Map bpServiceActorInfo1 = bpServiceActorInfo.get(0); + Map bpServiceActorInfo2 = bpServiceActorInfo.get(1); + + long lastHeartbeatSent1 = + Long.parseLong(bpServiceActorInfo1.get(lastHeartbeat)); + long lastHeartbeatSent2 = + Long.parseLong(bpServiceActorInfo2.get(lastHeartbeat)); + + Assert.assertTrue(lastHeartbeat + " for first bp service actor is higher than 5s", + lastHeartbeatSent1 < 5L); + Assert.assertTrue(lastHeartbeat + " for second bp service actor is higher than 5s", + lastHeartbeatSent2 < 5L); + } + } From 8714403dc72cc2df308f41404ae5707601a16ef2 Mon Sep 17 00:00:00 2001 From: gardenia Date: Tue, 7 Feb 2023 12:01:57 +0000 Subject: [PATCH 20/50] HADOOP-18621. Resource leak in CryptoOutputStream.close() (#5347) When closing we need to wrap the flush() in a try .. finally, otherwise when flush throws it will stop completion of the remainder of the close activities and in particular the close of the underlying wrapped stream object resulting in a resource leak. Contributed by Colm Dougan --- .../hadoop/crypto/CryptoOutputStream.java | 13 +++++++----- .../crypto/TestCryptoOutputStreamClosing.java | 20 +++++++++++++++++++ 2 files changed, 28 insertions(+), 5 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java index 2a1335b6e74..2892aaf85b2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java @@ -241,12 +241,15 @@ public class CryptoOutputStream extends FilterOutputStream implements return; } try { - flush(); - if (closeOutputStream) { - super.close(); - codec.close(); + try { + flush(); + } finally { + if (closeOutputStream) { + super.close(); + codec.close(); + } + freeBuffers(); } - freeBuffers(); } finally { closed = true; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoOutputStreamClosing.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoOutputStreamClosing.java index 39e4bb85880..04cdb962ac9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoOutputStreamClosing.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoOutputStreamClosing.java @@ -17,12 +17,14 @@ */ package org.apache.hadoop.crypto; +import java.io.IOException; import java.io.OutputStream; import org.apache.hadoop.conf.Configuration; import org.junit.BeforeClass; import org.junit.Test; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.Mockito.*; /** @@ -54,4 +56,22 @@ public class TestCryptoOutputStreamClosing { verify(outputStream, never()).close(); } + @Test + public void testUnderlyingOutputStreamClosedWhenExceptionClosing() throws Exception { + OutputStream outputStream = mock(OutputStream.class); + CryptoOutputStream cos = spy(new CryptoOutputStream(outputStream, codec, + new byte[16], new byte[16], 0L, true)); + + // exception while flushing during close + doThrow(new IOException("problem flushing wrapped stream")) + .when(cos).flush(); + + intercept(IOException.class, + () -> cos.close()); + + // We expect that the close of the CryptoOutputStream closes the + // wrapped OutputStream even though we got an exception + // during CryptoOutputStream::close (in the flush method) + verify(outputStream).close(); + } } From d3106426262daa2431b4d25a310ca20cf7b617a0 Mon Sep 17 00:00:00 2001 From: hchaverri <55413673+hchaverri@users.noreply.github.com> Date: Tue, 7 Feb 2023 10:45:05 -0800 Subject: [PATCH 21/50] HDFS-16895. [RBF] NamenodeHeartbeatService should use credentials of logged in user --- .../router/NamenodeHeartbeatService.java | 12 ++++++- .../router/TestRouterNamenodeHeartbeat.java | 32 +++++++++++++++++++ 2 files changed, 43 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java index b2f60d93149..86e24a0b24a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java @@ -26,6 +26,7 @@ import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.URI; +import java.security.PrivilegedExceptionAction; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -45,6 +46,7 @@ import org.apache.hadoop.hdfs.tools.DFSHAAdmin; import org.apache.hadoop.hdfs.tools.NNHAServiceTarget; import org.apache.hadoop.hdfs.web.URLConnectionFactory; import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.SecurityUtil; import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; @@ -236,7 +238,15 @@ public class NamenodeHeartbeatService extends PeriodicService { @Override public void periodicInvoke() { - updateState(); + try { + // Run using the login user credentials + SecurityUtil.doAsLoginUser((PrivilegedExceptionAction) () -> { + updateState(); + return null; + }); + } catch (IOException e) { + LOG.error("Cannot update namenode state", e); + } } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterNamenodeHeartbeat.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterNamenodeHeartbeat.java index 04b4b58bcb6..bb284c49ce3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterNamenodeHeartbeat.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterNamenodeHeartbeat.java @@ -26,6 +26,7 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_RP import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMENODES; import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMESERVICES; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; @@ -36,6 +37,7 @@ import java.util.Iterator; import java.util.List; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.router.SecurityConfUtil; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.server.federation.MockResolver; @@ -44,6 +46,7 @@ import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.NamenodeCon import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver; import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext; import org.apache.hadoop.net.MockDomainNameResolver; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.service.Service.STATE; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -318,4 +321,33 @@ public class TestRouterNamenodeHeartbeat { return conf; } + + @Test + public void testNamenodeHeartbeatWithSecurity() throws Exception { + Configuration conf = SecurityConfUtil.initSecurity(); + MiniRouterDFSCluster testCluster = null; + try { + testCluster = new MiniRouterDFSCluster(true, 1, conf); + // Start Namenodes and routers + testCluster.startCluster(conf); + testCluster.startRouters(); + + // Register Namenodes to generate a NamenodeStatusReport + testCluster.registerNamenodes(); + testCluster.waitNamenodeRegistration(); + + for (MiniRouterDFSCluster.RouterContext routerContext : testCluster.getRouters()) { + ActiveNamenodeResolver resolver = routerContext.getRouter().getNamenodeResolver(); + // Validate that NamenodeStatusReport has been registered + assertNotNull(resolver.getNamespaces()); + assertFalse(resolver.getNamespaces().isEmpty()); + } + } finally { + if (testCluster != null) { + testCluster.shutdown(); + } + UserGroupInformation.reset(); + SecurityConfUtil.destroy(); + } + } } From eb04ecd29d4a38b1b00612cbb3a56ed3a34a9572 Mon Sep 17 00:00:00 2001 From: hfutatzhanghb <1036798979@qq.com> Date: Wed, 8 Feb 2023 10:33:28 +0800 Subject: [PATCH 22/50] HDFS-16898. Make write lock fine-grain in method processCommandFromActor (#5330). Contributed by ZhangHB. Reviewed-by: zhangshuyan Reviewed-by: Viraj Jasani Signed-off-by: He Xiaoqiao --- .../hdfs/server/datanode/BPOfferService.java | 31 +++++++++++-------- .../hdfs/server/datanode/BPServiceActor.java | 2 +- 2 files changed, 19 insertions(+), 14 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java index d660970c725..fdd66cb05d3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java @@ -679,15 +679,20 @@ class BPOfferService { actor.reRegister(); return false; } - writeLock(); + boolean isActiveActor; + InetSocketAddress nnSocketAddress; + readLock(); try { - if (actor == bpServiceToActive) { - return processCommandFromActive(cmd, actor); - } else { - return processCommandFromStandby(cmd, actor); - } + isActiveActor = (actor == bpServiceToActive); + nnSocketAddress = actor.getNNSocketAddress(); } finally { - writeUnlock(); + readUnlock(); + } + + if (isActiveActor) { + return processCommandFromActive(cmd, nnSocketAddress); + } else { + return processCommandFromStandby(cmd, nnSocketAddress); } } @@ -715,7 +720,7 @@ class BPOfferService { * @throws IOException */ private boolean processCommandFromActive(DatanodeCommand cmd, - BPServiceActor actor) throws IOException { + InetSocketAddress nnSocketAddress) throws IOException { final BlockCommand bcmd = cmd instanceof BlockCommand? (BlockCommand)cmd: null; final BlockIdCommand blockIdCmd = @@ -768,7 +773,7 @@ class BPOfferService { dn.finalizeUpgradeForPool(bp); break; case DatanodeProtocol.DNA_RECOVERBLOCK: - String who = "NameNode at " + actor.getNNSocketAddress(); + String who = "NameNode at " + nnSocketAddress; dn.getBlockRecoveryWorker().recoverBlocks(who, ((BlockRecoveryCommand)cmd).getRecoveringBlocks()); break; @@ -810,11 +815,11 @@ class BPOfferService { * DNA_REGISTER which should be handled earlier itself. */ private boolean processCommandFromStandby(DatanodeCommand cmd, - BPServiceActor actor) throws IOException { + InetSocketAddress nnSocketAddress) throws IOException { switch(cmd.getAction()) { case DatanodeProtocol.DNA_ACCESSKEYUPDATE: LOG.info("DatanodeCommand action from standby NN {}: DNA_ACCESSKEYUPDATE", - actor.getNNSocketAddress()); + nnSocketAddress); if (dn.isBlockTokenEnabled) { dn.blockPoolTokenSecretManager.addKeys( getBlockPoolId(), @@ -831,11 +836,11 @@ class BPOfferService { case DatanodeProtocol.DNA_UNCACHE: case DatanodeProtocol.DNA_ERASURE_CODING_RECONSTRUCTION: LOG.warn("Got a command from standby NN {} - ignoring command: {}", - actor.getNNSocketAddress(), cmd.getAction()); + nnSocketAddress, cmd.getAction()); break; default: LOG.warn("Unknown DatanodeCommand action: {} from standby NN {}", - cmd.getAction(), actor.getNNSocketAddress()); + cmd.getAction(), nnSocketAddress); } return true; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index 35ab6193142..e9f424604b4 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -1499,7 +1499,7 @@ class BPServiceActor implements Runnable { dn.getMetrics().addNumProcessedCommands(processCommandsMs); } if (processCommandsMs > dnConf.getProcessCommandsThresholdMs()) { - LOG.info("Took {} ms to process {} commands from NN", + LOG.warn("Took {} ms to process {} commands from NN", processCommandsMs, cmds.length); } } From 7e919212c4672c99072dbecf8a1ec48fcb6de5f0 Mon Sep 17 00:00:00 2001 From: He Xiaoqiao Date: Wed, 8 Feb 2023 11:15:22 +0800 Subject: [PATCH 23/50] Revert "HDFS-16898. Make write lock fine-grain in method processCommandFromActor (#5330). Contributed by ZhangHB." This reverts commit eb04ecd29d4a38b1b00612cbb3a56ed3a34a9572. --- .../hdfs/server/datanode/BPOfferService.java | 31 ++++++++----------- .../hdfs/server/datanode/BPServiceActor.java | 2 +- 2 files changed, 14 insertions(+), 19 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java index fdd66cb05d3..d660970c725 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java @@ -679,20 +679,15 @@ class BPOfferService { actor.reRegister(); return false; } - boolean isActiveActor; - InetSocketAddress nnSocketAddress; - readLock(); + writeLock(); try { - isActiveActor = (actor == bpServiceToActive); - nnSocketAddress = actor.getNNSocketAddress(); + if (actor == bpServiceToActive) { + return processCommandFromActive(cmd, actor); + } else { + return processCommandFromStandby(cmd, actor); + } } finally { - readUnlock(); - } - - if (isActiveActor) { - return processCommandFromActive(cmd, nnSocketAddress); - } else { - return processCommandFromStandby(cmd, nnSocketAddress); + writeUnlock(); } } @@ -720,7 +715,7 @@ class BPOfferService { * @throws IOException */ private boolean processCommandFromActive(DatanodeCommand cmd, - InetSocketAddress nnSocketAddress) throws IOException { + BPServiceActor actor) throws IOException { final BlockCommand bcmd = cmd instanceof BlockCommand? (BlockCommand)cmd: null; final BlockIdCommand blockIdCmd = @@ -773,7 +768,7 @@ class BPOfferService { dn.finalizeUpgradeForPool(bp); break; case DatanodeProtocol.DNA_RECOVERBLOCK: - String who = "NameNode at " + nnSocketAddress; + String who = "NameNode at " + actor.getNNSocketAddress(); dn.getBlockRecoveryWorker().recoverBlocks(who, ((BlockRecoveryCommand)cmd).getRecoveringBlocks()); break; @@ -815,11 +810,11 @@ class BPOfferService { * DNA_REGISTER which should be handled earlier itself. */ private boolean processCommandFromStandby(DatanodeCommand cmd, - InetSocketAddress nnSocketAddress) throws IOException { + BPServiceActor actor) throws IOException { switch(cmd.getAction()) { case DatanodeProtocol.DNA_ACCESSKEYUPDATE: LOG.info("DatanodeCommand action from standby NN {}: DNA_ACCESSKEYUPDATE", - nnSocketAddress); + actor.getNNSocketAddress()); if (dn.isBlockTokenEnabled) { dn.blockPoolTokenSecretManager.addKeys( getBlockPoolId(), @@ -836,11 +831,11 @@ class BPOfferService { case DatanodeProtocol.DNA_UNCACHE: case DatanodeProtocol.DNA_ERASURE_CODING_RECONSTRUCTION: LOG.warn("Got a command from standby NN {} - ignoring command: {}", - nnSocketAddress, cmd.getAction()); + actor.getNNSocketAddress(), cmd.getAction()); break; default: LOG.warn("Unknown DatanodeCommand action: {} from standby NN {}", - cmd.getAction(), nnSocketAddress); + cmd.getAction(), actor.getNNSocketAddress()); } return true; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index e9f424604b4..35ab6193142 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -1499,7 +1499,7 @@ class BPServiceActor implements Runnable { dn.getMetrics().addNumProcessedCommands(processCommandsMs); } if (processCommandsMs > dnConf.getProcessCommandsThresholdMs()) { - LOG.warn("Took {} ms to process {} commands from NN", + LOG.info("Took {} ms to process {} commands from NN", processCommandsMs, cmds.length); } } From 3ba058a894b14c13651a52219b72eff48d07f4f4 Mon Sep 17 00:00:00 2001 From: He Xiaoqiao Date: Wed, 8 Feb 2023 11:19:07 +0800 Subject: [PATCH 24/50] HDFS-16898. Remove write lock for processCommandFromActor of DataNode to reduce impact on heartbeat (#5330). Contributed by ZhangHB. Reviewed-by: zhangshuyan Reviewed-by: Viraj Jasani Signed-off-by: He Xiaoqiao --- .../hdfs/server/datanode/BPOfferService.java | 31 +++++++++++-------- .../hdfs/server/datanode/BPServiceActor.java | 2 +- 2 files changed, 19 insertions(+), 14 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java index d660970c725..fdd66cb05d3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java @@ -679,15 +679,20 @@ class BPOfferService { actor.reRegister(); return false; } - writeLock(); + boolean isActiveActor; + InetSocketAddress nnSocketAddress; + readLock(); try { - if (actor == bpServiceToActive) { - return processCommandFromActive(cmd, actor); - } else { - return processCommandFromStandby(cmd, actor); - } + isActiveActor = (actor == bpServiceToActive); + nnSocketAddress = actor.getNNSocketAddress(); } finally { - writeUnlock(); + readUnlock(); + } + + if (isActiveActor) { + return processCommandFromActive(cmd, nnSocketAddress); + } else { + return processCommandFromStandby(cmd, nnSocketAddress); } } @@ -715,7 +720,7 @@ class BPOfferService { * @throws IOException */ private boolean processCommandFromActive(DatanodeCommand cmd, - BPServiceActor actor) throws IOException { + InetSocketAddress nnSocketAddress) throws IOException { final BlockCommand bcmd = cmd instanceof BlockCommand? (BlockCommand)cmd: null; final BlockIdCommand blockIdCmd = @@ -768,7 +773,7 @@ class BPOfferService { dn.finalizeUpgradeForPool(bp); break; case DatanodeProtocol.DNA_RECOVERBLOCK: - String who = "NameNode at " + actor.getNNSocketAddress(); + String who = "NameNode at " + nnSocketAddress; dn.getBlockRecoveryWorker().recoverBlocks(who, ((BlockRecoveryCommand)cmd).getRecoveringBlocks()); break; @@ -810,11 +815,11 @@ class BPOfferService { * DNA_REGISTER which should be handled earlier itself. */ private boolean processCommandFromStandby(DatanodeCommand cmd, - BPServiceActor actor) throws IOException { + InetSocketAddress nnSocketAddress) throws IOException { switch(cmd.getAction()) { case DatanodeProtocol.DNA_ACCESSKEYUPDATE: LOG.info("DatanodeCommand action from standby NN {}: DNA_ACCESSKEYUPDATE", - actor.getNNSocketAddress()); + nnSocketAddress); if (dn.isBlockTokenEnabled) { dn.blockPoolTokenSecretManager.addKeys( getBlockPoolId(), @@ -831,11 +836,11 @@ class BPOfferService { case DatanodeProtocol.DNA_UNCACHE: case DatanodeProtocol.DNA_ERASURE_CODING_RECONSTRUCTION: LOG.warn("Got a command from standby NN {} - ignoring command: {}", - actor.getNNSocketAddress(), cmd.getAction()); + nnSocketAddress, cmd.getAction()); break; default: LOG.warn("Unknown DatanodeCommand action: {} from standby NN {}", - cmd.getAction(), actor.getNNSocketAddress()); + cmd.getAction(), nnSocketAddress); } return true; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index 35ab6193142..e9f424604b4 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -1499,7 +1499,7 @@ class BPServiceActor implements Runnable { dn.getMetrics().addNumProcessedCommands(processCommandsMs); } if (processCommandsMs > dnConf.getProcessCommandsThresholdMs()) { - LOG.info("Took {} ms to process {} commands from NN", + LOG.warn("Took {} ms to process {} commands from NN", processCommandsMs, cmds.length); } } From 08f58ecf07ffef907b06ee4703374b7a3fc4edf6 Mon Sep 17 00:00:00 2001 From: Steve Vaughan Date: Wed, 8 Feb 2023 12:15:42 -0500 Subject: [PATCH 25/50] HDFS-16904. Close webhdfs during TestSymlinkHdfs teardown (#5342) Contributed by Steve Vaughan Jr --- .../src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java index fd81a1e23fb..650a75e5698 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java @@ -23,6 +23,7 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.net.URI; +import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSConfigKeys; @@ -99,6 +100,7 @@ abstract public class TestSymlinkHdfs extends SymlinkBaseTest { if (cluster != null) { cluster.shutdown(); } + IOUtils.closeQuietly(webhdfs); } @Test(timeout=10000) From af20841fb1f437b30d333eba0a6ef21fdb7a82ba Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Thu, 9 Feb 2023 03:48:38 +0800 Subject: [PATCH 26/50] YARN-11217. [Federation] Add dumpSchedulerLogs REST APIs for Router. (#5272) --- .../yarn/server/router/RouterMetrics.java | 31 ++++++++++ .../webapp/FederationInterceptorREST.java | 62 ++++++++++++++++++- .../yarn/server/router/TestRouterMetrics.java | 33 ++++++++++ .../MockDefaultRequestInterceptorREST.java | 20 ++++++ .../webapp/TestFederationInterceptorREST.java | 36 ++++++++++- 5 files changed, 180 insertions(+), 2 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java index 033aa076658..47396371ff4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java @@ -135,6 +135,8 @@ public final class RouterMetrics { private MutableGaugeInt numRenewDelegationTokenFailedRetrieved; @Metric("# of renewDelegationToken failed to be retrieved") private MutableGaugeInt numCancelDelegationTokenFailedRetrieved; + @Metric("# of dumpSchedulerLogs failed to be retrieved") + private MutableGaugeInt numDumpSchedulerLogsFailedRetrieved; @Metric("# of getActivities failed to be retrieved") private MutableGaugeInt numGetActivitiesFailedRetrieved; @Metric("# of getBulkActivities failed to be retrieved") @@ -241,6 +243,8 @@ public final class RouterMetrics { private MutableRate totalSucceededRenewDelegationTokenRetrieved; @Metric("Total number of successful Retrieved CancelDelegationToken and latency(ms)") private MutableRate totalSucceededCancelDelegationTokenRetrieved; + @Metric("Total number of successful Retrieved DumpSchedulerLogs and latency(ms)") + private MutableRate totalSucceededDumpSchedulerLogsRetrieved; @Metric("Total number of successful Retrieved GetActivities and latency(ms)") private MutableRate totalSucceededGetActivitiesRetrieved; @Metric("Total number of successful Retrieved GetBulkActivities and latency(ms)") @@ -303,6 +307,7 @@ public final class RouterMetrics { private MutableQuantiles getDelegationTokenLatency; private MutableQuantiles renewDelegationTokenLatency; private MutableQuantiles cancelDelegationTokenLatency; + private MutableQuantiles dumpSchedulerLogsLatency; private MutableQuantiles getActivitiesLatency; private MutableQuantiles getBulkActivitiesLatency; private MutableQuantiles getSchedulerInfoRetrievedLatency; @@ -482,6 +487,9 @@ public final class RouterMetrics { cancelDelegationTokenLatency = registry.newQuantiles("cancelDelegationTokenLatency", "latency of cancel delegation token timeouts", "ops", "latency", 10); + dumpSchedulerLogsLatency = registry.newQuantiles("dumpSchedulerLogsLatency", + "latency of dump scheduler logs timeouts", "ops", "latency", 10); + getActivitiesLatency = registry.newQuantiles("getActivitiesLatency", "latency of get activities timeouts", "ops", "latency", 10); @@ -752,6 +760,11 @@ public final class RouterMetrics { return totalSucceededCancelDelegationTokenRetrieved.lastStat().numSamples(); } + @VisibleForTesting + public long getNumSucceededDumpSchedulerLogsRetrieved() { + return totalSucceededDumpSchedulerLogsRetrieved.lastStat().numSamples(); + } + @VisibleForTesting public long getNumSucceededGetActivitiesRetrieved() { return totalSucceededGetActivitiesRetrieved.lastStat().numSamples(); @@ -1007,6 +1020,11 @@ public final class RouterMetrics { return totalSucceededCancelDelegationTokenRetrieved.lastStat().mean(); } + @VisibleForTesting + public double getLatencySucceededDumpSchedulerLogsRetrieved() { + return totalSucceededDumpSchedulerLogsRetrieved.lastStat().mean(); + } + @VisibleForTesting public double getLatencySucceededGetActivitiesRetrieved() { return totalSucceededGetActivitiesRetrieved.lastStat().mean(); @@ -1245,6 +1263,10 @@ public final class RouterMetrics { return numCancelDelegationTokenFailedRetrieved.value(); } + public int getDumpSchedulerLogsFailedRetrieved() { + return numDumpSchedulerLogsFailedRetrieved.value(); + } + public int getActivitiesFailedRetrieved() { return numGetActivitiesFailedRetrieved.value(); } @@ -1492,6 +1514,11 @@ public final class RouterMetrics { cancelDelegationTokenLatency.add(duration); } + public void succeededDumpSchedulerLogsRetrieved(long duration) { + totalSucceededDumpSchedulerLogsRetrieved.add(duration); + dumpSchedulerLogsLatency.add(duration); + } + public void succeededGetActivitiesLatencyRetrieved(long duration) { totalSucceededGetActivitiesRetrieved.add(duration); getActivitiesLatency.add(duration); @@ -1713,6 +1740,10 @@ public final class RouterMetrics { numCancelDelegationTokenFailedRetrieved.incr(); } + public void incrDumpSchedulerLogsFailedRetrieved() { + numDumpSchedulerLogsFailedRetrieved.incr(); + } + public void incrGetActivitiesFailedRetrieved() { numGetActivitiesFailedRetrieved.incr(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java index 69dba5b07e6..7cc403a492a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java @@ -1183,10 +1183,70 @@ public class FederationInterceptorREST extends AbstractRESTRequestInterceptor { throw new RuntimeException("getSchedulerInfo error."); } + /** + * This method dumps the scheduler logs for the time got in input, and it is + * reachable by using {@link RMWSConsts#SCHEDULER_LOGS}. + * + * @param time the period of time. It is a FormParam. + * @param hsr the servlet request + * @return the result of the operation + * @throws IOException when it cannot create dump log file + */ @Override public String dumpSchedulerLogs(String time, HttpServletRequest hsr) throws IOException { - throw new NotImplementedException("Code is not implemented"); + + // Step1. We will check the time parameter to + // ensure that the time parameter is not empty and greater than 0. + + if (StringUtils.isBlank(time)) { + routerMetrics.incrDumpSchedulerLogsFailedRetrieved(); + throw new IllegalArgumentException("Parameter error, the time is empty or null."); + } + + try { + int period = Integer.parseInt(time); + if (period <= 0) { + throw new IllegalArgumentException("time must be greater than 0."); + } + } catch (NumberFormatException e) { + routerMetrics.incrDumpSchedulerLogsFailedRetrieved(); + throw new IllegalArgumentException("time must be a number."); + } catch (IllegalArgumentException e) { + routerMetrics.incrDumpSchedulerLogsFailedRetrieved(); + throw e; + } + + // Step2. Call dumpSchedulerLogs of each subcluster. + try { + long startTime = clock.getTime(); + Map subClustersActive = getActiveSubclusters(); + final HttpServletRequest hsrCopy = clone(hsr); + Class[] argsClasses = new Class[]{String.class, HttpServletRequest.class}; + Object[] args = new Object[]{time, hsrCopy}; + ClientMethod remoteMethod = new ClientMethod("dumpSchedulerLogs", argsClasses, args); + Map dumpSchedulerLogsMap = invokeConcurrent( + subClustersActive.values(), remoteMethod, String.class); + StringBuilder stringBuilder = new StringBuilder(); + dumpSchedulerLogsMap.forEach((subClusterInfo, msg) -> { + SubClusterId subClusterId = subClusterInfo.getSubClusterId(); + stringBuilder.append("subClusterId" + subClusterId + " : " + msg + "; "); + }); + long stopTime = clock.getTime(); + routerMetrics.succeededDumpSchedulerLogsRetrieved(stopTime - startTime); + return stringBuilder.toString(); + } catch (IllegalArgumentException e) { + routerMetrics.incrDumpSchedulerLogsFailedRetrieved(); + RouterServerUtil.logAndThrowRunTimeException(e, + "Unable to dump SchedulerLogs by time: %s.", time); + } catch (YarnException e) { + routerMetrics.incrDumpSchedulerLogsFailedRetrieved(); + RouterServerUtil.logAndThrowRunTimeException(e, + "dumpSchedulerLogs by time = %s error .", time); + } + + routerMetrics.incrDumpSchedulerLogsFailedRetrieved(); + throw new RuntimeException("dumpSchedulerLogs Failed."); } /** diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java index c26df63c954..b86d85a94fd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java @@ -534,6 +534,11 @@ public class TestRouterMetrics { metrics.incrRenewDelegationTokenFailedRetrieved(); } + public void getDumpSchedulerLogsFailed() { + LOG.info("Mocked: failed DumpSchedulerLogs call"); + metrics.incrDumpSchedulerLogsFailedRetrieved(); + } + public void getActivitiesFailed() { LOG.info("Mocked: failed getBulkActivitie call"); metrics.incrGetActivitiesFailedRetrieved(); @@ -774,6 +779,11 @@ public class TestRouterMetrics { metrics.succeededRenewDelegationTokenRetrieved(duration); } + public void getDumpSchedulerLogsRetrieved(long duration) { + LOG.info("Mocked: successful DumpSchedulerLogs call with duration {}", duration); + metrics.succeededDumpSchedulerLogsRetrieved(duration); + } + public void getActivitiesRetrieved(long duration) { LOG.info("Mocked: successful GetActivities call with duration {}", duration); metrics.succeededGetActivitiesLatencyRetrieved(duration); @@ -1618,6 +1628,29 @@ public class TestRouterMetrics { metrics.getRenewDelegationTokenFailedRetrieved()); } + @Test + public void testDumpSchedulerLogsRetrieved() { + long totalGoodBefore = metrics.getNumSucceededDumpSchedulerLogsRetrieved(); + goodSubCluster.getDumpSchedulerLogsRetrieved(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededDumpSchedulerLogsRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededDumpSchedulerLogsRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getDumpSchedulerLogsRetrieved(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededDumpSchedulerLogsRetrieved()); + Assert.assertEquals(225, + metrics.getLatencySucceededDumpSchedulerLogsRetrieved(), ASSERT_DOUBLE_DELTA); + } + + @Test + public void testDumpSchedulerLogsRetrievedFailed() { + long totalBadBefore = metrics.getDumpSchedulerLogsFailedRetrieved(); + badSubCluster.getDumpSchedulerLogsFailed(); + Assert.assertEquals(totalBadBefore + 1, + metrics.getDumpSchedulerLogsFailedRetrieved()); + } + @Test public void testGetActivitiesRetrieved() { long totalGoodBefore = metrics.getNumSucceededGetActivitiesRetrieved(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java index 2e118d172c1..91f3a7d4cea 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java @@ -1214,6 +1214,26 @@ public class MockDefaultRequestInterceptorREST return new RMQueueAclInfo(true, user.getUserName(), ""); } + + public String dumpSchedulerLogs(String time, HttpServletRequest hsr) + throws IOException { + + int period = Integer.parseInt(time); + if (period <= 0) { + throw new BadRequestException("Period must be greater than 0"); + } + + return "Capacity scheduler logs are being created."; + } + } + + @Override + public String dumpSchedulerLogs(String time, HttpServletRequest hsr) throws IOException { + ResourceManager mockResourceManager = mock(ResourceManager.class); + Configuration conf = new YarnConfiguration(); + MockRMWebServices webSvc = new MockRMWebServices(mockResourceManager, conf, + mock(HttpServletResponse.class)); + return webSvc.dumpSchedulerLogs(time, hsr); } @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java index edaa1e26e93..b7ddda7d30b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java @@ -134,7 +134,6 @@ import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import static org.apache.hadoop.yarn.conf.YarnConfiguration.RM_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT; import static org.apache.hadoop.yarn.conf.YarnConfiguration.RM_DELEGATION_KEY_UPDATE_INTERVAL_KEY; import static org.apache.hadoop.yarn.conf.YarnConfiguration.RM_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT; @@ -1785,6 +1784,41 @@ public class TestFederationInterceptorREST extends BaseRouterWebServicesTest { Assert.assertEquals(response.getStatus(), Status.OK.getStatusCode()); } + @Test + public void testDumpSchedulerLogs() throws Exception { + HttpServletRequest mockHsr = mockHttpServletRequestByUserName("admin"); + String dumpSchedulerLogsMsg = interceptor.dumpSchedulerLogs("1", mockHsr); + + // We cannot guarantee the calling order of the sub-clusters, + // We guarantee that the returned result contains the information of each subCluster. + Assert.assertNotNull(dumpSchedulerLogsMsg); + subClusters.stream().forEach(subClusterId -> { + String subClusterMsg = + "subClusterId" + subClusterId + " : Capacity scheduler logs are being created.; "; + Assert.assertTrue(dumpSchedulerLogsMsg.contains(subClusterMsg)); + }); + } + + @Test + public void testDumpSchedulerLogsError() throws Exception { + HttpServletRequest mockHsr = mockHttpServletRequestByUserName("admin"); + + // time is empty + LambdaTestUtils.intercept(IllegalArgumentException.class, + "Parameter error, the time is empty or null.", + () -> interceptor.dumpSchedulerLogs(null, mockHsr)); + + // time is negative + LambdaTestUtils.intercept(IllegalArgumentException.class, + "time must be greater than 0.", + () -> interceptor.dumpSchedulerLogs("-1", mockHsr)); + + // time is non-numeric + LambdaTestUtils.intercept(IllegalArgumentException.class, + "time must be a number.", + () -> interceptor.dumpSchedulerLogs("abc", mockHsr)); + } + @Test public void testGetActivitiesNormal() { ActivitiesInfo activitiesInfo = interceptor.getActivities(null, "1", "DIAGNOSTIC"); From 4fcceff535552342c225066c4150f3997535a06a Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Wed, 8 Feb 2023 18:45:07 -0800 Subject: [PATCH 27/50] HADOOP-18620 Avoid using grizzly-http-* APIs (#5356) --- hadoop-common-project/hadoop-common/pom.xml | 5 - .../TestProxyUserAuthenticationFilter.java | 194 +++++++++++++++++- hadoop-project/pom.xml | 17 -- .../nodemanager/webapp/TestNMWebFilter.java | 157 +++++++++++++- .../hadoop-yarn-server-router/pom.xml | 6 - .../hadoop/yarn/server/router/TestRouter.java | 185 ++++++++++++++++- .../hadoop-yarn-server-web-proxy/pom.xml | 5 - .../webproxy/amfilter/TestAmFilter.java | 156 +++++++++++++- pom.xml | 8 + 9 files changed, 692 insertions(+), 41 deletions(-) diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml index 4391995d209..5f0302fd077 100644 --- a/hadoop-common-project/hadoop-common/pom.xml +++ b/hadoop-common-project/hadoop-common/pom.xml @@ -200,11 +200,6 @@ assertj-core test - - org.glassfish.grizzly - grizzly-http-servlet - test - commons-beanutils commons-beanutils diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authentication/server/TestProxyUserAuthenticationFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authentication/server/TestProxyUserAuthenticationFilter.java index 16c0e1eb112..978c15d8f2a 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authentication/server/TestProxyUserAuthenticationFilter.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authentication/server/TestProxyUserAuthenticationFilter.java @@ -18,21 +18,27 @@ package org.apache.hadoop.security.authentication.server; +import java.io.IOException; +import java.io.PrintWriter; import java.security.Principal; +import java.util.Collection; import java.util.Collections; import java.util.Enumeration; import java.util.HashMap; +import java.util.Locale; import java.util.Map; import javax.servlet.FilterConfig; import javax.servlet.FilterChain; import javax.servlet.ServletContext; +import javax.servlet.ServletOutputStream; import javax.servlet.ServletResponse; import javax.servlet.ServletRequest; +import javax.servlet.http.Cookie; import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; import static org.assertj.core.api.Assertions.assertThat; -import org.glassfish.grizzly.servlet.HttpServletResponseImpl; import org.junit.Test; import org.mockito.Mockito; @@ -76,8 +82,192 @@ public class TestProxyUserAuthenticationFilter { } } - private class HttpServletResponseForTest extends HttpServletResponseImpl { + private class HttpServletResponseForTest implements HttpServletResponse { + @Override + public void addCookie(Cookie cookie) { + + } + + @Override + public boolean containsHeader(String name) { + return false; + } + + @Override + public String encodeURL(String url) { + return null; + } + + @Override + public String encodeRedirectURL(String url) { + return null; + } + + @Override + public String encodeUrl(String url) { + return null; + } + + @Override + public String encodeRedirectUrl(String url) { + return null; + } + + @Override + public void sendError(int sc, String msg) throws IOException { + + } + + @Override + public void sendError(int sc) throws IOException { + + } + + @Override + public void sendRedirect(String location) throws IOException { + + } + + @Override + public void setDateHeader(String name, long date) { + + } + + @Override + public void addDateHeader(String name, long date) { + + } + + @Override + public void setHeader(String name, String value) { + + } + + @Override + public void addHeader(String name, String value) { + + } + + @Override + public void setIntHeader(String name, int value) { + + } + + @Override + public void addIntHeader(String name, int value) { + + } + + @Override + public void setStatus(int sc) { + + } + + @Override + public void setStatus(int sc, String sm) { + + } + + @Override + public int getStatus() { + return 0; + } + + @Override + public String getHeader(String name) { + return null; + } + + @Override + public Collection getHeaders(String name) { + return null; + } + + @Override + public Collection getHeaderNames() { + return null; + } + + @Override + public String getCharacterEncoding() { + return null; + } + + @Override + public String getContentType() { + return null; + } + + @Override + public ServletOutputStream getOutputStream() throws IOException { + return null; + } + + @Override + public PrintWriter getWriter() throws IOException { + return null; + } + + @Override + public void setCharacterEncoding(String charset) { + + } + + @Override + public void setContentLength(int len) { + + } + + @Override + public void setContentLengthLong(long len) { + + } + + @Override + public void setContentType(String type) { + + } + + @Override + public void setBufferSize(int size) { + + } + + @Override + public int getBufferSize() { + return 0; + } + + @Override + public void flushBuffer() throws IOException { + + } + + @Override + public void resetBuffer() { + + } + + @Override + public boolean isCommitted() { + return false; + } + + @Override + public void reset() { + + } + + @Override + public void setLocale(Locale loc) { + + } + + @Override + public Locale getLocale() { + return null; + } } diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index aada4e9b6de..73dc8eb2906 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -137,7 +137,6 @@ 1.6.20 2.0.6.1 5.2.0 - 2.2.21 2.9.0 3.2.4 4.1.77.Final @@ -1535,22 +1534,6 @@ ${jersey.version} - - org.glassfish.grizzly - grizzly-http-servlet - ${grizzly.version} - - - org.glassfish.grizzly - grizzly-http - ${grizzly.version} - - - org.glassfish.grizzly - grizzly-http-server - ${grizzly.version} - - ${leveldbjni.group} leveldbjni-all diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebFilter.java index 266c638a898..4ef988422e8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebFilter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebFilter.java @@ -27,14 +27,18 @@ import com.google.inject.Injector; import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; +import java.util.Collection; import java.util.HashMap; +import java.util.Locale; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import javax.servlet.FilterChain; import javax.servlet.ServletException; +import javax.servlet.ServletOutputStream; import javax.servlet.ServletRequest; import javax.servlet.ServletResponse; +import javax.servlet.http.Cookie; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import org.apache.hadoop.conf.Configuration; @@ -46,7 +50,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService; import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; -import org.glassfish.grizzly.servlet.HttpServletResponseImpl; import org.junit.Test; /** @@ -126,7 +129,7 @@ public class TestNMWebFilter { assertTrue(redirect.contains("start=10")); } - private class HttpServletResponseForTest extends HttpServletResponseImpl { + private class HttpServletResponseForTest implements HttpServletResponse { String redirectLocation = ""; int status; private String contentType; @@ -142,34 +145,184 @@ public class TestNMWebFilter { redirectLocation = location; } + @Override + public void setDateHeader(String name, long date) { + + } + + @Override + public void addDateHeader(String name, long date) { + + } + + @Override + public void addCookie(Cookie cookie) { + + } + + @Override + public boolean containsHeader(String name) { + return false; + } + + @Override + public String encodeURL(String url) { + return null; + } + @Override public String encodeRedirectURL(String url) { return url; } + @Override + public String encodeUrl(String url) { + return null; + } + + @Override + public String encodeRedirectUrl(String url) { + return null; + } + + @Override + public void sendError(int sc, String msg) throws IOException { + + } + + @Override + public void sendError(int sc) throws IOException { + + } + @Override public void setStatus(int status) { this.status = status; } + @Override + public void setStatus(int sc, String sm) { + + } + + @Override + public int getStatus() { + return 0; + } + @Override public void setContentType(String type) { this.contentType = type; } + @Override + public void setBufferSize(int size) { + + } + + @Override + public int getBufferSize() { + return 0; + } + + @Override + public void flushBuffer() throws IOException { + + } + + @Override + public void resetBuffer() { + + } + + @Override + public boolean isCommitted() { + return false; + } + + @Override + public void reset() { + + } + + @Override + public void setLocale(Locale loc) { + + } + + @Override + public Locale getLocale() { + return null; + } + @Override public void setHeader(String name, String value) { headers.put(name, value); } + @Override + public void addHeader(String name, String value) { + + } + + @Override + public void setIntHeader(String name, int value) { + + } + + @Override + public void addIntHeader(String name, int value) { + + } + public String getHeader(String name) { return headers.get(name); } + @Override + public Collection getHeaders(String name) { + return null; + } + + @Override + public Collection getHeaderNames() { + return null; + } + + @Override + public String getCharacterEncoding() { + return null; + } + + @Override + public String getContentType() { + return null; + } + + @Override + public ServletOutputStream getOutputStream() throws IOException { + return null; + } + @Override public PrintWriter getWriter() throws IOException { body = new StringWriter(); return new PrintWriter(body); } + + @Override + public void setCharacterEncoding(String charset) { + + } + + @Override + public void setContentLength(int len) { + + } + + @Override + public void setContentLengthLong(long len) { + + } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml index 0ba39653a5c..ec4fe86e7c2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml @@ -150,12 +150,6 @@ test-jar - - org.glassfish.grizzly - grizzly-http-servlet - test - - diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouter.java index d5501d74440..89438d72d0c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouter.java @@ -32,16 +32,21 @@ import org.apache.hadoop.yarn.webapp.WebApp; import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletHandler; import org.eclipse.jetty.webapp.WebAppContext; -import org.glassfish.grizzly.servlet.HttpServletResponseImpl; import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; import javax.servlet.FilterChain; import javax.servlet.ServletException; +import javax.servlet.ServletOutputStream; +import javax.servlet.http.Cookie; import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; import java.io.IOException; +import java.io.PrintWriter; +import java.util.Collection; import java.util.HashMap; +import java.util.Locale; import java.util.Map; /** @@ -180,20 +185,196 @@ public class TestRouter { router.stop(); } - private class HttpServletResponseForRouterTest extends HttpServletResponseImpl { + private class HttpServletResponseForRouterTest implements HttpServletResponse { private final Map headers = new HashMap<>(1); + + @Override + public void addCookie(Cookie cookie) { + + } + + @Override + public boolean containsHeader(String name) { + return false; + } + + @Override + public String encodeURL(String url) { + return null; + } + + @Override + public String encodeRedirectURL(String url) { + return null; + } + + @Override + public String encodeUrl(String url) { + return null; + } + + @Override + public String encodeRedirectUrl(String url) { + return null; + } + + @Override + public void sendError(int sc, String msg) throws IOException { + + } + + @Override + public void sendError(int sc) throws IOException { + + } + + @Override + public void sendRedirect(String location) throws IOException { + + } + + @Override + public void setDateHeader(String name, long date) { + + } + + @Override + public void addDateHeader(String name, long date) { + + } + @Override public void setHeader(String name, String value) { headers.put(name, value); } + @Override + public void addHeader(String name, String value) { + + } + + @Override + public void setIntHeader(String name, int value) { + + } + + @Override + public void addIntHeader(String name, int value) { + + } + + @Override + public void setStatus(int sc) { + + } + + @Override + public void setStatus(int sc, String sm) { + + } + + @Override + public int getStatus() { + return 0; + } + public String getHeader(String name) { return headers.get(name); } + @Override + public Collection getHeaders(String name) { + return null; + } + + @Override + public Collection getHeaderNames() { + return null; + } + public Map getHeaders() { return headers; } + + @Override + public String getCharacterEncoding() { + return null; + } + + @Override + public String getContentType() { + return null; + } + + @Override + public ServletOutputStream getOutputStream() throws IOException { + return null; + } + + @Override + public PrintWriter getWriter() throws IOException { + return null; + } + + @Override + public void setCharacterEncoding(String charset) { + + } + + @Override + public void setContentLength(int len) { + + } + + @Override + public void setContentLengthLong(long len) { + + } + + @Override + public void setContentType(String type) { + + } + + @Override + public void setBufferSize(int size) { + + } + + @Override + public int getBufferSize() { + return 0; + } + + @Override + public void flushBuffer() throws IOException { + + } + + @Override + public void resetBuffer() { + + } + + @Override + public boolean isCommitted() { + return false; + } + + @Override + public void reset() { + + } + + @Override + public void setLocale(Locale loc) { + + } + + @Override + public Locale getLocale() { + return null; + } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml index 9681dfa4c1d..15df5456810 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml @@ -99,11 +99,6 @@ jetty-server - - org.glassfish.grizzly - grizzly-http-servlet - test - org.junit.jupiter junit-jupiter-api diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilter.java index 07302e65276..f4827920335 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilter.java @@ -22,10 +22,12 @@ import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; import java.net.HttpURLConnection; +import java.util.Collection; import java.util.Collections; import java.util.Enumeration; import java.util.HashMap; import java.util.HashSet; +import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; @@ -35,6 +37,7 @@ import javax.servlet.FilterChain; import javax.servlet.FilterConfig; import javax.servlet.ServletContext; import javax.servlet.ServletException; +import javax.servlet.ServletOutputStream; import javax.servlet.ServletRequest; import javax.servlet.ServletResponse; import javax.servlet.http.Cookie; @@ -46,7 +49,6 @@ import org.eclipse.jetty.server.ServerConnector; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.util.thread.QueuedThreadPool; -import org.glassfish.grizzly.servlet.HttpServletResponseImpl; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; @@ -307,7 +309,7 @@ public class TestAmFilter { } - private class HttpServletResponseForTest extends HttpServletResponseImpl { + private class HttpServletResponseForTest implements HttpServletResponse { String redirectLocation = ""; int status; private String contentType; @@ -324,35 +326,185 @@ public class TestAmFilter { redirectLocation = location; } + @Override + public void setDateHeader(String name, long date) { + + } + + @Override + public void addDateHeader(String name, long date) { + + } + + @Override + public void addCookie(Cookie cookie) { + + } + + @Override + public boolean containsHeader(String name) { + return false; + } + + @Override + public String encodeURL(String url) { + return null; + } + @Override public String encodeRedirectURL(String url) { return url; } + @Override + public String encodeUrl(String url) { + return null; + } + + @Override + public String encodeRedirectUrl(String url) { + return null; + } + + @Override + public void sendError(int sc, String msg) throws IOException { + + } + + @Override + public void sendError(int sc) throws IOException { + + } + @Override public void setStatus(int status) { this.status = status; } + @Override + public void setStatus(int sc, String sm) { + + } + + @Override + public int getStatus() { + return 0; + } + @Override public void setContentType(String type) { this.contentType = type; } + @Override + public void setBufferSize(int size) { + + } + + @Override + public int getBufferSize() { + return 0; + } + + @Override + public void flushBuffer() throws IOException { + + } + + @Override + public void resetBuffer() { + + } + + @Override + public boolean isCommitted() { + return false; + } + + @Override + public void reset() { + + } + + @Override + public void setLocale(Locale loc) { + + } + + @Override + public Locale getLocale() { + return null; + } + @Override public void setHeader(String name, String value) { headers.put(name, value); } + @Override + public void addHeader(String name, String value) { + + } + + @Override + public void setIntHeader(String name, int value) { + + } + + @Override + public void addIntHeader(String name, int value) { + + } + public String getHeader(String name) { return headers.get(name); } + @Override + public Collection getHeaders(String name) { + return null; + } + + @Override + public Collection getHeaderNames() { + return null; + } + + @Override + public String getCharacterEncoding() { + return null; + } + + @Override + public String getContentType() { + return null; + } + + @Override + public ServletOutputStream getOutputStream() throws IOException { + return null; + } + @Override public PrintWriter getWriter() throws IOException { body = new StringWriter(); return new PrintWriter(body); } + + @Override + public void setCharacterEncoding(String charset) { + + } + + @Override + public void setContentLength(int len) { + + } + + @Override + public void setContentLengthLong(long len) { + + } } diff --git a/pom.xml b/pom.xml index 68d890a8b48..4e2887f1df4 100644 --- a/pom.xml +++ b/pom.xml @@ -280,6 +280,14 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x org.codehaus.jackson.** + + true + Use HttpServlet APIs instead + + org.glassfish.grizzly + org.glassfish.grizzly.** + + From d5c046518e9b519747d746e233ca256fbd66176d Mon Sep 17 00:00:00 2001 From: huhaiyang Date: Thu, 9 Feb 2023 10:47:57 +0800 Subject: [PATCH 28/50] HDFS-16910. Fix incorrectly initializing RandomAccessFile caused flush performance decreased for JN (#5359) --- .../hadoop/hdfs/server/namenode/EditLogFileOutputStream.java | 4 ++-- .../hadoop-hdfs/src/main/resources/hdfs-default.xml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java index c5d7ea7eb41..9dd3c24d5b4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java @@ -84,9 +84,9 @@ public class EditLogFileOutputStream extends EditLogOutputStream { doubleBuf = new EditsDoubleBuffer(size); RandomAccessFile rp; if (shouldSyncWritesAndSkipFsync) { - rp = new RandomAccessFile(name, "rw"); + rp = new RandomAccessFile(name, "rwd"); } else { - rp = new RandomAccessFile(name, "rws"); + rp = new RandomAccessFile(name, "rw"); } try { fp = new FileOutputStream(rp.getFD()); // open for append diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index 49f75aac955..5643a9b5c5e 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -2721,10 +2721,10 @@ Specifies whether to flush edit log file channel. When set, expensive FileChannel#force calls are skipped and synchronous disk writes are - enabled instead by opening the edit log file with RandomAccessFile("rws") + enabled instead by opening the edit log file with RandomAccessFile("rwd") flags. This can significantly improve the performance of edit log writes on the Windows platform. - Note that the behavior of the "rws" flags is platform and hardware specific + Note that the behavior of the "rwd" flags is platform and hardware specific and might not provide the same level of guarantees as FileChannel#force. For example, the write will skip the disk-cache on SAS and SCSI devices while it might not on SATA devices. This is an expert level setting, From 113a9e40cbf2b8303910547d8d4476af60846996 Mon Sep 17 00:00:00 2001 From: huhaiyang Date: Thu, 9 Feb 2023 21:28:34 +0800 Subject: [PATCH 29/50] HADOOP-18625. Fix method name of RPC.Builder#setnumReaders (#5301) Changes method name of RPC.Builder#setnumReaders to setNumReaders() The original method is still there, just marked deprecated. It is the one which should be used when working with older branches. Contributed by Haiyang Hu --- .../src/main/java/org/apache/hadoop/ipc/RPC.java | 16 +++++++++++++++- .../test/java/org/apache/hadoop/ipc/TestRPC.java | 2 +- .../federation/router/RouterRpcServer.java | 2 +- 3 files changed, 17 insertions(+), 3 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java index fc562b525ad..c7ca09c60ba 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java @@ -896,15 +896,29 @@ public class RPC { this.numHandlers = numHandlers; return this; } - + /** * @return Default: -1. * @param numReaders input numReaders. + * @deprecated call {@link #setNumReaders(int value)} instead. */ + @Deprecated public Builder setnumReaders(int numReaders) { this.numReaders = numReaders; return this; } + + /** + * Set the number of reader threads. + * + * @return this builder. + * @param value input numReaders. + * @since HADOOP-18625. + */ + public Builder setNumReaders(int value) { + this.numReaders = value; + return this; + } /** * @return Default: -1. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java index 084a3dbd4ae..a7727716c48 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java @@ -378,7 +378,7 @@ public class TestRPC extends TestRpcBase { assertEquals(confReaders, server.getNumReaders()); server = newServerBuilder(conf) - .setNumHandlers(1).setnumReaders(3).setQueueSizePerHandler(200) + .setNumHandlers(1).setNumReaders(3).setQueueSizePerHandler(200) .setVerbose(false).build(); assertEquals(3, server.getNumReaders()); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java index 45676cea63a..9d4c2caaa19 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java @@ -333,7 +333,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol, .setBindAddress(confRpcAddress.getHostName()) .setPort(confRpcAddress.getPort()) .setNumHandlers(handlerCount) - .setnumReaders(readerCount) + .setNumReaders(readerCount) .setQueueSizePerHandler(handlerQueueSize) .setVerbose(false) .setAlignmentContext(routerStateIdContext) From 9e4f50d8a0168b2127707bf12fdef29954530ef8 Mon Sep 17 00:00:00 2001 From: Mehakmeet Singh Date: Thu, 9 Feb 2023 21:31:09 +0530 Subject: [PATCH 30/50] HADOOP-18596. Distcp -update to use modification time while checking for file skip. (#5308) Adding toggleable support for modification time during distcp -update between two stores with incompatible checksum comparison. Contributed by: Mehakmeet Singh --- .../apache/hadoop/tools/DistCpConstants.java | 20 +++ .../hadoop/tools/mapred/CopyMapper.java | 79 +++++++++- .../apache/hadoop/tools/util/DistCpUtils.java | 30 +++- .../src/site/markdown/DistCp.md.vm | 39 ++++- .../contract/AbstractContractDistCpTest.java | 145 ++++++++++++++++++ .../tools/mapred/TestCopyCommitter.java | 9 +- 6 files changed, 297 insertions(+), 25 deletions(-) diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java index 289d552b862..6838d4f7757 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java @@ -142,6 +142,26 @@ public final class DistCpConstants { "distcp.blocks.per.chunk"; public static final String CONF_LABEL_USE_ITERATOR = "distcp.use.iterator"; + + /** + * Enabling {@code distcp -update} to use modification time of source and + * target file to check while copying same file with same size but + * different content. + * + * The check would verify if the target file is perceived as older than the + * source then it indicates that the source has been recently updated and it + * is a newer version than what was synced, so we should not skip the copy. + * {@value} + */ + public static final String CONF_LABEL_UPDATE_MOD_TIME = + "distcp.update.modification.time"; + + /** + * Default value for 'distcp.update.modification.time' configuration. + */ + public static final boolean CONF_LABEL_UPDATE_MOD_TIME_DEFAULT = + true; + /** * Constants for DistCp return code to shell / consumer of ToolRunner's run */ diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java index c1a11ef091b..ad17e574ca9 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java @@ -41,6 +41,8 @@ import org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.CopyReadException import org.apache.hadoop.tools.util.DistCpUtils; import org.apache.hadoop.util.StringUtils; +import static org.apache.hadoop.tools.DistCpConstants.CONF_LABEL_UPDATE_MOD_TIME_DEFAULT; + /** * Mapper class that executes the DistCp copy operation. * Implements the o.a.h.mapreduce.Mapper interface. @@ -74,6 +76,15 @@ public class CopyMapper extends Mapper OVERWRITE, // Overwrite the whole file } + /** + * Indicates the checksum comparison result. + */ + public enum ChecksumComparison { + TRUE, // checksum comparison is compatible and true. + FALSE, // checksum comparison is compatible and false. + INCOMPATIBLE, // checksum comparison is not compatible. + } + private static Logger LOG = LoggerFactory.getLogger(CopyMapper.class); private Configuration conf; @@ -85,6 +96,7 @@ public class CopyMapper extends Mapper private boolean append = false; private boolean verboseLog = false; private boolean directWrite = false; + private boolean useModTimeToUpdate; private EnumSet preserve = EnumSet.noneOf(FileAttribute.class); private FileSystem targetFS = null; @@ -114,6 +126,9 @@ public class CopyMapper extends Mapper PRESERVE_STATUS.getConfigLabel())); directWrite = conf.getBoolean( DistCpOptionSwitch.DIRECT_WRITE.getConfigLabel(), false); + useModTimeToUpdate = + conf.getBoolean(DistCpConstants.CONF_LABEL_UPDATE_MOD_TIME, + CONF_LABEL_UPDATE_MOD_TIME_DEFAULT); targetWorkPath = new Path(conf.get(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH)); Path targetFinalPath = new Path(conf.get( @@ -354,13 +369,65 @@ public class CopyMapper extends Mapper boolean sameLength = target.getLen() == source.getLen(); boolean sameBlockSize = source.getBlockSize() == target.getBlockSize() || !preserve.contains(FileAttribute.BLOCKSIZE); - if (sameLength && sameBlockSize) { - return skipCrc || - DistCpUtils.checksumsAreEqual(sourceFS, source.getPath(), null, - targetFS, target.getPath(), source.getLen()); - } else { - return false; + // Skip the copy if a 0 size file is being copied. + if (sameLength && source.getLen() == 0) { + return true; } + // If the src and target file have same size and block size, we would + // check if the checkCrc flag is enabled or not. If enabled, and the + // modTime comparison is enabled then return true if target file is older + // than the source file, since this indicates that the target file is + // recently updated and the source is not changed more recently than the + // update, we can skip the copy else we would copy. + // If skipCrc flag is disabled, we would check the checksum comparison + // which is an enum representing 3 values, of which if the comparison + // returns NOT_COMPATIBLE, we'll try to check modtime again, else return + // the result of checksum comparison which are compatible(true or false). + // + // Note: Different object stores can have different checksum algorithms + // resulting in no checksum comparison that results in return true + // always, having the modification time enabled can help in these + // scenarios to not incorrectly skip a copy. Refer: HADOOP-18596. + + if (sameLength && sameBlockSize) { + if (skipCrc) { + return maybeUseModTimeToCompare(source, target); + } else { + ChecksumComparison checksumComparison = DistCpUtils + .checksumsAreEqual(sourceFS, source.getPath(), null, + targetFS, target.getPath(), source.getLen()); + LOG.debug("Result of checksum comparison between src {} and target " + + "{} : {}", source, target, checksumComparison); + if (checksumComparison.equals(ChecksumComparison.INCOMPATIBLE)) { + return maybeUseModTimeToCompare(source, target); + } + // if skipCrc is disabled and checksumComparison is compatible we + // need not check the mod time. + return checksumComparison.equals(ChecksumComparison.TRUE); + } + } + return false; + } + + /** + * If the mod time comparison is enabled, check the mod time else return + * false. + * Comparison: If the target file perceives to have greater or equal mod time + * (older) than the source file, we can assume that there has been no new + * changes that occurred in the source file, hence we should return true to + * skip the copy of the file. + * + * @param source Source fileStatus. + * @param target Target fileStatus. + * @return boolean representing result of modTime check. + */ + private boolean maybeUseModTimeToCompare( + CopyListingFileStatus source, FileStatus target) { + if (useModTimeToUpdate) { + return source.getModificationTime() <= target.getModificationTime(); + } + // if we cannot check mod time, return true (skip the copy). + return true; } @Override diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java index 1af434e19f8..e77b2031a76 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java @@ -41,6 +41,7 @@ import org.apache.hadoop.tools.CopyListing.XAttrsNotSupportedException; import org.apache.hadoop.tools.CopyListingFileStatus; import org.apache.hadoop.tools.DistCpContext; import org.apache.hadoop.tools.DistCpOptions.FileAttribute; +import org.apache.hadoop.tools.mapred.CopyMapper; import org.apache.hadoop.tools.mapred.UniformSizeInputFormat; import org.apache.hadoop.util.StringUtils; @@ -568,10 +569,12 @@ public class DistCpUtils { * and false otherwise. * @throws IOException if there's an exception while retrieving checksums. */ - public static boolean checksumsAreEqual(FileSystem sourceFS, Path source, - FileChecksum sourceChecksum, - FileSystem targetFS, - Path target, long sourceLen) + public static CopyMapper.ChecksumComparison checksumsAreEqual( + FileSystem sourceFS, + Path source, + FileChecksum sourceChecksum, + FileSystem targetFS, + Path target, long sourceLen) throws IOException { FileChecksum targetChecksum = null; try { @@ -585,8 +588,15 @@ public class DistCpUtils { } catch (IOException e) { LOG.error("Unable to retrieve checksum for " + source + " or " + target, e); } - return (sourceChecksum == null || targetChecksum == null || - sourceChecksum.equals(targetChecksum)); + // If the source or target checksum is null, that means there is no + // comparison that took place and return not compatible. + // else if matched, return compatible with the matched result. + if (sourceChecksum == null || targetChecksum == null) { + return CopyMapper.ChecksumComparison.INCOMPATIBLE; + } else if (sourceChecksum.equals(targetChecksum)) { + return CopyMapper.ChecksumComparison.TRUE; + } + return CopyMapper.ChecksumComparison.FALSE; } /** @@ -613,8 +623,12 @@ public class DistCpUtils { //At this point, src & dest lengths are same. if length==0, we skip checksum if ((srcLen != 0) && (!skipCrc)) { - if (!checksumsAreEqual(sourceFS, source, sourceChecksum, - targetFS, target, srcLen)) { + CopyMapper.ChecksumComparison + checksumComparison = checksumsAreEqual(sourceFS, source, sourceChecksum, + targetFS, target, srcLen); + // If Checksum comparison is false set it to false, else set to true. + boolean checksumResult = !checksumComparison.equals(CopyMapper.ChecksumComparison.FALSE); + if (!checksumResult) { StringBuilder errorMessage = new StringBuilder(DistCpConstants.CHECKSUM_MISMATCH_ERROR_MSG) .append(source).append(" and ").append(target).append("."); diff --git a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm index a86e41c6668..2d77619d5cf 100644 --- a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm +++ b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm @@ -631,14 +631,37 @@ hadoop distcp -update -numListstatusThreads 20 \ Because object stores are slow to list files, consider setting the `-numListstatusThreads` option when performing a `-update` operation on a large directory tree (the limit is 40 threads). -When `DistCp -update` is used with object stores, -generally only the modification time and length of the individual files are compared, -not any checksums. The fact that most object stores do have valid timestamps -for directories is irrelevant; only the file timestamps are compared. -However, it is important to have the clock of the client computers close -to that of the infrastructure, so that timestamps are consistent between -the client/HDFS cluster and that of the object store. Otherwise, changed files may be -missed/copied too often. +When `DistCp -update` is used with object stores, generally only the +modification time and length of the individual files are compared, not any +checksums if the checksum algorithm between the two stores is different. + +* The `distcp -update` between two object stores with different checksum + algorithm compares the modification times of source and target files along + with the file size to determine whether to skip the file copy. The behavior + is controlled by the property `distcp.update.modification.time`, which is + set to true by default. If the source file is more recently modified than + the target file, it is assumed that the content has changed, and the file + should be updated. + We need to ensure that there is no clock skew between the machines. + The fact that most object stores do have valid timestamps for directories + is irrelevant; only the file timestamps are compared. However, it is + important to have the clock of the client computers close to that of the + infrastructure, so that timestamps are consistent between the client/HDFS + cluster and that of the object store. Otherwise, changed files may be + missed/copied too often. + +* `distcp.update.modification.time` would only be used if either of the two + stores don't have checksum validation resulting in incompatible checksum + comparison between the two. Even if the property is set to true, it won't + be used if their is valid checksum comparison between the two stores. + +To turn off the modification time check, set this in your core-site.xml +```xml + + distcp.update.modification.time + false + +``` **Notes** diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java index 8545df30bac..532abc2aa40 100644 --- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java +++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java @@ -29,6 +29,7 @@ import java.util.HashMap; import java.util.Map; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; @@ -72,6 +73,9 @@ public abstract class AbstractContractDistCpTest private static final Logger LOG = LoggerFactory.getLogger(AbstractContractDistCpTest.class); + /** Using offset to change modification time in tests. */ + private static final long MODIFICATION_TIME_OFFSET = 10000; + public static final String SCALE_TEST_DISTCP_FILE_SIZE_KB = "scale.test.distcp.file.size.kb"; @@ -354,6 +358,29 @@ public abstract class AbstractContractDistCpTest .withOverwrite(false))); } + /** + * Run distcp -update srcDir destDir. + * @param srcDir local source directory + * @param destDir remote destination directory. + * @return the completed job + * @throws Exception any failure. + */ + private Job distCpUpdateWithFs(final Path srcDir, final Path destDir, + FileSystem sourceFs, FileSystem targetFs) + throws Exception { + describe("\nDistcp -update from " + srcDir + " to " + destDir); + lsR("Source Fs to update", sourceFs, srcDir); + lsR("Target Fs before update", targetFs, destDir); + return runDistCp(buildWithStandardOptions( + new DistCpOptions.Builder( + Collections.singletonList(srcDir), destDir) + .withDeleteMissing(true) + .withSyncFolder(true) + .withSkipCRC(false) + .withDirectWrite(shouldUseDirectWrite()) + .withOverwrite(false))); + } + /** * Update the source directories as various tests expect, * including adding a new file. @@ -857,4 +884,122 @@ public abstract class AbstractContractDistCpTest verifyFileContents(localFS, dest, block); } + @Test + public void testDistCpUpdateCheckFileSkip() throws Exception { + describe("Distcp update to check file skips."); + + Path source = new Path(remoteDir, "file"); + Path dest = new Path(localDir, "file"); + + Path source0byte = new Path(remoteDir, "file_0byte"); + Path dest0byte = new Path(localDir, "file_0byte"); + dest = localFS.makeQualified(dest); + dest0byte = localFS.makeQualified(dest0byte); + + // Creating a source file with certain dataset. + byte[] sourceBlock = dataset(10, 'a', 'z'); + + // Write the dataset and as well create the target path. + ContractTestUtils.createFile(localFS, dest, true, sourceBlock); + ContractTestUtils + .writeDataset(remoteFS, source, sourceBlock, sourceBlock.length, + 1024, true); + + // Create 0 byte source and target files. + ContractTestUtils.createFile(remoteFS, source0byte, true, new byte[0]); + ContractTestUtils.createFile(localFS, dest0byte, true, new byte[0]); + + // Execute the distcp -update job. + Job job = distCpUpdateWithFs(remoteDir, localDir, remoteFS, localFS); + + // First distcp -update would normally copy the source to dest. + verifyFileContents(localFS, dest, sourceBlock); + // Verify 1 file was skipped in the distcp -update (The 0 byte file). + // Verify 1 file was copied in the distcp -update (The new source file). + verifySkipAndCopyCounter(job, 1, 1); + + // Remove the source file and replace with a file with same name and size + // but different content. + remoteFS.delete(source, false); + Path updatedSource = new Path(remoteDir, "file"); + byte[] updatedSourceBlock = dataset(10, 'b', 'z'); + ContractTestUtils.writeDataset(remoteFS, updatedSource, + updatedSourceBlock, updatedSourceBlock.length, 1024, true); + + // For testing purposes we would take the modification time of the + // updated Source file and add an offset or subtract the offset and set + // that time as the modification time for target file, this way we can + // ensure that our test can emulate a scenario where source is either more + // recently changed after -update so that copy takes place or target file + // is more recently changed which would skip the copying since the source + // has not been recently updated. + FileStatus fsSourceUpd = remoteFS.getFileStatus(updatedSource); + long modTimeSourceUpd = fsSourceUpd.getModificationTime(); + + // Add by an offset which would ensure enough gap for the test to + // not fail due to race conditions. + long newTargetModTimeNew = modTimeSourceUpd + MODIFICATION_TIME_OFFSET; + localFS.setTimes(dest, newTargetModTimeNew, -1); + + // Execute the distcp -update job. + Job updatedSourceJobOldSrc = + distCpUpdateWithFs(remoteDir, localDir, remoteFS, + localFS); + + // File contents should remain same since the mod time for target is + // newer than the updatedSource which indicates that the sync happened + // more recently and there is no update. + verifyFileContents(localFS, dest, sourceBlock); + // Skipped both 0 byte file and sourceFile (since mod time of target is + // older than the source it is perceived that source is of older version + // and we can skip it's copy). + verifySkipAndCopyCounter(updatedSourceJobOldSrc, 2, 0); + + // Subtract by an offset which would ensure enough gap for the test to + // not fail due to race conditions. + long newTargetModTimeOld = + Math.min(modTimeSourceUpd - MODIFICATION_TIME_OFFSET, 0); + localFS.setTimes(dest, newTargetModTimeOld, -1); + + // Execute the distcp -update job. + Job updatedSourceJobNewSrc = distCpUpdateWithFs(remoteDir, localDir, + remoteFS, + localFS); + + // Verifying the target directory have both 0 byte file and the content + // file. + Assertions + .assertThat(RemoteIterators.toList(localFS.listFiles(localDir, true))) + .hasSize(2); + // Now the copy should take place and the file contents should change + // since the mod time for target is older than the source file indicating + // that there was an update to the source after the last sync took place. + verifyFileContents(localFS, dest, updatedSourceBlock); + // Verifying we skipped the 0 byte file and copied the updated source + // file (since the modification time of the new source is older than the + // target now). + verifySkipAndCopyCounter(updatedSourceJobNewSrc, 1, 1); + } + + /** + * Method to check the skipped and copied counters of a distcp job. + * + * @param job job to check. + * @param skipExpectedValue expected skip counter value. + * @param copyExpectedValue expected copy counter value. + * @throws IOException throw in case of failures. + */ + private void verifySkipAndCopyCounter(Job job, + int skipExpectedValue, int copyExpectedValue) throws IOException { + // get the skip and copy counters from the job. + long skipActualValue = job.getCounters() + .findCounter(CopyMapper.Counter.SKIP).getValue(); + long copyActualValue = job.getCounters() + .findCounter(CopyMapper.Counter.COPY).getValue(); + // Verify if the actual values equals the expected ones. + assertEquals("Mismatch in COPY counter value", copyExpectedValue, + copyActualValue); + assertEquals("Mismatch in SKIP counter value", skipExpectedValue, + skipActualValue); + } } \ No newline at end of file diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyCommitter.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyCommitter.java index bda80a3d25e..f2dd246db5a 100644 --- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyCommitter.java +++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyCommitter.java @@ -563,9 +563,12 @@ public class TestCopyCommitter { Path sourcePath = new Path(sourceBase + srcFilename); CopyListingFileStatus sourceCurrStatus = new CopyListingFileStatus(fs.getFileStatus(sourcePath)); - Assert.assertFalse(DistCpUtils.checksumsAreEqual( - fs, new Path(sourceBase + srcFilename), null, - fs, new Path(targetBase + srcFilename), sourceCurrStatus.getLen())); + Assert.assertEquals("Checksum should not be equal", + CopyMapper.ChecksumComparison.FALSE, + DistCpUtils.checksumsAreEqual( + fs, new Path(sourceBase + srcFilename), null, + fs, new Path(targetBase + srcFilename), + sourceCurrStatus.getLen())); } catch(IOException exception) { if (skipCrc) { LOG.error("Unexpected exception is found", exception); From 17c8cdf63c27425633f96031fca325abb87f7ddf Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Thu, 9 Feb 2023 11:03:20 -0800 Subject: [PATCH 31/50] HDFS-16907. ADDENDUM: Remove unused variables from testDataNodeMXBeanLastHeartbeats. (#5373) Signed-off-by: Ayush Saxena --- .../hadoop/hdfs/server/datanode/TestDataNodeMXBean.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java index ad4c892b22f..28330139bc4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java @@ -310,10 +310,6 @@ public class TestDataNodeMXBean extends SaslDataTransferTestCase { DataNode datanode = cluster.getDataNodes().get(0); - MBeanServer mbs = ManagementFactory.getPlatformMBeanServer(); - ObjectName mxbeanName = new ObjectName( - "Hadoop:service=DataNode,name=DataNodeInfo"); - // Verify and wait until one of the BP service actor identifies active namenode as active // and another as standby. cluster.waitDatanodeConnectedToActive(datanode, 5000); From 151b71d7affbbaadab5af7943f824f6ae6a6f47b Mon Sep 17 00:00:00 2001 From: Tamas Domok Date: Fri, 10 Feb 2023 17:40:21 +0100 Subject: [PATCH 32/50] MAPREDUCE-7431. ShuffleHandler refactor and fix after Netty4 upgrade. (#5311) --- .../hadoop-mapreduce-client-shuffle/pom.xml | 6 + .../hadoop/mapred/ShuffleChannelHandler.java | 715 +++++++ .../mapred/ShuffleChannelHandlerContext.java | 140 ++ .../mapred/ShuffleChannelInitializer.java | 74 + .../apache/hadoop/mapred/ShuffleHandler.java | 1047 ++------- .../mapred/TestShuffleChannelHandler.java | 562 +++++ .../hadoop/mapred/TestShuffleHandler.java | 1871 ++--------------- .../hadoop/mapred/TestShuffleHandlerBase.java | 172 ++ .../src/test/resources/cert.pem | 27 + .../src/test/resources/key.pem | 52 + .../src/test/resources/log4j.properties | 4 +- 11 files changed, 2066 insertions(+), 2604 deletions(-) create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelHandler.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelHandlerContext.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelInitializer.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleChannelHandler.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandlerBase.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/cert.pem create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/key.pem diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/pom.xml index 4e24a3d25cd..7117b4d9770 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/pom.xml @@ -55,6 +55,12 @@ ${leveldbjni.group} leveldbjni-all + + ch.qos.logback + logback-classic + 1.1.2 + test + diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelHandler.java new file mode 100644 index 00000000000..49c0bb288b5 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelHandler.java @@ -0,0 +1,715 @@ +/* + * 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.mapred; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.handler.codec.TooLongFrameException; +import io.netty.handler.codec.http.DefaultFullHttpResponse; +import io.netty.handler.codec.http.DefaultHttpResponse; +import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpUtil; +import io.netty.handler.codec.http.LastHttpContent; +import io.netty.handler.codec.http.QueryStringDecoder; +import io.netty.handler.ssl.SslHandler; +import io.netty.util.CharsetUtil; + +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.net.URL; +import java.nio.channels.ClosedChannelException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicInteger; + +import javax.crypto.SecretKey; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.io.SecureIOUtils; +import org.apache.hadoop.mapreduce.security.SecureShuffleUtils; +import org.apache.hadoop.mapreduce.task.reduce.ShuffleHeader; +import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; +import org.eclipse.jetty.http.HttpHeader; + +import static io.netty.buffer.Unpooled.wrappedBuffer; +import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_TYPE; +import static io.netty.handler.codec.http.HttpMethod.GET; +import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST; +import static io.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN; +import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR; +import static io.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED; +import static io.netty.handler.codec.http.HttpResponseStatus.OK; +import static io.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED; +import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; +import static org.apache.hadoop.mapred.ShuffleHandler.AUDITLOG; +import static org.apache.hadoop.mapred.ShuffleHandler.CONNECTION_CLOSE; +import static org.apache.hadoop.mapred.ShuffleHandler.FETCH_RETRY_DELAY; +import static org.apache.hadoop.mapred.ShuffleHandler.IGNORABLE_ERROR_MESSAGE; +import static org.apache.hadoop.mapred.ShuffleHandler.RETRY_AFTER_HEADER; +import static org.apache.hadoop.mapred.ShuffleHandler.TIMEOUT_HANDLER; +import static org.apache.hadoop.mapred.ShuffleHandler.TOO_MANY_REQ_STATUS; +import static org.apache.hadoop.mapred.ShuffleHandler.LOG; + +/** + * ShuffleChannelHandler verifies the map request then servers the attempts in a http stream. + * Before each attempt a serialised ShuffleHeader object is written with the details. + * + *
+ * Example Request
+ * ===================
+ * GET /mapOutput?job=job_1111111111111_0001&reduce=0&
+ *     map=attempt_1111111111111_0001_m_000001_0,
+ *     attempt_1111111111111_0002_m_000002_0,
+ *     attempt_1111111111111_0003_m_000003_0 HTTP/1.1
+ * name: mapreduce
+ * version: 1.0.0
+ * UrlHash: 9zS++qE0/7/D2l1Rg0TqRoSguAk=
+ *
+ * Example Response
+ * ===================
+ * HTTP/1.1 200 OK
+ * ReplyHash: GcuojWkAxXUyhZHPnwoV/MW2tGA=
+ * name: mapreduce
+ * version: 1.0.0
+ * connection: close
+ * content-length: 138
+ *
+ * +--------+-------------------------------------------------+----------------+
+ * |00000000| 25 61 74 74 65 6d 70 74 5f 31 31 31 31 31 31 31 |%attempt_1111111|
+ * |00000010| 31 31 31 31 31 31 5f 30 30 30 31 5f 6d 5f 30 30 |111111_0001_m_00|
+ * |00000020| 30 30 30 31 5f 30 05 0a 00                      |0001_0...       |
+ * +--------+-------------------------------------------------+----------------+
+ * |00000000| 61 61 61 61 61                                  |aaaaa           |
+ * +--------+-------------------------------------------------+----------------+
+ * |00000000| 25 61 74 74 65 6d 70 74 5f 31 31 31 31 31 31 31 |%attempt_1111111|
+ * |00000010| 31 31 31 31 31 31 5f 30 30 30 32 5f 6d 5f 30 30 |111111_0002_m_00|
+ * |00000020| 30 30 30 32 5f 30 05 0a 00                      |0002_0...       |
+ * +--------+-------------------------------------------------+----------------+
+ * |00000000| 62 62 62 62 62                                  |bbbbb           |
+ * +--------+-------------------------------------------------+----------------+
+ * |00000000| 25 61 74 74 65 6d 70 74 5f 31 31 31 31 31 31 31 |%attempt_1111111|
+ * |00000010| 31 31 31 31 31 31 5f 30 30 30 33 5f 6d 5f 30 30 |111111_0003_m_00|
+ * |00000020| 30 30 30 33 5f 30 05 0a 00                      |0003_0...       |
+ * +--------+-------------------------------------------------+----------------+
+ * |00000000| 63 63 63 63 63                                  |ccccc           |
+ * +--------+-------------------------------------------------+----------------+
+ * 
+ */ +public class ShuffleChannelHandler extends SimpleChannelInboundHandler { + private final ShuffleChannelHandlerContext handlerCtx; + + ShuffleChannelHandler(ShuffleChannelHandlerContext ctx) { + handlerCtx = ctx; + } + + private List splitMaps(List mapq) { + if (null == mapq) { + return null; + } + final List ret = new ArrayList<>(); + for (String s : mapq) { + Collections.addAll(ret, s.split(",")); + } + return ret; + } + + @Override + public void channelActive(ChannelHandlerContext ctx) + throws Exception { + LOG.debug("Executing channelActive; channel='{}'", ctx.channel().id()); + int numConnections = handlerCtx.activeConnections.incrementAndGet(); + if ((handlerCtx.maxShuffleConnections > 0) && + (numConnections > handlerCtx.maxShuffleConnections)) { + LOG.info(String.format("Current number of shuffle connections (%d) is " + + "greater than the max allowed shuffle connections (%d)", + handlerCtx.allChannels.size(), handlerCtx.maxShuffleConnections)); + + Map headers = new HashMap<>(1); + // notify fetchers to backoff for a while before closing the connection + // if the shuffle connection limit is hit. Fetchers are expected to + // handle this notification gracefully, that is, not treating this as a + // fetch failure. + headers.put(RETRY_AFTER_HEADER, String.valueOf(FETCH_RETRY_DELAY)); + sendError(ctx, "", TOO_MANY_REQ_STATUS, headers); + } else { + super.channelActive(ctx); + handlerCtx.allChannels.add(ctx.channel()); + LOG.debug("Added channel: {}, channel id: {}. Accepted number of connections={}", + ctx.channel(), ctx.channel().id(), handlerCtx.activeConnections.get()); + } + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + LOG.debug("Executing channelInactive; channel='{}'", ctx.channel().id()); + super.channelInactive(ctx); + int noOfConnections = handlerCtx.activeConnections.decrementAndGet(); + LOG.debug("New value of Accepted number of connections={}", noOfConnections); + } + + @Override + public void channelRead0(ChannelHandlerContext ctx, FullHttpRequest request) { + Channel channel = ctx.channel(); + LOG.debug("Received HTTP request: {}, channel='{}'", request, channel.id()); + + if (request.method() != GET) { + sendError(ctx, METHOD_NOT_ALLOWED); + return; + } + // Check whether the shuffle version is compatible + String shuffleVersion = ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION; + String httpHeaderName = ShuffleHeader.DEFAULT_HTTP_HEADER_NAME; + if (request.headers() != null) { + shuffleVersion = request.headers().get(ShuffleHeader.HTTP_HEADER_VERSION); + httpHeaderName = request.headers().get(ShuffleHeader.HTTP_HEADER_NAME); + LOG.debug("Received from request header: ShuffleVersion={} header name={}, channel id: {}", + shuffleVersion, httpHeaderName, channel.id()); + } + if (request.headers() == null || + !ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(httpHeaderName) || + !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(shuffleVersion)) { + sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST); + return; + } + final Map> q = + new QueryStringDecoder(request.uri()).parameters(); + + final List keepAliveList = q.get("keepAlive"); + boolean keepAliveParam = false; + if (keepAliveList != null && keepAliveList.size() == 1) { + keepAliveParam = Boolean.parseBoolean(keepAliveList.get(0)); + if (LOG.isDebugEnabled()) { + LOG.debug("KeepAliveParam: {} : {}, channel id: {}", + keepAliveList, keepAliveParam, channel.id()); + } + } + final List mapIds = splitMaps(q.get("map")); + final List reduceQ = q.get("reduce"); + final List jobQ = q.get("job"); + if (LOG.isDebugEnabled()) { + LOG.debug("RECV: " + request.uri() + + "\n mapId: " + mapIds + + "\n reduceId: " + reduceQ + + "\n jobId: " + jobQ + + "\n keepAlive: " + keepAliveParam + + "\n channel id: " + channel.id()); + } + + if (mapIds == null || reduceQ == null || jobQ == null) { + sendError(ctx, "Required param job, map and reduce", BAD_REQUEST); + return; + } + if (reduceQ.size() != 1 || jobQ.size() != 1) { + sendError(ctx, "Too many job/reduce parameters", BAD_REQUEST); + return; + } + + int reduceId; + String jobId; + try { + reduceId = Integer.parseInt(reduceQ.get(0)); + jobId = jobQ.get(0); + } catch (NumberFormatException e) { + sendError(ctx, "Bad reduce parameter", BAD_REQUEST); + return; + } catch (IllegalArgumentException e) { + sendError(ctx, "Bad job parameter", BAD_REQUEST); + return; + } + final String reqUri = request.uri(); + if (null == reqUri) { + // TODO? add upstream? + sendError(ctx, FORBIDDEN); + return; + } + HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK); + try { + verifyRequest(jobId, ctx, request, response, + new URL("http", "", handlerCtx.port, reqUri)); + } catch (IOException e) { + LOG.warn("Shuffle failure ", e); + sendError(ctx, e.getMessage(), UNAUTHORIZED); + return; + } + + Map mapOutputInfoMap = new HashMap<>(); + ChannelPipeline pipeline = channel.pipeline(); + ShuffleHandler.TimeoutHandler timeoutHandler = + (ShuffleHandler.TimeoutHandler)pipeline.get(TIMEOUT_HANDLER); + timeoutHandler.setEnabledTimeout(false); + String user = handlerCtx.userRsrc.get(jobId); + + try { + populateHeaders(mapIds, jobId, user, reduceId, + response, keepAliveParam, mapOutputInfoMap); + } catch(IOException e) { + LOG.error("Shuffle error while populating headers. Channel id: " + channel.id(), e); + sendError(ctx, getErrorMessage(e), INTERNAL_SERVER_ERROR); + return; + } + + channel.write(response); + + //Initialize one ReduceContext object per channelRead call + boolean keepAlive = keepAliveParam || handlerCtx.connectionKeepAliveEnabled; + ReduceContext reduceContext = new ReduceContext(mapIds, reduceId, ctx, + user, mapOutputInfoMap, jobId, keepAlive); + + sendMap(reduceContext); + } + + /** + * Calls sendMapOutput for the mapId pointed by ReduceContext.mapsToSend + * and increments it. This method is first called by messageReceived() + * maxSessionOpenFiles times and then on the completion of every + * sendMapOutput operation. This limits the number of open files on a node, + * which can get really large(exhausting file descriptors on the NM) if all + * sendMapOutputs are called in one go, as was done previous to this change. + * @param reduceContext used to call sendMapOutput with correct params. + */ + public void sendMap(ReduceContext reduceContext) { + LOG.trace("Executing sendMap; channel='{}'", reduceContext.ctx.channel().id()); + if (reduceContext.getMapsToSend().get() < + reduceContext.getMapIds().size()) { + int nextIndex = reduceContext.getMapsToSend().getAndIncrement(); + String mapId = reduceContext.getMapIds().get(nextIndex); + + try { + MapOutputInfo info = reduceContext.getInfoMap().get(mapId); + if (info == null) { + info = getMapOutputInfo(mapId, reduceContext.getReduceId(), + reduceContext.getJobId(), reduceContext.getUser()); + } + LOG.trace("Calling sendMapOutput; channel='{}'", reduceContext.ctx.channel().id()); + ChannelFuture nextMap = sendMapOutput( + reduceContext.getCtx().channel(), + reduceContext.getUser(), mapId, + reduceContext.getReduceId(), info); + nextMap.addListener(new ReduceMapFileCount(this, reduceContext)); + } catch (IOException e) { + LOG.error("Shuffle error: {}; channel={}", e, reduceContext.ctx.channel().id()); + + // It is not possible to sendError, the success HttpResponse has been already sent + reduceContext.ctx.channel().close(); + } + } + } + + private String getErrorMessage(Throwable t) { + StringBuilder sb = new StringBuilder(t.getMessage()); + while (t.getCause() != null) { + sb.append(t.getCause().getMessage()); + t = t.getCause(); + } + return sb.toString(); + } + + protected MapOutputInfo getMapOutputInfo(String mapId, int reduce, String jobId, String user) + throws IOException { + ShuffleHandler.AttemptPathInfo pathInfo; + try { + ShuffleHandler.AttemptPathIdentifier identifier = new ShuffleHandler.AttemptPathIdentifier( + jobId, user, mapId); + pathInfo = handlerCtx.pathCache.get(identifier); + if (LOG.isDebugEnabled()) { + LOG.debug("Retrieved pathInfo for " + identifier + + " check for corresponding loaded messages to determine whether" + + " it was loaded or cached"); + } + } catch (ExecutionException e) { + if (e.getCause() instanceof IOException) { + throw (IOException) e.getCause(); + } else { + throw new RuntimeException(e.getCause()); + } + } + + IndexRecord info = + handlerCtx.indexCache.getIndexInformation(mapId, reduce, pathInfo.indexPath, user); + + if (LOG.isDebugEnabled()) { + LOG.debug("getMapOutputInfo: jobId=" + jobId + ", mapId=" + mapId + + ",dataFile=" + pathInfo.dataPath + ", indexFile=" + + pathInfo.indexPath); + LOG.debug("getMapOutputInfo: startOffset={}, partLength={} rawLength={}", + info.startOffset, info.partLength, info.rawLength); + } + + return new MapOutputInfo(pathInfo.dataPath, info); + } + + protected void populateHeaders(List mapIds, String jobId, + String user, int reduce, HttpResponse response, + boolean keepAliveParam, + Map mapOutputInfoMap) + throws IOException { + + long contentLength = 0; + for (String mapId : mapIds) { + MapOutputInfo outputInfo = getMapOutputInfo(mapId, reduce, jobId, user); + if (mapOutputInfoMap.size() < handlerCtx.mapOutputMetaInfoCacheSize) { + mapOutputInfoMap.put(mapId, outputInfo); + } + + ShuffleHeader header = + new ShuffleHeader(mapId, outputInfo.indexRecord.partLength, + outputInfo.indexRecord.rawLength, reduce); + DataOutputBuffer dob = new DataOutputBuffer(); + header.write(dob); + contentLength += outputInfo.indexRecord.partLength; + contentLength += dob.getLength(); + + // verify file access to data file to send an actually correct http error + final File spillFile = new File(outputInfo.mapOutputFileName.toString()); + RandomAccessFile r = SecureIOUtils.openForRandomRead(spillFile, "r", user, null); + r.close(); + } + + // Now set the response headers. + setResponseHeaders(response, keepAliveParam, contentLength); + + // this audit log is disabled by default, + // to turn it on please enable this audit log + // on log4j.properties by uncommenting the setting + if (AUDITLOG.isDebugEnabled()) { + StringBuilder sb = new StringBuilder("shuffle for "); + sb.append(jobId).append(" reducer ").append(reduce); + sb.append(" length ").append(contentLength); + if (AUDITLOG.isTraceEnabled()) { + // For trace level logging, append the list of mappers + sb.append(" mappers: ").append(mapIds); + AUDITLOG.trace(sb.toString()); + } else { + AUDITLOG.debug(sb.toString()); + } + } + } + + protected void setResponseHeaders(HttpResponse response, + boolean keepAliveParam, long contentLength) { + if (!handlerCtx.connectionKeepAliveEnabled && !keepAliveParam) { + response.headers().set(HttpHeader.CONNECTION.asString(), CONNECTION_CLOSE); + } else { + response.headers().set(HttpHeader.CONNECTION.asString(), + HttpHeader.KEEP_ALIVE.asString()); + response.headers().set(HttpHeader.KEEP_ALIVE.asString(), + "timeout=" + handlerCtx.connectionKeepAliveTimeOut); + } + + // Content length must be set (https://www.rfc-editor.org/rfc/rfc7230#section-3.3.3) + HttpUtil.setContentLength(response, contentLength); + } + + @SuppressWarnings("checkstyle:VisibilityModifier") + static class MapOutputInfo { + final Path mapOutputFileName; + final IndexRecord indexRecord; + + MapOutputInfo(Path mapOutputFileName, IndexRecord indexRecord) { + this.mapOutputFileName = mapOutputFileName; + this.indexRecord = indexRecord; + } + } + + protected void verifyRequest(String appid, ChannelHandlerContext ctx, + HttpRequest request, HttpResponse response, URL requestUri) + throws IOException { + SecretKey tokenSecret = handlerCtx.secretManager.retrieveTokenSecret(appid); + if (null == tokenSecret) { + LOG.info("Request for unknown token {}, channel id: {}", appid, ctx.channel().id()); + throw new IOException("Could not find jobid"); + } + // encrypting URL + String encryptedURL = SecureShuffleUtils.buildMsgFrom(requestUri); + // hash from the fetcher + String urlHashStr = + request.headers().get(SecureShuffleUtils.HTTP_HEADER_URL_HASH); + if (urlHashStr == null) { + LOG.info("Missing header hash for {}, channel id: {}", appid, ctx.channel().id()); + throw new IOException("fetcher cannot be authenticated"); + } + if (LOG.isDebugEnabled()) { + int len = urlHashStr.length(); + LOG.debug("Verifying request. encryptedURL:{}, hash:{}, channel id: " + + "{}", encryptedURL, + urlHashStr.substring(len - len / 2, len - 1), ctx.channel().id()); + } + // verify - throws exception + SecureShuffleUtils.verifyReply(urlHashStr, encryptedURL, tokenSecret); + // verification passed - encode the reply + String reply = SecureShuffleUtils.generateHash(urlHashStr.getBytes(Charsets.UTF_8), + tokenSecret); + response.headers().set( + SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply); + // Put shuffle version into http header + response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, + ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); + response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, + ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + if (LOG.isDebugEnabled()) { + int len = reply.length(); + LOG.debug("Fetcher request verified. " + + "encryptedURL: {}, reply: {}, channel id: {}", + encryptedURL, reply.substring(len - len / 2, len - 1), + ctx.channel().id()); + } + } + + public static ByteBuf shuffleHeaderToBytes(ShuffleHeader header) throws IOException { + final DataOutputBuffer dob = new DataOutputBuffer(); + header.write(dob); + return wrappedBuffer(dob.getData(), 0, dob.getLength()); + } + + protected ChannelFuture sendMapOutput(Channel ch, String user, String mapId, int reduce, + MapOutputInfo mapOutputInfo) + throws IOException { + final IndexRecord info = mapOutputInfo.indexRecord; + ch.write(shuffleHeaderToBytes( + new ShuffleHeader(mapId, info.partLength, info.rawLength, reduce))); + final File spillFile = + new File(mapOutputInfo.mapOutputFileName.toString()); + RandomAccessFile spill = SecureIOUtils.openForRandomRead(spillFile, "r", user, null); + ChannelFuture writeFuture; + if (ch.pipeline().get(SslHandler.class) == null) { + final FadvisedFileRegion partition = new FadvisedFileRegion(spill, + info.startOffset, info.partLength, handlerCtx.manageOsCache, handlerCtx.readaheadLength, + handlerCtx.readaheadPool, spillFile.getAbsolutePath(), + handlerCtx.shuffleBufferSize, handlerCtx.shuffleTransferToAllowed); + writeFuture = ch.writeAndFlush(partition); + // TODO error handling; distinguish IO/connection failures, + // attribute to appropriate spill output + writeFuture.addListener((ChannelFutureListener) future -> { + if (future.isSuccess()) { + partition.transferSuccessful(); + } + partition.deallocate(); + }); + } else { + // HTTPS cannot be done with zero copy. + final FadvisedChunkedFile chunk = new FadvisedChunkedFile(spill, + info.startOffset, info.partLength, handlerCtx.sslFileBufferSize, + handlerCtx.manageOsCache, handlerCtx.readaheadLength, handlerCtx.readaheadPool, + spillFile.getAbsolutePath()); + writeFuture = ch.writeAndFlush(chunk); + } + + handlerCtx.metrics.shuffleConnections.incr(); + handlerCtx.metrics.shuffleOutputBytes.incr(info.partLength); // optimistic + return writeFuture; + } + + protected void sendError(ChannelHandlerContext ctx, + HttpResponseStatus status) { + sendError(ctx, "", status); + } + + protected void sendError(ChannelHandlerContext ctx, String message, + HttpResponseStatus status) { + sendError(ctx, message, status, Collections.emptyMap()); + } + + protected void sendError(ChannelHandlerContext ctx, String msg, + HttpResponseStatus status, Map headers) { + FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, status, + Unpooled.copiedBuffer(msg, CharsetUtil.UTF_8)); + response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8"); + // Put shuffle version into http header + response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, + ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); + response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, + ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + for (Map.Entry header : headers.entrySet()) { + response.headers().set(header.getKey(), header.getValue()); + } + HttpUtil.setContentLength(response, response.content().readableBytes()); + + // Close the connection as soon as the error message is sent. + ctx.channel().writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); + // TODO: missing keep-alive handling + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) + throws Exception { + Channel ch = ctx.channel(); + if (cause instanceof TooLongFrameException) { + LOG.trace("TooLongFrameException, channel id: {}", ch.id()); + sendError(ctx, BAD_REQUEST); + return; + } else if (cause instanceof IOException) { + if (cause instanceof ClosedChannelException) { + LOG.debug("Ignoring closed channel error, channel id: " + ch.id(), cause); + return; + } + String message = String.valueOf(cause.getMessage()); + if (IGNORABLE_ERROR_MESSAGE.matcher(message).matches()) { + LOG.debug("Ignoring client socket close, channel id: " + ch.id(), cause); + return; + } + } + + LOG.error("Shuffle error. Channel id: " + ch.id(), cause); + if (ch.isActive()) { + sendError(ctx, INTERNAL_SERVER_ERROR); + } + } + + /** + * Maintain parameters per messageReceived() Netty context. + * Allows sendMapOutput calls from operationComplete() + */ + public static class ReduceContext { + private final List mapIds; + private final AtomicInteger mapsToWait; + private final AtomicInteger mapsToSend; + private final int reduceId; + private final ChannelHandlerContext ctx; + private final String user; + private final Map infoMap; + private final String jobId; + private final boolean keepAlive; + + ReduceContext(List mapIds, int rId, + ChannelHandlerContext context, String usr, + Map mapOutputInfoMap, + String jobId, boolean keepAlive) { + + this.mapIds = mapIds; + this.reduceId = rId; + /* + * Atomic count for tracking the no. of map outputs that are yet to + * complete. Multiple futureListeners' operationComplete() can decrement + * this value asynchronously. It is used to decide when the channel should + * be closed. + */ + this.mapsToWait = new AtomicInteger(mapIds.size()); + /* + * Atomic count for tracking the no. of map outputs that have been sent. + * Multiple sendMap() calls can increment this value + * asynchronously. Used to decide which mapId should be sent next. + */ + this.mapsToSend = new AtomicInteger(0); + this.ctx = context; + this.user = usr; + this.infoMap = mapOutputInfoMap; + this.jobId = jobId; + this.keepAlive = keepAlive; + } + + public int getReduceId() { + return reduceId; + } + + public ChannelHandlerContext getCtx() { + return ctx; + } + + public String getUser() { + return user; + } + + public Map getInfoMap() { + return infoMap; + } + + public String getJobId() { + return jobId; + } + + public List getMapIds() { + return mapIds; + } + + public AtomicInteger getMapsToSend() { + return mapsToSend; + } + + public AtomicInteger getMapsToWait() { + return mapsToWait; + } + + public boolean getKeepAlive() { + return keepAlive; + } + } + + static class ReduceMapFileCount implements ChannelFutureListener { + private final ShuffleChannelHandler handler; + private final ReduceContext reduceContext; + + ReduceMapFileCount(ShuffleChannelHandler handler, ReduceContext rc) { + this.handler = handler; + this.reduceContext = rc; + } + + @Override + public void operationComplete(ChannelFuture future) throws Exception { + LOG.trace("SendMap operation complete; mapsToWait='{}', channel='{}'", + this.reduceContext.getMapsToWait().get(), future.channel().id()); + if (!future.isSuccess()) { + LOG.error("Future is unsuccessful. channel='{}' Cause: ", + future.channel().id(), future.cause()); + future.channel().close(); + return; + } + int waitCount = this.reduceContext.getMapsToWait().decrementAndGet(); + if (waitCount == 0) { + ChannelFuture lastContentFuture = + future.channel().writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); + handler.handlerCtx.metrics.operationComplete(future); + + // Let the idle timer handler close keep-alive connections + if (reduceContext.getKeepAlive()) { + LOG.trace("SendMap operation complete, keeping alive the connection; channel='{}'", + future.channel().id()); + ChannelPipeline pipeline = future.channel().pipeline(); + ShuffleHandler.TimeoutHandler timeoutHandler = + (ShuffleHandler.TimeoutHandler)pipeline.get(TIMEOUT_HANDLER); + timeoutHandler.setEnabledTimeout(true); + } else { + LOG.trace("SendMap operation complete, closing connection; channel='{}'", + future.channel().id()); + lastContentFuture.addListener(ChannelFutureListener.CLOSE); + } + } else { + LOG.trace("SendMap operation complete, waitCount > 0, " + + "invoking sendMap with reduceContext; channel='{}'", + future.channel().id()); + handler.sendMap(reduceContext); + } + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelHandlerContext.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelHandlerContext.java new file mode 100644 index 00000000000..fa037e98e83 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelHandlerContext.java @@ -0,0 +1,140 @@ +/* + * 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.mapred; + +import io.netty.channel.group.ChannelGroup; + +import org.apache.hadoop.thirdparty.com.google.common.cache.LoadingCache; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.ReadaheadPool; +import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager; +import org.apache.hadoop.util.Shell; + +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_MAX_SHUFFLE_CONNECTIONS; +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_BUFFER_SIZE; +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED; +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT; +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_MANAGE_OS_CACHE; +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE; +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES; +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_READAHEAD_BYTES; +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED; +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE; +import static org.apache.hadoop.mapred.ShuffleHandler.MAX_SHUFFLE_CONNECTIONS; +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_BUFFER_SIZE; +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED; +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT; +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_MANAGE_OS_CACHE; +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE; +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_MAX_SESSION_OPEN_FILES; +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_READAHEAD_BYTES; +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_TRANSFERTO_ALLOWED; +import static org.apache.hadoop.mapred.ShuffleHandler.SUFFLE_SSL_FILE_BUFFER_SIZE_KEY; +import static org.apache.hadoop.mapred.ShuffleHandler.WINDOWS_DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED; + +@SuppressWarnings("checkstyle:VisibilityModifier") +public class ShuffleChannelHandlerContext { + + public final Configuration conf; + public final JobTokenSecretManager secretManager; + public final Map userRsrc; + public final LoadingCache pathCache; + public final IndexCache indexCache; + public final ShuffleHandler.ShuffleMetrics metrics; + public final ChannelGroup allChannels; + + + public final boolean connectionKeepAliveEnabled; + public final int sslFileBufferSize; + public final int connectionKeepAliveTimeOut; + public final int mapOutputMetaInfoCacheSize; + + public final AtomicInteger activeConnections = new AtomicInteger(); + + /** + * Should the shuffle use posix_fadvise calls to manage the OS cache during + * sendfile. + */ + public final boolean manageOsCache; + public final int readaheadLength; + public final int maxShuffleConnections; + public final int shuffleBufferSize; + public final boolean shuffleTransferToAllowed; + public final int maxSessionOpenFiles; + public final ReadaheadPool readaheadPool = ReadaheadPool.getInstance(); + + public int port = -1; + + public ShuffleChannelHandlerContext(Configuration conf, + Map userRsrc, + JobTokenSecretManager secretManager, + LoadingCache patCache, + IndexCache indexCache, + ShuffleHandler.ShuffleMetrics metrics, + ChannelGroup allChannels) { + this.conf = conf; + this.userRsrc = userRsrc; + this.secretManager = secretManager; + this.pathCache = patCache; + this.indexCache = indexCache; + this.metrics = metrics; + this.allChannels = allChannels; + + sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY, + DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE); + connectionKeepAliveEnabled = + conf.getBoolean(SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, + DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED); + connectionKeepAliveTimeOut = + Math.max(1, conf.getInt(SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, + DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT)); + mapOutputMetaInfoCacheSize = + Math.max(1, conf.getInt(SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE, + DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE)); + + manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE, + DEFAULT_SHUFFLE_MANAGE_OS_CACHE); + + readaheadLength = conf.getInt(SHUFFLE_READAHEAD_BYTES, + DEFAULT_SHUFFLE_READAHEAD_BYTES); + + maxShuffleConnections = conf.getInt(MAX_SHUFFLE_CONNECTIONS, + DEFAULT_MAX_SHUFFLE_CONNECTIONS); + + shuffleBufferSize = conf.getInt(SHUFFLE_BUFFER_SIZE, + DEFAULT_SHUFFLE_BUFFER_SIZE); + + shuffleTransferToAllowed = conf.getBoolean(SHUFFLE_TRANSFERTO_ALLOWED, + (Shell.WINDOWS)?WINDOWS_DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED: + DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED); + + maxSessionOpenFiles = conf.getInt(SHUFFLE_MAX_SESSION_OPEN_FILES, + DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES); + } + + void setPort(int port) { + this.port = port; + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelInitializer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelInitializer.java new file mode 100644 index 00000000000..25f01322df9 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelInitializer.java @@ -0,0 +1,74 @@ +/* + * 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.mapred; + +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.socket.SocketChannel; +import io.netty.handler.codec.http.HttpObjectAggregator; +import io.netty.handler.codec.http.HttpServerCodec; +import io.netty.handler.ssl.SslHandler; +import io.netty.handler.stream.ChunkedWriteHandler; + +import java.io.IOException; +import java.security.GeneralSecurityException; + +import org.apache.hadoop.security.ssl.SSLFactory; + +import static org.apache.hadoop.mapred.ShuffleHandler.TIMEOUT_HANDLER; +import static org.apache.hadoop.mapred.ShuffleHandler.LOG; + +public class ShuffleChannelInitializer extends ChannelInitializer { + + public static final int MAX_CONTENT_LENGTH = 1 << 16; + + private final ShuffleChannelHandlerContext handlerContext; + private final SSLFactory sslFactory; + + + public ShuffleChannelInitializer(ShuffleChannelHandlerContext ctx, SSLFactory sslFactory) { + this.handlerContext = ctx; + this.sslFactory = sslFactory; + } + + @Override + public void initChannel(SocketChannel ch) throws GeneralSecurityException, IOException { + LOG.debug("ShuffleChannelInitializer init; channel='{}'", ch.id()); + + ChannelPipeline pipeline = ch.pipeline(); + if (sslFactory != null) { + pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine())); + } + pipeline.addLast("http", new HttpServerCodec()); + pipeline.addLast("aggregator", new HttpObjectAggregator(MAX_CONTENT_LENGTH)); + pipeline.addLast("chunking", new ChunkedWriteHandler()); + + // An EventExecutorGroup could be specified to run in a + // different thread than an I/O thread so that the I/O thread + // is not blocked by a time-consuming task: + // https://netty.io/4.1/api/io/netty/channel/ChannelPipeline.html + pipeline.addLast("shuffle", new ShuffleChannelHandler(handlerContext)); + + pipeline.addLast(TIMEOUT_HANDLER, + new ShuffleHandler.TimeoutHandler(handlerContext.connectionKeepAliveTimeOut)); + // TODO factor security manager into pipeline + // TODO factor out encode/decode to permit binary shuffle + // TODO factor out decode of index to permit alt. models + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java index e4a43f85b94..2fcfbf36db6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -18,94 +18,52 @@ package org.apache.hadoop.mapred; -import static io.netty.buffer.Unpooled.wrappedBuffer; -import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_TYPE; -import static io.netty.handler.codec.http.HttpMethod.GET; -import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST; -import static io.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN; -import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR; -import static io.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED; -import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND; -import static io.netty.handler.codec.http.HttpResponseStatus.OK; -import static io.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED; -import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; -import static org.apache.hadoop.mapred.ShuffleHandler.NettyChannelHelper.*; import static org.fusesource.leveldbjni.JniDBFactory.asString; import static org.fusesource.leveldbjni.JniDBFactory.bytes; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; -import java.io.RandomAccessFile; import java.net.InetSocketAddress; -import java.net.URL; import java.nio.ByteBuffer; -import java.nio.channels.ClosedChannelException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; -import javax.crypto.SecretKey; - import io.netty.bootstrap.ServerBootstrap; -import io.netty.buffer.Unpooled; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; -import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelInboundHandlerAdapter; -import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelOption; -import io.netty.channel.ChannelOutboundHandlerAdapter; -import io.netty.channel.ChannelPipeline; -import io.netty.channel.ChannelPromise; import io.netty.channel.EventLoopGroup; import io.netty.channel.group.ChannelGroup; import io.netty.channel.group.DefaultChannelGroup; import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.channel.socket.SocketChannel; import io.netty.channel.socket.nio.NioServerSocketChannel; -import io.netty.handler.codec.TooLongFrameException; -import io.netty.handler.codec.http.DefaultFullHttpResponse; -import io.netty.handler.codec.http.DefaultHttpResponse; -import io.netty.handler.codec.http.FullHttpResponse; -import io.netty.handler.codec.http.HttpObjectAggregator; -import io.netty.handler.codec.http.HttpRequest; -import io.netty.handler.codec.http.HttpRequestDecoder; -import io.netty.handler.codec.http.HttpResponse; -import io.netty.handler.codec.http.HttpResponseEncoder; import io.netty.handler.codec.http.HttpResponseStatus; -import io.netty.handler.codec.http.LastHttpContent; -import io.netty.handler.codec.http.QueryStringDecoder; -import io.netty.handler.ssl.SslHandler; -import io.netty.handler.stream.ChunkedWriteHandler; import io.netty.handler.timeout.IdleState; import io.netty.handler.timeout.IdleStateEvent; import io.netty.handler.timeout.IdleStateHandler; -import io.netty.util.CharsetUtil; -import io.netty.util.concurrent.DefaultEventExecutorGroup; +import io.netty.util.concurrent.GlobalEventExecutor; + +import javax.annotation.Nonnull; + +import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder; +import org.apache.hadoop.thirdparty.com.google.common.cache.CacheLoader; +import org.apache.hadoop.thirdparty.com.google.common.cache.LoadingCache; +import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalListener; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataInputByteBuffer; import org.apache.hadoop.io.DataOutputBuffer; -import org.apache.hadoop.io.ReadaheadPool; -import org.apache.hadoop.io.SecureIOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.proto.ShuffleHandlerRecoveryProtos.JobShuffleInfoProto; import org.apache.hadoop.mapreduce.MRConfig; -import org.apache.hadoop.mapreduce.security.SecureShuffleUtils; import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier; import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager; -import org.apache.hadoop.mapreduce.task.reduce.ShuffleHeader; import org.apache.hadoop.metrics2.MetricsSystem; import org.apache.hadoop.metrics2.annotation.Metric; import org.apache.hadoop.metrics2.annotation.Metrics; @@ -116,8 +74,6 @@ import org.apache.hadoop.metrics2.lib.MutableGaugeInt; import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.security.ssl.SSLFactory; import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.util.DiskChecker; -import org.apache.hadoop.util.Shell; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto; import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext; @@ -132,23 +88,17 @@ import org.fusesource.leveldbjni.internal.NativeDB; import org.iq80.leveldb.DB; import org.iq80.leveldb.DBException; import org.iq80.leveldb.Options; -import org.eclipse.jetty.http.HttpHeader; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; -import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder; -import org.apache.hadoop.thirdparty.com.google.common.cache.CacheLoader; -import org.apache.hadoop.thirdparty.com.google.common.cache.LoadingCache; -import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalListener; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.thirdparty.protobuf.ByteString; public class ShuffleHandler extends AuxiliaryService { - private static final org.slf4j.Logger LOG = + public static final org.slf4j.Logger LOG = LoggerFactory.getLogger(ShuffleHandler.class); - private static final org.slf4j.Logger AUDITLOG = + public static final org.slf4j.Logger AUDITLOG = LoggerFactory.getLogger(ShuffleHandler.class.getName()+".audit"); public static final String SHUFFLE_MANAGE_OS_CACHE = "mapreduce.shuffle.manage.os.cache"; public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true; @@ -170,7 +120,7 @@ public class ShuffleHandler extends AuxiliaryService { // pattern to identify errors related to the client closing the socket early // idea borrowed from Netty SslHandler - private static final Pattern IGNORABLE_ERROR_MESSAGE = Pattern.compile( + public static final Pattern IGNORABLE_ERROR_MESSAGE = Pattern.compile( "^.*(?:connection.*reset|connection.*closed|broken.*pipe).*$", Pattern.CASE_INSENSITIVE); @@ -187,37 +137,21 @@ public class ShuffleHandler extends AuxiliaryService { // This should be kept in sync with Fetcher.FETCH_RETRY_DELAY_DEFAULT public static final long FETCH_RETRY_DELAY = 1000L; public static final String RETRY_AFTER_HEADER = "Retry-After"; - static final String ENCODER_HANDLER_NAME = "encoder"; private int port; private EventLoopGroup bossGroup; private EventLoopGroup workerGroup; - private ServerBootstrap bootstrap; - private Channel ch; - private final ChannelGroup accepted = - new DefaultChannelGroup(new DefaultEventExecutorGroup(5).next()); - private final AtomicInteger activeConnections = new AtomicInteger(); - protected HttpPipelineFactory pipelineFact; - private int sslFileBufferSize; - //TODO snemeth add a config option for these later, this is temporarily disabled for now. - private boolean useOutboundExceptionHandler = false; - private boolean useOutboundLogger = false; - - /** - * Should the shuffle use posix_fadvise calls to manage the OS cache during - * sendfile. - */ - private boolean manageOsCache; - private int readaheadLength; - private int maxShuffleConnections; - private int shuffleBufferSize; - private boolean shuffleTransferToAllowed; - private int maxSessionOpenFiles; - private ReadaheadPool readaheadPool = ReadaheadPool.getInstance(); + @SuppressWarnings("checkstyle:VisibilityModifier") + protected final ChannelGroup allChannels = + new DefaultChannelGroup(GlobalEventExecutor.INSTANCE); - private Map userRsrc; - private JobTokenSecretManager secretManager; + private SSLFactory sslFactory; + + @SuppressWarnings("checkstyle:VisibilityModifier") + protected JobTokenSecretManager secretManager; + @SuppressWarnings("checkstyle:VisibilityModifier") + protected Map userRsrc; private DB stateDb = null; @@ -276,9 +210,6 @@ public class ShuffleHandler extends AuxiliaryService { "mapreduce.shuffle.max.session-open-files"; public static final int DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES = 3; - boolean connectionKeepAliveEnabled = false; - private int connectionKeepAliveTimeOut; - private int mapOutputMetaInfoCacheSize; @Metrics(about="Shuffle output metrics", context="mapred") static class ShuffleMetrics implements ChannelFutureListener { @@ -302,170 +233,11 @@ public class ShuffleHandler extends AuxiliaryService { } } - static class NettyChannelHelper { - static ChannelFuture writeToChannel(Channel ch, Object obj) { - LOG.debug("Writing {} to channel: {}", obj.getClass().getSimpleName(), ch.id()); - return ch.writeAndFlush(obj); - } - - static ChannelFuture writeToChannelAndClose(Channel ch, Object obj) { - return writeToChannel(ch, obj).addListener(ChannelFutureListener.CLOSE); - } - - static ChannelFuture writeToChannelAndAddLastHttpContent(Channel ch, HttpResponse obj) { - writeToChannel(ch, obj); - return writeLastHttpContentToChannel(ch); - } - - static ChannelFuture writeLastHttpContentToChannel(Channel ch) { - LOG.debug("Writing LastHttpContent, channel id: {}", ch.id()); - return ch.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); - } - - static ChannelFuture closeChannel(Channel ch) { - LOG.debug("Closing channel, channel id: {}", ch.id()); - return ch.close(); - } - - static void closeChannels(ChannelGroup channelGroup) { - channelGroup.close().awaitUninterruptibly(10, TimeUnit.SECONDS); - } - - public static ChannelFuture closeAsIdle(Channel ch, int timeout) { - LOG.debug("Closing channel as writer was idle for {} seconds", timeout); - return closeChannel(ch); - } - - public static void channelActive(Channel ch) { - LOG.debug("Executing channelActive, channel id: {}", ch.id()); - } - - public static void channelInactive(Channel ch) { - LOG.debug("Executing channelInactive, channel id: {}", ch.id()); - } - } - - private final MetricsSystem ms; + @SuppressWarnings("checkstyle:VisibilityModifier") + protected final MetricsSystem ms; + @SuppressWarnings("checkstyle:VisibilityModifier") final ShuffleMetrics metrics; - class ReduceMapFileCount implements ChannelFutureListener { - - private ReduceContext reduceContext; - - ReduceMapFileCount(ReduceContext rc) { - this.reduceContext = rc; - } - - @Override - public void operationComplete(ChannelFuture future) throws Exception { - LOG.trace("operationComplete"); - if (!future.isSuccess()) { - LOG.error("Future is unsuccessful. Cause: ", future.cause()); - closeChannel(future.channel()); - return; - } - int waitCount = this.reduceContext.getMapsToWait().decrementAndGet(); - if (waitCount == 0) { - LOG.trace("Finished with all map outputs"); - //HADOOP-15327: Need to send an instance of LastHttpContent to define HTTP - //message boundaries. See details in jira. - writeLastHttpContentToChannel(future.channel()); - metrics.operationComplete(future); - // Let the idle timer handler close keep-alive connections - if (reduceContext.getKeepAlive()) { - ChannelPipeline pipeline = future.channel().pipeline(); - TimeoutHandler timeoutHandler = - (TimeoutHandler)pipeline.get(TIMEOUT_HANDLER); - timeoutHandler.setEnabledTimeout(true); - } else { - closeChannel(future.channel()); - } - } else { - LOG.trace("operationComplete, waitCount > 0, invoking sendMap with reduceContext"); - pipelineFact.getSHUFFLE().sendMap(reduceContext); - } - } - } - - /** - * Maintain parameters per messageReceived() Netty context. - * Allows sendMapOutput calls from operationComplete() - */ - private static class ReduceContext { - private List mapIds; - private AtomicInteger mapsToWait; - private AtomicInteger mapsToSend; - private int reduceId; - private ChannelHandlerContext ctx; - private String user; - private Map infoMap; - private String jobId; - private final boolean keepAlive; - - ReduceContext(List mapIds, int rId, - ChannelHandlerContext context, String usr, - Map mapOutputInfoMap, - String jobId, boolean keepAlive) { - - this.mapIds = mapIds; - this.reduceId = rId; - /** - * Atomic count for tracking the no. of map outputs that are yet to - * complete. Multiple futureListeners' operationComplete() can decrement - * this value asynchronously. It is used to decide when the channel should - * be closed. - */ - this.mapsToWait = new AtomicInteger(mapIds.size()); - /** - * Atomic count for tracking the no. of map outputs that have been sent. - * Multiple sendMap() calls can increment this value - * asynchronously. Used to decide which mapId should be sent next. - */ - this.mapsToSend = new AtomicInteger(0); - this.ctx = context; - this.user = usr; - this.infoMap = mapOutputInfoMap; - this.jobId = jobId; - this.keepAlive = keepAlive; - } - - public int getReduceId() { - return reduceId; - } - - public ChannelHandlerContext getCtx() { - return ctx; - } - - public String getUser() { - return user; - } - - public Map getInfoMap() { - return infoMap; - } - - public String getJobId() { - return jobId; - } - - public List getMapIds() { - return mapIds; - } - - public AtomicInteger getMapsToSend() { - return mapsToSend; - } - - public AtomicInteger getMapsToWait() { - return mapsToWait; - } - - public boolean getKeepAlive() { - return keepAlive; - } - } - ShuffleHandler(MetricsSystem ms) { super(MAPREDUCE_SHUFFLE_SERVICEID); this.ms = ms; @@ -480,18 +252,20 @@ public class ShuffleHandler extends AuxiliaryService { * Serialize the shuffle port into a ByteBuffer for use later on. * @param port the port to be sent to the ApplciationMaster * @return the serialized form of the port. + * @throws IOException on failure */ public static ByteBuffer serializeMetaData(int port) throws IOException { //TODO these bytes should be versioned - DataOutputBuffer port_dob = new DataOutputBuffer(); - port_dob.writeInt(port); - return ByteBuffer.wrap(port_dob.getData(), 0, port_dob.getLength()); + DataOutputBuffer portDob = new DataOutputBuffer(); + portDob.writeInt(port); + return ByteBuffer.wrap(portDob.getData(), 0, portDob.getLength()); } /** * A helper function to deserialize the metadata returned by ShuffleHandler. * @param meta the metadata returned by the ShuffleHandler * @return the port the Shuffle Handler is listening on to serve shuffle data. + * @throws IOException on failure */ public static int deserializeMetaData(ByteBuffer meta) throws IOException { //TODO this should be returning a class not just an int @@ -507,16 +281,18 @@ public class ShuffleHandler extends AuxiliaryService { * @param jobToken the job token to be used for authentication of * shuffle data requests. * @return the serialized version of the jobToken. + * @throws IOException on failure */ public static ByteBuffer serializeServiceData(Token jobToken) throws IOException { //TODO these bytes should be versioned - DataOutputBuffer jobToken_dob = new DataOutputBuffer(); - jobToken.write(jobToken_dob); - return ByteBuffer.wrap(jobToken_dob.getData(), 0, jobToken_dob.getLength()); + DataOutputBuffer jobTokenDob = new DataOutputBuffer(); + jobToken.write(jobTokenDob); + return ByteBuffer.wrap(jobTokenDob.getData(), 0, jobTokenDob.getLength()); } - static Token deserializeServiceData(ByteBuffer secret) throws IOException { + public static Token deserializeServiceData(ByteBuffer secret) + throws IOException { DataInputByteBuffer in = new DataInputByteBuffer(); in.reset(secret); Token jt = new Token(); @@ -556,14 +332,6 @@ public class ShuffleHandler extends AuxiliaryService { @Override protected void serviceInit(Configuration conf) throws Exception { - manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE, - DEFAULT_SHUFFLE_MANAGE_OS_CACHE); - - readaheadLength = conf.getInt(SHUFFLE_READAHEAD_BYTES, - DEFAULT_SHUFFLE_READAHEAD_BYTES); - - maxShuffleConnections = conf.getInt(MAX_SHUFFLE_CONNECTIONS, - DEFAULT_MAX_SHUFFLE_CONNECTIONS); int maxShuffleThreads = conf.getInt(MAX_SHUFFLE_THREADS, DEFAULT_MAX_SHUFFLE_THREADS); // Since Netty 4.x, the value of 0 threads would default to: @@ -574,16 +342,6 @@ public class ShuffleHandler extends AuxiliaryService { if (maxShuffleThreads == 0) { maxShuffleThreads = 2 * Runtime.getRuntime().availableProcessors(); } - - shuffleBufferSize = conf.getInt(SHUFFLE_BUFFER_SIZE, - DEFAULT_SHUFFLE_BUFFER_SIZE); - - shuffleTransferToAllowed = conf.getBoolean(SHUFFLE_TRANSFERTO_ALLOWED, - (Shell.WINDOWS)?WINDOWS_DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED: - DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED); - - maxSessionOpenFiles = conf.getInt(SHUFFLE_MAX_SESSION_OPEN_FILES, - DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES); ThreadFactory bossFactory = new ThreadFactoryBuilder() .setNameFormat("ShuffleHandler Netty Boss #%d") @@ -592,66 +350,117 @@ public class ShuffleHandler extends AuxiliaryService { .setNameFormat("ShuffleHandler Netty Worker #%d") .build(); - bossGroup = new NioEventLoopGroup(maxShuffleThreads, bossFactory); + bossGroup = new NioEventLoopGroup(1, bossFactory); workerGroup = new NioEventLoopGroup(maxShuffleThreads, workerFactory); super.serviceInit(new Configuration(conf)); } + protected ShuffleChannelHandlerContext createHandlerContext() { + Configuration conf = getConfig(); + + final LoadingCache pathCache = + CacheBuilder.newBuilder().expireAfterAccess( + conf.getInt(EXPIRE_AFTER_ACCESS_MINUTES, DEFAULT_EXPIRE_AFTER_ACCESS_MINUTES), + TimeUnit.MINUTES).softValues().concurrencyLevel(conf.getInt(CONCURRENCY_LEVEL, + DEFAULT_CONCURRENCY_LEVEL)). + removalListener( + (RemovalListener) notification -> { + if (LOG.isDebugEnabled()) { + LOG.debug("PathCache Eviction: " + notification.getKey() + + ", Reason=" + notification.getCause()); + } + } + ).maximumWeight(conf.getInt(MAX_WEIGHT, DEFAULT_MAX_WEIGHT)).weigher( + (key, value) -> key.jobId.length() + key.user.length() + + key.attemptId.length()+ + value.indexPath.toString().length() + + value.dataPath.toString().length() + ).build(new CacheLoader() { + @Override + public AttemptPathInfo load(@Nonnull AttemptPathIdentifier key) throws + Exception { + String base = getBaseLocation(key.jobId, key.user); + String attemptBase = base + key.attemptId; + Path indexFileName = getAuxiliaryLocalPathHandler() + .getLocalPathForRead(attemptBase + "/" + INDEX_FILE_NAME); + Path mapOutputFileName = getAuxiliaryLocalPathHandler() + .getLocalPathForRead(attemptBase + "/" + DATA_FILE_NAME); + + if (LOG.isDebugEnabled()) { + LOG.debug("Loaded : " + key + " via loader"); + } + return new AttemptPathInfo(indexFileName, mapOutputFileName); + } + }); + + return new ShuffleChannelHandlerContext(conf, + userRsrc, + secretManager, + pathCache, + new IndexCache(new JobConf(conf)), + metrics, + allChannels + ); + } + // TODO change AbstractService to throw InterruptedException @Override protected void serviceStart() throws Exception { Configuration conf = getConfig(); - userRsrc = new ConcurrentHashMap(); + userRsrc = new ConcurrentHashMap<>(); secretManager = new JobTokenSecretManager(); recoverState(conf); - try { - pipelineFact = new HttpPipelineFactory(conf); - } catch (Exception ex) { - throw new RuntimeException(ex); + + if (conf.getBoolean(MRConfig.SHUFFLE_SSL_ENABLED_KEY, + MRConfig.SHUFFLE_SSL_ENABLED_DEFAULT)) { + LOG.info("Encrypted shuffle is enabled."); + sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf); + sslFactory.init(); } - bootstrap = new ServerBootstrap(); + ShuffleChannelHandlerContext handlerContext = createHandlerContext(); + ServerBootstrap bootstrap = new ServerBootstrap(); bootstrap.group(bossGroup, workerGroup) .channel(NioServerSocketChannel.class) .option(ChannelOption.SO_BACKLOG, conf.getInt(SHUFFLE_LISTEN_QUEUE_SIZE, DEFAULT_SHUFFLE_LISTEN_QUEUE_SIZE)) .childOption(ChannelOption.SO_KEEPALIVE, true) - .childHandler(pipelineFact); + .childHandler(new ShuffleChannelInitializer( + handlerContext, + sslFactory) + ); port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT); - ch = bootstrap.bind(new InetSocketAddress(port)).sync().channel(); - accepted.add(ch); + Channel ch = bootstrap.bind(new InetSocketAddress(port)).sync().channel(); port = ((InetSocketAddress)ch.localAddress()).getPort(); + allChannels.add(ch); conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port)); - pipelineFact.SHUFFLE.setPort(port); + handlerContext.setPort(port); LOG.info(getName() + " listening on port " + port); super.serviceStart(); - - sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY, - DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE); - connectionKeepAliveEnabled = - conf.getBoolean(SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, - DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED); - connectionKeepAliveTimeOut = - Math.max(1, conf.getInt(SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, - DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT)); - mapOutputMetaInfoCacheSize = - Math.max(1, conf.getInt(SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE, - DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE)); } @Override protected void serviceStop() throws Exception { - closeChannels(accepted); + allChannels.close().awaitUninterruptibly(10, TimeUnit.SECONDS); - if (pipelineFact != null) { - pipelineFact.destroy(); + if (sslFactory != null) { + sslFactory.destroy(); } if (stateDb != null) { stateDb.close(); } ms.unregisterSource(ShuffleMetrics.class.getSimpleName()); + + if (bossGroup != null) { + bossGroup.shutdownGracefully(); + } + + if (workerGroup != null) { + workerGroup.shutdownGracefully(); + } + super.serviceStop(); } @@ -666,10 +475,6 @@ public class ShuffleHandler extends AuxiliaryService { } } - protected Shuffle getShuffle(Configuration conf) { - return new Shuffle(conf); - } - private void recoverState(Configuration conf) throws IOException { Path recoveryRoot = getRecoveryPath(); if (recoveryRoot != null) { @@ -845,11 +650,6 @@ public class ShuffleHandler extends AuxiliaryService { } } - @VisibleForTesting - public void setUseOutboundExceptionHandler(boolean useHandler) { - this.useOutboundExceptionHandler = useHandler; - } - static class TimeoutHandler extends IdleStateHandler { private final int connectionKeepAliveTimeOut; private boolean enabledTimeout; @@ -862,11 +662,6 @@ public class ShuffleHandler extends AuxiliaryService { this.connectionKeepAliveTimeOut = connectionKeepAliveTimeOut; } - @VisibleForTesting - public int getConnectionKeepAliveTimeOut() { - return connectionKeepAliveTimeOut; - } - void setEnabledTimeout(boolean enabledTimeout) { this.enabledTimeout = enabledTimeout; } @@ -874,607 +669,18 @@ public class ShuffleHandler extends AuxiliaryService { @Override public void channelIdle(ChannelHandlerContext ctx, IdleStateEvent e) { if (e.state() == IdleState.WRITER_IDLE && enabledTimeout) { - closeAsIdle(ctx.channel(), connectionKeepAliveTimeOut); - } - } - } - - class HttpPipelineFactory extends ChannelInitializer { - private static final int MAX_CONTENT_LENGTH = 1 << 16; - - final Shuffle SHUFFLE; - private SSLFactory sslFactory; - - HttpPipelineFactory(Configuration conf) throws Exception { - SHUFFLE = getShuffle(conf); - if (conf.getBoolean(MRConfig.SHUFFLE_SSL_ENABLED_KEY, - MRConfig.SHUFFLE_SSL_ENABLED_DEFAULT)) { - LOG.info("Encrypted shuffle is enabled."); - sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf); - sslFactory.init(); - } - } - - public Shuffle getSHUFFLE() { - return SHUFFLE; - } - - public void destroy() { - if (sslFactory != null) { - sslFactory.destroy(); - } - } - - @Override protected void initChannel(SocketChannel ch) throws Exception { - ChannelPipeline pipeline = ch.pipeline(); - if (sslFactory != null) { - pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine())); - } - pipeline.addLast("decoder", new HttpRequestDecoder()); - pipeline.addLast("aggregator", new HttpObjectAggregator(MAX_CONTENT_LENGTH)); - pipeline.addLast(ENCODER_HANDLER_NAME, useOutboundLogger ? - new LoggingHttpResponseEncoder(false) : new HttpResponseEncoder()); - pipeline.addLast("chunking", new ChunkedWriteHandler()); - pipeline.addLast("shuffle", SHUFFLE); - if (useOutboundExceptionHandler) { - //https://stackoverflow.com/questions/50612403/catch-all-exception-handling-for-outbound-channelhandler - pipeline.addLast("outboundExceptionHandler", new ChannelOutboundHandlerAdapter() { - @Override - public void write(ChannelHandlerContext ctx, Object msg, - ChannelPromise promise) throws Exception { - promise.addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE); - super.write(ctx, msg, promise); - } - }); - } - pipeline.addLast(TIMEOUT_HANDLER, new TimeoutHandler(connectionKeepAliveTimeOut)); - // TODO factor security manager into pipeline - // TODO factor out encode/decode to permit binary shuffle - // TODO factor out decode of index to permit alt. models - } - } - - @ChannelHandler.Sharable - class Shuffle extends ChannelInboundHandlerAdapter { - private final IndexCache indexCache; - private final LoadingCache pathCache; - - private int port; - - Shuffle(Configuration conf) { - this.port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT); - this.indexCache = new IndexCache(new JobConf(conf)); - this.pathCache = CacheBuilder.newBuilder() - .expireAfterAccess(conf.getInt(EXPIRE_AFTER_ACCESS_MINUTES, - DEFAULT_EXPIRE_AFTER_ACCESS_MINUTES), TimeUnit.MINUTES) - .softValues() - .concurrencyLevel(conf.getInt(CONCURRENCY_LEVEL, - DEFAULT_CONCURRENCY_LEVEL)) - .removalListener((RemovalListener) notification -> - LOG.debug("PathCache Eviction: {}, Reason={}", - notification.getKey(), notification.getCause())) - .maximumWeight(conf.getInt(MAX_WEIGHT, DEFAULT_MAX_WEIGHT)) - .weigher((key, value) -> key.jobId.length() + key.user.length() + - key.attemptId.length()+ value.indexPath.toString().length() + - value.dataPath.toString().length()) - .build(new CacheLoader() { - @Override - public AttemptPathInfo load(AttemptPathIdentifier key) throws - Exception { - String base = getBaseLocation(key.jobId, key.user); - String attemptBase = base + key.attemptId; - Path indexFileName = getAuxiliaryLocalPathHandler() - .getLocalPathForRead(attemptBase + "/" + INDEX_FILE_NAME); - Path mapOutputFileName = getAuxiliaryLocalPathHandler() - .getLocalPathForRead(attemptBase + "/" + DATA_FILE_NAME); - LOG.debug("Loaded : {} via loader", key); - return new AttemptPathInfo(indexFileName, mapOutputFileName); - } - }); - } - - public void setPort(int port) { - this.port = port; - } - - private List splitMaps(List mapq) { - if (null == mapq) { - return null; - } - final List ret = new ArrayList(); - for (String s : mapq) { - Collections.addAll(ret, s.split(",")); - } - return ret; - } - - @Override - public void channelActive(ChannelHandlerContext ctx) - throws Exception { - NettyChannelHelper.channelActive(ctx.channel()); - int numConnections = activeConnections.incrementAndGet(); - if ((maxShuffleConnections > 0) && (numConnections > maxShuffleConnections)) { - LOG.info(String.format("Current number of shuffle connections (%d) is " + - "greater than the max allowed shuffle connections (%d)", - accepted.size(), maxShuffleConnections)); - - Map headers = new HashMap<>(1); - // notify fetchers to backoff for a while before closing the connection - // if the shuffle connection limit is hit. Fetchers are expected to - // handle this notification gracefully, that is, not treating this as a - // fetch failure. - headers.put(RETRY_AFTER_HEADER, String.valueOf(FETCH_RETRY_DELAY)); - sendError(ctx, "", TOO_MANY_REQ_STATUS, headers); - } else { - super.channelActive(ctx); - accepted.add(ctx.channel()); - LOG.debug("Added channel: {}, channel id: {}. Accepted number of connections={}", - ctx.channel(), ctx.channel().id(), activeConnections.get()); - } - } - - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - NettyChannelHelper.channelInactive(ctx.channel()); - super.channelInactive(ctx); - int noOfConnections = activeConnections.decrementAndGet(); - LOG.debug("New value of Accepted number of connections={}", noOfConnections); - } - - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) - throws Exception { - Channel channel = ctx.channel(); - LOG.trace("Executing channelRead, channel id: {}", channel.id()); - HttpRequest request = (HttpRequest) msg; - LOG.debug("Received HTTP request: {}, channel id: {}", request, channel.id()); - if (request.method() != GET) { - sendError(ctx, METHOD_NOT_ALLOWED); - return; - } - // Check whether the shuffle version is compatible - String shuffleVersion = ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION; - String httpHeaderName = ShuffleHeader.DEFAULT_HTTP_HEADER_NAME; - if (request.headers() != null) { - shuffleVersion = request.headers().get(ShuffleHeader.HTTP_HEADER_VERSION); - httpHeaderName = request.headers().get(ShuffleHeader.HTTP_HEADER_NAME); - LOG.debug("Received from request header: ShuffleVersion={} header name={}, channel id: {}", - shuffleVersion, httpHeaderName, channel.id()); - } - if (request.headers() == null || - !ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(httpHeaderName) || - !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(shuffleVersion)) { - sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST); - } - final Map> q = - new QueryStringDecoder(request.uri()).parameters(); - final List keepAliveList = q.get("keepAlive"); - boolean keepAliveParam = false; - if (keepAliveList != null && keepAliveList.size() == 1) { - keepAliveParam = Boolean.valueOf(keepAliveList.get(0)); - if (LOG.isDebugEnabled()) { - LOG.debug("KeepAliveParam: {} : {}, channel id: {}", - keepAliveList, keepAliveParam, channel.id()); - } - } - final List mapIds = splitMaps(q.get("map")); - final List reduceQ = q.get("reduce"); - final List jobQ = q.get("job"); - if (LOG.isDebugEnabled()) { - LOG.debug("RECV: " + request.uri() + - "\n mapId: " + mapIds + - "\n reduceId: " + reduceQ + - "\n jobId: " + jobQ + - "\n keepAlive: " + keepAliveParam + - "\n channel id: " + channel.id()); - } - - if (mapIds == null || reduceQ == null || jobQ == null) { - sendError(ctx, "Required param job, map and reduce", BAD_REQUEST); - return; - } - if (reduceQ.size() != 1 || jobQ.size() != 1) { - sendError(ctx, "Too many job/reduce parameters", BAD_REQUEST); - return; - } - - int reduceId; - String jobId; - try { - reduceId = Integer.parseInt(reduceQ.get(0)); - jobId = jobQ.get(0); - } catch (NumberFormatException e) { - sendError(ctx, "Bad reduce parameter", BAD_REQUEST); - return; - } catch (IllegalArgumentException e) { - sendError(ctx, "Bad job parameter", BAD_REQUEST); - return; - } - final String reqUri = request.uri(); - if (null == reqUri) { - // TODO? add upstream? - sendError(ctx, FORBIDDEN); - return; - } - HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK); - try { - verifyRequest(jobId, ctx, request, response, - new URL("http", "", this.port, reqUri)); - } catch (IOException e) { - LOG.warn("Shuffle failure ", e); - sendError(ctx, e.getMessage(), UNAUTHORIZED); - return; - } - - Map mapOutputInfoMap = - new HashMap(); - ChannelPipeline pipeline = channel.pipeline(); - TimeoutHandler timeoutHandler = - (TimeoutHandler)pipeline.get(TIMEOUT_HANDLER); - timeoutHandler.setEnabledTimeout(false); - String user = userRsrc.get(jobId); - - try { - populateHeaders(mapIds, jobId, user, reduceId, request, - response, keepAliveParam, mapOutputInfoMap); - } catch(IOException e) { - //HADOOP-15327 - // Need to send an instance of LastHttpContent to define HTTP - // message boundaries. - //Sending a HTTP 200 OK + HTTP 500 later (sendError) - // is quite a non-standard way of crafting HTTP responses, - // but we need to keep backward compatibility. - // See more details in jira. - writeToChannelAndAddLastHttpContent(channel, response); - LOG.error("Shuffle error while populating headers. Channel id: " + channel.id(), e); - sendError(ctx, getErrorMessage(e), INTERNAL_SERVER_ERROR); - return; - } - writeToChannel(channel, response).addListener((ChannelFutureListener) future -> { - if (future.isSuccess()) { - LOG.debug("Written HTTP response object successfully. Channel id: {}", channel.id()); - } else { - LOG.error("Error while writing HTTP response object: {}. " + - "Cause: {}, channel id: {}", response, future.cause(), channel.id()); - } - }); - //Initialize one ReduceContext object per channelRead call - boolean keepAlive = keepAliveParam || connectionKeepAliveEnabled; - ReduceContext reduceContext = new ReduceContext(mapIds, reduceId, ctx, - user, mapOutputInfoMap, jobId, keepAlive); - for (int i = 0; i < Math.min(maxSessionOpenFiles, mapIds.size()); i++) { - ChannelFuture nextMap = sendMap(reduceContext); - if(nextMap == null) { - return; - } - } - } - - /** - * Calls sendMapOutput for the mapId pointed by ReduceContext.mapsToSend - * and increments it. This method is first called by messageReceived() - * maxSessionOpenFiles times and then on the completion of every - * sendMapOutput operation. This limits the number of open files on a node, - * which can get really large(exhausting file descriptors on the NM) if all - * sendMapOutputs are called in one go, as was done previous to this change. - * @param reduceContext used to call sendMapOutput with correct params. - * @return the ChannelFuture of the sendMapOutput, can be null. - */ - public ChannelFuture sendMap(ReduceContext reduceContext) { - LOG.trace("Executing sendMap"); - ChannelFuture nextMap = null; - if (reduceContext.getMapsToSend().get() < - reduceContext.getMapIds().size()) { - int nextIndex = reduceContext.getMapsToSend().getAndIncrement(); - String mapId = reduceContext.getMapIds().get(nextIndex); - - try { - MapOutputInfo info = reduceContext.getInfoMap().get(mapId); - if (info == null) { - info = getMapOutputInfo(mapId, reduceContext.getReduceId(), - reduceContext.getJobId(), reduceContext.getUser()); - } - LOG.trace("Calling sendMapOutput"); - nextMap = sendMapOutput( - reduceContext.getCtx(), - reduceContext.getCtx().channel(), - reduceContext.getUser(), mapId, - reduceContext.getReduceId(), info); - if (nextMap == null) { - //This can only happen if spill file was not found - sendError(reduceContext.getCtx(), NOT_FOUND); - LOG.trace("Returning nextMap: null"); - return null; - } - nextMap.addListener(new ReduceMapFileCount(reduceContext)); - } catch (IOException e) { - if (e instanceof DiskChecker.DiskErrorException) { - LOG.error("Shuffle error: " + e); - } else { - LOG.error("Shuffle error: ", e); - } - String errorMessage = getErrorMessage(e); - sendError(reduceContext.getCtx(), errorMessage, - INTERNAL_SERVER_ERROR); - return null; - } - } - return nextMap; - } - - private String getErrorMessage(Throwable t) { - StringBuffer sb = new StringBuffer(t.getMessage()); - while (t.getCause() != null) { - sb.append(t.getCause().getMessage()); - t = t.getCause(); - } - return sb.toString(); - } - - private String getBaseLocation(String jobId, String user) { - final JobID jobID = JobID.forName(jobId); - final ApplicationId appID = - ApplicationId.newInstance(Long.parseLong(jobID.getJtIdentifier()), - jobID.getId()); - final String baseStr = - ContainerLocalizer.USERCACHE + "/" + user + "/" - + ContainerLocalizer.APPCACHE + "/" - + appID.toString() + "/output" + "/"; - return baseStr; - } - - protected MapOutputInfo getMapOutputInfo(String mapId, int reduce, - String jobId, String user) throws IOException { - AttemptPathInfo pathInfo; - try { - AttemptPathIdentifier identifier = new AttemptPathIdentifier( - jobId, user, mapId); - pathInfo = pathCache.get(identifier); - if (LOG.isDebugEnabled()) { - LOG.debug("Retrieved pathInfo for " + identifier + - " check for corresponding loaded messages to determine whether" + - " it was loaded or cached"); - } - } catch (ExecutionException e) { - if (e.getCause() instanceof IOException) { - throw (IOException) e.getCause(); - } else { - throw new RuntimeException(e.getCause()); - } - } - - IndexRecord info = indexCache.getIndexInformation(mapId, reduce, pathInfo.indexPath, user); - - if (LOG.isDebugEnabled()) { - LOG.debug("getMapOutputInfo: jobId=" + jobId + ", mapId=" + mapId + - ",dataFile=" + pathInfo.dataPath + ", indexFile=" + - pathInfo.indexPath); - } - - MapOutputInfo outputInfo = new MapOutputInfo(pathInfo.dataPath, info); - return outputInfo; - } - - protected void populateHeaders(List mapIds, String jobId, - String user, int reduce, HttpRequest request, HttpResponse response, - boolean keepAliveParam, Map mapOutputInfoMap) - throws IOException { - - long contentLength = 0; - for (String mapId : mapIds) { - MapOutputInfo outputInfo = getMapOutputInfo(mapId, reduce, jobId, user); - if (mapOutputInfoMap.size() < mapOutputMetaInfoCacheSize) { - mapOutputInfoMap.put(mapId, outputInfo); - } - - ShuffleHeader header = - new ShuffleHeader(mapId, outputInfo.indexRecord.partLength, - outputInfo.indexRecord.rawLength, reduce); - DataOutputBuffer dob = new DataOutputBuffer(); - header.write(dob); - contentLength += outputInfo.indexRecord.partLength; - contentLength += dob.getLength(); - } - - // Now set the response headers. - setResponseHeaders(response, keepAliveParam, contentLength); - - // this audit log is disabled by default, - // to turn it on please enable this audit log - // on log4j.properties by uncommenting the setting - if (AUDITLOG.isDebugEnabled()) { - StringBuilder sb = new StringBuilder("shuffle for "); - sb.append(jobId).append(" reducer ").append(reduce); - sb.append(" length ").append(contentLength); - if (AUDITLOG.isTraceEnabled()) { - // For trace level logging, append the list of mappers - sb.append(" mappers: ").append(mapIds); - AUDITLOG.trace(sb.toString()); - } else { - AUDITLOG.debug(sb.toString()); - } - } - } - - protected void setResponseHeaders(HttpResponse response, - boolean keepAliveParam, long contentLength) { - if (!connectionKeepAliveEnabled && !keepAliveParam) { - response.headers().set(HttpHeader.CONNECTION.asString(), CONNECTION_CLOSE); - } else { - response.headers().set(HttpHeader.CONTENT_LENGTH.asString(), - String.valueOf(contentLength)); - response.headers().set(HttpHeader.CONNECTION.asString(), - HttpHeader.KEEP_ALIVE.asString()); - response.headers().set(HttpHeader.KEEP_ALIVE.asString(), - "timeout=" + connectionKeepAliveTimeOut); - LOG.info("Content Length in shuffle : " + contentLength); - } - } - - class MapOutputInfo { - final Path mapOutputFileName; - final IndexRecord indexRecord; - - MapOutputInfo(Path mapOutputFileName, IndexRecord indexRecord) { - this.mapOutputFileName = mapOutputFileName; - this.indexRecord = indexRecord; - } - } - - protected void verifyRequest(String appid, ChannelHandlerContext ctx, - HttpRequest request, HttpResponse response, URL requestUri) - throws IOException { - SecretKey tokenSecret = secretManager.retrieveTokenSecret(appid); - if (null == tokenSecret) { - LOG.info("Request for unknown token {}, channel id: {}", appid, ctx.channel().id()); - throw new IOException("Could not find jobid"); - } - // encrypting URL - String encryptedURL = SecureShuffleUtils.buildMsgFrom(requestUri); - // hash from the fetcher - String urlHashStr = - request.headers().get(SecureShuffleUtils.HTTP_HEADER_URL_HASH); - if (urlHashStr == null) { - LOG.info("Missing header hash for {}, channel id: {}", appid, ctx.channel().id()); - throw new IOException("fetcher cannot be authenticated"); - } - if (LOG.isDebugEnabled()) { - int len = urlHashStr.length(); - LOG.debug("Verifying request. encryptedURL:{}, hash:{}, channel id: " + - "{}", encryptedURL, - urlHashStr.substring(len - len / 2, len - 1), ctx.channel().id()); - } - // verify - throws exception - SecureShuffleUtils.verifyReply(urlHashStr, encryptedURL, tokenSecret); - // verification passed - encode the reply - String reply = SecureShuffleUtils.generateHash(urlHashStr.getBytes(Charsets.UTF_8), - tokenSecret); - response.headers().set( - SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply); - // Put shuffle version into http header - response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, - ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); - response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, - ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); - if (LOG.isDebugEnabled()) { - int len = reply.length(); - LOG.debug("Fetcher request verified. " + - "encryptedURL: {}, reply: {}, channel id: {}", - encryptedURL, reply.substring(len - len / 2, len - 1), - ctx.channel().id()); - } - } - - protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, - String user, String mapId, int reduce, MapOutputInfo mapOutputInfo) - throws IOException { - final IndexRecord info = mapOutputInfo.indexRecord; - final ShuffleHeader header = new ShuffleHeader(mapId, info.partLength, info.rawLength, - reduce); - final DataOutputBuffer dob = new DataOutputBuffer(); - header.write(dob); - writeToChannel(ch, wrappedBuffer(dob.getData(), 0, dob.getLength())); - final File spillfile = - new File(mapOutputInfo.mapOutputFileName.toString()); - RandomAccessFile spill; - try { - spill = SecureIOUtils.openForRandomRead(spillfile, "r", user, null); - } catch (FileNotFoundException e) { - LOG.info("{} not found. Channel id: {}", spillfile, ctx.channel().id()); - return null; - } - ChannelFuture writeFuture; - if (ch.pipeline().get(SslHandler.class) == null) { - final FadvisedFileRegion partition = new FadvisedFileRegion(spill, - info.startOffset, info.partLength, manageOsCache, readaheadLength, - readaheadPool, spillfile.getAbsolutePath(), - shuffleBufferSize, shuffleTransferToAllowed); - writeFuture = writeToChannel(ch, partition); - writeFuture.addListener(new ChannelFutureListener() { - // TODO error handling; distinguish IO/connection failures, - // attribute to appropriate spill output - @Override - public void operationComplete(ChannelFuture future) { - if (future.isSuccess()) { - partition.transferSuccessful(); - } - partition.deallocate(); - } - }); - } else { - // HTTPS cannot be done with zero copy. - final FadvisedChunkedFile chunk = new FadvisedChunkedFile(spill, - info.startOffset, info.partLength, sslFileBufferSize, - manageOsCache, readaheadLength, readaheadPool, - spillfile.getAbsolutePath()); - writeFuture = writeToChannel(ch, chunk); - } - metrics.shuffleConnections.incr(); - metrics.shuffleOutputBytes.incr(info.partLength); // optimistic - return writeFuture; - } - - protected void sendError(ChannelHandlerContext ctx, - HttpResponseStatus status) { - sendError(ctx, "", status); - } - - protected void sendError(ChannelHandlerContext ctx, String message, - HttpResponseStatus status) { - sendError(ctx, message, status, Collections.emptyMap()); - } - - protected void sendError(ChannelHandlerContext ctx, String msg, - HttpResponseStatus status, Map headers) { - FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, status, - Unpooled.copiedBuffer(msg, CharsetUtil.UTF_8)); - response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8"); - // Put shuffle version into http header - response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, - ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); - response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, - ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); - for (Map.Entry header : headers.entrySet()) { - response.headers().set(header.getKey(), header.getValue()); - } - - // Close the connection as soon as the error message is sent. - writeToChannelAndClose(ctx.channel(), response); - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) - throws Exception { - Channel ch = ctx.channel(); - if (cause instanceof TooLongFrameException) { - LOG.trace("TooLongFrameException, channel id: {}", ch.id()); - sendError(ctx, BAD_REQUEST); - return; - } else if (cause instanceof IOException) { - if (cause instanceof ClosedChannelException) { - LOG.debug("Ignoring closed channel error, channel id: " + ch.id(), cause); - return; - } - String message = String.valueOf(cause.getMessage()); - if (IGNORABLE_ERROR_MESSAGE.matcher(message).matches()) { - LOG.debug("Ignoring client socket close, channel id: " + ch.id(), cause); - return; - } - } - - LOG.error("Shuffle error. Channel id: " + ch.id(), cause); - if (ch.isActive()) { - sendError(ctx, INTERNAL_SERVER_ERROR); + LOG.debug("Closing channel as writer was idle for {} seconds", connectionKeepAliveTimeOut); + ctx.channel().close(); } } } + @SuppressWarnings("checkstyle:VisibilityModifier") static class AttemptPathInfo { // TODO Change this over to just store local dir indices, instead of the // entire path. Far more efficient. - private final Path indexPath; - private final Path dataPath; + public final Path indexPath; + public final Path dataPath; AttemptPathInfo(Path indexPath, Path dataPath) { this.indexPath = indexPath; @@ -1482,10 +688,11 @@ public class ShuffleHandler extends AuxiliaryService { } } + @SuppressWarnings("checkstyle:VisibilityModifier") static class AttemptPathIdentifier { - private final String jobId; - private final String user; - private final String attemptId; + public final String jobId; + public final String user; + public final String attemptId; AttemptPathIdentifier(String jobId, String user, String attemptId) { this.jobId = jobId; @@ -1529,4 +736,14 @@ public class ShuffleHandler extends AuxiliaryService { '}'; } } + + private static String getBaseLocation(String jobId, String user) { + final JobID jobID = JobID.forName(jobId); + final ApplicationId appID = + ApplicationId.newInstance(Long.parseLong(jobID.getJtIdentifier()), + jobID.getId()); + return ContainerLocalizer.USERCACHE + "/" + user + "/" + + ContainerLocalizer.APPCACHE + "/" + + appID + "/output" + "/"; + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleChannelHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleChannelHandler.java new file mode 100644 index 00000000000..7fedc7bb2dc --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleChannelHandler.java @@ -0,0 +1,562 @@ +/* + * 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.mapred; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.ByteBufUtil; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.FileRegion; +import io.netty.channel.embedded.EmbeddedChannel; +import io.netty.channel.group.DefaultChannelGroup; +import io.netty.handler.codec.MessageToMessageEncoder; +import io.netty.handler.codec.http.DefaultFullHttpRequest; +import io.netty.handler.codec.http.DefaultHttpResponse; +import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.HttpHeaders; +import io.netty.handler.codec.http.HttpMethod; +import io.netty.handler.codec.http.HttpObjectAggregator; +import io.netty.handler.codec.http.HttpResponseDecoder; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpServerCodec; +import io.netty.handler.codec.http.HttpUtil; +import io.netty.handler.codec.http.HttpVersion; +import io.netty.handler.codec.http.LastHttpContent; +import io.netty.handler.ssl.SslContext; +import io.netty.handler.ssl.SslContextBuilder; +import io.netty.handler.ssl.SslHandler; +import io.netty.handler.stream.ChunkedWriteHandler; +import io.netty.util.concurrent.GlobalEventExecutor; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.nio.charset.StandardCharsets; +import java.security.cert.X509Certificate; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; + +import javax.crypto.SecretKey; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLEngine; +import javax.net.ssl.TrustManager; +import javax.net.ssl.X509TrustManager; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.security.SecureShuffleUtils; +import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier; +import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager; +import org.apache.hadoop.mapreduce.task.reduce.ShuffleHeader; +import org.apache.hadoop.metrics2.MetricsSystem; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.security.token.SecretManager; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; +import org.eclipse.jetty.http.HttpHeader; +import org.junit.Test; +import org.slf4j.LoggerFactory; + +import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_LENGTH; +import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_TYPE; +import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; +import static org.apache.hadoop.mapred.ShuffleChannelHandler.shuffleHeaderToBytes; +import static org.apache.hadoop.mapred.ShuffleChannelInitializer.MAX_CONTENT_LENGTH; +import static org.apache.hadoop.mapred.ShuffleHandler.CONNECTION_CLOSE; +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED; +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT; +import static org.apache.hadoop.mapred.ShuffleHandler.TIMEOUT_HANDLER; +import static org.apache.hadoop.mapreduce.security.SecureShuffleUtils.HTTP_HEADER_URL_HASH; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class TestShuffleChannelHandler extends TestShuffleHandlerBase { + private static final org.slf4j.Logger LOG = + LoggerFactory.getLogger(TestShuffleChannelHandler.class); + + @Test + public void testGetMapsFileRegion() throws IOException { + final ShuffleTest t = createShuffleTest(); + final EmbeddedChannel shuffle = t.createShuffleHandlerChannelFileRegion(); + t.testGetAllAttemptsForReduce0NoKeepAlive(shuffle.outboundMessages(), shuffle); + } + + @Test + public void testGetMapsChunkedFileSSl() throws Exception { + final ShuffleTest t = createShuffleTest(); + final LinkedList unencryptedMessages = new LinkedList<>(); + final EmbeddedChannel shuffle = t.createShuffleHandlerSSL(unencryptedMessages); + t.testGetAllAttemptsForReduce0NoKeepAlive(unencryptedMessages, shuffle); + } + + @Test + public void testKeepAlive() throws Exception { + // TODO: problems with keep-alive + // current behaviour: + // a) mapreduce.shuffle.connection-keep-alive.enable=false + // + client request with &keepAlive=true + // ==> connection is kept + // b) mapreduce.shuffle.connection-keep-alive.enable=true + // ==> connection is kept + // + // a) seems like a bug + // b) might be ok, because it's the default in HTTP/1.1 + Configuration conf = new Configuration(); + conf.set(SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, "false"); + conf.set(SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, "15"); + final ShuffleTest t = createShuffleTest(conf); + final EmbeddedChannel shuffle = t.createShuffleHandlerChannelFileRegion(); + t.testKeepAlive(shuffle.outboundMessages(), shuffle); + } + + @Test + public void testKeepAliveSSL() throws Exception { + Configuration conf = new Configuration(); + conf.set(SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, "false"); + conf.set(SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, "15"); + final ShuffleTest t = createShuffleTest(conf); + final LinkedList unencryptedMessages = new LinkedList<>(); + final EmbeddedChannel shuffle = t.createShuffleHandlerSSL(unencryptedMessages); + t.testKeepAlive(unencryptedMessages, shuffle); + } + + @Test + public void tetKeepAliveTimeout() throws InterruptedException, IOException { + Configuration conf = new Configuration(); + conf.set(SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, "true"); + conf.set(SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, "1"); + final ShuffleTest t = createShuffleTest(conf); + final EmbeddedChannel shuffle = t.createShuffleHandlerChannelFileRegion(); + + FullHttpRequest req = t.createRequest(getUri(TEST_JOB_ID, 0, + Collections.singletonList(TEST_ATTEMPT_1), true)); + shuffle.writeInbound(req); + t.assertResponse(shuffle.outboundMessages(), + t.getExpectedHttpResponse(req, true, 46), + t.getAttemptData(new Attempt(TEST_ATTEMPT_1, TEST_DATA_A)) + ); + assertTrue("keep-alive", shuffle.isActive()); + + TimeUnit.SECONDS.sleep(3); + shuffle.runScheduledPendingTasks(); + + assertFalse("closed", shuffle.isActive()); + } + + @Test + public void testIncompatibleShuffleVersion() { + Configuration conf = new Configuration(); + conf.set(SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, "true"); + final ShuffleTest t = createShuffleTest(conf); + final EmbeddedChannel shuffle = t.createShuffleHandlerChannelFileRegion(); + FullHttpRequest req = t.createRequest(getUri(TEST_JOB_ID, 0, + Collections.singletonList(TEST_ATTEMPT_1), true)); + req.headers().set(ShuffleHeader.HTTP_HEADER_NAME, "invalid"); + shuffle.writeInbound(req); + + final EmbeddedChannel decoder = t.createHttpResponseChannel(); + for (Object obj : shuffle.outboundMessages()) { + decoder.writeInbound(obj); + } + DefaultHttpResponse actual = decoder.readInbound(); + assertFalse(actual.headers().get(CONTENT_LENGTH).isEmpty()); + actual.headers().set(CONTENT_LENGTH, 0); + + assertEquals(getExpectedHttpResponse(HttpResponseStatus.BAD_REQUEST).toString(), + actual.toString()); + + assertFalse("closed", shuffle.isActive()); // known-issue + } + + @Test + public void testInvalidMapNoIndexFile() { + final ShuffleTest t = createShuffleTest(); + final EmbeddedChannel shuffle = t.createShuffleHandlerChannelFileRegion(); + FullHttpRequest req = t.createRequest(getUri(TEST_JOB_ID, 0, + Arrays.asList(TEST_ATTEMPT_1, "non-existing"), true)); + shuffle.writeInbound(req); + + final EmbeddedChannel decoder = t.createHttpResponseChannel(); + for (Object obj : shuffle.outboundMessages()) { + decoder.writeInbound(obj); + } + + DefaultHttpResponse actual = decoder.readInbound(); + assertFalse(actual.headers().get(CONTENT_LENGTH).isEmpty()); + actual.headers().set(CONTENT_LENGTH, 0); + + assertEquals(getExpectedHttpResponse(HttpResponseStatus.INTERNAL_SERVER_ERROR).toString(), + actual.toString()); + + assertFalse("closed", shuffle.isActive()); + } + + @Test + public void testInvalidMapNoDataFile() { + final ShuffleTest t = createShuffleTest(); + final EmbeddedChannel shuffle = t.createShuffleHandlerChannelFileRegion(); + + String dataFile = getDataFile(tempDir.toAbsolutePath().toString(), TEST_ATTEMPT_2); + assertTrue("should delete", new File(dataFile).delete()); + + FullHttpRequest req = t.createRequest(getUri(TEST_JOB_ID, 0, + Arrays.asList(TEST_ATTEMPT_1, TEST_ATTEMPT_2), false)); + shuffle.writeInbound(req); + + final EmbeddedChannel decoder = t.createHttpResponseChannel(); + for (Object obj : shuffle.outboundMessages()) { + decoder.writeInbound(obj); + } + + DefaultHttpResponse actual = decoder.readInbound(); + assertFalse(actual.headers().get(CONTENT_LENGTH).isEmpty()); + actual.headers().set(CONTENT_LENGTH, 0); + + assertEquals(getExpectedHttpResponse(HttpResponseStatus.INTERNAL_SERVER_ERROR).toString(), + actual.toString()); + + assertFalse("closed", shuffle.isActive()); + } + + private DefaultHttpResponse getExpectedHttpResponse(HttpResponseStatus status) { + DefaultHttpResponse response = new DefaultHttpResponse(HTTP_1_1, status); + response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8"); + response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, + ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); + response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, + ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + response.headers().set(CONTENT_LENGTH, 0); + return response; + } + + private ShuffleTest createShuffleTest() { + return createShuffleTest(new Configuration()); + } + + private ShuffleTest createShuffleTest(Configuration conf) { + return new ShuffleTest(conf); + } + + private File getResourceFile(String resourceName) { + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + return new File(Objects.requireNonNull(classLoader.getResource(resourceName)).getFile()); + } + + @SuppressWarnings("checkstyle:VisibilityModifier") + static class Attempt { + final String id; + final String content; + + Attempt(String attempt, String content) { + this.id = attempt; + this.content = content; + } + } + + private class ShuffleTest { + private final ShuffleChannelHandlerContext ctx; + private final SecretKey shuffleSecretKey; + + ShuffleTest(Configuration conf) { + JobConf jobConf = new JobConf(conf); + MetricsSystem ms = DefaultMetricsSystem.instance(); + this.ctx = new ShuffleChannelHandlerContext(conf, + new ConcurrentHashMap<>(), + new JobTokenSecretManager(), + createLoadingCache(), + new IndexCache(jobConf), + ms.register(new ShuffleHandler.ShuffleMetrics()), + new DefaultChannelGroup(GlobalEventExecutor.INSTANCE) + ); + + JobTokenIdentifier tokenId = new JobTokenIdentifier(new Text(TEST_JOB_ID)); + Token token = new Token<>(tokenId, ctx.secretManager); + shuffleSecretKey = JobTokenSecretManager.createSecretKey(token.getPassword()); + + ctx.userRsrc.put(TEST_JOB_ID, TEST_USER); + ctx.secretManager.addTokenForJob(TEST_JOB_ID, token); + } + + public FullHttpRequest createRequest(String uri) { + FullHttpRequest request = + new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, uri); + request.headers().set(ShuffleHeader.HTTP_HEADER_NAME, + ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); + request.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, + ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + request.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, + ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + try { + String msgToEncode = SecureShuffleUtils.buildMsgFrom(new URL("http", "", ctx.port, uri)); + request.headers().set(HTTP_HEADER_URL_HASH, + SecureShuffleUtils.hashFromString(msgToEncode, shuffleSecretKey)); + } catch (IOException e) { + e.printStackTrace(); + fail("Could not create URL hash for test request"); + } + + return request; + } + + public DefaultHttpResponse getExpectedHttpResponse( + FullHttpRequest request, boolean keepAlive, long contentLength) { + DefaultHttpResponse response = + new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + HttpHeaders headers = response.headers(); + try { + SecretKey tokenSecret = ctx.secretManager.retrieveTokenSecret(TEST_JOB_ID); + headers.set(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, + SecureShuffleUtils.generateHash( + request.headers().get(HTTP_HEADER_URL_HASH).getBytes(Charsets.UTF_8), + tokenSecret)); + } catch (SecretManager.InvalidToken e) { + fail("Could not generate reply hash"); + } + headers.set(ShuffleHeader.HTTP_HEADER_NAME, ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); + headers.set(ShuffleHeader.HTTP_HEADER_VERSION, ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + if (keepAlive) { + headers.set(HttpHeader.CONNECTION.asString(), HttpHeader.KEEP_ALIVE.asString()); + headers.set(HttpHeader.KEEP_ALIVE.asString(), "timeout=" + ctx.connectionKeepAliveTimeOut); + } else { + response.headers().set(HttpHeader.CONNECTION.asString(), CONNECTION_CLOSE); + } + HttpUtil.setContentLength(response, contentLength); + return response; + } + + private void testGetAllAttemptsForReduce0NoKeepAlive( + java.util.Queue outboundMessages, EmbeddedChannel shuffle) throws IOException { + final FullHttpRequest request = createRequest( + getUri(TEST_JOB_ID, 0, + Arrays.asList(TEST_ATTEMPT_1, TEST_ATTEMPT_2, TEST_ATTEMPT_3), false)); + shuffle.writeInbound(request); + assertResponse(outboundMessages, + getExpectedHttpResponse(request, false, 138), + getAllAttemptsForReduce0() + ); + assertFalse("no keep-alive", shuffle.isActive()); + } + + private void testKeepAlive(java.util.Queue messages, + EmbeddedChannel shuffle) throws IOException { + final FullHttpRequest req1 = createRequest( + getUri(TEST_JOB_ID, 0, Collections.singletonList(TEST_ATTEMPT_1), true)); + shuffle.writeInbound(req1); + assertResponse(messages, + getExpectedHttpResponse(req1, true, 46), + getAttemptData(new Attempt(TEST_ATTEMPT_1, TEST_DATA_A)) + ); + assertTrue("keep-alive", shuffle.isActive()); + messages.clear(); + + final FullHttpRequest req2 = createRequest( + getUri(TEST_JOB_ID, 0, Collections.singletonList(TEST_ATTEMPT_2), true)); + shuffle.writeInbound(req2); + assertResponse(messages, + getExpectedHttpResponse(req2, true, 46), + getAttemptData(new Attempt(TEST_ATTEMPT_2, TEST_DATA_B)) + ); + assertTrue("keep-alive", shuffle.isActive()); + messages.clear(); + + final FullHttpRequest req3 = createRequest( + getUri(TEST_JOB_ID, 0, Collections.singletonList(TEST_ATTEMPT_3), false)); + shuffle.writeInbound(req3); + assertResponse(messages, + getExpectedHttpResponse(req3, false, 46), + getAttemptData(new Attempt(TEST_ATTEMPT_3, TEST_DATA_C)) + ); + assertFalse("no keep-alive", shuffle.isActive()); + } + + private ArrayList getAllAttemptsForReduce0() throws IOException { + return getAttemptData( + new Attempt(TEST_ATTEMPT_1, TEST_DATA_A), + new Attempt(TEST_ATTEMPT_2, TEST_DATA_B), + new Attempt(TEST_ATTEMPT_3, TEST_DATA_C) + ); + } + + private ArrayList getAttemptData(Attempt... attempts) throws IOException { + ArrayList data = new ArrayList<>(); + for (Attempt attempt : attempts) { + data.add(shuffleHeaderToBytes(new ShuffleHeader(attempt.id, attempt.content.length(), + attempt.content.length() * 2L, 0))); + data.add(Unpooled.copiedBuffer(attempt.content.getBytes(StandardCharsets.UTF_8))); + } + return data; + } + + private void assertResponse(java.util.Queue outboundMessages, + DefaultHttpResponse response, + List content) { + final EmbeddedChannel decodeChannel = createHttpResponseChannel(); + + content.add(LastHttpContent.EMPTY_LAST_CONTENT.content()); + + int i = 0; + for (Object outboundMessage : outboundMessages) { + ByteBuf actualBytes = ((ByteBuf) outboundMessage); + String actualHexdump = ByteBufUtil.prettyHexDump(actualBytes); + LOG.info("\n{}", actualHexdump); + + decodeChannel.writeInbound(actualBytes); + Object obj = decodeChannel.readInbound(); + LOG.info("Decoded object: {}", obj); + + if (i == 0) { + DefaultHttpResponse resp = (DefaultHttpResponse) obj; + assertEquals(response.toString(), resp.toString()); + } + if (i > 0 && i <= content.size()) { + assertEquals("data should match", + ByteBufUtil.prettyHexDump(content.get(i - 1)), actualHexdump); + } + + i++; + } + + // This check is done after to have better debug logs on failure. + assertEquals("all data should match", content.size() + 1, outboundMessages.size()); + } + + public EmbeddedChannel createShuffleHandlerChannelFileRegion() { + final EmbeddedChannel channel = createShuffleHandlerChannel(); + + channel.pipeline().addFirst( + new MessageToMessageEncoder() { + @Override + protected void encode( + ChannelHandlerContext cCtx, FileRegion msg, List out) throws Exception { + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + WritableByteChannel wbc = Channels.newChannel(stream); + msg.transferTo(wbc, msg.position()); + out.add(Unpooled.wrappedBuffer(stream.toByteArray())); + } + } + ); + + return channel; + } + + public EmbeddedChannel createSSLClient() throws Exception { + final EmbeddedChannel channel = createShuffleHandlerChannel(); + + SSLContext sc = SSLContext.getInstance("SSL"); + + final TrustManager trm = new X509TrustManager() { + public X509Certificate[] getAcceptedIssuers() { + return null; + } + + public void checkClientTrusted(X509Certificate[] certs, String authType) { + } + + public void checkServerTrusted(X509Certificate[] certs, String authType) { + } + }; + + sc.init(null, new TrustManager[]{trm}, null); + + final SSLEngine sslEngine = sc.createSSLEngine(); + sslEngine.setUseClientMode(true); + channel.pipeline().addFirst("ssl", new SslHandler(sslEngine)); + + return channel; + } + + public EmbeddedChannel createShuffleHandlerSSL(java.util.Queue unencryptedMessages) + throws Exception { + final EmbeddedChannel channel = createShuffleHandlerChannel(); + // SelfSignedCertificate was generated manually with: + // openssl req -x509 -newkey rsa:4096 -keyout key.pem \ + // -out cert.pem -sha256 -days 3650 -nodes -subj '/CN=localhost' + // Because: + // SelfSignedCertificate ssc = new SelfSignedCertificate(); + // Throws: Failed to generate a self-signed X.509 certificate using Bouncy Castle + final SslContext sslCtx = SslContextBuilder + .forServer(getResourceFile("cert.pem"), getResourceFile("key.pem")) + .build(); + final SslHandler sslHandler = sslCtx.newHandler(ByteBufAllocator.DEFAULT); + channel.pipeline().addFirst("ssl", sslHandler); + + channel.pipeline().addAfter("ssl", "unencrypted", new MessageToMessageEncoder() { + @Override + protected void encode(ChannelHandlerContext cCtx, ByteBuf msg, List out) { + unencryptedMessages.add(msg.copy()); + out.add(msg.retain()); + } + }); + + channel.pipeline().addLast(new ChannelInboundHandlerAdapter() { + @Override + public void userEventTriggered(ChannelHandlerContext cCtx, Object evt) { + LOG.info("EVENT: {}", evt); + } + }); + + // SSLHandshake must be done, otherwise messages are buffered + final EmbeddedChannel client = createSSLClient(); + for (Object obj : client.outboundMessages()) { + channel.writeInbound(obj); + } + client.outboundMessages().clear(); + for (Object obj : channel.outboundMessages()) { + client.writeInbound(obj); + } + channel.outboundMessages().clear(); + for (Object obj : client.outboundMessages()) { + channel.writeInbound(obj); + } + client.outboundMessages().clear(); + + return channel; + } + + public EmbeddedChannel createShuffleHandlerChannel() { + final EmbeddedChannel channel = new EmbeddedChannel(); + channel.pipeline().addLast("http", new HttpServerCodec()); + channel.pipeline().addLast("aggregator", new HttpObjectAggregator(MAX_CONTENT_LENGTH)); + channel.pipeline().addLast("chunking", new ChunkedWriteHandler()); + channel.pipeline().addLast("shuffle", new ShuffleChannelHandler(ctx)); + channel.pipeline().addLast(TIMEOUT_HANDLER, + new ShuffleHandler.TimeoutHandler(ctx.connectionKeepAliveTimeOut)); + return channel; + } + + public EmbeddedChannel createHttpResponseChannel() { + return new EmbeddedChannel( + new HttpResponseDecoder() + ); + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java index 38500032ef3..37a9210286c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java @@ -17,78 +17,50 @@ */ package org.apache.hadoop.mapred; -import io.netty.channel.ChannelFutureListener; -import io.netty.channel.DefaultFileRegion; -import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; -import io.netty.channel.AbstractChannel; -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelPipeline; -import io.netty.channel.socket.SocketChannel; -import io.netty.handler.codec.http.HttpMethod; -import io.netty.handler.codec.http.HttpRequest; -import io.netty.handler.codec.http.HttpResponse; -import io.netty.handler.codec.http.HttpResponseEncoder; -import io.netty.handler.codec.http.HttpResponseStatus; -import io.netty.handler.timeout.IdleStateEvent; -import org.apache.hadoop.test.GenericTestUtils; -import static io.netty.buffer.Unpooled.wrappedBuffer; -import static java.util.stream.Collectors.toList; +import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; + +import io.netty.channel.ChannelFuture; +import io.netty.handler.codec.http.HttpResponseStatus; + +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY; +import static org.apache.hadoop.mapreduce.security.SecureShuffleUtils.HTTP_HEADER_URL_HASH; import static org.apache.hadoop.test.MetricsAsserts.assertCounter; import static org.apache.hadoop.test.MetricsAsserts.assertGauge; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; -import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; import static org.junit.Assume.assumeTrue; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import java.io.ByteArrayOutputStream; +import java.io.BufferedReader; import java.io.DataInputStream; -import java.io.EOFException; import java.io.File; import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; -import java.io.InputStream; +import java.io.InputStreamReader; import java.net.HttpURLConnection; -import java.net.InetSocketAddress; -import java.net.Proxy; -import java.net.Socket; +import java.net.MalformedURLException; import java.net.URL; -import java.net.SocketAddress; -import java.net.URLConnection; import java.nio.ByteBuffer; -import java.nio.channels.ClosedChannelException; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; -import java.util.zip.CheckedOutputStream; -import java.util.zip.Checksum; + +import javax.crypto.SecretKey; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.nativeio.NativeIO; -import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.mapreduce.security.SecureShuffleUtils; import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier; import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager; @@ -100,761 +72,22 @@ import org.apache.hadoop.metrics2.impl.MetricsSystemImpl; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.service.ServiceStateException; -import org.apache.hadoop.util.DiskChecker; -import org.apache.hadoop.util.PureJavaCrc32; import org.apache.hadoop.util.Sets; -import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext; import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext; -import org.apache.hadoop.yarn.server.api.AuxiliaryLocalPathHandler; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer; import org.apache.hadoop.yarn.server.records.Version; -import org.hamcrest.CoreMatchers; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TestName; -import org.mockito.Mockito; -import org.eclipse.jetty.http.HttpHeader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class TestShuffleHandler { - static final long MiB = 1024 * 1024; +public class TestShuffleHandler extends TestShuffleHandlerBase { + static final long MIB = 1024 * 1024; private static final Logger LOG = LoggerFactory.getLogger(TestShuffleHandler.class); - private static final File ABS_LOG_DIR = GenericTestUtils.getTestDir( - TestShuffleHandler.class.getSimpleName() + "LocDir"); - private static final long ATTEMPT_ID = 12345L; - private static final long ATTEMPT_ID_2 = 12346L; + private static final HttpResponseStatus OK_STATUS = new HttpResponseStatus(200, "OK"); - - - //Control test execution properties with these flags - private static final boolean DEBUG_MODE = false; - //WARNING: If this is set to true and proxy server is not running, tests will fail! - private static final boolean USE_PROXY = false; - private static final int HEADER_WRITE_COUNT = 100000; - private static final int ARBITRARY_NEGATIVE_TIMEOUT_SECONDS = -100; - private static TestExecution TEST_EXECUTION; - - private static class TestExecution { - private static final int DEFAULT_KEEP_ALIVE_TIMEOUT_SECONDS = 1; - private static final int DEBUG_KEEP_ALIVE_SECONDS = 1000; - private static final int DEFAULT_PORT = 0; //random port - private static final int FIXED_PORT = 8088; - private static final String PROXY_HOST = "127.0.0.1"; - private static final int PROXY_PORT = 8888; - private static final int CONNECTION_DEBUG_TIMEOUT = 1000000; - private final boolean debugMode; - private final boolean useProxy; - - TestExecution(boolean debugMode, boolean useProxy) { - this.debugMode = debugMode; - this.useProxy = useProxy; - } - - int getKeepAliveTimeout() { - if (debugMode) { - return DEBUG_KEEP_ALIVE_SECONDS; - } - return DEFAULT_KEEP_ALIVE_TIMEOUT_SECONDS; - } - - HttpURLConnection openConnection(URL url) throws IOException { - HttpURLConnection conn; - if (useProxy) { - Proxy proxy - = new Proxy(Proxy.Type.HTTP, new InetSocketAddress(PROXY_HOST, PROXY_PORT)); - conn = (HttpURLConnection) url.openConnection(proxy); - } else { - conn = (HttpURLConnection) url.openConnection(); - } - return conn; - } - - int shuffleHandlerPort() { - if (debugMode) { - return FIXED_PORT; - } else { - return DEFAULT_PORT; - } - } - - void parameterizeConnection(URLConnection conn) { - if (DEBUG_MODE) { - conn.setReadTimeout(CONNECTION_DEBUG_TIMEOUT); - conn.setConnectTimeout(CONNECTION_DEBUG_TIMEOUT); - } - } - } - - private static class ResponseConfig { - private final int headerWriteCount; - private final int mapOutputCount; - private final int contentLengthOfOneMapOutput; - private long headerSize; - public long contentLengthOfResponse; - - ResponseConfig(int headerWriteCount, int mapOutputCount, - int contentLengthOfOneMapOutput) { - if (mapOutputCount <= 0 && contentLengthOfOneMapOutput > 0) { - throw new IllegalStateException("mapOutputCount should be at least 1"); - } - this.headerWriteCount = headerWriteCount; - this.mapOutputCount = mapOutputCount; - this.contentLengthOfOneMapOutput = contentLengthOfOneMapOutput; - } - - private void setHeaderSize(long headerSize) { - this.headerSize = headerSize; - long contentLengthOfAllHeaders = headerWriteCount * headerSize; - this.contentLengthOfResponse = computeContentLengthOfResponse(contentLengthOfAllHeaders); - LOG.debug("Content-length of all headers: {}", contentLengthOfAllHeaders); - LOG.debug("Content-length of one MapOutput: {}", contentLengthOfOneMapOutput); - LOG.debug("Content-length of final HTTP response: {}", contentLengthOfResponse); - } - - private long computeContentLengthOfResponse(long contentLengthOfAllHeaders) { - int mapOutputCountMultiplier = mapOutputCount; - if (mapOutputCount == 0) { - mapOutputCountMultiplier = 1; - } - return (contentLengthOfAllHeaders + contentLengthOfOneMapOutput) * mapOutputCountMultiplier; - } - } - - private enum ShuffleUrlType { - SIMPLE, WITH_KEEPALIVE, WITH_KEEPALIVE_MULTIPLE_MAP_IDS, WITH_KEEPALIVE_NO_MAP_IDS - } - - private static class InputStreamReadResult { - final String asString; - int totalBytesRead; - - InputStreamReadResult(byte[] bytes, int totalBytesRead) { - this.asString = new String(bytes, StandardCharsets.UTF_8); - this.totalBytesRead = totalBytesRead; - } - } - - private static abstract class AdditionalMapOutputSenderOperations { - public abstract ChannelFuture perform(ChannelHandlerContext ctx, Channel ch) throws IOException; - } - - private class ShuffleHandlerForKeepAliveTests extends ShuffleHandler { - final LastSocketAddress lastSocketAddress = new LastSocketAddress(); - final ArrayList failures = new ArrayList<>(); - final ShuffleHeaderProvider shuffleHeaderProvider; - final HeaderPopulator headerPopulator; - MapOutputSender mapOutputSender; - private Consumer channelIdleCallback; - private CustomTimeoutHandler customTimeoutHandler; - private boolean failImmediatelyOnErrors = false; - private boolean closeChannelOnError = true; - private ResponseConfig responseConfig; - - ShuffleHandlerForKeepAliveTests(long attemptId, ResponseConfig responseConfig, - Consumer channelIdleCallback) throws IOException { - this(attemptId, responseConfig); - this.channelIdleCallback = channelIdleCallback; - } - - ShuffleHandlerForKeepAliveTests(long attemptId, ResponseConfig responseConfig) - throws IOException { - this.responseConfig = responseConfig; - this.shuffleHeaderProvider = new ShuffleHeaderProvider(attemptId); - this.responseConfig.setHeaderSize(shuffleHeaderProvider.getShuffleHeaderSize()); - this.headerPopulator = new HeaderPopulator(this, responseConfig, shuffleHeaderProvider, true); - this.mapOutputSender = new MapOutputSender(responseConfig, lastSocketAddress, - shuffleHeaderProvider); - setUseOutboundExceptionHandler(true); - } - - public void setFailImmediatelyOnErrors(boolean failImmediatelyOnErrors) { - this.failImmediatelyOnErrors = failImmediatelyOnErrors; - } - - public void setCloseChannelOnError(boolean closeChannelOnError) { - this.closeChannelOnError = closeChannelOnError; - } - - @Override - protected Shuffle getShuffle(final Configuration conf) { - // replace the shuffle handler with one stubbed for testing - return new Shuffle(conf) { - @Override - protected MapOutputInfo getMapOutputInfo(String mapId, int reduce, - String jobId, String user) { - return null; - } - @Override - protected void verifyRequest(String appid, ChannelHandlerContext ctx, - HttpRequest request, HttpResponse response, URL requestUri) { - } - - @Override - protected void populateHeaders(List mapIds, String jobId, - String user, int reduce, HttpRequest request, - HttpResponse response, boolean keepAliveParam, - Map infoMap) throws IOException { - long contentLength = headerPopulator.populateHeaders( - keepAliveParam); - super.setResponseHeaders(response, keepAliveParam, contentLength); - } - - @Override - protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, - Channel ch, String user, String mapId, int reduce, - MapOutputInfo info) throws IOException { - return mapOutputSender.send(ctx, ch); - } - - @Override - public void channelActive(ChannelHandlerContext ctx) throws Exception { - ctx.pipeline().replace(HttpResponseEncoder.class, ENCODER_HANDLER_NAME, - new LoggingHttpResponseEncoder(false)); - replaceTimeoutHandlerWithCustom(ctx); - LOG.debug("Modified pipeline: {}", ctx.pipeline()); - super.channelActive(ctx); - } - - private void replaceTimeoutHandlerWithCustom(ChannelHandlerContext ctx) { - TimeoutHandler oldTimeoutHandler = - (TimeoutHandler)ctx.pipeline().get(TIMEOUT_HANDLER); - int timeoutValue = - oldTimeoutHandler.getConnectionKeepAliveTimeOut(); - customTimeoutHandler = new CustomTimeoutHandler(timeoutValue, channelIdleCallback); - ctx.pipeline().replace(TIMEOUT_HANDLER, TIMEOUT_HANDLER, customTimeoutHandler); - } - - @Override - protected void sendError(ChannelHandlerContext ctx, - HttpResponseStatus status) { - String message = "Error while processing request. Status: " + status; - handleError(ctx, message); - if (failImmediatelyOnErrors) { - stop(); - } - } - - @Override - protected void sendError(ChannelHandlerContext ctx, String message, - HttpResponseStatus status) { - String errMessage = String.format("Error while processing request. " + - "Status: " + - "%s, message: %s", status, message); - handleError(ctx, errMessage); - if (failImmediatelyOnErrors) { - stop(); - } - } - }; - } - - private void handleError(ChannelHandlerContext ctx, String message) { - LOG.error(message); - failures.add(new Error(message)); - if (closeChannelOnError) { - LOG.warn("sendError: Closing channel"); - ctx.channel().close(); - } - } - - private class CustomTimeoutHandler extends TimeoutHandler { - private boolean channelIdle = false; - private final Consumer channelIdleCallback; - - CustomTimeoutHandler(int connectionKeepAliveTimeOut, - Consumer channelIdleCallback) { - super(connectionKeepAliveTimeOut); - this.channelIdleCallback = channelIdleCallback; - } - - @Override - public void channelIdle(ChannelHandlerContext ctx, IdleStateEvent e) { - LOG.debug("Channel idle"); - this.channelIdle = true; - if (channelIdleCallback != null) { - LOG.debug("Calling channel idle callback.."); - channelIdleCallback.accept(e); - } - super.channelIdle(ctx, e); - } - } - } - - private static class MapOutputSender { - private final ResponseConfig responseConfig; - private final LastSocketAddress lastSocketAddress; - private final ShuffleHeaderProvider shuffleHeaderProvider; - private AdditionalMapOutputSenderOperations additionalMapOutputSenderOperations; - - MapOutputSender(ResponseConfig responseConfig, LastSocketAddress lastSocketAddress, - ShuffleHeaderProvider shuffleHeaderProvider) { - this.responseConfig = responseConfig; - this.lastSocketAddress = lastSocketAddress; - this.shuffleHeaderProvider = shuffleHeaderProvider; - } - - public ChannelFuture send(ChannelHandlerContext ctx, Channel ch) throws IOException { - LOG.debug("In MapOutputSender#send"); - lastSocketAddress.setAddress(ch.remoteAddress()); - ShuffleHeader header = shuffleHeaderProvider.createNewShuffleHeader(); - ChannelFuture future = writeHeaderNTimes(ch, header, responseConfig.headerWriteCount); - // This is the last operation - // It's safe to increment ShuffleHeader counter for better identification - shuffleHeaderProvider.incrementCounter(); - if (additionalMapOutputSenderOperations != null) { - return additionalMapOutputSenderOperations.perform(ctx, ch); - } - return future; - } - - private ChannelFuture writeHeaderNTimes(Channel ch, ShuffleHeader header, int iterations) - throws IOException { - DataOutputBuffer dob = new DataOutputBuffer(); - for (int i = 0; i < iterations; ++i) { - header.write(dob); - } - LOG.debug("MapOutputSender#writeHeaderNTimes WriteAndFlush big chunk of data, " + - "outputBufferSize: " + dob.size()); - return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); - } - } - - private static class ShuffleHeaderProvider { - private final long attemptId; - private int attemptCounter = 0; - private int cachedSize = Integer.MIN_VALUE; - - ShuffleHeaderProvider(long attemptId) { - this.attemptId = attemptId; - } - - ShuffleHeader createNewShuffleHeader() { - return new ShuffleHeader(String.format("attempt_%s_1_m_1_0%s", attemptId, attemptCounter), - 5678, 5678, 1); - } - - void incrementCounter() { - attemptCounter++; - } - - private int getShuffleHeaderSize() throws IOException { - if (cachedSize != Integer.MIN_VALUE) { - return cachedSize; - } - DataOutputBuffer dob = new DataOutputBuffer(); - ShuffleHeader header = createNewShuffleHeader(); - header.write(dob); - cachedSize = dob.size(); - return cachedSize; - } - } - - private static class HeaderPopulator { - private final ShuffleHandler shuffleHandler; - private final boolean disableKeepAliveConfig; - private final ShuffleHeaderProvider shuffleHeaderProvider; - private final ResponseConfig responseConfig; - - HeaderPopulator(ShuffleHandler shuffleHandler, - ResponseConfig responseConfig, - ShuffleHeaderProvider shuffleHeaderProvider, - boolean disableKeepAliveConfig) { - this.shuffleHandler = shuffleHandler; - this.responseConfig = responseConfig; - this.disableKeepAliveConfig = disableKeepAliveConfig; - this.shuffleHeaderProvider = shuffleHeaderProvider; - } - - public long populateHeaders(boolean keepAliveParam) throws IOException { - // Send some dummy data (populate content length details) - DataOutputBuffer dob = new DataOutputBuffer(); - for (int i = 0; i < responseConfig.headerWriteCount; ++i) { - ShuffleHeader header = - shuffleHeaderProvider.createNewShuffleHeader(); - header.write(dob); - } - // for testing purpose; - // disable connectionKeepAliveEnabled if keepAliveParam is available - if (keepAliveParam && disableKeepAliveConfig) { - shuffleHandler.connectionKeepAliveEnabled = false; - } - return responseConfig.contentLengthOfResponse; - } - } - - private static final class HttpConnectionData { - private final Map> headers; - private HttpURLConnection conn; - private final int payloadLength; - private final SocketAddress socket; - private int responseCode = -1; - - private HttpConnectionData(HttpURLConnection conn, int payloadLength, - SocketAddress socket) { - this.headers = conn.getHeaderFields(); - this.conn = conn; - this.payloadLength = payloadLength; - this.socket = socket; - try { - this.responseCode = conn.getResponseCode(); - } catch (IOException e) { - fail("Failed to read response code from connection: " + conn); - } - } - - static HttpConnectionData create(HttpURLConnection conn, int payloadLength, - SocketAddress socket) { - return new HttpConnectionData(conn, payloadLength, socket); - } - } - - private static final class HttpConnectionAssert { - private final HttpConnectionData connData; - - private HttpConnectionAssert(HttpConnectionData connData) { - this.connData = connData; - } - - static HttpConnectionAssert create(HttpConnectionData connData) { - return new HttpConnectionAssert(connData); - } - - public static void assertKeepAliveConnectionsAreSame( - HttpConnectionHelper httpConnectionHelper) { - assertTrue("At least two connection data " + - "is required to perform this assertion", - httpConnectionHelper.connectionData.size() >= 2); - SocketAddress firstAddress = httpConnectionHelper.getConnectionData(0).socket; - SocketAddress secondAddress = httpConnectionHelper.getConnectionData(1).socket; - Assert.assertNotNull("Initial shuffle address should not be null", - firstAddress); - Assert.assertNotNull("Keep-Alive shuffle address should not be null", - secondAddress); - assertEquals("Initial shuffle address and keep-alive shuffle " - + "address should be the same", firstAddress, secondAddress); - } - - public HttpConnectionAssert expectKeepAliveWithTimeout(long timeout) { - assertEquals(HttpURLConnection.HTTP_OK, connData.responseCode); - assertHeaderValue(HttpHeader.CONNECTION, HttpHeader.KEEP_ALIVE.asString()); - assertHeaderValue(HttpHeader.KEEP_ALIVE, "timeout=" + timeout); - return this; - } - - public HttpConnectionAssert expectBadRequest(long timeout) { - assertEquals(HttpURLConnection.HTTP_BAD_REQUEST, connData.responseCode); - assertHeaderValue(HttpHeader.CONNECTION, HttpHeader.KEEP_ALIVE.asString()); - assertHeaderValue(HttpHeader.KEEP_ALIVE, "timeout=" + timeout); - return this; - } - - public HttpConnectionAssert expectResponseContentLength(long size) { - assertEquals(size, connData.payloadLength); - return this; - } - - private void assertHeaderValue(HttpHeader header, String expectedValue) { - List headerList = connData.headers.get(header.asString()); - Assert.assertNotNull("Got null header value for header: " + header, headerList); - Assert.assertFalse("Got empty header value for header: " + header, headerList.isEmpty()); - assertEquals("Unexpected size of header list for header: " + header, 1, - headerList.size()); - assertEquals(expectedValue, headerList.get(0)); - } - } - - private static class HttpConnectionHelper { - private final LastSocketAddress lastSocketAddress; - List connectionData = new ArrayList<>(); - - HttpConnectionHelper(LastSocketAddress lastSocketAddress) { - this.lastSocketAddress = lastSocketAddress; - } - - public void connectToUrls(String[] urls, ResponseConfig responseConfig) throws IOException { - connectToUrlsInternal(urls, responseConfig, HttpURLConnection.HTTP_OK); - } - - public void connectToUrls(String[] urls, ResponseConfig responseConfig, int expectedHttpStatus) - throws IOException { - connectToUrlsInternal(urls, responseConfig, expectedHttpStatus); - } - - private void connectToUrlsInternal(String[] urls, ResponseConfig responseConfig, - int expectedHttpStatus) throws IOException { - int requests = urls.length; - int expectedConnections = urls.length; - LOG.debug("Will connect to URLs: {}", Arrays.toString(urls)); - for (int reqIdx = 0; reqIdx < requests; reqIdx++) { - String urlString = urls[reqIdx]; - LOG.debug("Connecting to URL: {}", urlString); - URL url = new URL(urlString); - HttpURLConnection conn = TEST_EXECUTION.openConnection(url); - conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME, - ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); - conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION, - ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); - TEST_EXECUTION.parameterizeConnection(conn); - conn.connect(); - if (expectedHttpStatus == HttpURLConnection.HTTP_BAD_REQUEST) { - //Catch exception as error are caught with overridden sendError method - //Caught errors will be validated later. - try { - DataInputStream input = new DataInputStream(conn.getInputStream()); - } catch (Exception e) { - expectedConnections--; - continue; - } - } - DataInputStream input = new DataInputStream(conn.getInputStream()); - LOG.debug("Opened DataInputStream for connection: {}/{}", (reqIdx + 1), requests); - ShuffleHeader header = new ShuffleHeader(); - header.readFields(input); - InputStreamReadResult result = readDataFromInputStream(input); - result.totalBytesRead += responseConfig.headerSize; - int expectedContentLength = - Integer.parseInt(conn.getHeaderField(HttpHeader.CONTENT_LENGTH.asString())); - - if (result.totalBytesRead != expectedContentLength) { - throw new IOException(String.format("Premature EOF InputStream. " + - "Expected content-length: %s, " + - "Actual content-length: %s", expectedContentLength, result.totalBytesRead)); - } - connectionData.add(HttpConnectionData - .create(conn, result.totalBytesRead, lastSocketAddress.getSocketAddres())); - input.close(); - LOG.debug("Finished all interactions with URL: {}. Progress: {}/{}", url, (reqIdx + 1), - requests); - } - assertEquals(expectedConnections, connectionData.size()); - } - - void validate(Consumer connDataValidator) { - for (int i = 0; i < connectionData.size(); i++) { - LOG.debug("Validating connection data #{}", (i + 1)); - HttpConnectionData connData = connectionData.get(i); - connDataValidator.accept(connData); - } - } - - HttpConnectionData getConnectionData(int i) { - return connectionData.get(i); - } - - private static InputStreamReadResult readDataFromInputStream( - InputStream input) throws IOException { - ByteArrayOutputStream dataStream = new ByteArrayOutputStream(); - byte[] buffer = new byte[1024]; - int bytesRead; - int totalBytesRead = 0; - while ((bytesRead = input.read(buffer)) != -1) { - dataStream.write(buffer, 0, bytesRead); - totalBytesRead += bytesRead; - } - LOG.debug("Read total bytes: " + totalBytesRead); - dataStream.flush(); - return new InputStreamReadResult(dataStream.toByteArray(), totalBytesRead); - } - } - - class ShuffleHandlerForTests extends ShuffleHandler { - public final ArrayList failures = new ArrayList<>(); - - ShuffleHandlerForTests() { - setUseOutboundExceptionHandler(true); - } - - ShuffleHandlerForTests(MetricsSystem ms) { - super(ms); - setUseOutboundExceptionHandler(true); - } - - @Override - protected Shuffle getShuffle(final Configuration conf) { - return new Shuffle(conf) { - @Override - public void exceptionCaught(ChannelHandlerContext ctx, - Throwable cause) throws Exception { - LOG.debug("ExceptionCaught"); - failures.add(cause); - super.exceptionCaught(ctx, cause); - } - }; - } - } - - class MockShuffleHandler extends org.apache.hadoop.mapred.ShuffleHandler { - final ArrayList failures = new ArrayList<>(); - - private final AuxiliaryLocalPathHandler pathHandler = - new TestAuxiliaryLocalPathHandler(); - - MockShuffleHandler() { - setUseOutboundExceptionHandler(true); - } - - MockShuffleHandler(MetricsSystem ms) { - super(ms); - setUseOutboundExceptionHandler(true); - } - - @Override - protected Shuffle getShuffle(final Configuration conf) { - return new Shuffle(conf) { - @Override - protected void verifyRequest(String appid, ChannelHandlerContext ctx, - HttpRequest request, HttpResponse response, URL requestUri) - throws IOException { - } - @Override - protected MapOutputInfo getMapOutputInfo(String mapId, int reduce, - String jobId, String user) { - // Do nothing. - return null; - } - @Override - protected void populateHeaders(List mapIds, String jobId, - String user, int reduce, HttpRequest request, - HttpResponse response, boolean keepAliveParam, - Map infoMap) { - // Do nothing. - } - @Override - protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, - Channel ch, String user, String mapId, int reduce, - MapOutputInfo info) throws IOException { - - ShuffleHeader header = - new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1); - DataOutputBuffer dob = new DataOutputBuffer(); - header.write(dob); - ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); - dob = new DataOutputBuffer(); - for (int i = 0; i < 100; ++i) { - header.write(dob); - } - return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, - Throwable cause) throws Exception { - LOG.debug("ExceptionCaught"); - failures.add(cause); - super.exceptionCaught(ctx, cause); - } - }; - } - - @Override - public AuxiliaryLocalPathHandler getAuxiliaryLocalPathHandler() { - return pathHandler; - } - } - - private class TestAuxiliaryLocalPathHandler - implements AuxiliaryLocalPathHandler { - @Override - public Path getLocalPathForRead(String path) { - return new Path(ABS_LOG_DIR.getAbsolutePath(), path); - } - - @Override - public Path getLocalPathForWrite(String path) { - return new Path(ABS_LOG_DIR.getAbsolutePath()); - } - - @Override - public Path getLocalPathForWrite(String path, long size) { - return new Path(ABS_LOG_DIR.getAbsolutePath()); - } - - @Override - public Iterable getAllLocalPathsForRead(String path) { - ArrayList paths = new ArrayList<>(); - paths.add(new Path(ABS_LOG_DIR.getAbsolutePath())); - return paths; - } - } - - private static class MockShuffleHandler2 extends - org.apache.hadoop.mapred.ShuffleHandler { - final ArrayList failures = new ArrayList<>(1); - boolean socketKeepAlive = false; - - MockShuffleHandler2() { - setUseOutboundExceptionHandler(true); - } - - MockShuffleHandler2(MetricsSystem ms) { - super(ms); - setUseOutboundExceptionHandler(true); - } - - @Override - protected Shuffle getShuffle(final Configuration conf) { - return new Shuffle(conf) { - @Override - protected void verifyRequest(String appid, ChannelHandlerContext ctx, - HttpRequest request, HttpResponse response, URL requestUri) { - SocketChannel channel = (SocketChannel)(ctx.channel()); - socketKeepAlive = channel.config().isKeepAlive(); - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, - Throwable cause) throws Exception { - LOG.debug("ExceptionCaught"); - failures.add(cause); - super.exceptionCaught(ctx, cause); - } - }; - } - - protected boolean isSocketKeepAlive() { - return socketKeepAlive; - } - } - - @Rule - public TestName name = new TestName(); - - @Before - public void setup() { - TEST_EXECUTION = new TestExecution(DEBUG_MODE, USE_PROXY); - } - - @After - public void tearDown() { - int port = TEST_EXECUTION.shuffleHandlerPort(); - if (isPortUsed(port)) { - String msg = String.format("Port is being used: %d. " + - "Current testcase name: %s", - port, name.getMethodName()); - throw new IllegalStateException(msg); - } - } - - private static boolean isPortUsed(int port) { - if (port == 0) { - //Don't check if port is 0 - return false; - } - try (Socket ignored = new Socket("localhost", port)) { - return true; - } catch (IOException e) { - LOG.error("Port: {}, port check result: {}", port, e.getMessage()); - return false; - } - } + private static final ApplicationId TEST_APP_ID = ApplicationId.newInstance(1111111111111L, 1); /** * Test the validation of ShuffleHandler's meta-data's serialization and @@ -862,8 +95,8 @@ public class TestShuffleHandler { * * @throws Exception exception */ - @Test (timeout = 10000) - public void testSerializeMeta() throws Exception { + @Test(timeout = 10000) + public void testSerializeMeta() throws Exception { assertEquals(1, ShuffleHandler.deserializeMetaData( ShuffleHandler.serializeMetaData(1))); assertEquals(-1, ShuffleHandler.deserializeMetaData( @@ -877,24 +110,24 @@ public class TestShuffleHandler { * * @throws Exception exception */ - @Test (timeout = 10000) + @Test(timeout = 10000) public void testShuffleMetrics() throws Exception { MetricsSystem ms = new MetricsSystemImpl(); - ShuffleHandler sh = new ShuffleHandlerForTests(ms); + ShuffleHandler sh = new ShuffleHandler(ms); ChannelFuture cf = mock(ChannelFuture.class); when(cf.isSuccess()).thenReturn(true).thenReturn(false); sh.metrics.shuffleConnections.incr(); - sh.metrics.shuffleOutputBytes.incr(MiB); + sh.metrics.shuffleOutputBytes.incr(MIB); sh.metrics.shuffleConnections.incr(); - sh.metrics.shuffleOutputBytes.incr(2*MiB); + sh.metrics.shuffleOutputBytes.incr(2 * MIB); - checkShuffleMetrics(ms, 3*MiB, 0, 0, 2); + checkShuffleMetrics(ms, 3 * MIB, 0, 0, 2); sh.metrics.operationComplete(cf); sh.metrics.operationComplete(cf); - checkShuffleMetrics(ms, 3*MiB, 1, 1, 0); + checkShuffleMetrics(ms, 3 * MIB, 1, 1, 0); sh.stop(); } @@ -909,447 +142,32 @@ public class TestShuffleHandler { assertGauge("ShuffleConnections", connections, rb); } - /** - * Verify client prematurely closing a connection. - * - * @throws Exception exception. - */ - @Test (timeout = 10000) - public void testClientClosesConnection() throws Exception { - Configuration conf = new Configuration(); - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort()); - ShuffleHandlerForTests shuffleHandler = new ShuffleHandlerForTests() { - - @Override - protected Shuffle getShuffle(Configuration conf) { - // replace the shuffle handler with one stubbed for testing - return new Shuffle(conf) { - @Override - protected MapOutputInfo getMapOutputInfo(String mapId, int reduce, - String jobId, String user) { - return null; - } - @Override - protected void populateHeaders(List mapIds, String jobId, - String user, int reduce, HttpRequest request, - HttpResponse response, boolean keepAliveParam, - Map infoMap) { - // Only set response headers and skip everything else - // send some dummy value for content-length - super.setResponseHeaders(response, keepAliveParam, 100); - } - @Override - protected void verifyRequest(String appid, ChannelHandlerContext ctx, - HttpRequest request, HttpResponse response, URL requestUri) { - } - @Override - protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, - Channel ch, String user, String mapId, int reduce, - MapOutputInfo info) - throws IOException { - ShuffleHeader header = - new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1); - DataOutputBuffer dob = new DataOutputBuffer(); - header.write(dob); - ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); - dob = new DataOutputBuffer(); - for (int i = 0; i < 100000; ++i) { - header.write(dob); - } - return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); - } - @Override - protected void sendError(ChannelHandlerContext ctx, - HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error()); - ctx.channel().close(); - } - } - @Override - protected void sendError(ChannelHandlerContext ctx, String message, - HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error()); - ctx.channel().close(); - } - } - }; - } - }; - shuffleHandler.init(conf); - shuffleHandler.start(); - - // simulate a reducer that closes early by reading a single shuffle header - // then closing the connection - URL url = new URL("http://127.0.0.1:" - + shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY) - + "/mapOutput?job=job_12345_1&reduce=1&map=attempt_12345_1_m_1_0"); - HttpURLConnection conn = TEST_EXECUTION.openConnection(url); - conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME, - ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); - conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION, - ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); - conn.connect(); - DataInputStream input = new DataInputStream(conn.getInputStream()); - assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); - assertEquals("close", - conn.getHeaderField(HttpHeader.CONNECTION.asString())); - ShuffleHeader header = new ShuffleHeader(); - header.readFields(input); - input.close(); - - assertEquals("sendError called when client closed connection", 0, - shuffleHandler.failures.size()); - assertEquals("Should have no caught exceptions", Collections.emptyList(), - shuffleHandler.failures); - - shuffleHandler.stop(); - } - - static class LastSocketAddress { - SocketAddress lastAddress; - void setAddress(SocketAddress lastAddress) { - this.lastAddress = lastAddress; - } - SocketAddress getSocketAddres() { - return lastAddress; - } - } - - @Test(timeout = 10000) - public void testKeepAliveInitiallyEnabled() throws Exception { - Configuration conf = new Configuration(); - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort()); - conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true); - conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, - TEST_EXECUTION.getKeepAliveTimeout()); - ResponseConfig responseConfig = new ResponseConfig(HEADER_WRITE_COUNT, 0, 0); - ShuffleHandlerForKeepAliveTests shuffleHandler = new ShuffleHandlerForKeepAliveTests( - ATTEMPT_ID, responseConfig); - testKeepAliveWithHttpOk(conf, shuffleHandler, ShuffleUrlType.SIMPLE, - ShuffleUrlType.WITH_KEEPALIVE); - } - - @Test(timeout = 1000000) - public void testKeepAliveInitiallyEnabledTwoKeepAliveUrls() throws Exception { - Configuration conf = new Configuration(); - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort()); - conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true); - conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, - TEST_EXECUTION.getKeepAliveTimeout()); - ResponseConfig responseConfig = new ResponseConfig(HEADER_WRITE_COUNT, 0, 0); - ShuffleHandlerForKeepAliveTests shuffleHandler = new ShuffleHandlerForKeepAliveTests( - ATTEMPT_ID, responseConfig); - testKeepAliveWithHttpOk(conf, shuffleHandler, ShuffleUrlType.WITH_KEEPALIVE, - ShuffleUrlType.WITH_KEEPALIVE); - } - - //TODO snemeth implement keepalive test that used properly mocked ShuffleHandler - @Test(timeout = 10000) - public void testKeepAliveInitiallyDisabled() throws Exception { - Configuration conf = new Configuration(); - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort()); - conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, false); - conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, - TEST_EXECUTION.getKeepAliveTimeout()); - ResponseConfig responseConfig = new ResponseConfig(HEADER_WRITE_COUNT, 0, 0); - ShuffleHandlerForKeepAliveTests shuffleHandler = new ShuffleHandlerForKeepAliveTests( - ATTEMPT_ID, responseConfig); - testKeepAliveWithHttpOk(conf, shuffleHandler, ShuffleUrlType.WITH_KEEPALIVE, - ShuffleUrlType.WITH_KEEPALIVE); - } - - @Test(timeout = 10000) - public void testKeepAliveMultipleMapAttemptIds() throws Exception { - final int mapOutputContentLength = 11; - final int mapOutputCount = 2; - - Configuration conf = new Configuration(); - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort()); - conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true); - conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, - TEST_EXECUTION.getKeepAliveTimeout()); - ResponseConfig responseConfig = new ResponseConfig(HEADER_WRITE_COUNT, - mapOutputCount, mapOutputContentLength); - ShuffleHandlerForKeepAliveTests shuffleHandler = new ShuffleHandlerForKeepAliveTests( - ATTEMPT_ID, responseConfig); - shuffleHandler.mapOutputSender.additionalMapOutputSenderOperations = - new AdditionalMapOutputSenderOperations() { - @Override - public ChannelFuture perform(ChannelHandlerContext ctx, Channel ch) throws IOException { - File tmpFile = File.createTempFile("test", ".tmp"); - Files.write(tmpFile.toPath(), - "dummytestcontent123456".getBytes(StandardCharsets.UTF_8)); - final DefaultFileRegion partition = new DefaultFileRegion(tmpFile, 0, - mapOutputContentLength); - LOG.debug("Writing response partition: {}, channel: {}", - partition, ch.id()); - return ch.writeAndFlush(partition) - .addListener((ChannelFutureListener) future -> - LOG.debug("Finished Writing response partition: {}, channel: " + - "{}", partition, ch.id())); - } - }; - testKeepAliveWithHttpOk(conf, shuffleHandler, - ShuffleUrlType.WITH_KEEPALIVE_MULTIPLE_MAP_IDS, - ShuffleUrlType.WITH_KEEPALIVE_MULTIPLE_MAP_IDS); - } - - @Test(timeout = 10000) - public void testKeepAliveWithoutMapAttemptIds() throws Exception { - Configuration conf = new Configuration(); - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort()); - conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true); - conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, - TEST_EXECUTION.getKeepAliveTimeout()); - ResponseConfig responseConfig = new ResponseConfig(HEADER_WRITE_COUNT, 0, 0); - ShuffleHandlerForKeepAliveTests shuffleHandler = new ShuffleHandlerForKeepAliveTests( - ATTEMPT_ID, responseConfig); - shuffleHandler.setFailImmediatelyOnErrors(true); - //Closing channels caused Netty to open another channel - // so 1 request was handled with 2 separate channels, - // ultimately generating 2 * HTTP 400 errors. - // We'd like to avoid this so disabling closing the channel here. - shuffleHandler.setCloseChannelOnError(false); - testKeepAliveWithHttpBadRequest(conf, shuffleHandler, ShuffleUrlType.WITH_KEEPALIVE_NO_MAP_IDS); - } - - private void testKeepAliveWithHttpOk( - Configuration conf, - ShuffleHandlerForKeepAliveTests shuffleHandler, - ShuffleUrlType... shuffleUrlTypes) throws IOException { - testKeepAliveWithHttpStatus(conf, shuffleHandler, shuffleUrlTypes, HttpURLConnection.HTTP_OK); - } - - private void testKeepAliveWithHttpBadRequest( - Configuration conf, - ShuffleHandlerForKeepAliveTests shuffleHandler, - ShuffleUrlType... shuffleUrlTypes) throws IOException { - testKeepAliveWithHttpStatus(conf, shuffleHandler, shuffleUrlTypes, - HttpURLConnection.HTTP_BAD_REQUEST); - } - - private void testKeepAliveWithHttpStatus(Configuration conf, - ShuffleHandlerForKeepAliveTests shuffleHandler, - ShuffleUrlType[] shuffleUrlTypes, - int expectedHttpStatus) throws IOException { - if (expectedHttpStatus != HttpURLConnection.HTTP_BAD_REQUEST) { - assertTrue("Expected at least two shuffle URL types ", - shuffleUrlTypes.length >= 2); - } - shuffleHandler.init(conf); - shuffleHandler.start(); - - String[] urls = new String[shuffleUrlTypes.length]; - for (int i = 0; i < shuffleUrlTypes.length; i++) { - ShuffleUrlType url = shuffleUrlTypes[i]; - if (url == ShuffleUrlType.SIMPLE) { - urls[i] = getShuffleUrl(shuffleHandler, ATTEMPT_ID, ATTEMPT_ID); - } else if (url == ShuffleUrlType.WITH_KEEPALIVE) { - urls[i] = getShuffleUrlWithKeepAlive(shuffleHandler, ATTEMPT_ID, ATTEMPT_ID); - } else if (url == ShuffleUrlType.WITH_KEEPALIVE_MULTIPLE_MAP_IDS) { - urls[i] = getShuffleUrlWithKeepAlive(shuffleHandler, ATTEMPT_ID, ATTEMPT_ID, ATTEMPT_ID_2); - } else if (url == ShuffleUrlType.WITH_KEEPALIVE_NO_MAP_IDS) { - urls[i] = getShuffleUrlWithKeepAlive(shuffleHandler, ATTEMPT_ID); - } - } - HttpConnectionHelper connHelper; - try { - connHelper = new HttpConnectionHelper(shuffleHandler.lastSocketAddress); - connHelper.connectToUrls(urls, shuffleHandler.responseConfig, expectedHttpStatus); - if (expectedHttpStatus == HttpURLConnection.HTTP_BAD_REQUEST) { - assertEquals(1, shuffleHandler.failures.size()); - assertThat(shuffleHandler.failures.get(0).getMessage(), - CoreMatchers.containsString("Status: 400 Bad Request, " + - "message: Required param job, map and reduce")); - } - } finally { - shuffleHandler.stop(); - } - - //Verify expectations - int configuredTimeout = TEST_EXECUTION.getKeepAliveTimeout(); - int expectedTimeout = configuredTimeout < 0 ? 1 : configuredTimeout; - - connHelper.validate(connData -> { - HttpConnectionAssert.create(connData) - .expectKeepAliveWithTimeout(expectedTimeout) - .expectResponseContentLength(shuffleHandler.responseConfig.contentLengthOfResponse); - }); - if (expectedHttpStatus == HttpURLConnection.HTTP_OK) { - HttpConnectionAssert.assertKeepAliveConnectionsAreSame(connHelper); - assertEquals("Unexpected ShuffleHandler failure", Collections.emptyList(), - shuffleHandler.failures); - } - } - - @Test(timeout = 10000) - public void testSocketKeepAlive() throws Exception { - Configuration conf = new Configuration(); - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort()); - conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true); - // try setting to negative keep alive timeout. - conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, - ARBITRARY_NEGATIVE_TIMEOUT_SECONDS); - HttpURLConnection conn = null; - MockShuffleHandler2 shuffleHandler = new MockShuffleHandler2(); - AuxiliaryLocalPathHandler pathHandler = - mock(AuxiliaryLocalPathHandler.class); - when(pathHandler.getLocalPathForRead(anyString())).thenThrow( - new DiskChecker.DiskErrorException("Test")); - shuffleHandler.setAuxiliaryLocalPathHandler(pathHandler); - try { - shuffleHandler.init(conf); - shuffleHandler.start(); - - String shuffleBaseURL = "http://127.0.0.1:" - + shuffleHandler.getConfig().get( - ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY); - URL url = - new URL(shuffleBaseURL + "/mapOutput?job=job_12345_1&reduce=1&" - + "map=attempt_12345_1_m_1_0"); - conn = TEST_EXECUTION.openConnection(url); - conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME, - ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); - conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION, - ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); - conn.connect(); - int rc = conn.getResponseCode(); - conn.getInputStream(); - assertEquals(HttpURLConnection.HTTP_OK, rc); - assertTrue("socket should be set KEEP_ALIVE", - shuffleHandler.isSocketKeepAlive()); - } finally { - if (conn != null) { - conn.disconnect(); - } - shuffleHandler.stop(); - } - assertEquals("Should have no caught exceptions", - Collections.emptyList(), shuffleHandler.failures); - } - - /** - * Simulate a reducer that sends an invalid shuffle-header - sometimes a wrong - * header_name and sometimes a wrong version. - * - * @throws Exception exception - */ - @Test (timeout = 10000) - public void testIncompatibleShuffleVersion() throws Exception { - final int failureNum = 3; - Configuration conf = new Configuration(); - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort()); - ShuffleHandler shuffleHandler = new ShuffleHandlerForTests(); - shuffleHandler.init(conf); - shuffleHandler.start(); - - // simulate a reducer that closes early by reading a single shuffle header - // then closing the connection - URL url = new URL("http://127.0.0.1:" - + shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY) - + "/mapOutput?job=job_12345_1&reduce=1&map=attempt_12345_1_m_1_0"); - for (int i = 0; i < failureNum; ++i) { - HttpURLConnection conn = TEST_EXECUTION.openConnection(url); - conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME, - i == 0 ? "mapreduce" : "other"); - conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION, - i == 1 ? "1.0.0" : "1.0.1"); - conn.connect(); - assertEquals( - HttpURLConnection.HTTP_BAD_REQUEST, conn.getResponseCode()); - } - - shuffleHandler.stop(); - shuffleHandler.close(); - } - /** * Validate the limit on number of shuffle connections. - * + * * @throws Exception exception */ - @Test (timeout = 10000) + @Test(timeout = 10000) public void testMaxConnections() throws Exception { - final ArrayList failures = new ArrayList<>(); final int maxAllowedConnections = 3; final int notAcceptedConnections = 1; final int connAttempts = maxAllowedConnections + notAcceptedConnections; - - Configuration conf = new Configuration(); - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort()); - conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, maxAllowedConnections); - ShuffleHandler shuffleHandler = new ShuffleHandler() { - @Override - protected Shuffle getShuffle(Configuration conf) { - // replace the shuffle handler with one stubbed for testing - return new Shuffle(conf) { - @Override - protected MapOutputInfo getMapOutputInfo(String mapId, int reduce, - String jobId, String user) { - // Do nothing. - return null; - } - @Override - protected void populateHeaders(List mapIds, String jobId, - String user, int reduce, HttpRequest request, - HttpResponse response, boolean keepAliveParam, - Map infoMap) { - // Do nothing. - } - @Override - protected void verifyRequest(String appid, ChannelHandlerContext ctx, - HttpRequest request, HttpResponse response, URL requestUri) { - // Do nothing. - } - @Override - protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, - Channel ch, String user, String mapId, int reduce, - MapOutputInfo info) - throws IOException { - // send a shuffle header and a lot of data down the channel - // to trigger a broken pipe - ShuffleHeader header = - new ShuffleHeader("dummy_header", 5678, 5678, 1); - DataOutputBuffer dob = new DataOutputBuffer(); - header.write(dob); - ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); - dob = new DataOutputBuffer(); - for (int i=0; i<100000; ++i) { - header.write(dob); - } - return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); - } - @Override - public void exceptionCaught(ChannelHandlerContext ctx, - Throwable cause) throws Exception { - LOG.debug("ExceptionCaught"); - failures.add(cause); - super.exceptionCaught(ctx, cause); - } - }; - } - }; - shuffleHandler.setUseOutboundExceptionHandler(true); + Configuration conf = new Configuration(); + conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, maxAllowedConnections); + ShuffleHandlerMock shuffleHandler = new ShuffleHandlerMock(); shuffleHandler.init(conf); shuffleHandler.start(); + final String port = shuffleHandler.getConfig().get(SHUFFLE_PORT_CONFIG_KEY); + final SecretKey secretKey = shuffleHandler.addTestApp(); // setup connections HttpURLConnection[] conns = new HttpURLConnection[connAttempts]; for (int i = 0; i < connAttempts; i++) { - String urlString = "http://127.0.0.1:" - + shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY) - + "/mapOutput?job=job_12345_1&reduce=1&map=attempt_12345_1_m_" - + i + "_0"; - URL url = new URL(urlString); - conns[i] = TEST_EXECUTION.openConnection(url); - conns[i].setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME, - ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); - conns[i].setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION, - ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + conns[i] = createRequest( + geURL(port, TEST_JOB_ID, 0, Collections.singletonList(TEST_ATTEMPT_1), true), + secretKey); } // Try to open numerous connections @@ -1381,7 +199,7 @@ public class TestShuffleHandler { HttpURLConnection.HTTP_OK, ShuffleHandler.TOO_MANY_REQ_STATUS.code()), mapOfConnections.keySet()); - + List successfulConnections = mapOfConnections.get(HttpURLConnection.HTTP_OK); assertEquals(String.format("Expected exactly %d requests " + @@ -1405,307 +223,196 @@ public class TestShuffleHandler { assertTrue("The backoff value cannot be negative.", backoff > 0); shuffleHandler.stop(); + } - //It's okay to get a ClosedChannelException. - //All other kinds of exceptions means something went wrong - assertEquals("Should have no caught exceptions", - Collections.emptyList(), failures.stream() - .filter(f -> !(f instanceof ClosedChannelException)) - .collect(toList())); + /** + * Validate the limit on number of shuffle connections. + * + * @throws Exception exception + */ + @Test(timeout = 10000) + public void testKeepAlive() throws Exception { + Configuration conf = new Configuration(); + ShuffleHandlerMock shuffleHandler = new ShuffleHandlerMock(); + shuffleHandler.init(conf); + shuffleHandler.start(); + final String port = shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY); + final SecretKey secretKey = shuffleHandler.addTestApp(); + + HttpURLConnection conn1 = createRequest( + geURL(port, TEST_JOB_ID, 0, Collections.singletonList(TEST_ATTEMPT_1), true), + secretKey); + conn1.connect(); + verifyContent(conn1, TEST_DATA_A); + + HttpURLConnection conn2 = createRequest( + geURL(port, TEST_JOB_ID, 0, Collections.singletonList(TEST_ATTEMPT_2), true), + secretKey); + conn2.connect(); + verifyContent(conn2, TEST_DATA_B); + + HttpURLConnection conn3 = createRequest( + geURL(port, TEST_JOB_ID, 0, Collections.singletonList(TEST_ATTEMPT_3), false), + secretKey); + conn3.connect(); + verifyContent(conn3, TEST_DATA_C); + + shuffleHandler.stop(); + + List actual = matchLogs("connections=\\d+"); + assertEquals("only one connection was used", + Arrays.asList("connections=1", "connections=0"), actual); } /** * Validate the ownership of the map-output files being pulled in. The * local-file-system owner of the file should match the user component in the * - * @throws Exception exception + * @throws IOException exception */ @Test(timeout = 100000) public void testMapFileAccess() throws IOException { - final ArrayList failures = new ArrayList<>(); // This will run only in NativeIO is enabled as SecureIOUtils need it assumeTrue(NativeIO.isAvailable()); Configuration conf = new Configuration(); - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort()); - conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); - conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, - "kerberos"); + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); UserGroupInformation.setConfiguration(conf); - conf.set(YarnConfiguration.NM_LOCAL_DIRS, ABS_LOG_DIR.getAbsolutePath()); - ApplicationId appId = ApplicationId.newInstance(12345, 1); - LOG.info(appId.toString()); - String appAttemptId = "attempt_12345_1_m_1_0"; - String user = "randomUser"; - String reducerId = "0"; - List fileMap = new ArrayList<>(); - createShuffleHandlerFiles(ABS_LOG_DIR, user, appId.toString(), appAttemptId, - conf, fileMap); - ShuffleHandler shuffleHandler = new ShuffleHandler() { - @Override - protected Shuffle getShuffle(Configuration conf) { - // replace the shuffle handler with one stubbed for testing - return new Shuffle(conf) { - @Override - protected void verifyRequest(String appid, ChannelHandlerContext ctx, - HttpRequest request, HttpResponse response, URL requestUri) { - // Do nothing. - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, - Throwable cause) throws Exception { - LOG.debug("ExceptionCaught"); - failures.add(cause); - super.exceptionCaught(ctx, cause); - } - - @Override - public void channelActive(ChannelHandlerContext ctx) throws Exception { - ctx.pipeline().replace(HttpResponseEncoder.class, - "loggingResponseEncoder", - new LoggingHttpResponseEncoder(false)); - LOG.debug("Modified pipeline: {}", ctx.pipeline()); - super.channelActive(ctx); - } - }; - } - }; - AuxiliaryLocalPathHandler pathHandler = new TestAuxiliaryLocalPathHandler(); - shuffleHandler.setUseOutboundExceptionHandler(true); - shuffleHandler.setAuxiliaryLocalPathHandler(pathHandler); + ShuffleHandlerMock shuffleHandler = new ShuffleHandlerMock(); shuffleHandler.init(conf); try { shuffleHandler.start(); - DataOutputBuffer outputBuffer = new DataOutputBuffer(); - outputBuffer.reset(); - Token jt = - new Token<>("identifier".getBytes(), - "password".getBytes(), new Text(user), new Text("shuffleService")); - jt.write(outputBuffer); - shuffleHandler - .initializeApplication(new ApplicationInitializationContext(user, - appId, ByteBuffer.wrap(outputBuffer.getData(), 0, - outputBuffer.getLength()))); - URL url = - new URL( - "http://127.0.0.1:" - + shuffleHandler.getConfig().get( - ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY) - + "/mapOutput?job=job_12345_0001&reduce=" + reducerId - + "&map=attempt_12345_1_m_1_0"); - HttpURLConnection conn = TEST_EXECUTION.openConnection(url); - conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME, - ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); - conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION, - ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + final String port = shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY); + final SecretKey secretKey = shuffleHandler.addTestApp(); + + HttpURLConnection conn = createRequest( + geURL(port, TEST_JOB_ID, 0, Collections.singletonList(TEST_ATTEMPT_1), false), + secretKey); conn.connect(); - DataInputStream is = new DataInputStream(conn.getInputStream()); - InputStreamReadResult result = HttpConnectionHelper.readDataFromInputStream(is); - String receivedString = result.asString; + BufferedReader in = new BufferedReader(new InputStreamReader(conn.getInputStream())); + StringBuilder builder = new StringBuilder(); + String inputLine; + while ((inputLine = in.readLine()) != null) { + System.out.println(inputLine); + builder.append(inputLine); + } + String receivedString = builder.toString(); //Retrieve file owner name - FileInputStream fis = new FileInputStream(fileMap.get(0)); - String owner = NativeIO.POSIX.getFstat(fis.getFD()).getOwner(); - fis.close(); + String indexFilePath = getIndexFile(tempDir.toAbsolutePath().toString(), TEST_ATTEMPT_1); + String owner; + try (FileInputStream fis = new FileInputStream(indexFilePath)) { + owner = NativeIO.POSIX.getFstat(fis.getFD()).getOwner(); + } String message = - "Owner '" + owner + "' for path " + fileMap.get(0).getAbsolutePath() - + " did not match expected owner '" + user + "'"; + "Owner '" + owner + "' for path " + indexFilePath + + " did not match expected owner '" + TEST_USER + "'"; assertTrue(String.format("Received string '%s' should contain " + - "message '%s'", receivedString, message), + "message '%s'", receivedString, message), receivedString.contains(message)); assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); LOG.info("received: " + receivedString); assertNotEquals("", receivedString); } finally { shuffleHandler.stop(); - FileUtil.fullyDelete(ABS_LOG_DIR); } - - assertEquals("Should have no caught exceptions", - Collections.emptyList(), failures); - } - - private static void createShuffleHandlerFiles(File logDir, String user, - String appId, String appAttemptId, Configuration conf, - List fileMap) throws IOException { - String attemptDir = - StringUtils.join(Path.SEPARATOR, - new String[] {logDir.getAbsolutePath(), - ContainerLocalizer.USERCACHE, user, - ContainerLocalizer.APPCACHE, appId, "output", appAttemptId }); - File appAttemptDir = new File(attemptDir); - appAttemptDir.mkdirs(); - System.out.println(appAttemptDir.getAbsolutePath()); - File indexFile = new File(appAttemptDir, "file.out.index"); - fileMap.add(indexFile); - createIndexFile(indexFile, conf); - File mapOutputFile = new File(appAttemptDir, "file.out"); - fileMap.add(mapOutputFile); - createMapOutputFile(mapOutputFile, conf); - } - - private static void createMapOutputFile(File mapOutputFile, Configuration conf) - throws IOException { - FileOutputStream out = new FileOutputStream(mapOutputFile); - out.write("Creating new dummy map output file. Used only for testing" - .getBytes()); - out.flush(); - out.close(); - } - - private static void createIndexFile(File indexFile, Configuration conf) - throws IOException { - if (indexFile.exists()) { - System.out.println("Deleting existing file"); - indexFile.delete(); - } - indexFile.createNewFile(); - FSDataOutputStream output = FileSystem.getLocal(conf).getRaw().append( - new Path(indexFile.getAbsolutePath())); - Checksum crc = new PureJavaCrc32(); - crc.reset(); - CheckedOutputStream chk = new CheckedOutputStream(output, crc); - String msg = "Writing new index file. This file will be used only " + - "for the testing."; - chk.write(Arrays.copyOf(msg.getBytes(), - MapTask.MAP_OUTPUT_INDEX_RECORD_LENGTH)); - output.writeLong(chk.getChecksum().getValue()); - output.close(); } @Test public void testRecovery() throws IOException { - final String user = "someuser"; - final ApplicationId appId = ApplicationId.newInstance(12345, 1); - final JobID jobId = JobID.downgrade(TypeConverter.fromYarn(appId)); final File tmpDir = new File(System.getProperty("test.build.data", System.getProperty("java.io.tmpdir")), TestShuffleHandler.class.getName()); - ShuffleHandler shuffle = new ShuffleHandlerForTests(); - AuxiliaryLocalPathHandler pathHandler = new TestAuxiliaryLocalPathHandler(); - shuffle.setAuxiliaryLocalPathHandler(pathHandler); + ShuffleHandlerMock shuffle = new ShuffleHandlerMock(); Configuration conf = new Configuration(); - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort()); conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); - conf.set(YarnConfiguration.NM_LOCAL_DIRS, - ABS_LOG_DIR.getAbsolutePath()); // emulate aux services startup with recovery enabled shuffle.setRecoveryPath(new Path(tmpDir.toString())); - tmpDir.mkdirs(); + assertTrue(tmpDir.mkdirs()); try { shuffle.init(conf); shuffle.start(); - - // set up a shuffle token for an application - DataOutputBuffer outputBuffer = new DataOutputBuffer(); - outputBuffer.reset(); - Token jt = new Token<>( - "identifier".getBytes(), "password".getBytes(), new Text(user), - new Text("shuffleService")); - jt.write(outputBuffer); - shuffle.initializeApplication(new ApplicationInitializationContext(user, - appId, ByteBuffer.wrap(outputBuffer.getData(), 0, - outputBuffer.getLength()))); + final String port = shuffle.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY); + final SecretKey secretKey = shuffle.addTestApp(); // verify we are authorized to shuffle - int rc = getShuffleResponseCode(shuffle, jt); + int rc = getShuffleResponseCode(port, secretKey); assertEquals(HttpURLConnection.HTTP_OK, rc); // emulate shuffle handler restart shuffle.close(); - shuffle = new ShuffleHandlerForTests(); - shuffle.setAuxiliaryLocalPathHandler(pathHandler); + shuffle = new ShuffleHandlerMock(); shuffle.setRecoveryPath(new Path(tmpDir.toString())); shuffle.init(conf); shuffle.start(); // verify we are still authorized to shuffle to the old application - rc = getShuffleResponseCode(shuffle, jt); + rc = getShuffleResponseCode(port, secretKey); assertEquals(HttpURLConnection.HTTP_OK, rc); // shutdown app and verify access is lost - shuffle.stopApplication(new ApplicationTerminationContext(appId)); - rc = getShuffleResponseCode(shuffle, jt); + shuffle.stopApplication(new ApplicationTerminationContext(TEST_APP_ID)); + rc = getShuffleResponseCode(port, secretKey); assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, rc); // emulate shuffle handler restart shuffle.close(); - shuffle = new ShuffleHandlerForTests(); + shuffle = new ShuffleHandlerMock(); shuffle.setRecoveryPath(new Path(tmpDir.toString())); shuffle.init(conf); shuffle.start(); // verify we still don't have access - rc = getShuffleResponseCode(shuffle, jt); + rc = getShuffleResponseCode(port, secretKey); assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, rc); } finally { - if (shuffle != null) { - shuffle.close(); - } + shuffle.close(); FileUtil.fullyDelete(tmpDir); } } - + @Test public void testRecoveryFromOtherVersions() throws IOException { - final String user = "someuser"; - final ApplicationId appId = ApplicationId.newInstance(12345, 1); final File tmpDir = new File(System.getProperty("test.build.data", System.getProperty("java.io.tmpdir")), TestShuffleHandler.class.getName()); Configuration conf = new Configuration(); - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort()); conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); - ShuffleHandler shuffle = new ShuffleHandlerForTests(); - AuxiliaryLocalPathHandler pathHandler = new TestAuxiliaryLocalPathHandler(); - shuffle.setAuxiliaryLocalPathHandler(pathHandler); - conf.set(YarnConfiguration.NM_LOCAL_DIRS, ABS_LOG_DIR.getAbsolutePath()); + ShuffleHandlerMock shuffle = new ShuffleHandlerMock(); // emulate aux services startup with recovery enabled shuffle.setRecoveryPath(new Path(tmpDir.toString())); - tmpDir.mkdirs(); + assertTrue(tmpDir.mkdirs()); try { shuffle.init(conf); shuffle.start(); - - // set up a shuffle token for an application - DataOutputBuffer outputBuffer = new DataOutputBuffer(); - outputBuffer.reset(); - Token jt = new Token<>( - "identifier".getBytes(), "password".getBytes(), new Text(user), - new Text("shuffleService")); - jt.write(outputBuffer); - shuffle.initializeApplication(new ApplicationInitializationContext(user, - appId, ByteBuffer.wrap(outputBuffer.getData(), 0, - outputBuffer.getLength()))); + final String port = shuffle.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY); + final SecretKey secretKey = shuffle.addTestApp(); // verify we are authorized to shuffle - int rc = getShuffleResponseCode(shuffle, jt); + int rc = getShuffleResponseCode(port, secretKey); assertEquals(HttpURLConnection.HTTP_OK, rc); // emulate shuffle handler restart shuffle.close(); - shuffle = new ShuffleHandlerForTests(); - shuffle.setAuxiliaryLocalPathHandler(pathHandler); + shuffle = new ShuffleHandlerMock(); shuffle.setRecoveryPath(new Path(tmpDir.toString())); shuffle.init(conf); shuffle.start(); // verify we are still authorized to shuffle to the old application - rc = getShuffleResponseCode(shuffle, jt); + rc = getShuffleResponseCode(port, secretKey); assertEquals(HttpURLConnection.HTTP_OK, rc); Version version = Version.newInstance(1, 0); assertEquals(version, shuffle.getCurrentVersion()); - + // emulate shuffle handler restart with compatible version Version version11 = Version.newInstance(1, 1); // update version info before close shuffle shuffle.storeVersion(version11); assertEquals(version11, shuffle.loadVersion()); shuffle.close(); - shuffle = new ShuffleHandlerForTests(); - shuffle.setAuxiliaryLocalPathHandler(pathHandler); + shuffle = new ShuffleHandlerMock(); shuffle.setRecoveryPath(new Path(tmpDir.toString())); shuffle.init(conf); shuffle.start(); @@ -1713,309 +420,99 @@ public class TestShuffleHandler { // successfully. assertEquals(version, shuffle.loadVersion()); // verify we are still authorized to shuffle to the old application - rc = getShuffleResponseCode(shuffle, jt); + rc = getShuffleResponseCode(port, secretKey); assertEquals(HttpURLConnection.HTTP_OK, rc); - + // emulate shuffle handler restart with incompatible version Version version21 = Version.newInstance(2, 1); shuffle.storeVersion(version21); assertEquals(version21, shuffle.loadVersion()); shuffle.close(); - shuffle = new ShuffleHandlerForTests(); - shuffle.setAuxiliaryLocalPathHandler(pathHandler); + shuffle = new ShuffleHandlerMock(); shuffle.setRecoveryPath(new Path(tmpDir.toString())); shuffle.init(conf); - + try { shuffle.start(); fail("Incompatible version, should expect fail here."); } catch (ServiceStateException e) { assertTrue("Exception message mismatch", e.getMessage().contains("Incompatible version for state DB schema:")); - } - - } finally { - if (shuffle != null) { - shuffle.close(); } + + } finally { + shuffle.close(); FileUtil.fullyDelete(tmpDir); } } - private static int getShuffleResponseCode(ShuffleHandler shuffle, - Token jt) throws IOException { - URL url = new URL("http://127.0.0.1:" - + shuffle.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY) - + "/mapOutput?job=job_12345_0001&reduce=0&map=attempt_12345_1_m_1_0"); - HttpURLConnection conn = TEST_EXECUTION.openConnection(url); - String encHash = SecureShuffleUtils.hashFromString( - SecureShuffleUtils.buildMsgFrom(url), - JobTokenSecretManager.createSecretKey(jt.getPassword())); - conn.addRequestProperty( - SecureShuffleUtils.HTTP_HEADER_URL_HASH, encHash); - conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME, - ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); - conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION, - ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + private static void verifyContent(HttpURLConnection conn, + String expectedContent) throws IOException { + DataInputStream input = new DataInputStream(conn.getInputStream()); + ShuffleHeader header = new ShuffleHeader(); + header.readFields(input); + byte[] data = new byte[expectedContent.length()]; + assertEquals(expectedContent.length(), input.read(data)); + assertEquals(expectedContent, new String(data)); + } + + private static int getShuffleResponseCode(String port, SecretKey key) throws IOException { + HttpURLConnection conn = createRequest( + geURL(port, TEST_JOB_ID, 0, Collections.singletonList(TEST_ATTEMPT_1), false), + key); conn.connect(); int rc = conn.getResponseCode(); conn.disconnect(); return rc; } - @Test(timeout = 100000) - public void testGetMapOutputInfo() throws Exception { - final ArrayList failures = new ArrayList<>(1); - Configuration conf = new Configuration(); - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort()); - conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); - conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, - "simple"); - UserGroupInformation.setConfiguration(conf); - conf.set(YarnConfiguration.NM_LOCAL_DIRS, ABS_LOG_DIR.getAbsolutePath()); - ApplicationId appId = ApplicationId.newInstance(12345, 1); - String appAttemptId = "attempt_12345_1_m_1_0"; - String user = "randomUser"; - String reducerId = "0"; - List fileMap = new ArrayList<>(); - createShuffleHandlerFiles(ABS_LOG_DIR, user, appId.toString(), appAttemptId, - conf, fileMap); - AuxiliaryLocalPathHandler pathHandler = new TestAuxiliaryLocalPathHandler(); - ShuffleHandler shuffleHandler = new ShuffleHandler() { - @Override - protected Shuffle getShuffle(Configuration conf) { - // replace the shuffle handler with one stubbed for testing - return new Shuffle(conf) { - @Override - protected void populateHeaders(List mapIds, - String outputBaseStr, String user, int reduce, - HttpRequest request, HttpResponse response, - boolean keepAliveParam, Map infoMap) - throws IOException { - // Only set response headers and skip everything else - // send some dummy value for content-length - super.setResponseHeaders(response, keepAliveParam, 100); - } - @Override - protected void verifyRequest(String appid, - ChannelHandlerContext ctx, HttpRequest request, - HttpResponse response, URL requestUri) { - // Do nothing. - } - @Override - protected void sendError(ChannelHandlerContext ctx, String message, - HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error(message)); - ctx.channel().close(); - } - } - @Override - protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, - Channel ch, String user, String mapId, int reduce, - MapOutputInfo info) throws IOException { - // send a shuffle header - ShuffleHeader header = - new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1); - DataOutputBuffer dob = new DataOutputBuffer(); - header.write(dob); - return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); - } - }; - } - }; - shuffleHandler.setUseOutboundExceptionHandler(true); - shuffleHandler.setAuxiliaryLocalPathHandler(pathHandler); - shuffleHandler.init(conf); - try { - shuffleHandler.start(); + private static URL geURL(String port, String jobId, int reduce, List maps, + boolean keepAlive) throws MalformedURLException { + return new URL(getURLString(port, getUri(jobId, reduce, maps, keepAlive))); + } + + private static String getURLString(String port, String uri) { + return String.format("http://127.0.0.1:%s%s", port, uri); + } + + private static HttpURLConnection createRequest(URL url, SecretKey secretKey) throws IOException { + HttpURLConnection connection = (HttpURLConnection) url.openConnection(); + connection.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME, + ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); + connection.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION, + ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + String msgToEncode = SecureShuffleUtils.buildMsgFrom(url); + connection.setRequestProperty(HTTP_HEADER_URL_HASH, + SecureShuffleUtils.hashFromString(msgToEncode, secretKey)); + return connection; + } + + class ShuffleHandlerMock extends ShuffleHandler { + + public SecretKey addTestApp() throws IOException { DataOutputBuffer outputBuffer = new DataOutputBuffer(); outputBuffer.reset(); - Token jt = - new Token<>("identifier".getBytes(), - "password".getBytes(), new Text(user), new Text("shuffleService")); + Token jt = new Token<>( + "identifier".getBytes(), "password".getBytes(), new Text(TEST_USER), + new Text("shuffleService")); jt.write(outputBuffer); - shuffleHandler - .initializeApplication(new ApplicationInitializationContext(user, - appId, ByteBuffer.wrap(outputBuffer.getData(), 0, - outputBuffer.getLength()))); - URL url = - new URL( - "http://127.0.0.1:" - + shuffleHandler.getConfig().get( - ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY) - + "/mapOutput?job=job_12345_0001&reduce=" + reducerId - + "&map=attempt_12345_1_m_1_0"); - HttpURLConnection conn = TEST_EXECUTION.openConnection(url); - conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME, - ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); - conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION, - ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); - conn.connect(); - try { - DataInputStream is = new DataInputStream(conn.getInputStream()); - ShuffleHeader header = new ShuffleHeader(); - header.readFields(is); - is.close(); - } catch (EOFException e) { - // ignore - } - assertEquals("sendError called due to shuffle error", - 0, failures.size()); - } finally { - shuffleHandler.stop(); - FileUtil.fullyDelete(ABS_LOG_DIR); - } - } + initializeApplication(new ApplicationInitializationContext(TEST_USER, TEST_APP_ID, + ByteBuffer.wrap(outputBuffer.getData(), 0, + outputBuffer.getLength()))); - @Test(timeout = 4000) - public void testSendMapCount() throws Exception { - final List listenerList = - new ArrayList<>(); - int connectionKeepAliveTimeOut = 5; //arbitrary value - final ChannelHandlerContext mockCtx = - mock(ChannelHandlerContext.class); - final Channel mockCh = mock(AbstractChannel.class); - final ChannelPipeline mockPipeline = mock(ChannelPipeline.class); - - // Mock HttpRequest and ChannelFuture - final HttpRequest mockHttpRequest = createMockHttpRequest(); - final ChannelFuture mockFuture = createMockChannelFuture(mockCh, - listenerList); - final ShuffleHandler.TimeoutHandler timerHandler = - new ShuffleHandler.TimeoutHandler(connectionKeepAliveTimeOut); - - // Mock Netty Channel Context and Channel behavior - Mockito.doReturn(mockCh).when(mockCtx).channel(); - when(mockCh.pipeline()).thenReturn(mockPipeline); - when(mockPipeline.get( - Mockito.any(String.class))).thenReturn(timerHandler); - when(mockCtx.channel()).thenReturn(mockCh); - Mockito.doReturn(mockFuture).when(mockCh).writeAndFlush(Mockito.any(Object.class)); - - final MockShuffleHandler sh = new MockShuffleHandler(); - Configuration conf = new Configuration(); - sh.init(conf); - sh.start(); - int maxOpenFiles =conf.getInt(ShuffleHandler.SHUFFLE_MAX_SESSION_OPEN_FILES, - ShuffleHandler.DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES); - sh.getShuffle(conf).channelRead(mockCtx, mockHttpRequest); - assertTrue("Number of Open files should not exceed the configured " + - "value!-Not Expected", - listenerList.size() <= maxOpenFiles); - while(!listenerList.isEmpty()) { - listenerList.remove(0).operationComplete(mockFuture); - assertTrue("Number of Open files should not exceed the configured " + - "value!-Not Expected", - listenerList.size() <= maxOpenFiles); - } - sh.close(); - sh.stop(); - - assertEquals("Should have no caught exceptions", - Collections.emptyList(), sh.failures); - } - - @Test(timeout = 10000) - public void testIdleStateHandlingSpecifiedTimeout() throws Exception { - int timeoutSeconds = 4; - int expectedTimeoutSeconds = timeoutSeconds; - testHandlingIdleState(timeoutSeconds, expectedTimeoutSeconds); - } - - @Test(timeout = 10000) - public void testIdleStateHandlingNegativeTimeoutDefaultsTo1Second() throws Exception { - int expectedTimeoutSeconds = 1; //expected by production code - testHandlingIdleState(ARBITRARY_NEGATIVE_TIMEOUT_SECONDS, expectedTimeoutSeconds); - } - - private String getShuffleUrlWithKeepAlive(ShuffleHandler shuffleHandler, long jobId, - long... attemptIds) { - String url = getShuffleUrl(shuffleHandler, jobId, attemptIds); - return url + "&keepAlive=true"; - } - - private String getShuffleUrl(ShuffleHandler shuffleHandler, long jobId, long... attemptIds) { - String port = shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY); - String shuffleBaseURL = "http://127.0.0.1:" + port; - - StringBuilder mapAttemptIds = new StringBuilder(); - for (int i = 0; i < attemptIds.length; i++) { - if (i == 0) { - mapAttemptIds.append("&map="); - } else { - mapAttemptIds.append(","); - } - mapAttemptIds.append(String.format("attempt_%s_1_m_1_0", attemptIds[i])); + return JobTokenSecretManager.createSecretKey(jt.getPassword()); } - String location = String.format("/mapOutput" + - "?job=job_%s_1" + - "&reduce=1" + - "%s", jobId, mapAttemptIds); - return shuffleBaseURL + location; + @Override + protected ShuffleChannelHandlerContext createHandlerContext() { + return new ShuffleChannelHandlerContext(getConfig(), + userRsrc, + secretManager, + createLoadingCache(), + new IndexCache(new JobConf(getConfig())), + ms.register(new ShuffleHandler.ShuffleMetrics()), + allChannels + ); + } } - - private void testHandlingIdleState(int configuredTimeoutSeconds, int expectedTimeoutSeconds) - throws IOException, - InterruptedException { - Configuration conf = new Configuration(); - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort()); - conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true); - conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, configuredTimeoutSeconds); - - final CountDownLatch countdownLatch = new CountDownLatch(1); - ResponseConfig responseConfig = new ResponseConfig(HEADER_WRITE_COUNT, 0, 0); - ShuffleHandlerForKeepAliveTests shuffleHandler = new ShuffleHandlerForKeepAliveTests( - ATTEMPT_ID, responseConfig, - event -> countdownLatch.countDown()); - shuffleHandler.init(conf); - shuffleHandler.start(); - - String shuffleUrl = getShuffleUrl(shuffleHandler, ATTEMPT_ID, ATTEMPT_ID); - String[] urls = new String[] {shuffleUrl}; - HttpConnectionHelper httpConnectionHelper = new HttpConnectionHelper( - shuffleHandler.lastSocketAddress); - long beforeConnectionTimestamp = System.currentTimeMillis(); - httpConnectionHelper.connectToUrls(urls, shuffleHandler.responseConfig); - countdownLatch.await(); - long channelClosedTimestamp = System.currentTimeMillis(); - long secondsPassed = - TimeUnit.SECONDS.convert(channelClosedTimestamp - beforeConnectionTimestamp, - TimeUnit.MILLISECONDS); - assertTrue(String.format("Expected at least %s seconds of timeout. " + - "Actual timeout seconds: %s", expectedTimeoutSeconds, secondsPassed), - secondsPassed >= expectedTimeoutSeconds); - shuffleHandler.stop(); - } - - public ChannelFuture createMockChannelFuture(Channel mockCh, - final List listenerList) { - final ChannelFuture mockFuture = mock(ChannelFuture.class); - when(mockFuture.channel()).thenReturn(mockCh); - Mockito.doReturn(true).when(mockFuture).isSuccess(); - Mockito.doAnswer(invocation -> { - //Add ReduceMapFileCount listener to a list - if (invocation.getArguments()[0].getClass() == ShuffleHandler.ReduceMapFileCount.class) { - listenerList.add((ShuffleHandler.ReduceMapFileCount) - invocation.getArguments()[0]); - } - return null; - }).when(mockFuture).addListener(Mockito.any( - ShuffleHandler.ReduceMapFileCount.class)); - return mockFuture; - } - - public HttpRequest createMockHttpRequest() { - HttpRequest mockHttpRequest = mock(HttpRequest.class); - Mockito.doReturn(HttpMethod.GET).when(mockHttpRequest).method(); - Mockito.doAnswer(invocation -> { - String uri = "/mapOutput?job=job_12345_1&reduce=1"; - for (int i = 0; i < 100; i++) { - uri = uri.concat("&map=attempt_12345_1_m_" + i + "_0"); - } - return uri; - }).when(mockHttpRequest).uri(); - return mockHttpRequest; - } -} +} \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandlerBase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandlerBase.java new file mode 100644 index 00000000000..1bce443381d --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandlerBase.java @@ -0,0 +1,172 @@ +/* + * 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.mapred; + +import io.netty.util.ResourceLeakDetector; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.PrintStream; +import java.io.PrintWriter; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import javax.annotation.Nonnull; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder; +import org.apache.hadoop.thirdparty.com.google.common.cache.CacheLoader; +import org.apache.hadoop.thirdparty.com.google.common.cache.LoadingCache; +import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalListener; +import org.junit.After; +import org.junit.Before; + +import static io.netty.util.ResourceLeakDetector.Level.PARANOID; +import static org.apache.hadoop.io.MapFile.DATA_FILE_NAME; +import static org.apache.hadoop.io.MapFile.INDEX_FILE_NAME; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TestShuffleHandlerBase { + public static final String TEST_ATTEMPT_1 = "attempt_1111111111111_0001_m_000001_0"; + public static final String TEST_ATTEMPT_2 = "attempt_1111111111111_0002_m_000002_0"; + public static final String TEST_ATTEMPT_3 = "attempt_1111111111111_0003_m_000003_0"; + public static final String TEST_JOB_ID = "job_1111111111111_0001"; + public static final String TEST_USER = "testUser"; + public static final String TEST_DATA_A = "aaaaa"; + public static final String TEST_DATA_B = "bbbbb"; + public static final String TEST_DATA_C = "ccccc"; + + private final PrintStream standardOut = System.out; + private final ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); + @SuppressWarnings("checkstyle:VisibilityModifier") + protected java.nio.file.Path tempDir; + + @Before + public void setup() throws IOException { + tempDir = Files.createTempDirectory("test-shuffle-channel-handler"); + tempDir.toFile().deleteOnExit(); + + generateMapOutput(tempDir.toAbsolutePath().toString(), TEST_ATTEMPT_1, + Arrays.asList(TEST_DATA_A, TEST_DATA_B, TEST_DATA_C)); + generateMapOutput(tempDir.toAbsolutePath().toString(), TEST_ATTEMPT_2, + Arrays.asList(TEST_DATA_B, TEST_DATA_A, TEST_DATA_C)); + generateMapOutput(tempDir.toAbsolutePath().toString(), TEST_ATTEMPT_3, + Arrays.asList(TEST_DATA_C, TEST_DATA_B, TEST_DATA_A)); + + outputStreamCaptor.reset(); + ResourceLeakDetector.setLevel(PARANOID); + System.setOut(new PrintStream(outputStreamCaptor)); + } + + @After + public void teardown() { + System.setOut(standardOut); + System.out.print(outputStreamCaptor); + // For this to work ch.qos.logback.classic is needed for some reason + assertFalse(outputStreamCaptor.toString() + .contains("LEAK: ByteBuf.release() was not called before")); + } + + public List matchLogs(String pattern) { + String logs = outputStreamCaptor.toString(); + Matcher m = Pattern.compile(pattern).matcher(logs); + List allMatches = new ArrayList<>(); + while (m.find()) { + allMatches.add(m.group()); + } + return allMatches; + } + + public static void generateMapOutput(String tempDir, String attempt, List maps) + throws IOException { + SpillRecord record = new SpillRecord(maps.size()); + + assertTrue(new File(getBasePath(tempDir, attempt)).mkdirs()); + try (PrintWriter writer = new PrintWriter(getDataFile(tempDir, attempt), "UTF-8")) { + long startOffset = 0; + int partition = 0; + for (String map : maps) { + record.putIndex(new IndexRecord( + startOffset, + map.length() * 2L, // doesn't matter in this test + map.length()), + partition); + startOffset += map.length() + 1; + partition++; + writer.write(map); + } + record.writeToFile(new Path(getIndexFile(tempDir, attempt)), + new JobConf(new Configuration())); + } + } + + public static String getIndexFile(String tempDir, String attempt) { + return String.format("%s/%s", getBasePath(tempDir, attempt), INDEX_FILE_NAME); + } + + public static String getDataFile(String tempDir, String attempt) { + return String.format("%s/%s", getBasePath(tempDir, attempt), DATA_FILE_NAME); + } + + private static String getBasePath(String tempDir, String attempt) { + return String.format("%s/%s/%s/%s", tempDir, TEST_JOB_ID, TEST_USER, attempt); + } + + public static String getUri(String jobId, int reduce, List maps, boolean keepAlive) { + return String.format("/mapOutput?job=%s&reduce=%d&map=%s%s", + jobId, reduce, String.join(",", maps), + keepAlive ? "&keepAlive=true" : ""); + } + + public LoadingCache createLoadingCache() { + return CacheBuilder.newBuilder().expireAfterAccess( + 5, + TimeUnit.MINUTES).softValues().concurrencyLevel(16). + removalListener( + (RemovalListener) notification -> { + } + ).maximumWeight(10 * 1024 * 1024).weigher( + (key, value) -> key.jobId.length() + key.user.length() + + key.attemptId.length() + + value.indexPath.toString().length() + + value.dataPath.toString().length() + ).build(new CacheLoader() { + @Override + public ShuffleHandler.AttemptPathInfo load( + @Nonnull ShuffleHandler.AttemptPathIdentifier key) { + String base = String.format("%s/%s/%s/", tempDir, key.jobId, key.user); + String attemptBase = base + key.attemptId; + Path indexFileName = new Path(attemptBase + "/" + INDEX_FILE_NAME); + Path mapOutputFileName = new Path(attemptBase + "/" + DATA_FILE_NAME); + return new ShuffleHandler.AttemptPathInfo(indexFileName, mapOutputFileName); + } + }); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/cert.pem b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/cert.pem new file mode 100644 index 00000000000..ec32a67152a --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/cert.pem @@ -0,0 +1,27 @@ +-----BEGIN CERTIFICATE----- +MIIEpDCCAowCCQDDMEtH5Wp0qTANBgkqhkiG9w0BAQsFADAUMRIwEAYDVQQDDAls +b2NhbGhvc3QwHhcNMjMwMTE2MTI0NjQ4WhcNMzMwMTEzMTI0NjQ4WjAUMRIwEAYD +VQQDDAlsb2NhbGhvc3QwggIiMA0GCSqGSIb3DQEBAQUAA4ICDwAwggIKAoICAQDO +FiF+sfoJYHPMPx4jaU11mCupytAFJzz9igaiaKAZCjVHBVWC31KDxHmRdKD066DO +clOJORNOe8Oe4aB5Lbu6wgKtlHEtKmqAU2WrYAEl0oXrZKEL0Xgs1KTTChbVSJ/I +m1WwmEthriQSul0WaEncNpS5NV4PORhiGu0plw+SWSJBFsbl29K6oHE1ClgVjm8j +iu4Y1NAilOPcjmhCmwRq5eq5H0mJ5LWxfvjLIJ9cPpMLG9eVLQkOIE9I01DJ37WM +OvljUMpmhxWDq2oZEmeyCJUFSUh1IlcUM1hTmRUzU/Vcf7EhpAYZxphvSIvDQkAw +cmnn0LQZmORCMP0HurR1o3NnzAVf/ahfpXwvA/BuCsEcW1Le+WATtxa2EvRCnEPa +I76W35FY69t/WYZNIzPgo9eYD7iDBbqxuBH+GlDuwWU6mjEc0nL11uGtcRPrXzKa +QhRMqAtwNW5I5S5HgPLbMiu/n+PpX6+S431eLHFHJ6WUvcwOIK4ZqLH4/Piks1fV +0Svdo47Jymlt6dOvYm85tFsWkYcmldO6aQilRuGBYdXJ06xDyH7EaD0Z2PmPjhl9 +zkt3gpaXxBn0gsJIn++qZ26pXFxVewlJi0m84Xd3x10h9MvpP8AZMhFkLWXR2nqw +eCfell4jzGNXBDLEcspv6HmuTvP7+gqgRCuFLrjOiQIDAQABMA0GCSqGSIb3DQEB +CwUAA4ICAQAexU5VTmT5VAqau0TGTGEgStGPWoliV4b+d8AcdJvOd1qmetgFhJ+X +TSutcFPdascOys0+tUV2GZwew3P8yTQyd35LDUC4OjGt2kISDplGAtTdDD19u5/R +hQf7VxJImOxsg2lPPRv3RXMbNF64reP9YsM9osWgJOFzrDf9FkP2HByslU2v7ler +sWQVu+Ei7r3/ZMOI7hHnN8MLqcj+BJwEHCTa8HPmr0Ic3lJ86vUVR4QQE5LgNvSu +oSOZlALsMNVx2rxmirhC6guLwPh7HylDFMzyVedCzmqwQ0R8SSU6SmJvXNLeBFLw +F5mZRh1jabiqsMTGnmMQ97GPs0q78M2sw3TjI+nexCcYZ3jQfR+1eFSg4DlSd55x +BMVfT2kYThzxOw3brtygXjl6gGr8v5M6PzOvbLyDtEa3iDp7Mslw2tJ5OmxxJV9g +QVvBQL1L2nySFk0ij2bIjD7fdpF/EpxrNf4IATOAf5YvxELUeXnyuqJZBtgC8b3I +qXHJIpGM7N4jdwhe0sMVH0OWlqzsL14QZCE6YdvXBEksJ/HBVUie6afYAZrUwUP1 +gtcq9uFpPteg9PsBLZ7hGfNt2278EXhPBtlIpeiPE8X19Lr3bCmCO1PbWNCTkweb +tGfwnH46DmWYUqYrofnKso1mq56yEbbuDy7a2FeHJ2d+18Fh97WnUw== +-----END CERTIFICATE----- diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/key.pem b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/key.pem new file mode 100644 index 00000000000..e064e5e8d03 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/key.pem @@ -0,0 +1,52 @@ +-----BEGIN PRIVATE KEY----- +MIIJRAIBADANBgkqhkiG9w0BAQEFAASCCS4wggkqAgEAAoICAQDOFiF+sfoJYHPM +Px4jaU11mCupytAFJzz9igaiaKAZCjVHBVWC31KDxHmRdKD066DOclOJORNOe8Oe +4aB5Lbu6wgKtlHEtKmqAU2WrYAEl0oXrZKEL0Xgs1KTTChbVSJ/Im1WwmEthriQS +ul0WaEncNpS5NV4PORhiGu0plw+SWSJBFsbl29K6oHE1ClgVjm8jiu4Y1NAilOPc +jmhCmwRq5eq5H0mJ5LWxfvjLIJ9cPpMLG9eVLQkOIE9I01DJ37WMOvljUMpmhxWD +q2oZEmeyCJUFSUh1IlcUM1hTmRUzU/Vcf7EhpAYZxphvSIvDQkAwcmnn0LQZmORC +MP0HurR1o3NnzAVf/ahfpXwvA/BuCsEcW1Le+WATtxa2EvRCnEPaI76W35FY69t/ +WYZNIzPgo9eYD7iDBbqxuBH+GlDuwWU6mjEc0nL11uGtcRPrXzKaQhRMqAtwNW5I +5S5HgPLbMiu/n+PpX6+S431eLHFHJ6WUvcwOIK4ZqLH4/Piks1fV0Svdo47Jymlt +6dOvYm85tFsWkYcmldO6aQilRuGBYdXJ06xDyH7EaD0Z2PmPjhl9zkt3gpaXxBn0 +gsJIn++qZ26pXFxVewlJi0m84Xd3x10h9MvpP8AZMhFkLWXR2nqweCfell4jzGNX +BDLEcspv6HmuTvP7+gqgRCuFLrjOiQIDAQABAoICAQDAe6UfK2YIugCN5OnmUyUY +z18AwD/YgFSTzgXyTNwzZvhp9A5xJNpx3eFZvN/Uwfs4t0lUom1o4WnNjJkQdWmg +vjI4I6wtbi942evcy9dmlyGjwSI14phm7tlfj03SOXmbqZG4VhYaDsb8gvoMwq0x +s/zmm3TVrRMcFmAqd0ABBaVbu8VbzRweWVpDGv04bQda4BkQMjyQABZu2seAZj8T +BNldvF44H9igBqKjPj35rywxtPh/CUgq3HyQ3WXYl0x+xFpHq57Pch3jFAgNkMYv +X5qoDFFTrhY89NPriNBnV2SU12L+s69LBdU8Izr+zXMcjNBjxudf/RA8znqWbIi8 +pbwXOwBUD4XP3coAzipVOJfeXb5OAkq+wjHnKb4YXJ5mNFb7LcRy6MJVqyxPNJGh +UlfGxsJ441K/9e+aoDEyB0xbjeZ+2yP021P2eObwj46M5sxP2BFSe8E1DUpQ5+ZX +kKipKDZETLc2e4G37Hziw2Wa5X0AAbKgSh1a5AMd0GUbrmJQzO0dok1ujJNu+zpn +K0qAG0c/HD+egIKPEc03+81fLzXKxGHOxrTnHPInWLeGjxEL3oM2Tl5QkYSjm8qg +uIY5up5K//R+fDy45/XRACPOo+yf2RTFnIjfIhxJaB2M7BrBUpWvX1xLJQfDS3Gb +4Rfo2Qlgh/adrNkr2m0NHQKCAQEA8KZK7eugKV/Gk5L0j5E59qytlVZTUoDWdbAq +vMnAgU6BGiTy/Av4jPCH5HDYD5ZX5nqD+GVkXSh2fiT8eSpgtIuBEdeiHHZXwCcb +IK7vKxSePQrs0aA53uk7LY0LGPMRhDheYTItTE+6pRp2HswDgRBw+1fm6Yt1ps32 +oqM7bOUSg6eCKISmAP8UV9ac1l6ZHLdhTIzrVnOc/YqIW4ruNbwpSK1fI7uTWH4i +5JqfPtTa7anJrt080vVUi6cS22G8QhlW3q6fo1GrH8QM4gInRF/4MwkAAP8p1787 +KlFHXxS0bWnJZhjKvh7naeVJi5EaMCWJ1gKF/OcvQYONrA6zdwKCAQEA2ztvxaPy +j4Pq2wpYWlHueCPPn5yMDQQqCLlcG50HzPbquSdZe7o0MEWqV8MaXB6fw1xLwCC4 +i5+HnL72KaBu6DVIhMYDmPzhiw4GbCASfR4v/biQ+047KfnQiHPUEhUCxGvHhcDR +Y3Zhzax6mC79Mfz2gunEx2ZI1MURn/sO+3tQtx+Gzsoj/W4LHpeEQGKsUhcIN48v +XAmeWqVwwmr0twQygnOQyvgZxtiunnIADVxJJh4AQLWGagDiMjaWJ4fZ7q8aBMLY +SFBlDqzf5xssraUAiaawsaRL0jliy0y8WXwSJHb4WKebH2QQcUq22c2y8IbKMcsz +AjLHf1nG0oEN/wKCAQEAypfkutnEEzDbVz+/feIQPzfuRqvPJ8TpR1jREfBzjSiP +DXiPy1s0m0qfzNSnQLAcPh9kXMjXLooy/02Z81OW6EgNl/zGMFn80/rYHrLHDKil +8tPwvSW7sor9VALKle2EEKD367G3augwRHC7gn/Ub2JtC1xcPL84g/4fJZpwG+PZ +q1ZpAD10F6Cm+q/lh59KHV/QnQaB1V0tjFGFLDQRCNHom5PBZa6zhCtkqrn1WIsP +6EcpUHpWi28YBx3XhTOJrsfwVzYBlRfbDboZ8mdHsYttw2FPTIeELWqDn8OfZ09h ++j6126sBe/8+aOsr+EBnIKNpn+6t6FSkdu4OZgxWTwKCAQEAxjRXWjVeUBgxFfWl +aYsXcXDW/nP3PrFcG2UR/DkdW1aFYf5MbYssMdRaLFxNEanyQRrJtZsfncQORE11 +mq7cdn3t4XRwvjkq5KA6eDkK3imTs+zQzxOsc8fSm/s0aWCrjs/upGNuK2FVDTD5 +6WraKP6OFE+rQ6ebAxpkU+IUapLTp6wOIhkpLq/1x6OuwtMy/kiqeiiu5aQgkc1v +Q6aVNn3J+Jzo9EgYbelq/f8NQwcDbz3Cdr5nFqFT35fway7sflm6yUErbz2YEAuF +ppiv7RH3iXu11fU3Q4n0Yt8ujiyY7nTNFMH7ggbiwrqM1B+fvsvuM9SFemBUczPE +iH57GwKCAQAdLm1mSeUPn3qEXy/ui7M7GPK43r1l2fn6UJhEGckm4YJ2DOlWisNW +2ilyzfdlYF1Cq22iKxi3/mZdNojKKL7yFCTwx2evHsSIt2vcyD25sFVh5u9O/xFa +1Zk3Pzq6XpaAfZCY4OizJb5zraWYWVNAP1DI4qT0Kg6LvNWZ5G9Dh+tptTmB9E05 +5GiBWD3OfWH5AMQ2UmprEivbaqN8Gm/W6m6Hraf+LbP4aFORwElNAZTymeNcW5O5 +ha2XU2TAINmhgPm1IZEGiSah+A+s2uW4Ox4nQJfksy+rtJOPRcnK4aIhURhzwJv/ +8JszrQ2Tq9fN/cO50CDeipqAtKkcWNjE +-----END PRIVATE KEY----- diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/log4j.properties b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/log4j.properties index b7d8ad36efc..471993fd590 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/log4j.properties +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/log4j.properties @@ -17,5 +17,5 @@ log4j.threshold=ALL log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n -log4j.logger.io.netty=INFO -log4j.logger.org.apache.hadoop.mapred=INFO \ No newline at end of file +log4j.logger.io.netty=TRACE +log4j.logger.org.apache.hadoop.mapred=TRACE \ No newline at end of file From 26fba8701c97928bb2ed2e6b456ab5ba9513e0fe Mon Sep 17 00:00:00 2001 From: Owen O'Malley Date: Fri, 10 Feb 2023 09:51:03 -0800 Subject: [PATCH 33/50] HDFS-18324. Fix race condition in closing IPC connections. (#5371) --- .../src/main/java/org/apache/hadoop/ipc/Client.java | 9 ++++++++- .../src/test/java/org/apache/hadoop/ipc/TestIPC.java | 2 +- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java index c0f90d98bc6..c52af06c5ad 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java @@ -1181,7 +1181,14 @@ public class Client implements AutoCloseable { final ResponseBuffer buf = new ResponseBuffer(); header.writeDelimitedTo(buf); RpcWritable.wrap(call.rpcRequest).writeTo(buf); - rpcRequestQueue.put(Pair.of(call, buf)); + // Wait for the message to be sent. We offer with timeout to + // prevent a race condition between checking the shouldCloseConnection + // and the stopping of the polling thread + while (!shouldCloseConnection.get()) { + if (rpcRequestQueue.offer(Pair.of(call, buf), 1, TimeUnit.SECONDS)) { + break; + } + } } /* Receive a response. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java index 25c69765494..b65f86a0f7b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java @@ -1336,7 +1336,7 @@ public class TestIPC { /** * Test the retry count while used in a retry proxy. */ - @Test(timeout=60000) + @Test(timeout=100000) public void testRetryProxy() throws IOException { final Client client = new Client(LongWritable.class, conf); From f3c4277576554dd4be45dc8c6cdfebcbfe8c0e71 Mon Sep 17 00:00:00 2001 From: hfutatzhanghb <1036798979@qq.com> Date: Sat, 11 Feb 2023 08:00:43 +0800 Subject: [PATCH 34/50] HDFS-16882. RBF: Add cache hit rate metric in MountTableResolver#getDestinationForPath (#5276) Reviewed-by: Inigo Goiri Signed-off-by: Tao Li --- .../federation/metrics/StateStoreMetrics.java | 14 +++++++ .../resolver/MountTableResolver.java | 22 ++++++++++- .../server/federation/store/RecordStore.java | 2 +- .../resolver/TestMountTableResolver.java | 39 ++++++++++++++++++- 4 files changed, 73 insertions(+), 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/StateStoreMetrics.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/StateStoreMetrics.java index f3ca4752ef7..371b33e05e2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/StateStoreMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/StateStoreMetrics.java @@ -30,6 +30,7 @@ import org.apache.hadoop.metrics2.annotation.Metrics; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.metrics2.lib.MetricsRegistry; import org.apache.hadoop.metrics2.lib.MutableGaugeInt; +import org.apache.hadoop.metrics2.lib.MutableGaugeLong; import org.apache.hadoop.metrics2.lib.MutableRate; import org.apache.hadoop.classification.VisibleForTesting; @@ -136,6 +137,19 @@ public class StateStoreMetrics implements StateStoreMBean { counter.set(size); } + /** + * set the count of the location cache access information. + * @param name Name of the record. + * @param count count of the record. + */ + public void setLocationCache(String name, long count) { + MutableGaugeLong counter = (MutableGaugeLong) registry.get(name); + if (counter == null) { + counter = registry.newGauge(name, name, count); + } + counter.set(count); + } + @VisibleForTesting public void reset() { reads.resetMinMax(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java index 1fdd4cdfba8..4b21ec0aa63 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java @@ -42,6 +42,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.LongAdder; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -51,6 +52,7 @@ import java.util.ArrayList; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hdfs.server.federation.metrics.StateStoreMetrics; import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder; import org.apache.hadoop.hdfs.server.federation.router.Router; import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; @@ -97,6 +99,8 @@ public class MountTableResolver private final TreeMap tree = new TreeMap<>(); /** Path -> Remote location. */ private final Cache locationCache; + private final LongAdder locCacheMiss = new LongAdder(); + private final LongAdder locCacheAccess = new LongAdder(); /** Default nameservice when no mount matches the math. */ private String defaultNameService = ""; @@ -408,6 +412,9 @@ public class MountTableResolver mountTable.getMountTableEntries(request); List records = response.getEntries(); refreshEntries(records); + StateStoreMetrics metrics = this.getMountTableStore().getDriver().getMetrics(); + metrics.setLocationCache("locationCacheMissed", this.getLocCacheMiss().sum()); + metrics.setLocationCache("locationCacheAccessed", this.getLocCacheAccess().sum()); } catch (IOException e) { LOG.error("Cannot fetch mount table entries from State Store", e); return false; @@ -441,9 +448,12 @@ public class MountTableResolver if (this.locationCache == null) { res = lookupLocation(processTrashPath(path)); } else { - Callable meh = (Callable) () -> - lookupLocation(processTrashPath(path)); + Callable meh = (Callable) () -> { + this.getLocCacheMiss().increment(); + return lookupLocation(processTrashPath(path)); + }; res = this.locationCache.get(processTrashPath(path), meh); + this.getLocCacheAccess().increment(); } if (isTrashPath(path)) { List remoteLocations = new ArrayList<>(); @@ -699,4 +709,12 @@ public class MountTableResolver public void setDisabled(boolean disable) { this.disabled = disable; } + + public LongAdder getLocCacheMiss() { + return locCacheMiss; + } + + public LongAdder getLocCacheAccess() { + return locCacheAccess; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java index 92aa5843e3d..a2e7adc8d74 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java @@ -73,7 +73,7 @@ public abstract class RecordStore { * * @return State Store driver. */ - protected StateStoreDriver getDriver() { + public StateStoreDriver getDriver() { return this.driver; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java index 32c54d9cb73..15d3caa5e4e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java @@ -729,4 +729,41 @@ public class TestMountTableResolver { assertEquals("2->/testInvalidateCache/foo", mountTable .getDestinationForPath("/testInvalidateCache/foo").toString()); } -} \ No newline at end of file + + /** + * Test location cache hit when get destination for path. + */ + @Test + public void testLocationCacheHitrate() throws Exception { + List entries = new ArrayList<>(); + + // Add entry and test location cache + Map map1 = getMountTableEntry("1", "/testlocationcache"); + MountTable entry1 = MountTable.newInstance("/testlocationcache", map1); + entries.add(entry1); + + Map map2 = getMountTableEntry("2", + "/anothertestlocationcache"); + MountTable entry2 = MountTable.newInstance("/anothertestlocationcache", + map2); + entries.add(entry2); + + mountTable.refreshEntries(entries); + mountTable.getLocCacheAccess().reset(); + mountTable.getLocCacheMiss().reset(); + assertEquals("1->/testlocationcache", + mountTable.getDestinationForPath("/testlocationcache").toString()); + assertEquals("2->/anothertestlocationcache", + mountTable.getDestinationForPath("/anothertestlocationcache") + .toString()); + + assertEquals(2, mountTable.getLocCacheMiss().intValue()); + assertEquals("1->/testlocationcache", + mountTable.getDestinationForPath("/testlocationcache").toString()); + assertEquals(3, mountTable.getLocCacheAccess().intValue()); + + // Cleanup before exit + mountTable.removeEntry("/testlocationcache"); + mountTable.removeEntry("/anothertestlocationcache"); + } +} From f42c89dffb93a02119dbdad0bb253950e8495893 Mon Sep 17 00:00:00 2001 From: Steve Vaughan Date: Mon, 13 Feb 2023 09:31:32 -0500 Subject: [PATCH 35/50] HDFS-16904. Close webhdfs during TestSymlinkHdfs teardown (#5372) This is a followup to the original patch, 08f58ecf07f, which it supercedes * Switch to org.apache.hadoop.io.IOUtils and closeStream. * Use cleanupWithLogger to include error logging Contributed by Steve Vaughan Jr --- .../test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java index 650a75e5698..12930d01acf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java @@ -23,7 +23,6 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.net.URI; -import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSConfigKeys; @@ -37,11 +36,14 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.web.WebHdfsConstants; import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; import org.apache.hadoop.hdfs.web.WebHdfsTestUtil; +import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.test.GenericTestUtils; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.slf4j.event.Level; /** @@ -53,6 +55,9 @@ abstract public class TestSymlinkHdfs extends SymlinkBaseTest { GenericTestUtils.setLogLevel(NameNode.stateChangeLog, Level.TRACE); } + private static final Logger LOG = LoggerFactory.getLogger( + TestSymlinkHdfs.class); + protected static MiniDFSCluster cluster; protected static WebHdfsFileSystem webhdfs; protected static DistributedFileSystem dfs; @@ -100,7 +105,7 @@ abstract public class TestSymlinkHdfs extends SymlinkBaseTest { if (cluster != null) { cluster.shutdown(); } - IOUtils.closeQuietly(webhdfs); + IOUtils.cleanupWithLogger(LOG, webhdfs); } @Test(timeout=10000) From e4b5314991142e4a11774340c54fa4a4b7df99bd Mon Sep 17 00:00:00 2001 From: Tamas Domok Date: Mon, 13 Feb 2023 16:21:27 +0100 Subject: [PATCH 36/50] MAPREDUCE-7433. Remove unused mapred/LoggingHttpResponseEncoder.java. (#5388) --- .../mapred/LoggingHttpResponseEncoder.java | 106 ------------------ 1 file changed, 106 deletions(-) delete mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/LoggingHttpResponseEncoder.java diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/LoggingHttpResponseEncoder.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/LoggingHttpResponseEncoder.java deleted file mode 100644 index c7b98ce166c..00000000000 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/LoggingHttpResponseEncoder.java +++ /dev/null @@ -1,106 +0,0 @@ -/* - * 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.mapred; - -import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelPromise; -import io.netty.handler.codec.http.HttpHeaders; -import io.netty.handler.codec.http.HttpResponse; -import io.netty.handler.codec.http.HttpResponseEncoder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.List; - -class LoggingHttpResponseEncoder extends HttpResponseEncoder { - private static final Logger LOG = LoggerFactory.getLogger(LoggingHttpResponseEncoder.class); - private final boolean logStacktraceOfEncodingMethods; - - LoggingHttpResponseEncoder(boolean logStacktraceOfEncodingMethods) { - this.logStacktraceOfEncodingMethods = logStacktraceOfEncodingMethods; - } - - @Override - public boolean acceptOutboundMessage(Object msg) throws Exception { - printExecutingMethod(); - LOG.info("OUTBOUND MESSAGE: " + msg); - return super.acceptOutboundMessage(msg); - } - - @Override - protected void encodeInitialLine(ByteBuf buf, HttpResponse response) throws Exception { - LOG.debug("Executing method: {}, response: {}", - getExecutingMethodName(), response); - logStacktraceIfRequired(); - super.encodeInitialLine(buf, response); - } - - @Override - protected void encode(ChannelHandlerContext ctx, Object msg, - List out) throws Exception { - LOG.debug("Encoding to channel {}: {}", ctx.channel(), msg); - printExecutingMethod(); - logStacktraceIfRequired(); - super.encode(ctx, msg, out); - } - - @Override - protected void encodeHeaders(HttpHeaders headers, ByteBuf buf) { - printExecutingMethod(); - super.encodeHeaders(headers, buf); - } - - @Override - public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise - promise) throws Exception { - LOG.debug("Writing to channel {}: {}", ctx.channel(), msg); - printExecutingMethod(); - super.write(ctx, msg, promise); - } - - private void logStacktraceIfRequired() { - if (logStacktraceOfEncodingMethods) { - LOG.debug("Stacktrace: ", new Throwable()); - } - } - - private void printExecutingMethod() { - String methodName = getExecutingMethodName(1); - LOG.debug("Executing method: {}", methodName); - } - - private String getExecutingMethodName() { - return getExecutingMethodName(0); - } - - private String getExecutingMethodName(int additionalSkipFrames) { - try { - StackTraceElement[] stackTrace = Thread.currentThread().getStackTrace(); - // Array items (indices): - // 0: java.lang.Thread.getStackTrace(...) - // 1: TestShuffleHandler$LoggingHttpResponseEncoder.getExecutingMethodName(...) - int skipFrames = 2 + additionalSkipFrames; - String methodName = stackTrace[skipFrames].getMethodName(); - String className = this.getClass().getSimpleName(); - return className + "#" + methodName; - } catch (Throwable t) { - LOG.error("Error while getting execution method name", t); - return "unknown"; - } - } -} From 30f560554d2534467272e5ac66cda0ebd6a1a760 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Tue, 14 Feb 2023 00:03:02 +0530 Subject: [PATCH 37/50] HADOOP-18524. Deploy Hadoop trunk version website. (#5386). Contributed by Ayush Saxena. Reviewed-by: Akira Ajisaka --- .github/workflows/website.yml | 55 +++++++++++++++++++++++++++++++++++ 1 file changed, 55 insertions(+) create mode 100644 .github/workflows/website.yml diff --git a/.github/workflows/website.yml b/.github/workflows/website.yml new file mode 100644 index 00000000000..b55f37ca544 --- /dev/null +++ b/.github/workflows/website.yml @@ -0,0 +1,55 @@ +# 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. + + +name: website + +# Controls when the action will run. +on: + push: + branches: [ trunk ] + +jobs: + build: + runs-on: ubuntu-latest + steps: + - name: Checkout Hadoop trunk + uses: actions/checkout@v3 + with: + repository: apache/hadoop + - name: Set up JDK 8 + uses: actions/setup-java@v3 + with: + java-version: '8' + distribution: 'temurin' + - name: Cache local Maven repository + uses: actions/cache@v3 + with: + path: ~/.m2/repository + key: ${{ runner.os }}-maven-${{ hashFiles('**/pom.xml') }} + restore-keys: | + ${{ runner.os }}-maven- + - name: Build Hadoop + run: mvn clean install site -DskipTests -DskipShade + - name: Stage document + run: mvn site:stage -DstagingDirectory=${GITHUB_WORKSPACE}/staging/ + - name: Deploy to GitHub Pages + uses: peaceiris/actions-gh-pages@v3 + with: + github_token: ${{ secrets.GITHUB_TOKEN }} + publish_dir: ./staging/hadoop-project + user_name: 'github-actions[bot]' + user_email: 'github-actions[bot]@users.noreply.github.com' + From 90de1ff151ede83a6f963aaf2407d3eb6220ae40 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Mon, 13 Feb 2023 11:24:06 -0800 Subject: [PATCH 38/50] HADOOP-18206 Cleanup the commons-logging references and restrict its usage in future (#5315) --- LICENSE-binary | 1 - hadoop-common-project/hadoop-common/pom.xml | 5 -- .../java/org/apache/hadoop/io/IOUtils.java | 25 ------ .../java/org/apache/hadoop/log/LogLevel.java | 40 ++-------- .../hadoop/service/ServiceOperations.java | 6 +- .../org/apache/hadoop/util/LogAdapter.java | 78 ------------------- .../apache/hadoop/util/ReflectionUtils.java | 13 ++-- .../org/apache/hadoop/util/SignalLogger.java | 24 +++--- .../org/apache/hadoop/util/StringUtils.java | 28 ++----- .../TestViewFileSystemLocalFileSystem.java | 9 +-- ...leSystemOverloadSchemeLocalFileSystem.java | 8 +- .../hadoop/http/TestHttpServerWithSpnego.java | 4 - .../org/apache/hadoop/log/TestLog4Json.java | 4 - .../org/apache/hadoop/log/TestLogLevel.java | 6 +- .../apache/hadoop/test/GenericTestUtils.java | 71 ++--------------- .../org/apache/hadoop/util/TestJarFinder.java | 4 +- .../apache/hadoop/util/TestSignalLogger.java | 4 +- hadoop-common-project/hadoop-nfs/pom.xml | 5 -- .../hadoop-hdfs-client/pom.xml | 4 - hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml | 5 -- hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml | 4 - .../federation/router/TestRouterRpc.java | 6 +- .../router/TestRouterRpcMultiDestination.java | 2 +- hadoop-hdfs-project/hadoop-hdfs/pom.xml | 5 -- .../hdfs/server/common/MetricsLoggerTask.java | 27 ++----- .../hdfs/server/datanode/BlockReceiver.java | 11 ++- .../hdfs/server/datanode/BlockSender.java | 9 +-- .../hadoop/hdfs/server/datanode/DataNode.java | 14 ++-- .../server/datanode/DataSetLockManager.java | 7 +- .../hdfs/server/datanode/DataXceiver.java | 28 +++---- .../hdfs/server/namenode/FSNamesystem.java | 20 ++--- .../server/namenode/FsImageValidation.java | 16 +--- .../hadoop/hdfs/server/namenode/NameNode.java | 9 +-- .../hdfs/TestBlockTokenWrappingQOP.java | 6 +- .../org/apache/hadoop/hdfs/TestDFSRename.java | 2 +- .../hadoop/hdfs/TestStripedFileAppend.java | 6 +- .../datanode/TestDataNodeMetricsLogger.java | 10 +-- .../server/namenode/TestAuditLogAtDebug.java | 5 +- .../hdfs/server/namenode/TestAuditLogger.java | 8 +- .../namenode/TestAuditLoggerWithCommands.java | 2 +- .../hdfs/server/namenode/TestAuditLogs.java | 11 ++- .../hadoop/hdfs/server/namenode/TestFsck.java | 5 +- .../namenode/TestNameNodeMetricsLogger.java | 10 +-- .../ha/TestDNFencingWithReplication.java | 2 +- .../TestSimpleExponentialForecast.java | 10 +-- .../v2/TestSpeculativeExecOnCluster.java | 7 +- .../hadoop-mapreduce-client/pom.xml | 5 -- .../hadoop-mapreduce-examples/pom.xml | 4 - hadoop-project/pom.xml | 6 -- hadoop-tools/hadoop-archive-logs/pom.xml | 5 -- .../hadoop/fs/azure/PageBlobInputStream.java | 6 +- .../hadoop/fs/azure/PageBlobOutputStream.java | 7 +- .../hadoop/fs/azure/SelfRenewingLease.java | 6 +- .../fs/azure/SelfThrottlingIntercept.java | 7 +- .../hadoop/fs/azure/SendRequestIntercept.java | 4 - .../hadoop/fs/azure/SimpleKeyProvider.java | 7 +- .../azure/metrics/BandwidthGaugeUpdater.java | 6 +- .../ResponseReceivedMetricUpdater.java | 4 - .../ITestFileSystemOperationsWithThreads.java | 6 +- ...estNativeAzureFileSystemClientLogging.java | 12 +-- .../azure/NativeAzureFileSystemBaseTest.java | 6 +- .../TestShellDecryptionKeyProvider.java | 7 +- .../application/AppCatalogSolrClient.java | 6 +- .../application/YarnServiceClient.java | 6 +- .../linux/runtime/RuncContainerRuntime.java | 7 +- .../runc/ImageTagToManifestPlugin.java | 7 +- .../yarn/server/resourcemanager/RMInfo.java | 7 +- .../webapp/JAXBContextResolver.java | 7 +- ...citySchedulerMultiNodesWithPreemption.java | 8 +- .../pom.xml | 16 ++++ pom.xml | 7 ++ 71 files changed, 223 insertions(+), 532 deletions(-) delete mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LogAdapter.java diff --git a/LICENSE-binary b/LICENSE-binary index 432dc5d28f7..60fdcb45e84 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -250,7 +250,6 @@ commons-codec:commons-codec:1.11 commons-collections:commons-collections:3.2.2 commons-daemon:commons-daemon:1.0.13 commons-io:commons-io:2.8.0 -commons-logging:commons-logging:1.1.3 commons-net:commons-net:3.9.0 de.ruedigermoeller:fst:2.50 io.grpc:grpc-api:1.26.0 diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml index 5f0302fd077..41efc183c3e 100644 --- a/hadoop-common-project/hadoop-common/pom.xml +++ b/hadoop-common-project/hadoop-common/pom.xml @@ -180,11 +180,6 @@ jersey-server compile - - commons-logging - commons-logging - compile - log4j log4j diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java index f0a9b0b6952..7d792f8dc7d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java @@ -32,7 +32,6 @@ import java.nio.file.StandardOpenOption; import java.util.ArrayList; import java.util.List; -import org.apache.commons.logging.Log; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -246,30 +245,6 @@ public class IOUtils { } } - /** - * Close the Closeable objects and ignore any {@link Throwable} or - * null pointers. Must only be used for cleanup in exception handlers. - * - * @param log the log to record problems to at debug level. Can be null. - * @param closeables the objects to close - * @deprecated use {@link #cleanupWithLogger(Logger, java.io.Closeable...)} - * instead - */ - @Deprecated - public static void cleanup(Log log, java.io.Closeable... closeables) { - for (java.io.Closeable c : closeables) { - if (c != null) { - try { - c.close(); - } catch(Throwable e) { - if (log != null && log.isDebugEnabled()) { - log.debug("Exception in closing " + c, e); - } - } - } - } - } - /** * Close the Closeable objects and ignore any {@link Throwable} or * null pointers. Must only be used for cleanup in exception handlers. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java index e2ad16fce2c..6785e2f672a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java @@ -34,10 +34,6 @@ import javax.servlet.http.HttpServletResponse; import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.commons.logging.impl.Jdk14Logger; -import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -51,6 +47,8 @@ import org.apache.hadoop.util.GenericOptionsParser; import org.apache.hadoop.util.ServletUtil; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; /** * Change log level in runtime. @@ -340,22 +338,14 @@ public class LogLevel { out.println(MARKER + "Submitted Class Name: " + logName + "
"); - Log log = LogFactory.getLog(logName); + Logger log = Logger.getLogger(logName); out.println(MARKER + "Log Class: " + log.getClass().getName() +"
"); if (level != null) { out.println(MARKER + "Submitted Level: " + level + "
"); } - if (log instanceof Log4JLogger) { - process(((Log4JLogger)log).getLogger(), level, out); - } - else if (log instanceof Jdk14Logger) { - process(((Jdk14Logger)log).getLogger(), level, out); - } - else { - out.println("Sorry, " + log.getClass() + " not supported.
"); - } + process(log, level, out); } out.println(FORMS); @@ -371,14 +361,14 @@ public class LogLevel { + "" + ""; - private static void process(org.apache.log4j.Logger log, String level, + private static void process(Logger log, String level, PrintWriter out) throws IOException { if (level != null) { - if (!level.equalsIgnoreCase(org.apache.log4j.Level.toLevel(level) + if (!level.equalsIgnoreCase(Level.toLevel(level) .toString())) { out.println(MARKER + "Bad Level : " + level + "
"); } else { - log.setLevel(org.apache.log4j.Level.toLevel(level)); + log.setLevel(Level.toLevel(level)); out.println(MARKER + "Setting Level to " + level + " ...
"); } } @@ -386,21 +376,5 @@ public class LogLevel { + "Effective Level: " + log.getEffectiveLevel() + "
"); } - private static void process(java.util.logging.Logger log, String level, - PrintWriter out) throws IOException { - if (level != null) { - String levelToUpperCase = level.toUpperCase(); - try { - log.setLevel(java.util.logging.Level.parse(levelToUpperCase)); - } catch (IllegalArgumentException e) { - out.println(MARKER + "Bad Level : " + level + "
"); - } - out.println(MARKER + "Setting Level to " + level + " ...
"); - } - - java.util.logging.Level lev; - for(; (lev = log.getLevel()) == null; log = log.getParent()); - out.println(MARKER + "Effective Level: " + lev + "
"); - } } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/ServiceOperations.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/ServiceOperations.java index 726a83da257..57f91886f49 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/ServiceOperations.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/ServiceOperations.java @@ -21,7 +21,6 @@ package org.apache.hadoop.service; import java.util.ArrayList; import java.util.List; -import org.apache.commons.logging.Log; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.slf4j.Logger; @@ -75,9 +74,10 @@ public final class ServiceOperations { * @param log the log to warn at * @param service a service; may be null * @return any exception that was caught; null if none was. - * @see ServiceOperations#stopQuietly(Service) + * @deprecated to be removed with 3.4.0. Use {@link #stopQuietly(Logger, Service)} instead. */ - public static Exception stopQuietly(Log log, Service service) { + @Deprecated + public static Exception stopQuietly(org.apache.commons.logging.Log log, Service service) { try { stop(service); } catch (Exception e) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LogAdapter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LogAdapter.java deleted file mode 100644 index b2bcbf57ef2..00000000000 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LogAdapter.java +++ /dev/null @@ -1,78 +0,0 @@ -/** - * 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.util; - -import org.apache.commons.logging.Log; -import org.slf4j.Logger; - -class LogAdapter { - private Log LOG; - private Logger LOGGER; - - private LogAdapter(Log LOG) { - this.LOG = LOG; - } - - private LogAdapter(Logger LOGGER) { - this.LOGGER = LOGGER; - } - - /** - * @deprecated use {@link #create(Logger)} instead - */ - @Deprecated - public static LogAdapter create(Log LOG) { - return new LogAdapter(LOG); - } - - public static LogAdapter create(Logger LOGGER) { - return new LogAdapter(LOGGER); - } - - public void info(String msg) { - if (LOG != null) { - LOG.info(msg); - } else if (LOGGER != null) { - LOGGER.info(msg); - } - } - - public void warn(String msg, Throwable t) { - if (LOG != null) { - LOG.warn(msg, t); - } else if (LOGGER != null) { - LOGGER.warn(msg, t); - } - } - - public void debug(Throwable t) { - if (LOG != null) { - LOG.debug(t); - } else if (LOGGER != null) { - LOGGER.debug("", t); - } - } - - public void error(String msg) { - if (LOG != null) { - LOG.error(msg); - } else if (LOGGER != null) { - LOGGER.error(msg); - } - } -} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java index 155c4f9c5f4..26bcd4a41cf 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java @@ -36,7 +36,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import org.apache.commons.logging.Log; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configurable; @@ -222,16 +221,18 @@ public class ReflectionUtils { } private static long previousLogTime = 0; - + /** * Log the current thread stacks at INFO level. * @param log the logger that logs the stack trace * @param title a descriptive title for the call stacks - * @param minInterval the minimum time from the last + * @param minInterval the minimum time from the last + * @deprecated to be removed with 3.4.0. Use {@link #logThreadInfo(Logger, String, long)} instead. */ - public static void logThreadInfo(Log log, - String title, - long minInterval) { + @Deprecated + public static void logThreadInfo(org.apache.commons.logging.Log log, + String title, + long minInterval) { boolean dumpStack = false; if (log.isInfoEnabled()) { synchronized (ReflectionUtils.class) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SignalLogger.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SignalLogger.java index 605352443e5..9f112906b2e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SignalLogger.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SignalLogger.java @@ -18,10 +18,10 @@ package org.apache.hadoop.util; +import org.slf4j.Logger; import sun.misc.Signal; import sun.misc.SignalHandler; -import org.apache.commons.logging.Log; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -42,11 +42,11 @@ public enum SignalLogger { * Our signal handler. */ private static class Handler implements SignalHandler { - final private LogAdapter LOG; + final private Logger log; final private SignalHandler prevHandler; - Handler(String name, LogAdapter LOG) { - this.LOG = LOG; + Handler(String name, Logger log) { + this.log = log; prevHandler = Signal.handle(new Signal(name), this); } @@ -57,7 +57,7 @@ public enum SignalLogger { */ @Override public void handle(Signal signal) { - LOG.error("RECEIVED SIGNAL " + signal.getNumber() + + log.error("RECEIVED SIGNAL " + signal.getNumber() + ": SIG" + signal.getName()); prevHandler.handle(signal); } @@ -66,13 +66,9 @@ public enum SignalLogger { /** * Register some signal handlers. * - * @param LOG The log4j logfile to use in the signal handlers. + * @param log The log4j logfile to use in the signal handlers. */ - public void register(final Log LOG) { - register(LogAdapter.create(LOG)); - } - - void register(final LogAdapter LOG) { + public void register(final Logger log) { if (registered) { throw new IllegalStateException("Can't re-install the signal handlers."); } @@ -83,15 +79,15 @@ public enum SignalLogger { String separator = ""; for (String signalName : SIGNALS) { try { - new Handler(signalName, LOG); + new Handler(signalName, log); bld.append(separator) .append(signalName); separator = ", "; } catch (Exception e) { - LOG.debug(e); + log.debug("Error: ", e); } } bld.append("]"); - LOG.info(bld.toString()); + log.info(bld.toString()); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java index b620ba73222..3debd36da78 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java @@ -740,42 +740,26 @@ public class StringUtils { * Print a log message for starting up and shutting down * @param clazz the class of the server * @param args arguments - * @param LOG the target log object + * @param log the target log object */ public static void startupShutdownMessage(Class clazz, String[] args, - final org.apache.commons.logging.Log LOG) { - startupShutdownMessage(clazz, args, LogAdapter.create(LOG)); - } - - /** - * Print a log message for starting up and shutting down - * @param clazz the class of the server - * @param args arguments - * @param LOG the target log object - */ - public static void startupShutdownMessage(Class clazz, String[] args, - final org.slf4j.Logger LOG) { - startupShutdownMessage(clazz, args, LogAdapter.create(LOG)); - } - - static void startupShutdownMessage(Class clazz, String[] args, - final LogAdapter LOG) { + final org.slf4j.Logger log) { final String hostname = NetUtils.getHostname(); final String classname = clazz.getSimpleName(); - LOG.info(createStartupShutdownMessage(classname, hostname, args)); + log.info(createStartupShutdownMessage(classname, hostname, args)); if (SystemUtils.IS_OS_UNIX) { try { - SignalLogger.INSTANCE.register(LOG); + SignalLogger.INSTANCE.register(log); } catch (Throwable t) { - LOG.warn("failed to register any UNIX signal loggers: ", t); + log.warn("failed to register any UNIX signal loggers: ", t); } } ShutdownHookManager.get().addShutdownHook( new Runnable() { @Override public void run() { - LOG.info(toStartupShutdownString("SHUTDOWN_MSG: ", new String[]{ + log.info(toStartupShutdownString("SHUTDOWN_MSG: ", new String[]{ "Shutting down " + classname + " at " + hostname})); LogManager.shutdown(); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java index adc5db87e77..d88730b005d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java @@ -25,8 +25,6 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.net.URI; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -39,7 +37,8 @@ import org.apache.hadoop.security.UserGroupInformation; import org.junit.After; import org.junit.Before; import org.junit.Test; - +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * @@ -51,8 +50,8 @@ import org.junit.Test; */ public class TestViewFileSystemLocalFileSystem extends ViewFileSystemBaseTest { - private static final Log LOG = - LogFactory.getLog(TestViewFileSystemLocalFileSystem.class); + private static final Logger LOG = + LoggerFactory.getLogger(TestViewFileSystemLocalFileSystem.class); @Override @Before diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeLocalFileSystem.java index ac7a1a68994..1e86a91c141 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeLocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeLocalFileSystem.java @@ -21,8 +21,6 @@ import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -35,6 +33,8 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * @@ -43,8 +43,8 @@ import org.junit.Test; */ public class TestViewFileSystemOverloadSchemeLocalFileSystem { private static final String FILE = "file"; - private static final Log LOG = - LogFactory.getLog(TestViewFileSystemOverloadSchemeLocalFileSystem.class); + private static final Logger LOG = + LoggerFactory.getLogger(TestViewFileSystemOverloadSchemeLocalFileSystem.class); private FileSystem fsTarget; private Configuration conf; private Path targetTestRoot; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpnego.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpnego.java index ea7c8cd4e68..dfcd98801de 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpnego.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpnego.java @@ -17,8 +17,6 @@ */ package org.apache.hadoop.http; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.minikdc.MiniKdc; @@ -53,8 +51,6 @@ import static org.junit.Assert.assertTrue; */ public class TestHttpServerWithSpnego { - static final Log LOG = LogFactory.getLog(TestHttpServerWithSpnego.class); - private static final String SECRET_STR = "secret"; private static final String HTTP_USER = "HTTP"; private static final String PREFIX = "hadoop.http.authentication."; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLog4Json.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLog4Json.java index d41a58782d0..519f14b7fd8 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLog4Json.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLog4Json.java @@ -22,8 +22,6 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.ContainerNode; import org.junit.Test; import static org.junit.Assert.*; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.util.Time; import org.apache.log4j.Appender; import org.apache.log4j.Category; @@ -44,8 +42,6 @@ import java.util.Vector; public class TestLog4Json { - private static final Log LOG = LogFactory.getLog(TestLog4Json.class); - @Test public void testConstruction() throws Throwable { Log4Json l4j = new Log4Json(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java index 3af70e95548..636c03a16d9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java @@ -22,9 +22,6 @@ import java.net.SocketException; import java.net.URI; import java.util.concurrent.Callable; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; @@ -70,8 +67,7 @@ public class TestLogLevel extends KerberosSecurityTestcase { private final String logName = TestLogLevel.class.getName(); private String clientPrincipal; private String serverPrincipal; - private final Log testlog = LogFactory.getLog(logName); - private final Logger log = ((Log4JLogger)testlog).getLogger(); + private final Logger log = Logger.getLogger(logName); private final static String PRINCIPAL = "loglevel.principal"; private final static String KEYTAB = "loglevel.keytab"; private static final String PREFIX = "hadoop.http.authentication."; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java index 61d5938494c..e54971e491c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java @@ -49,8 +49,6 @@ import java.util.function.Supplier; import java.util.regex.Pattern; import org.apache.commons.lang3.RandomStringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; @@ -117,29 +115,11 @@ public abstract class GenericTestUtils { public static final String ERROR_INVALID_ARGUMENT = "Total wait time should be greater than check interval time"; - /** - * @deprecated use {@link #disableLog(org.slf4j.Logger)} instead - */ - @Deprecated - @SuppressWarnings("unchecked") - public static void disableLog(Log log) { - // We expect that commons-logging is a wrapper around Log4j. - disableLog((Log4JLogger) log); - } - @Deprecated public static Logger toLog4j(org.slf4j.Logger logger) { return LogManager.getLogger(logger.getName()); } - /** - * @deprecated use {@link #disableLog(org.slf4j.Logger)} instead - */ - @Deprecated - public static void disableLog(Log4JLogger log) { - log.getLogger().setLevel(Level.OFF); - } - /** * @deprecated use {@link #disableLog(org.slf4j.Logger)} instead */ @@ -152,45 +132,6 @@ public abstract class GenericTestUtils { disableLog(toLog4j(logger)); } - /** - * @deprecated - * use {@link #setLogLevel(org.slf4j.Logger, org.slf4j.event.Level)} instead - */ - @Deprecated - @SuppressWarnings("unchecked") - public static void setLogLevel(Log log, Level level) { - // We expect that commons-logging is a wrapper around Log4j. - setLogLevel((Log4JLogger) log, level); - } - - /** - * A helper used in log4j2 migration to accept legacy - * org.apache.commons.logging apis. - *

- * And will be removed after migration. - * - * @param log a log - * @param level level to be set - */ - @Deprecated - public static void setLogLevel(Log log, org.slf4j.event.Level level) { - setLogLevel(log, Level.toLevel(level.toString())); - } - - /** - * @deprecated - * use {@link #setLogLevel(org.slf4j.Logger, org.slf4j.event.Level)} instead - */ - @Deprecated - public static void setLogLevel(Log4JLogger log, Level level) { - log.getLogger().setLevel(level); - } - - /** - * @deprecated - * use {@link #setLogLevel(org.slf4j.Logger, org.slf4j.event.Level)} instead - */ - @Deprecated public static void setLogLevel(Logger logger, Level level) { logger.setLevel(level); } @@ -535,13 +476,15 @@ public abstract class GenericTestUtils { private WriterAppender appender; private Logger logger; - public static LogCapturer captureLogs(Log l) { - Logger logger = ((Log4JLogger)l).getLogger(); - return new LogCapturer(logger); + public static LogCapturer captureLogs(org.slf4j.Logger logger) { + if (logger.getName().equals("root")) { + return new LogCapturer(org.apache.log4j.Logger.getRootLogger()); + } + return new LogCapturer(toLog4j(logger)); } - public static LogCapturer captureLogs(org.slf4j.Logger logger) { - return new LogCapturer(toLog4j(logger)); + public static LogCapturer captureLogs(Logger logger) { + return new LogCapturer(logger); } private LogCapturer(Logger logger) { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJarFinder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJarFinder.java index e58fb3bffde..109cb191b48 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJarFinder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJarFinder.java @@ -18,10 +18,10 @@ package org.apache.hadoop.util; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.test.GenericTestUtils; import org.junit.Assert; import org.junit.Test; +import org.slf4j.LoggerFactory; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -43,7 +43,7 @@ public class TestJarFinder { public void testJar() throws Exception { //picking a class that is for sure in a JAR in the classpath - String jar = JarFinder.getJar(LogFactory.class); + String jar = JarFinder.getJar(LoggerFactory.class); Assert.assertTrue(new File(jar).exists()); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSignalLogger.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSignalLogger.java index b61cebc0a62..f6b272e1c61 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSignalLogger.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSignalLogger.java @@ -32,9 +32,9 @@ public class TestSignalLogger { @Test(timeout=60000) public void testInstall() throws Exception { Assume.assumeTrue(SystemUtils.IS_OS_UNIX); - SignalLogger.INSTANCE.register(LogAdapter.create(LOG)); + SignalLogger.INSTANCE.register(LOG); try { - SignalLogger.INSTANCE.register(LogAdapter.create(LOG)); + SignalLogger.INSTANCE.register(LOG); Assert.fail("expected IllegalStateException from double registration"); } catch (IllegalStateException e) { // fall through diff --git a/hadoop-common-project/hadoop-nfs/pom.xml b/hadoop-common-project/hadoop-nfs/pom.xml index 33d8b3710c5..1da5a25ad1e 100644 --- a/hadoop-common-project/hadoop-nfs/pom.xml +++ b/hadoop-common-project/hadoop-nfs/pom.xml @@ -63,11 +63,6 @@ mockito-core test - - commons-logging - commons-logging - compile - javax.servlet javax.servlet-api diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml index 3337f7d4089..b362e001ea6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml @@ -61,10 +61,6 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> hadoop-common provided - - commons-logging - commons-logging - log4j log4j diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml index 24599a2b244..39bc6683fcf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml @@ -133,11 +133,6 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> commons-io compile - - commons-logging - commons-logging - compile - commons-daemon commons-daemon diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml index d4d5c1eb339..9fb868f79f3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml @@ -49,10 +49,6 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> hadoop-common provided - - commons-logging - commons-logging - log4j log4j diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java index 35b640d4067..cd98b635b50 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java @@ -2054,7 +2054,7 @@ public class TestRouterRpc { @Test public void testMkdirsWithCallerContext() throws IOException { GenericTestUtils.LogCapturer auditlog = - GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.auditLog); + GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); // Current callerContext is null assertNull(CallerContext.getCurrent()); @@ -2092,7 +2092,7 @@ public class TestRouterRpc { @Test public void testAddClientIpPortToCallerContext() throws IOException { GenericTestUtils.LogCapturer auditLog = - GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.auditLog); + GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); // 1. ClientIp and ClientPort are not set on the client. // Set client context. @@ -2127,7 +2127,7 @@ public class TestRouterRpc { @Test public void testAddClientIdAndCallIdToCallerContext() throws IOException { GenericTestUtils.LogCapturer auditLog = - GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.auditLog); + GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); // 1. ClientId and ClientCallId are not set on the client. // Set client context. diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java index 6ade57326e3..30a2bc11020 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java @@ -440,7 +440,7 @@ public class TestRouterRpcMultiDestination extends TestRouterRpc { @Test public void testCallerContextWithMultiDestinations() throws IOException { GenericTestUtils.LogCapturer auditLog = - GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.auditLog); + GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); // set client context CallerContext.setCurrent( diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml index ab8934f9368..5f156499ee0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml @@ -117,11 +117,6 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> commons-io compile - - commons-logging - commons-logging - compile - commons-daemon commons-daemon diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/MetricsLoggerTask.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/MetricsLoggerTask.java index 051e2d2c525..66685f6cc18 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/MetricsLoggerTask.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/MetricsLoggerTask.java @@ -31,8 +31,6 @@ import javax.management.MBeanServer; import javax.management.MalformedObjectNameException; import javax.management.ObjectName; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.impl.Log4JLogger; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.metrics2.util.MBeans; @@ -58,13 +56,12 @@ public class MetricsLoggerTask implements Runnable { } } - private Log metricsLog; + private org.apache.log4j.Logger metricsLog; private String nodeName; private short maxLogLineLength; - public MetricsLoggerTask(Log metricsLog, String nodeName, - short maxLogLineLength) { - this.metricsLog = metricsLog; + public MetricsLoggerTask(String metricsLog, String nodeName, short maxLogLineLength) { + this.metricsLog = org.apache.log4j.Logger.getLogger(metricsLog); this.nodeName = nodeName; this.maxLogLineLength = maxLogLineLength; } @@ -118,13 +115,8 @@ public class MetricsLoggerTask implements Runnable { .substring(0, maxLogLineLength) + "..."); } - private static boolean hasAppenders(Log logger) { - if (!(logger instanceof Log4JLogger)) { - // Don't bother trying to determine the presence of appenders. - return true; - } - Log4JLogger log4JLogger = ((Log4JLogger) logger); - return log4JLogger.getLogger().getAllAppenders().hasMoreElements(); + private static boolean hasAppenders(org.apache.log4j.Logger logger) { + return logger.getAllAppenders().hasMoreElements(); } /** @@ -150,13 +142,8 @@ public class MetricsLoggerTask implements Runnable { * Make the metrics logger async and add all pre-existing appenders to the * async appender. */ - public static void makeMetricsLoggerAsync(Log metricsLog) { - if (!(metricsLog instanceof Log4JLogger)) { - LOG.warn("Metrics logging will not be async since " - + "the logger is not log4j"); - return; - } - org.apache.log4j.Logger logger = ((Log4JLogger) metricsLog).getLogger(); + public static void makeMetricsLoggerAsync(String metricsLog) { + org.apache.log4j.Logger logger = org.apache.log4j.Logger.getLogger(metricsLog); logger.setAdditivity(false); // Don't pollute actual logs with metrics dump @SuppressWarnings("unchecked") diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java index 77e0be6c7b3..1c077098a9d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java @@ -35,7 +35,6 @@ import java.util.Queue; import java.util.concurrent.atomic.AtomicLong; import java.util.zip.Checksum; -import org.apache.commons.logging.Log; import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.fs.FSOutputSummer; import org.apache.hadoop.fs.StorageType; @@ -73,7 +72,7 @@ import org.slf4j.Logger; **/ class BlockReceiver implements Closeable { public static final Logger LOG = DataNode.LOG; - static final Log ClientTraceLog = DataNode.ClientTraceLog; + static final Logger CLIENT_TRACE_LOG = DataNode.CLIENT_TRACE_LOG; @VisibleForTesting static long CACHE_DROP_LAG_BYTES = 8 * 1024 * 1024; @@ -1398,7 +1397,7 @@ class BlockReceiver implements Closeable { public void run() { datanode.metrics.incrDataNodePacketResponderCount(); boolean lastPacketInBlock = false; - final long startTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0; + final long startTime = CLIENT_TRACE_LOG.isInfoEnabled() ? System.nanoTime() : 0; while (isRunning() && !lastPacketInBlock) { long totalAckTimeNanos = 0; boolean isInterrupted = false; @@ -1553,7 +1552,7 @@ class BlockReceiver implements Closeable { // Hold a volume reference to finalize block. try (ReplicaHandler handler = BlockReceiver.this.claimReplicaHandler()) { BlockReceiver.this.close(); - endTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0; + endTime = CLIENT_TRACE_LOG.isInfoEnabled() ? System.nanoTime() : 0; block.setNumBytes(replicaInfo.getNumBytes()); datanode.data.finalizeBlock(block, dirSyncOnFinalize); } @@ -1564,11 +1563,11 @@ class BlockReceiver implements Closeable { datanode.closeBlock(block, null, replicaInfo.getStorageUuid(), replicaInfo.isOnTransientStorage()); - if (ClientTraceLog.isInfoEnabled() && isClient) { + if (CLIENT_TRACE_LOG.isInfoEnabled() && isClient) { long offset = 0; DatanodeRegistration dnR = datanode.getDNRegistrationForBP(block .getBlockPoolId()); - ClientTraceLog.info(String.format(DN_CLIENTTRACE_FORMAT, inAddr, + CLIENT_TRACE_LOG.info(String.format(DN_CLIENTTRACE_FORMAT, inAddr, myAddr, replicaInfo.getVolume(), block.getNumBytes(), "HDFS_WRITE", clientname, offset, dnR.getDatanodeUuid(), block, endTime - startTime)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java index 5c4212fea53..b2bc09bc392 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java @@ -32,7 +32,6 @@ import java.nio.channels.FileChannel; import java.util.Arrays; import java.util.concurrent.TimeUnit; -import org.apache.commons.logging.Log; import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.fs.FsTracer; import org.apache.hadoop.hdfs.DFSUtilClient; @@ -103,7 +102,7 @@ import org.slf4j.Logger; */ class BlockSender implements java.io.Closeable { static final Logger LOG = DataNode.LOG; - static final Log ClientTraceLog = DataNode.ClientTraceLog; + static final Logger CLIENT_TRACE_LOG = DataNode.CLIENT_TRACE_LOG; private static final boolean is32Bit = System.getProperty("sun.arch.data.model").equals("32"); /** @@ -784,7 +783,7 @@ class BlockSender implements java.io.Closeable { // Trigger readahead of beginning of file if configured. manageOsCache(); - final long startTime = ClientTraceLog.isDebugEnabled() ? System.nanoTime() : 0; + final long startTime = CLIENT_TRACE_LOG.isDebugEnabled() ? System.nanoTime() : 0; try { int maxChunksPerPacket; int pktBufSize = PacketHeader.PKT_MAX_HEADER_LEN; @@ -831,9 +830,9 @@ class BlockSender implements java.io.Closeable { sentEntireByteRange = true; } } finally { - if ((clientTraceFmt != null) && ClientTraceLog.isDebugEnabled()) { + if ((clientTraceFmt != null) && CLIENT_TRACE_LOG.isDebugEnabled()) { final long endTime = System.nanoTime(); - ClientTraceLog.debug(String.format(clientTraceFmt, totalRead, + CLIENT_TRACE_LOG.debug(String.format(clientTraceFmt, totalRead, initialOffset, endTime - startTime)); } close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index d8149b6f3e4..ce566885983 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -140,8 +140,6 @@ import javax.annotation.Nullable; import javax.management.ObjectName; import javax.net.SocketFactory; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -318,9 +316,9 @@ public class DataNode extends ReconfigurableBase ", srvID: %s" + // DatanodeRegistration ", blockid: %s" + // block id ", duration(ns): %s"; // duration time - - static final Log ClientTraceLog = - LogFactory.getLog(DataNode.class.getName() + ".clienttrace"); + + static final Logger CLIENT_TRACE_LOG = + LoggerFactory.getLogger(DataNode.class.getName() + ".clienttrace"); private static final String USAGE = "Usage: hdfs datanode [-regular | -rollback | -rollingupgrade rollback" + @@ -360,7 +358,7 @@ public class DataNode extends ReconfigurableBase FS_GETSPACEUSED_JITTER_KEY, FS_GETSPACEUSED_CLASSNAME)); - public static final Log METRICS_LOG = LogFactory.getLog("DataNodeMetricsLog"); + public static final String METRICS_LOG_NAME = "DataNodeMetricsLog"; private static final String DATANODE_HTRACE_PREFIX = "datanode.htrace."; private final FileIoProvider fileIoProvider; @@ -4060,12 +4058,12 @@ public class DataNode extends ReconfigurableBase return; } - MetricsLoggerTask.makeMetricsLoggerAsync(METRICS_LOG); + MetricsLoggerTask.makeMetricsLoggerAsync(METRICS_LOG_NAME); // Schedule the periodic logging. metricsLoggerTimer = new ScheduledThreadPoolExecutor(1); metricsLoggerTimer.setExecuteExistingDelayedTasksAfterShutdownPolicy(false); - metricsLoggerTimer.scheduleWithFixedDelay(new MetricsLoggerTask(METRICS_LOG, + metricsLoggerTimer.scheduleWithFixedDelay(new MetricsLoggerTask(METRICS_LOG_NAME, "DataNode", (short) 0), metricsLoggerPeriodSec, metricsLoggerPeriodSec, TimeUnit.SECONDS); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataSetLockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataSetLockManager.java index eac1259fb84..913c289cfea 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataSetLockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataSetLockManager.java @@ -18,8 +18,6 @@ package org.apache.hadoop.hdfs.server.datanode; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.server.common.AutoCloseDataSetLock; @@ -29,11 +27,14 @@ import java.util.HashMap; import java.util.Stack; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Class for maintain a set of lock for fsDataSetImpl. */ public class DataSetLockManager implements DataNodeLockManager { - public static final Log LOG = LogFactory.getLog(DataSetLockManager.class); + public static final Logger LOG = LoggerFactory.getLogger(DataSetLockManager.class); private final HashMap threadCountMap = new HashMap<>(); private final LockMap lockMap = new LockMap(); private boolean isFair = true; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java index 77041023016..3cf4bde3d45 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java @@ -21,7 +21,6 @@ import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.protobuf.ByteString; import javax.crypto.SecretKey; -import org.apache.commons.logging.Log; import org.apache.hadoop.fs.FsTracer; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.DFSUtilClient; @@ -105,7 +104,7 @@ import static org.apache.hadoop.util.Time.monotonicNow; */ class DataXceiver extends Receiver implements Runnable { public static final Logger LOG = DataNode.LOG; - static final Log ClientTraceLog = DataNode.ClientTraceLog; + static final Logger CLIENT_TRACE_LOG = DataNode.CLIENT_TRACE_LOG; private Peer peer; private final String remoteAddress; // address of remote side @@ -426,10 +425,10 @@ class DataXceiver extends Receiver implements Runnable { registeredSlotId); datanode.shortCircuitRegistry.unregisterSlot(registeredSlotId); } - if (ClientTraceLog.isInfoEnabled()) { + if (CLIENT_TRACE_LOG.isInfoEnabled()) { DatanodeRegistration dnR = datanode.getDNRegistrationForBP(blk .getBlockPoolId()); - BlockSender.ClientTraceLog.info(String.format( + BlockSender.CLIENT_TRACE_LOG.info(String.format( "src: 127.0.0.1, dest: 127.0.0.1, op: REQUEST_SHORT_CIRCUIT_FDS," + " blockid: %s, srvID: %s, success: %b", blk.getBlockId(), dnR.getDatanodeUuid(), success)); @@ -466,8 +465,8 @@ class DataXceiver extends Receiver implements Runnable { bld.build().writeDelimitedTo(socketOut); success = true; } finally { - if (ClientTraceLog.isInfoEnabled()) { - BlockSender.ClientTraceLog.info(String.format( + if (CLIENT_TRACE_LOG.isInfoEnabled()) { + BlockSender.CLIENT_TRACE_LOG.info(String.format( "src: 127.0.0.1, dest: 127.0.0.1, op: RELEASE_SHORT_CIRCUIT_FDS," + " shmId: %016x%016x, slotIdx: %d, srvID: %s, success: %b", slotId.getShmId().getHi(), slotId.getShmId().getLo(), @@ -526,9 +525,9 @@ class DataXceiver extends Receiver implements Runnable { sendShmSuccessResponse(sock, shmInfo); success = true; } finally { - if (ClientTraceLog.isInfoEnabled()) { + if (CLIENT_TRACE_LOG.isInfoEnabled()) { if (success) { - BlockSender.ClientTraceLog.info(String.format( + BlockSender.CLIENT_TRACE_LOG.info(String.format( "cliID: %s, src: 127.0.0.1, dest: 127.0.0.1, " + "op: REQUEST_SHORT_CIRCUIT_SHM," + " shmId: %016x%016x, srvID: %s, success: true", @@ -536,7 +535,7 @@ class DataXceiver extends Receiver implements Runnable { shmInfo.getShmId().getLo(), datanode.getDatanodeUuid())); } else { - BlockSender.ClientTraceLog.info(String.format( + BlockSender.CLIENT_TRACE_LOG.info(String.format( "cliID: %s, src: 127.0.0.1, dest: 127.0.0.1, " + "op: REQUEST_SHORT_CIRCUIT_SHM, " + "shmId: n/a, srvID: %s, success: false", @@ -587,13 +586,10 @@ class DataXceiver extends Receiver implements Runnable { BlockSender blockSender = null; DatanodeRegistration dnR = datanode.getDNRegistrationForBP(block.getBlockPoolId()); - final String clientTraceFmt = - clientName.length() > 0 && ClientTraceLog.isInfoEnabled() - ? String.format(DN_CLIENTTRACE_FORMAT, localAddress, remoteAddress, - "", "%d", "HDFS_READ", clientName, "%d", - dnR.getDatanodeUuid(), block, "%d") - : dnR + " Served block " + block + " to " + - remoteAddress; + final String clientTraceFmt = clientName.length() > 0 && CLIENT_TRACE_LOG.isInfoEnabled() ? + String.format(DN_CLIENTTRACE_FORMAT, localAddress, remoteAddress, "", "%d", "HDFS_READ", + clientName, "%d", dnR.getDatanodeUuid(), block, "%d") : + dnR + " Served block " + block + " to " + remoteAddress; try { try { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 5e4f0d520a9..95b855e8af4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -185,9 +185,6 @@ import javax.management.NotCompliantMBeanException; import javax.management.ObjectName; import javax.management.StandardMBean; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -405,7 +402,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, private final String contextFieldSeparator; boolean isAuditEnabled() { - return (!isDefaultAuditLogger || auditLog.isInfoEnabled()) + return (!isDefaultAuditLogger || AUDIT_LOG.isInfoEnabled()) && !auditLoggers.isEmpty(); } @@ -491,8 +488,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, * perm=<permissions (optional)> * */ - public static final Log auditLog = LogFactory.getLog( - FSNamesystem.class.getName() + ".audit"); + public static final Logger AUDIT_LOG = Logger.getLogger(FSNamesystem.class.getName() + ".audit"); private final int maxCorruptFileBlocksReturn; private final boolean isPermissionEnabled; @@ -8783,8 +8779,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, FileStatus status, CallerContext callerContext, UserGroupInformation ugi, DelegationTokenSecretManager dtSecretManager) { - if (auditLog.isDebugEnabled() || - (auditLog.isInfoEnabled() && !debugCmdSet.contains(cmd))) { + if (AUDIT_LOG.isDebugEnabled() || + (AUDIT_LOG.isInfoEnabled() && !debugCmdSet.contains(cmd))) { final StringBuilder sb = STRING_BUILDER.get(); src = escapeJava(src); dst = escapeJava(dst); @@ -8853,16 +8849,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, } public void logAuditMessage(String message) { - auditLog.info(message); + AUDIT_LOG.info(message); } } private static void enableAsyncAuditLog(Configuration conf) { - if (!(auditLog instanceof Log4JLogger)) { - LOG.warn("Log4j is required to enable async auditlog"); - return; - } - Logger logger = ((Log4JLogger)auditLog).getLogger(); + Logger logger = AUDIT_LOG; @SuppressWarnings("unchecked") List appenders = Collections.list(logger.getAllAppenders()); // failsafe against trying to async it more than once diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsImageValidation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsImageValidation.java index 3325222267d..067ea5e9a8d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsImageValidation.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsImageValidation.java @@ -17,9 +17,6 @@ */ package org.apache.hadoop.hdfs.server.namenode; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; @@ -125,15 +122,10 @@ public class FsImageValidation { } static void setLogLevel(Class clazz, Level level) { - final Log log = LogFactory.getLog(clazz); - if (log instanceof Log4JLogger) { - final org.apache.log4j.Logger logger = ((Log4JLogger) log).getLogger(); - logger.setLevel(level); - LOG.info("setLogLevel {} to {}, getEffectiveLevel() = {}", - clazz.getName(), level, logger.getEffectiveLevel()); - } else { - LOG.warn("Failed setLogLevel {} to {}", clazz.getName(), level); - } + final org.apache.log4j.Logger logger = org.apache.log4j.Logger.getLogger(clazz); + logger.setLevel(level); + LOG.info("setLogLevel {} to {}, getEffectiveLevel() = {}", clazz.getName(), level, + logger.getEffectiveLevel()); } static String toCommaSeparatedNumber(long n) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java index 36ea5c2f646..ddd9fd8087f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java @@ -25,8 +25,6 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; import org.apache.hadoop.util.Preconditions; import java.util.Set; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -427,8 +425,7 @@ public class NameNode extends ReconfigurableBase implements private static final String NAMENODE_HTRACE_PREFIX = "namenode.htrace."; - public static final Log MetricsLog = - LogFactory.getLog("NameNodeMetricsLog"); + public static final String METRICS_LOG_NAME = "NameNodeMetricsLog"; protected FSNamesystem namesystem; protected final NamenodeRole role; @@ -949,13 +946,13 @@ public class NameNode extends ReconfigurableBase implements return; } - MetricsLoggerTask.makeMetricsLoggerAsync(MetricsLog); + MetricsLoggerTask.makeMetricsLoggerAsync(METRICS_LOG_NAME); // Schedule the periodic logging. metricsLoggerTimer = new ScheduledThreadPoolExecutor(1); metricsLoggerTimer.setExecuteExistingDelayedTasksAfterShutdownPolicy( false); - metricsLoggerTimer.scheduleWithFixedDelay(new MetricsLoggerTask(MetricsLog, + metricsLoggerTimer.scheduleWithFixedDelay(new MetricsLoggerTask(METRICS_LOG_NAME, "NameNode", (short) 128), metricsLoggerPeriodSec, metricsLoggerPeriodSec, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockTokenWrappingQOP.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockTokenWrappingQOP.java index c224c4916b5..5a9dcb2465d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockTokenWrappingQOP.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockTokenWrappingQOP.java @@ -21,8 +21,6 @@ import java.net.URI; import java.util.Arrays; import java.util.Collection; import java.util.EnumSet; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FSDataOutputStream; @@ -41,6 +39,8 @@ import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.apache.hadoop.hdfs.DFSConfigKeys.*; import static org.junit.Assert.*; @@ -52,7 +52,7 @@ import static org.junit.Assert.*; */ @RunWith(Parameterized.class) public class TestBlockTokenWrappingQOP extends SaslDataTransferTestCase { - public static final Log LOG = LogFactory.getLog(TestPermission.class); + public static final Logger LOG = LoggerFactory.getLogger(TestPermission.class); private HdfsConfiguration conf; private MiniDFSCluster cluster; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java index 427dc43d3bb..b16f0237b1e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java @@ -190,7 +190,7 @@ public class TestDFSRename { Path path = new Path("/test"); dfs.mkdirs(path); GenericTestUtils.LogCapturer auditLog = - GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.auditLog); + GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); dfs.rename(path, new Path("/dir1"), new Rename[] {Rename.OVERWRITE, Rename.TO_TRASH}); String auditOut = auditLog.getOutput(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStripedFileAppend.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStripedFileAppend.java index a00f67ac3b5..848cedd9892 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStripedFileAppend.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStripedFileAppend.java @@ -17,8 +17,6 @@ */ package org.apache.hadoop.hdfs; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; @@ -29,6 +27,8 @@ import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType; import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.slf4j.event.Level; import java.io.IOException; @@ -47,7 +47,7 @@ import static org.junit.Assert.fail; * Tests append on erasure coded file. */ public class TestStripedFileAppend { - public static final Log LOG = LogFactory.getLog(TestStripedFileAppend.class); + public static final Logger LOG = LoggerFactory.getLogger(TestStripedFileAppend.class); static { DFSTestUtil.setNameNodeLogLevel(Level.TRACE); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java index 8443c368354..29619cc4e31 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java @@ -32,8 +32,6 @@ import java.util.Random; import java.util.concurrent.TimeoutException; import java.util.regex.Pattern; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.impl.Log4JLogger; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -135,8 +133,7 @@ public class TestDataNodeMetricsLogger { public void testMetricsLoggerIsAsync() throws IOException { startDNForTest(true); assertNotNull(dn); - org.apache.log4j.Logger logger = ((Log4JLogger) DataNode.METRICS_LOG) - .getLogger(); + org.apache.log4j.Logger logger = org.apache.log4j.Logger.getLogger(DataNode.METRICS_LOG_NAME); @SuppressWarnings("unchecked") List appenders = Collections.list(logger.getAllAppenders()); assertTrue(appenders.get(0) instanceof AsyncAppender); @@ -156,7 +153,7 @@ public class TestDataNodeMetricsLogger { assertNotNull(dn); final PatternMatchingAppender appender = new PatternMatchingAppender( "^.*FakeMetric.*$"); - addAppender(DataNode.METRICS_LOG, appender); + addAppender(org.apache.log4j.Logger.getLogger(DataNode.METRICS_LOG_NAME), appender); // Ensure that the supplied pattern was matched. GenericTestUtils.waitFor(new Supplier() { @@ -169,8 +166,7 @@ public class TestDataNodeMetricsLogger { dn.shutdown(); } - private void addAppender(Log log, Appender appender) { - org.apache.log4j.Logger logger = ((Log4JLogger) log).getLogger(); + private void addAppender(org.apache.log4j.Logger logger, Appender appender) { @SuppressWarnings("unchecked") List appenders = Collections.list(logger.getAllAppenders()); ((AsyncAppender) appenders.get(0)).addAppender(appender); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogAtDebug.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogAtDebug.java index c86b04cb773..dad4fa306c7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogAtDebug.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogAtDebug.java @@ -26,10 +26,11 @@ import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.FSNamesystemAuditLogger; import org.apache.hadoop.test.GenericTestUtils; +import org.apache.log4j.Level; + import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; -import org.slf4j.event.Level; import java.net.Inet4Address; import java.util.Arrays; @@ -61,7 +62,7 @@ public class TestAuditLogAtDebug { Joiner.on(",").join(debugCommands.get())); } logger.initialize(conf); - GenericTestUtils.setLogLevel(FSNamesystem.auditLog, level); + GenericTestUtils.setLogLevel(FSNamesystem.AUDIT_LOG, level); return spy(logger); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java index 1cc950723be..c00649a9db5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java @@ -258,7 +258,7 @@ public class TestAuditLogger { conf.setInt(HADOOP_CALLER_CONTEXT_SIGNATURE_MAX_SIZE_KEY, 40); try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build()) { - LogCapturer auditlog = LogCapturer.captureLogs(FSNamesystem.auditLog); + LogCapturer auditlog = LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); cluster.waitClusterUp(); final FileSystem fs = cluster.getFileSystem(); final long time = System.currentTimeMillis(); @@ -568,7 +568,7 @@ public class TestAuditLogger { Configuration conf = new HdfsConfiguration(); MiniDFSCluster cluster1 = new MiniDFSCluster.Builder(conf).build(); try { - LogCapturer auditLog = LogCapturer.captureLogs(FSNamesystem.auditLog); + LogCapturer auditLog = LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); cluster1.waitClusterUp(); FileSystem fs = cluster1.getFileSystem(); long time = System.currentTimeMillis(); @@ -585,7 +585,7 @@ public class TestAuditLogger { conf.setBoolean(HADOOP_CALLER_CONTEXT_ENABLED_KEY, true); MiniDFSCluster cluster2 = new MiniDFSCluster.Builder(conf).build(); try { - LogCapturer auditLog = LogCapturer.captureLogs(FSNamesystem.auditLog); + LogCapturer auditLog = LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); cluster2.waitClusterUp(); FileSystem fs = cluster2.getFileSystem(); long time = System.currentTimeMillis(); @@ -606,7 +606,7 @@ public class TestAuditLogger { conf.setInt(HADOOP_CALLER_CONTEXT_SIGNATURE_MAX_SIZE_KEY, 40); try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build()) { - LogCapturer auditlog = LogCapturer.captureLogs(FSNamesystem.auditLog); + LogCapturer auditlog = LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); cluster.waitClusterUp(); final FileSystem fs = cluster.getFileSystem(); final long time = System.currentTimeMillis(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java index 4d379b177af..d34d6ca7379 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java @@ -93,7 +93,7 @@ public class TestAuditLoggerWithCommands { user2 = UserGroupInformation.createUserForTesting("theEngineer", new String[]{"hadoop"}); - auditlog = LogCapturer.captureLogs(FSNamesystem.auditLog); + auditlog = LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); proto = cluster.getNameNodeRpc(); fileSys = DFSTestUtil.getFileSystemAs(user1, conf); fs2 = DFSTestUtil.getFileSystemAs(user2, conf); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java index 2832782caf9..54fcc17cdcf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java @@ -32,7 +32,6 @@ import java.util.Enumeration; import java.util.List; import java.util.regex.Pattern; -import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -130,7 +129,7 @@ public class TestAuditLogs { util.createFiles(fs, fileName); // make sure the appender is what it's supposed to be - Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger(); + Logger logger = FSNamesystem.AUDIT_LOG; @SuppressWarnings("unchecked") List appenders = Collections.list(logger.getAllAppenders()); assertEquals(1, appenders.size()); @@ -283,7 +282,7 @@ public class TestAuditLogs { /** Sets up log4j logger for auditlogs */ private void setupAuditLogs() throws IOException { - Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger(); + Logger logger = FSNamesystem.AUDIT_LOG; // enable logging now that the test is ready to run logger.setLevel(Level.INFO); } @@ -303,7 +302,7 @@ public class TestAuditLogs { disableAuditLog(); PatternLayout layout = new PatternLayout("%m%n"); RollingFileAppender appender = new RollingFileAppender(layout, auditLogFile); - Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger(); + Logger logger = FSNamesystem.AUDIT_LOG; logger.addAppender(appender); } @@ -319,7 +318,7 @@ public class TestAuditLogs { disableAuditLog(); // Close the appenders and force all logs to be flushed - Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger(); + Logger logger = FSNamesystem.AUDIT_LOG; Enumeration appenders = logger.getAllAppenders(); while (appenders.hasMoreElements()) { Appender appender = (Appender)appenders.nextElement(); @@ -352,7 +351,7 @@ public class TestAuditLogs { disableAuditLog(); // Close the appenders and force all logs to be flushed - Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger(); + Logger logger = FSNamesystem.AUDIT_LOG; Enumeration appenders = logger.getAllAppenders(); while (appenders.hasMoreElements()) { Appender appender = (Appender)appenders.nextElement(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java index 60442c6bd04..0f8ca10174a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java @@ -61,7 +61,6 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.function.Supplier; -import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.fs.FSDataOutputStream; @@ -252,7 +251,7 @@ public class TestFsck { if (file.exists()) { file.delete(); } - Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger(); + Logger logger = FSNamesystem.AUDIT_LOG; logger.removeAllAppenders(); logger.setLevel(Level.INFO); PatternLayout layout = new PatternLayout("%m%n"); @@ -291,7 +290,7 @@ public class TestFsck { if (reader != null) { reader.close(); } - Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger(); + Logger logger = FSNamesystem.AUDIT_LOG; if (logger != null) { logger.removeAllAppenders(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetricsLogger.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetricsLogger.java index 9b5e9884c52..7548adbd5d2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetricsLogger.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetricsLogger.java @@ -19,8 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode; import java.util.function.Supplier; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.impl.Log4JLogger; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -70,8 +68,7 @@ public class TestNameNodeMetricsLogger { @Test public void testMetricsLoggerIsAsync() throws IOException { makeNameNode(true); - org.apache.log4j.Logger logger = - ((Log4JLogger) NameNode.MetricsLog).getLogger(); + org.apache.log4j.Logger logger = org.apache.log4j.Logger.getLogger(NameNode.METRICS_LOG_NAME); @SuppressWarnings("unchecked") List appenders = Collections.list(logger.getAllAppenders()); assertTrue(appenders.get(0) instanceof AsyncAppender); @@ -90,7 +87,7 @@ public class TestNameNodeMetricsLogger { makeNameNode(true); // Log metrics early and often. final PatternMatchingAppender appender = new PatternMatchingAppender("^.*FakeMetric42.*$"); - addAppender(NameNode.MetricsLog, appender); + addAppender(org.apache.log4j.Logger.getLogger(NameNode.METRICS_LOG_NAME), appender); // Ensure that the supplied pattern was matched. GenericTestUtils.waitFor(new Supplier() { @@ -118,8 +115,7 @@ public class TestNameNodeMetricsLogger { return new TestNameNode(conf); } - private void addAppender(Log log, Appender appender) { - org.apache.log4j.Logger logger = ((Log4JLogger) log).getLogger(); + private void addAppender(org.apache.log4j.Logger logger, Appender appender) { @SuppressWarnings("unchecked") List appenders = Collections.list(logger.getAllAppenders()); ((AsyncAppender) appenders.get(0)).addAppender(appender); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java index 3f86d4521e4..a243255cdad 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java @@ -45,7 +45,7 @@ import java.util.function.Supplier; */ public class TestDNFencingWithReplication { static { - GenericTestUtils.setLogLevel(FSNamesystem.auditLog, Level.WARN); + GenericTestUtils.setLogLevel(FSNamesystem.AUDIT_LOG, org.apache.log4j.Level.WARN); GenericTestUtils.setLogLevel(Server.LOG, Level.ERROR); GenericTestUtils.setLogLevel(RetryInvocationHandler.LOG, Level.ERROR); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/forecast/TestSimpleExponentialForecast.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/forecast/TestSimpleExponentialForecast.java index 5324e0cff7e..4bd23141304 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/forecast/TestSimpleExponentialForecast.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/forecast/TestSimpleExponentialForecast.java @@ -18,18 +18,18 @@ package org.apache.hadoop.mapreduce.v2.app.speculate.forecast; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.yarn.util.ControlledClock; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.yarn.util.ControlledClock; /** * Testing the statistical model of simple exponential estimator. */ public class TestSimpleExponentialForecast { - private static final Log LOG = - LogFactory.getLog(TestSimpleExponentialForecast.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSimpleExponentialForecast.class); private static long clockTicks = 1000L; private ControlledClock clock; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecOnCluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecOnCluster.java index 02e4358a075..36dcce4dfac 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecOnCluster.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecOnCluster.java @@ -28,8 +28,6 @@ import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -66,6 +64,8 @@ import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Test speculation on Mini Cluster. @@ -73,8 +73,7 @@ import org.junit.runners.Parameterized; @Ignore @RunWith(Parameterized.class) public class TestSpeculativeExecOnCluster { - private static final Log LOG = LogFactory - .getLog(TestSpeculativeExecOnCluster.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSpeculativeExecOnCluster.class); private static final int NODE_MANAGERS_COUNT = 2; private static final boolean ENABLE_SPECULATIVE_MAP = true; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml index fdcab2f2ffb..b9e181d1ae3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml @@ -132,11 +132,6 @@ io.netty netty-all - - commons-logging - commons-logging - provided - org.apache.hadoop.thirdparty hadoop-shaded-guava diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml index e5426a08b3c..fac2ac0561e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml @@ -38,10 +38,6 @@ commons-cli commons-cli - - commons-logging - commons-logging - org.apache.hadoop hadoop-mapreduce-client-jobclient diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 73dc8eb2906..5887d5b7fa9 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -121,7 +121,6 @@ 2.11.0 3.12.0 1.1.3 - 1.1 3.6.1 3.9.0 1.10.0 @@ -1094,11 +1093,6 @@ - - commons-logging - commons-logging-api - ${commons-logging-api.version} - log4j log4j diff --git a/hadoop-tools/hadoop-archive-logs/pom.xml b/hadoop-tools/hadoop-archive-logs/pom.xml index f6154484ad2..bd64495dcae 100644 --- a/hadoop-tools/hadoop-archive-logs/pom.xml +++ b/hadoop-tools/hadoop-archive-logs/pom.xml @@ -101,11 +101,6 @@ commons-io provided - - commons-logging - commons-logging - provided - commons-cli commons-cli diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobInputStream.java index 40bf6f4ae1a..0d8936582e9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobInputStream.java @@ -30,8 +30,6 @@ import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.fs.azure.StorageInterface.CloudPageBlobWrapper; @@ -39,6 +37,8 @@ import com.microsoft.azure.storage.OperationContext; import com.microsoft.azure.storage.StorageException; import com.microsoft.azure.storage.blob.BlobRequestOptions; import com.microsoft.azure.storage.blob.PageRange; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * An input stream that reads file data from a page blob stored @@ -46,7 +46,7 @@ import com.microsoft.azure.storage.blob.PageRange; */ final class PageBlobInputStream extends InputStream { - private static final Log LOG = LogFactory.getLog(PageBlobInputStream.class); + private static final Logger LOG = LoggerFactory.getLogger(PageBlobInputStream.class); // The blob we're reading from. private final CloudPageBlobWrapper blob; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobOutputStream.java index 3c98405ff9f..f77a6b80514 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobOutputStream.java @@ -39,8 +39,6 @@ import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.Syncable; import org.apache.hadoop.fs.azure.StorageInterface.CloudPageBlobWrapper; import org.apache.commons.lang3.exception.ExceptionUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.classification.VisibleForTesting; @@ -48,7 +46,8 @@ import com.microsoft.azure.storage.OperationContext; import com.microsoft.azure.storage.StorageException; import com.microsoft.azure.storage.blob.BlobRequestOptions; import com.microsoft.azure.storage.blob.CloudPageBlob; - +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * An output stream that write file data to a page blob stored using ASV's @@ -120,7 +119,7 @@ final class PageBlobOutputStream extends OutputStream implements Syncable, Strea // Whether the stream has been closed. private boolean closed = false; - public static final Log LOG = LogFactory.getLog(AzureNativeFileSystemStore.class); + public static final Logger LOG = LoggerFactory.getLogger(AzureNativeFileSystemStore.class); // Set the minimum page blob file size to 128MB, which is >> the default // block size of 32MB. This default block size is often used as the diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java index 01ab06cb024..989c3ba6d93 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java @@ -18,8 +18,6 @@ package org.apache.hadoop.fs.azure; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.azure.StorageInterface.CloudBlobWrapper; import org.apache.hadoop.classification.VisibleForTesting; @@ -27,6 +25,8 @@ import org.apache.hadoop.classification.VisibleForTesting; import com.microsoft.azure.storage.AccessCondition; import com.microsoft.azure.storage.StorageException; import com.microsoft.azure.storage.blob.CloudBlob; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.concurrent.atomic.AtomicInteger; @@ -58,7 +58,7 @@ public class SelfRenewingLease { // Time to wait to renew lease in milliseconds public static final int LEASE_RENEWAL_PERIOD = 40000; - private static final Log LOG = LogFactory.getLog(SelfRenewingLease.class); + private static final Logger LOG = LoggerFactory.getLogger(SelfRenewingLease.class); // Used to allocate thread serial numbers in thread name private static AtomicInteger threadNumber = new AtomicInteger(0); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfThrottlingIntercept.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfThrottlingIntercept.java index a9e3df907f9..ad71016a745 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfThrottlingIntercept.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfThrottlingIntercept.java @@ -21,8 +21,6 @@ package org.apache.hadoop.fs.azure; import java.net.HttpURLConnection; import java.util.Date; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import com.microsoft.azure.storage.OperationContext; @@ -30,6 +28,8 @@ import com.microsoft.azure.storage.RequestResult; import com.microsoft.azure.storage.ResponseReceivedEvent; import com.microsoft.azure.storage.SendingRequestEvent; import com.microsoft.azure.storage.StorageEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /* * Self throttling is implemented by hooking into send & response callbacks @@ -63,8 +63,7 @@ import com.microsoft.azure.storage.StorageEvent; */ @InterfaceAudience.Private public class SelfThrottlingIntercept { - public static final Log LOG = LogFactory - .getLog(SelfThrottlingIntercept.class); + public static final Logger LOG = LoggerFactory.getLogger(SelfThrottlingIntercept.class); private final float readFactor; private final float writeFactor; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SendRequestIntercept.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SendRequestIntercept.java index 924ecd30b31..98f9de7bff5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SendRequestIntercept.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SendRequestIntercept.java @@ -21,8 +21,6 @@ package org.apache.hadoop.fs.azure; import java.net.HttpURLConnection; import java.security.InvalidKeyException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import com.microsoft.azure.storage.Constants.HeaderConstants; @@ -40,8 +38,6 @@ import com.microsoft.azure.storage.StorageException; @InterfaceAudience.Private public final class SendRequestIntercept extends StorageEvent { - public static final Log LOG = LogFactory.getLog(SendRequestIntercept.class); - private static final String ALLOW_ALL_REQUEST_PRECONDITIONS = "*"; /** diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SimpleKeyProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SimpleKeyProvider.java index 5596f7e67c2..64811e13ee9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SimpleKeyProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SimpleKeyProvider.java @@ -20,8 +20,9 @@ package org.apache.hadoop.fs.azure; import java.io.IOException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.ProviderUtils; @@ -32,7 +33,7 @@ import org.apache.hadoop.security.ProviderUtils; */ @InterfaceAudience.Private public class SimpleKeyProvider implements KeyProvider { - private static final Log LOG = LogFactory.getLog(SimpleKeyProvider.class); + private static final Logger LOG = LoggerFactory.getLogger(SimpleKeyProvider.class); protected static final String KEY_ACCOUNT_KEY_PREFIX = "fs.azure.account.key."; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/metrics/BandwidthGaugeUpdater.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/metrics/BandwidthGaugeUpdater.java index 699fde7dee7..d3fe4aefeb0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/metrics/BandwidthGaugeUpdater.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/metrics/BandwidthGaugeUpdater.java @@ -21,8 +21,6 @@ package org.apache.hadoop.fs.azure.metrics; import java.util.ArrayList; import java.util.Date; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; /** @@ -31,9 +29,7 @@ import org.apache.hadoop.classification.InterfaceAudience; */ @InterfaceAudience.Private public final class BandwidthGaugeUpdater { - public static final Log LOG = LogFactory - .getLog(BandwidthGaugeUpdater.class); - + public static final String THREAD_NAME = "AzureNativeFilesystemStore-UploadBandwidthUpdater"; private static final int DEFAULT_WINDOW_SIZE_MS = 1000; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/metrics/ResponseReceivedMetricUpdater.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/metrics/ResponseReceivedMetricUpdater.java index de503bf1907..4c61f6817cf 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/metrics/ResponseReceivedMetricUpdater.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/metrics/ResponseReceivedMetricUpdater.java @@ -20,8 +20,6 @@ package org.apache.hadoop.fs.azure.metrics; import java.net.HttpURLConnection; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import com.microsoft.azure.storage.Constants.HeaderConstants; @@ -38,8 +36,6 @@ import com.microsoft.azure.storage.StorageEvent; @InterfaceAudience.Private public final class ResponseReceivedMetricUpdater extends StorageEvent { - public static final Log LOG = LogFactory.getLog(ResponseReceivedMetricUpdater.class); - private final AzureFileSystemInstrumentation instrumentation; private final BandwidthGaugeUpdater blockUploadGaugeUpdater; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsWithThreads.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsWithThreads.java index 4389fda393c..1e7330fbd0b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsWithThreads.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsWithThreads.java @@ -28,7 +28,6 @@ import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -41,6 +40,8 @@ import org.junit.rules.ExpectedException; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Tests the Native Azure file system (WASB) using parallel threads for rename and delete operations. @@ -70,8 +71,7 @@ public class ITestFileSystemOperationsWithThreads extends AbstractWasbTestBase { fs.initialize(uri, conf); // Capture logs - logs = LogCapturer.captureLogs(new Log4JLogger(org.apache.log4j.Logger - .getRootLogger())); + logs = LogCapturer.captureLogs(LoggerFactory.getLogger(Logger.ROOT_LOGGER_NAME)); } /* diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemClientLogging.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemClientLogging.java index f73a7638a3e..476d7a4f01e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemClientLogging.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemClientLogging.java @@ -21,12 +21,13 @@ package org.apache.hadoop.fs.azure; import java.net.URI; import java.util.StringTokenizer; -import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.test.GenericTestUtils.LogCapturer; -import org.apache.log4j.Logger; + import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Test to validate Azure storage client side logging. Tests works only when @@ -94,8 +95,8 @@ public class ITestNativeAzureFileSystemClientLogging @Test public void testLoggingEnabled() throws Exception { - LogCapturer logs = LogCapturer.captureLogs(new Log4JLogger(Logger - .getRootLogger())); + LogCapturer logs = + LogCapturer.captureLogs(LoggerFactory.getLogger(org.slf4j.Logger.ROOT_LOGGER_NAME)); // Update configuration based on the Test. updateFileSystemConfiguration(true); @@ -116,8 +117,7 @@ public class ITestNativeAzureFileSystemClientLogging @Test public void testLoggingDisabled() throws Exception { - LogCapturer logs = LogCapturer.captureLogs(new Log4JLogger(Logger - .getRootLogger())); + LogCapturer logs = LogCapturer.captureLogs(LoggerFactory.getLogger(Logger.ROOT_LOGGER_NAME)); // Update configuration based on the Test. updateFileSystemConfiguration(false); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java index 8ac36c299b6..9a75ef55335 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java @@ -30,8 +30,6 @@ import java.util.Date; import java.util.EnumSet; import java.util.TimeZone; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -49,6 +47,8 @@ import org.apache.hadoop.fs.azure.NativeAzureFileSystem.FolderRenamePending; import com.microsoft.azure.storage.AccessCondition; import com.microsoft.azure.storage.StorageException; import com.microsoft.azure.storage.blob.CloudBlob; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.readStringFromFile; import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.writeStringToFile; @@ -73,7 +73,7 @@ public abstract class NativeAzureFileSystemBaseTest private static final EnumSet CREATE_FLAG = EnumSet.of(XAttrSetFlag.CREATE); private static final EnumSet REPLACE_FLAG = EnumSet.of(XAttrSetFlag.REPLACE); - public static final Log LOG = LogFactory.getLog(NativeAzureFileSystemBaseTest.class); + public static final Logger LOG = LoggerFactory.getLogger(NativeAzureFileSystemBaseTest.class); protected NativeAzureFileSystem fs; @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java index b8df38eed0a..1f027411720 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java @@ -23,10 +23,10 @@ import java.nio.charset.Charset; import org.junit.Assert; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.commons.io.FileUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException; @@ -39,8 +39,7 @@ import static org.junit.Assert.assertEquals; * */ public class TestShellDecryptionKeyProvider { - public static final Log LOG = LogFactory - .getLog(TestShellDecryptionKeyProvider.class); + public static final Logger LOG = LoggerFactory.getLogger(TestShellDecryptionKeyProvider.class); private static final File TEST_ROOT_DIR = new File(System.getProperty( "test.build.data", "/tmp"), "TestShellDecryptionKeyProvider"); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/src/main/java/org/apache/hadoop/yarn/appcatalog/application/AppCatalogSolrClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/src/main/java/org/apache/hadoop/yarn/appcatalog/application/AppCatalogSolrClient.java index b1515a5b6fb..ac8dbbac61d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/src/main/java/org/apache/hadoop/yarn/appcatalog/application/AppCatalogSolrClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/src/main/java/org/apache/hadoop/yarn/appcatalog/application/AppCatalogSolrClient.java @@ -34,8 +34,6 @@ import org.apache.hadoop.yarn.appcatalog.model.Application; import org.apache.hadoop.yarn.appcatalog.utils.RandomWord; import org.apache.hadoop.yarn.appcatalog.utils.WordLengthException; import org.apache.hadoop.yarn.service.api.records.Service; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.solr.client.solrj.SolrClient; import org.apache.solr.client.solrj.SolrQuery; import org.apache.solr.client.solrj.SolrQuery.ORDER; @@ -48,13 +46,15 @@ import org.apache.solr.common.SolrInputDocument; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Driver class for accessing Solr. */ public class AppCatalogSolrClient { - private static final Log LOG = LogFactory.getLog(AppCatalogSolrClient.class); + private static final Logger LOG = LoggerFactory.getLogger(AppCatalogSolrClient.class); private static String urlString; public AppCatalogSolrClient() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/src/main/java/org/apache/hadoop/yarn/appcatalog/application/YarnServiceClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/src/main/java/org/apache/hadoop/yarn/appcatalog/application/YarnServiceClient.java index 79838a9e4fb..185b1c8ddeb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/src/main/java/org/apache/hadoop/yarn/appcatalog/application/YarnServiceClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/src/main/java/org/apache/hadoop/yarn/appcatalog/application/YarnServiceClient.java @@ -21,8 +21,6 @@ package org.apache.hadoop.yarn.appcatalog.application; import java.io.IOException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.appcatalog.model.AppEntry; @@ -39,13 +37,15 @@ import com.sun.jersey.api.client.ClientResponse; import com.sun.jersey.api.client.UniformInterfaceException; import com.sun.jersey.api.client.config.ClientConfig; import com.sun.jersey.api.client.config.DefaultClientConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Driver class for calling YARN Resource Manager REST API. */ public class YarnServiceClient { - private static final Log LOG = LogFactory.getLog(YarnServiceClient.class); + private static final Logger LOG = LoggerFactory.getLogger(YarnServiceClient.class); private static Configuration conf = new Configuration(); private static ClientConfig getClientConfig() { ClientConfig config = new DefaultClientConfig(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/RuncContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/RuncContainerRuntime.java index e43f7788d78..2c327c04eba 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/RuncContainerRuntime.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/RuncContainerRuntime.java @@ -21,8 +21,6 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime; import org.apache.hadoop.classification.VisibleForTesting; -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; @@ -81,6 +79,8 @@ import java.util.regex.Matcher; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_NM_RUNC_IMAGE_TAG_TO_MANIFEST_PLUGIN; import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_NM_RUNC_LAYER_MOUNTS_TO_KEEP; @@ -136,8 +136,7 @@ import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.r @InterfaceStability.Unstable public class RuncContainerRuntime extends OCIContainerRuntime { - private static final Log LOG = LogFactory.getLog( - RuncContainerRuntime.class); + private static final Logger LOG = LoggerFactory.getLogger(RuncContainerRuntime.class); @InterfaceAudience.Private private static final String RUNTIME_TYPE = "RUNC"; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/runc/ImageTagToManifestPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/runc/ImageTagToManifestPlugin.java index fbec3ee6f5e..457939c9a17 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/runc/ImageTagToManifestPlugin.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/runc/ImageTagToManifestPlugin.java @@ -20,8 +20,6 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc; import org.apache.commons.io.IOUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -45,6 +43,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import com.fasterxml.jackson.databind.ObjectMapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_NM_RUNC_CACHE_REFRESH_INTERVAL; import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_NM_RUNC_IMAGE_TOPLEVEL_DIR; @@ -78,8 +78,7 @@ public class ImageTagToManifestPlugin extends AbstractService private String manifestDir; private String localImageTagToHashFile; - private static final Log LOG = LogFactory.getLog( - ImageTagToManifestPlugin.class); + private static final Logger LOG = LoggerFactory.getLogger(ImageTagToManifestPlugin.class); private static final int SHA256_HASH_LENGTH = 64; private static final String ALPHA_NUMERIC = "[a-zA-Z0-9]+"; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMInfo.java index 84d49cd25b3..f58dd911fac 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMInfo.java @@ -17,8 +17,6 @@ */ package org.apache.hadoop.yarn.server.resourcemanager; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.metrics2.util.MBeans; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.UserGroupInformation; @@ -27,11 +25,14 @@ import javax.management.NotCompliantMBeanException; import javax.management.ObjectName; import javax.management.StandardMBean; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * JMX bean for RM info. */ public class RMInfo implements RMInfoMXBean { - private static final Log LOG = LogFactory.getLog(RMNMInfo.class); + private static final Logger LOG = LoggerFactory.getLogger(RMNMInfo.class); private ResourceManager resourceManager; private ObjectName rmStatusBeanName; 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/JAXBContextResolver.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java index 24428b3bb83..2277c36beb1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java @@ -22,6 +22,8 @@ import com.google.inject.Inject; import com.google.inject.Singleton; import com.sun.jersey.api.json.JSONConfiguration; import com.sun.jersey.api.json.JSONJAXBContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.*; @@ -29,8 +31,6 @@ import javax.ws.rs.ext.ContextResolver; import javax.ws.rs.ext.Provider; import javax.xml.bind.JAXBContext; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.UserInfo; @@ -41,8 +41,7 @@ import org.apache.hadoop.yarn.webapp.RemoteExceptionData; @Provider public class JAXBContextResolver implements ContextResolver { - private static final Log LOG = - LogFactory.getLog(JAXBContextResolver.class.getName()); + private static final Logger LOG = LoggerFactory.getLogger(JAXBContextResolver.class.getName()); private final Map typesContextMap; 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/TestCapacitySchedulerMultiNodesWithPreemption.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/TestCapacitySchedulerMultiNodesWithPreemption.java index dc15857bfa4..c895b58b296 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/TestCapacitySchedulerMultiNodesWithPreemption.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/TestCapacitySchedulerMultiNodesWithPreemption.java @@ -22,8 +22,6 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.C import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.Resource; @@ -49,14 +47,16 @@ import org.apache.hadoop.yarn.util.resource.Resources; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; public class TestCapacitySchedulerMultiNodesWithPreemption { - private static final Log LOG = LogFactory - .getLog(TestCapacitySchedulerMultiNodesWithPreemption.class); + private static final Logger LOG = + LoggerFactory.getLogger(TestCapacitySchedulerMultiNodesWithPreemption.class); private CapacitySchedulerConfiguration conf; private static final String POLICY_CLASS_NAME = "org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement." diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml index a323eaff855..6f2fce097df 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml @@ -104,6 +104,10 @@ com.sun.jersey jersey-json + + commons-logging + commons-logging + @@ -336,6 +340,10 @@ com.sun.jersey jersey-json + + commons-logging + commons-logging + @@ -351,6 +359,10 @@ org.apache.hadoop hadoop-hdfs-client + + commons-logging + commons-logging + @@ -367,6 +379,10 @@ org.apache.hadoop hadoop-hdfs-client + + commons-logging + commons-logging + diff --git a/pom.xml b/pom.xml index 4e2887f1df4..51a80c76341 100644 --- a/pom.xml +++ b/pom.xml @@ -288,6 +288,13 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x org.glassfish.grizzly.** + + true + Use slf4j based Logger + + org.apache.commons.logging.** + + From 1def35d802bd06ffc0be6470e3454da698e67b78 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Tue, 14 Feb 2023 11:05:41 +0530 Subject: [PATCH 39/50] HADOOP-18524. Addendum: Deploy Hadoop trunk version website. (#5389). Contributed by Ayush Saxena. Reviewed-by: Vinayakumar B --- .github/workflows/website.yml | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/.github/workflows/website.yml b/.github/workflows/website.yml index b55f37ca544..6d925f3dcff 100644 --- a/.github/workflows/website.yml +++ b/.github/workflows/website.yml @@ -41,8 +41,12 @@ jobs: key: ${{ runner.os }}-maven-${{ hashFiles('**/pom.xml') }} restore-keys: | ${{ runner.os }}-maven- + - name: Build Hadoop maven plugins + run: cd hadoop-maven-plugins && mvn --batch-mode install - name: Build Hadoop - run: mvn clean install site -DskipTests -DskipShade + run: mvn clean install -DskipTests -DskipShade + - name: Build document + run: mvn clean site - name: Stage document run: mvn site:stage -DstagingDirectory=${GITHUB_WORKSPACE}/staging/ - name: Deploy to GitHub Pages From fe0541b58d8a1aead015bb440528ec84e25ec1c9 Mon Sep 17 00:00:00 2001 From: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Date: Tue, 14 Feb 2023 15:29:38 +0800 Subject: [PATCH 40/50] HDFS-16913. Fix flaky some unit tests since they offen timeout (#5377) Co-authored-by: gf13871 Reviewed-by: Tao Li Reviewed-by: Shilun Fan Signed-off-by: Shilun Fan --- .../apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java | 2 +- .../java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java | 6 +++--- .../hadoop/hdfs/server/balancer/TestBalancerService.java | 2 +- .../hdfs/server/namenode/ha/TestPipelinesFailover.java | 2 +- .../hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java index c8420ca82a8..f993db0f1fa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java @@ -32,7 +32,7 @@ public class TestFileLengthOnClusterRestart { * Tests the fileLength when we sync the file and restart the cluster and * Datanodes not report to Namenode yet. */ - @Test(timeout = 60000) + @Test(timeout = 120000) public void testFileLengthWithHSyncAndClusterRestartWithOutDNsRegister() throws Exception { final Configuration conf = new HdfsConfiguration(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java index bfa3deaa6b1..5d472439550 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java @@ -493,17 +493,17 @@ public class TestLeaseRecovery2 { * * @throws Exception */ - @Test(timeout = 30000) + @Test(timeout = 60000) public void testHardLeaseRecoveryAfterNameNodeRestart() throws Exception { hardLeaseRecoveryRestartHelper(false, -1); } - @Test(timeout = 30000) + @Test(timeout = 60000) public void testHardLeaseRecoveryAfterNameNodeRestart2() throws Exception { hardLeaseRecoveryRestartHelper(false, 1535); } - @Test(timeout = 30000) + @Test(timeout = 60000) public void testHardLeaseRecoveryWithRenameAfterNameNodeRestart() throws Exception { hardLeaseRecoveryRestartHelper(true, -1); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerService.java index a75b5092df7..eb0a8ef860a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerService.java @@ -168,7 +168,7 @@ public class TestBalancerService { } } - @Test(timeout = 60000) + @Test(timeout = 120000) public void testBalancerServiceOnError() throws Exception { Configuration conf = new HdfsConfiguration(); // retry for every 5 seconds diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java index 260dd7049d5..cc80af6fb1f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java @@ -210,7 +210,7 @@ public class TestPipelinesFailover { doTestWriteOverFailoverWithDnFail(TestScenario.GRACEFUL_FAILOVER); } - @Test(timeout=30000) + @Test(timeout=60000) public void testWriteOverCrashFailoverWithDnFail() throws Exception { doTestWriteOverFailoverWithDnFail(TestScenario.ORIGINAL_ACTIVE_CRASHED); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java index 0a262f899ab..9737d1d31f1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java @@ -548,7 +548,7 @@ public class TestSnapshot { * paths work and the NN can load the resulting edits. This test if for * snapshots at the root level. */ - @Test(timeout = 60000) + @Test(timeout = 120000) public void testSnapshotOpsOnRootReservedPath() throws Exception { Path dir = new Path("/"); Path sub = new Path(dir, "sub"); From 021fcc6c5e931fd454f10dc8ede47dbcc34d6139 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Tue, 14 Feb 2023 03:48:48 -0800 Subject: [PATCH 41/50] HADOOP-18628. IPC Server Connection should log host name before returning VersionMismatch error (#5385) Contributed by Viraj Jasani --- .../java/org/apache/hadoop/ipc/Server.java | 42 +++++++++++++------ .../java/org/apache/hadoop/ipc/TestIPC.java | 4 ++ .../java/org/apache/hadoop/ipc/TestRPC.java | 5 +++ 3 files changed, 39 insertions(+), 12 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index a79fc2eeb57..26f519716fe 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -1985,11 +1985,26 @@ public abstract class Server { private long lastContact; private int dataLength; private Socket socket; + // Cache the remote host & port info so that even if the socket is // disconnected, we can say where it used to connect to. - private String hostAddress; - private int remotePort; - private InetAddress addr; + + /** + * Client Host IP address from where the socket connection is being established to the Server. + */ + private final String hostAddress; + /** + * Client remote port used for the given socket connection. + */ + private final int remotePort; + /** + * Address to which the socket is connected to. + */ + private final InetAddress addr; + /** + * Client Host address from where the socket connection is being established to the Server. + */ + private final String hostName; IpcConnectionContextProto connectionContext; String protocolName; @@ -2033,8 +2048,12 @@ public abstract class Server { this.isOnAuxiliaryPort = isOnAuxiliaryPort; if (addr == null) { this.hostAddress = "*Unknown*"; + this.hostName = this.hostAddress; } else { + // host IP address this.hostAddress = addr.getHostAddress(); + // host name for the IP address + this.hostName = addr.getHostName(); } this.remotePort = socket.getPort(); this.responseQueue = new LinkedList(); @@ -2050,7 +2069,7 @@ public abstract class Server { @Override public String toString() { - return getHostAddress() + ":" + remotePort; + return hostName + ":" + remotePort + " / " + hostAddress + ":" + remotePort; } boolean setShouldClose() { @@ -2463,19 +2482,18 @@ public abstract class Server { return -1; } - if(!RpcConstants.HEADER.equals(dataLengthBuffer)) { - LOG.warn("Incorrect RPC Header length from {}:{} " - + "expected length: {} got length: {}", - hostAddress, remotePort, RpcConstants.HEADER, dataLengthBuffer); + if (!RpcConstants.HEADER.equals(dataLengthBuffer)) { + LOG.warn("Incorrect RPC Header length from {}:{} / {}:{}. Expected: {}. Actual: {}", + hostName, remotePort, hostAddress, remotePort, RpcConstants.HEADER, + dataLengthBuffer); setupBadVersionResponse(version); return -1; } if (version != CURRENT_VERSION) { //Warning is ok since this is not supposed to happen. - LOG.warn("Version mismatch from " + - hostAddress + ":" + remotePort + - " got version " + version + - " expected version " + CURRENT_VERSION); + LOG.warn("Version mismatch from {}:{} / {}:{}. " + + "Expected version: {}. Actual version: {} ", hostName, + remotePort, hostAddress, remotePort, CURRENT_VERSION, version); setupBadVersionResponse(version); return -1; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java index b65f86a0f7b..59966b340c8 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java @@ -1168,6 +1168,10 @@ public class TestIPC { call(client, addr, serviceClass, conf); Connection connection = server.getConnections()[0]; + LOG.info("Connection is from: {}", connection); + assertEquals( + "Connection string representation should include both IP address and Host name", 2, + connection.toString().split(" / ").length); int serviceClass2 = connection.getServiceClass(); assertFalse(noChanged ^ serviceClass == serviceClass2); client.stop(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java index a7727716c48..9126316fca6 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java @@ -1849,6 +1849,11 @@ public class TestRPC extends TestRpcBase { // if it wasn't fatal, verify there's only one open connection. Connection[] conns = server.getConnections(); assertEquals(reqName, 1, conns.length); + String connectionInfo = conns[0].toString(); + LOG.info("Connection is from: {}", connectionInfo); + assertEquals( + "Connection string representation should include both IP address and Host name", 2, + connectionInfo.split(" / ").length); // verify whether the connection should have been reused. if (isDisconnected) { assertNotSame(reqName, lastConn, conns[0]); From e2ab35084a1ef67db9519d28ac9bb015442abe05 Mon Sep 17 00:00:00 2001 From: SimhadriGovindappa Date: Tue, 14 Feb 2023 18:13:29 +0530 Subject: [PATCH 42/50] HADOOP-18630. Add gh-pages in asf.yaml to deploy the current trunk doc (#5393). Contributed by Simhadri Govindappa. Signed-off-by: Ayush Saxena --- .asf.yaml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.asf.yaml b/.asf.yaml index e1c11790c8b..a8445f78879 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -14,6 +14,8 @@ # limitations under the License. github: + ghp_path: / + ghp_branch: gh-pages enabled_merge_buttons: squash: true merge: false From d56977e9098591f838c1cbdf9433da568fa19c1d Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 14 Feb 2023 17:22:59 +0000 Subject: [PATCH 43/50] HADOOP-18470. More in the 3.3.5 index.html about security (#5383) Expands on the comments in cluster config to tell people they shouldn't be running a cluster without a private VLAN in cloud, that Knox is good here, and unsecured clusters without a VLAN are just computation-as-a-service to crypto miners Contributed by Steve Loughran --- .../src/site/markdown/SingleCluster.md.vm | 2 + hadoop-project/src/site/markdown/index.md.vm | 63 +++++++++++++++++-- 2 files changed, 59 insertions(+), 6 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm b/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm index 3c8af8fd6e9..bbea16855e5 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm +++ b/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm @@ -35,6 +35,8 @@ These instructions do not cover integration with any Kerberos services, -everyone bringing up a production cluster should include connecting to their organisation's Kerberos infrastructure as a key part of the deployment. +See [Security](./SecureMode.html) for details on how to secure a cluster. + Prerequisites ------------- diff --git a/hadoop-project/src/site/markdown/index.md.vm b/hadoop-project/src/site/markdown/index.md.vm index 5e0a46449fa..e7ed0fe8066 100644 --- a/hadoop-project/src/site/markdown/index.md.vm +++ b/hadoop-project/src/site/markdown/index.md.vm @@ -24,7 +24,7 @@ Users are encouraged to read the full set of release notes. This page provides an overview of the major changes. Azure ABFS: Critical Stream Prefetch Fix ---------------------------------------------- +---------------------------------------- The abfs has a critical bug fix [HADOOP-18546](https://issues.apache.org/jira/browse/HADOOP-18546). @@ -120,25 +120,76 @@ be vulnerable, and the ugprades should also reduce the number of false positives security scanners report. We have not been able to upgrade every single dependency to the latest -version there is. Some of those changes are just going to be incompatible. -If you have concerns about the state of a specific library, consult the pache JIRA -issue tracker to see whether a JIRA has been filed, discussions have taken place about +version there is. Some of those changes are fundamentally incompatible. +If you have concerns about the state of a specific library, consult the Apache JIRA +issue tracker to see if an issue has been filed, discussions have taken place about the library in question, and whether or not there is already a fix in the pipeline. *Please don't file new JIRAs about dependency-X.Y.Z having a CVE without searching for any existing issue first* -As an open source project, contributions in this area are always welcome, +As an open-source project, contributions in this area are always welcome, especially in testing the active branches, testing applications downstream of those branches and of whether updated dependencies trigger regressions. + +Security Advisory +================= + +Hadoop HDFS is a distributed filesystem allowing remote +callers to read and write data. + +Hadoop YARN is a distributed job submission/execution +engine allowing remote callers to submit arbitrary +work into the cluster. + +Unless a Hadoop cluster is deployed with +[caller authentication with Kerberos](./hadoop-project-dist/hadoop-common/SecureMode.html), +anyone with network access to the servers has unrestricted access to the data +and the ability to run whatever code they want in the system. + +In production, there are generally three deployment patterns which +can, with care, keep data and computing resources private. +1. Physical cluster: *configure Hadoop security*, usually bonded to the + enterprise Kerberos/Active Directory systems. + Good. +1. Cloud: transient or persistent single or multiple user/tenant cluster + with private VLAN *and security*. + Good. + Consider [Apache Knox](https://knox.apache.org/) for managing remote + access to the cluster. +1. Cloud: transient single user/tenant cluster with private VLAN + *and no security at all*. + Requires careful network configuration as this is the sole + means of securing the cluster.. + Consider [Apache Knox](https://knox.apache.org/) for managing + remote access to the cluster. + +*If you deploy a Hadoop cluster in-cloud without security, and without configuring a VLAN +to restrict access to trusted users, you are implicitly sharing your data and +computing resources with anyone with network access* + +If you do deploy an insecure cluster this way then port scanners will inevitably +find it and submit crypto-mining jobs. If this happens to you, please do not report +this as a CVE or security issue: it is _utterly predictable_. Secure *your cluster* if +you want to remain exclusively *your cluster*. + +Finally, if you are using Hadoop as a service deployed/managed by someone else, +do determine what security their products offer and make sure it meets your requirements. + + Getting Started =============== The Hadoop documentation includes the information you need to get started using -Hadoop. Begin with the +Hadoop. Begin with the [Single Node Setup](./hadoop-project-dist/hadoop-common/SingleCluster.html) which shows you how to set up a single-node Hadoop installation. Then move on to the [Cluster Setup](./hadoop-project-dist/hadoop-common/ClusterSetup.html) to learn how to set up a multi-node Hadoop installation. +Before deploying Hadoop in production, read +[Hadoop in Secure Mode](./hadoop-project-dist/hadoop-common/SecureMode.html), +and follow its instructions to secure your cluster. + + From 4cbe19f3a222576970e300db60a41f78545e9524 Mon Sep 17 00:00:00 2001 From: Zita Dombi <50611074+dombizita@users.noreply.github.com> Date: Wed, 15 Feb 2023 17:16:04 +0100 Subject: [PATCH 44/50] HDFS-16761. Namenode UI for Datanodes page not loading if any data node is down (#5390) --- .../src/main/webapps/router/federationhealth.html | 1 + .../hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html | 1 + 2 files changed, 2 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/webapps/router/federationhealth.html b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/webapps/router/federationhealth.html index 9fa43a30858..f9a8d7a49b1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/webapps/router/federationhealth.html +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/webapps/router/federationhealth.html @@ -390,6 +390,7 @@ + {/DeadNodes} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html index 4bf2e2bd7f5..889bc8aff51 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html @@ -361,6 +361,7 @@ + {/DeadNodes} From f4f2793f3b62d4d5519775e8d1031524027d1f71 Mon Sep 17 00:00:00 2001 From: Ankit Saurabh Date: Wed, 15 Feb 2023 18:28:42 +0000 Subject: [PATCH 45/50] HADOOP-18351. Reduce excess logging of errors during S3A prefetching reads (#5274) Contributed by Ankit Saurabh --- .../fs/impl/prefetch/CachingBlockManager.java | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/CachingBlockManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/CachingBlockManager.java index 31084c7bf26..a0db4b308b6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/CachingBlockManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/CachingBlockManager.java @@ -302,7 +302,12 @@ public abstract class CachingBlockManager extends BlockManager { private void read(BufferData data) throws IOException { synchronized (data) { - readBlock(data, false, BufferData.State.BLANK); + try { + readBlock(data, false, BufferData.State.BLANK); + } catch (IOException e) { + LOG.error("error reading block {}", data.getBlockNumber(), e); + throw e; + } } } @@ -362,9 +367,6 @@ public abstract class CachingBlockManager extends BlockManager { buffer.flip(); data.setReady(expectedState); } catch (Exception e) { - String message = String.format("error during readBlock(%s)", data.getBlockNumber()); - LOG.error(message, e); - if (isPrefetch && tracker != null) { tracker.failed(); } @@ -406,7 +408,8 @@ public abstract class CachingBlockManager extends BlockManager { try { blockManager.prefetch(data, taskQueuedStartTime); } catch (Exception e) { - LOG.error("error during prefetch", e); + LOG.info("error prefetching block {}. {}", data.getBlockNumber(), e.getMessage()); + LOG.debug("error prefetching block {}", data.getBlockNumber(), e); } return null; } @@ -493,7 +496,8 @@ public abstract class CachingBlockManager extends BlockManager { return; } } catch (Exception e) { - LOG.error("error waiting on blockFuture: {}", data, e); + LOG.info("error waiting on blockFuture: {}. {}", data, e.getMessage()); + LOG.debug("error waiting on blockFuture: {}", data, e); data.setDone(); return; } @@ -523,8 +527,8 @@ public abstract class CachingBlockManager extends BlockManager { data.setDone(); } catch (Exception e) { numCachingErrors.incrementAndGet(); - String message = String.format("error adding block to cache after wait: %s", data); - LOG.error(message, e); + LOG.info("error adding block to cache after wait: {}. {}", data, e.getMessage()); + LOG.debug("error adding block to cache after wait: {}", data, e); data.setDone(); } From c3706597a3f2105da83e25ed2681ffbadb463262 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Thu, 16 Feb 2023 06:38:41 +0800 Subject: [PATCH 46/50] YARN-11349. [Federation] Router Support DelegationToken With SQL. (#5244) --- .../AbstractDelegationTokenSecretManager.java | 4 +- .../MySQL/FederationStateStoreStoredProcs.sql | 73 ++- .../MySQL/FederationStateStoreTables.sql | 21 + .../MySQL/dropStoreProcedures.sql | 14 + .../FederationStateStore/MySQL/dropTables.sql | 6 + .../FederationStateStoreStoredProcs.sql | 271 ++++++++- .../SQLServer/FederationStateStoreTables.sql | 154 +++++ .../SQLServer/FederationStateStoreUser.sql | 4 +- .../SQLServer/dropStoreProcedures.sql | 82 ++- .../SQLServer/dropTables.sql | 28 +- .../SQLServer/dropUser.sql | 8 +- .../impl/MemoryFederationStateStore.java | 25 +- .../store/impl/SQLFederationStateStore.java | 557 ++++++++++++++++-- .../records/RouterRMDTSecretManagerState.java | 4 +- .../store/records/RouterStoreToken.java | 25 +- .../impl/pb/RouterStoreTokenPBImpl.java | 28 + .../federation/store/sql/DatabaseProduct.java | 126 ++++ .../store/sql/FederationQueryRunner.java | 310 ++++++++++ .../store/sql/FederationSQLOutParameter.java | 91 +++ .../store/sql/ResultSetHandler.java | 30 + .../store/sql/RouterMasterKeyHandler.java | 68 +++ .../store/sql/RouterStoreTokenHandler.java | 83 +++ .../federation/store/sql/RowCountHandler.java | 56 ++ .../federation/store/sql/package-info.java | 17 + ...FederationRouterRMTokenInputValidator.java | 1 - .../utils/FederationStateStoreUtils.java | 69 ++- .../utils/FederationStateStoreFacade.java | 36 ++ .../proto/yarn_server_federation_protos.proto | 1 + .../impl/FederationStateStoreBaseTest.java | 33 +- .../impl/HSQLDBFederationStateStore.java | 125 ++++ .../impl/TestMemoryFederationStateStore.java | 2 +- .../impl/TestSQLFederationStateStore.java | 281 ++++----- .../RouterDelegationTokenSupport.java | 65 ++ .../token/delegation/package-info.java | 20 + .../RouterDelegationTokenSecretManager.java | 84 +++ .../TestFederationClientInterceptor.java | 18 +- 36 files changed, 2586 insertions(+), 234 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/DatabaseProduct.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationQueryRunner.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationSQLOutParameter.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/ResultSetHandler.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterMasterKeyHandler.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterStoreTokenHandler.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RowCountHandler.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/package-info.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/RouterDelegationTokenSupport.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/package-info.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java index d0c0fac6e88..8aaf9bbd8de 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java @@ -866,9 +866,9 @@ extends AbstractDelegationTokenIdentifier> /** * Add token stats to the owner to token count mapping. * - * @param id + * @param id token id. */ - private void addTokenForOwnerStats(TokenIdent id) { + protected void addTokenForOwnerStats(TokenIdent id) { String realOwner = getTokenRealOwner(id); tokenOwnerStats.put(realOwner, tokenOwnerStats.getOrDefault(realOwner, 0L)+1); diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql index 6461cf2bd75..2edda86cd3c 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql @@ -183,7 +183,7 @@ BEGIN SELECT ROW_COUNT() INTO rowCount_OUT; SELECT homeSubCluster INTO storedHomeSubCluster_OUT FROM reservationsHomeSubCluster - WHERE applicationId = reservationId_IN; + WHERE reservationId = reservationId_IN; END // CREATE PROCEDURE sp_getReservationHomeSubCluster( @@ -219,4 +219,75 @@ BEGIN SELECT ROW_COUNT() INTO rowCount_OUT; END // +CREATE PROCEDURE sp_addMasterKey( + IN keyId_IN bigint, IN masterKey_IN varchar(1024), + OUT rowCount_OUT int) +BEGIN + INSERT INTO masterKeys(keyId, masterKey) + (SELECT keyId_IN, masterKey_IN + FROM masterKeys + WHERE keyId = keyId_IN + HAVING COUNT(*) = 0); + SELECT ROW_COUNT() INTO rowCount_OUT; +END // + +CREATE PROCEDURE sp_getMasterKey( + IN keyId_IN bigint, + OUT masterKey_OUT varchar(1024)) +BEGIN + SELECT masterKey INTO masterKey_OUT + FROM masterKeys + WHERE keyId = keyId_IN; +END // + +CREATE PROCEDURE sp_deleteMasterKey( + IN keyId_IN bigint, OUT rowCount_OUT int) +BEGIN + DELETE FROM masterKeys + WHERE keyId = keyId_IN; + SELECT ROW_COUNT() INTO rowCount_OUT; +END // + +CREATE PROCEDURE sp_addDelegationToken( + IN sequenceNum_IN bigint, IN tokenIdent_IN varchar(1024), + IN token_IN varchar(1024), IN renewDate_IN bigint, + OUT rowCount_OUT int) +BEGIN + INSERT INTO delegationTokens(sequenceNum, tokenIdent, token, renewDate) + (SELECT sequenceNum_IN, tokenIdent_IN, token_IN, renewDate_IN + FROM delegationTokens + WHERE sequenceNum = sequenceNum_IN + HAVING COUNT(*) = 0); + SELECT ROW_COUNT() INTO rowCount_OUT; +END // + +CREATE PROCEDURE sp_getDelegationToken( + IN sequenceNum_IN bigint, OUT tokenIdent_OUT varchar(1024), + OUT token_OUT varchar(1024), OUT renewDate_OUT bigint) +BEGIN + SELECT tokenIdent, token, renewDate INTO tokenIdent_OUT, token_OUT, renewDate_OUT + FROM delegationTokens + WHERE sequenceNum = sequenceNum_IN; +END // + +CREATE PROCEDURE sp_updateDelegationToken( + IN sequenceNum_IN bigint, IN tokenIdent_IN varchar(1024), + IN token_IN varchar(1024), IN renewDate_IN bigint, OUT rowCount_OUT int) +BEGIN + UPDATE delegationTokens + SET tokenIdent = tokenIdent_IN, + token = token_IN, + renewDate = renewDate_IN + WHERE sequenceNum = sequenceNum_IN; + SELECT ROW_COUNT() INTO rowCount_OUT; +END // + +CREATE PROCEDURE sp_deleteDelegationToken( + IN sequenceNum_IN bigint, OUT rowCount_OUT int) +BEGIN + DELETE FROM delegationTokens + WHERE sequenceNum = sequenceNum_IN; + SELECT ROW_COUNT() INTO rowCount_OUT; +END // + DELIMITER ; diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreTables.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreTables.sql index d61a10f998b..9e864eb5ed7 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreTables.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreTables.sql @@ -52,4 +52,25 @@ CREATE TABLE reservationsHomeSubCluster ( reservationId varchar(128) NOT NULL, homeSubCluster varchar(256) NOT NULL, CONSTRAINT pk_reservationId PRIMARY KEY (reservationId) +); + +CREATE TABLE masterKeys ( + keyId bigint NOT NULL, + masterKey varchar(1024) NOT NULL, + CONSTRAINT pk_keyId PRIMARY KEY (keyId) +); + +CREATE TABLE delegationTokens +( + sequenceNum bigint NOT NULL, + tokenIdent varchar(1024) NOT NULL, + token varchar(1024) NOT NULL, + renewDate bigint NOT NULL, + CONSTRAINT pk_sequenceNum PRIMARY KEY (sequenceNum) +); + +CREATE TABLE sequenceTable ( + sequenceName varchar(255) NOT NULL, + nextVal bigint(20) NOT NULL, + CONSTRAINT pk_sequenceName PRIMARY KEY (sequenceName) ); \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropStoreProcedures.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropStoreProcedures.sql index a2f0b882b3f..e7a16c81deb 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropStoreProcedures.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropStoreProcedures.sql @@ -55,3 +55,17 @@ DROP PROCEDURE sp_getReservationsHomeSubCluster; DROP PROCEDURE sp_deleteReservationHomeSubCluster; DROP PROCEDURE sp_updateReservationHomeSubCluster; + +DROP PROCEDURE sp_addMasterKey; + +DROP PROCEDURE sp_getMasterKey; + +DROP PROCEDURE sp_deleteMasterKey; + +DROP PROCEDURE sp_addDelegationToken; + +DROP PROCEDURE sp_getDelegationToken; + +DROP PROCEDURE sp_updateDelegationToken; + +DROP PROCEDURE sp_deleteDelegationToken; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropTables.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropTables.sql index d29f8652c15..38d00d3cb10 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropTables.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropTables.sql @@ -27,3 +27,9 @@ DROP TABLE membership; DROP TABLE policies; DROP TABLE reservationsHomeSubCluster; + +DROP TABLE masterKeys; + +DROP TABLE delegationTokens; + +DROP TABLE sequenceTable; diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreStoredProcs.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreStoredProcs.sql index cc8a79d6273..d82b53e73aa 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreStoredProcs.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreStoredProcs.sql @@ -128,7 +128,7 @@ AS BEGIN [createTime], row_number() over(order by [createTime] desc) AS app_rank FROM [dbo].[applicationsHomeSubCluster] - WHERE [homeSubCluster] = @homeSubCluster_IN OR @homeSubCluster = '') AS applicationsHomeSubCluster + WHERE [homeSubCluster] = @homeSubCluster_IN OR @homeSubCluster_IN = '') AS applicationsHomeSubCluster WHERE app_rank <= @limit_IN; END TRY @@ -699,4 +699,273 @@ AS BEGIN ) WITH log END CATCH END; +GO + +IF OBJECT_ID ( '[sp_addMasterKey]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_addMasterKey]; +GO + +CREATE PROCEDURE [dbo].[sp_addMasterKey] + @keyId_IN BIGINT, + @masterKey_IN VARCHAR(1024), + @rowCount_OUT int OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + BEGIN TRAN + -- If application to sub-cluster map doesn't exist, insert it. + -- Otherwise don't change the current mapping. + IF NOT EXISTS (SELECT TOP 1 * + FROM [dbo].[masterKeys] + WHERE [keyId] = @keyId_IN) + + INSERT INTO [dbo].[masterKeys] ( + [keyId], + [masterKey]) + VALUES ( + @keyId_IN, + @masterKey_IN); + -- End of the IF block + + SELECT @rowCount_OUT = @@ROWCOUNT; + + COMMIT TRAN + END TRY + + BEGIN CATCH + ROLLBACK TRAN + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_getMasterKey]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_getMasterKey]; +GO + +CREATE PROCEDURE [dbo].[sp_getMasterKey] + @keyId_IN bigint, + @masterKey_OUT VARCHAR(1024) OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + + SELECT @masterKey_OUT = [masterKey] + FROM [dbo].[masterKeys] + WHERE [keyId] = @keyId_IN; + + END TRY + + BEGIN CATCH + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_deleteMasterKey]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_deleteMasterKey]; +GO + +CREATE PROCEDURE [dbo].[sp_deleteMasterKey] + @keyId_IN bigint, + @rowCount_OUT int OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + BEGIN TRAN + + DELETE FROM [dbo].[masterKeys] + WHERE [keyId] = @keyId_IN; + SELECT @rowCount_OUT = @@ROWCOUNT; + + COMMIT TRAN + END TRY + + BEGIN CATCH + ROLLBACK TRAN + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_addDelegationToken]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_addDelegationToken]; +GO + +CREATE PROCEDURE [dbo].[sp_addDelegationToken] + @sequenceNum_IN BIGINT, + @tokenIdent_IN VARCHAR(1024), + @token_IN VARCHAR(1024), + @renewDate_IN BIGINT, + @rowCount_OUT int OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + BEGIN TRAN + -- If application to sub-cluster map doesn't exist, insert it. + -- Otherwise don't change the current mapping. + IF NOT EXISTS (SELECT TOP 1 * + FROM [dbo].[delegationTokens] + WHERE [sequenceNum] = @sequenceNum_IN) + + INSERT INTO [dbo].[delegationTokens] ( + [sequenceNum], + [tokenIdent], + [token], + [renewDate]) + VALUES ( + @sequenceNum_IN, + @tokenIdent_IN, + @token_IN, + @renewDate_IN); + -- End of the IF block + + SELECT @rowCount_OUT = @@ROWCOUNT; + + COMMIT TRAN + END TRY + + BEGIN CATCH + ROLLBACK TRAN + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_getDelegationToken]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_getDelegationToken]; +GO + +CREATE PROCEDURE [dbo].[sp_getDelegationToken] + @sequenceNum_IN BIGINT, + @tokenIdent_OUT VARCHAR(1024) OUTPUT, + @token_OUT VARCHAR(1024) OUTPUT, + @renewDate_OUT BIGINT OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + + SELECT @tokenIdent_OUT = [tokenIdent], + @token_OUT = [token], + @renewDate_OUT = [renewDate] + FROM [dbo].[delegationTokens] + WHERE [sequenceNum] = @sequenceNum_IN; + + END TRY + + BEGIN CATCH + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_updateDelegationToken]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_updateDelegationToken]; +GO + +CREATE PROCEDURE [dbo].[sp_updateDelegationToken] + @sequenceNum_IN BIGINT, + @tokenIdent_IN VARCHAR(1024), + @token_IN VARCHAR(1024), + @renewDate_IN BIGINT, + @rowCount_OUT BIGINT OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + + UPDATE [dbo].[delegationTokens] + SET [tokenIdent] = @tokenIdent_IN, + [token] = @token_IN, + [renewDate] = @renewDate_IN + WHERE [sequenceNum] = @sequenceNum_IN; + SELECT @rowCount_OUT = @@ROWCOUNT; + + END TRY + + BEGIN CATCH + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_deleteDelegationToken]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_deleteDelegationToken]; +GO + +CREATE PROCEDURE [dbo].[sp_deleteDelegationToken] + @sequenceNum_IN bigint, + @rowCount_OUT int OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + BEGIN TRAN + + DELETE FROM [dbo].[delegationTokens] + WHERE [sequenceNum] = @sequenceNum_IN; + SELECT @rowCount_OUT = @@ROWCOUNT; + + COMMIT TRAN + END TRY + + BEGIN CATCH + ROLLBACK TRAN + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; GO \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreTables.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreTables.sql index fb8a1bff554..4d187d4459a 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreTables.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreTables.sql @@ -155,4 +155,158 @@ IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables ELSE PRINT 'Table reservationsHomeSubCluster exists, no operation required...' GO +GO + +IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables + WHERE name = 'masterKeys' + AND schema_id = SCHEMA_ID('dbo')) + BEGIN + PRINT 'Table masterKeys does not exist, create it...' + + SET ANSI_NULLS ON + + SET QUOTED_IDENTIFIER ON + + SET ANSI_PADDING ON + + CREATE TABLE [dbo].[masterKeys]( + keyId BIGINT NOT NULL, + masterKey VARCHAR(1024) NOT NULL, + CONSTRAINT [pk_keyId] PRIMARY KEY + ( + [keyId] + ) + ) + + SET ANSI_PADDING OFF + + PRINT 'Table masterKeys created.' + END +ELSE + PRINT 'Table masterKeys exists, no operation required...' + GO +GO + +IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables + WHERE name = 'delegationTokens' + AND schema_id = SCHEMA_ID('dbo')) + BEGIN + PRINT 'Table delegationTokens does not exist, create it...' + + SET ANSI_NULLS ON + + SET QUOTED_IDENTIFIER ON + + SET ANSI_PADDING ON + + CREATE TABLE [dbo].[delegationTokens]( + sequenceNum BIGINT NOT NULL, + tokenIdent VARCHAR(1024) NOT NULL, + token VARCHAR(1024) NOT NULL, + renewDate BIGINT NOT NULL, + CONSTRAINT [pk_sequenceNum] PRIMARY KEY + ( + [sequenceNum] + ) + ) + + SET ANSI_PADDING OFF + + PRINT 'Table delegationTokens created.' + END +ELSE + PRINT 'Table delegationTokens exists, no operation required...' + GO +GO + +IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables + WHERE name = 'masterKeys' + AND schema_id = SCHEMA_ID('dbo')) + BEGIN + PRINT 'Table masterKeys does not exist, create it...' + + SET ANSI_NULLS ON + + SET QUOTED_IDENTIFIER ON + + SET ANSI_PADDING ON + + CREATE TABLE [dbo].[masterKeys]( + keyId BIGINT NOT NULL, + masterKey VARCHAR(1024) NOT NULL, + CONSTRAINT [pk_keyId] PRIMARY KEY + ( + [keyId] + ) + ) + + SET ANSI_PADDING OFF + + PRINT 'Table masterKeys created.' + END +ELSE + PRINT 'Table masterKeys exists, no operation required...' + GO +GO + +IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables + WHERE name = 'delegationTokens' + AND schema_id = SCHEMA_ID('dbo')) + BEGIN + PRINT 'Table delegationTokens does not exist, create it...' + + SET ANSI_NULLS ON + + SET QUOTED_IDENTIFIER ON + + SET ANSI_PADDING ON + + CREATE TABLE [dbo].[delegationTokens]( + sequenceNum BIGINT NOT NULL, + tokenIdent VARCHAR(1024) NOT NULL, + token VARCHAR(1024) NOT NULL, + renewDate BIGINT NOT NULL, + CONSTRAINT [pk_sequenceNum] PRIMARY KEY + ( + [sequenceNum] + ) + ) + + SET ANSI_PADDING OFF + + PRINT 'Table delegationTokens created.' + END +ELSE + PRINT 'Table delegationTokens exists, no operation required...' + GO +GO + +IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables + WHERE name = 'sequenceTable' + AND schema_id = SCHEMA_ID('dbo')) + BEGIN + PRINT 'Table sequenceTable does not exist, create it...' + + SET ANSI_NULLS ON + + SET QUOTED_IDENTIFIER ON + + SET ANSI_PADDING ON + + CREATE TABLE [dbo].[sequenceTable]( + sequenceName VARCHAR(255) NOT NULL, + nextVal bigint NOT NULL + CONSTRAINT [pk_sequenceName] PRIMARY KEY + ( + [sequenceName] + ) + ) + + SET ANSI_PADDING OFF + + PRINT 'Table sequenceTable created.' + END +ELSE + PRINT 'Table sequenceTable exists, no operation required...' + GO GO \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreUser.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreUser.sql index 3f9553fbe32..1bee729f849 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreUser.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreUser.sql @@ -21,10 +21,10 @@ USE [FederationStateStore] GO -CREATE LOGIN 'FederationUser' with password = 'FederationPassword', default_database=[FederationStateStore] ; +CREATE LOGIN FederationUser with password = 'Federation@Password', default_database=[FederationStateStore]; GO -CREATE USER 'FederationUser' FOR LOGIN 'FederationUser' WITH default_schema=dbo; +CREATE USER FederationUser FOR LOGIN FederationUser WITH default_schema=dbo; GO EXEC sp_addrolemember 'db_owner', 'FederationUser'; diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropStoreProcedures.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropStoreProcedures.sql index 6204df2f418..a6e35df1af4 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropStoreProcedures.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropStoreProcedures.sql @@ -21,56 +21,102 @@ USE [FederationStateStore] GO -DROP PROCEDURE IF EXISTS [sp_addApplicationHomeSubCluster]; +IF OBJECT_ID ('[sp_addApplicationHomeSubCluster]', 'P') IS NOT NULL + DROP PROCEDURE [sp_addApplicationHomeSubCluster]; GO -DROP PROCEDURE IF EXISTS [sp_updateApplicationHomeSubCluster]; +IF OBJECT_ID ('[sp_updateApplicationHomeSubCluster]', 'P') IS NOT NULL + DROP PROCEDURE [sp_updateApplicationHomeSubCluster]; GO -DROP PROCEDURE IF EXISTS [sp_getApplicationsHomeSubCluster]; +IF OBJECT_ID ('[sp_getApplicationsHomeSubCluster]', 'P') IS NOT NULL + DROP PROCEDURE [sp_getApplicationsHomeSubCluster]; GO -DROP PROCEDURE IF EXISTS [sp_getApplicationHomeSubCluster]; +IF OBJECT_ID ('[sp_getApplicationHomeSubCluster]', 'P') IS NOT NULL + DROP PROCEDURE [sp_getApplicationHomeSubCluster]; GO -DROP PROCEDURE IF EXISTS [sp_deleteApplicationHomeSubCluster]; +IF OBJECT_ID ('[sp_deleteApplicationHomeSubCluster]', 'P') IS NOT NULL + DROP PROCEDURE [sp_deleteApplicationHomeSubCluster]; GO -DROP PROCEDURE IF EXISTS [sp_registerSubCluster]; +IF OBJECT_ID ('[sp_registerSubCluster]', 'P') IS NOT NULL + DROP PROCEDURE [sp_registerSubCluster]; GO -DROP PROCEDURE IF EXISTS [sp_getSubClusters]; +IF OBJECT_ID ('[sp_getSubClusters]', 'P') IS NOT NULL + DROP PROCEDURE [sp_getSubClusters]; GO -DROP PROCEDURE IF EXISTS [sp_getSubCluster]; +IF OBJECT_ID ('[sp_getSubCluster]', 'P') IS NOT NULL + DROP PROCEDURE [sp_getSubCluster]; GO -DROP PROCEDURE IF EXISTS [sp_subClusterHeartbeat]; +IF OBJECT_ID ('[sp_subClusterHeartbeat]', 'P') IS NOT NULL + DROP PROCEDURE [sp_subClusterHeartbeat]; GO -DROP PROCEDURE IF EXISTS [sp_deregisterSubCluster]; +IF OBJECT_ID ('[sp_deregisterSubCluster]', 'P') IS NOT NULL + DROP PROCEDURE [sp_deregisterSubCluster]; GO -DROP PROCEDURE IF EXISTS [sp_setPolicyConfiguration]; +IF OBJECT_ID ('[sp_setPolicyConfiguration]', 'P') IS NOT NULL + DROP PROCEDURE [sp_setPolicyConfiguration]; GO -DROP PROCEDURE IF EXISTS [sp_getPolicyConfiguration]; +IF OBJECT_ID ('[sp_getPolicyConfiguration]', 'P') IS NOT NULL + DROP PROCEDURE [sp_getPolicyConfiguration]; GO -DROP PROCEDURE IF EXISTS [sp_getPoliciesConfigurations]; +IF OBJECT_ID ('[sp_getPoliciesConfigurations]', 'P') IS NOT NULL + DROP PROCEDURE [sp_getPoliciesConfigurations]; GO -DROP PROCEDURE IF EXISTS [sp_addApplicationHomeSubCluster]; +IF OBJECT_ID ('[sp_addReservationHomeSubCluster]', 'P') IS NOT NULL + DROP PROCEDURE [sp_addReservationHomeSubCluster]; GO -DROP PROCEDURE IF EXISTS [sp_updateReservationHomeSubCluster]; +IF OBJECT_ID ('[sp_updateReservationHomeSubCluster]', 'P') IS NOT NULL + DROP PROCEDURE [sp_updateReservationHomeSubCluster]; GO -DROP PROCEDURE IF EXISTS [sp_getReservationsHomeSubCluster]; +IF OBJECT_ID ('[sp_getReservationsHomeSubCluster]', 'P') IS NOT NULL + DROP PROCEDURE [sp_getReservationsHomeSubCluster]; GO -DROP PROCEDURE IF EXISTS [sp_getReservationHomeSubCluster]; +IF OBJECT_ID ('[sp_getReservationHomeSubCluster]', 'P') IS NOT NULL + DROP PROCEDURE [sp_getReservationHomeSubCluster]; GO -DROP PROCEDURE IF EXISTS [sp_deleteReservationHomeSubCluster]; +IF OBJECT_ID ('[sp_deleteReservationHomeSubCluster]', 'P') IS NOT NULL + DROP PROCEDURE [sp_deleteReservationHomeSubCluster]; GO + +IF OBJECT_ID ('[sp_addMasterKey]', 'P') IS NOT NULL + DROP PROCEDURE [sp_addMasterKey]; +GO + +IF OBJECT_ID ('[sp_getMasterKey]', 'P') IS NOT NULL + DROP PROCEDURE [sp_getMasterKey]; +GO + +IF OBJECT_ID ('[sp_deleteMasterKey]', 'P') IS NOT NULL + DROP PROCEDURE [sp_deleteMasterKey]; +GO + +IF OBJECT_ID ('[sp_addDelegationToken]', 'P') IS NOT NULL + DROP PROCEDURE [sp_addDelegationToken]; +GO + +IF OBJECT_ID ('[sp_getDelegationToken]', 'P') IS NOT NULL + DROP PROCEDURE [sp_getDelegationToken]; +GO + +IF OBJECT_ID ('[sp_updateDelegationToken]', 'P') IS NOT NULL + DROP PROCEDURE [sp_updateDelegationToken]; +GO + +IF OBJECT_ID ('[sp_deleteDelegationToken]', 'P') IS NOT NULL + DROP PROCEDURE [sp_deleteDelegationToken]; +GO \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropTables.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropTables.sql index 9bcacb7f885..9a2188cbe19 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropTables.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropTables.sql @@ -21,14 +21,34 @@ USE [FederationStateStore] GO -DROP TABLE [applicationsHomeSubCluster]; +IF OBJECT_ID ( '[sp_deregisterSubCluster]', 'U' ) IS NOT NULL + DROP TABLE [sp_deregisterSubCluster]; GO -DROP TABLE [membership]; +IF OBJECT_ID ( '[membership]', 'U' ) IS NOT NULL + DROP TABLE [membership]; GO -DROP TABLE [policies]; +IF OBJECT_ID ( '[policies]', 'U' ) IS NOT NULL + DROP TABLE [policies]; GO -DROP TABLE [reservationsHomeSubCluster]; +IF OBJECT_ID ( '[applicationsHomeSubCluster]', 'U' ) IS NOT NULL + DROP TABLE [applicationsHomeSubCluster]; +GO + +IF OBJECT_ID ( '[reservationsHomeSubCluster]', 'U' ) IS NOT NULL + DROP TABLE [reservationsHomeSubCluster]; +GO + +IF OBJECT_ID ( '[masterKeys]', 'U' ) IS NOT NULL + DROP TABLE [masterKeys]; +GO + +IF OBJECT_ID ( '[delegationTokens]', 'U' ) IS NOT NULL + DROP TABLE [delegationTokens]; +GO + +IF OBJECT_ID ( '[sequenceTable]', 'U' ) IS NOT NULL + DROP TABLE [sequenceTable]; GO diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropUser.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropUser.sql index 6d5203a52e5..2d338606e76 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropUser.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropUser.sql @@ -18,5 +18,11 @@ -- Script to drop the user from Federation StateStore in MySQL -DROP USER 'FederationUser'; +USE [FederationStateStore] +GO + +DROP USER FederationUser; +GO + +DROP LOGIN FederationUser; GO \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java index 41ade680be2..d44c30eef28 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java @@ -494,8 +494,7 @@ public class MemoryFederationStateStore implements FederationStateStore { RouterStoreToken storeToken = request.getRouterStoreToken(); RMDelegationTokenIdentifier tokenIdentifier = (RMDelegationTokenIdentifier) storeToken.getTokenIdentifier(); - Long renewDate = storeToken.getRenewDate(); - storeOrUpdateRouterRMDT(tokenIdentifier, renewDate, false); + storeOrUpdateRouterRMDT(tokenIdentifier, storeToken, false); return RouterRMTokenResponse.newInstance(storeToken); } @@ -505,10 +504,10 @@ public class MemoryFederationStateStore implements FederationStateStore { RouterStoreToken storeToken = request.getRouterStoreToken(); RMDelegationTokenIdentifier tokenIdentifier = (RMDelegationTokenIdentifier) storeToken.getTokenIdentifier(); - Long renewDate = storeToken.getRenewDate(); - Map rmDTState = routerRMSecretManagerState.getTokenState(); + Map rmDTState = + routerRMSecretManagerState.getTokenState(); rmDTState.remove(tokenIdentifier); - storeOrUpdateRouterRMDT(tokenIdentifier, renewDate, true); + storeOrUpdateRouterRMDT(tokenIdentifier, storeToken, true); return RouterRMTokenResponse.newInstance(storeToken); } @@ -518,7 +517,8 @@ public class MemoryFederationStateStore implements FederationStateStore { RouterStoreToken storeToken = request.getRouterStoreToken(); RMDelegationTokenIdentifier tokenIdentifier = (RMDelegationTokenIdentifier) storeToken.getTokenIdentifier(); - Map rmDTState = routerRMSecretManagerState.getTokenState(); + Map rmDTState = + routerRMSecretManagerState.getTokenState(); rmDTState.remove(tokenIdentifier); return RouterRMTokenResponse.newInstance(storeToken); } @@ -529,13 +529,13 @@ public class MemoryFederationStateStore implements FederationStateStore { RouterStoreToken storeToken = request.getRouterStoreToken(); RMDelegationTokenIdentifier tokenIdentifier = (RMDelegationTokenIdentifier) storeToken.getTokenIdentifier(); - Map rmDTState = routerRMSecretManagerState.getTokenState(); + Map rmDTState = + routerRMSecretManagerState.getTokenState(); if (!rmDTState.containsKey(tokenIdentifier)) { LOG.info("Router RMDelegationToken: {} does not exist.", tokenIdentifier); throw new IOException("Router RMDelegationToken: " + tokenIdentifier + " does not exist."); } - RouterStoreToken resultToken = - RouterStoreToken.newInstance(tokenIdentifier, rmDTState.get(tokenIdentifier)); + RouterStoreToken resultToken = rmDTState.get(tokenIdentifier); return RouterRMTokenResponse.newInstance(resultToken); } @@ -565,13 +565,14 @@ public class MemoryFederationStateStore implements FederationStateStore { } private void storeOrUpdateRouterRMDT(RMDelegationTokenIdentifier rmDTIdentifier, - Long renewDate, boolean isUpdate) throws IOException { - Map rmDTState = routerRMSecretManagerState.getTokenState(); + RouterStoreToken routerStoreToken, boolean isUpdate) throws IOException { + Map rmDTState = + routerRMSecretManagerState.getTokenState(); if (rmDTState.containsKey(rmDTIdentifier)) { LOG.info("Error storing info for RMDelegationToken: {}.", rmDTIdentifier); throw new IOException("Router RMDelegationToken: " + rmDTIdentifier + "is already stored."); } - rmDTState.put(rmDTIdentifier, renewDate); + rmDTState.put(rmDTIdentifier, routerStoreToken); if (!isUpdate) { routerRMSecretManagerState.setDtSequenceNumber(rmDTIdentifier.getSequenceNumber()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java index 1e3f3a12f3d..f16fe673ce3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java @@ -33,10 +33,12 @@ import java.util.TimeZone; import org.apache.commons.lang3.NotImplementedException; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.token.delegation.DelegationKey; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ReservationId; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; import org.apache.hadoop.yarn.server.federation.store.FederationStateStore; import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException; import org.apache.hadoop.yarn.server.federation.store.metrics.FederationStateStoreClientMetrics; @@ -86,11 +88,19 @@ import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyReq import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyResponse; import org.apache.hadoop.yarn.server.federation.store.records.RouterRMTokenRequest; import org.apache.hadoop.yarn.server.federation.store.records.RouterRMTokenResponse; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; +import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; import org.apache.hadoop.yarn.server.federation.store.utils.FederationApplicationHomeSubClusterStoreInputValidator; import org.apache.hadoop.yarn.server.federation.store.utils.FederationMembershipStateStoreInputValidator; import org.apache.hadoop.yarn.server.federation.store.utils.FederationPolicyStoreInputValidator; -import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils; import org.apache.hadoop.yarn.server.federation.store.utils.FederationReservationHomeSubClusterStoreInputValidator; +import org.apache.hadoop.yarn.server.federation.store.utils.FederationRouterRMTokenInputValidator; +import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils; +import org.apache.hadoop.yarn.server.federation.store.sql.FederationSQLOutParameter; +import org.apache.hadoop.yarn.server.federation.store.sql.FederationQueryRunner; +import org.apache.hadoop.yarn.server.federation.store.sql.RouterMasterKeyHandler; +import org.apache.hadoop.yarn.server.federation.store.sql.RouterStoreTokenHandler; +import org.apache.hadoop.yarn.server.federation.store.sql.RowCountHandler; import org.apache.hadoop.yarn.server.records.Version; import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.MonotonicClock; @@ -100,6 +110,13 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.VisibleForTesting; import com.zaxxer.hikari.HikariDataSource; +import static java.sql.Types.INTEGER; +import static java.sql.Types.VARCHAR; +import static java.sql.Types.BIGINT; +import static org.apache.hadoop.yarn.server.federation.store.sql.FederationQueryRunner.YARN_ROUTER_CURRENT_KEY_ID; +import static org.apache.hadoop.yarn.server.federation.store.sql.FederationQueryRunner.YARN_ROUTER_SEQUENCE_NUM; +import static org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils.convertMasterKeyToDelegationKey; + /** * SQL implementation of {@link FederationStateStore}. */ @@ -164,6 +181,27 @@ public class SQLFederationStateStore implements FederationStateStore { protected static final String CALL_SP_UPDATE_RESERVATION_HOME_SUBCLUSTER = "{call sp_updateReservationHomeSubCluster(?, ?, ?)}"; + protected static final String CALL_SP_ADD_MASTERKEY = + "{call sp_addMasterKey(?, ?, ?)}"; + + protected static final String CALL_SP_GET_MASTERKEY = + "{call sp_getMasterKey(?, ?)}"; + + protected static final String CALL_SP_DELETE_MASTERKEY = + "{call sp_deleteMasterKey(?, ?)}"; + + protected static final String CALL_SP_ADD_DELEGATIONTOKEN = + "{call sp_addDelegationToken(?, ?, ?, ?, ?)}"; + + protected static final String CALL_SP_GET_DELEGATIONTOKEN = + "{call sp_getDelegationToken(?, ?, ?, ?)}"; + + protected static final String CALL_SP_UPDATE_DELEGATIONTOKEN = + "{call sp_updateDelegationToken(?, ?, ?, ?, ?)}"; + + protected static final String CALL_SP_DELETE_DELEGATIONTOKEN = + "{call sp_deleteDelegationToken(?, ?)}"; + private Calendar utcCalendar = Calendar.getInstance(TimeZone.getTimeZone("UTC")); @@ -247,7 +285,7 @@ public class SQLFederationStateStore implements FederationStateStore { cstmt.setString("state_IN", subClusterInfo.getState().toString()); cstmt.setLong("lastStartTime_IN", subClusterInfo.getLastStartTime()); cstmt.setString("capability_IN", subClusterInfo.getCapability()); - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -302,7 +340,7 @@ public class SQLFederationStateStore implements FederationStateStore { // Set the parameters for the stored procedure cstmt.setString("subClusterId_IN", subClusterId.getId()); cstmt.setString("state_IN", state.toString()); - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -356,7 +394,7 @@ public class SQLFederationStateStore implements FederationStateStore { cstmt.setString("subClusterId_IN", subClusterId.getId()); cstmt.setString("state_IN", state.toString()); cstmt.setString("capability_IN", subClusterHeartbeatRequest.getCapability()); - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -408,14 +446,14 @@ public class SQLFederationStateStore implements FederationStateStore { cstmt.setString("subClusterId_IN", subClusterId.getId()); // Set the parameters for the stored procedure - cstmt.registerOutParameter("amRMServiceAddress_OUT", java.sql.Types.VARCHAR); - cstmt.registerOutParameter("clientRMServiceAddress_OUT", java.sql.Types.VARCHAR); - cstmt.registerOutParameter("rmAdminServiceAddress_OUT", java.sql.Types.VARCHAR); - cstmt.registerOutParameter("rmWebServiceAddress_OUT", java.sql.Types.VARCHAR); + cstmt.registerOutParameter("amRMServiceAddress_OUT", VARCHAR); + cstmt.registerOutParameter("clientRMServiceAddress_OUT", VARCHAR); + cstmt.registerOutParameter("rmAdminServiceAddress_OUT", VARCHAR); + cstmt.registerOutParameter("rmWebServiceAddress_OUT", VARCHAR); cstmt.registerOutParameter("lastHeartBeat_OUT", java.sql.Types.TIMESTAMP); - cstmt.registerOutParameter("state_OUT", java.sql.Types.VARCHAR); - cstmt.registerOutParameter("lastStartTime_OUT", java.sql.Types.BIGINT); - cstmt.registerOutParameter("capability_OUT", java.sql.Types.VARCHAR); + cstmt.registerOutParameter("state_OUT", VARCHAR); + cstmt.registerOutParameter("lastStartTime_OUT", BIGINT); + cstmt.registerOutParameter("capability_OUT", VARCHAR); // Execute the query long startTime = clock.getTime(); @@ -548,8 +586,8 @@ public class SQLFederationStateStore implements FederationStateStore { // Set the parameters for the stored procedure cstmt.setString("applicationId_IN", appId.toString()); cstmt.setString("homeSubCluster_IN", subClusterId.getId()); - cstmt.registerOutParameter("storedHomeSubCluster_OUT", java.sql.Types.VARCHAR); - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("storedHomeSubCluster_OUT", VARCHAR); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -625,7 +663,7 @@ public class SQLFederationStateStore implements FederationStateStore { // Set the parameters for the stored procedure cstmt.setString("applicationId_IN", appId.toString()); cstmt.setString("homeSubCluster_IN", subClusterId.getId()); - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -677,7 +715,7 @@ public class SQLFederationStateStore implements FederationStateStore { // Set the parameters for the stored procedure cstmt.setString("applicationId_IN", applicationId.toString()); - cstmt.registerOutParameter("homeSubCluster_OUT", java.sql.Types.VARCHAR); + cstmt.registerOutParameter("homeSubCluster_OUT", VARCHAR); // Execute the query long startTime = clock.getTime(); @@ -775,7 +813,7 @@ public class SQLFederationStateStore implements FederationStateStore { // Set the parameters for the stored procedure cstmt.setString("applicationId_IN", applicationId.toString()); - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -825,7 +863,7 @@ public class SQLFederationStateStore implements FederationStateStore { // Set the parameters for the stored procedure cstmt.setString("queue_IN", request.getQueue()); - cstmt.registerOutParameter("policyType_OUT", java.sql.Types.VARCHAR); + cstmt.registerOutParameter("policyType_OUT", VARCHAR); cstmt.registerOutParameter("params_OUT", java.sql.Types.VARBINARY); // Execute the query @@ -877,7 +915,7 @@ public class SQLFederationStateStore implements FederationStateStore { cstmt.setString("queue_IN", policyConf.getQueue()); cstmt.setString("policyType_IN", policyConf.getType()); cstmt.setBytes("params_IN", getByteArray(policyConf.getParams())); - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -984,6 +1022,22 @@ public class SQLFederationStateStore implements FederationStateStore { return dataSource.getConnection(); } + /** + * Get a connection from the DataSource pool. + * + * @param isCommitted Whether to enable automatic transaction commit. + * If set to true, turn on transaction autocommit, + * if set to false, turn off transaction autocommit. + * + * @return a connection from the DataSource pool. + * @throws SQLException on failure. + */ + protected Connection getConnection(boolean isCommitted) throws SQLException { + Connection dbConn = getConnection(); + dbConn.setAutoCommit(isCommitted); + return dbConn; + } + @VisibleForTesting protected CallableStatement getCallableStatement(String procedure) throws SQLException { @@ -1029,9 +1083,9 @@ public class SQLFederationStateStore implements FederationStateStore { // 2)IN homeSubCluster_IN varchar(256) cstmt.setString("homeSubCluster_IN", subClusterId.getId()); // 3) OUT storedHomeSubCluster_OUT varchar(256) - cstmt.registerOutParameter("storedHomeSubCluster_OUT", java.sql.Types.VARCHAR); + cstmt.registerOutParameter("storedHomeSubCluster_OUT", VARCHAR); // 4) OUT rowCount_OUT int - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -1119,7 +1173,7 @@ public class SQLFederationStateStore implements FederationStateStore { // 1)IN reservationId_IN varchar(128) cstmt.setString("reservationId_IN", reservationId.toString()); // 2)OUT homeSubCluster_OUT varchar(256) - cstmt.registerOutParameter("homeSubCluster_OUT", java.sql.Types.VARCHAR); + cstmt.registerOutParameter("homeSubCluster_OUT", VARCHAR); // Execute the query long startTime = clock.getTime(); @@ -1237,7 +1291,7 @@ public class SQLFederationStateStore implements FederationStateStore { // 1)IN reservationId_IN varchar(128) cstmt.setString("reservationId_IN", reservationId.toString()); // 2)OUT rowCount_OUT int - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -1306,7 +1360,7 @@ public class SQLFederationStateStore implements FederationStateStore { // 2)IN homeSubCluster_IN varchar(256) cstmt.setString("homeSubCluster_IN", subClusterId.getId()); // 3)OUT rowCount_OUT int - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -1353,70 +1407,503 @@ public class SQLFederationStateStore implements FederationStateStore { return conn; } + /** + * SQLFederationStateStore Supports Store New MasterKey. + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey. + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterMasterKeyResponse storeNewMasterKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + // Step1: Verify parameters to ensure that key fields are not empty. + FederationRouterRMTokenInputValidator.validate(request); + + // Step2: Parse the parameters and serialize the DelegationKey as a string. + DelegationKey delegationKey = convertMasterKeyToDelegationKey(request); + int keyId = delegationKey.getKeyId(); + String delegationKeyStr = FederationStateStoreUtils.encodeWritable(delegationKey); + + // Step3. store data in database. + try { + + FederationSQLOutParameter rowCountOUT = + new FederationSQLOutParameter<>("rowCount_OUT", INTEGER, Integer.class); + + // Execute the query + long startTime = clock.getTime(); + Integer rowCount = getRowCountByProcedureSQL(CALL_SP_ADD_MASTERKEY, keyId, + delegationKeyStr, rowCountOUT); + long stopTime = clock.getTime(); + + // We hope that 1 record can be written to the database. + // If the number of records is not 1, it means that the data was written incorrectly. + if (rowCount != 1) { + FederationStateStoreUtils.logAndThrowStoreException(LOG, + "Wrong behavior during the insertion of masterKey, keyId = %s. " + + "please check the records of the database.", String.valueOf(keyId)); + } + FederationStateStoreClientMetrics.succeededStateStoreCall(stopTime - startTime); + } catch (SQLException e) { + FederationStateStoreClientMetrics.failedStateStoreCall(); + FederationStateStoreUtils.logAndThrowRetriableException(e, LOG, + "Unable to insert the newly masterKey, keyId = %s.", String.valueOf(keyId)); + } + + // Step4. Query Data from the database and return the result. + return getMasterKeyByDelegationKey(request); } + /** + * SQLFederationStateStore Supports Remove MasterKey. + * + * Defined the sp_deleteMasterKey procedure. + * This procedure requires 1 input parameters, 1 output parameters. + * Input parameters + * 1. IN keyId_IN int + * Output parameters + * 2. OUT rowCount_OUT int + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterMasterKeyResponse removeStoredMasterKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + // Step1: Verify parameters to ensure that key fields are not empty. + FederationRouterRMTokenInputValidator.validate(request); + + // Step2: Parse parameters and get KeyId. + RouterMasterKey paramMasterKey = request.getRouterMasterKey(); + int paramKeyId = paramMasterKey.getKeyId(); + + // Step3. Clear data from database. + try { + + // Execute the query + long startTime = clock.getTime(); + FederationSQLOutParameter rowCountOUT = + new FederationSQLOutParameter<>("rowCount_OUT", INTEGER, Integer.class); + Integer rowCount = getRowCountByProcedureSQL(CALL_SP_DELETE_MASTERKEY, + paramKeyId, rowCountOUT); + long stopTime = clock.getTime(); + + // if it is equal to 0 it means the call + // did not delete the reservation from FederationStateStore + if (rowCount == 0) { + FederationStateStoreUtils.logAndThrowStoreException(LOG, + "masterKeyId = %s does not exist.", String.valueOf(paramKeyId)); + } else if (rowCount != 1) { + // if it is different from 1 it means the call + // had a wrong behavior. Maybe the database is not set correctly. + FederationStateStoreUtils.logAndThrowStoreException(LOG, + "Wrong behavior during deleting the keyId %s. " + + "The database is expected to delete 1 record, " + + "but the number of deleted records returned by the database is greater than 1, " + + "indicating that a duplicate masterKey occurred during the deletion process.", + paramKeyId); + } + + LOG.info("Delete from the StateStore the keyId: {}.", paramKeyId); + FederationStateStoreClientMetrics.succeededStateStoreCall(stopTime - startTime); + return RouterMasterKeyResponse.newInstance(paramMasterKey); + } catch (SQLException e) { + FederationStateStoreClientMetrics.failedStateStoreCall(); + FederationStateStoreUtils.logAndThrowRetriableException(e, LOG, + "Unable to delete the keyId %s.", paramKeyId); + } + + throw new YarnException("Unable to delete the masterKey, keyId = " + paramKeyId); } + /** + * SQLFederationStateStore Supports Remove MasterKey. + * + * Defined the sp_getMasterKey procedure. + * this procedure requires 2 parameters. + * Input parameters: + * 1. IN keyId_IN int + * Output parameters: + * 2. OUT masterKey_OUT varchar(1024) + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterMasterKeyResponse getMasterKeyByDelegationKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + // Step1: Verify parameters to ensure that key fields are not empty. + FederationRouterRMTokenInputValidator.validate(request); + + // Step2: Parse parameters and get KeyId. + RouterMasterKey paramMasterKey = request.getRouterMasterKey(); + int paramKeyId = paramMasterKey.getKeyId(); + + // Step3: Call the stored procedure to get the result. + try { + + FederationQueryRunner runner = new FederationQueryRunner(); + FederationSQLOutParameter masterKeyOUT = + new FederationSQLOutParameter<>("masterKey_OUT", VARCHAR, String.class); + + // Execute the query + long startTime = clock.getTime(); + RouterMasterKey routerMasterKey = runner.execute( + conn, CALL_SP_GET_MASTERKEY, new RouterMasterKeyHandler(), paramKeyId, masterKeyOUT); + long stopTime = clock.getTime(); + + LOG.info("Got the information about the specified masterKey = {} according to keyId = {}.", + routerMasterKey, paramKeyId); + + FederationStateStoreClientMetrics.succeededStateStoreCall(stopTime - startTime); + + // Return query result. + return RouterMasterKeyResponse.newInstance(routerMasterKey); + + } catch (SQLException e) { + FederationStateStoreClientMetrics.failedStateStoreCall(); + FederationStateStoreUtils.logAndThrowRetriableException(e, LOG, + "Unable to obtain the masterKey information according to %s.", + String.valueOf(paramKeyId)); + } + + // Throw exception information + throw new YarnException( + "Unable to obtain the masterKey information according to " + paramKeyId); } + /** + * SQLFederationStateStore Supports Store RMDelegationTokenIdentifier. + * + * Defined the sp_addDelegationToken procedure. + * This procedure requires 4 input parameters, 1 output parameters. + * Input parameters: + * 1. IN sequenceNum_IN int + * 2. IN tokenIdent_IN varchar(1024) + * 3. IN token_IN varchar(1024) + * 4. IN renewDate_IN bigint + * Output parameters: + * 5. OUT rowCount_OUT int + * + * @param request The request contains RouterRMToken (RMDelegationTokenIdentifier and renewDate) + * @return routerRMTokenResponse, the response contains the RouterStoreToken. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterRMTokenResponse storeNewToken(RouterRMTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + // Step1: Verify parameters to ensure that key fields are not empty. + FederationRouterRMTokenInputValidator.validate(request); + + // Step2. store data in database. + try { + long duration = addOrUpdateToken(request, true); + FederationStateStoreClientMetrics.succeededStateStoreCall(duration); + } catch (SQLException e) { + FederationStateStoreClientMetrics.failedStateStoreCall(); + throw new YarnException(e); + } + + // Step3. Query Data from the database and return the result. + return getTokenByRouterStoreToken(request); } + /** + * SQLFederationStateStore Supports Update RMDelegationTokenIdentifier. + * + * Defined the sp_updateDelegationToken procedure. + * This procedure requires 4 input parameters, 1 output parameters. + * Input parameters: + * 1. IN sequenceNum_IN int + * 2. IN tokenIdent_IN varchar(1024) + * 3. IN token_IN varchar(1024) + * 4. IN renewDate_IN bigint + * Output parameters: + * 5. OUT rowCount_OUT int + * + * @param request The request contains RouterRMToken (RMDelegationTokenIdentifier and renewDate) + * @return routerRMTokenResponse, the response contains the RouterStoreToken. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterRMTokenResponse updateStoredToken(RouterRMTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + // Step1: Verify parameters to ensure that key fields are not empty. + FederationRouterRMTokenInputValidator.validate(request); + + // Step2. update data in database. + try { + long duration = addOrUpdateToken(request, false); + FederationStateStoreClientMetrics.succeededStateStoreCall(duration); + } catch (SQLException e) { + FederationStateStoreClientMetrics.failedStateStoreCall(); + throw new YarnException(e); + } + + // Step3. Query Data from the database and return the result. + return getTokenByRouterStoreToken(request); } + /** + * Add Or Update RMDelegationTokenIdentifier. + * + * @param request The request contains RouterRMToken (RMDelegationTokenIdentifier and renewDate) + * @param isAdd true, addData; false, updateData. + * @return method operation time. + * @throws IOException An IO Error occurred. + * @throws SQLException An SQL Error occurred. + * @throws YarnException if the call to the state store is unsuccessful. + */ + private long addOrUpdateToken(RouterRMTokenRequest request, boolean isAdd) + throws IOException, SQLException, YarnException { + + // Parse parameters and get KeyId. + RouterStoreToken routerStoreToken = request.getRouterStoreToken(); + YARNDelegationTokenIdentifier identifier = routerStoreToken.getTokenIdentifier(); + String tokenIdentifier = FederationStateStoreUtils.encodeWritable(identifier); + String tokenInfo = routerStoreToken.getTokenInfo(); + long renewDate = routerStoreToken.getRenewDate(); + int sequenceNum = identifier.getSequenceNumber(); + + FederationQueryRunner runner = new FederationQueryRunner(); + FederationSQLOutParameter rowCountOUT = + new FederationSQLOutParameter<>("rowCount_OUT", INTEGER, Integer.class); + + // Execute the query + long startTime = clock.getTime(); + String procedure = isAdd ? CALL_SP_ADD_DELEGATIONTOKEN : CALL_SP_UPDATE_DELEGATIONTOKEN; + Integer rowCount = runner.execute(conn, procedure, new RowCountHandler("rowCount_OUT"), + sequenceNum, tokenIdentifier, tokenInfo, renewDate, rowCountOUT); + long stopTime = clock.getTime(); + + // Get rowCount + // In the process of updating the code, rowCount may be 0 or 1; + // if rowCount=1, it is as expected, indicating that we have updated the Token correctly; + // if rowCount=0, it is not as expected, + // indicating that we have not updated the Token correctly. + if (rowCount != 1) { + FederationStateStoreUtils.logAndThrowStoreException(LOG, + "Wrong behavior during the insertion of delegationToken, tokenId = %s. " + + "Please check the records of the database.", String.valueOf(sequenceNum)); + } + + // return execution time + return (stopTime - startTime); + } + + /** + * SQLFederationStateStore Supports Remove RMDelegationTokenIdentifier. + * + * Defined the sp_deleteDelegationToken procedure. + * This procedure requires 1 input parameters, 1 output parameters. + * Input parameters: + * 1. IN sequenceNum_IN bigint + * Output parameters: + * 2. OUT rowCount_OUT int + * + * @param request The request contains RouterRMToken (RMDelegationTokenIdentifier and renewDate) + * @return routerRMTokenResponse, the response contains the RouterStoreToken. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterRMTokenResponse removeStoredToken(RouterRMTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + // Step1: Verify parameters to ensure that key fields are not empty. + FederationRouterRMTokenInputValidator.validate(request); + + // Step2: Parse parameters and get KeyId. + RouterStoreToken routerStoreToken = request.getRouterStoreToken(); + YARNDelegationTokenIdentifier identifier = routerStoreToken.getTokenIdentifier(); + int sequenceNum = identifier.getSequenceNumber(); + + try { + + FederationSQLOutParameter rowCountOUT = + new FederationSQLOutParameter<>("rowCount_OUT", INTEGER, Integer.class); + + // Execute the query + long startTime = clock.getTime(); + Integer rowCount = getRowCountByProcedureSQL(CALL_SP_DELETE_DELEGATIONTOKEN, + sequenceNum, rowCountOUT); + long stopTime = clock.getTime(); + + // if it is equal to 0 it means the call + // did not delete the reservation from FederationStateStore + if (rowCount == 0) { + FederationStateStoreUtils.logAndThrowStoreException(LOG, + "TokenId %s does not exist", String.valueOf(sequenceNum)); + } else if (rowCount != 1) { + // if it is different from 1 it means the call + // had a wrong behavior. Maybe the database is not set correctly. + FederationStateStoreUtils.logAndThrowStoreException(LOG, + "Wrong behavior during deleting the delegationToken %s. " + + "The database is expected to delete 1 record, " + + "but the number of deleted records returned by the database is greater than 1, " + + "indicating that a duplicate tokenId occurred during the deletion process.", + String.valueOf(sequenceNum)); + } + + LOG.info("Delete from the StateStore the delegationToken, tokenId = {}.", sequenceNum); + FederationStateStoreClientMetrics.succeededStateStoreCall(stopTime - startTime); + return RouterRMTokenResponse.newInstance(routerStoreToken); + } catch (SQLException e) { + FederationStateStoreClientMetrics.failedStateStoreCall(); + FederationStateStoreUtils.logAndThrowRetriableException(e, LOG, + "Unable to delete the delegationToken, tokenId = %s.", sequenceNum); + } + throw new YarnException("Unable to delete the delegationToken, tokenId = " + sequenceNum); } + /** + * The Router Supports GetTokenByRouterStoreToken. + * + * @param request The request contains RouterRMToken (RMDelegationTokenIdentifier and renewDate) + * @return RouterRMTokenResponse. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterRMTokenResponse getTokenByRouterStoreToken(RouterRMTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + // Step1: Verify parameters to ensure that key fields are not empty. + FederationRouterRMTokenInputValidator.validate(request); + + // Step2: Parse parameters and get KeyId. + RouterStoreToken routerStoreToken = request.getRouterStoreToken(); + YARNDelegationTokenIdentifier identifier = routerStoreToken.getTokenIdentifier(); + int sequenceNum = identifier.getSequenceNumber(); + + try { + FederationQueryRunner runner = new FederationQueryRunner(); + FederationSQLOutParameter tokenIdentOUT = + new FederationSQLOutParameter<>("tokenIdent_OUT", VARCHAR, String.class); + FederationSQLOutParameter tokenOUT = + new FederationSQLOutParameter<>("token_OUT", VARCHAR, String.class); + FederationSQLOutParameter renewDateOUT = + new FederationSQLOutParameter<>("renewDate_OUT", BIGINT, Long.class); + + // Execute the query + long startTime = clock.getTime(); + RouterStoreToken resultToken = runner.execute(conn, CALL_SP_GET_DELEGATIONTOKEN, + new RouterStoreTokenHandler(), sequenceNum, tokenIdentOUT, tokenOUT, renewDateOUT); + long stopTime = clock.getTime(); + + FederationStateStoreClientMetrics.succeededStateStoreCall(stopTime - startTime); + return RouterRMTokenResponse.newInstance(resultToken); + } catch (SQLException e) { + FederationStateStoreClientMetrics.failedStateStoreCall(); + FederationStateStoreUtils.logAndThrowRetriableException(e, LOG, + "Unable to get the delegationToken, tokenId = %s.", String.valueOf(sequenceNum)); + } + + // Throw exception information + throw new YarnException("Unable to get the delegationToken, tokenId = " + sequenceNum); } + /** + * Call Procedure to get RowCount. + * + * @param procedure procedureSQL. + * @param params procedure params. + * @return RowCount. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + private int getRowCountByProcedureSQL(String procedure, Object... params) throws SQLException { + FederationQueryRunner runner = new FederationQueryRunner(); + // Execute the query + Integer rowCount = runner.execute(conn, procedure, + new RowCountHandler("rowCount_OUT"), params); + return rowCount; + } + + /** + * Increment DelegationToken SeqNum. + * + * @return delegationTokenSeqNum. + */ @Override public int incrementDelegationTokenSeqNum() { - return 0; + return querySequenceTable(YARN_ROUTER_SEQUENCE_NUM, true); } + /** + * Get DelegationToken SeqNum. + * + * @return delegationTokenSeqNum. + */ @Override public int getDelegationTokenSeqNum() { - return 0; + return querySequenceTable(YARN_ROUTER_SEQUENCE_NUM, false); } @Override public void setDelegationTokenSeqNum(int seqNum) { - return; + Connection connection = null; + try { + connection = getConnection(false); + FederationQueryRunner runner = new FederationQueryRunner(); + runner.updateSequenceTable(connection, YARN_ROUTER_SEQUENCE_NUM, seqNum); + } catch (Exception e) { + throw new RuntimeException("Could not update sequence table!!", e); + } finally { + // Return to the pool the CallableStatement + try { + FederationStateStoreUtils.returnToPool(LOG, null, connection); + } catch (YarnException e) { + LOG.error("close connection error.", e); + } + } } + /** + * Get Current KeyId. + * + * @return currentKeyId. + */ @Override public int getCurrentKeyId() { - return 0; + return querySequenceTable(YARN_ROUTER_CURRENT_KEY_ID, false); } + /** + * The Router Supports incrementCurrentKeyId. + * + * @return CurrentKeyId. + */ @Override public int incrementCurrentKeyId() { - return 0; + return querySequenceTable(YARN_ROUTER_CURRENT_KEY_ID, true); + } + + private int querySequenceTable(String sequenceName, boolean isUpdate){ + Connection connection = null; + try { + connection = getConnection(false); + FederationQueryRunner runner = new FederationQueryRunner(); + return runner.selectOrUpdateSequenceTable(connection, sequenceName, isUpdate); + } catch (Exception e) { + throw new RuntimeException("Could not query sequence table!!", e); + } finally { + // Return to the pool the CallableStatement + try { + FederationStateStoreUtils.returnToPool(LOG, null, connection); + } catch (YarnException e) { + LOG.error("close connection error.", e); + } + } } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterRMDTSecretManagerState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterRMDTSecretManagerState.java index 85a8002c91c..62a89f419dd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterRMDTSecretManagerState.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterRMDTSecretManagerState.java @@ -28,13 +28,13 @@ import java.util.Set; public class RouterRMDTSecretManagerState { // DTIdentifier -> renewDate - private Map delegationTokenState = new HashMap<>(); + private Map delegationTokenState = new HashMap<>(); private Set masterKeyState = new HashSet<>(); private int dtSequenceNumber = 0; - public Map getTokenState() { + public Map getTokenState() { return delegationTokenState; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java index 29f86903f91..a20297399b7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java @@ -18,7 +18,9 @@ package org.apache.hadoop.yarn.server.federation.store.records; import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; import org.apache.hadoop.yarn.util.Records; @@ -39,6 +41,17 @@ public abstract class RouterStoreToken { return storeToken; } + @Private + @Unstable + public static RouterStoreToken newInstance(YARNDelegationTokenIdentifier identifier, + Long renewdate, String tokenInfo) { + RouterStoreToken storeToken = Records.newRecord(RouterStoreToken.class); + storeToken.setIdentifier(identifier); + storeToken.setRenewDate(renewdate); + storeToken.setTokenInfo(tokenInfo); + return storeToken; + } + @Private @Unstable public abstract YARNDelegationTokenIdentifier getTokenIdentifier() throws IOException; @@ -47,8 +60,8 @@ public abstract class RouterStoreToken { @Unstable public abstract void setIdentifier(YARNDelegationTokenIdentifier identifier); - @Private - @Unstable + @Public + @Stable public abstract Long getRenewDate(); @Private @@ -62,4 +75,12 @@ public abstract class RouterStoreToken { @Private @Unstable public abstract void readFields(DataInput in) throws IOException; + + @Public + @Stable + public abstract String getTokenInfo(); + + @Private + @Unstable + public abstract void setTokenInfo(String tokenInfo); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/RouterStoreTokenPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/RouterStoreTokenPBImpl.java index df6030a3f0d..a89cf06f0b1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/RouterStoreTokenPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/RouterStoreTokenPBImpl.java @@ -47,6 +47,7 @@ public class RouterStoreTokenPBImpl extends RouterStoreToken { private YARNDelegationTokenIdentifier rMDelegationTokenIdentifier = null; private Long renewDate; + private String tokenInfo; public RouterStoreTokenPBImpl() { builder = RouterStoreTokenProto.newBuilder(); @@ -84,6 +85,10 @@ public class RouterStoreTokenPBImpl extends RouterStoreToken { if (this.renewDate != null) { builder.setRenewDate(this.renewDate); } + + if (this.tokenInfo != null) { + builder.setTokenInfo(this.tokenInfo); + } } private void maybeInitBuilder() { @@ -164,6 +169,29 @@ public class RouterStoreTokenPBImpl extends RouterStoreToken { this.renewDate = renewDate; this.builder.setRenewDate(renewDate); } + @Override + public String getTokenInfo() { + RouterStoreTokenProtoOrBuilder p = viaProto ? proto : builder; + if (this.tokenInfo != null) { + return this.tokenInfo; + } + if (!p.hasTokenInfo()) { + return null; + } + this.tokenInfo = p.getTokenInfo(); + return this.tokenInfo; + } + + @Override + public void setTokenInfo(String tokenInfo) { + maybeInitBuilder(); + if (tokenInfo == null) { + builder.clearTokenInfo(); + return; + } + this.tokenInfo = tokenInfo; + this.builder.setTokenInfo(tokenInfo); + } private YARNDelegationTokenIdentifierProto convertToProtoFormat( YARNDelegationTokenIdentifier delegationTokenIdentifier) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/DatabaseProduct.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/DatabaseProduct.java new file mode 100644 index 00000000000..3b3af242390 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/DatabaseProduct.java @@ -0,0 +1,126 @@ +/** + * 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.yarn.server.federation.store.sql; + +import org.apache.hadoop.classification.InterfaceAudience.Private; + +import java.sql.Connection; +import java.sql.SQLException; + +@Private +public final class DatabaseProduct { + + public enum DbType {MYSQL, SQLSERVER, POSTGRES, UNDEFINED, HSQLDB} + + private static final String SQL_SERVER_NAME = "sqlserver"; + private static final String MYSQL_NAME = "mysql"; + private static final String MARIADB_NAME = "mariadb"; + private static final String HSQLDB_NAME = "hsqldatabase"; + + private DatabaseProduct() { + } + + public static DbType getDbType(Connection conn) throws SQLException { + if (conn == null) { + return DbType.UNDEFINED; + } + String productName = getProductName(conn); + return getDbType(productName); + } + + /** + * We get DBType based on ProductName. + * + * @param productName productName. + * @return DbType. + */ + private static DbType getDbType(String productName) { + DbType dbt; + productName = productName.replaceAll("\\s+", "").toLowerCase(); + if (productName.contains(SQL_SERVER_NAME)) { + dbt = DbType.SQLSERVER; + } else if (productName.contains(MYSQL_NAME) || productName.contains(MARIADB_NAME)) { + dbt = DbType.MYSQL; + } else if (productName.contains(HSQLDB_NAME)) { + dbt = DbType.HSQLDB; + } else { + dbt = DbType.UNDEFINED; + } + return dbt; + } + + /** + * We get ProductName based on metadata in SQL Connection. + * + * @param conn SQL Connection + * @return DB ProductName (Like MySQL SQLSERVER etc.) + */ + private static String getProductName(Connection conn) throws SQLException { + return conn.getMetaData().getDatabaseProductName(); + } + + /** + * We add for update to SQL according to different database types. + * This statement can ensure that a row of records in the database is only updated by one thread. + * + * @param dbType type of database. + * @param selectStatement querySQL. + * @return SQL after adding for update. + * @throws SQLException SQL exception. + */ + public static String addForUpdateClause(DbType dbType, String selectStatement) + throws SQLException { + switch (dbType) { + case MYSQL: + case HSQLDB: + return selectStatement + " for update"; + case SQLSERVER: + String modifier = " with (updlock)"; + int wherePos = selectStatement.toUpperCase().indexOf(" WHERE "); + if (wherePos < 0) { + return selectStatement + modifier; + } + return selectStatement.substring(0, wherePos) + modifier + + selectStatement.substring(wherePos, selectStatement.length()); + default: + String msg = "Unrecognized database product name <" + dbType + ">"; + throw new SQLException(msg); + } + } + + public static boolean isDuplicateKeyError(DbType dbType, SQLException ex) { + switch (dbType) { + case MYSQL: + if((ex.getErrorCode() == 1022 || ex.getErrorCode() == 1062 || ex.getErrorCode() == 1586) && + "23000".equals(ex.getSQLState())) { + return true; + } + break; + case SQLSERVER: + if ((ex.getErrorCode() == 2627 || ex.getErrorCode() == 2601) + && "23000".equals(ex.getSQLState())) { + return true; + } + break; + default: + return false; + } + return false; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationQueryRunner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationQueryRunner.java new file mode 100644 index 00000000000..0cca531d0d9 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationQueryRunner.java @@ -0,0 +1,310 @@ +/** + * 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.yarn.server.federation.store.sql; + +import org.apache.hadoop.classification.VisibleForTesting; + +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.CallableStatement; +import java.sql.ResultSet; +import java.util.Arrays; + +import org.apache.hadoop.yarn.server.federation.store.sql.DatabaseProduct.DbType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.hadoop.yarn.server.federation.store.sql.DatabaseProduct.isDuplicateKeyError; + +/** + * QueryRunner is used to execute stored procedure SQL and parse the returned results. + */ +public class FederationQueryRunner { + + public final static String YARN_ROUTER_SEQUENCE_NUM = "YARN_ROUTER_SEQUENCE_NUM"; + + public final static String YARN_ROUTER_CURRENT_KEY_ID = "YARN_ROUTER_CURRENT_KEY_ID"; + + public final static String QUERY_SEQUENCE_TABLE_SQL = + "SELECT nextVal FROM sequenceTable WHERE sequenceName = %s"; + + public final static String INSERT_SEQUENCE_TABLE_SQL = "" + + "INSERT INTO sequenceTable(sequenceName, nextVal) VALUES(%s, %d)"; + + public final static String UPDATE_SEQUENCE_TABLE_SQL = "" + + "UPDATE sequenceTable SET nextVal = %d WHERE sequenceName = %s"; + + public static final Logger LOG = LoggerFactory.getLogger(FederationQueryRunner.class); + + /** + * Execute Stored Procedure SQL. + * + * @param conn Database Connection. + * @param procedure Stored Procedure SQL. + * @param rsh Result Set handler. + * @param params List of stored procedure parameters. + * @param Generic T. + * @return Stored Procedure Result Set. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + public T execute(Connection conn, String procedure, ResultSetHandler rsh, Object... params) + throws SQLException { + if (conn == null) { + throw new SQLException("Null connection"); + } + + if (procedure == null) { + throw new SQLException("Null Procedure SQL statement"); + } + + if (rsh == null) { + throw new SQLException("Null ResultSetHandler"); + } + + CallableStatement stmt = null; + T results = null; + + try { + stmt = this.getCallableStatement(conn, procedure); + this.fillStatement(stmt, params); + stmt.executeUpdate(); + this.retrieveOutParameters(stmt, params); + results = rsh.handle(params); + } catch (SQLException e) { + this.rethrow(e, procedure, params); + } finally { + close(stmt); + } + return results; + } + + /** + * Get CallableStatement from Conn. + * + * @param conn Database Connection. + * @param procedure Stored Procedure SQL. + * @return CallableStatement. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + @VisibleForTesting + protected CallableStatement getCallableStatement(Connection conn, String procedure) + throws SQLException { + return conn.prepareCall(procedure); + } + + /** + * Set Statement parameters. + * + * @param stmt CallableStatement. + * @param params Stored procedure parameters. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + public void fillStatement(CallableStatement stmt, Object... params) + throws SQLException { + for (int i = 0; i < params.length; i++) { + if (params[i] != null) { + if (stmt != null) { + if (params[i] instanceof FederationSQLOutParameter) { + FederationSQLOutParameter sqlOutParameter = (FederationSQLOutParameter) params[i]; + sqlOutParameter.register(stmt, i + 1); + } else { + stmt.setObject(i + 1, params[i]); + } + } + } + } + } + + /** + * Close Statement. + * + * @param stmt CallableStatement. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + public void close(Statement stmt) throws SQLException { + if (stmt != null) { + stmt.close(); + stmt = null; + } + } + + /** + * Retrieve execution result from CallableStatement. + * + * @param stmt CallableStatement. + * @param params Stored procedure parameters. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + private void retrieveOutParameters(CallableStatement stmt, Object[] params) throws SQLException { + if (params != null && stmt != null) { + for (int i = 0; i < params.length; i++) { + if (params[i] instanceof FederationSQLOutParameter) { + FederationSQLOutParameter sqlOutParameter = (FederationSQLOutParameter) params[i]; + sqlOutParameter.setValue(stmt, i + 1); + } + } + } + } + + /** + * Re-throw SQL exception. + * + * @param cause SQLException. + * @param sql Stored Procedure SQL. + * @param params Stored procedure parameters. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + protected void rethrow(SQLException cause, String sql, Object... params) + throws SQLException { + + String causeMessage = cause.getMessage(); + if (causeMessage == null) { + causeMessage = ""; + } + + StringBuffer msg = new StringBuffer(causeMessage); + msg.append(" Query: "); + msg.append(sql); + msg.append(" Parameters: "); + + if (params == null) { + msg.append("[]"); + } else { + msg.append(Arrays.deepToString(params)); + } + + SQLException e = new SQLException(msg.toString(), cause.getSQLState(), cause.getErrorCode()); + e.setNextException(cause); + throw e; + } + + /** + * We query or update the SequenceTable. + * + * @param connection database conn. + * @param sequenceName sequenceName, We currently have 2 sequences, + * YARN_ROUTER_SEQUENCE_NUM and YARN_ROUTER_CURRENT_KEY_ID. + * @param isUpdate true, means we will update the SequenceTable, + * false, we query the SequenceTable. + * + * @return SequenceValue. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + public int selectOrUpdateSequenceTable(Connection connection, String sequenceName, + boolean isUpdate) throws SQLException { + + int maxSequenceValue = 0; + boolean insertDone = false; + boolean committed = false; + Statement statement = null; + + try { + + // Step1. Query SequenceValue. + while (maxSequenceValue == 0) { + // Query SQL. + String sql = String.format(QUERY_SEQUENCE_TABLE_SQL, quoteString(sequenceName)); + DbType dbType = DatabaseProduct.getDbType(connection); + String forUpdateSQL = DatabaseProduct.addForUpdateClause(dbType, sql); + statement = connection.createStatement(); + ResultSet rs = statement.executeQuery(forUpdateSQL); + if (rs.next()) { + maxSequenceValue = rs.getInt("nextVal"); + } else if (insertDone) { + throw new SQLException("Invalid state of SEQUENCE_TABLE for " + sequenceName); + } else { + insertDone = true; + close(statement); + statement = connection.createStatement(); + String insertSQL = String.format(INSERT_SEQUENCE_TABLE_SQL, quoteString(sequenceName), 1); + try { + statement.executeUpdate(insertSQL); + } catch (SQLException e) { + // If the record is already inserted by some other thread continue to select. + if (isDuplicateKeyError(dbType, e)) { + continue; + } + LOG.error("Unable to insert into SEQUENCE_TABLE for {}.", sequenceName, e); + throw e; + } finally { + close(statement); + } + } + } + + // Step2. Increase SequenceValue. + if (isUpdate) { + int nextSequenceValue = maxSequenceValue + 1; + close(statement); + statement = connection.createStatement(); + String updateSQL = + String.format(UPDATE_SEQUENCE_TABLE_SQL, nextSequenceValue, quoteString(sequenceName)); + statement.executeUpdate(updateSQL); + maxSequenceValue = nextSequenceValue; + } + + connection.commit(); + committed = true; + return maxSequenceValue; + } catch(SQLException e){ + throw new SQLException("Unable to selectOrUpdateSequenceTable due to: " + e.getMessage(), e); + } finally { + if (!committed) { + rollbackDBConn(connection); + } + close(statement); + } + } + + public void updateSequenceTable(Connection connection, String sequenceName, int sequenceValue) + throws SQLException { + String updateSQL = + String.format(UPDATE_SEQUENCE_TABLE_SQL, sequenceValue, quoteString(sequenceName)); + boolean committed = false; + Statement statement = null; + try { + statement = connection.createStatement(); + statement.executeUpdate(updateSQL); + connection.commit(); + committed = true; + } catch (SQLException e) { + throw new SQLException("Unable to updateSequenceTable due to: " + e.getMessage()); + } finally { + if (!committed) { + rollbackDBConn(connection); + } + close(statement); + } + } + + static void rollbackDBConn(Connection dbConn) { + try { + if (dbConn != null && !dbConn.isClosed()) { + dbConn.rollback(); + } + } catch (SQLException e) { + LOG.warn("Failed to rollback db connection ", e); + } + } + + static String quoteString(String input) { + return "'" + input + "'"; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationSQLOutParameter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationSQLOutParameter.java new file mode 100644 index 00000000000..890e3e1e851 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationSQLOutParameter.java @@ -0,0 +1,91 @@ +/** + * 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.yarn.server.federation.store.sql; + +import java.sql.CallableStatement; +import java.sql.SQLException; + +/** + * SQLOutParameter is used to set the output parameters of the stored procedure. + * @param Generic T. + */ +public class FederationSQLOutParameter { + private final int sqlType; + private final Class javaType; + private T value = null; + private String paramName; + + public FederationSQLOutParameter(String paramName, int sqlType, Class javaType) { + this.paramName = paramName; + this.sqlType = sqlType; + this.javaType = javaType; + } + + public FederationSQLOutParameter(int sqlType, Class javaType, T value) { + this.sqlType = sqlType; + this.javaType = javaType; + this.value = value; + } + + public int getSqlType() { + return sqlType; + } + + public Class getJavaType() { + return javaType; + } + + public T getValue() { + return value; + } + + public void setValue(T value) { + this.value = value; + } + + public String getParamName() { + return paramName; + } + + public void setParamName(String paramName) { + this.paramName = paramName; + } + + void setValue(CallableStatement stmt, int index) throws SQLException { + Object object = stmt.getObject(index); + value = javaType.cast(object); + } + + void register(CallableStatement stmt, int index) throws SQLException { + stmt.registerOutParameter(index, sqlType); + if (value != null) { + stmt.setObject(index, value); + } + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("OutParameter: [") + .append("SqlType: ").append(sqlType).append(", ") + .append("JavaType: ").append(javaType).append(", ") + .append("Value: ").append(value) + .append("]"); + return sb.toString(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/ResultSetHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/ResultSetHandler.java new file mode 100644 index 00000000000..1e793d0ecbd --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/ResultSetHandler.java @@ -0,0 +1,30 @@ +/** + * 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.yarn.server.federation.store.sql; + +import java.sql.SQLException; + +/** + * Result Set Handler. + * + * @param Generic T. + */ +public interface ResultSetHandler { + T handle(Object... params) throws SQLException; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterMasterKeyHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterMasterKeyHandler.java new file mode 100644 index 00000000000..3f7b3641a51 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterMasterKeyHandler.java @@ -0,0 +1,68 @@ +/** + * 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.yarn.server.federation.store.sql; + +import org.apache.hadoop.security.token.delegation.DelegationKey; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; +import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils; +import org.apache.hadoop.yarn.util.Records; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.sql.SQLException; + +/** + * RouterMasterKey Handler. + * Used to parse the result information of the output parameter into the RouterMasterKey type. + */ +public class RouterMasterKeyHandler implements ResultSetHandler { + + private final static String MASTERKEY_OUT = "masterKey_OUT"; + + @Override + public RouterMasterKey handle(Object... params) throws SQLException { + RouterMasterKey routerMasterKey = Records.newRecord(RouterMasterKey.class); + for (Object param : params) { + if (param instanceof FederationSQLOutParameter) { + FederationSQLOutParameter parameter = (FederationSQLOutParameter) param; + String paramName = parameter.getParamName(); + Object parmaValue = parameter.getValue(); + if (StringUtils.equalsIgnoreCase(paramName, MASTERKEY_OUT)) { + DelegationKey key = getDelegationKey(parmaValue); + routerMasterKey.setKeyId(key.getKeyId()); + routerMasterKey.setKeyBytes(ByteBuffer.wrap(key.getEncodedKey())); + routerMasterKey.setExpiryDate(key.getExpiryDate()); + } + } + } + return routerMasterKey; + } + + private DelegationKey getDelegationKey(Object paramMasterKey) throws SQLException { + try { + DelegationKey key = new DelegationKey(); + String masterKey = String.valueOf(paramMasterKey); + FederationStateStoreUtils.decodeWritable(key, masterKey); + return key; + } catch (IOException e) { + throw new SQLException(e); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterStoreTokenHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterStoreTokenHandler.java new file mode 100644 index 00000000000..465fe706f42 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterStoreTokenHandler.java @@ -0,0 +1,83 @@ +/** + * 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.yarn.server.federation.store.sql; + +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; +import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; +import org.apache.hadoop.yarn.util.Records; + +import java.io.IOException; +import java.sql.SQLException; + +import static org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils.decodeWritable; + +/** + * RouterStoreToken Handler. + * Used to parse the result information of the output parameter into the RouterStoreToken type. + */ +public class RouterStoreTokenHandler implements ResultSetHandler { + + private final static String TOKENIDENT_OUT = "tokenIdent_OUT"; + private final static String TOKEN_OUT = "token_OUT"; + private final static String RENEWDATE_OUT = "renewDate_OUT"; + + @Override + public RouterStoreToken handle(Object... params) throws SQLException { + RouterStoreToken storeToken = Records.newRecord(RouterStoreToken.class); + for (Object param : params) { + if (param instanceof FederationSQLOutParameter) { + FederationSQLOutParameter parameter = (FederationSQLOutParameter) param; + String paramName = parameter.getParamName(); + Object parmaValue = parameter.getValue(); + if (StringUtils.equalsIgnoreCase(paramName, TOKENIDENT_OUT)) { + YARNDelegationTokenIdentifier identifier = getYARNDelegationTokenIdentifier(parmaValue); + storeToken.setIdentifier(identifier); + } else if (StringUtils.equalsIgnoreCase(paramName, TOKEN_OUT)) { + String tokenInfo = getTokenInfo(parmaValue); + storeToken.setTokenInfo(tokenInfo); + } else if(StringUtils.equalsIgnoreCase(paramName, RENEWDATE_OUT)){ + Long renewDate = getRenewDate(parmaValue); + storeToken.setRenewDate(renewDate); + } + } + } + return storeToken; + } + + private YARNDelegationTokenIdentifier getYARNDelegationTokenIdentifier(Object tokenIdent) + throws SQLException { + try { + YARNDelegationTokenIdentifier resultIdentifier = + Records.newRecord(YARNDelegationTokenIdentifier.class); + decodeWritable(resultIdentifier, String.valueOf(tokenIdent)); + return resultIdentifier; + } catch (IOException e) { + throw new SQLException(e); + } + } + + private String getTokenInfo(Object tokenInfo) { + return String.valueOf(tokenInfo); + } + + private Long getRenewDate(Object renewDate) { + return Long.parseLong(String.valueOf(renewDate)); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RowCountHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RowCountHandler.java new file mode 100644 index 00000000000..e9438d1198d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RowCountHandler.java @@ -0,0 +1,56 @@ +/** + * 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.yarn.server.federation.store.sql; + +import org.apache.hadoop.util.StringUtils; + +import java.sql.SQLException; + +/** + * RowCount Handler. + * Used to parse out the rowCount information of the output parameter. + */ +public class RowCountHandler implements ResultSetHandler { + + private String rowCountParamName; + + public RowCountHandler(String paramName) { + this.rowCountParamName = paramName; + } + + @Override + public Integer handle(Object... params) throws SQLException { + Integer result = 0; + for (Object param : params) { + if (param instanceof FederationSQLOutParameter) { + FederationSQLOutParameter parameter = (FederationSQLOutParameter) param; + String paramName = parameter.getParamName(); + Object parmaValue = parameter.getValue(); + if (StringUtils.equalsIgnoreCase(paramName, rowCountParamName)) { + result = getRowCount(parmaValue); + } + } + } + return result; + } + + private Integer getRowCount(Object rowCount) { + return Integer.parseInt(String.valueOf(rowCount)); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/package-info.java new file mode 100644 index 00000000000..d6bca3fd236 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/package-info.java @@ -0,0 +1,17 @@ +/** + * 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.yarn.server.federation.store.sql; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationRouterRMTokenInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationRouterRMTokenInputValidator.java index 40fe1f36cfb..1ad7cc1ce2f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationRouterRMTokenInputValidator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationRouterRMTokenInputValidator.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.yarn.server.federation.store.utils; import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java index f14867a0e65..aba8ddac2f8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java @@ -18,16 +18,27 @@ package org.apache.hadoop.yarn.server.federation.store.utils; +import java.io.IOException; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.nio.ByteBuffer; import java.sql.CallableStatement; import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; +import java.util.Base64; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.security.token.delegation.DelegationKey; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreException; import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException; import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreRetriableException; import org.apache.hadoop.yarn.server.federation.store.metrics.FederationStateStoreClientMetrics; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyRequest; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,7 +47,6 @@ import com.zaxxer.hikari.HikariDataSource; /** * Common utility methods used by the store implementations. - * */ public final class FederationStateStoreUtils { @@ -329,4 +339,61 @@ public final class FederationStateStoreUtils { return false; } + + /** + * Encode for Writable objects. + * This method will convert the writable object to a base64 string. + * + * @param key Writable Key. + * @return base64 string. + * @throws IOException raised on errors performing I/O. + */ + public static String encodeWritable(Writable key) throws IOException { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(bos); + key.write(dos); + dos.flush(); + return Base64.getUrlEncoder().encodeToString(bos.toByteArray()); + } + + /** + * Decode Base64 string to Writable object. + * + * @param w Writable Key. + * @param idStr base64 string. + * @throws IOException raised on errors performing I/O. + */ + public static void decodeWritable(Writable w, String idStr) throws IOException { + DataInputStream in = new DataInputStream( + new ByteArrayInputStream(Base64.getUrlDecoder().decode(idStr))); + w.readFields(in); + } + + /** + * Convert MasterKey to DelegationKey. + * + * Before using this function, + * please use FederationRouterRMTokenInputValidator to verify the request. + * By default, the request is not empty, and the internal object is not empty. + * + * @param request RouterMasterKeyRequest + * @return DelegationKey. + */ + public static DelegationKey convertMasterKeyToDelegationKey(RouterMasterKeyRequest request) { + RouterMasterKey masterKey = request.getRouterMasterKey(); + return convertMasterKeyToDelegationKey(masterKey); + } + + /** + * Convert MasterKey to DelegationKey. + * + * @param masterKey masterKey. + * @return DelegationKey. + */ + private static DelegationKey convertMasterKeyToDelegationKey(RouterMasterKey masterKey) { + ByteBuffer keyByteBuf = masterKey.getKeyBytes(); + byte[] keyBytes = new byte[keyByteBuf.remaining()]; + keyByteBuf.get(keyBytes); + return new DelegationKey(masterKey.getKeyId(), masterKey.getExpiryDate(), keyBytes); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java index 8c36fba1f1e..12625a60e94 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java @@ -809,6 +809,24 @@ public final class FederationStateStoreFacade { stateStore.storeNewToken(request); } + /** + * The Router Supports Store RMDelegationTokenIdentifier{@link RMDelegationTokenIdentifier}. + * + * @param identifier delegation tokens from the RM. + * @param renewDate renewDate. + * @param tokenInfo tokenInfo. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ + public void storeNewToken(RMDelegationTokenIdentifier identifier, + long renewDate, String tokenInfo) throws YarnException, IOException { + LOG.info("storing RMDelegation token with sequence number: {}.", + identifier.getSequenceNumber()); + RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate, tokenInfo); + RouterRMTokenRequest request = RouterRMTokenRequest.newInstance(storeToken); + stateStore.storeNewToken(request); + } + /** * The Router Supports Update RMDelegationTokenIdentifier{@link RMDelegationTokenIdentifier}. * @@ -826,6 +844,24 @@ public final class FederationStateStoreFacade { stateStore.updateStoredToken(request); } + /** + * The Router Supports Update RMDelegationTokenIdentifier{@link RMDelegationTokenIdentifier}. + * + * @param identifier delegation tokens from the RM + * @param renewDate renewDate + * @param tokenInfo tokenInfo. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ + public void updateStoredToken(RMDelegationTokenIdentifier identifier, + long renewDate, String tokenInfo) throws YarnException, IOException { + LOG.info("updating RMDelegation token with sequence number: {}.", + identifier.getSequenceNumber()); + RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate, tokenInfo); + RouterRMTokenRequest request = RouterRMTokenRequest.newInstance(storeToken); + stateStore.updateStoredToken(request); + } + /** * The Router Supports Remove RMDelegationTokenIdentifier{@link RMDelegationTokenIdentifier}. * diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto index 0544a26e4c5..26fc77e01f2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto @@ -234,6 +234,7 @@ message RouterMasterKeyResponseProto { message RouterStoreTokenProto { optional YARNDelegationTokenIdentifierProto token_identifier = 1; optional int64 renew_date = 2; + optional string token_info = 3; } message RouterRMTokenRequestProto { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java index 7fb1e327e85..c93115ccfd3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java @@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.federation.store.impl; import java.io.IOException; import java.nio.ByteBuffer; +import java.sql.SQLException; import java.util.Calendar; import java.util.List; import java.util.Set; @@ -98,10 +99,10 @@ public abstract class FederationStateStoreBaseTest { protected abstract FederationStateStore createStateStore(); protected abstract void checkRouterMasterKey(DelegationKey delegationKey, - RouterMasterKey routerMasterKey) throws YarnException, IOException; + RouterMasterKey routerMasterKey) throws YarnException, IOException, SQLException; protected abstract void checkRouterStoreToken(RMDelegationTokenIdentifier identifier, - RouterStoreToken token) throws YarnException, IOException; + RouterStoreToken token) throws YarnException, IOException, SQLException; private Configuration conf; @@ -937,16 +938,17 @@ public abstract class FederationStateStoreBaseTest { } @Test - public void testStoreNewToken() throws IOException, YarnException { + public void testStoreNewToken() throws IOException, YarnException, SQLException { // prepare parameters RMDelegationTokenIdentifier identifier = new RMDelegationTokenIdentifier( new Text("owner1"), new Text("renewer1"), new Text("realuser1")); int sequenceNumber = 1; identifier.setSequenceNumber(sequenceNumber); Long renewDate = Time.now(); + String tokenInfo = "tokenInfo"; // store new rm-token - RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate); + RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate, tokenInfo); RouterRMTokenRequest request = RouterRMTokenRequest.newInstance(storeToken); RouterRMTokenResponse routerRMTokenResponse = stateStore.storeNewToken(request); @@ -957,33 +959,33 @@ public abstract class FederationStateStoreBaseTest { Assert.assertNotNull(storeTokenResp); Assert.assertEquals(storeToken.getRenewDate(), storeTokenResp.getRenewDate()); Assert.assertEquals(storeToken.getTokenIdentifier(), storeTokenResp.getTokenIdentifier()); + Assert.assertEquals(storeToken.getTokenInfo(), storeTokenResp.getTokenInfo()); - checkRouterStoreToken(identifier, storeToken); checkRouterStoreToken(identifier, storeTokenResp); } @Test - public void testUpdateStoredToken() throws IOException, YarnException { + public void testUpdateStoredToken() throws IOException, YarnException, SQLException { // prepare saveToken parameters RMDelegationTokenIdentifier identifier = new RMDelegationTokenIdentifier( new Text("owner2"), new Text("renewer2"), new Text("realuser2")); int sequenceNumber = 2; + String tokenInfo = "tokenInfo"; identifier.setSequenceNumber(sequenceNumber); Long renewDate = Time.now(); // store new rm-token - RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate); + RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate, tokenInfo); RouterRMTokenRequest request = RouterRMTokenRequest.newInstance(storeToken); RouterRMTokenResponse routerRMTokenResponse = stateStore.storeNewToken(request); Assert.assertNotNull(routerRMTokenResponse); // prepare updateToken parameters Long renewDate2 = Time.now(); - int sequenceNumber2 = 3; - identifier.setSequenceNumber(sequenceNumber2); + String tokenInfo2 = "tokenInfo2"; // update rm-token - RouterStoreToken updateToken = RouterStoreToken.newInstance(identifier, renewDate2); + RouterStoreToken updateToken = RouterStoreToken.newInstance(identifier, renewDate2, tokenInfo2); RouterRMTokenRequest updateTokenRequest = RouterRMTokenRequest.newInstance(updateToken); RouterRMTokenResponse updateTokenResponse = stateStore.updateStoredToken(updateTokenRequest); @@ -992,6 +994,7 @@ public abstract class FederationStateStoreBaseTest { Assert.assertNotNull(updateTokenResp); Assert.assertEquals(updateToken.getRenewDate(), updateTokenResp.getRenewDate()); Assert.assertEquals(updateToken.getTokenIdentifier(), updateTokenResp.getTokenIdentifier()); + Assert.assertEquals(updateToken.getTokenInfo(), updateTokenResp.getTokenInfo()); checkRouterStoreToken(identifier, updateTokenResp); } @@ -1004,9 +1007,10 @@ public abstract class FederationStateStoreBaseTest { int sequenceNumber = 3; identifier.setSequenceNumber(sequenceNumber); Long renewDate = Time.now(); + String tokenInfo = "tokenInfo"; // store new rm-token - RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate); + RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate, tokenInfo); RouterRMTokenRequest request = RouterRMTokenRequest.newInstance(storeToken); RouterRMTokenResponse routerRMTokenResponse = stateStore.storeNewToken(request); Assert.assertNotNull(routerRMTokenResponse); @@ -1021,16 +1025,17 @@ public abstract class FederationStateStoreBaseTest { } @Test - public void testGetTokenByRouterStoreToken() throws IOException, YarnException { + public void testGetTokenByRouterStoreToken() throws IOException, YarnException, SQLException { // prepare saveToken parameters RMDelegationTokenIdentifier identifier = new RMDelegationTokenIdentifier( new Text("owner4"), new Text("renewer4"), new Text("realuser4")); int sequenceNumber = 4; identifier.setSequenceNumber(sequenceNumber); Long renewDate = Time.now(); + String tokenInfo = "tokenInfo"; // store new rm-token - RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate); + RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate, tokenInfo); RouterRMTokenRequest request = RouterRMTokenRequest.newInstance(storeToken); RouterRMTokenResponse routerRMTokenResponse = stateStore.storeNewToken(request); Assert.assertNotNull(routerRMTokenResponse); @@ -1041,7 +1046,7 @@ public abstract class FederationStateStoreBaseTest { RouterStoreToken getStoreTokenResp = getRouterRMTokenResp.getRouterStoreToken(); Assert.assertNotNull(getStoreTokenResp); Assert.assertEquals(getStoreTokenResp.getRenewDate(), storeToken.getRenewDate()); - Assert.assertEquals(getStoreTokenResp.getTokenIdentifier(), storeToken.getTokenIdentifier()); + Assert.assertEquals(storeToken.getTokenInfo(), getStoreTokenResp.getTokenInfo()); checkRouterStoreToken(identifier, getStoreTokenResp); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/HSQLDBFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/HSQLDBFederationStateStore.java index b3bb0764dfa..73b65feb48e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/HSQLDBFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/HSQLDBFederationStateStore.java @@ -75,6 +75,26 @@ public class HSQLDBFederationStateStore extends SQLFederationStateStore { + " homeSubCluster varchar(256) NOT NULL," + " CONSTRAINT pk_reservationId PRIMARY KEY (reservationId))"; + private static final String TABLE_MASTERKEYS = + " CREATE TABLE masterKeys (" + + " keyId bigint NOT NULL," + + " masterKey varchar(1024) NOT NULL," + + " CONSTRAINT pk_keyId PRIMARY KEY (keyId))"; + + private static final String TABLE_DELEGATIONTOKENS = + " CREATE TABLE delegationTokens (" + + " sequenceNum bigint NOT NULL," + + " tokenIdent varchar(1024) NOT NULL," + + " token varchar(1024) NOT NULL," + + " renewDate bigint NOT NULL," + + " CONSTRAINT pk_sequenceNum PRIMARY KEY (sequenceNum))"; + + private static final String TABLE_SEQUENCETABLE = + " CREATE TABLE sequenceTable (" + + " sequenceName varchar(255) NOT NULL," + + " nextVal bigint NOT NULL," + + " CONSTRAINT pk_sequenceName PRIMARY KEY (sequenceName))"; + private static final String SP_REGISTERSUBCLUSTER = "CREATE PROCEDURE sp_registerSubCluster(" + " IN subClusterId_IN varchar(256)," @@ -318,6 +338,99 @@ public class HSQLDBFederationStateStore extends SQLFederationStateStore { + " WHERE reservationId = reservationId_IN;" + " SET rowCount_OUT = 2; END"; + protected static final String SP_DROP_ADDMASTERKEY = "DROP PROCEDURE sp_addMasterKey"; + + protected static final String SP_ADDMASTERKEY = + "CREATE PROCEDURE sp_addMasterKey(" + + " IN keyId_IN int, IN masterKey_IN varchar(1024)," + + " OUT rowCount_OUT int)" + + " MODIFIES SQL DATA BEGIN ATOMIC " + + " INSERT INTO masterKeys(keyId, masterKey)" + + " (SELECT keyId_IN, masterKey_IN" + + " FROM masterKeys " + + " WHERE keyId = keyId_IN " + + " HAVING COUNT(*) = 0);" + + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT;" + + " END"; + + protected static final String SP_DROP_GETMASTERKEY = "DROP PROCEDURE sp_getMasterKey"; + + protected static final String SP_GETMASTERKEY = + "CREATE PROCEDURE sp_getMasterKey(" + + " IN keyId_IN int," + + " OUT masterKey_OUT varchar(1024))" + + " MODIFIES SQL DATA BEGIN ATOMIC " + + " SELECT masterKey INTO masterKey_OUT " + + " FROM masterKeys " + + " WHERE keyId = keyId_IN; " + + " END "; + + protected static final String SP_DROP_DELETEMASTERKEY = "DROP PROCEDURE sp_deleteMasterKey"; + + protected static final String SP_DELETEMASTERKEY = + "CREATE PROCEDURE sp_deleteMasterKey(" + + " IN keyId_IN int, OUT rowCount_OUT int)" + + " MODIFIES SQL DATA BEGIN ATOMIC" + + " DELETE FROM masterKeys WHERE keyId = keyId_IN;" + + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT; END"; + + protected static final String SP_DROP_ADD_DELEGATIONTOKEN = + "DROP PROCEDURE sp_addDelegationToken"; + + protected static final String SP_ADD_DELEGATIONTOKEN = + "CREATE PROCEDURE sp_addDelegationToken(" + + " IN sequenceNum_IN bigint, IN tokenIdent_IN varchar(1024)," + + " IN token_IN varchar(1024), IN renewDate_IN bigint, OUT rowCount_OUT int)" + + " MODIFIES SQL DATA BEGIN ATOMIC " + + " INSERT INTO delegationTokens(sequenceNum, tokenIdent, token, renewDate)" + + " (SELECT sequenceNum_IN, tokenIdent_IN, token_IN, renewDate_IN" + + " FROM delegationTokens" + + " WHERE sequenceNum = sequenceNum_IN" + + " HAVING COUNT(*) = 0);" + + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT;" + + " END"; + + protected static final String SP_DROP_GET_DELEGATIONTOKEN = + "DROP PROCEDURE sp_getDelegationToken"; + + protected static final String SP_GET_DELEGATIONTOKEN = + "CREATE PROCEDURE sp_getDelegationToken(" + + " IN sequenceNum_IN bigint, OUT tokenIdent_OUT varchar(1024), " + + " OUT token_OUT varchar(1024), OUT renewDate_OUT bigint)" + + " MODIFIES SQL DATA BEGIN ATOMIC " + + " SELECT tokenIdent, token, renewDate INTO " + + " tokenIdent_OUT, token_OUT, renewDate_OUT" + + " FROM delegationTokens" + + " WHERE sequenceNum = sequenceNum_IN; " + + " END "; + + protected static final String SP_DROP_UPDATE_DELEGATIONTOKEN = + "DROP PROCEDURE sp_updateDelegationToken"; + + protected static final String SP_UPDATE_DELEGATIONTOKEN = + "CREATE PROCEDURE sp_updateDelegationToken(" + + " IN sequenceNum_IN bigint, IN tokenIdent_IN varchar(1024)," + + " IN token_IN varchar(1024), IN renewDate_IN bigint, OUT rowCount_OUT int)" + + " MODIFIES SQL DATA BEGIN ATOMIC" + + " UPDATE delegationTokens" + + " SET tokenIdent = tokenIdent_IN," + + " token = token_IN, renewDate = renewDate_IN" + + " WHERE sequenceNum = sequenceNum_IN;" + + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT; " + + " END "; + + protected static final String SP_DROP_DELETE_DELEGATIONTOKEN = + "DROP PROCEDURE sp_deleteDelegationToken"; + + protected static final String SP_DELETE_DELEGATIONTOKEN = + "CREATE PROCEDURE sp_deleteDelegationToken(" + + " IN sequenceNum_IN bigint, OUT rowCount_OUT int)" + + " MODIFIES SQL DATA BEGIN ATOMIC" + + " DELETE FROM delegationTokens" + + " WHERE sequenceNum = sequenceNum_IN;" + + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT; " + + " END "; + private List tables = new ArrayList<>(); @Override @@ -333,6 +446,9 @@ public class HSQLDBFederationStateStore extends SQLFederationStateStore { conn.prepareStatement(TABLE_MEMBERSHIP).execute(); conn.prepareStatement(TABLE_POLICIES).execute(); conn.prepareStatement(TABLE_RESERVATIONSHOMESUBCLUSTER).execute(); + conn.prepareStatement(TABLE_MASTERKEYS).execute(); + conn.prepareStatement(TABLE_DELEGATIONTOKENS).execute(); + conn.prepareStatement(TABLE_SEQUENCETABLE).execute(); conn.prepareStatement(SP_REGISTERSUBCLUSTER).execute(); conn.prepareStatement(SP_DEREGISTERSUBCLUSTER).execute(); @@ -356,6 +472,15 @@ public class HSQLDBFederationStateStore extends SQLFederationStateStore { conn.prepareStatement(SP_DELETERESERVATIONHOMESUBCLUSTER).execute(); conn.prepareStatement(SP_UPDATERESERVATIONHOMESUBCLUSTER).execute(); + conn.prepareStatement(SP_ADDMASTERKEY).execute(); + conn.prepareStatement(SP_GETMASTERKEY).execute(); + conn.prepareStatement(SP_DELETEMASTERKEY).execute(); + + conn.prepareStatement(SP_ADD_DELEGATIONTOKEN).execute(); + conn.prepareStatement(SP_GET_DELEGATIONTOKEN).execute(); + conn.prepareStatement(SP_UPDATE_DELEGATIONTOKEN).execute(); + conn.prepareStatement(SP_DELETE_DELEGATIONTOKEN).execute(); + LOG.info("Database Init: Complete"); } catch (Exception e) { LOG.error("ERROR: failed to initialize HSQLDB {}.", e.getMessage()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestMemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestMemoryFederationStateStore.java index 0ea714ff06e..5548dab1b8c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestMemoryFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestMemoryFederationStateStore.java @@ -78,7 +78,7 @@ public class TestMemoryFederationStateStore extends FederationStateStoreBaseTest memoryStateStore.getRouterRMSecretManagerState(); assertNotNull(secretManagerState); - Map tokenStateMap = + Map tokenStateMap = secretManagerState.getTokenState(); assertNotNull(tokenStateMap); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java index befdf489763..91414ebc70d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java @@ -17,7 +17,6 @@ package org.apache.hadoop.yarn.server.federation.store.impl; -import org.apache.commons.lang3.NotImplementedException; import org.apache.hadoop.security.token.delegation.DelegationKey; import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.util.Time; @@ -37,8 +36,14 @@ import org.apache.hadoop.yarn.server.federation.store.records.UpdateReservationH import org.apache.hadoop.yarn.server.federation.store.records.DeleteReservationHomeSubClusterRequest; import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyRequest; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyResponse; +import org.apache.hadoop.yarn.server.federation.store.sql.DatabaseProduct; +import org.apache.hadoop.yarn.server.federation.store.sql.FederationSQLOutParameter; +import org.apache.hadoop.yarn.server.federation.store.sql.FederationQueryRunner; +import org.apache.hadoop.yarn.server.federation.store.sql.RouterMasterKeyHandler; +import org.apache.hadoop.yarn.server.federation.store.sql.RouterStoreTokenHandler; import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils; -import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,20 +53,30 @@ import java.sql.CallableStatement; import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; +import java.sql.DatabaseMetaData; import java.util.ArrayList; import java.util.List; +import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_GET_MASTERKEY; import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_ADD_RESERVATION_HOME_SUBCLUSTER; import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_GET_RESERVATION_HOME_SUBCLUSTER; import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_GET_RESERVATIONS_HOME_SUBCLUSTER; -import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_UPDATE_RESERVATION_HOME_SUBCLUSTER; import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_DELETE_RESERVATION_HOME_SUBCLUSTER; +import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_GET_DELEGATIONTOKEN; +import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_UPDATE_RESERVATION_HOME_SUBCLUSTER; +import static org.apache.hadoop.yarn.server.federation.store.sql.DatabaseProduct.DbType; import static org.apache.hadoop.yarn.server.federation.store.impl.HSQLDBFederationStateStore.SP_DROP_ADDRESERVATIONHOMESUBCLUSTER; import static org.apache.hadoop.yarn.server.federation.store.impl.HSQLDBFederationStateStore.SP_ADDRESERVATIONHOMESUBCLUSTER2; import static org.apache.hadoop.yarn.server.federation.store.impl.HSQLDBFederationStateStore.SP_DROP_UPDATERESERVATIONHOMESUBCLUSTER; import static org.apache.hadoop.yarn.server.federation.store.impl.HSQLDBFederationStateStore.SP_UPDATERESERVATIONHOMESUBCLUSTER2; import static org.apache.hadoop.yarn.server.federation.store.impl.HSQLDBFederationStateStore.SP_DROP_DELETERESERVATIONHOMESUBCLUSTER; import static org.apache.hadoop.yarn.server.federation.store.impl.HSQLDBFederationStateStore.SP_DELETERESERVATIONHOMESUBCLUSTER2; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static java.sql.Types.VARCHAR; +import static java.sql.Types.BIGINT; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; /** * Unit tests for SQLFederationStateStore. @@ -74,6 +89,7 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { private static final String DATABASE_URL = "jdbc:hsqldb:mem:state"; private static final String DATABASE_USERNAME = "SA"; private static final String DATABASE_PASSWORD = ""; + private SQLFederationStateStore sqlFederationStateStore = null; @Override protected FederationStateStore createStateStore() { @@ -90,7 +106,8 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { DATABASE_URL + System.currentTimeMillis()); conf.setInt(YarnConfiguration.FEDERATION_STATESTORE_MAX_APPLICATIONS, 10); super.setConf(conf); - return new HSQLDBFederationStateStore(); + sqlFederationStateStore = new HSQLDBFederationStateStore(); + return sqlFederationStateStore; } @Test @@ -103,13 +120,13 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { stateStore.registerSubCluster( SubClusterRegisterRequest.newInstance(subClusterInfo)); - Assert.assertEquals(subClusterInfo, querySubClusterInfo(subClusterId)); + assertEquals(subClusterInfo, querySubClusterInfo(subClusterId)); addApplicationHomeSC(appId, subClusterId); - Assert.assertEquals(subClusterId, queryApplicationHomeSC(appId)); + assertEquals(subClusterId, queryApplicationHomeSC(appId)); // Verify if connection is created only once at statestore init - Assert.assertEquals(1, + assertEquals(1, FederationStateStoreClientMetrics.getNumConnections()); } @@ -125,8 +142,7 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { } } - private ReservationHomeSC addReservationHomeSubCluster( - SQLFederationStateStore sqlFederationStateStore, String procedure, + private ReservationHomeSC addReservationHomeSubCluster(String procedure, String reservationId, String subHomeClusterId) throws SQLException, YarnException { // procedure call parameter preparation CallableStatement cstmt = sqlFederationStateStore.getCallableStatement(procedure); @@ -148,8 +164,7 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { return new ReservationHomeSC(reservationId, dbStoredHomeSubCluster, dbRowCount); } - private ReservationHomeSC getReservationHomeSubCluster( - SQLFederationStateStore sqlFederationStateStore, String procedure, + private ReservationHomeSC getReservationHomeSubCluster(String procedure, String reservationId) throws SQLException, YarnException { // procedure call parameter preparation @@ -170,8 +185,7 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { return new ReservationHomeSC(reservationId, dBSubClusterHomeId, 0); } - private List getReservationsHomeSubCluster( - SQLFederationStateStore sqlFederationStateStore, String procedure) + private List getReservationsHomeSubCluster(String procedure) throws SQLException, IOException, YarnException { List results = new ArrayList<>(); @@ -197,8 +211,7 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { return results; } - private ReservationHomeSC updateReservationHomeSubCluster( - SQLFederationStateStore sqlFederationStateStore, String procedure, + private ReservationHomeSC updateReservationHomeSubCluster(String procedure, String reservationId, String subHomeClusterId) throws SQLException, IOException { @@ -222,8 +235,7 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { return new ReservationHomeSC(reservationId, subHomeClusterId, rowCount); } - private ReservationHomeSC deleteReservationHomeSubCluster( - SQLFederationStateStore sqlFederationStateStore, String procedure, + private ReservationHomeSC deleteReservationHomeSubCluster(String procedure, String reservationId) throws SQLException { // procedure call parameter preparation CallableStatement cstmt = sqlFederationStateStore.getCallableStatement(procedure); @@ -254,21 +266,17 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { */ @Test public void testCheckAddReservationHomeSubCluster() throws Exception { - FederationStateStore stateStore = getStateStore(); - Assert.assertTrue(stateStore instanceof SQLFederationStateStore); - - SQLFederationStateStore sqlFederationStateStore = (SQLFederationStateStore) stateStore; // procedure call parameter preparation ReservationId reservationId = ReservationId.newInstance(Time.now(), 1); String subHomeClusterId = "SC-1"; - ReservationHomeSC resultHC = addReservationHomeSubCluster(sqlFederationStateStore, + ReservationHomeSC resultHC = addReservationHomeSubCluster( CALL_SP_ADD_RESERVATION_HOME_SUBCLUSTER, reservationId.toString(), subHomeClusterId); // validation results - Assert.assertNotNull(resultHC); - Assert.assertEquals(subHomeClusterId, resultHC.subHomeClusterId); - Assert.assertEquals(1, resultHC.dbUpdateCount); + assertNotNull(resultHC); + assertEquals(subHomeClusterId, resultHC.subHomeClusterId); + assertEquals(1, resultHC.dbUpdateCount); } /** @@ -282,24 +290,20 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { */ @Test public void testCheckGetReservationHomeSubCluster() throws Exception { - FederationStateStore stateStore = getStateStore(); - Assert.assertTrue(stateStore instanceof SQLFederationStateStore); - - SQLFederationStateStore sqlFederationStateStore = (SQLFederationStateStore) stateStore; // procedure call parameter preparation ReservationId reservationId = ReservationId.newInstance(Time.now(), 1); String subHomeClusterId = "SC-1"; - addReservationHomeSubCluster(sqlFederationStateStore, + addReservationHomeSubCluster( CALL_SP_ADD_RESERVATION_HOME_SUBCLUSTER, reservationId.toString(), subHomeClusterId); // Call getReservationHomeSubCluster to get the result - ReservationHomeSC resultHC = getReservationHomeSubCluster(sqlFederationStateStore, + ReservationHomeSC resultHC = getReservationHomeSubCluster( CALL_SP_GET_RESERVATION_HOME_SUBCLUSTER, reservationId.toString()); - Assert.assertNotNull(resultHC); - Assert.assertEquals(subHomeClusterId, resultHC.subHomeClusterId); - Assert.assertEquals(reservationId.toString(), resultHC.reservationId); + assertNotNull(resultHC); + assertEquals(subHomeClusterId, resultHC.subHomeClusterId); + assertEquals(reservationId.toString(), resultHC.reservationId); } /** @@ -315,38 +319,34 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { */ @Test public void testCheckGetReservationsHomeSubCluster() throws Exception { - FederationStateStore stateStore = getStateStore(); - Assert.assertTrue(stateStore instanceof SQLFederationStateStore); - - SQLFederationStateStore sqlFederationStateStore = (SQLFederationStateStore) stateStore; // add 1st record ReservationId reservationId1 = ReservationId.newInstance(Time.now(), 1); String subHomeClusterId1 = "SC-1"; - addReservationHomeSubCluster(sqlFederationStateStore, + addReservationHomeSubCluster( CALL_SP_ADD_RESERVATION_HOME_SUBCLUSTER, reservationId1.toString(), subHomeClusterId1); // add 2nd record ReservationId reservationId2 = ReservationId.newInstance(Time.now(), 2); String subHomeClusterId2 = "SC-2"; - addReservationHomeSubCluster(sqlFederationStateStore, + addReservationHomeSubCluster( CALL_SP_ADD_RESERVATION_HOME_SUBCLUSTER, reservationId2.toString(), subHomeClusterId2); List reservationHomeSubClusters = getReservationsHomeSubCluster( - sqlFederationStateStore, CALL_SP_GET_RESERVATIONS_HOME_SUBCLUSTER); + CALL_SP_GET_RESERVATIONS_HOME_SUBCLUSTER); - Assert.assertNotNull(reservationHomeSubClusters); - Assert.assertEquals(2, reservationHomeSubClusters.size()); + assertNotNull(reservationHomeSubClusters); + assertEquals(2, reservationHomeSubClusters.size()); ReservationHomeSC resultHC1 = reservationHomeSubClusters.get(0); - Assert.assertNotNull(resultHC1); - Assert.assertEquals(reservationId1.toString(), resultHC1.reservationId); - Assert.assertEquals(subHomeClusterId1, resultHC1.subHomeClusterId); + assertNotNull(resultHC1); + assertEquals(reservationId1.toString(), resultHC1.reservationId); + assertEquals(subHomeClusterId1, resultHC1.subHomeClusterId); ReservationHomeSC resultHC2 = reservationHomeSubClusters.get(1); - Assert.assertNotNull(resultHC2); - Assert.assertEquals(reservationId2.toString(), resultHC2.reservationId); - Assert.assertEquals(subHomeClusterId2, resultHC2.subHomeClusterId); + assertNotNull(resultHC2); + assertEquals(reservationId2.toString(), resultHC2.reservationId); + assertEquals(subHomeClusterId2, resultHC2.subHomeClusterId); } /** @@ -364,37 +364,33 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { */ @Test public void testCheckUpdateReservationHomeSubCluster() throws Exception { - FederationStateStore stateStore = getStateStore(); - Assert.assertTrue(stateStore instanceof SQLFederationStateStore); - - SQLFederationStateStore sqlFederationStateStore = (SQLFederationStateStore) stateStore; // procedure call parameter preparation ReservationId reservationId = ReservationId.newInstance(Time.now(), 1); String subHomeClusterId = "SC-1"; - addReservationHomeSubCluster(sqlFederationStateStore, + addReservationHomeSubCluster( CALL_SP_ADD_RESERVATION_HOME_SUBCLUSTER, reservationId.toString(), subHomeClusterId); // verify that the subHomeClusterId corresponding to reservationId is SC-1 - ReservationHomeSC resultHC = getReservationHomeSubCluster(sqlFederationStateStore, + ReservationHomeSC resultHC = getReservationHomeSubCluster( CALL_SP_GET_RESERVATION_HOME_SUBCLUSTER, reservationId.toString()); - Assert.assertNotNull(resultHC); - Assert.assertEquals(subHomeClusterId, resultHC.subHomeClusterId); + assertNotNull(resultHC); + assertEquals(subHomeClusterId, resultHC.subHomeClusterId); // prepare to update parameters String newSubHomeClusterId = "SC-2"; ReservationHomeSC reservationHomeSubCluster = - updateReservationHomeSubCluster(sqlFederationStateStore, + updateReservationHomeSubCluster( CALL_SP_UPDATE_RESERVATION_HOME_SUBCLUSTER, reservationId.toString(), newSubHomeClusterId); - Assert.assertNotNull(reservationHomeSubCluster); - Assert.assertEquals(1, reservationHomeSubCluster.dbUpdateCount); + assertNotNull(reservationHomeSubCluster); + assertEquals(1, reservationHomeSubCluster.dbUpdateCount); // verify that the subHomeClusterId corresponding to reservationId is SC-2 - ReservationHomeSC resultHC2 = getReservationHomeSubCluster(sqlFederationStateStore, + ReservationHomeSC resultHC2 = getReservationHomeSubCluster( CALL_SP_GET_RESERVATION_HOME_SUBCLUSTER, reservationId.toString()); - Assert.assertNotNull(resultHC2); - Assert.assertEquals(newSubHomeClusterId, resultHC2.subHomeClusterId); + assertNotNull(resultHC2); + assertEquals(newSubHomeClusterId, resultHC2.subHomeClusterId); } /** @@ -411,29 +407,25 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { */ @Test public void testCheckDeleteReservationHomeSubCluster() throws Exception { - FederationStateStore stateStore = getStateStore(); - Assert.assertTrue(stateStore instanceof SQLFederationStateStore); - - SQLFederationStateStore sqlFederationStateStore = (SQLFederationStateStore) stateStore; // procedure call parameter preparation ReservationId reservationId = ReservationId.newInstance(Time.now(), 1); String subHomeClusterId = "SC-1"; - addReservationHomeSubCluster(sqlFederationStateStore, + addReservationHomeSubCluster( CALL_SP_ADD_RESERVATION_HOME_SUBCLUSTER, reservationId.toString(), subHomeClusterId); // call the delete method of the reservation - ReservationHomeSC resultHC = deleteReservationHomeSubCluster(sqlFederationStateStore, + ReservationHomeSC resultHC = deleteReservationHomeSubCluster( CALL_SP_DELETE_RESERVATION_HOME_SUBCLUSTER, reservationId.toString()); - Assert.assertNotNull(resultHC); - Assert.assertEquals(1, resultHC.dbUpdateCount); + assertNotNull(resultHC); + assertEquals(1, resultHC.dbUpdateCount); // call getReservationHomeSubCluster to get the result - ReservationHomeSC resultHC1 = getReservationHomeSubCluster(sqlFederationStateStore, + ReservationHomeSC resultHC1 = getReservationHomeSubCluster( CALL_SP_GET_RESERVATION_HOME_SUBCLUSTER, reservationId.toString()); - Assert.assertNotNull(resultHC1); - Assert.assertEquals(null, resultHC1.subHomeClusterId); + assertNotNull(resultHC1); + assertEquals(null, resultHC1.subHomeClusterId); } /** @@ -446,10 +438,6 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { */ @Test public void testAddReservationHomeSubClusterAbnormalSituation() throws Exception { - FederationStateStore stateStore = getStateStore(); - Assert.assertTrue(stateStore instanceof SQLFederationStateStore); - - SQLFederationStateStore sqlFederationStateStore = (SQLFederationStateStore) stateStore; Connection conn = sqlFederationStateStore.getConn(); conn.prepareStatement(SP_DROP_ADDRESERVATIONHOMESUBCLUSTER).execute(); @@ -470,7 +458,7 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { "please check the records of the database.", subClusterId, reservationId); LambdaTestUtils.intercept(YarnException.class, errorMsg, - () -> stateStore.addReservationHomeSubCluster(request)); + () -> sqlFederationStateStore.addReservationHomeSubCluster(request)); } /** @@ -483,10 +471,6 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { */ @Test public void testUpdateReservationHomeSubClusterAbnormalSituation() throws Exception { - FederationStateStore stateStore = getStateStore(); - Assert.assertTrue(stateStore instanceof SQLFederationStateStore); - - SQLFederationStateStore sqlFederationStateStore = (SQLFederationStateStore) stateStore; Connection conn = sqlFederationStateStore.getConn(); conn.prepareStatement(SP_DROP_UPDATERESERVATIONHOMESUBCLUSTER).execute(); @@ -500,7 +484,7 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { ReservationHomeSubCluster.newInstance(reservationId, subClusterId1); AddReservationHomeSubClusterRequest addRequest = AddReservationHomeSubClusterRequest.newInstance(reservationHomeSubCluster); - stateStore.addReservationHomeSubCluster(addRequest); + sqlFederationStateStore.addReservationHomeSubCluster(addRequest); SubClusterId subClusterId2 = SubClusterId.newInstance("SC2"); ReservationHomeSubCluster reservationHomeSubCluster2 = @@ -516,7 +500,7 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { subClusterId2, reservationId); LambdaTestUtils.intercept(YarnException.class, errorMsg, - () -> stateStore.updateReservationHomeSubCluster(updateRequest)); + () -> sqlFederationStateStore.updateReservationHomeSubCluster(updateRequest)); } /** @@ -529,10 +513,6 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { */ @Test public void testDeleteReservationHomeSubClusterAbnormalSituation() throws Exception { - FederationStateStore stateStore = getStateStore(); - Assert.assertTrue(stateStore instanceof SQLFederationStateStore); - - SQLFederationStateStore sqlFederationStateStore = (SQLFederationStateStore) stateStore; Connection conn = sqlFederationStateStore.getConn(); conn.prepareStatement(SP_DROP_DELETERESERVATIONHOMESUBCLUSTER).execute(); @@ -546,7 +526,7 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { ReservationHomeSubCluster.newInstance(reservationId, subClusterId1); AddReservationHomeSubClusterRequest addRequest = AddReservationHomeSubClusterRequest.newInstance(reservationHomeSubCluster); - stateStore.addReservationHomeSubCluster(addRequest); + sqlFederationStateStore.addReservationHomeSubCluster(addRequest); DeleteReservationHomeSubClusterRequest delRequest = DeleteReservationHomeSubClusterRequest.newInstance(reservationId); @@ -559,55 +539,92 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { reservationId); LambdaTestUtils.intercept(YarnException.class, errorMsg, - () -> stateStore.deleteReservationHomeSubCluster(delRequest)); - } - - @Test(expected = NotImplementedException.class) - public void testStoreNewMasterKey() throws Exception { - super.testStoreNewMasterKey(); - } - - @Test(expected = NotImplementedException.class) - public void testGetMasterKeyByDelegationKey() throws YarnException, IOException { - super.testGetMasterKeyByDelegationKey(); - } - - @Test(expected = NotImplementedException.class) - public void testRemoveStoredMasterKey() throws YarnException, IOException { - super.testRemoveStoredMasterKey(); - } - - @Test(expected = NotImplementedException.class) - public void testStoreNewToken() throws IOException, YarnException { - super.testStoreNewToken(); - } - - @Test(expected = NotImplementedException.class) - public void testUpdateStoredToken() throws IOException, YarnException { - super.testUpdateStoredToken(); - } - - @Test(expected = NotImplementedException.class) - public void testRemoveStoredToken() throws IOException, YarnException { - super.testRemoveStoredToken(); - } - - @Test(expected = NotImplementedException.class) - public void testGetTokenByRouterStoreToken() throws IOException, YarnException { - super.testGetTokenByRouterStoreToken(); + () -> sqlFederationStateStore.deleteReservationHomeSubCluster(delRequest)); } @Override protected void checkRouterMasterKey(DelegationKey delegationKey, - RouterMasterKey routerMasterKey) throws YarnException, IOException { - // TODO: This part of the code will be completed in YARN-11349 and - // will be used to verify whether the RouterMasterKey stored in the DB is as expected. + RouterMasterKey routerMasterKey) throws YarnException, IOException, SQLException { + // Check for MasterKey stored in DB. + RouterMasterKeyRequest routerMasterKeyRequest = + RouterMasterKeyRequest.newInstance(routerMasterKey); + + // Query Data from DB. + Connection conn = sqlFederationStateStore.getConn(); + int paramKeyId = delegationKey.getKeyId(); + FederationQueryRunner runner = new FederationQueryRunner(); + FederationSQLOutParameter masterKeyOUT = + new FederationSQLOutParameter<>("masterKey_OUT", VARCHAR, String.class); + RouterMasterKey sqlRouterMasterKey = runner.execute( + conn, CALL_SP_GET_MASTERKEY, new RouterMasterKeyHandler(), paramKeyId, masterKeyOUT); + + // Check Data. + RouterMasterKeyResponse response = getStateStore(). + getMasterKeyByDelegationKey(routerMasterKeyRequest); + assertNotNull(response); + RouterMasterKey respRouterMasterKey = response.getRouterMasterKey(); + assertEquals(routerMasterKey, respRouterMasterKey); + assertEquals(routerMasterKey, sqlRouterMasterKey); + assertEquals(sqlRouterMasterKey, respRouterMasterKey); } @Override protected void checkRouterStoreToken(RMDelegationTokenIdentifier identifier, - RouterStoreToken token) throws YarnException, IOException { - // TODO: This part of the code will be completed in YARN-11349 and - // will be used to verify whether the RouterStoreToken stored in the DB is as expected. + RouterStoreToken token) throws YarnException, IOException, SQLException { + // Get SequenceNum. + int sequenceNum = identifier.getSequenceNumber(); + + // Query Data from DB. + Connection conn = sqlFederationStateStore.getConn(); + FederationQueryRunner runner = new FederationQueryRunner(); + FederationSQLOutParameter tokenIdentOUT = + new FederationSQLOutParameter<>("tokenIdent_OUT", VARCHAR, String.class); + FederationSQLOutParameter tokenOUT = + new FederationSQLOutParameter<>("token_OUT", VARCHAR, String.class); + FederationSQLOutParameter renewDateOUT = + new FederationSQLOutParameter<>("renewDate_OUT", BIGINT, Long.class); + RouterStoreToken sqlRouterStoreToken = runner.execute(conn, CALL_SP_GET_DELEGATIONTOKEN, + new RouterStoreTokenHandler(), sequenceNum, tokenIdentOUT, tokenOUT, renewDateOUT); + + assertEquals(token, sqlRouterStoreToken); + } + + @Test + public void testCheckHSQLDB() throws SQLException { + Connection conn = sqlFederationStateStore.getConn(); + DbType dbType = DatabaseProduct.getDbType(conn); + assertEquals(DbType.HSQLDB, dbType); + } + + @Test + public void testGetDbTypeNullConn() throws SQLException { + DbType dbType = DatabaseProduct.getDbType(null); + assertEquals(DbType.UNDEFINED, dbType); + } + + @Test + public void testGetDBTypeEmptyConn() throws SQLException { + Connection connection = mock(Connection.class); + DatabaseMetaData metaData = mock(DatabaseMetaData.class); + when(metaData.getDatabaseProductName()).thenReturn(""); + when(connection.getMetaData()).thenReturn(metaData); + DbType dbType = DatabaseProduct.getDbType(connection); + assertEquals(DbType.UNDEFINED, dbType); + } + + @Test + public void testCheckForHSQLDBUpdateSQL() throws SQLException { + String sql = "select sequenceName, nextVal from sequenceTable"; + String hsqlDBSQL = DatabaseProduct.addForUpdateClause(DbType.HSQLDB, sql); + String expectUpdateSQL = "select sequenceName, nextVal from sequenceTable for update"; + assertEquals(expectUpdateSQL, hsqlDBSQL); + } + + @Test + public void testCheckForSqlServerDBUpdateSQL() throws SQLException { + String sql = "select sequenceName, nextVal from sequenceTable"; + String sqlServerDBSQL = DatabaseProduct.addForUpdateClause(DbType.SQLSERVER, sql); + String expectUpdateSQL = "select sequenceName, nextVal from sequenceTable with (updlock)"; + assertEquals(expectUpdateSQL, sqlServerDBSQL); } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/RouterDelegationTokenSupport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/RouterDelegationTokenSupport.java new file mode 100644 index 00000000000..d530f751cb2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/RouterDelegationTokenSupport.java @@ -0,0 +1,65 @@ +/* + * 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.security.token.delegation; + +import org.apache.hadoop.io.WritableUtils; +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.DelegationTokenInformation; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.Base64; + +/** + * Workaround for serialization of {@link DelegationTokenInformation} through package access. + * Future version of Hadoop should add this to DelegationTokenInformation itself. + */ +public final class RouterDelegationTokenSupport { + + private RouterDelegationTokenSupport() { + } + + public static String encodeDelegationTokenInformation(DelegationTokenInformation token) { + try { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(bos); + WritableUtils.writeVInt(out, token.password.length); + out.write(token.password); + out.writeLong(token.renewDate); + out.flush(); + byte[] tokenInfoBytes = bos.toByteArray(); + return Base64.getUrlEncoder().encodeToString(tokenInfoBytes); + } catch (IOException ex) { + throw new RuntimeException("Failed to encode token.", ex); + } + } + + public static DelegationTokenInformation decodeDelegationTokenInformation(byte[] tokenBytes) + throws IOException { + DataInputStream in = new DataInputStream(new ByteArrayInputStream(tokenBytes)); + DelegationTokenInformation token = new DelegationTokenInformation(0, null); + int len = WritableUtils.readVInt(in); + token.password = new byte[len]; + in.readFully(token.password); + token.renewDate = in.readLong(); + return token; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/package-info.java new file mode 100644 index 00000000000..3a1cb3e69ae --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Router security token delegation. **/ +package org.apache.hadoop.security.token.delegation; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/security/RouterDelegationTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/security/RouterDelegationTokenSecretManager.java index 918bf16e4f4..57d2aaa4bf1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/security/RouterDelegationTokenSecretManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/security/RouterDelegationTokenSecretManager.java @@ -21,13 +21,16 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager; import org.apache.hadoop.security.token.delegation.DelegationKey; +import org.apache.hadoop.security.token.delegation.RouterDelegationTokenSupport; import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier; import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyResponse; import org.apache.hadoop.yarn.server.federation.store.records.RouterRMTokenResponse; +import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,6 +41,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.Base64; /** * A Router specific delegation token secret manager. @@ -137,6 +141,29 @@ public class RouterDelegationTokenSecretManager } } + /** + * The Router Supports Store new Token. + * + * @param identifier RMDelegationToken. + * @param tokenInfo DelegationTokenInformation. + */ + public void storeNewToken(RMDelegationTokenIdentifier identifier, + DelegationTokenInformation tokenInfo) { + try { + String token = + RouterDelegationTokenSupport.encodeDelegationTokenInformation(tokenInfo); + long renewDate = tokenInfo.getRenewDate(); + + federationFacade.storeNewToken(identifier, renewDate, token); + } catch (Exception e) { + if (!shouldIgnoreException(e)) { + LOG.error("Error in storing RMDelegationToken with sequence number: {}.", + identifier.getSequenceNumber()); + ExitUtil.terminate(1, e); + } + } + } + /** * The Router Supports Update Token. * @@ -157,6 +184,27 @@ public class RouterDelegationTokenSecretManager } } + /** + * The Router Supports Update Token. + * + * @param identifier RMDelegationToken. + * @param tokenInfo DelegationTokenInformation. + */ + public void updateStoredToken(RMDelegationTokenIdentifier identifier, + DelegationTokenInformation tokenInfo) { + try { + long renewDate = tokenInfo.getRenewDate(); + String token = RouterDelegationTokenSupport.encodeDelegationTokenInformation(tokenInfo); + federationFacade.updateStoredToken(identifier, renewDate, token); + } catch (Exception e) { + if (!shouldIgnoreException(e)) { + LOG.error("Error in updating persisted RMDelegationToken with sequence number: {}.", + identifier.getSequenceNumber()); + ExitUtil.terminate(1, e); + } + } + } + /** * The Router Supports Remove Token. * @@ -267,6 +315,42 @@ public class RouterDelegationTokenSecretManager return federationFacade.incrementDelegationTokenSeqNum(); } + @Override + protected void storeToken(RMDelegationTokenIdentifier rmDelegationTokenIdentifier, + DelegationTokenInformation tokenInfo) throws IOException { + this.currentTokens.put(rmDelegationTokenIdentifier, tokenInfo); + this.addTokenForOwnerStats(rmDelegationTokenIdentifier); + storeNewToken(rmDelegationTokenIdentifier, tokenInfo); + } + + @Override + protected void updateToken(RMDelegationTokenIdentifier rmDelegationTokenIdentifier, + DelegationTokenInformation tokenInfo) throws IOException { + this.currentTokens.put(rmDelegationTokenIdentifier, tokenInfo); + updateStoredToken(rmDelegationTokenIdentifier, tokenInfo); + } + + @Override + protected DelegationTokenInformation getTokenInfo( + RMDelegationTokenIdentifier ident) { + // First check if I have this.. + DelegationTokenInformation tokenInfo = currentTokens.get(ident); + if (tokenInfo == null) { + try { + RouterRMTokenResponse response = federationFacade.getTokenByRouterStoreToken(ident); + RouterStoreToken routerStoreToken = response.getRouterStoreToken(); + String tokenStr = routerStoreToken.getTokenInfo(); + byte[] tokenBytes = Base64.getUrlDecoder().decode(tokenStr); + tokenInfo = RouterDelegationTokenSupport.decodeDelegationTokenInformation(tokenBytes); + } catch (Exception e) { + LOG.error("Error retrieving tokenInfo [" + ident.getSequenceNumber() + + "] from StateStore.", e); + throw new YarnRuntimeException(e); + } + } + return tokenInfo; + } + @Override protected synchronized int getDelegationTokenSeqNum() { return federationFacade.getDelegationTokenSeqNum(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java index 2488fc73b07..6f7248a0866 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java @@ -138,6 +138,7 @@ import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier; import org.apache.hadoop.yarn.server.federation.policies.manager.UniformBroadcastPolicyManager; import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore; import org.apache.hadoop.yarn.server.federation.store.records.RouterRMDTSecretManagerState; +import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo; import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade; @@ -1617,14 +1618,17 @@ public class TestFederationClientInterceptor extends BaseRouterClientRMTest { RouterRMDTSecretManagerState managerState = stateStore.getRouterRMSecretManagerState(); Assert.assertNotNull(managerState); - Map delegationTokenState = managerState.getTokenState(); + Map delegationTokenState = + managerState.getTokenState(); Assert.assertNotNull(delegationTokenState); Assert.assertTrue(delegationTokenState.containsKey(rMDelegationTokenIdentifier)); long tokenRenewInterval = this.getConf().getLong( YarnConfiguration.RM_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, YarnConfiguration.RM_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT); - long renewDate = delegationTokenState.get(rMDelegationTokenIdentifier); + RouterStoreToken resultRouterStoreToken = delegationTokenState.get(rMDelegationTokenIdentifier); + Assert.assertNotNull(resultRouterStoreToken); + long renewDate = resultRouterStoreToken.getRenewDate(); Assert.assertEquals(issueDate + tokenRenewInterval, renewDate); } @@ -1667,10 +1671,13 @@ public class TestFederationClientInterceptor extends BaseRouterClientRMTest { // Step3. Compare whether the expirationTime returned to // the client is consistent with the renewDate in the stateStore RouterRMDTSecretManagerState managerState = stateStore.getRouterRMSecretManagerState(); - Map delegationTokenState = managerState.getTokenState(); + Map delegationTokenState = + managerState.getTokenState(); Assert.assertNotNull(delegationTokenState); Assert.assertTrue(delegationTokenState.containsKey(rMDelegationTokenIdentifier)); - long renewDate = delegationTokenState.get(rMDelegationTokenIdentifier); + RouterStoreToken resultRouterStoreToken = delegationTokenState.get(rMDelegationTokenIdentifier); + Assert.assertNotNull(resultRouterStoreToken); + long renewDate = resultRouterStoreToken.getRenewDate(); Assert.assertEquals(expDate, renewDate); } @@ -1700,7 +1707,8 @@ public class TestFederationClientInterceptor extends BaseRouterClientRMTest { // Step3. Query the data in the StateStore and confirm that the Delegation has been deleted. // At this point, the size of delegationTokenState should be 0. RouterRMDTSecretManagerState managerState = stateStore.getRouterRMSecretManagerState(); - Map delegationTokenState = managerState.getTokenState(); + Map delegationTokenState = + managerState.getTokenState(); Assert.assertNotNull(delegationTokenState); Assert.assertEquals(0, delegationTokenState.size()); } From a5f48eacca4b80c0cc31f7de96652ff120179c01 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Thu, 16 Feb 2023 06:40:34 +0800 Subject: [PATCH 47/50] YARN-11425. [Federation] Router Supports SubClusterCleaner. (#5326) --- .../hadoop/yarn/conf/YarnConfiguration.java | 22 +++ .../src/main/resources/yarn-default.xml | 40 +++++ .../impl/MemoryFederationStateStore.java | 11 ++ .../utils/FederationStateStoreFacade.java | 24 +++ .../hadoop/yarn/server/router/Router.java | 37 +++- .../router/cleaner/SubClusterCleaner.java | 92 ++++++++++ .../server/router/cleaner/package-info.java | 20 +++ .../router/cleaner/TestSubClusterCleaner.java | 158 ++++++++++++++++++ 8 files changed, 398 insertions(+), 6 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/cleaner/SubClusterCleaner.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/cleaner/package-info.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/cleaner/TestSubClusterCleaner.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index 316a6421889..699059f068f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -4299,6 +4299,28 @@ public class YarnConfiguration extends Configuration { ROUTER_PREFIX + "interceptor.allow-partial-result.enable"; public static final boolean DEFAULT_ROUTER_INTERCEPTOR_ALLOW_PARTIAL_RESULT_ENABLED = false; + /** Router SubCluster Cleaner Thread Clean Interval Time. **/ + public static final String ROUTER_SUBCLUSTER_CLEANER_INTERVAL_TIME = + ROUTER_PREFIX + "subcluster.cleaner.interval.time"; + public static final long DEFAULT_ROUTER_SUBCLUSTER_CLEANER_INTERVAL_TIME = + TimeUnit.SECONDS.toMillis(60); + + /** Router SubCluster Timeout Allowed by Router. **/ + public static final String ROUTER_SUBCLUSTER_EXPIRATION_TIME = + ROUTER_PREFIX + "subcluster.heartbeat.expiration.time"; + public static final long DEFAULT_ROUTER_SUBCLUSTER_EXPIRATION_TIME = + TimeUnit.MINUTES.toMillis(30); + + /** Router Thread Pool Schedule Thread Number. **/ + public static final String ROUTER_SCHEDULED_EXECUTOR_THREADS = + ROUTER_PREFIX + "scheduled.executor.threads"; + public static final int DEFAULT_ROUTER_SCHEDULED_EXECUTOR_THREADS = 1; + + /** Enable DeregisterSubCluster, enabled by default. **/ + public static final String ROUTER_DEREGISTER_SUBCLUSTER_ENABLED = + ROUTER_PREFIX + "deregister.subcluster.enabled"; + public static final boolean DEFAULT_ROUTER_DEREGISTER_SUBCLUSTER_ENABLED = true; + //////////////////////////////// // CSI Volume configs //////////////////////////////// diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml index 80672fb1cc8..dc58f2f8285 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml @@ -5117,4 +5117,44 @@ + + + The number of threads to use for the Router scheduled executor service. + + yarn.router.subcluster.cleaner.interval.time + 1 + + + + + The interval at which the subClusterCleaner runs. Default is 60s. + + yarn.router.subcluster.cleaner.interval.time + 60s + + + + + SubCluster heartbeat timeout. Default is 30mins. + + yarn.router.subcluster.heartbeat.expiration.time + 30m + + + + + Whether to enable deregisterSubCluster. Default is true. + + yarn.router.deregister.subcluster.enabled + true + + + + + Number of Router Scheduler Threads. + + yarn.router.scheduled.executor.threads + 1 + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java index d44c30eef28..b91de3ae808 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java @@ -34,6 +34,7 @@ import java.util.Comparator; import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.token.delegation.DelegationKey; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ReservationId; @@ -606,4 +607,14 @@ public class MemoryFederationStateStore implements FederationStateStore { public void setMembership(Map membership) { this.membership = membership; } + + @VisibleForTesting + public void setExpiredHeartbeat(SubClusterId subClusterId, long heartBearTime) + throws YarnRuntimeException { + if(!membership.containsKey(subClusterId)){ + throw new YarnRuntimeException("subClusterId = " + subClusterId + "not exist"); + } + SubClusterInfo subClusterInfo = membership.get(subClusterId); + subClusterInfo.setLastHeartBeat(heartBearTime); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java index 12625a60e94..ebad527b6d4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java @@ -89,6 +89,9 @@ import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; import org.apache.hadoop.yarn.server.federation.store.records.RouterRMTokenRequest; import org.apache.hadoop.yarn.server.federation.store.records.RouterRMTokenResponse; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterResponse; import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1187,4 +1190,25 @@ public final class FederationStateStoreFacade { reservationHomeSubCluster); } } + + /** + * Deregister subCluster, Update the subCluster state to + * SC_LOST、SC_DECOMMISSIONED etc. + * + * @param subClusterId subClusterId. + * @param subClusterState The state of the subCluster to be updated. + * @throws YarnException yarn exception. + * @return If Deregister subCluster is successful, return true, otherwise, return false. + */ + public boolean deregisterSubCluster(SubClusterId subClusterId, + SubClusterState subClusterState) throws YarnException { + SubClusterDeregisterRequest deregisterRequest = + SubClusterDeregisterRequest.newInstance(subClusterId, subClusterState); + SubClusterDeregisterResponse response = stateStore.deregisterSubCluster(deregisterRequest); + // If the response is not empty, deregisterSubCluster is successful. + if (response != null) { + return true; + } + return false; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java index 24e9ad23c93..77abf18bd5f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java @@ -21,8 +21,11 @@ package org.apache.hadoop.yarn.server.router; import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.commons.lang.time.DurationFormatUtils; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; @@ -37,6 +40,7 @@ import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebAppUtil; +import org.apache.hadoop.yarn.server.router.cleaner.SubClusterCleaner; import org.apache.hadoop.yarn.server.router.clientrm.RouterClientRMService; import org.apache.hadoop.yarn.server.router.rmadmin.RouterRMAdminService; import org.apache.hadoop.yarn.server.router.webapp.RouterWebApp; @@ -50,6 +54,13 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.VisibleForTesting; +import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_ROUTER_DEREGISTER_SUBCLUSTER_ENABLED; +import static org.apache.hadoop.yarn.conf.YarnConfiguration.ROUTER_DEREGISTER_SUBCLUSTER_ENABLED; +import static org.apache.hadoop.yarn.conf.YarnConfiguration.ROUTER_SUBCLUSTER_CLEANER_INTERVAL_TIME; +import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_ROUTER_SUBCLUSTER_CLEANER_INTERVAL_TIME; +import static org.apache.hadoop.yarn.conf.YarnConfiguration.ROUTER_SCHEDULED_EXECUTOR_THREADS; +import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_ROUTER_SCHEDULED_EXECUTOR_THREADS; + /** * The router is a stateless YARN component which is the entry point to the * cluster. It can be deployed on multiple nodes behind a Virtual IP (VIP) with @@ -88,6 +99,9 @@ public class Router extends CompositeService { private static final String METRICS_NAME = "Router"; + private ScheduledThreadPoolExecutor scheduledExecutorService; + private SubClusterCleaner subClusterCleaner; + public Router() { super(Router.class.getName()); } @@ -117,6 +131,12 @@ public class Router extends CompositeService { addService(pauseMonitor); jm.setPauseMonitor(pauseMonitor); + // Initialize subClusterCleaner + this.subClusterCleaner = new SubClusterCleaner(this.conf); + int scheduledExecutorThreads = conf.getInt(ROUTER_SCHEDULED_EXECUTOR_THREADS, + DEFAULT_ROUTER_SCHEDULED_EXECUTOR_THREADS); + this.scheduledExecutorService = new ScheduledThreadPoolExecutor(scheduledExecutorThreads); + WebServiceClient.initialize(config); super.serviceInit(conf); } @@ -128,6 +148,16 @@ public class Router extends CompositeService { } catch (IOException e) { throw new YarnRuntimeException("Failed Router login", e); } + boolean isDeregisterSubClusterEnabled = this.conf.getBoolean( + ROUTER_DEREGISTER_SUBCLUSTER_ENABLED, DEFAULT_ROUTER_DEREGISTER_SUBCLUSTER_ENABLED); + if (isDeregisterSubClusterEnabled) { + long scCleanerIntervalMs = this.conf.getTimeDuration(ROUTER_SUBCLUSTER_CLEANER_INTERVAL_TIME, + DEFAULT_ROUTER_SUBCLUSTER_CLEANER_INTERVAL_TIME, TimeUnit.MINUTES); + this.scheduledExecutorService.scheduleAtFixedRate(this.subClusterCleaner, + 0, scCleanerIntervalMs, TimeUnit.MILLISECONDS); + LOG.info("Scheduled SubClusterCleaner With Interval: {}.", + DurationFormatUtils.formatDurationISO(scCleanerIntervalMs)); + } startWepApp(); super.serviceStart(); } @@ -146,12 +176,7 @@ public class Router extends CompositeService { } protected void shutDown() { - new Thread() { - @Override - public void run() { - Router.this.stop(); - } - }.start(); + new Thread(() -> Router.this.stop()).start(); } protected RouterClientRMService createClientRMProxyService() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/cleaner/SubClusterCleaner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/cleaner/SubClusterCleaner.java new file mode 100644 index 00000000000..1147f7742d5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/cleaner/SubClusterCleaner.java @@ -0,0 +1,92 @@ +/** + * 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.yarn.server.router.cleaner; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState; +import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Date; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * The SubClusterCleaner thread is used to check whether the SubCluster + * has exceeded the heartbeat time. + * If the SubCluster heartbeat time exceeds 30 mins, set the SubCluster to LOST. + * Check the thread every 1 mins, check once. + */ +public class SubClusterCleaner implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(SubClusterCleaner.class); + private FederationStateStoreFacade federationFacade; + private long heartbeatExpirationMillis; + + public SubClusterCleaner(Configuration conf) { + federationFacade = FederationStateStoreFacade.getInstance(); + this.heartbeatExpirationMillis = + conf.getTimeDuration(YarnConfiguration.ROUTER_SUBCLUSTER_EXPIRATION_TIME, + YarnConfiguration.DEFAULT_ROUTER_SUBCLUSTER_EXPIRATION_TIME, TimeUnit.MINUTES); + } + + @Override + public void run() { + try { + // Step1. Get Current Time. + Date now = new Date(); + LOG.info("SubClusterCleaner at {}.", now); + + Map subClusters = federationFacade.getSubClusters(true); + + for (Map.Entry subCluster : subClusters.entrySet()) { + // Step2. Get information about subClusters. + SubClusterId subClusterId = subCluster.getKey(); + SubClusterInfo subClusterInfo = subCluster.getValue(); + SubClusterState subClusterState = subClusterInfo.getState(); + long lastHeartBeatTime = subClusterInfo.getLastHeartBeat(); + + // We Only Check SubClusters in NEW and RUNNING states + if (!subClusterState.isUnusable()) { + long heartBeatInterval = now.getTime() - lastHeartBeatTime; + try { + // HeartBeat Interval Exceeds Expiration Time + if (heartBeatInterval > heartbeatExpirationMillis) { + LOG.info("Deregister SubCluster {} in state {} last heartbeat at {}.", + subClusterId, subClusterState, new Date(lastHeartBeatTime)); + federationFacade.deregisterSubCluster(subClusterId, SubClusterState.SC_LOST); + } + } catch (YarnException e) { + LOG.error("deregisterSubCluster failed on SubCluster {}.", subClusterId, e); + } + } else { + LOG.debug("SubCluster {} in state {} last heartbeat at {}, " + + "heartbeat interval < 30mins, no need for Deregister.", + subClusterId, subClusterState, new Date(lastHeartBeatTime)); + } + } + } catch (Throwable e) { + LOG.error("SubClusterCleaner Fails.", e); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/cleaner/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/cleaner/package-info.java new file mode 100644 index 00000000000..75477508cb1 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/cleaner/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Router Cleaner package. **/ +package org.apache.hadoop.yarn.server.router.cleaner; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/cleaner/TestSubClusterCleaner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/cleaner/TestSubClusterCleaner.java new file mode 100644 index 00000000000..57d427581a3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/cleaner/TestSubClusterCleaner.java @@ -0,0 +1,158 @@ +/** + * 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.yarn.server.router.cleaner; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.Time; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatResponse; +import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Map; +import java.util.concurrent.TimeoutException; + +public class TestSubClusterCleaner { + + //////////////////////////////// + // Router Constants + //////////////////////////////// + private Configuration conf; + private MemoryFederationStateStore stateStore; + private FederationStateStoreFacade facade; + private SubClusterCleaner cleaner; + private final static int NUM_SUBCLUSTERS = 4; + private final static long EXPIRATION_TIME = Time.now() - 5000; + + @Before + public void setup() throws YarnException { + conf = new YarnConfiguration(); + conf.setLong(YarnConfiguration.ROUTER_SUBCLUSTER_EXPIRATION_TIME, 1000); + conf.setInt(YarnConfiguration.FEDERATION_CACHE_TIME_TO_LIVE_SECS, 0); + + stateStore = new MemoryFederationStateStore(); + stateStore.init(conf); + + facade = FederationStateStoreFacade.getInstance(); + facade.reinitialize(stateStore, conf); + + cleaner = new SubClusterCleaner(conf); + for (int i = 0; i < NUM_SUBCLUSTERS; i++){ + // Create sub cluster id and info + SubClusterId subClusterId = SubClusterId.newInstance("SC-" + i); + SubClusterInfo subClusterInfo = SubClusterInfo.newInstance(subClusterId, + "127.0.0.1:1", "127.0.0.1:2", "127.0.0.1:3", "127.0.0.1:4", + SubClusterState.SC_RUNNING, Time.now(), ""); + // Register the subCluster + stateStore.registerSubCluster( + SubClusterRegisterRequest.newInstance(subClusterInfo)); + } + } + + @Test + public void testSubClustersWithOutHeartBeat() + throws InterruptedException, TimeoutException, YarnException { + + // We set up such a unit test, We set the status of all subClusters to RUNNING, + // and Manually set subCluster heartbeat expiration. + // At this time, the size of the Active SubCluster is 0. + Map subClustersMap = facade.getSubClusters(false); + + // Step1. Manually set subCluster heartbeat expiration. + // subCluster has no heartbeat, and all subClusters will expire. + subClustersMap.keySet().forEach(subClusterId -> + stateStore.setExpiredHeartbeat(subClusterId, EXPIRATION_TIME)); + + // Step2. Run the Cleaner to change the status of the expired SubCluster to SC_LOST. + cleaner.run(); + + // Step3. All clusters have expired, + // so the current Federation has no active subClusters. + int count = facade.getActiveSubClustersCount(); + Assert.assertEquals(0, count); + + // Step4. Check Active SubCluster Status. + // We want all subClusters to be SC_LOST. + subClustersMap.values().forEach(subClusterInfo -> { + SubClusterState subClusterState = subClusterInfo.getState(); + Assert.assertEquals(SubClusterState.SC_LOST, subClusterState); + }); + } + + @Test + public void testSubClustersPartWithHeartBeat() throws YarnException, InterruptedException { + + // Step1. Manually set subCluster heartbeat expiration. + for (int i = 0; i < NUM_SUBCLUSTERS; i++) { + // Create subCluster id and info. + expiredSubcluster("SC-" + i); + } + + // Step2. Run the Cleaner to change the status of the expired SubCluster to SC_LOST. + cleaner.run(); + + // Step3. Let SC-0, SC-1 resume heartbeat. + resumeSubClusterHeartbeat("SC-0"); + resumeSubClusterHeartbeat("SC-1"); + + // Step4. At this point we should have 2 subClusters that are surviving clusters. + int count = facade.getActiveSubClustersCount(); + Assert.assertEquals(2, count); + + // Step5. The result we expect is that SC-0 and SC-1 are in the RUNNING state, + // and SC-2 and SC-3 are in the SC_LOST state. + checkSubClusterState("SC-0", SubClusterState.SC_RUNNING); + checkSubClusterState("SC-1", SubClusterState.SC_RUNNING); + checkSubClusterState("SC-2", SubClusterState.SC_LOST); + checkSubClusterState("SC-3", SubClusterState.SC_LOST); + } + + private void resumeSubClusterHeartbeat(String pSubClusterId) + throws YarnException { + SubClusterId subClusterId = SubClusterId.newInstance(pSubClusterId); + SubClusterHeartbeatRequest request = SubClusterHeartbeatRequest.newInstance( + subClusterId, Time.now(), SubClusterState.SC_RUNNING, "test"); + SubClusterHeartbeatResponse response = stateStore.subClusterHeartbeat(request); + Assert.assertNotNull(response); + } + + private void expiredSubcluster(String pSubClusterId) { + SubClusterId subClusterId = SubClusterId.newInstance(pSubClusterId); + stateStore.setExpiredHeartbeat(subClusterId, EXPIRATION_TIME); + } + + private void checkSubClusterState(String pSubClusterId, SubClusterState expectState) + throws YarnException { + Map subClustersMap = facade.getSubClusters(false); + SubClusterId subClusterId = SubClusterId.newInstance(pSubClusterId); + SubClusterInfo subClusterInfo = subClustersMap.get(subClusterId); + if (subClusterInfo == null) { + throw new YarnException("subClusterId=" + pSubClusterId + " does not exist."); + } + Assert.assertEquals(expectState, subClusterInfo.getState()); + } +} From 723535b788070f6b103be3bae621fefe3b753081 Mon Sep 17 00:00:00 2001 From: hfutatzhanghb <1036798979@qq.com> Date: Thu, 16 Feb 2023 09:25:50 +0800 Subject: [PATCH 48/50] HDFS-16914. Add some logs for updateBlockForPipeline RPC. (#5381) Reviewed-by: Shilun Fan Signed-off-by: Tao Li --- .../hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java | 3 ++- .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java index 2f2bd99a275..9db5e1e9cc9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java @@ -1557,7 +1557,8 @@ class FsDatasetImpl implements FsDatasetSpi { @Override // FsDatasetSpi public Replica recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException { - LOG.info("Recover failed close " + b); + LOG.info("Recover failed close {}, new GS:{}, expectedBlockLen:{}", + b, newGS, expectedBlockLen); while (true) { try { try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 95b855e8af4..44b2a21779f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -5939,6 +5939,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, } // Ensure we record the new generation stamp getEditLog().logSync(); + LOG.info("bumpBlockGenerationStamp({}, client={}) success", + locatedBlock.getBlock(), clientName); return locatedBlock; } From 7a0903b7436e7f3fb4f07c40bed2218a2653ba38 Mon Sep 17 00:00:00 2001 From: Mehakmeet Singh Date: Thu, 16 Feb 2023 10:09:06 +0530 Subject: [PATCH 49/50] HADOOP-18633. fix test AbstractContractDistCpTest#testDistCpUpdateCheckFileSkip (#5401) Contributed by: Mehakmeet Singh --- hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm | 2 +- .../hadoop/tools/contract/AbstractContractDistCpTest.java | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm index 2d77619d5cf..0e11b24529b 100644 --- a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm +++ b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm @@ -653,7 +653,7 @@ checksums if the checksum algorithm between the two stores is different. * `distcp.update.modification.time` would only be used if either of the two stores don't have checksum validation resulting in incompatible checksum comparison between the two. Even if the property is set to true, it won't - be used if their is valid checksum comparison between the two stores. + be used if there is valid checksum comparison between the two stores. To turn off the modification time check, set this in your core-site.xml ```xml diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java index 532abc2aa40..aa42cb968d6 100644 --- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java +++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java @@ -899,8 +899,7 @@ public abstract class AbstractContractDistCpTest // Creating a source file with certain dataset. byte[] sourceBlock = dataset(10, 'a', 'z'); - // Write the dataset and as well create the target path. - ContractTestUtils.createFile(localFS, dest, true, sourceBlock); + // Write the dataset. ContractTestUtils .writeDataset(remoteFS, source, sourceBlock, sourceBlock.length, 1024, true); From 7e19bc31b65f86be91451a0ec7590023b6b57a12 Mon Sep 17 00:00:00 2001 From: Bryan Beaudreault Date: Thu, 16 Feb 2023 13:13:25 -0500 Subject: [PATCH 50/50] HADOOP-18215. Enhance WritableName to be able to return aliases for classes that use serializers (#4215) --- .../org/apache/hadoop/io/WritableName.java | 2 +- .../apache/hadoop/io/TestSequenceFile.java | 119 ++++++++++++++++++ .../apache/hadoop/io/TestWritableName.java | 50 +++++++- 3 files changed, 169 insertions(+), 2 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableName.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableName.java index e5e74875225..683d6c099b5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableName.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableName.java @@ -92,7 +92,7 @@ public class WritableName { ) throws IOException { Class writableClass = NAME_TO_CLASS.get(name); if (writableClass != null) - return writableClass.asSubclass(Writable.class); + return writableClass; try { return conf.getClassByName(name); } catch (ClassNotFoundException e) { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFile.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFile.java index 8b604d9c7a4..8944cae70f9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFile.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFile.java @@ -26,6 +26,9 @@ import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.SequenceFile.Metadata; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.io.serializer.Deserializer; +import org.apache.hadoop.io.serializer.Serialization; +import org.apache.hadoop.io.serializer.Serializer; import org.apache.hadoop.io.serializer.avro.AvroReflectSerialization; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.ReflectionUtils; @@ -756,6 +759,122 @@ public class TestSequenceFile { } } + @Test + public void testSerializationUsingWritableNameAlias() throws IOException { + Configuration config = new Configuration(); + config.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, SimpleSerializer.class.getName()); + Path path = new Path(System.getProperty("test.build.data", "."), + "SerializationUsingWritableNameAlias"); + + // write with the original serializable class + SequenceFile.Writer writer = SequenceFile.createWriter( + config, + SequenceFile.Writer.file(path), + SequenceFile.Writer.keyClass(SimpleSerializable.class), + SequenceFile.Writer.valueClass(SimpleSerializable.class)); + + int max = 10; + try { + SimpleSerializable val = new SimpleSerializable(); + val.setId(-1); + for (int i = 0; i < max; i++) { + SimpleSerializable key = new SimpleSerializable(); + key.setId(i); + writer.append(key, val); + } + } finally { + writer.close(); + } + + // override name so it gets forced to the new serializable + WritableName.setName(AnotherSimpleSerializable.class, SimpleSerializable.class.getName()); + + // read and expect our new serializable, and all the correct values read + SequenceFile.Reader reader = new SequenceFile.Reader( + config, + SequenceFile.Reader.file(path)); + + AnotherSimpleSerializable key = new AnotherSimpleSerializable(); + int count = 0; + while (true) { + key = (AnotherSimpleSerializable) reader.next(key); + if (key == null) { + // make sure we exhausted all the ints we wrote + assertEquals(count, max); + break; + } + assertEquals(count++, key.getId()); + } + } + + public static class SimpleSerializable implements Serializable { + + private int id; + + public int getId() { + return id; + } + + public void setId(int id) { + this.id = id; + } + } + + public static class AnotherSimpleSerializable extends SimpleSerializable { + } + + public static class SimpleSerializer implements Serialization { + + @Override + public boolean accept(Class c) { + return SimpleSerializable.class.isAssignableFrom(c); + } + + @Override + public Serializer getSerializer(Class c) { + return new Serializer() { + private DataOutputStream out; + @Override + public void open(OutputStream out) throws IOException { + this.out = new DataOutputStream(out); + } + + @Override + public void serialize(SimpleSerializable simpleSerializable) throws IOException { + out.writeInt(simpleSerializable.getId()); + } + + @Override + public void close() throws IOException { + out.close(); + } + }; + } + + @Override + public Deserializer getDeserializer(Class c) { + return new Deserializer() { + private DataInputStream dis; + @Override + public void open(InputStream in) throws IOException { + dis = new DataInputStream(in); + } + + @Override + public SimpleSerializable deserialize(SimpleSerializable simpleSerializable) + throws IOException { + simpleSerializable.setId(dis.readInt()); + return simpleSerializable; + } + + @Override + public void close() throws IOException { + dis.close(); + } + }; + } + } + /** For debugging and testing. */ public static void main(String[] args) throws Exception { int count = 1024 * 1024; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestWritableName.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestWritableName.java index 5950142220e..22f2aee62ad 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestWritableName.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestWritableName.java @@ -24,8 +24,14 @@ import java.io.IOException; import java.util.Random; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.io.serializer.Deserializer; +import org.apache.hadoop.io.serializer.Serialization; +import org.apache.hadoop.io.serializer.SerializationFactory; +import org.apache.hadoop.io.serializer.Serializer; import org.junit.Test; - +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; /** Unit tests for WritableName. */ @@ -63,6 +69,28 @@ public class TestWritableName { } } + private static class SimpleSerializable { + + } + + private static class SimpleSerializer implements Serialization { + + @Override + public boolean accept(Class c) { + return c.equals(SimpleSerializable.class); + } + + @Override + public Serializer getSerializer(Class c) { + return null; + } + + @Override + public Deserializer getDeserializer(Class c) { + return null; + } + } + private static final String testName = "mystring"; @Test @@ -95,7 +123,27 @@ public class TestWritableName { // check original name still works test = WritableName.getClass(testName, conf); assertTrue(test.equals(SimpleWritable.class)); + } + @Test + public void testAddNameSerializable() throws Exception { + Configuration conf = new Configuration(); + conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, SimpleSerializer.class.getName()); + SerializationFactory serializationFactory = + new SerializationFactory(conf); + + String altName = testName + ".alt"; + + WritableName.addName(SimpleSerializable.class, altName); + + Class test = WritableName.getClass(altName, conf); + assertEquals(test, SimpleSerializable.class); + assertNotNull(serializationFactory.getSerialization(test)); + + // check original name still works + test = WritableName.getClass(SimpleSerializable.class.getName(), conf); + assertEquals(test, SimpleSerializable.class); + assertNotNull(serializationFactory.getSerialization(test)); } @Test