diff --git a/LICENSE.txt b/LICENSE.txt index 1d97b95c10718..e100d26aeb1b0 100644 --- a/LICENSE.txt +++ b/LICENSE.txt @@ -1636,7 +1636,6 @@ derived.) The binary distribution of this product bundles these dependencies under the following license: -JUnit 4.11 Eclipse JDT Core 3.1.1 -------------------------------------------------------------------------------- (EPL v1.0) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java index 2757f8846ed4b..6251cd498e224 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java @@ -1840,7 +1840,7 @@ public long getTimeDuration(String name, long defaultValue, TimeUnit defaultUnit, TimeUnit returnUnit) { String vStr = get(name); if (null == vStr) { - return defaultValue; + return returnUnit.convert(defaultValue, defaultUnit); } else { return getTimeDurationHelper(name, vStr, defaultUnit, returnUnit); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java index dc6cd2bc2b07f..6e82543ca850a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java @@ -51,6 +51,7 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.LambdaUtils; import org.apache.hadoop.util.Progressable; @@ -456,8 +457,16 @@ public Path getInitialWorkingDirectory() { * @return current user's home directory. */ public Path getHomeDirectory() { - return new Path("/user/"+System.getProperty("user.name")).makeQualified( - getUri(), null); + String username; + try { + username = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch(IOException ex) { + LOG.warn("Unable to get user name. Fall back to system property " + + "user.name", ex); + username = System.getProperty("user.name"); + } + return new Path("/user/" + username) + .makeQualified(getUri(), null); } /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index efb675cf6fb2c..19f38af69998f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -2241,8 +2241,16 @@ public LocatedFileStatus next() throws IOException { * The default implementation returns {@code "/user/$USER/"}. */ public Path getHomeDirectory() { + String username; + try { + username = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch(IOException ex) { + LOGGER.warn("Unable to get user name. Fall back to system property " + + "user.name", ex); + username = System.getProperty("user.name"); + } return this.makeQualified( - new Path(USER_HOME_PREFIX + "/" + System.getProperty("user.name"))); + new Path(USER_HOME_PREFIX + "/" + username)); } 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 9259d877e12fd..f329accec7553 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 @@ -145,7 +145,14 @@ public synchronized void reset() { setDelegationTokenSeqNum(0); currentTokens.clear(); } - + + /** + * Total count of active delegation tokens. + */ + public long getCurrentTokensSize() { + return currentTokens.size(); + } + /** * Add a previously used master key to cache (when NN restarts), * should be called before activate(). diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java index 1cf7a20c610b3..64beb7b484296 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java @@ -1401,6 +1401,10 @@ public void testEnumFromXml() throws IOException { @Test public void testTimeDuration() { Configuration conf = new Configuration(false); + + assertEquals(7000L, + conf.getTimeDuration("test.time.a", 7L, SECONDS, MILLISECONDS)); + conf.setTimeDuration("test.time.a", 7L, SECONDS); assertEquals("7s", conf.get("test.time.a")); assertEquals(0L, conf.getTimeDuration("test.time.a", 30, MINUTES)); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java index ad12f0babac50..df685cf681a2d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java @@ -259,6 +259,29 @@ public void testGetUserWithOwnerAndReal() { ugi.getRealUser().getAuthenticationMethod()); } + @Test + public void testDelegationTokenCount() throws Exception { + final TestDelegationTokenSecretManager dtSecretManager = + new TestDelegationTokenSecretManager(24*60*60*1000, + 3*1000, 1*1000, 3600000); + try { + dtSecretManager.startThreads(); + Assert.assertEquals(dtSecretManager.getCurrentTokensSize(), 0); + final Token token1 = + generateDelegationToken(dtSecretManager, "SomeUser", "JobTracker"); + Assert.assertEquals(dtSecretManager.getCurrentTokensSize(), 1); + final Token token2 = + generateDelegationToken(dtSecretManager, "SomeUser", "JobTracker"); + Assert.assertEquals(dtSecretManager.getCurrentTokensSize(), 2); + dtSecretManager.cancelToken(token1, "JobTracker"); + Assert.assertEquals(dtSecretManager.getCurrentTokensSize(), 1); + dtSecretManager.cancelToken(token2, "JobTracker"); + Assert.assertEquals(dtSecretManager.getCurrentTokensSize(), 0); + } finally { + dtSecretManager.stopThreads(); + } + } + @Test public void testDelegationTokenSecretManager() throws Exception { final TestDelegationTokenSecretManager dtSecretManager = diff --git a/hadoop-hdds/common/pom.xml b/hadoop-hdds/common/pom.xml index 2385a9ec0cdf3..5d1bb52844dff 100644 --- a/hadoop-hdds/common/pom.xml +++ b/hadoop-hdds/common/pom.xml @@ -137,6 +137,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> opentracing-util 0.31.0 + + org.yaml + snakeyaml + 1.16 + diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java index b097321507739..2c267fb176474 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java @@ -368,6 +368,8 @@ public final class ScmConfigKeys { "hdds.scm.http.kerberos.keytab"; // Network topology + public static final String OZONE_SCM_NETWORK_TOPOLOGY_SCHEMA_FILE_TYPE = + "ozone.scm.network.topology.schema.file.type"; public static final String OZONE_SCM_NETWORK_TOPOLOGY_SCHEMA_FILE = "ozone.scm.network.topology.schema.file"; public static final String OZONE_SCM_NETWORK_TOPOLOGY_SCHEMA_FILE_DEFAULT = diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchema.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchema.java index 8c289f7617679..47e5de880d6d7 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchema.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchema.java @@ -19,6 +19,8 @@ import org.apache.hadoop.HadoopIllegalArgumentException; +import java.util.List; + /** * Network topology schema to housekeeper relevant information. */ @@ -59,13 +61,15 @@ public static LayerType getType(String typeStr) { } // default cost - private final int cost; + private int cost; // layer Type, mandatory property - private final LayerType type; + private LayerType type; // default name, can be null or "" - private final String defaultName; + private String defaultName; // layer prefix, can be null or "" - private final String prefix; + private String prefix; + // sublayer + private List sublayer; /** * Builder for NodeSchema. @@ -123,6 +127,14 @@ public NodeSchema(LayerType type, int cost, String prefix, this.defaultName = defaultName; } + /** + * Constructor. This constructor is only used when build NodeSchema from + * YAML file. + */ + public NodeSchema() { + this.type = LayerType.INNER_NODE; + } + public boolean matchPrefix(String name) { if (name == null || name.isEmpty() || prefix == null || prefix.isEmpty()) { return false; @@ -134,15 +146,38 @@ public LayerType getType() { return this.type; } + public void setType(LayerType type) { + this.type = type; + } + public String getPrefix() { return this.prefix; } + public void setPrefix(String prefix) { + this.prefix = prefix; + } + public String getDefaultName() { return this.defaultName; } + public void setDefaultName(String name) { + this.defaultName = name; + } + public int getCost() { return this.cost; } + public void setCost(int cost) { + this.cost = cost; + } + + public void setSublayer(List sublayer) { + this.sublayer = sublayer; + } + + public List getSublayer() { + return sublayer; + } } \ No newline at end of file diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchemaLoader.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchemaLoader.java index 9125fb7b8cc18..32d7f16a9969d 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchemaLoader.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchemaLoader.java @@ -30,6 +30,7 @@ import javax.xml.parsers.DocumentBuilderFactory; import javax.xml.parsers.ParserConfigurationException; import java.io.File; +import java.io.FileInputStream; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -37,6 +38,7 @@ import java.util.Map; import org.apache.hadoop.hdds.scm.net.NodeSchema.LayerType; +import org.yaml.snakeyaml.Yaml; /** * A Network topology layer schema loading tool that loads user defined network @@ -95,7 +97,7 @@ public List getSchemaList() { * @param schemaFilePath path of schema file * @return all valid node schemas defined in schema file */ - public NodeSchemaLoadResult loadSchemaFromFile(String schemaFilePath) + public NodeSchemaLoadResult loadSchemaFromXml(String schemaFilePath) throws IllegalArgumentException { try { File schemaFile = new File(schemaFilePath); @@ -165,6 +167,88 @@ private NodeSchemaLoadResult loadSchema(File schemaFile) throws return schemaList; } + /** + * Load user defined network layer schemas from a YAML configuration file. + * @param schemaFilePath path of schema file + * @return all valid node schemas defined in schema file + */ + public NodeSchemaLoadResult loadSchemaFromYaml(String schemaFilePath) + throws IllegalArgumentException { + try { + File schemaFile = new File(schemaFilePath); + if (!schemaFile.exists()) { + String msg = "Network topology layer schema file " + schemaFilePath + + " is not found."; + LOG.warn(msg); + throw new IllegalArgumentException(msg); + } + return loadSchemaFromYaml(schemaFile); + } catch (Exception e) { + throw new IllegalArgumentException("Fail to load network topology node" + + " schema file: " + schemaFilePath + " , error:" + + e.getMessage()); + } + } + + /** + * Load network topology layer schemas from a YAML configuration file. + * @param schemaFile schema file + * @return all valid node schemas defined in schema file + * @throws ParserConfigurationException ParserConfigurationException happen + * @throws IOException no such schema file + * @throws SAXException xml file has some invalid elements + * @throws IllegalArgumentException xml file content is logically invalid + */ + private NodeSchemaLoadResult loadSchemaFromYaml(File schemaFile) { + LOG.info("Loading network topology layer schema file {}", schemaFile); + NodeSchemaLoadResult finalSchema; + + try { + Yaml yaml = new Yaml(); + NodeSchema nodeTree; + + try (FileInputStream fileInputStream = new FileInputStream(schemaFile)) { + nodeTree = yaml.loadAs(fileInputStream, NodeSchema.class); + } + List schemaList = new ArrayList<>(); + if (nodeTree.getType() != LayerType.ROOT) { + throw new IllegalArgumentException("First layer is not a ROOT node." + + " schema file: " + schemaFile.getAbsolutePath()); + } + schemaList.add(nodeTree); + if (nodeTree.getSublayer() != null) { + nodeTree = nodeTree.getSublayer().get(0); + } + + while (nodeTree != null) { + if (nodeTree.getType() == LayerType.LEAF_NODE + && nodeTree.getSublayer() != null) { + throw new IllegalArgumentException("Leaf node in the middle of path." + + " schema file: " + schemaFile.getAbsolutePath()); + } + if (nodeTree.getType() == LayerType.ROOT) { + throw new IllegalArgumentException("Multiple root nodes are defined." + + " schema file: " + schemaFile.getAbsolutePath()); + } + schemaList.add(nodeTree); + if (nodeTree.getSublayer() != null) { + nodeTree = nodeTree.getSublayer().get(0); + } else { + break; + } + } + finalSchema = new NodeSchemaLoadResult(schemaList, true); + } catch (RuntimeException e) { + throw e; + } catch (Exception e) { + throw new IllegalArgumentException("Fail to load network topology node" + + " schema file: " + schemaFile.getAbsolutePath() + " , error:" + + e.getMessage()); + } + + return finalSchema; + } + /** * Load layoutVersion from root element in the XML configuration file. * @param root root element diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchemaManager.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchemaManager.java index 8f2fac75f59a3..8e5d935e6175d 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchemaManager.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchemaManager.java @@ -59,13 +59,20 @@ public void init(Configuration conf) { /** * Load schemas from network topology schema configuration file */ + String schemaFileType = conf.get( + ScmConfigKeys.OZONE_SCM_NETWORK_TOPOLOGY_SCHEMA_FILE_TYPE); + String schemaFile = conf.get( ScmConfigKeys.OZONE_SCM_NETWORK_TOPOLOGY_SCHEMA_FILE, ScmConfigKeys.OZONE_SCM_NETWORK_TOPOLOGY_SCHEMA_FILE_DEFAULT); NodeSchemaLoadResult result; try { - result = NodeSchemaLoader.getInstance().loadSchemaFromFile(schemaFile); + if (schemaFileType.toLowerCase().compareTo("yaml") == 0) { + result = NodeSchemaLoader.getInstance().loadSchemaFromYaml(schemaFile); + } else { + result = NodeSchemaLoader.getInstance().loadSchemaFromXml(schemaFile); + } allSchema = result.getSchemaList(); enforcePrefix = result.isEnforePrefix(); maxLevel = allSchema.size(); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java index 71f9831bc1000..559022f286974 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java @@ -190,7 +190,7 @@ private String useFirstLine(String message) { public ScmInfo getScmInfo() throws IOException { HddsProtos.GetScmInfoRequestProto request = HddsProtos.GetScmInfoRequestProto.getDefaultInstance(); - HddsProtos.GetScmInfoRespsonseProto resp; + HddsProtos.GetScmInfoResponseProto resp; try { resp = rpcProxy.getScmInfo(NULL_RPC_CONTROLLER, request); } catch (ServiceException e) { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java index acd446ede3050..ffb4686b3d378 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java @@ -357,7 +357,7 @@ public ScmInfo getScmInfo() throws IOException { .setTraceID(TracingUtil.exportCurrentSpan()) .build(); try { - HddsProtos.GetScmInfoRespsonseProto resp = rpcProxy.getScmInfo( + HddsProtos.GetScmInfoResponseProto resp = rpcProxy.getScmInfo( NULL_RPC_CONTROLLER, request); ScmInfo.Builder builder = new ScmInfo.Builder() .setClusterId(resp.getClusterId()) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/ScmBlockLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/ScmBlockLocationProtocolServerSideTranslatorPB.java index 9ff6879573fbe..65f0a973ce412 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/ScmBlockLocationProtocolServerSideTranslatorPB.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/ScmBlockLocationProtocolServerSideTranslatorPB.java @@ -133,7 +133,7 @@ public DeleteScmKeyBlocksResponseProto deleteScmKeyBlocks( } @Override - public HddsProtos.GetScmInfoRespsonseProto getScmInfo( + public HddsProtos.GetScmInfoResponseProto getScmInfo( RpcController controller, HddsProtos.GetScmInfoRequestProto req) throws ServiceException { ScmInfo scmInfo; @@ -142,7 +142,7 @@ public HddsProtos.GetScmInfoRespsonseProto getScmInfo( } catch (IOException ex) { throw new ServiceException(ex); } - return HddsProtos.GetScmInfoRespsonseProto.newBuilder() + return HddsProtos.GetScmInfoResponseProto.newBuilder() .setClusterId(scmInfo.getClusterId()) .setScmId(scmInfo.getScmId()) .build(); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java index a994ed7ea5029..ea96cfa0aaea1 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java @@ -265,13 +265,13 @@ public ClosePipelineResponseProto closePipeline( } @Override - public HddsProtos.GetScmInfoRespsonseProto getScmInfo( + public HddsProtos.GetScmInfoResponseProto getScmInfo( RpcController controller, HddsProtos.GetScmInfoRequestProto req) throws ServiceException { try (Scope scope = TracingUtil .importAndCreateScope("getScmInfo", req.getTraceID())) { ScmInfo scmInfo = impl.getScmInfo(); - return HddsProtos.GetScmInfoRespsonseProto.newBuilder() + return HddsProtos.GetScmInfoResponseProto.newBuilder() .setClusterId(scmInfo.getClusterId()) .setScmId(scmInfo.getScmId()) .build(); diff --git a/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto b/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto index eebda0a823877..6745c6ee14bb5 100644 --- a/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto +++ b/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto @@ -138,5 +138,5 @@ service ScmBlockLocationProtocolService { * Gets the scmInfo from SCM. */ rpc getScmInfo(hadoop.hdds.GetScmInfoRequestProto) - returns (hadoop.hdds.GetScmInfoRespsonseProto); + returns (hadoop.hdds.GetScmInfoResponseProto); } diff --git a/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto b/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto index 2c397efbbf63f..7ad19b112a6b1 100644 --- a/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto +++ b/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto @@ -262,7 +262,7 @@ service StorageContainerLocationProtocolService { * Returns information about SCM. */ rpc getScmInfo(GetScmInfoRequestProto) - returns (GetScmInfoRespsonseProto); + returns (GetScmInfoResponseProto); /** * Checks if SCM is in SafeMode. diff --git a/hadoop-hdds/common/src/main/proto/hdds.proto b/hadoop-hdds/common/src/main/proto/hdds.proto index f5570503e52b6..ddde7ea93acb5 100644 --- a/hadoop-hdds/common/src/main/proto/hdds.proto +++ b/hadoop-hdds/common/src/main/proto/hdds.proto @@ -169,7 +169,7 @@ message GetScmInfoRequestProto { optional string traceID = 1; } -message GetScmInfoRespsonseProto { +message GetScmInfoResponseProto { required string clusterId = 1; required string scmId = 2; } @@ -222,8 +222,8 @@ message ContainerBlockID { */ message BlockTokenSecretProto { /** - * File access permissions mode. - */ + * File access permissions mode. + */ enum AccessModeProto { READ = 1; WRITE = 2; diff --git a/hadoop-hdds/common/src/main/resources/network-topology-default.yaml b/hadoop-hdds/common/src/main/resources/network-topology-default.yaml new file mode 100644 index 0000000000000..561869fb43b54 --- /dev/null +++ b/hadoop-hdds/common/src/main/resources/network-topology-default.yaml @@ -0,0 +1,61 @@ +# 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. + +--- +# Cost: The cost of crossing this layer. +# The value should be positive integer or 0. This field is optional. +# When it's not defined, it's value is default "1". +cost: 1 + +# The prefix of this layer. +# If the prefix is "dc", then every name in this layer should start with "dc", +# such as "dc1", "dc2". +# Note that unlike XML schema, the prefix must be specified explicitly if the type is InnerNode. +prefix: / + +# Layer type, optional field, default value InnerNode. +# Current value range : {ROOT, INNER_NODE, LEAF_NODE} +type: ROOT + +# Layer name +defaultName: root + +# Sub layer +# The sub layer property defines as a list which can reflect a node tree, though +# in schema template it always has only one child. +sublayer: + - + cost: 1 + prefix: dc + defaultName: datacenter + type: INNER_NODE + sublayer: + - + cost: 1 + prefix: rack + defaultName: rack + type: INNER_NODE + sublayer: + - + cost: 1 + prefix: ng + defaultName: nodegroup + type: INNER_NODE + sublayer: + - + defaultName: node + type: LEAF_NODE + prefix: node +... \ No newline at end of file diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/net/TestNodeSchemaLoader.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/net/TestNodeSchemaLoader.java index 6d9057cb550e2..30799b1099b37 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/net/TestNodeSchemaLoader.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/net/TestNodeSchemaLoader.java @@ -44,7 +44,7 @@ public TestNodeSchemaLoader(String schemaFile, String errMsg) { try { String filePath = classLoader.getResource( "./networkTopologyTestFiles/" + schemaFile).getPath(); - NodeSchemaLoader.getInstance().loadSchemaFromFile(filePath); + NodeSchemaLoader.getInstance().loadSchemaFromXml(filePath); fail("expect exceptions"); } catch (Throwable e) { assertTrue(e.getMessage().contains(errMsg)); @@ -83,7 +83,7 @@ public void testGood() { try { String filePath = classLoader.getResource( "./networkTopologyTestFiles/good.xml").getPath(); - NodeSchemaLoader.getInstance().loadSchemaFromFile(filePath); + NodeSchemaLoader.getInstance().loadSchemaFromXml(filePath); } catch (Throwable e) { fail("should succeed"); } @@ -94,7 +94,7 @@ public void testNotExist() { String filePath = classLoader.getResource( "./networkTopologyTestFiles/good.xml").getPath() + ".backup"; try { - NodeSchemaLoader.getInstance().loadSchemaFromFile(filePath); + NodeSchemaLoader.getInstance().loadSchemaFromXml(filePath); fail("should fail"); } catch (Throwable e) { assertTrue(e.getMessage().contains("file " + filePath + " is not found")); diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/net/TestYamlSchemaLoader.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/net/TestYamlSchemaLoader.java new file mode 100644 index 0000000000000..580a7fb485e80 --- /dev/null +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/net/TestYamlSchemaLoader.java @@ -0,0 +1,90 @@ +/** + * 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.hdds.scm.net; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collection; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** Test the node schema loader. */ +@RunWith(Parameterized.class) +public class TestYamlSchemaLoader { + private static final Logger LOG = + LoggerFactory.getLogger(TestYamlSchemaLoader.class); + private ClassLoader classLoader = + Thread.currentThread().getContextClassLoader(); + + public TestYamlSchemaLoader(String schemaFile, String errMsg) { + try { + String filePath = classLoader.getResource( + "./networkTopologyTestFiles/" + schemaFile).getPath(); + NodeSchemaLoader.getInstance().loadSchemaFromYaml(filePath); + fail("expect exceptions"); + } catch (Throwable e) { + assertTrue(e.getMessage().contains(errMsg)); + } + } + + @Rule + public Timeout testTimeout = new Timeout(30000); + + @Parameters + public static Collection getSchemaFiles() { + Object[][] schemaFiles = new Object[][]{ + {"multiple-root.yaml", "Multiple root"}, + {"middle-leaf.yaml", "Leaf node in the middle"}, + }; + return Arrays.asList(schemaFiles); + } + + + @Test + public void testGood() { + try { + String filePath = classLoader.getResource( + "./networkTopologyTestFiles/good.yaml").getPath(); + NodeSchemaLoader.getInstance().loadSchemaFromYaml(filePath); + } catch (Throwable e) { + fail("should succeed"); + } + } + + @Test + public void testNotExist() { + String filePath = classLoader.getResource( + "./networkTopologyTestFiles/good.xml").getPath() + ".backup"; + try { + NodeSchemaLoader.getInstance().loadSchemaFromXml(filePath); + fail("should fail"); + } catch (Throwable e) { + assertTrue(e.getMessage().contains("file " + filePath + " is not found")); + } + } + +} diff --git a/hadoop-hdds/common/src/test/resources/networkTopologyTestFiles/good.yaml b/hadoop-hdds/common/src/test/resources/networkTopologyTestFiles/good.yaml new file mode 100644 index 0000000000000..d5092ad0dbc75 --- /dev/null +++ b/hadoop-hdds/common/src/test/resources/networkTopologyTestFiles/good.yaml @@ -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. + +--- +# Cost: The cost of crossing this layer. +# The value should be positive integer or 0. This field is optional. +# When it's not defined, it's value is default "1". +cost: 1 + +# The prefix of this layer. +# If the prefix is "dc", then every name in this layer should start with "dc", +# such as "dc1", "dc2". +# Note that unlike XML schema, the prefix must be specified explicitly if the type is InnerNode. +prefix: / + +# Layer type, optional field, default value InnerNode. +# Current value range : {ROOT, INNER_NODE, LEAF_NODE} +type: ROOT + +# Layer name +defaultName: root + +# The sub layer of current layer. We use list +sublayer: + - + cost: 1 + prefix: dc + defaultName: datacenter + type: INNER_NODE + sublayer: + - + cost: 1 + prefix: rack + defaultName: rack + type: INNER_NODE + sublayer: + - + cost: 1 + prefix: ng + defaultName: nodegroup + type: INNER_NODE + sublayer: + - + defaultName: node + type: LEAF_NODE + prefix: node +... \ No newline at end of file diff --git a/hadoop-hdds/common/src/test/resources/networkTopologyTestFiles/middle-leaf.yaml b/hadoop-hdds/common/src/test/resources/networkTopologyTestFiles/middle-leaf.yaml new file mode 100644 index 0000000000000..0a2d490d5fa94 --- /dev/null +++ b/hadoop-hdds/common/src/test/resources/networkTopologyTestFiles/middle-leaf.yaml @@ -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. + +--- +# Cost: The cost of crossing this layer. +# The value should be positive integer or 0. This field is optional. +# When it's not defined, it's value is default "1". +cost: 1 + +# The prefix of this layer. +# If the prefix is "dc", then every name in this layer should start with "dc", +# such as "dc1", "dc2". +# Note that unlike XML schema, the prefix must be specified explicitly if the type is InnerNode. +prefix: / + +# Layer type, optional field, default value InnerNode. +# Current value range : {ROOT, INNER_NODE, LEAF_NODE} +type: ROOT + +# Layer name +defaultName: root + +# The sub layer of current layer. We use list +sublayer: + - + cost: 1 + prefix: dc + defaultName: datacenter + type: INNER_NODE + sublayer: + - + cost: 1 + prefix: node + defaultName: rack + type: LEAF_NODE + sublayer: + - + cost: 1 + prefix: ng + defaultName: nodegroup + type: INNER_NODE + sublayer: + - + defaultName: node + type: LEAF_NODE + prefix: node +... \ No newline at end of file diff --git a/hadoop-hdds/common/src/test/resources/networkTopologyTestFiles/multiple-root.yaml b/hadoop-hdds/common/src/test/resources/networkTopologyTestFiles/multiple-root.yaml new file mode 100644 index 0000000000000..536ed23eb6c84 --- /dev/null +++ b/hadoop-hdds/common/src/test/resources/networkTopologyTestFiles/multiple-root.yaml @@ -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. + +--- +# Cost: The cost of crossing this layer. +# The value should be positive integer or 0. This field is optional. +# When it's not defined, it's value is default "1". +cost: 1 + +# The prefix of this layer. +# If the prefix is "dc", then every name in this layer should start with "dc", +# such as "dc1", "dc2". +# Note that unlike XML schema, the prefix must be specified explicitly if the type is InnerNode. +prefix: / + +# Layer type, optional field, default value InnerNode. +# Current value range : {ROOT, INNER_NODE, LEAF_NODE} +type: ROOT + +# Layer name +defaultName: root + +# The sub layer of current layer. We use list +sublayer: + - + cost: 1 + prefix: root + defaultName: root + type: ROOT + sublayer: + - + cost: 1 + prefix: rack + defaultName: rack + type: INNER_NODE + sublayer: + - + cost: 1 + prefix: ng + defaultName: nodegroup + type: INNER_NODE + sublayer: + - + defaultName: node + type: LEAF_NODE + prefix: node +... \ No newline at end of file diff --git a/hadoop-hdds/container-service/pom.xml b/hadoop-hdds/container-service/pom.xml index 6d3e8085a9f38..c74d68690e360 100644 --- a/hadoop-hdds/container-service/pom.xml +++ b/hadoop-hdds/container-service/pom.xml @@ -48,7 +48,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> org.yaml snakeyaml - 1.8 + 1.16 com.google.code.findbugs diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java index 7e06473006328..56151f87d1441 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java @@ -19,9 +19,10 @@ import com.google.common.base.Preconditions; import com.google.protobuf.GeneratedMessage; import java.util.Map; -import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdds.protocol.proto + .StorageContainerDatanodeProtocolProtos.SCMCommandProto; import org.apache.hadoop.hdds.protocol.proto .StorageContainerDatanodeProtocolProtos.PipelineAction; import org.apache.hadoop.hdds.protocol.proto @@ -34,8 +35,6 @@ import org.apache.hadoop.ozone.container.common.states.datanode .RunningDatanodeState; import org.apache.hadoop.ozone.protocol.commands.CommandStatus; -import org.apache.hadoop.ozone.protocol.commands.CommandStatus - .CommandStatusBuilder; import org.apache.hadoop.ozone.protocol.commands .DeleteBlockCommandStatus.DeleteBlockCommandStatusBuilder; import org.apache.hadoop.ozone.protocol.commands.SCMCommand; @@ -432,27 +431,14 @@ public void addCmdStatus(Long key, CommandStatus status) { * @param cmd - {@link SCMCommand}. */ public void addCmdStatus(SCMCommand cmd) { - final Optional cmdStatusBuilder; - switch (cmd.getType()) { - case replicateContainerCommand: - cmdStatusBuilder = Optional.of(CommandStatusBuilder.newBuilder()); - break; - case deleteBlocksCommand: - cmdStatusBuilder = Optional.of( - DeleteBlockCommandStatusBuilder.newBuilder()); - break; - case deleteContainerCommand: - cmdStatusBuilder = Optional.of(CommandStatusBuilder.newBuilder()); - break; - default: - cmdStatusBuilder = Optional.empty(); + if (cmd.getType() == SCMCommandProto.Type.deleteBlocksCommand) { + addCmdStatus(cmd.getId(), + DeleteBlockCommandStatusBuilder.newBuilder() + .setCmdId(cmd.getId()) + .setStatus(Status.PENDING) + .setType(cmd.getType()) + .build()); } - cmdStatusBuilder.ifPresent(statusBuilder -> - addCmdStatus(cmd.getId(), statusBuilder - .setCmdId(cmd.getId()) - .setStatus(Status.PENDING) - .setType(cmd.getType()) - .build())); } /** diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java index 4a6787ee4dac0..b54fb1a17ac05 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java @@ -56,11 +56,7 @@ public void handle(final SCMCommand command, final ContainerController controller = ozoneContainer.getController(); controller.deleteContainer(deleteContainerCommand.getContainerID(), deleteContainerCommand.isForce()); - updateCommandStatus(context, command, - (cmdStatus) -> cmdStatus.setStatus(true), LOG); } catch (IOException e) { - updateCommandStatus(context, command, - (cmdStatus) -> cmdStatus.setStatus(false), LOG); LOG.error("Exception occurred while deleting the container.", e); } finally { totalTime += Time.monotonicNow() - startTime; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReplicateContainerCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReplicateContainerCommandHandler.java index 81d162d28037e..a028041b19675 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReplicateContainerCommandHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReplicateContainerCommandHandler.java @@ -61,25 +61,17 @@ public ReplicateContainerCommandHandler( public void handle(SCMCommand command, OzoneContainer container, StateContext context, SCMConnectionManager connectionManager) { - ReplicateContainerCommand replicateCommand = + final ReplicateContainerCommand replicateCommand = (ReplicateContainerCommand) command; - try { - List sourceDatanodes = - replicateCommand.getSourceDatanodes(); - long containerID = replicateCommand.getContainerID(); - - Preconditions.checkArgument(sourceDatanodes.size() > 0, - String.format("Replication command is received for container %d " - + "but the size of source datanodes was 0.", containerID)); - - ReplicationTask replicationTask = - new ReplicationTask(containerID, sourceDatanodes); - supervisor.addTask(replicationTask); - - } finally { - updateCommandStatus(context, command, - (cmdStatus) -> cmdStatus.setStatus(true), LOG); - } + final List sourceDatanodes = + replicateCommand.getSourceDatanodes(); + final long containerID = replicateCommand.getContainerID(); + + Preconditions.checkArgument(sourceDatanodes.size() > 0, + String.format("Replication command is received for container %d " + + "but the size of source datanodes was 0.", containerID)); + + supervisor.addTask(new ReplicationTask(containerID, sourceDatanodes)); } @Override diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/command/CommandStatusReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/command/CommandStatusReportHandler.java index 0ef02a338b1f3..d1479f7c2252c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/command/CommandStatusReportHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/command/CommandStatusReportHandler.java @@ -18,9 +18,10 @@ package org.apache.hadoop.hdds.scm.command; import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.protocol.proto + .StorageContainerDatanodeProtocolProtos.SCMCommandProto; import org.apache.hadoop.hdds.protocol.proto .StorageContainerDatanodeProtocolProtos.CommandStatus; -import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager; import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher .CommandStatusReportFromDatanode; @@ -54,32 +55,14 @@ public void onMessage(CommandStatusReportFromDatanode report, cmdStatusList.forEach(cmdStatus -> { LOGGER.trace("Emitting command status for id:{} type: {}", cmdStatus .getCmdId(), cmdStatus.getType()); - switch (cmdStatus.getType()) { - case replicateContainerCommand: - publisher.fireEvent(SCMEvents.REPLICATION_STATUS, new - ReplicationStatus(cmdStatus)); - if (cmdStatus.getStatus() == CommandStatus.Status.EXECUTED) { - publisher.fireEvent(SCMEvents.REPLICATION_COMPLETE, - new ReplicationManager.ReplicationCompleted( - cmdStatus.getCmdId())); - } - break; - case deleteBlocksCommand: + if (cmdStatus.getType() == SCMCommandProto.Type.deleteBlocksCommand) { if (cmdStatus.getStatus() == CommandStatus.Status.EXECUTED) { publisher.fireEvent(SCMEvents.DELETE_BLOCK_STATUS, new DeleteBlockStatus(cmdStatus)); } - break; - case deleteContainerCommand: - if (cmdStatus.getStatus() == CommandStatus.Status.EXECUTED) { - publisher.fireEvent(SCMEvents.DELETE_CONTAINER_COMMAND_COMPLETE, - new ReplicationManager.DeleteContainerCommandCompleted( - cmdStatus.getCmdId())); - } - default: + } else { LOGGER.debug("CommandStatus of type:{} not handled in " + "CommandStatusReportHandler.", cmdStatus.getType()); - break; } }); } @@ -109,24 +92,6 @@ public long getId() { } } - /** - * Wrapper event for Replicate Command. - */ - public static class ReplicationStatus extends CommandStatusEvent { - public ReplicationStatus(CommandStatus cmdStatus) { - super(cmdStatus); - } - } - - /** - * Wrapper event for CloseContainer Command. - */ - public static class CloseContainerStatus extends CommandStatusEvent { - public CloseContainerStatus(CommandStatus cmdStatus) { - super(cmdStatus); - } - } - /** * Wrapper event for DeleteBlock Command. */ diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/DeleteContainerCommandWatcher.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/DeleteContainerCommandWatcher.java deleted file mode 100644 index 0b1e4c8233217..0000000000000 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/DeleteContainerCommandWatcher.java +++ /dev/null @@ -1,56 +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.hdds.scm.container; - -import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager - .DeletionRequestToRepeat; -import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager - .DeleteContainerCommandCompleted; -import org.apache.hadoop.hdds.scm.events.SCMEvents; -import org.apache.hadoop.hdds.server.events.Event; -import org.apache.hadoop.hdds.server.events.EventPublisher; -import org.apache.hadoop.hdds.server.events.EventWatcher; -import org.apache.hadoop.ozone.lease.LeaseManager; - -/** - * Command watcher to track the delete container commands. - */ -public class DeleteContainerCommandWatcher extends - EventWatcher { - - public DeleteContainerCommandWatcher( - Event startEvent, - Event completionEvent, - LeaseManager leaseManager) { - super(startEvent, completionEvent, leaseManager); - } - - @Override - protected void onTimeout(EventPublisher publisher, - DeletionRequestToRepeat payload) { - //put back to the original queue - publisher.fireEvent(SCMEvents.REPLICATE_CONTAINER, - payload.getRequest()); - } - - - @Override - protected void onFinished(EventPublisher publisher, - DeletionRequestToRepeat payload) { - - } -} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationCommandWatcher.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationCommandWatcher.java deleted file mode 100644 index 03a81a7db867d..0000000000000 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationCommandWatcher.java +++ /dev/null @@ -1,56 +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.hdds.scm.container.replication; - -import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager - .ReplicationCompleted; -import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager - .ReplicationRequestToRepeat; -import org.apache.hadoop.hdds.scm.events.SCMEvents; -import org.apache.hadoop.hdds.server.events.Event; -import org.apache.hadoop.hdds.server.events.EventPublisher; -import org.apache.hadoop.hdds.server.events.EventWatcher; -import org.apache.hadoop.ozone.lease.LeaseManager; - -/** - * Command watcher to track the replication commands. - */ -public class ReplicationCommandWatcher - extends - EventWatcher { - - public ReplicationCommandWatcher(Event startEvent, - Event completionEvent, - LeaseManager leaseManager) { - super(startEvent, completionEvent, leaseManager); - } - - @Override - protected void onTimeout(EventPublisher publisher, - ReplicationRequestToRepeat payload) { - //put back to the original queue - publisher.fireEvent(SCMEvents.REPLICATE_CONTAINER, - payload.getRequest()); - } - - @Override - protected void onFinished(EventPublisher publisher, - ReplicationRequestToRepeat payload) { - - } -} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java deleted file mode 100644 index b904666222910..0000000000000 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java +++ /dev/null @@ -1,384 +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.hdds.scm.container.replication; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Comparator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.ThreadFactory; -import java.util.stream.Collectors; - -import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; -import org.apache.hadoop.hdds.scm.container.ContainerID; -import org.apache.hadoop.hdds.scm.container.ContainerInfo; -import org.apache.hadoop.hdds.scm.container.ContainerManager; -import org.apache.hadoop.hdds.scm.container.ContainerReplica; -import org.apache.hadoop.hdds.scm.container.DeleteContainerCommandWatcher; -import org.apache.hadoop.hdds.scm.container.placement.algorithms - .ContainerPlacementPolicy; -import org.apache.hadoop.hdds.scm.events.SCMEvents; -import org.apache.hadoop.hdds.server.events.EventPublisher; -import org.apache.hadoop.hdds.server.events.EventQueue; -import org.apache.hadoop.hdds.server.events.IdentifiableEventPayload; -import org.apache.hadoop.ozone.lease.LeaseManager; -import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode; -import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand; -import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.util.concurrent.ThreadFactoryBuilder; - -import static org.apache.hadoop.hdds.scm.events.SCMEvents - .TRACK_DELETE_CONTAINER_COMMAND; -import static org.apache.hadoop.hdds.scm.events.SCMEvents - .TRACK_REPLICATE_COMMAND; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Replication Manager manages the replication of the closed container. - */ -public class ReplicationManager implements Runnable { - - private static final Logger LOG = - LoggerFactory.getLogger(ReplicationManager.class); - - private ReplicationQueue replicationQueue; - - private ContainerPlacementPolicy containerPlacement; - - private EventPublisher eventPublisher; - - private ReplicationCommandWatcher replicationCommandWatcher; - private DeleteContainerCommandWatcher deleteContainerCommandWatcher; - - private boolean running = true; - - private ContainerManager containerManager; - - public ReplicationManager(ContainerPlacementPolicy containerPlacement, - ContainerManager containerManager, EventQueue eventQueue, - LeaseManager commandWatcherLeaseManager) { - - this.containerPlacement = containerPlacement; - this.containerManager = containerManager; - this.eventPublisher = eventQueue; - - this.replicationCommandWatcher = - new ReplicationCommandWatcher(TRACK_REPLICATE_COMMAND, - SCMEvents.REPLICATION_COMPLETE, commandWatcherLeaseManager); - - this.deleteContainerCommandWatcher = - new DeleteContainerCommandWatcher(TRACK_DELETE_CONTAINER_COMMAND, - SCMEvents.DELETE_CONTAINER_COMMAND_COMPLETE, - commandWatcherLeaseManager); - - this.replicationQueue = new ReplicationQueue(); - - eventQueue.addHandler(SCMEvents.REPLICATE_CONTAINER, - (replicationRequest, publisher) -> replicationQueue - .add(replicationRequest)); - - this.replicationCommandWatcher.start(eventQueue); - - } - - public void start() { - - ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat("Replication Manager").build(); - - threadFactory.newThread(this).start(); - } - - @Override - public void run() { - - while (running) { - ReplicationRequest request = null; - try { - //TODO: add throttling here - request = replicationQueue.take(); - - ContainerID containerID = new ContainerID(request.getContainerId()); - ContainerInfo container = containerManager.getContainer(containerID); - final HddsProtos.LifeCycleState state = container.getState(); - - if (state != LifeCycleState.CLOSED && - state != LifeCycleState.QUASI_CLOSED) { - LOG.warn("Cannot replicate the container {} when in {} state.", - containerID, state); - continue; - } - - //check the current replication - List containerReplicas = - new ArrayList<>(getCurrentReplicas(request)); - - if (containerReplicas.size() == 0) { - LOG.warn( - "Container {} should be replicated but can't find any existing " - + "replicas", - containerID); - return; - } - - final ReplicationRequest finalRequest = request; - - int inFlightReplications = replicationCommandWatcher.getTimeoutEvents( - e -> e.getRequest().getContainerId() - == finalRequest.getContainerId()) - .size(); - - int inFlightDelete = deleteContainerCommandWatcher.getTimeoutEvents( - e -> e.getRequest().getContainerId() - == finalRequest.getContainerId()) - .size(); - - int deficit = - (request.getExpecReplicationCount() - containerReplicas.size()) - - (inFlightReplications - inFlightDelete); - - if (deficit > 0) { - - List datanodes = containerReplicas.stream() - .sorted((r1, r2) -> - r2.getSequenceId().compareTo(r1.getSequenceId())) - .map(ContainerReplica::getDatanodeDetails) - .collect(Collectors.toList()); - List selectedDatanodes = containerPlacement - .chooseDatanodes(datanodes, deficit, container.getUsedBytes()); - - //send the command - for (DatanodeDetails datanode : selectedDatanodes) { - - LOG.info("Container {} is under replicated." + - " Expected replica count is {}, but found {}." + - " Re-replicating it on {}.", - container.containerID(), request.getExpecReplicationCount(), - containerReplicas.size(), datanode); - - ReplicateContainerCommand replicateCommand = - new ReplicateContainerCommand(containerID.getId(), datanodes); - - eventPublisher.fireEvent(SCMEvents.DATANODE_COMMAND, - new CommandForDatanode<>( - datanode.getUuid(), replicateCommand)); - - ReplicationRequestToRepeat timeoutEvent = - new ReplicationRequestToRepeat(replicateCommand.getId(), - request); - - eventPublisher.fireEvent(TRACK_REPLICATE_COMMAND, timeoutEvent); - - } - - } else if (deficit < 0) { - - int numberOfReplicasToDelete = Math.abs(deficit); - - final Map> originIdToDnMap = - new LinkedHashMap<>(); - - containerReplicas.stream() - .sorted(Comparator.comparing(ContainerReplica::getSequenceId)) - .forEach(replica -> { - originIdToDnMap.computeIfAbsent( - replica.getOriginDatanodeId(), key -> new ArrayList<>()); - originIdToDnMap.get(replica.getOriginDatanodeId()) - .add(replica.getDatanodeDetails()); - }); - - for (List listOfReplica : originIdToDnMap.values()) { - if (listOfReplica.size() > 1) { - final int toDelete = Math.min(listOfReplica.size() - 1, - numberOfReplicasToDelete); - final DeleteContainerCommand deleteContainer = - new DeleteContainerCommand(containerID.getId(), true); - for (int i = 0; i < toDelete; i++) { - LOG.info("Container {} is over replicated." + - " Expected replica count is {}, but found {}." + - " Deleting the replica on {}.", - container.containerID(), request.getExpecReplicationCount(), - containerReplicas.size(), listOfReplica.get(i)); - eventPublisher.fireEvent(SCMEvents.DATANODE_COMMAND, - new CommandForDatanode<>(listOfReplica.get(i).getUuid(), - deleteContainer)); - DeletionRequestToRepeat timeoutEvent = - new DeletionRequestToRepeat(deleteContainer.getId(), - request); - - eventPublisher.fireEvent( - TRACK_DELETE_CONTAINER_COMMAND, timeoutEvent); - } - numberOfReplicasToDelete -= toDelete; - } - if (numberOfReplicasToDelete == 0) { - break; - } - } - - if (numberOfReplicasToDelete != 0) { - final int expectedReplicaCount = container - .getReplicationFactor().getNumber(); - - LOG.warn("Not able to delete the container replica of Container" + - " {} even though it is over replicated. Expected replica" + - " count is {}, current replica count is {}.", - containerID, expectedReplicaCount, - expectedReplicaCount + numberOfReplicasToDelete); - } - } - - } catch (Exception e) { - LOG.error("Can't replicate container {}", request, e); - } - } - - } - - @VisibleForTesting - protected Set getCurrentReplicas(ReplicationRequest request) - throws IOException { - return containerManager - .getContainerReplicas(new ContainerID(request.getContainerId())); - } - - @VisibleForTesting - public ReplicationQueue getReplicationQueue() { - return replicationQueue; - } - - public void stop() { - running = false; - } - - /** - * Event for the ReplicationCommandWatcher to repeat the embedded request. - * in case fof timeout. - */ - public static class ReplicationRequestToRepeat - extends ContainerRequestToRepeat { - - public ReplicationRequestToRepeat( - long commandId, ReplicationRequest request) { - super(commandId, request); - } - } - - /** - * Event for the DeleteContainerCommandWatcher to repeat the - * embedded request. In case fof timeout. - */ - public static class DeletionRequestToRepeat - extends ContainerRequestToRepeat { - - public DeletionRequestToRepeat( - long commandId, ReplicationRequest request) { - super(commandId, request); - } - } - - /** - * Container Request wrapper which will be used by ReplicationManager to - * perform the intended operation. - */ - public static class ContainerRequestToRepeat - implements IdentifiableEventPayload { - - private final long commandId; - - private final ReplicationRequest request; - - ContainerRequestToRepeat(long commandId, - ReplicationRequest request) { - this.commandId = commandId; - this.request = request; - } - - public ReplicationRequest getRequest() { - return request; - } - - @Override - public long getId() { - return commandId; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ContainerRequestToRepeat that = (ContainerRequestToRepeat) o; - return Objects.equals(request, that.request); - } - - @Override - public int hashCode() { - - return Objects.hash(request); - } - } - - /** - * Event which indicates that the replicate operation is completed. - */ - public static class ReplicationCompleted - implements IdentifiableEventPayload { - - private final long uuid; - - public ReplicationCompleted(long uuid) { - this.uuid = uuid; - } - - @Override - public long getId() { - return uuid; - } - } - - /** - * Event which indicates that the container deletion operation is completed. - */ - public static class DeleteContainerCommandCompleted - implements IdentifiableEventPayload { - - private final long uuid; - - public DeleteContainerCommandCompleted(long uuid) { - this.uuid = uuid; - } - - @Override - public long getId() { - return uuid; - } - } -} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationQueue.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationQueue.java deleted file mode 100644 index 4ca67be4e1181..0000000000000 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationQueue.java +++ /dev/null @@ -1,73 +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.hdds.scm.container.replication; - -import java.util.List; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.PriorityBlockingQueue; - -/** - * Priority queue to handle under-replicated and over replicated containers - * in ozone. ReplicationManager will consume these messages and decide - * accordingly. - */ -public class ReplicationQueue { - - private final BlockingQueue queue; - - public ReplicationQueue() { - queue = new PriorityBlockingQueue<>(); - } - - public boolean add(ReplicationRequest repObj) { - if (this.queue.contains(repObj)) { - // Remove the earlier message and insert this one - this.queue.remove(repObj); - } - return this.queue.add(repObj); - } - - public boolean remove(ReplicationRequest repObj) { - return queue.remove(repObj); - } - - /** - * Retrieves, but does not remove, the head of this queue, - * or returns {@code null} if this queue is empty. - * - * @return the head of this queue, or {@code null} if this queue is empty - */ - public ReplicationRequest peek() { - return queue.peek(); - } - - /** - * Retrieves and removes the head of this queue (blocking queue). - */ - public ReplicationRequest take() throws InterruptedException { - return queue.take(); - } - - public boolean removeAll(List repObjs) { - return queue.removeAll(repObjs); - } - - public int size() { - return queue.size(); - } -} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationRequest.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationRequest.java deleted file mode 100644 index d40cd9cd17dfa..0000000000000 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationRequest.java +++ /dev/null @@ -1,123 +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.hdds.scm.container.replication; - -import java.io.Serializable; - -import org.apache.commons.lang3.builder.EqualsBuilder; -import org.apache.commons.lang3.builder.HashCodeBuilder; - -/** - * Wrapper class for hdds replication queue. Implements its natural - * ordering for priority queue. - */ -public class ReplicationRequest implements Comparable, - Serializable { - private final long containerId; - private final int replicationCount; - private final int expecReplicationCount; - private final long timestamp; - - public ReplicationRequest(long containerId, int replicationCount, - long timestamp, int expecReplicationCount) { - this.containerId = containerId; - this.replicationCount = replicationCount; - this.timestamp = timestamp; - this.expecReplicationCount = expecReplicationCount; - } - - public ReplicationRequest(long containerId, int replicationCount, - int expecReplicationCount) { - this(containerId, replicationCount, System.currentTimeMillis(), - expecReplicationCount); - } - - /** - * Compares this object with the specified object for order. Returns a - * negative integer, zero, or a positive integer as this object is less - * than, equal to, or greater than the specified object. - * @param o the object to be compared. - * @return a negative integer, zero, or a positive integer as this object - * is less than, equal to, or greater than the specified object. - * @throws NullPointerException if the specified object is null - * @throws ClassCastException if the specified object's type prevents it - * from being compared to this object. - */ - @Override - public int compareTo(ReplicationRequest o) { - if (o == null) { - return 1; - } - if (this == o) { - return 0; - } - int retVal = Integer - .compare(getReplicationCount() - getExpecReplicationCount(), - o.getReplicationCount() - o.getExpecReplicationCount()); - if (retVal != 0) { - return retVal; - } - return Long.compare(getTimestamp(), o.getTimestamp()); - } - - @Override - public int hashCode() { - return new HashCodeBuilder(91, 1011) - .append(getContainerId()) - .toHashCode(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ReplicationRequest that = (ReplicationRequest) o; - return new EqualsBuilder().append(getContainerId(), that.getContainerId()) - .isEquals(); - } - - public long getContainerId() { - return containerId; - } - - public int getReplicationCount() { - return replicationCount; - } - - public long getTimestamp() { - return timestamp; - } - - public int getExpecReplicationCount() { - return expecReplicationCount; - } - - @Override - public String toString() { - return "ReplicationRequest{" + - "containerId=" + containerId + - ", replicationCount=" + replicationCount + - ", expecReplicationCount=" + expecReplicationCount + - ", timestamp=" + timestamp + - '}'; - } -} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java index ed9727a8a8bfb..43d396e0cb127 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java @@ -23,8 +23,6 @@ import org.apache.hadoop.hdds.scm.block.PendingDeleteStatusList; import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager.SafeModeStatus; import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler; -import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler - .ReplicationStatus; import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher .IncrementalContainerReportFromDatanode; @@ -40,14 +38,8 @@ .ContainerReportFromDatanode; import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher .NodeReportFromDatanode; -import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager; -import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager - .DeleteContainerCommandCompleted; -import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager - .ReplicationCompleted; -import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest; - -import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer.NodeRegistrationContainerReport; +import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer + .NodeRegistrationContainerReport; import org.apache.hadoop.hdds.server.events.Event; import org.apache.hadoop.hdds.server.events.TypedEvent; import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode; @@ -184,12 +176,6 @@ public final class SCMEvents { public static final TypedEvent NON_HEALTHY_TO_HEALTHY_NODE = new TypedEvent<>(DatanodeDetails.class, "NON_HEALTHY_TO_HEALTHY_NODE"); - /** - * This event will be triggered by CommandStatusReportHandler whenever a - * status for Replication SCMCommand is received. - */ - public static final Event REPLICATION_STATUS = new - TypedEvent<>(ReplicationStatus.class, "Replicate_Command_Status"); /** * This event will be triggered by CommandStatusReportHandler whenever a * status for DeleteBlock SCMCommand is received. @@ -207,53 +193,6 @@ public final class SCMEvents { public static final Event PENDING_DELETE_STATUS = new TypedEvent<>(PendingDeleteStatusList.class, "Pending_Delete_Status"); - /** - * This is the command for ReplicationManager to handle under/over - * replication. Sent by the ContainerReportHandler after processing the - * heartbeat. - */ - public static final TypedEvent REPLICATE_CONTAINER = - new TypedEvent<>(ReplicationRequest.class); - - /** - * This event is sent by the ReplicaManager to the - * ReplicationCommandWatcher to track the in-progress replication. - */ - public static final TypedEvent - TRACK_REPLICATE_COMMAND = - new TypedEvent<>(ReplicationManager.ReplicationRequestToRepeat.class); - - /** - * This event is sent by the ReplicaManager to the - * DeleteContainerCommandWatcher to track the in-progress delete commands. - */ - public static final TypedEvent - TRACK_DELETE_CONTAINER_COMMAND = - new TypedEvent<>(ReplicationManager.DeletionRequestToRepeat.class); - /** - * This event comes from the Heartbeat dispatcher (in fact from the - * datanode) to notify the scm that the replication is done. This is - * received by the replicate command watcher to mark in-progress task as - * finished. -

- * TODO: Temporary event, should be replaced by specific Heartbeat - * ActionRequred event. - */ - public static final TypedEvent REPLICATION_COMPLETE = - new TypedEvent<>(ReplicationCompleted.class); - - public static final TypedEvent - DELETE_CONTAINER_COMMAND_COMPLETE = - new TypedEvent<>(DeleteContainerCommandCompleted.class); - - /** - * Signal for all the components (but especially for the replication - * manager and container report handler) that the replication could be - * started. Should be send only if (almost) all the container state are - * available from the datanodes. - */ - public static final TypedEvent START_REPLICATION = - new TypedEvent<>(Boolean.class); public static final TypedEvent SAFE_MODE_STATUS = new TypedEvent<>(SafeModeStatus.class); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java index a75a51a60f572..17e1fedd95251 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java @@ -18,121 +18,155 @@ package org.apache.hadoop.hdds.scm.node; -import java.util.Set; +import java.io.IOException; +import java.util.Optional; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.container.ContainerException; -import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.ContainerManager; import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; -import org.apache.hadoop.hdds.scm.container.ContainerReplica; -import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest; -import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; +import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; +import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException; import org.apache.hadoop.hdds.server.events.EventHandler; import org.apache.hadoop.hdds.server.events.EventPublisher; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.hdds.scm.events.SCMEvents.CLOSE_CONTAINER; + /** * Handles Dead Node event. */ public class DeadNodeHandler implements EventHandler { - private final ContainerManager containerManager; - private final NodeManager nodeManager; + private final PipelineManager pipelineManager; + private final ContainerManager containerManager; private static final Logger LOG = LoggerFactory.getLogger(DeadNodeHandler.class); - public DeadNodeHandler(NodeManager nodeManager, - ContainerManager containerManager) { - this.containerManager = containerManager; + public DeadNodeHandler(final NodeManager nodeManager, + final PipelineManager pipelineManager, + final ContainerManager containerManager) { this.nodeManager = nodeManager; + this.pipelineManager = pipelineManager; + this.containerManager = containerManager; } @Override - public void onMessage(DatanodeDetails datanodeDetails, - EventPublisher publisher) { + public void onMessage(final DatanodeDetails datanodeDetails, + final EventPublisher publisher) { - // TODO: check if there are any pipeline on this node and fire close - // pipeline event - Set ids = - null; try { - ids = nodeManager.getContainers(datanodeDetails); - } catch (NodeNotFoundException e) { + + /* + * We should have already destroyed all the pipelines on this datanode + * when it was marked as stale. Destroy pipeline should also have closed + * all the containers on this datanode. + * + * Ideally we should not have any pipeline or OPEN containers now. + * + * To be on a safer side, we double check here and take appropriate + * action. + */ + + destroyPipelines(datanodeDetails); + closeContainers(datanodeDetails, publisher); + + // Remove the container replicas associated with the dead node. + removeContainerReplicas(datanodeDetails); + + } catch (NodeNotFoundException ex) { // This should not happen, we cannot get a dead node event for an - // unregistered node! + // unregistered datanode! LOG.error("DeadNode event for a unregistered node: {}!", datanodeDetails); } - if (ids == null) { - LOG.info("There's no containers in dead datanode {}, no replica will be" - + " removed from the in-memory state.", datanodeDetails.getUuid()); - return; - } - LOG.info("Datanode {} is dead. Removing replications from the in-memory" + - " state.", datanodeDetails.getUuid()); - for (ContainerID id : ids) { - try { - final ContainerInfo container = containerManager.getContainer(id); - // TODO: For open containers, trigger close on other nodes - if (!container.isOpen()) { - Set replicas = containerManager - .getContainerReplicas(id); - replicas.stream() - .filter(r -> r.getDatanodeDetails().equals(datanodeDetails)) - .findFirst() - .ifPresent(replica -> { - try { - containerManager.removeContainerReplica(id, replica); - ContainerInfo containerInfo = - containerManager.getContainer(id); - replicateIfNeeded(containerInfo, publisher); - } catch (ContainerException ex) { - LOG.warn("Exception while removing container replica #{} " + - "for container #{}.", replica, container, ex); - } - }); - } - } catch (ContainerNotFoundException cnfe) { - LOG.warn("Container Not found!", cnfe); - } - } } /** - * Compare the existing replication number with the expected one. + * Destroys all the pipelines on the given datanode if there are any. + * + * @param datanodeDetails DatanodeDetails */ - private void replicateIfNeeded(ContainerInfo container, - EventPublisher publisher) throws ContainerNotFoundException { - // Replicate only closed and Quasi closed containers - if (container.getState() == HddsProtos.LifeCycleState.CLOSED || - container.getState() == HddsProtos.LifeCycleState.QUASI_CLOSED) { - final int existingReplicas = containerManager - .getContainerReplicas(container.containerID()).size(); - final int expectedReplicas = container.getReplicationFactor().getNumber(); - if (existingReplicas != expectedReplicas) { - LOG.debug("Replicate Request fired for container {}, exisiting " + - "replica count {}, expected replica count {}", - container.getContainerID(), existingReplicas, expectedReplicas); - publisher.fireEvent(SCMEvents.REPLICATE_CONTAINER, - new ReplicationRequest( - container.getContainerID(), existingReplicas, - expectedReplicas)); - } - } + private void destroyPipelines(final DatanodeDetails datanodeDetails) { + Optional.ofNullable(nodeManager.getPipelines(datanodeDetails)) + .ifPresent(pipelines -> + pipelines.forEach(id -> { + try { + pipelineManager.finalizeAndDestroyPipeline( + pipelineManager.getPipeline(id), false); + } catch (PipelineNotFoundException ignore) { + // Pipeline is not there in pipeline manager, + // should we care? + } catch (IOException ex) { + LOG.warn("Exception while finalizing pipeline {}", + id, ex); + } + })); + } + + /** + * Sends CloseContainerCommand to all the open containers on the + * given datanode. + * + * @param datanodeDetails DatanodeDetails + * @param publisher EventPublisher + * @throws NodeNotFoundException + */ + private void closeContainers(final DatanodeDetails datanodeDetails, + final EventPublisher publisher) + throws NodeNotFoundException { + nodeManager.getContainers(datanodeDetails) + .forEach(id -> { + try { + final ContainerInfo container = containerManager.getContainer(id); + if (container.getState() == HddsProtos.LifeCycleState.OPEN) { + publisher.fireEvent(CLOSE_CONTAINER, id); + } + } catch (ContainerNotFoundException cnfe) { + LOG.warn("Container {} is not managed by ContainerManager.", + id, cnfe); + } + }); } /** - * Returns logger. - * */ - // TODO: remove this. - public static Logger getLogger() { - return LOG; + * Removes the ContainerReplica of the dead datanode from the containers + * which are hosted by that datanode. + * + * @param datanodeDetails DatanodeDetails + * @throws NodeNotFoundException + */ + private void removeContainerReplicas(final DatanodeDetails datanodeDetails) + throws NodeNotFoundException { + nodeManager.getContainers(datanodeDetails) + .forEach(id -> { + try { + final ContainerInfo container = containerManager.getContainer(id); + // Identify and remove the ContainerReplica of dead node + containerManager.getContainerReplicas(id) + .stream() + .filter(r -> r.getDatanodeDetails().equals(datanodeDetails)) + .findFirst() + .ifPresent(replica -> { + try { + containerManager.removeContainerReplica(id, replica); + } catch (ContainerException ex) { + LOG.warn("Exception while removing container replica #{} " + + "of container {}.", replica, container, ex); + } + }); + } catch (ContainerNotFoundException cnfe) { + LOG.warn("Container {} is not managed by ContainerManager.", + id, cnfe); + } + }); } + + } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java index 17f72f6798901..2ab7295cd8091 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hdds.scm.node; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.hadoop.hdds.protocol.proto .StorageContainerDatanodeProtocolProtos; @@ -83,8 +82,7 @@ */ public class SCMNodeManager implements NodeManager { - @VisibleForTesting - static final Logger LOG = + private static final Logger LOG = LoggerFactory.getLogger(SCMNodeManager.class); private final NodeStateManager nodeStateManager; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java index 100534a0a2921..270d356a901aa 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java @@ -297,7 +297,7 @@ public StorageContainerManager(OzoneConfiguration conf, StaleNodeHandler staleNodeHandler = new StaleNodeHandler(scmNodeManager, pipelineManager, conf); DeadNodeHandler deadNodeHandler = new DeadNodeHandler(scmNodeManager, - containerManager); + pipelineManager, containerManager); NonHealthyToHealthyNodeHandler nonHealthyToHealthyNodeHandler = new NonHealthyToHealthyNodeHandler(pipelineManager, conf); ContainerActionsHandler actionsHandler = new ContainerActionsHandler(); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java index afa7fcbfde844..6a98a346f73c4 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java @@ -37,8 +37,6 @@ import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.scm.server.SCMConfigurator; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; -import org.apache.hadoop.hdds.server.events.EventHandler; -import org.apache.hadoop.hdds.server.events.EventPublisher; import org.apache.hadoop.hdds.server.events.EventQueue; import org.apache.hadoop.ozone.container.common.SCMTestUtils; import org.apache.hadoop.test.GenericTestUtils; @@ -57,7 +55,7 @@ /** * Tests for SCM Block Manager. */ -public class TestBlockManager implements EventHandler { +public class TestBlockManager { private StorageContainerManager scm; private SCMContainerManager mapping; private MockNodeManager nodeManager; @@ -103,7 +101,8 @@ public void setUp() throws Exception { eventQueue = new EventQueue(); eventQueue.addHandler(SCMEvents.SAFE_MODE_STATUS, scm.getSafeModeHandler()); - eventQueue.addHandler(SCMEvents.START_REPLICATION, this); + eventQueue.addHandler(SCMEvents.SAFE_MODE_STATUS, + scm.getSafeModeHandler()); CloseContainerEventHandler closeContainerHandler = new CloseContainerEventHandler(pipelineManager, mapping); eventQueue.addHandler(SCMEvents.CLOSE_CONTAINER, closeContainerHandler); @@ -282,8 +281,4 @@ public void testBlockAllocationWithNoAvailablePipelines() Assert.assertEquals(1, pipelineManager.getPipelines(type, factor).size()); } - @Override - public void onMessage(Boolean aBoolean, EventPublisher publisher) { - System.out.println("test"); - } } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/command/TestCommandStatusReportHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/command/TestCommandStatusReportHandler.java index 9fea7db606153..8877b2b5c4e8c 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/command/TestCommandStatusReportHandler.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/command/TestCommandStatusReportHandler.java @@ -74,13 +74,8 @@ public void testCommandStatusReport() { cmdStatusReportHandler.onMessage(report, this); assertTrue(logCapturer.getOutput().contains("firing event of type " + "Delete_Block_Status")); - assertTrue(logCapturer.getOutput().contains("firing event of type " + - "Replicate_Command_Status")); - assertTrue(logCapturer.getOutput().contains("type: " + "deleteBlocksCommand")); - assertTrue(logCapturer.getOutput().contains("type: " + - "replicateContainerCommand")); } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java deleted file mode 100644 index fbe26411492aa..0000000000000 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java +++ /dev/null @@ -1,290 +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.hdds.scm.container.replication; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.stream.IntStream; - -import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.ContainerReplicaProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.ReplicateContainerCommandProto; -import org.apache.hadoop.hdds.scm.TestUtils; -import org.apache.hadoop.hdds.scm.container.ContainerID; -import org.apache.hadoop.hdds.scm.container.ContainerManager; -import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; -import org.apache.hadoop.hdds.scm.container.ContainerReplica; -import org.apache.hadoop.hdds.scm.container.ContainerInfo; -import org.apache.hadoop.hdds.scm.container.placement.algorithms - .ContainerPlacementPolicy; -import org.apache.hadoop.hdds.scm.container.replication - .ReplicationManager.ReplicationRequestToRepeat; -import org.apache.hadoop.hdds.scm.container.replication - .ReplicationManager.DeletionRequestToRepeat; -import org.apache.hadoop.hdds.scm.events.SCMEvents; -import org.apache.hadoop.hdds.server.events.EventQueue; -import org.apache.hadoop.ozone.lease.LeaseManager; -import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode; - -import static org.apache.hadoop.hdds.scm.events.SCMEvents - .TRACK_DELETE_CONTAINER_COMMAND; -import static org.apache.hadoop.hdds.scm.events.SCMEvents - .TRACK_REPLICATE_COMMAND; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import static org.mockito.Matchers.anyObject; -import org.mockito.Mockito; -import static org.mockito.Mockito.when; - -/** - * Test behaviour of the TestReplication. - */ -public class TestReplicationManager { - - private EventQueue queue; - - private List trackReplicationEvents; - private List trackDeleteEvents; - - private List> copyEvents; - - private ContainerManager containerManager; - - private ContainerPlacementPolicy containerPlacementPolicy; - private List listOfDatanodeDetails; - private List listOfContainerReplica; - private LeaseManager leaseManager; - private ReplicationManager replicationManager; - - @Before - public void initReplicationManager() throws IOException { - - listOfDatanodeDetails = new ArrayList<>(); - listOfContainerReplica = new ArrayList<>(); - IntStream.range(1, 6).forEach(i -> { - DatanodeDetails dd = TestUtils.randomDatanodeDetails(); - listOfDatanodeDetails.add(dd); - listOfContainerReplica.add(ContainerReplica.newBuilder() - .setContainerID(ContainerID.valueof(i)) - .setContainerState(ContainerReplicaProto.State.CLOSED) - .setSequenceId(10000L) - .setOriginNodeId(dd.getUuid()) - .setDatanodeDetails(dd).build()); - }); - - containerPlacementPolicy = - (excludedNodes, nodesRequired, sizeRequired) -> listOfDatanodeDetails - .subList(2, 2 + nodesRequired); - - containerManager = Mockito.mock(ContainerManager.class); - - ContainerInfo containerInfo = new ContainerInfo.Builder() - .setState(LifeCycleState.CLOSED) - .build(); - - when(containerManager.getContainer(anyObject())) - .thenReturn(containerInfo); - - when(containerManager.getContainerReplicas(new ContainerID(1L))) - .thenReturn(new HashSet<>(Arrays.asList( - listOfContainerReplica.get(0), - listOfContainerReplica.get(1) - ))); - - - when(containerManager.getContainerReplicas(new ContainerID(3L))) - .thenReturn(new HashSet<>()); - - queue = new EventQueue(); - - trackReplicationEvents = new ArrayList<>(); - queue.addHandler(TRACK_REPLICATE_COMMAND, - (event, publisher) -> trackReplicationEvents.add(event)); - - trackDeleteEvents = new ArrayList<>(); - queue.addHandler(TRACK_DELETE_CONTAINER_COMMAND, - (event, publisher) -> trackDeleteEvents.add(event)); - - copyEvents = new ArrayList<>(); - queue.addHandler(SCMEvents.DATANODE_COMMAND, - (event, publisher) -> copyEvents.add(event)); - - leaseManager = new LeaseManager<>("Test", 100000L); - - replicationManager = new ReplicationManager(containerPlacementPolicy, - containerManager, queue, leaseManager); - - } - - /** - * Container should be replicated but no source replicas. - */ - @Test() - public void testNoExistingReplicas() throws InterruptedException { - try { - leaseManager.start(); - replicationManager.start(); - - //WHEN - queue.fireEvent(SCMEvents.REPLICATE_CONTAINER, - new ReplicationRequest(3L, (short) 2, System.currentTimeMillis(), - (short) 3)); - - Thread.sleep(500L); - queue.processAll(1000L); - - //THEN - Assert.assertEquals(0, trackReplicationEvents.size()); - Assert.assertEquals(0, copyEvents.size()); - - } finally { - if (leaseManager != null) { - leaseManager.shutdown(); - } - } - } - - @Test - public void testOverReplication() throws ContainerNotFoundException, - InterruptedException { - try { - leaseManager.start(); - replicationManager.start(); - - final ContainerID containerID = ContainerID.valueof(5L); - - final ContainerReplica duplicateReplicaOne = ContainerReplica.newBuilder() - .setContainerID(containerID) - .setContainerState(ContainerReplicaProto.State.CLOSED) - .setSequenceId(10000L) - .setOriginNodeId(listOfDatanodeDetails.get(0).getUuid()) - .setDatanodeDetails(listOfDatanodeDetails.get(3)) - .build(); - - final ContainerReplica duplicateReplicaTwo = ContainerReplica.newBuilder() - .setContainerID(containerID) - .setContainerState(ContainerReplicaProto.State.CLOSED) - .setSequenceId(10000L) - .setOriginNodeId(listOfDatanodeDetails.get(1).getUuid()) - .setDatanodeDetails(listOfDatanodeDetails.get(4)) - .build(); - - when(containerManager.getContainerReplicas(new ContainerID(5L))) - .thenReturn(new HashSet<>(Arrays.asList( - listOfContainerReplica.get(0), - listOfContainerReplica.get(1), - listOfContainerReplica.get(2), - duplicateReplicaOne, - duplicateReplicaTwo - ))); - - queue.fireEvent(SCMEvents.REPLICATE_CONTAINER, - new ReplicationRequest(5L, (short) 5, System.currentTimeMillis(), - (short) 3)); - Thread.sleep(500L); - queue.processAll(1000L); - - //THEN - Assert.assertEquals(2, trackDeleteEvents.size()); - Assert.assertEquals(2, copyEvents.size()); - - } finally { - if (leaseManager != null) { - leaseManager.shutdown(); - } - } - } - - @Test - public void testEventSending() throws InterruptedException, IOException { - - //GIVEN - try { - leaseManager.start(); - - replicationManager.start(); - - //WHEN - queue.fireEvent(SCMEvents.REPLICATE_CONTAINER, - new ReplicationRequest(1L, (short) 2, System.currentTimeMillis(), - (short) 3)); - - Thread.sleep(500L); - queue.processAll(1000L); - - //THEN - Assert.assertEquals(1, trackReplicationEvents.size()); - Assert.assertEquals(1, copyEvents.size()); - } finally { - if (leaseManager != null) { - leaseManager.shutdown(); - } - } - } - - @Test - public void testCommandWatcher() throws InterruptedException, IOException { - LeaseManager rapidLeaseManager = - new LeaseManager<>("Test", 1000L); - - replicationManager = new ReplicationManager(containerPlacementPolicy, - containerManager, queue, rapidLeaseManager); - - try { - leaseManager.start(); - rapidLeaseManager.start(); - replicationManager.start(); - - queue.fireEvent(SCMEvents.REPLICATE_CONTAINER, - new ReplicationRequest(1L, (short) 2, System.currentTimeMillis(), - (short) 3)); - - Thread.sleep(500L); - - queue.processAll(1000L); - - Assert.assertEquals(1, trackReplicationEvents.size()); - Assert.assertEquals(1, copyEvents.size()); - - Assert.assertEquals(trackReplicationEvents.get(0).getId(), - copyEvents.get(0).getCommand().getId()); - - //event is timed out - Thread.sleep(1500); - - queue.processAll(1000L); - - //original copy command + retry - Assert.assertEquals(2, trackReplicationEvents.size()); - Assert.assertEquals(2, copyEvents.size()); - - } finally { - rapidLeaseManager.shutdown(); - if (leaseManager != null) { - leaseManager.shutdown(); - } - } - } - -} \ No newline at end of file diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationQueue.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationQueue.java deleted file mode 100644 index 9dd4fe31c50fb..0000000000000 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationQueue.java +++ /dev/null @@ -1,134 +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.hdds.scm.container.replication; - -import java.util.Random; -import java.util.UUID; -import org.apache.hadoop.util.Time; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -/** - * Test class for ReplicationQueue. - */ -public class TestReplicationQueue { - - private ReplicationQueue replicationQueue; - private Random random; - - @Before - public void setUp() { - replicationQueue = new ReplicationQueue(); - random = new Random(); - } - - @Test - public void testDuplicateAddOp() throws InterruptedException { - long contId = random.nextLong(); - String nodeId = UUID.randomUUID().toString(); - ReplicationRequest obj1, obj2, obj3; - long time = Time.monotonicNow(); - obj1 = new ReplicationRequest(contId, (short) 2, time, (short) 3); - obj2 = new ReplicationRequest(contId, (short) 2, time + 1, (short) 3); - obj3 = new ReplicationRequest(contId, (short) 1, time+2, (short) 3); - - replicationQueue.add(obj1); - replicationQueue.add(obj2); - replicationQueue.add(obj3); - Assert.assertEquals("Should add only 1 msg as second one is duplicate", - 1, replicationQueue.size()); - ReplicationRequest temp = replicationQueue.take(); - Assert.assertEquals(temp, obj3); - } - - @Test - public void testPollOp() throws InterruptedException { - long contId = random.nextLong(); - String nodeId = UUID.randomUUID().toString(); - ReplicationRequest msg1, msg2, msg3, msg4, msg5; - msg1 = new ReplicationRequest(contId, (short) 1, Time.monotonicNow(), - (short) 3); - long time = Time.monotonicNow(); - msg2 = new ReplicationRequest(contId + 1, (short) 4, time, (short) 3); - msg3 = new ReplicationRequest(contId + 2, (short) 0, time, (short) 3); - msg4 = new ReplicationRequest(contId, (short) 2, time, (short) 3); - // Replication message for same container but different nodeId - msg5 = new ReplicationRequest(contId + 1, (short) 2, time, (short) 3); - - replicationQueue.add(msg1); - replicationQueue.add(msg2); - replicationQueue.add(msg3); - replicationQueue.add(msg4); - replicationQueue.add(msg5); - Assert.assertEquals("Should have 3 objects", - 3, replicationQueue.size()); - - // Since Priority queue orders messages according to replication count, - // message with lowest replication should be first - ReplicationRequest temp; - temp = replicationQueue.take(); - Assert.assertEquals("Should have 2 objects", - 2, replicationQueue.size()); - Assert.assertEquals(temp, msg3); - - temp = replicationQueue.take(); - Assert.assertEquals("Should have 1 objects", - 1, replicationQueue.size()); - Assert.assertEquals(temp, msg5); - - // Message 2 should be ordered before message 5 as both have same - // replication number but message 2 has earlier timestamp. - temp = replicationQueue.take(); - Assert.assertEquals("Should have 0 objects", - replicationQueue.size(), 0); - Assert.assertEquals(temp, msg4); - } - - @Test - public void testRemoveOp() { - long contId = random.nextLong(); - String nodeId = UUID.randomUUID().toString(); - ReplicationRequest obj1, obj2, obj3; - obj1 = new ReplicationRequest(contId, (short) 1, Time.monotonicNow(), - (short) 3); - obj2 = new ReplicationRequest(contId + 1, (short) 2, Time.monotonicNow(), - (short) 3); - obj3 = new ReplicationRequest(contId + 2, (short) 3, Time.monotonicNow(), - (short) 3); - - replicationQueue.add(obj1); - replicationQueue.add(obj2); - replicationQueue.add(obj3); - Assert.assertEquals("Should have 3 objects", - 3, replicationQueue.size()); - - replicationQueue.remove(obj3); - Assert.assertEquals("Should have 2 objects", - 2, replicationQueue.size()); - - replicationQueue.remove(obj2); - Assert.assertEquals("Should have 1 objects", - 1, replicationQueue.size()); - - replicationQueue.remove(obj1); - Assert.assertEquals("Should have 0 objects", - 0, replicationQueue.size()); - } - -} diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDeadNodeHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDeadNodeHandler.java index 6805210043efa..7657b54373f3e 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDeadNodeHandler.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDeadNodeHandler.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; +import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.scm.pipeline.PipelineProvider; import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager; import org.apache.hadoop.hdds.scm.pipeline.MockRatisPipelineProvider; @@ -62,7 +63,6 @@ import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; -import org.slf4j.event.Level; /** * Test DeadNodeHandler. @@ -95,7 +95,8 @@ public void setup() throws IOException, AuthenticationException { manager.setPipelineProvider(HddsProtos.ReplicationType.RATIS, mockRatisProvider); containerManager = scm.getContainerManager(); - deadNodeHandler = new DeadNodeHandler(nodeManager, containerManager); + deadNodeHandler = new DeadNodeHandler(nodeManager, + Mockito.mock(PipelineManager.class), containerManager); eventQueue.addHandler(SCMEvents.DEAD_NODE, deadNodeHandler); publisher = Mockito.mock(EventPublisher.class); nodeReportHandler = new NodeReportHandler(nodeManager); @@ -168,10 +169,6 @@ public void testOnMessage() throws IOException, NodeNotFoundException { TestUtils.closeContainer(containerManager, container2.containerID()); TestUtils.quasiCloseContainer(containerManager, container3.containerID()); - GenericTestUtils.setLogLevel(DeadNodeHandler.getLogger(), Level.DEBUG); - GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer - .captureLogs(DeadNodeHandler.getLogger()); - deadNodeHandler.onMessage(datanode1, publisher); Set container1Replicas = containerManager @@ -191,60 +188,6 @@ public void testOnMessage() throws IOException, NodeNotFoundException { Assert.assertEquals(1, container3Replicas.size()); Assert.assertEquals(datanode3, container3Replicas.iterator().next().getDatanodeDetails()); - - // Replicate should be fired for container 1 and container 2 as now - // datanode 1 is dead, these 2 will not match with expected replica count - // and their state is one of CLOSED/QUASI_CLOSE. - Assert.assertTrue(logCapturer.getOutput().contains( - "Replicate Request fired for container " + - container1.getContainerID())); - Assert.assertTrue(logCapturer.getOutput().contains( - "Replicate Request fired for container " + - container2.getContainerID())); - - // as container4 is still in open state, replicate event should not have - // fired for this. - Assert.assertFalse(logCapturer.getOutput().contains( - "Replicate Request fired for container " + - container4.getContainerID())); - - - } - - @Test - public void testOnMessageReplicaFailure() throws Exception { - - DatanodeDetails datanode1 = TestUtils.randomDatanodeDetails(); - DatanodeDetails datanode2 = TestUtils.randomDatanodeDetails(); - DatanodeDetails datanode3 = TestUtils.randomDatanodeDetails(); - - String storagePath = GenericTestUtils.getRandomizedTempPath() - .concat("/" + datanode1.getUuidString()); - - StorageReportProto storageOne = TestUtils.createStorageReport( - datanode1.getUuid(), storagePath, 100, 10, 90, null); - - nodeManager.register(datanode1, - TestUtils.createNodeReport(storageOne), null); - nodeManager.register(datanode2, - TestUtils.createNodeReport(storageOne), null); - nodeManager.register(datanode3, - TestUtils.createNodeReport(storageOne), null); - - DatanodeDetails dn1 = TestUtils.randomDatanodeDetails(); - GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer - .captureLogs(DeadNodeHandler.getLogger()); - - nodeReportHandler.onMessage(getNodeReport(dn1, storageOne), - Mockito.mock(EventPublisher.class)); - - ContainerInfo container1 = - TestUtils.allocateContainer(containerManager); - TestUtils.closeContainer(containerManager, container1.containerID()); - - deadNodeHandler.onMessage(dn1, eventQueue); - Assert.assertTrue(logCapturer.getOutput().contains( - "DeadNode event for a unregistered node")); } private void registerReplicas(ContainerManager contManager, diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestStatisticsUpdate.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestStatisticsUpdate.java index e62295f8ec19d..9bce94b2b1d82 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestStatisticsUpdate.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestStatisticsUpdate.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric; import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat; import org.apache.hadoop.hdds.scm.events.SCMEvents; +import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher .NodeReportFromDatanode; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; @@ -69,7 +70,8 @@ public void setup() throws IOException, AuthenticationException { final StorageContainerManager scm = HddsTestUtils.getScm(conf); nodeManager = scm.getScmNodeManager(); final DeadNodeHandler deadNodeHandler = new DeadNodeHandler( - nodeManager, scm.getContainerManager()); + nodeManager, Mockito.mock(PipelineManager.class), + scm.getContainerManager()); eventQueue.addHandler(SCMEvents.DEAD_NODE, deadNodeHandler); nodeReportHandler = new NodeReportHandler(nodeManager); } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java index bceec9256c622..4b03474f38d6c 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java @@ -421,15 +421,10 @@ public void testHeartbeatWithCommandStatusReport() throws Exception { serverAddress, 3000); Map map = stateContext.getCommandStatusMap(); assertNotNull(map); - assertEquals("Should have 2 objects", 2, map.size()); - assertTrue(map.containsKey(Long.valueOf(2))); - assertTrue(map.containsKey(Long.valueOf(3))); - assertTrue(map.get(Long.valueOf(2)).getType() - .equals(Type.replicateContainerCommand)); - assertTrue( - map.get(Long.valueOf(3)).getType().equals(Type.deleteBlocksCommand)); - assertTrue(map.get(Long.valueOf(2)).getStatus().equals(Status.PENDING)); - assertTrue(map.get(Long.valueOf(3)).getStatus().equals(Status.PENDING)); + assertEquals("Should have 1 objects", 1, map.size()); + assertTrue(map.containsKey(3L)); + assertEquals(Type.deleteBlocksCommand, map.get(3L).getType()); + assertEquals(Status.PENDING, map.get(3L).getStatus()); scmServerImpl.clearScmCommandRequests(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java index 0029a7643cd64..54425b1c6da88 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java @@ -1152,7 +1152,7 @@ public void run() { one = ackQueue.getFirst(); } if (one.getSeqno() != seqno) { - throw new IOException("ResponseProcessor: Expecting seqno " + + throw new IOException("ResponseProcessor: Expecting seqno" + " for block " + block + one.getSeqno() + " but received " + seqno); } @@ -1386,7 +1386,7 @@ private void addDatanode2ExistingPipeline() throws IOException { if (dfsClient.dtpReplaceDatanodeOnFailureReplication > 0 && nodes.length >= dfsClient.dtpReplaceDatanodeOnFailureReplication) { DFSClient.LOG.warn( - "Failed to find a new datanode to add to the write pipeline, " + "Failed to find a new datanode to add to the write pipeline," + " continue to write to the pipeline with " + nodes.length + " nodes since it's no less than minimum replication: " + dfsClient.dtpReplaceDatanodeOnFailureReplication diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ObserverReadProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ObserverReadProxyProvider.java index b4130d714b3fc..0df5e1e71d67e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ObserverReadProxyProvider.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ObserverReadProxyProvider.java @@ -44,6 +44,7 @@ import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RpcInvocationHandler; +import org.apache.hadoop.ipc.StandbyException; import org.apache.hadoop.util.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -67,7 +68,8 @@ @InterfaceStability.Evolving public class ObserverReadProxyProvider extends AbstractNNFailoverProxyProvider { - private static final Logger LOG = LoggerFactory.getLogger( + @VisibleForTesting + static final Logger LOG = LoggerFactory.getLogger( ObserverReadProxyProvider.class); /** Configuration key for {@link #autoMsyncPeriodMs}. */ @@ -251,20 +253,38 @@ private synchronized NNProxyInfo changeProxy(NNProxyInfo initial) { } currentIndex = (currentIndex + 1) % nameNodeProxies.size(); currentProxy = createProxyIfNeeded(nameNodeProxies.get(currentIndex)); - try { - HAServiceState state = currentProxy.proxy.getHAServiceState(); - currentProxy.setCachedState(state); - } catch (IOException e) { - LOG.info("Failed to connect to {}. Setting cached state to Standby", - currentProxy.getAddress(), e); - currentProxy.setCachedState(HAServiceState.STANDBY); - } + currentProxy.setCachedState(getHAServiceState(currentProxy)); LOG.debug("Changed current proxy from {} to {}", initial == null ? "none" : initial.proxyInfo, currentProxy.proxyInfo); return currentProxy; } + /** + * Fetch the service state from a proxy. If it is unable to be fetched, + * assume it is in standby state, but log the exception. + */ + private HAServiceState getHAServiceState(NNProxyInfo proxyInfo) { + IOException ioe; + try { + return proxyInfo.proxy.getHAServiceState(); + } catch (RemoteException re) { + // Though a Standby will allow a getHAServiceState call, it won't allow + // delegation token lookup, so if DT is used it throws StandbyException + if (re.unwrapRemoteException() instanceof StandbyException) { + LOG.debug("NameNode {} threw StandbyException when fetching HAState", + proxyInfo.getAddress()); + return HAServiceState.STANDBY; + } + ioe = re; + } catch (IOException e) { + ioe = e; + } + LOG.info("Failed to connect to {}. Assuming Standby state", + proxyInfo.getAddress(), ioe); + return HAServiceState.STANDBY; + } + /** * This will call {@link ClientProtocol#msync()} on the active NameNode * (via the {@link #failoverProxy}) to initialize the state of this client. 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 762aa6900c28c..ba80a3fe3b990 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -1724,7 +1724,7 @@ A comma-separated list of paths to use when creating file descriptors that will be shared between the DataNode and the DFSClient. Typically we use /dev/shm, so that the file descriptors will not be written to disk. - Systems that don't have /dev/shm will fall back to /tmp by default. + It tries paths in order until creation of shared memory segment succeeds. @@ -2803,18 +2803,6 @@ - - dfs.datanode.shared.file.descriptor.paths - /dev/shm,/tmp - - Comma separated paths to the directory on which - shared memory segments are created. - The client and the DataNode exchange information via - this shared memory segment. - It tries paths in order until creation of shared memory segment succeeds. - - - dfs.namenode.audit.log.debug.cmdlist diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java index d34654edf97bb..23539923402a6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java @@ -736,7 +736,7 @@ public void testTrySendErrorReportWhenNNThrowsIOException() Mockito.anyInt(), Mockito.anyString()); String errorString = "Can't send invalid block " + FAKE_BLOCK; bpos.trySendErrorReport(DatanodeProtocol.INVALID_BLOCK, errorString); - Thread.sleep(10000); + GenericTestUtils.waitFor(() -> secondCallTime != 0, 100, 20000); assertTrue("Active namenode didn't add the report back to the queue " + "when errorReport threw IOException", secondCallTime != 0); } finally { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java index f4843ac1359ff..261bf8cf6af2e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java @@ -276,24 +276,34 @@ public static void setFailoverConfigurations(MiniDFSCluster cluster, /** Sets the required configurations for performing failover. */ public static void setFailoverConfigurations(MiniDFSCluster cluster, Configuration conf, String logicalName, int nsIndex) { + setFailoverConfigurations(cluster, conf, logicalName, nsIndex, + ConfiguredFailoverProxyProvider.class); + } + + /** Sets the required configurations for performing failover. */ + public static

> void + setFailoverConfigurations(MiniDFSCluster cluster, Configuration conf, + String logicalName, int nsIndex, Class

classFPP) { MiniDFSCluster.NameNodeInfo[] nns = cluster.getNameNodeInfos(nsIndex); List nnAddresses = new ArrayList(3); for (MiniDFSCluster.NameNodeInfo nn : nns) { nnAddresses.add(nn.nameNode.getNameNodeAddress()); } - setFailoverConfigurations(conf, logicalName, nnAddresses); + setFailoverConfigurations(conf, logicalName, nnAddresses, classFPP); } public static void setFailoverConfigurations(Configuration conf, String logicalName, InetSocketAddress ... nnAddresses){ - setFailoverConfigurations(conf, logicalName, Arrays.asList(nnAddresses)); + setFailoverConfigurations(conf, logicalName, Arrays.asList(nnAddresses), + ConfiguredFailoverProxyProvider.class); } /** * Sets the required configurations for performing failover */ - public static void setFailoverConfigurations(Configuration conf, - String logicalName, List nnAddresses) { + public static

> void + setFailoverConfigurations(Configuration conf, String logicalName, + List nnAddresses, Class

classFPP) { setFailoverConfigurations(conf, logicalName, Iterables.transform(nnAddresses, new Function() { @@ -302,7 +312,7 @@ public static void setFailoverConfigurations(Configuration conf, public String apply(InetSocketAddress addr) { return "hdfs://" + addr.getHostName() + ":" + addr.getPort(); } - }), ConfiguredFailoverProxyProvider.class); + }), classFPP); } public static

> diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java index 7076ec674bf43..718d13f124ba9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java @@ -25,6 +25,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; import org.apache.hadoop.hdfs.*; import org.apache.hadoop.hdfs.protocol.HdfsConstants; @@ -47,6 +48,7 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.slf4j.event.Level; import java.io.ByteArrayInputStream; import java.io.DataInputStream; @@ -112,6 +114,50 @@ public void shutdownCluster() throws IOException { } } + /** + * Test that, when using ObserverReadProxyProvider with DT authentication, + * the ORPP gracefully handles when the Standby NN throws a StandbyException. + */ + @Test(timeout = 300000) + public void testObserverReadProxyProviderWithDT() throws Exception { + // Make the first node standby, so that the ORPP will try it first + // instead of just using and succeeding on the active + cluster.transitionToStandby(0); + cluster.transitionToActive(1); + + HATestUtil.setFailoverConfigurations(cluster, conf, + HATestUtil.getLogicalHostname(cluster), 0, + ObserverReadProxyProvider.class); + conf.setBoolean("fs.hdfs.impl.disable.cache", true); + + dfs = (DistributedFileSystem) FileSystem.get(conf); + final UserGroupInformation ugi = UserGroupInformation + .createRemoteUser("JobTracker"); + final Token token = + getDelegationToken(dfs, ugi.getShortUserName()); + ugi.addToken(token); + // Recreate the DFS, this time authenticating using a DT + dfs = ugi.doAs((PrivilegedExceptionAction) + () -> (DistributedFileSystem) FileSystem.get(conf)); + + GenericTestUtils.setLogLevel(ObserverReadProxyProvider.LOG, Level.DEBUG); + GenericTestUtils.LogCapturer logCapture = GenericTestUtils.LogCapturer + .captureLogs(ObserverReadProxyProvider.LOG); + try { + dfs.access(new Path("/"), FsAction.READ); + assertTrue(logCapture.getOutput() + .contains("threw StandbyException when fetching HAState")); + HATestUtil.isSentToAnyOfNameNodes(dfs, cluster, 1); + + cluster.shutdownNameNode(0); + logCapture.clearOutput(); + dfs.access(new Path("/"), FsAction.READ); + assertTrue(logCapture.getOutput().contains("Assuming Standby state")); + } finally { + logCapture.stopCapturing(); + } + } + @Test(timeout = 300000) public void testDelegationTokenDFSApi() throws Exception { final Token token = diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java index c1f195f14caa9..b12ef51b9fdda 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java @@ -316,8 +316,8 @@ private void handleWrite(byte[] b, int off, long len, boolean retry) BlockOutputStreamEntry current = streamEntries.get(currentStreamIndex); // length(len) will be in int range if the call is happening through - // write API of blockOutputStream. Length can be in long range if it comes - // via Exception path. + // write API of blockOutputStream. Length can be in long range if it + // comes via Exception path. int writeLen = Math.min((int) len, (int) current.getRemaining()); long currentPos = current.getWrittenDataLength(); try { @@ -619,8 +619,8 @@ private void handleFlushOrClose(StreamAction op) throws IOException { try { Collection failedServers = entry.getFailedServers(); - // failed servers can be null in case there is no data written in the - // stream + // failed servers can be null in case there is no data written in + // the stream if (failedServers != null && !failedServers.isEmpty()) { excludeList.addDatanodes(failedServers); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java index a1fd17ccd638e..463e42889a351 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java @@ -728,8 +728,9 @@ public static void waitForContainerClose(OzoneOutputStream outputStream, List containerIdList = new ArrayList<>(); for (OmKeyLocationInfo info : locationInfoList) { long id = info.getContainerID(); - if (!containerIdList.contains(id)) - containerIdList.add(id); + if (!containerIdList.contains(id)) { + containerIdList.add(id); + } } Assert.assertTrue(!containerIdList.isEmpty()); waitForContainerClose(cluster, containerIdList.toArray(new Long[0])); diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 1dc0baa000968..865e03d863be2 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -1100,6 +1100,11 @@ mockito-core 2.23.4 + + org.mockito + mockito-all + 1.8.5 + org.objenesis objenesis diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/CliConstants.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/CliConstants.java index f952aff46a87b..00190f0ad6450 100644 --- a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/CliConstants.java +++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/CliConstants.java @@ -38,6 +38,7 @@ public class CliConstants { public static final String TENSORBOARD_RESOURCES = "tensorboard_resources"; public static final String TENSORBOARD_DEFAULT_RESOURCES = "memory=4G,vcores=1"; + public static final String ARG_CONF = "conf"; public static final String WORKER_LAUNCH_CMD = "worker_launch_cmd"; public static final String SERVING_LAUNCH_CMD = "serving_launch_cmd"; @@ -57,4 +58,5 @@ public class CliConstants { public static final String PRINCIPAL = "principal"; public static final String DISTRIBUTE_KEYTAB = "distribute_keytab"; public static final String YAML_CONFIG = "f"; + public static final String INSECURE_CLUSTER = "insecure"; } diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/RunJobCli.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/RunJobCli.java index f9583c6aa3d7a..b38bddfa815e5 100644 --- a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/RunJobCli.java +++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/RunJobCli.java @@ -166,6 +166,9 @@ private Options generateOptions() { " parameter" + CliConstants.KEYTAB + " on cluster machines will be " + "used"); options.addOption("h", "help", false, "Print help"); + options.addOption("insecure", false, "Cluster is not Kerberos enabled."); + options.addOption("conf", true, + "User specified configuration, as key=val pairs."); return options; } diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/RunJobParameters.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/RunJobParameters.java index 479214431171b..e7b1e2f2beefc 100644 --- a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/RunJobParameters.java +++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/RunJobParameters.java @@ -56,9 +56,11 @@ public class RunJobParameters extends RunParameters { private boolean waitJobFinish = false; private boolean distributed = false; + private boolean securityDisabled = false; private String keytab; private String principal; private boolean distributeKeytab = false; + private List confPairs = new ArrayList<>(); @Override public void updateParameters(ParametersHolder parametersHolder, @@ -97,6 +99,10 @@ public void updateParameters(ParametersHolder parametersHolder, + "please double check."); } + if (parametersHolder.hasOption(CliConstants.INSECURE_CLUSTER)) { + setSecurityDisabled(true); + } + String kerberosKeytab = parametersHolder.getOptionValue( CliConstants.KEYTAB); String kerberosPrincipal = parametersHolder.getOptionValue( @@ -181,6 +187,9 @@ public void updateParameters(ParametersHolder parametersHolder, boolean distributeKerberosKeytab = parametersHolder.hasOption(CliConstants .DISTRIBUTE_KEYTAB); + List configPairs = parametersHolder + .getOptionValues(CliConstants.ARG_CONF); + this.setInputPath(input).setCheckpointPath(jobDir) .setNumPS(nPS).setNumWorkers(nWorkers) .setPSLaunchCmd(psLaunchCommand).setWorkerLaunchCmd(workerLaunchCmd) @@ -188,7 +197,8 @@ public void updateParameters(ParametersHolder parametersHolder, .setTensorboardEnabled(tensorboard) .setKeytab(kerberosKeytab) .setPrincipal(kerberosPrincipal) - .setDistributeKeytab(distributeKerberosKeytab); + .setDistributeKeytab(distributeKerberosKeytab) + .setConfPairs(configPairs); super.updateParameters(parametersHolder, clientContext); } @@ -329,6 +339,14 @@ public RunJobParameters setPrincipal(String kerberosPrincipal) { return this; } + public boolean isSecurityDisabled() { + return securityDisabled; + } + + public void setSecurityDisabled(boolean securityDisabled) { + this.securityDisabled = securityDisabled; + } + public boolean isDistributeKeytab() { return distributeKeytab; } @@ -339,6 +357,15 @@ public RunJobParameters setDistributeKeytab( return this; } + public List getConfPairs() { + return confPairs; + } + + public RunJobParameters setConfPairs(List confPairs) { + this.confPairs = confPairs; + return this; + } + @VisibleForTesting public static class UnderscoreConverterPropertyUtils extends PropertyUtils { @Override diff --git a/hadoop-submarine/hadoop-submarine-core/src/site/markdown/QuickStart.md b/hadoop-submarine/hadoop-submarine-core/src/site/markdown/QuickStart.md index 21db6bbadc743..071e1a8db3f6f 100644 --- a/hadoop-submarine/hadoop-submarine-core/src/site/markdown/QuickStart.md +++ b/hadoop-submarine/hadoop-submarine-core/src/site/markdown/QuickStart.md @@ -89,9 +89,9 @@ usage: job run ``` #### Notes: -When using `localization` option to make a collection of dependency Python +When using `localization` option to make a collection of dependency Python scripts available to entry python script in the container, you may also need to - set `PYTHONPATH` environment variable as below to avoid module import error +set `PYTHONPATH` environment variable as below to avoid module import error reported from `entry_script.py`. ``` diff --git a/hadoop-submarine/hadoop-submarine-tony-runtime/README.md b/hadoop-submarine/hadoop-submarine-tony-runtime/README.md new file mode 100644 index 0000000000000..988565bd1b1df --- /dev/null +++ b/hadoop-submarine/hadoop-submarine-tony-runtime/README.md @@ -0,0 +1,25 @@ + + +# Overview +[TonY](https://github.com/linkedin/TonY/) is an open source project that TonY +is a framework to natively run deep learning frameworks on Apache Hadoop. +As Submarine supports multiple runtimes, this module serves as an adaptor so + that Submarine could leverage TonY as a Runtime implementation to run +TensorFlow with 2.x version of Hadoop and Hadoop installations w/o docker +support or native service. + +Please jump to [QuickStart](src/site/markdown/QuickStart.md) to understand how +to run Submarine with TonY runtime. + \ No newline at end of file diff --git a/hadoop-submarine/hadoop-submarine-tony-runtime/pom.xml b/hadoop-submarine/hadoop-submarine-tony-runtime/pom.xml new file mode 100644 index 0000000000000..8dbda98183369 --- /dev/null +++ b/hadoop-submarine/hadoop-submarine-tony-runtime/pom.xml @@ -0,0 +1,66 @@ + + + + + hadoop-submarine + org.apache.hadoop + 0.2.0-SNAPSHOT + + 4.0.0 + + hadoop-submarine-tony-runtime + + + org.apache.hadoop + hadoop-submarine-core + 0.2.0-SNAPSHOT + compile + + + com.linkedin.tony + tony-core + 0.3.3 + + + com.linkedin.tony + tony-mini + + + com.linkedin.azkaban + az-hadoop-jobtype-plugin + + + + + junit + junit + + + org.apache.hadoop + hadoop-submarine-core + test-jar + test + 0.2.0-SNAPSHOT + + + org.mockito + mockito-core + test + + + + diff --git a/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/TonyJobMonitor.java b/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/TonyJobMonitor.java new file mode 100644 index 0000000000000..7540da8103c1c --- /dev/null +++ b/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/TonyJobMonitor.java @@ -0,0 +1,52 @@ +/** + * Licensed 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. See accompanying LICENSE file. + */ +package org.apache.hadoop.yarn.submarine.runtimes.tony; + +import com.linkedin.tony.TonyClient; +import com.linkedin.tony.client.TaskUpdateListener; +import com.linkedin.tony.rpc.TaskInfo; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.submarine.common.ClientContext; +import org.apache.hadoop.yarn.submarine.common.api.JobStatus; +import org.apache.hadoop.yarn.submarine.runtimes.common.JobMonitor; +import org.apache.hadoop.yarn.submarine.runtimes.tony.buider.JobStatusBuilder; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; + +/** + * An implementation of JobMonitor with TonY library. + */ +public class TonyJobMonitor extends JobMonitor implements TaskUpdateListener { + private Set taskInfos = new HashSet<>(); + + public TonyJobMonitor(ClientContext clientContext, TonyClient client) { + super(clientContext); + client.addListener(this); + } + + @Override + public JobStatus getTrainingJobStatus(String jobName) + throws IOException, YarnException { + JobStatus jobStatus = JobStatusBuilder.fromTaskInfoSet(taskInfos); + jobStatus.setJobName(jobName); + return jobStatus; + } + + @Override + public void onTaskInfosUpdated(Set taskInfoSet) { + this.taskInfos = taskInfoSet; + } +} diff --git a/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/TonyJobSubmitter.java b/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/TonyJobSubmitter.java new file mode 100644 index 0000000000000..bd50167e4a4a6 --- /dev/null +++ b/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/TonyJobSubmitter.java @@ -0,0 +1,97 @@ +/** + * Licensed 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. See accompanying LICENSE file. + */ + +package org.apache.hadoop.yarn.submarine.runtimes.tony; + +import com.linkedin.tony.Constants; +import com.linkedin.tony.TonyClient; +import com.linkedin.tony.client.CallbackHandler; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.submarine.client.cli.param.RunJobParameters; +import org.apache.hadoop.yarn.submarine.runtimes.common.JobSubmitter; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; + +/** + * Implementation of JobSumitter with TonY runtime. + */ +public class TonyJobSubmitter implements JobSubmitter, CallbackHandler { + + private static final Log LOG = LogFactory.getLog(TonyJobSubmitter.class); + private ApplicationId applicationId; + private TonyClient tonyClient; + + public TonyJobSubmitter() { } + public void setTonyClient(TonyClient client) { + this.tonyClient = client; + } + + @Override + public ApplicationId submitJob(RunJobParameters parameters) + throws IOException, YarnException { + LOG.info("Starting Tony runtime.."); + + File tonyFinalConfPath = File.createTempFile("temp", + Constants.TONY_FINAL_XML); + // Write user's overridden conf to an xml to be localized. + Configuration tonyConf = TonyUtils.tonyConfFromClientContext(parameters); + try (OutputStream os = new FileOutputStream(tonyFinalConfPath)) { + tonyConf.writeXml(os); + } catch (IOException e) { + throw new RuntimeException("Failed to create " + tonyFinalConfPath + + " conf file. Exiting.", e); + } + + try { + tonyClient.init(new String[]{ + "--conf_file", tonyFinalConfPath.getAbsolutePath() + }); + } catch (Exception e) { + LOG.error("Failed to init TonyClient: ", e); + } + Thread clientThread = new Thread(tonyClient::start); + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + try { + tonyClient.forceKillApplication(); + } catch (YarnException | IOException e) { + LOG.error("Failed to kill application during shutdown.", e); + } + })); + clientThread.start(); + while (clientThread.isAlive()) { + if (applicationId != null) { + LOG.info("TonyClient returned applicationId: " + applicationId); + return applicationId; + } + try { + Thread.sleep(5000); + } catch (InterruptedException e) { + LOG.error(e); + } + } + return null; + } + + @Override + public void onApplicationIdReceived(ApplicationId appId) { + applicationId = appId; + } +} diff --git a/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/TonyRuntimeFactory.java b/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/TonyRuntimeFactory.java new file mode 100644 index 0000000000000..7a0d1703b91e7 --- /dev/null +++ b/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/TonyRuntimeFactory.java @@ -0,0 +1,55 @@ +/** + * Licensed 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. See accompanying LICENSE file. + */ +package org.apache.hadoop.yarn.submarine.runtimes.tony; + +import com.linkedin.tony.TonyClient; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.submarine.common.ClientContext; +import org.apache.hadoop.yarn.submarine.runtimes.RuntimeFactory; +import org.apache.hadoop.yarn.submarine.runtimes.common.FSBasedSubmarineStorageImpl; +import org.apache.hadoop.yarn.submarine.runtimes.common.JobMonitor; +import org.apache.hadoop.yarn.submarine.runtimes.common.JobSubmitter; +import org.apache.hadoop.yarn.submarine.runtimes.common.SubmarineStorage; + +/** + * Implementation of RuntimeFactory with Tony Runtime + */ +public class TonyRuntimeFactory extends RuntimeFactory { + private TonyClient tonyClient; + private TonyJobSubmitter submitter; + private TonyJobMonitor monitor; + + public TonyRuntimeFactory(ClientContext clientContext) { + super(clientContext); + submitter = new TonyJobSubmitter(); + tonyClient = new TonyClient(submitter, new Configuration()); + monitor = new TonyJobMonitor(clientContext, tonyClient); + submitter.setTonyClient(tonyClient); + } + + @Override + protected JobSubmitter internalCreateJobSubmitter() { + return submitter; + } + + @Override + protected JobMonitor internalCreateJobMonitor() { + return monitor; + } + + @Override + protected SubmarineStorage internalCreateSubmarineStorage() { + return new FSBasedSubmarineStorageImpl(super.clientContext); + } +} diff --git a/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/TonyUtils.java b/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/TonyUtils.java new file mode 100644 index 0000000000000..7dc49b3a3390e --- /dev/null +++ b/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/TonyUtils.java @@ -0,0 +1,164 @@ +/** + * Licensed 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. See accompanying LICENSE file. + */ +package org.apache.hadoop.yarn.submarine.runtimes.tony; + +import com.linkedin.tony.Constants; +import com.linkedin.tony.TonyConfigurationKeys; +import com.linkedin.tony.util.Utils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException; +import org.apache.hadoop.yarn.submarine.client.cli.param.RunJobParameters; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +/** + * Utilities for Tony Runtime. + */ +public final class TonyUtils { + private static final Log LOG = LogFactory.getLog(TonyUtils.class); + + public static Configuration tonyConfFromClientContext( + RunJobParameters parameters) { + Configuration tonyConf = new Configuration(); + tonyConf.setInt( + TonyConfigurationKeys.getInstancesKey(Constants.WORKER_JOB_NAME), + parameters.getNumWorkers()); + tonyConf.setInt( + TonyConfigurationKeys.getInstancesKey(Constants.PS_JOB_NAME), + parameters.getNumPS()); + // Resources for PS & Worker + if (parameters.getPsResource() != null) { + tonyConf.setInt( + TonyConfigurationKeys.getResourceKey(Constants.PS_JOB_NAME, + Constants.VCORES), + parameters.getPsResource().getVirtualCores()); + tonyConf.setLong( + TonyConfigurationKeys.getResourceKey(Constants.PS_JOB_NAME, + Constants.MEMORY), + parameters.getPsResource().getMemorySize()); + } + if (parameters.getWorkerResource() != null) { + tonyConf.setInt( + TonyConfigurationKeys.getResourceKey(Constants.WORKER_JOB_NAME, + Constants.VCORES), + parameters.getWorkerResource().getVirtualCores()); + tonyConf.setLong( + TonyConfigurationKeys.getResourceKey(Constants.WORKER_JOB_NAME, + Constants.MEMORY), + parameters.getWorkerResource().getMemorySize()); + try { + tonyConf.setLong( + TonyConfigurationKeys.getResourceKey(Constants.WORKER_JOB_NAME, + Constants.GPUS), + parameters.getWorkerResource() + .getResourceValue(ResourceInformation.GPU_URI)); + } catch (ResourceNotFoundException rnfe) { + LOG.error("GPU resources not enabled."); + } + } + if (parameters.getQueue() != null) { + tonyConf.set( + TonyConfigurationKeys.YARN_QUEUE_NAME, + parameters.getQueue()); + } + // Set up Docker for PS & Worker + if (parameters.getDockerImageName() != null) { + tonyConf.set(TonyConfigurationKeys.getContainerDockerKey(), + parameters.getDockerImageName()); + tonyConf.setBoolean(TonyConfigurationKeys.DOCKER_ENABLED, true); + } + if (parameters.getWorkerDockerImage() != null) { + tonyConf.set( + TonyConfigurationKeys.getDockerImageKey(Constants.WORKER_JOB_NAME), + parameters.getWorkerDockerImage()); + tonyConf.setBoolean(TonyConfigurationKeys.DOCKER_ENABLED, true); + } + if (parameters.getPsDockerImage() != null) { + tonyConf.set( + TonyConfigurationKeys.getDockerImageKey(Constants.PS_JOB_NAME), + parameters.getPsDockerImage()); + tonyConf.setBoolean(TonyConfigurationKeys.DOCKER_ENABLED, true); + } + + // Set up container environment + List envs = parameters.getEnvars(); + tonyConf.setStrings( + TonyConfigurationKeys.CONTAINER_LAUNCH_ENV, + envs.toArray(new String[0])); + tonyConf.setStrings(TonyConfigurationKeys.EXECUTION_ENV, + envs.stream() + .map(env -> env.replaceAll("DOCKER_", "")) + .toArray(String[]::new)); + tonyConf.setStrings(TonyConfigurationKeys.CONTAINER_LAUNCH_ENV, + envs.stream().map(env -> env.replaceAll("DOCKER_", "")) + .toArray(String[]::new)); + + // Set up running command + if (parameters.getWorkerLaunchCmd() != null) { + tonyConf.set( + TonyConfigurationKeys.getExecuteCommandKey(Constants.WORKER_JOB_NAME), + parameters.getWorkerLaunchCmd()); + } + + if (parameters.getPSLaunchCmd() != null) { + tonyConf.set( + TonyConfigurationKeys.getExecuteCommandKey(Constants.PS_JOB_NAME), + parameters.getPSLaunchCmd()); + } + + tonyConf.setBoolean(TonyConfigurationKeys.SECURITY_ENABLED, + !parameters.isSecurityDisabled()); + + // Set up container resources + if (parameters.getLocalizations() != null) { + tonyConf.setStrings(TonyConfigurationKeys.getContainerResourcesKey(), + parameters.getLocalizations().stream() + .map(lo -> lo.getRemoteUri() + Constants.RESOURCE_DIVIDER + + lo.getLocalPath()) + .toArray(String[]::new)); + } + + if (parameters.getConfPairs() != null) { + String[] confArray = parameters.getConfPairs().toArray(new String[0]); + for (Map.Entry cliConf : Utils + .parseKeyValue(confArray).entrySet()) { + String[] existingValue = tonyConf.getStrings(cliConf.getKey()); + if (existingValue != null + && TonyConfigurationKeys + .MULTI_VALUE_CONF.contains(cliConf.getKey())) { + ArrayList newValues = new ArrayList<>(Arrays + .asList(existingValue)); + newValues.add(cliConf.getValue()); + tonyConf.setStrings(cliConf.getKey(), + newValues.toArray(new String[0])); + } else { + tonyConf.set(cliConf.getKey(), cliConf.getValue()); + } + } + } + + LOG.info("Resources: " + tonyConf.get( + TonyConfigurationKeys.getContainerResourcesKey())); + return tonyConf; + } + + private TonyUtils() { + } +} diff --git a/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/buider/JobStatusBuilder.java b/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/buider/JobStatusBuilder.java new file mode 100644 index 0000000000000..c9e72dca0c0ef --- /dev/null +++ b/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/buider/JobStatusBuilder.java @@ -0,0 +1,61 @@ +/** + * Licensed 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. See accompanying LICENSE file. + */ +package org.apache.hadoop.yarn.submarine.runtimes.tony.buider; + +import com.linkedin.tony.rpc.TaskInfo; +import org.apache.hadoop.yarn.submarine.common.api.JobComponentStatus; +import org.apache.hadoop.yarn.submarine.common.api.JobStatus; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * JobStatusBuilder builds the job status from a set of TaskInfos. + */ +public final class JobStatusBuilder { + public static JobStatus fromTaskInfoSet(final Set taskInfos) { + JobStatus status = new JobStatus(); + Set jobNames = + taskInfos.stream().map(TaskInfo::getName).collect(Collectors.toSet()); + List jobComponentStatusList = new ArrayList<>(); + for (String jobName : jobNames) { + Set filterTasks = taskInfos.stream() + .filter(taskInfo -> taskInfo.getName().equals(jobName)) + .collect(Collectors.toSet()); + long numReadyContainers = 0; + long numRunningContainers = 0; + long totalAskedContainers = 0; + for (TaskInfo taskInfo : filterTasks) { + totalAskedContainers += 1; + switch (taskInfo.getStatus()) { + case READY: + numReadyContainers += 1; + break; + case RUNNING: + numRunningContainers += 1; + break; + default: + } + } + jobComponentStatusList.add(new JobComponentStatus(jobName, + numReadyContainers, numRunningContainers, totalAskedContainers)); + } + status.setComponentStatus(jobComponentStatusList); + return status; + } + + private JobStatusBuilder() { } +} diff --git a/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/buider/package-info.java b/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/buider/package-info.java new file mode 100644 index 0000000000000..5dfd5a35d5c9e --- /dev/null +++ b/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/buider/package-info.java @@ -0,0 +1,14 @@ +/** + * Licensed 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. See accompanying LICENSE file. + */ +package org.apache.hadoop.yarn.submarine.runtimes.tony.buider; \ No newline at end of file diff --git a/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/package-info.java b/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/package-info.java new file mode 100644 index 0000000000000..4596202bcc0e3 --- /dev/null +++ b/hadoop-submarine/hadoop-submarine-tony-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/tony/package-info.java @@ -0,0 +1,14 @@ +/** + * Licensed 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. See accompanying LICENSE file. + */ +package org.apache.hadoop.yarn.submarine.runtimes.tony; \ No newline at end of file diff --git a/hadoop-submarine/hadoop-submarine-tony-runtime/src/site/markdown/QuickStart.md b/hadoop-submarine/hadoop-submarine-tony-runtime/src/site/markdown/QuickStart.md new file mode 100644 index 0000000000000..b6503e8123140 --- /dev/null +++ b/hadoop-submarine/hadoop-submarine-tony-runtime/src/site/markdown/QuickStart.md @@ -0,0 +1,198 @@ + + +# Quick Start Guide + +## Prerequisite + +Must: + +- Apache Hadoop 2.7 or above. + +Optional: + +- Enable GPU on YARN support (when GPU-based training is required, Hadoop 3.1 and above). +- Enable Docker support on Hadoop (Hadoop 2.9 and above). + +## Run jobs + +### Commandline options + +```$xslt +usage: + -docker_image Docker image name/tag + -env Common environment variable of worker/ps + -name Name of the job + -num_ps Number of PS tasks of the job, by default + it's 0 + -num_workers Numnber of worker tasks of the job, by + default it's 1 + -ps_docker_image Specify docker image for PS, when this is + not specified, PS uses --docker_image as + default. + -ps_launch_cmd Commandline of worker, arguments will be + directly used to launch the PS + -ps_resources Resource of each PS, for example + memory-mb=2048,vcores=2,yarn.io/gpu=2 + -queue Name of queue to run the job, by default it + uses default queue + -saved_model_path Model exported path (savedmodel) of the job, + which is needed when exported model is not + placed under ${checkpoint_path}could be + local or other FS directory. This will be + used to serve. + -tensorboard Should we run TensorBoard for this job? By + default it's true + -verbose Print verbose log for troubleshooting + -wait_job_finish Specified when user want to wait the job + finish + -worker_docker_image Specify docker image for WORKER, when this + is not specified, WORKER uses --docker_image + as default. + -worker_launch_cmd Commandline of worker, arguments will be + directly used to launch the worker + -worker_resources Resource of each worker, for example + memory-mb=2048,vcores=2,yarn.io/gpu=2 + -localization Specify localization to remote/local + file/directory available to all container(Docker). + Argument format is "RemoteUri:LocalFilePath[:rw]" + (ro permission is not supported yet). + The RemoteUri can be a file or directory in local + or HDFS or s3 or abfs or http .etc. + The LocalFilePath can be absolute or relative. + If relative, it'll be under container's implied + working directory. + This option can be set mutiple times. + Examples are + -localization "hdfs:///user/yarn/mydir2:/opt/data" + -localization "s3a:///a/b/myfile1:./" + -localization "https:///a/b/myfile2:./myfile" + -localization "/user/yarn/mydir3:/opt/mydir3" + -localization "./mydir1:." + -insecure Whether running in an insecure cluster + -conf Override configurations via commandline +``` + +### Submarine Configuration + +For submarine internal configuration, please create a `submarine.xml` which should be placed under `$HADOOP_CONF_DIR`. +Make sure you set `submarine.runtime.class` to `org.apache.hadoop.yarn.submarine.runtimes.tony.TonyRuntimeFactory` + +|Configuration Name | Description | +|:---- |:---- | +| `submarine.runtime.class` | org.apache.hadoop.yarn.submarine.runtimes.tony.TonyRuntimeFactory +| `submarine.localization.max-allowed-file-size-mb` | Optional. This sets a size limit to the file/directory to be localized in "-localization" CLI option. 2GB by default. | + + + +### Launch TensorFlow Application: + +#### Commandline + +### Without Docker + +You need: +* Build a Python virtual environment with TensorFlow 1.13.1 installed +* A cluster with Hadoop 2.7 or above. + +### Building a Python virtual environment with TensorFlow + +TonY requires a Python virtual environment zip with TensorFlow and any needed Python libraries already installed. + +``` +wget https://files.pythonhosted.org/packages/33/bc/fa0b5347139cd9564f0d44ebd2b147ac97c36b2403943dbee8a25fd74012/virtualenv-16.0.0.tar.gz +tar xf virtualenv-16.0.0.tar.gz + +# Make sure to install using Python 3, as TensorFlow only provides Python 3 artifacts +python virtualenv-16.0.0/virtualenv.py venv +. venv/bin/activate +pip install tensorflow==1.13.1 +zip -r venv.zip venv +``` + +### TensorFlow version + + - Version 1.13.1 + +**Note:** If you require a past version of TensorFlow and TensorBoard, take a look at [this](https://github.com/linkedin/TonY/issues/42) issue. + + +### Installing Hadoop + +TonY only requires YARN, not HDFS. Please see the [open-source documentation](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/SingleCluster.html) on how to set YARN up. + +### Get the training examples + +Get mnist_distributed.py from https://github.com/linkedin/TonY/tree/master/tony-examples/mnist-tensorflow + + +``` +CLASSPATH=$(hadoop classpath --glob): \ +./hadoop-submarine-core/target/hadoop-submarine-core-0.2.0-SNAPSHOT.jar: \ +./hadoop-submarine-yarnservice-runtime/target/hadoop-submarine-score-yarnservice-runtime-0.2.0-SNAPSHOT.jar: \ +./hadoop-submarine-tony-runtime/target/hadoop-submarine-tony-runtime-0.2.0-SNAPSHOT.jar: \ +/home/pi/hadoop/TonY/tony-cli/build/libs/tony-cli-0.3.2-all.jar \ + +java org.apache.hadoop.yarn.submarine.client.cli.Cli job run --name tf-job-001 \ + --num_workers 2 \ + --worker_resources memory=3G,vcores=2 \ + --num_ps 2 \ + --ps_resources memory=3G,vcores=2 \ + --worker_launch_cmd "venv.zip/venv/bin/python --steps 1000 --data_dir /tmp/data --working_dir /tmp/mode" \ + --ps_launch_cmd "venv.zip/venv/bin/python --steps 1000 --data_dir /tmp/data --working_dir /tmp/mode" \ + --container_resources /home/pi/hadoop/TonY/tony-cli/build/libs/tony-cli-0.3.2-all.jar + --insecure + --conf tony.containers.resources=PATH_TO_VENV_YOU_CREATED/venv.zip#archive,PATH_TO_MNIST_EXAMPLE/mnist_distributed.py, \ +PATH_TO_TONY_CLI_JAR/tony-cli-0.3.2-all.jar + +``` +You should then be able to see links and status of the jobs from command line: + +``` +2019-04-22 20:30:42,611 INFO tony.TonyClient: Tasks Status Updated: [TaskInfo] name: worker index: 0 url: http://pi-aw:8042/node/containerlogs/container_1555916523933_0030_01_000003/pi status: RUNNING +2019-04-22 20:30:42,612 INFO tony.TonyClient: Tasks Status Updated: [TaskInfo] name: worker index: 1 url: http://pi-aw:8042/node/containerlogs/container_1555916523933_0030_01_000004/pi status: RUNNING +2019-04-22 20:30:42,612 INFO tony.TonyClient: Tasks Status Updated: [TaskInfo] name: ps index: 0 url: http://pi-aw:8042/node/containerlogs/container_1555916523933_0030_01_000002/pi status: RUNNING +2019-04-22 20:30:42,612 INFO tony.TonyClient: Logs for ps 0 at: http://pi-aw:8042/node/containerlogs/container_1555916523933_0030_01_000002/pi +2019-04-22 20:30:42,612 INFO tony.TonyClient: Logs for worker 0 at: http://pi-aw:8042/node/containerlogs/container_1555916523933_0030_01_000003/pi +2019-04-22 20:30:42,612 INFO tony.TonyClient: Logs for worker 1 at: http://pi-aw:8042/node/containerlogs/container_1555916523933_0030_01_000004/pi +2019-04-22 20:30:44,625 INFO tony.TonyClient: Tasks Status Updated: [TaskInfo] name: ps index: 0 url: http://pi-aw:8042/node/containerlogs/container_1555916523933_0030_01_000002/pi status: FINISHED +2019-04-22 20:30:44,625 INFO tony.TonyClient: Tasks Status Updated: [TaskInfo] name: worker index: 0 url: http://pi-aw:8042/node/containerlogs/container_1555916523933_0030_01_000003/pi status: FINISHED +2019-04-22 20:30:44,626 INFO tony.TonyClient: Tasks Status Updated: [TaskInfo] name: worker index: 1 url: http://pi-aw:8042/node/containerlogs/container_1555916523933_0030_01_000004/pi status: FINISHED + +``` + +### With Docker + +``` +CLASSPATH=$(hadoop classpath --glob): \ +./hadoop-submarine-core/target/hadoop-submarine-core-0.2.0-SNAPSHOT.jar: \ +./hadoop-submarine-yarnservice-runtime/target/hadoop-submarine-score-yarnservice-runtime-0.2.0-SNAPSHOT.jar: \ +./hadoop-submarine-tony-runtime/target/hadoop-submarine-tony-runtime-0.2.0-SNAPSHOT.jar: \ +/home/pi/hadoop/TonY/tony-cli/build/libs/tony-cli-0.3.2-all.jar \ + +java org.apache.hadoop.yarn.submarine.client.cli.Cli job run --name tf-job-001 \ + --docker_image hadoopsubmarine/tf-1.8.0-cpu:0.0.3 \ + --input_path hdfs://pi-aw:9000/dataset/cifar-10-data \ + --worker_resources memory=3G,vcores=2 \ + --worker_launch_cmd "export CLASSPATH=\$(/hadoop-3.1.0/bin/hadoop classpath --glob) && cd /test/models/tutorials/image/cifar10_estimator && python cifar10_main.py --data-dir=%input_path% --job-dir=%checkpoint_path% --train-steps=10000 --eval-batch-size=16 --train-batch-size=16 --variable-strategy=CPU --num-gpus=0 --sync" \ + --env JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64 \ + --env DOCKER_HADOOP_HDFS_HOME=/hadoop-3.1.0 \ + --env DOCKER_JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64 \ + --env HADOOP_HOME=/hadoop-3.1.0 \ + --env HADOOP_YARN_HOME=/hadoop-3.1.0 \ + --env HADOOP_COMMON_HOME=/hadoop-3.1.0 \ + --env HADOOP_HDFS_HOME=/hadoop-3.1.0 \ + --env HADOOP_CONF_DIR=/hadoop-3.1.0/etc/hadoop \ + --conf tony.containers.resources=--conf tony.containers.resources=/home/pi/hadoop/TonY/tony-cli/build/libs/tony-cli-0.3.2-all.jar +``` diff --git a/hadoop-submarine/hadoop-submarine-tony-runtime/src/site/resources/css/site.css b/hadoop-submarine/hadoop-submarine-tony-runtime/src/site/resources/css/site.css new file mode 100644 index 0000000000000..7315db31e53ca --- /dev/null +++ b/hadoop-submarine/hadoop-submarine-tony-runtime/src/site/resources/css/site.css @@ -0,0 +1,29 @@ +/* +* 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. +*/ +#banner { + height: 93px; + background: none; +} + +#bannerLeft img { + margin-left: 30px; + margin-top: 10px; +} + +#bannerRight img { + margin: 17px; +} diff --git a/hadoop-submarine/hadoop-submarine-tony-runtime/src/site/site.xml b/hadoop-submarine/hadoop-submarine-tony-runtime/src/site/site.xml new file mode 100644 index 0000000000000..5feae9a879b8d --- /dev/null +++ b/hadoop-submarine/hadoop-submarine-tony-runtime/src/site/site.xml @@ -0,0 +1,28 @@ + + + + + org.apache.maven.skins + maven-stylus-skin + ${maven-stylus-skin.version} + + + + + + + + + diff --git a/hadoop-submarine/hadoop-submarine-tony-runtime/src/test/java/TestTonyUtils.java b/hadoop-submarine/hadoop-submarine-tony-runtime/src/test/java/TestTonyUtils.java new file mode 100644 index 0000000000000..60e2c26c421c6 --- /dev/null +++ b/hadoop-submarine/hadoop-submarine-tony-runtime/src/test/java/TestTonyUtils.java @@ -0,0 +1,113 @@ +/** + * Licensed 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. See accompanying LICENSE file. + */ +import com.linkedin.tony.Constants; +import com.linkedin.tony.TonyConfigurationKeys; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.submarine.client.cli.RunJobCli; +import org.apache.hadoop.yarn.submarine.client.cli.param.RunJobParameters; +import org.apache.hadoop.yarn.submarine.common.MockClientContext; +import org.apache.hadoop.yarn.submarine.common.conf.SubmarineLogs; +import org.apache.hadoop.yarn.submarine.runtimes.RuntimeFactory; +import org.apache.hadoop.yarn.submarine.runtimes.common.JobMonitor; +import org.apache.hadoop.yarn.submarine.runtimes.common.JobSubmitter; +import org.apache.hadoop.yarn.submarine.runtimes.common.SubmarineStorage; +import org.apache.hadoop.yarn.submarine.runtimes.tony.TonyUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * 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. + */ + +public class TestTonyUtils { + + private MockClientContext getMockClientContext() + throws IOException, YarnException { + MockClientContext mockClientContext = new MockClientContext(); + JobSubmitter mockJobSubmitter = mock(JobSubmitter.class); + when(mockJobSubmitter.submitJob(any(RunJobParameters.class))).thenReturn( + ApplicationId.newInstance(1234L, 1)); + JobMonitor mockJobMonitor = mock(JobMonitor.class); + SubmarineStorage storage = mock(SubmarineStorage.class); + RuntimeFactory rtFactory = mock(RuntimeFactory.class); + + when(rtFactory.getJobSubmitterInstance()).thenReturn(mockJobSubmitter); + when(rtFactory.getJobMonitorInstance()).thenReturn(mockJobMonitor); + when(rtFactory.getSubmarineStorage()).thenReturn(storage); + + mockClientContext.setRuntimeFactory(rtFactory); + return mockClientContext; + } + + @Before + public void before() { + SubmarineLogs.verboseOff(); + } + + @Test + public void testTonyConfFromClientContext() throws Exception { + RunJobCli runJobCli = new RunJobCli(getMockClientContext()); + runJobCli.run( + new String[] {"--name", "my-job", "--docker_image", "tf-docker:1.1.0", + "--input_path", "hdfs://input", + "--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd", + "python run-job.py", "--worker_resources", "memory=2048M,vcores=2", + "--ps_resources", "memory=4G,vcores=4", "--ps_launch_cmd", + "python run-ps.py"}); + RunJobParameters jobRunParameters = runJobCli.getRunJobParameters(); + Configuration tonyConf = TonyUtils + .tonyConfFromClientContext(jobRunParameters); + Assert.assertEquals(jobRunParameters.getDockerImageName(), + tonyConf.get(TonyConfigurationKeys.getContainerDockerKey())); + Assert.assertEquals("3", tonyConf.get(TonyConfigurationKeys + .getInstancesKey("worker"))); + Assert.assertEquals(jobRunParameters.getWorkerLaunchCmd(), + tonyConf.get(TonyConfigurationKeys + .getExecuteCommandKey("worker"))); + Assert.assertEquals("2048", tonyConf.get(TonyConfigurationKeys + .getResourceKey(Constants.WORKER_JOB_NAME, Constants.MEMORY))); + Assert.assertEquals("2", tonyConf.get(TonyConfigurationKeys + .getResourceKey(Constants.WORKER_JOB_NAME, Constants.VCORES))); + Assert.assertEquals("4096", tonyConf.get(TonyConfigurationKeys + .getResourceKey(Constants.PS_JOB_NAME, Constants.MEMORY))); + Assert.assertEquals("4", tonyConf.get(TonyConfigurationKeys + .getResourceKey(Constants.PS_JOB_NAME, + Constants.VCORES))); + Assert.assertEquals(jobRunParameters.getPSLaunchCmd(), + tonyConf.get(TonyConfigurationKeys.getExecuteCommandKey("ps"))); + } +} diff --git a/hadoop-submarine/hadoop-submarine-yarnservice-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/yarnservice/YarnServiceJobSubmitter.java b/hadoop-submarine/hadoop-submarine-yarnservice-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/yarnservice/YarnServiceJobSubmitter.java index 842f4ad4c39ff..58a33cf321dbf 100644 --- a/hadoop-submarine/hadoop-submarine-yarnservice-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/yarnservice/YarnServiceJobSubmitter.java +++ b/hadoop-submarine/hadoop-submarine-yarnservice-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/yarnservice/YarnServiceJobSubmitter.java @@ -110,7 +110,7 @@ private Resource getServiceResourceFromYarnResource( return serviceResource; } - private String getValueOfEnvionment(String envar) { + private String getValueOfEnvironment(String envar) { // extract value from "key=value" form if (envar == null || !envar.contains("=")) { return ""; @@ -133,10 +133,10 @@ private void addHdfsClassPathIfNeeded(RunJobParameters parameters, for (String envar : parameters.getEnvars()) { if (envar.startsWith("DOCKER_HADOOP_HDFS_HOME=")) { - hdfsHome = getValueOfEnvionment(envar); + hdfsHome = getValueOfEnvironment(envar); hadoopEnv = true; } else if (envar.startsWith("DOCKER_JAVA_HOME=")) { - javaHome = getValueOfEnvionment(envar); + javaHome = getValueOfEnvironment(envar); } } diff --git a/hadoop-submarine/pom.xml b/hadoop-submarine/pom.xml index bc2a0cec1f10a..1f44556905527 100644 --- a/hadoop-submarine/pom.xml +++ b/hadoop-submarine/pom.xml @@ -37,6 +37,7 @@ hadoop-submarine-core hadoop-submarine-yarnservice-runtime + hadoop-submarine-tony-runtime diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml index eaab7e3c47cac..332d8feff1299 100644 --- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml @@ -685,7 +685,6 @@ - @@ -693,7 +692,6 @@ - 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 860227e2167ab..b21d76369e544 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 @@ -402,6 +402,17 @@ public static boolean isAclEnabled(Configuration conf) { public static final boolean DEFAULT_OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED = false; + /** + * Maximum number of opportunistic containers to be allocated in + * AM heartbeat. + */ + @Unstable + public static final String + OPP_CONTAINER_MAX_ALLOCATIONS_PER_AM_HEARTBEAT = + RM_PREFIX + "opportunistic.max.container-allocation.per.am.heartbeat"; + public static final int + DEFAULT_OPP_CONTAINER_MAX_ALLOCATIONS_PER_AM_HEARTBEAT = -1; + /** Number of nodes to be used by the Opportunistic Container allocator for * dispatching containers during container allocation. */ @Unstable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-docker/src/main/scripts/entrypoint.sh b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-docker/src/main/scripts/entrypoint.sh index dfbd7e353a23a..e8ab488c9350b 100755 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-docker/src/main/scripts/entrypoint.sh +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-docker/src/main/scripts/entrypoint.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/bin/bash -x # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -32,25 +32,49 @@ template_generator() { export JAVA_HOME=/usr/lib/jvm/jre export HADOOP_CONF_DIR=/etc/hadoop/conf -/opt/apache/solr/bin/solr start -p 8983 -force -/opt/apache/solr/bin/solr create_core -c appcatalog -force -/opt/apache/solr/bin/post -c appcatalog /tmp/samples.xml -if [ -d /etc/hadoop/conf ]; then - sed -i.bak 's/shared.loader=.*$/shared.loader=\/etc\/hadoop\/conf/g' /etc/tomcat/catalina.properties + +SOLR_OPTS=() + +if [ "${SOLR_STORAGE_TYPE}" == "hdfs" ]; then + SOLR_OPTS+=("-Dsolr.directoryFactory=HdfsDirectoryFactory") + SOLR_OPTS+=("-Dsolr.lock.type=hdfs") + if [ -e "$HADOOP_CONF_DIR" ]; then + SOLR_OPTS+=("-Dsolr.hdfs.confdir=${HADOOP_CONF_DIR}") + fi fi + +if [ "${SOLR_DATA_DIR}" != "" ]; then + SOLR_OPTS+=("-Dsolr.data.dir=$SOLR_DATA_DIR") + fi + if [ -e "$KEYTAB" ]; then + SOLR_OPTS+=("-Dsolr.hdfs.security.kerberos.enabled=true") + SOLR_OPTS+=("-Dsolr.hdfs.security.kerberos.keytabfile=${KEYTAB}") + SOLR_OPTS+=("-Dsolr.hdfs.security.kerberos.principal=${PRINCIPAL}") export JAVA_OPTS="$JAVA_OPTS -Djava.security.auth.login.config=/etc/tomcat/jaas.config -Djava.security.krb5.conf=/etc/krb5.conf -Djavax.security.auth.useSubjectCredsOnly=false" template_generator /etc/tomcat/jaas.config.template /etc/tomcat/jaas.config fi + +export SOLR_OPTS + +/opt/apache/solr/bin/solr start "${SOLR_OPTS[@]}" -p 8983 -force +/opt/apache/solr/bin/solr create_core -c appcatalog -force +/opt/apache/solr/bin/post -c appcatalog /tmp/samples.xml +if [ -d /etc/hadoop/conf ]; then + sed -i.bak 's/shared.loader=.*$/shared.loader=\/etc\/hadoop\/conf/g' /etc/tomcat/catalina.properties +fi + if [ -e "$SPNEGO_KEYTAB" ]; then sed -i.bak 's/authentication.type=.*$/authentication.type=kerberos/g' /etc/tomcat/catalina.properties sed -i.bak 's/simple.anonymous.allowed=.*$/simple.anonymous.allowed=false/g' /etc/tomcat/catalina.properties - if [ -z "$SPNEGO_PRINCIPAL" ]; then - echo "kerberos.principal=HTTP/$HOSTNAME" >> /etc/tomcat/catalina.properties - else - echo "kerberos.principal=$SPNEGO_PRINCIPAL" >> /etc/tomcat/catalina.properties - fi - echo "kerberos.keytab=$SPNEGO_KEYTAB" >> /etc/tomcat/catalina.properties - echo "hostname=$HOSTNAME" >> /etc/tomcat/catalina.properties + { + if [ -z "$SPNEGO_PRINCIPAL" ]; then + echo "kerberos.principal=HTTP/$HOSTNAME" + else + echo "kerberos.principal=$SPNEGO_PRINCIPAL" + fi + echo "kerberos.keytab=$SPNEGO_KEYTAB" + echo "hostname=$HOSTNAME" + } >> /etc/tomcat/catalina.properties fi /usr/libexec/tomcat/server start diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/dev-support/findbugs-exclude.xml new file mode 100644 index 0000000000000..e4e59d93779b2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/dev-support/findbugs-exclude.xml @@ -0,0 +1,17 @@ + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/dev-support/findbugs-exclude.xml new file mode 100644 index 0000000000000..e4e59d93779b2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/dev-support/findbugs-exclude.xml @@ -0,0 +1,17 @@ + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/pom.xml new file mode 100644 index 0000000000000..02e0fdce70fae --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/pom.xml @@ -0,0 +1,173 @@ + + + + + hadoop-applications-mawo + org.apache.hadoop.applications.mawo + 3.3.0-SNAPSHOT + + 4.0.0 + + + hadoop-applications-mawo-core + jar + + Apache Hadoop YARN Application MaWo Core + http://maven.apache.org + + + UTF-8 + + + + + junit + junit + test + + + + org.apache.hadoop + hadoop-common + + + + org.apache.hadoop + hadoop-common + test-jar + test + + + + com.google.inject + guice + + + + org.apache.curator + curator-framework + + + + org.apache.curator + curator-client + + + + org.apache.curator + curator-test + test + + + + commons-io + commons-io + + + + commons-cli + commons-cli + + + + org.apache.zookeeper + zookeeper + + + + junit + junit + + + com.sun.jdmk + jmxtools + + + com.sun.jmx + jmxri + + + org.jboss.netty + netty + + + + + + org.slf4j + slf4j-api + + + + com.googlecode.json-simple + json-simple + + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + + org.mockito + mockito-all + test + + + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + src/assembly/bin.xml + + hadoop-applications-mawo-core-${project.version} + + + + package + + single + + + + + + org.apache.rat + apache-rat-plugin + + + target/**/* + + + + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/assembly/bin.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/assembly/bin.xml new file mode 100644 index 0000000000000..6f2d9a82dde0d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/assembly/bin.xml @@ -0,0 +1,41 @@ + + + + bin + + tar.gz + + + + + README* + + + + + + src/main/bin + bin + + + + + target + / + + *.jar + + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/AbstractTask.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/AbstractTask.java new file mode 100644 index 0000000000000..f27c4ee780589 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/AbstractTask.java @@ -0,0 +1,246 @@ +/** + * 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.applications.mawo.server.common; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.WritableUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +/** + * Abstract class for MaWo Task. + */ +public abstract class AbstractTask implements Task { + /** + * Task identifier. + */ + private TaskId taskID = new TaskId(); + /** + * Task environment. + */ + private Map environment = new HashMap(); + /** + * Command which need to be executed as Task. + */ + private String taskCmd; + /** + * Type of task. + */ + private TaskType taskType; + /** + * Task timeout. + */ + private long timeout; + /** + * logger for abstract class. + */ + static final Logger LOG = LoggerFactory.getLogger(AbstractTask.class); + + /** + * AbstractTask constructor. + */ + public AbstractTask() { + } + + /** + * AbstrackTask constructor. + * @param taskId : Task identifier + * @param localenvironment : Task environment vars + * @param taskCMD : Cmd to run + * @param localtimeout : Task timeout in seconds + */ + + public AbstractTask(final TaskId taskId, + final Map localenvironment, + final String taskCMD, final long localtimeout) { + this(); + setTaskId(taskId); + setEnvironment(localenvironment); + setTaskCmd(taskCMD); + setTimeout(localtimeout); + LOG.info("Created Task - type: " + this.taskType + ", TaskId: " + + this.taskID.toString() + ", cmd: '" + taskCMD + "' Timeout: " + + timeout); + } + + /** + * Get environment for a Task. + * @return environment of a Task + */ + @Override + public final Map getEnvironment() { + return environment; + } + + /** + * Set environment for a Task. + * @param localenvironment : Map of environment vars + */ + @Override + public final void setEnvironment(final Map localenvironment) { + this.environment = localenvironment; + } + + /** + * Get TaskCmd for a Task. + * @return TaskCMD: Its a task command line such as sleep 10 + */ + @Override + public final String getTaskCmd() { + return taskCmd; + } + + /** + * Set TaskCmd for a Task. + * @param taskCMD : Task command line + */ + @Override + public final void setTaskCmd(final String taskCMD) { + this.taskCmd = taskCMD; + } + + /** + * Get TaskId for a Task. + * @return TaskID: Task command line + */ + @Override + public final TaskId getTaskId() { + return taskID; + } + + /** + * Set Task Id. + * @param taskId : Task Identifier + */ + @Override + public final void setTaskId(final TaskId taskId) { + if (taskId != null) { + this.taskID = taskId; + } + } + + /** + * Get TaskType for a Task. + * @return TaskType: Type of Task + */ + @Override + public final TaskType getTaskType() { + return taskType; + } + + /** + * Set TaskType for a Task. + * @param type Simple or Composite Task + */ + public final void setTaskType(final TaskType type) { + this.taskType = type; + } + + /** + * Get Timeout for a Task. + * @return timeout in seconds + */ + @Override + public final long getTimeout() { + return this.timeout; + } + + /** + * Set Task Timeout in seconds. + * @param taskTimeout : Timeout in seconds + */ + @Override + public final void setTimeout(final long taskTimeout) { + this.timeout = taskTimeout; + } + + /** + * Write Task. + * @param out : dataoutout object. + * @throws IOException : Throws IO exception if any error occurs. + */ + @Override + public final void write(final DataOutput out) throws IOException { + taskID.write(out); + int environmentSize = 0; + if (environment == null) { + environmentSize = 0; + } else { + environmentSize = environment.size(); + } + new IntWritable(environmentSize).write(out); + if (environmentSize != 0) { + for (Entry envEntry : environment.entrySet()) { + new Text(envEntry.getKey()).write(out); + new Text(envEntry.getValue()).write(out); + } + } + Text taskCmdText; + if (taskCmd == null) { + taskCmdText = new Text(""); + } else { + taskCmdText = new Text(taskCmd); + } + taskCmdText.write(out); + WritableUtils.writeEnum(out, taskType); + WritableUtils.writeVLong(out, timeout); + } + + /** + * Read Fields from file. + * @param in : datainput object. + * @throws IOException : Throws IOException in case of error. + */ + @Override + public final void readFields(final DataInput in) throws IOException { + this.taskID = new TaskId(); + taskID.readFields(in); + IntWritable envSize = new IntWritable(0); + envSize.readFields(in); + for (int i = 0; i < envSize.get(); i++) { + Text key = new Text(); + Text value = new Text(); + key.readFields(in); + value.readFields(in); + environment.put(key.toString(), value.toString()); + } + Text taskCmdText = new Text(); + taskCmdText.readFields(in); + taskCmd = taskCmdText.toString(); + taskType = WritableUtils.readEnum(in, TaskType.class); + timeout = WritableUtils.readVLong(in); + } + + /** + * ToString. + * @return String representation of Task + */ + @Override + public final String toString() { + return "TaskId: " + this.taskID.toString() + ", TaskType: " + this.taskType + + ", cmd: '" + taskCmd + "'"; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/CompositeTask.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/CompositeTask.java new file mode 100644 index 0000000000000..05c17f8743c09 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/CompositeTask.java @@ -0,0 +1,42 @@ +/** + * 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.applications.mawo.server.common; + +/** + * Composite Task is Task with multiple commands. + */ +public class CompositeTask extends AbstractTask { + /** + * Composite Task initializer. + */ + public CompositeTask() { + super(); + setTaskType(TaskType.COMPOSITE); + } + + /** + * Composite initializer for specific task. + * @param task : Task object + */ + public CompositeTask(final Task task) { + super(task.getTaskId(), task.getEnvironment(), task.getTaskCmd(), + task.getTimeout()); + this.setTaskType(TaskType.COMPOSITE); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/DieTask.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/DieTask.java new file mode 100644 index 0000000000000..6ac778fe7033f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/DieTask.java @@ -0,0 +1,32 @@ +/** + * 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.applications.mawo.server.common; + +/** + * Die Task is a type of task which indicates app to die. + */ +public class DieTask extends AbstractTask { + /** + * Die Task constructor. + */ + public DieTask() { + super(); + setTaskType(TaskType.DIE); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/MawoConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/MawoConfiguration.java new file mode 100644 index 0000000000000..a019913ccde0d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/MawoConfiguration.java @@ -0,0 +1,502 @@ +/** + * 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.applications.mawo.server.common; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.inject.Singleton; + +/** + * MaWo configuration class. + */ +@Singleton +public final class MawoConfiguration { + /** + * logger initialization for mawo config class. + */ + static final Logger LOG = LoggerFactory.getLogger(MawoConfiguration.class); + /** + * Define comma separator. + */ + static final String COMMA_SPLITTER = ","; + /** + * MaWo config file name. + */ + public static final String CONFIG_FILE = "mawo.properties"; + + /** + * RPC server hostname. + */ + private static final String RPC_SERVER_HOSTNAME = "rpc.server.hostname"; + /** + * RPC server port. + */ + private static final String RPC_SERVER_PORT = "rpc.server.port"; + + // Default values + /** + * RPC server hostname default value. + */ + private static final String RPC_SERVER_HOSTNAME_DEFAULT = "localhost"; + /** + * RPC server port default value. + */ + private static final String RPC_SERVER_PORT_DEFAULT = "5121"; + + // Curator related Configurations + /** + * Config to check id Job Queue Storage is enabled. + */ + private static final String JOB_QUEUE_STORAGE_ENABLED = + "mawo.job-queue-storage.enabled"; + + /** + * ZooKeeper property prefix. + */ + private static final String ZK_PREFIX = "zookeeper."; + /** + * Property for ZooKeeper address. + */ + private static final String ZK_ADDRESS = ZK_PREFIX + "address"; + /** + * Default value for ZooKeeper address. + */ + private static final String ZK_ADDRESS_DEFAULT = "localhost:2181"; + + /** + * Property for ZooKeeper parent path. + */ + private static final String ZK_PARENT_PATH = ZK_PREFIX + "parent.path"; + /** + * Property for ZooKeeper parent path default value. + */ + private static final String ZK_PARENT_PATH_DEFAULT = "/mawoRoot"; + + /** + * Property for ZooKeeper retry interval. + */ + private static final String ZK_RETRY_INTERVAL_MS = + ZK_PREFIX + "retry.interval.ms"; + /** + * Default value for ZooKeeper retry interval. + */ + private static final String ZK_RETRY_INTERVAL_MS_DEFAULT = "1000"; + + /** + * Property for Zookeeper session timeout. + */ + private static final String ZK_SESSION_TIMEOUT_MS = + ZK_PREFIX + "session.timeout.ms"; + /** + * Default value for ZooKeeper session timeout. + */ + private static final String ZK_SESSION_TIMEOUT_MS_DEFAULT = "10000"; + + /** + * Property for ZooKeeper retry number. + */ + private static final String ZK_RETRIES_NUM = ZK_PREFIX + "retries.num"; + /** + * Default value for ZooKeeper retry number. + */ + private static final String ZK_RETRIES_NUM_DEFAULT = "1000"; + + /** + * Property for ZooKeeper acl. + */ + private static final String ZK_ACL = ZK_PREFIX + "acl"; + /** + * Default value for ZooKeeper acl. + */ + private static final String ZK_ACL_DEFAULT = "world:anyone:rwcda"; + + /** + * Property for setting num of workers. + */ + private static final String WORKER_NUM_TASKS = "worker.num.tasks"; + /** + * Default value for num of workers. + */ + private static final String WORKER_NUM_TASKS_DEFAULT = "10"; + + /** + * Property for setting job builder class. + */ + public static final String JOB_BUILDER_CLASS = "mawo.job-builder.class"; + /** + * Default value for job builder class = simpleTaskJobBuilder. + */ + private static final String JOB_BUILDER_CLASS_DEFAULT = + "org.apache.hadoop.applications.mawo.server.master.job." + + "SimpleTaskJobBuilder"; + + /** + * Property for setting worker workspace. + */ + private static final String WORKER_WORK_SPACE = "worker.workspace"; + /** + * Default value for worker workspace. + */ + private static final String WORKER_WORK_SPACE_DEFAULT = "/tmp"; + + /** + * Property for resource manager url. + */ + public static final String CLUSTER_MANAGER_URL = "ycloud.url"; + /** + * Default value for resource manager url. + */ + private static final String DEFAULT_CLUSTER_MANAGER_URL = "0.0.0.0:9191"; + + /** + * Property for setting auto shutdown for worker. + */ + public static final String AUTO_SHUTDOWN_WORKERS = + "mawo.master.auto-shutdown-workers"; + /** + * Set auto shutdown of workers to False by default. + */ + private static final boolean DEFAULT_AUTO_SHUTDOWN_WORKERS = false; + + /** + * Property for task status log path in master node. + */ + public static final String MASTER_TASKS_STATUS_LOG_PATH + = "master.tasks-status.log.path"; + /** + * Default value for task status log path. + */ + private static final String MASTER_TASKS_STATUS_LOG_PATH_DEFAULT = "/tmp"; + + /** + * Property for drain event timeout. + */ + private static final String MASTER_DRAIN_EVENTS_TIMEOUT = + "master.drain-events.timeout"; + /** + * Default value for drain event timeout. + */ + private static final long MASTER_DRAIN_EVENTS_TIMEOUT_DEFAULT = 60000; + + /** + * Property for worker white list env. + * This environment variables will be set for all tasks. + */ + private static final String WORKER_WHITELIST_ENV = "worker.whitelist.env"; + /** + * Default value for worker white list env. + */ + private static final String WORKER_WHITELIST_ENV_DEFAULT = ""; + + /** + * Property for teardown worker validity. + */ + private static final String MASTER_TEARDOWN_WORKER_VALIDITY_INTERVAL_MS = + "master.teardown-worker.validity-interval.ms"; + /** + * Default value for teardown worker validity. + */ + private static final String + MASTER_TEARDOWN_WORKER_VALIDITY_INTERVAL_MS_DEFAULT = "120000"; + + /** + * Map of MaWo Configs. + */ + private Map configsMap; + + /** + * Mowo configuration initializer. + */ + public MawoConfiguration() { + this(readConfigFile()); + } + + /** + * Set up MaWo properties. + * @param properties : Map of properties + */ + private MawoConfiguration(final Properties properties) { + + configsMap = new HashMap(); + + configsMap.put(RPC_SERVER_HOSTNAME, properties + .getProperty(RPC_SERVER_HOSTNAME, RPC_SERVER_HOSTNAME_DEFAULT)); + configsMap.put(RPC_SERVER_PORT, + properties.getProperty(RPC_SERVER_PORT, RPC_SERVER_PORT_DEFAULT)); + + configsMap.put(ZK_ADDRESS, + properties.getProperty(ZK_ADDRESS, ZK_ADDRESS_DEFAULT)); + configsMap.put(ZK_PARENT_PATH, + properties.getProperty(ZK_PARENT_PATH, ZK_PARENT_PATH_DEFAULT)); + configsMap.put(ZK_RETRY_INTERVAL_MS, properties + .getProperty(ZK_RETRY_INTERVAL_MS, ZK_RETRY_INTERVAL_MS_DEFAULT)); + configsMap.put(ZK_SESSION_TIMEOUT_MS, properties + .getProperty(ZK_SESSION_TIMEOUT_MS, ZK_SESSION_TIMEOUT_MS_DEFAULT)); + configsMap.put(ZK_RETRIES_NUM, + properties.getProperty(ZK_RETRIES_NUM, ZK_RETRIES_NUM_DEFAULT)); + configsMap.put(ZK_ACL, properties.getProperty(ZK_ACL, ZK_ACL_DEFAULT)); + + configsMap.put(JOB_BUILDER_CLASS, + properties.getProperty(JOB_BUILDER_CLASS, JOB_BUILDER_CLASS_DEFAULT)); + + configsMap.put(JOB_QUEUE_STORAGE_ENABLED, + properties.getProperty(JOB_QUEUE_STORAGE_ENABLED, "false")); + + configsMap.put(CLUSTER_MANAGER_URL, properties + .getProperty(CLUSTER_MANAGER_URL, DEFAULT_CLUSTER_MANAGER_URL)); + + configsMap.put(WORKER_NUM_TASKS, + properties.getProperty(WORKER_NUM_TASKS, WORKER_NUM_TASKS_DEFAULT)); + + configsMap.put(WORKER_WORK_SPACE, + properties.getProperty(WORKER_WORK_SPACE, WORKER_WORK_SPACE_DEFAULT)); + + configsMap.put(AUTO_SHUTDOWN_WORKERS, properties.getProperty( + AUTO_SHUTDOWN_WORKERS, String.valueOf(DEFAULT_AUTO_SHUTDOWN_WORKERS))); + + configsMap.put(MASTER_TASKS_STATUS_LOG_PATH, properties.getProperty( + MASTER_TASKS_STATUS_LOG_PATH, + String.valueOf(MASTER_TASKS_STATUS_LOG_PATH_DEFAULT))); + + configsMap.put(MASTER_DRAIN_EVENTS_TIMEOUT, + properties.getProperty(MASTER_DRAIN_EVENTS_TIMEOUT, + String.valueOf(MASTER_DRAIN_EVENTS_TIMEOUT_DEFAULT))); + + configsMap.put(WORKER_WHITELIST_ENV, properties.getProperty( + WORKER_WHITELIST_ENV, WORKER_WHITELIST_ENV_DEFAULT)); + + configsMap.put(MASTER_TEARDOWN_WORKER_VALIDITY_INTERVAL_MS, + properties.getProperty(MASTER_TEARDOWN_WORKER_VALIDITY_INTERVAL_MS, + MASTER_TEARDOWN_WORKER_VALIDITY_INTERVAL_MS_DEFAULT)); + + } + + /** + * Get MaWo config map. + * @return the config map for MaWo properties + */ + + public Map getConfigsMap() { + return configsMap; + } + + /** + * Find, read, and parse the configuration file. + * + * @return the properties that were found or empty if no file was found + */ + private static Properties readConfigFile() { + Properties properties = new Properties(); + + // Get property file stream from classpath + LOG.info("Configuration file being loaded: " + CONFIG_FILE + + ". Found in classpath at " + + MawoConfiguration.class.getClassLoader().getResource(CONFIG_FILE)); + InputStream inputStream = MawoConfiguration.class.getClassLoader() + .getResourceAsStream(CONFIG_FILE); + + if (inputStream == null) { + throw new RuntimeException(CONFIG_FILE + " not found in classpath"); + } + + // load the properties + try { + properties.load(inputStream); + inputStream.close(); + } catch (FileNotFoundException fnf) { + LOG.error( + "No configuration file " + CONFIG_FILE + " found in classpath."); + } catch (IOException ie) { + throw new IllegalArgumentException( + "Can't read configuration file " + CONFIG_FILE, ie); + } + + return properties; + } + + /** + * Get MaWo RPC server Port. + * @return value of rpc.server.port + */ + public int getRpcServerPort() { + return Integer.parseInt(configsMap.get(RPC_SERVER_PORT)); + } + + /** + * Get RPC Host map. + * @return value of rpc.server.hostname + */ + public String getRpcHostName() { + return configsMap.get(RPC_SERVER_HOSTNAME); + } + + /** + * Check if Job Queue Storage is Enabled. + * @return True if Job queue storage is enabled otherwise False + */ + public boolean getJobQueueStorageEnabled() { + return Boolean.parseBoolean(configsMap.get(JOB_QUEUE_STORAGE_ENABLED)); + } + + /** + * Get ZooKeeper Address. + * @return value of ZooKeeper.address + */ + public String getZKAddress() { + return configsMap.get(ZK_ADDRESS); + } + + /** + * Get ZooKeeper parent Path. + * @return value of ZooKeeper.parent.path + */ + public String getZKParentPath() { + return configsMap.get(ZK_PARENT_PATH); + } + + /** + * Get ZooKeeper retry interval value in milli seconds. + * @return value of ZooKeeper.retry.interval.ms + */ + public int getZKRetryIntervalMS() { + return Integer.parseInt(configsMap.get(ZK_RETRY_INTERVAL_MS)); + } + + /** + * Get ZooKeeper session timeout in milli seconds. + * @return value of ZooKeeper.session.timeout.ms + */ + public int getZKSessionTimeoutMS() { + return Integer.parseInt(configsMap.get(ZK_SESSION_TIMEOUT_MS)); + } + + /** + * Get ZooKeeper retries number. + * @return value of ZooKeeper.retries.num + */ + public int getZKRetriesNum() { + return Integer.parseInt(configsMap.get(ZK_RETRIES_NUM)); + } + + /** + * Get ZooKeeper Acls. + * @return value of ZooKeeper.acl + */ + public String getZKAcl() { + return configsMap.get(ZK_ACL); + } + + /** + * Get number of tasks a worker can run in parallel. + * @return value of worker.num.tasks + */ + public int getWorkerConcurrentTasksLimit() { + return Integer.parseInt(configsMap.get(WORKER_NUM_TASKS)); + } + + /** + * Get job builder class. + * @return value of mawo.job-builder.class + */ + public String getJobBuilderClass() { + return configsMap.get(JOB_BUILDER_CLASS); + } + + /** + * Get worker work space. + * @return value of worker.workspace + */ + public String getWorkerWorkSpace() { + return configsMap.get(WORKER_WORK_SPACE); + } + + /** + * Get cluster manager URL. + * @return value of ycloud.url + */ + public String getClusterManagerURL() { + return configsMap.get(CLUSTER_MANAGER_URL); + } + + /** + * Check if worker auto shutdown feature is enabled. + * @return value of mawo.master.auto-shutdown-workers + */ + public boolean getAutoShutdownWorkers() { + return Boolean.parseBoolean(configsMap.get(AUTO_SHUTDOWN_WORKERS)); + } + + /** + * Get Task status log file path on master host. + * @return value of master.tasks-status.log.path + */ + public String getMasterTasksStatusLogPath() { + return configsMap.get(MASTER_TASKS_STATUS_LOG_PATH); + } + + /** + * Get Master drain event timeout. + * @return value of master.drain-events.timeout + */ + public long getMasterDrainEventsTimeout() { + return Long.parseLong(configsMap.get(MASTER_DRAIN_EVENTS_TIMEOUT)); + } + + /** + * Get Worker whitelist env params. + * These params will be set in all tasks. + * @return list of white list environment + */ + public List getWorkerWhiteListEnv() { + List whiteList = new ArrayList(); + String env = configsMap.get(WORKER_WHITELIST_ENV); + if (env != null && !env.isEmpty()) { + String[] variables = env.split(COMMA_SPLITTER); + for (String variable : variables) { + variable = variable.trim(); + if (variable.startsWith("$")) { + variable = variable.substring(1); + } + if (!variable.isEmpty()) { + whiteList.add(variable); + } + } + } + return whiteList; + } + + /** + * Get Teardown worker validity interval. + * @return value of master.teardown-worker.validity-interval.ms + */ + public long getTeardownWorkerValidityInterval() { + return Long.parseLong(configsMap.get( + MASTER_TEARDOWN_WORKER_VALIDITY_INTERVAL_MS)); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/NullTask.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/NullTask.java new file mode 100644 index 0000000000000..c93491310836d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/NullTask.java @@ -0,0 +1,33 @@ +/** + * 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.applications.mawo.server.common; + +/** + * Define Null Task. + */ +public class NullTask extends AbstractTask { + + /** + * Null Task initializer. + */ + public NullTask() { + super(); + this.setTaskType(TaskType.NULL); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/SimpleTask.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/SimpleTask.java new file mode 100644 index 0000000000000..1dd87a67bcea0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/SimpleTask.java @@ -0,0 +1,57 @@ +/** + * 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.applications.mawo.server.common; + +import java.util.Map; + +/** + * Define Simple Task. + * Each Task has only one command + */ +public class SimpleTask extends AbstractTask { + /** + * Simple Task default initializer. + */ + public SimpleTask() { + super(); + this.setTaskType(TaskType.SIMPLE); + } + + /** + * Set up Simple Task with Task object. + * @param task : Task object + */ + public SimpleTask(final Task task) { + this(task.getTaskId(), task.getEnvironment(), task.getTaskCmd(), + task.getTimeout()); + } + + /** + * Create Simple Task with Task details. + * @param taskId : task identifier + * @param environment : task environment + * @param taskCMD : task command + * @param timeout : task timeout + */ + public SimpleTask(final TaskId taskId, final Map environment, + final String taskCMD, final long timeout) { + super(taskId, environment, taskCMD, timeout); + this.setTaskType(TaskType.SIMPLE); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/Task.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/Task.java new file mode 100644 index 0000000000000..e6b42ac9c356a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/Task.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.applications.mawo.server.common; + +import java.util.Map; + +import org.apache.hadoop.io.Writable; + +/** + * Define Task Interface. + */ +public interface Task extends Writable { + + /** + * Get TaskId of a Task. + * @return value of TaskId + */ + TaskId getTaskId(); + + /** + * Get Environment of Task. + * @return map of environment + */ + Map getEnvironment(); + + /** + * Get Task cmd. + * @return value of Task cmd such "sleep 1" + */ + String getTaskCmd(); + + /** + * Get Task type such as Simple, Composite. + * @return value of TaskType + */ + TaskType getTaskType(); + + /** + * Set TaskId. + * @param taskId : Task identifier + */ + void setTaskId(TaskId taskId); + + /** + * Set Task environment such as {"HOME":"/user/A"}. + * @param environment : Map of environment variables + */ + void setEnvironment(Map environment); + + /** + * Set Task command. + * @param taskCMD : Task command to be executed + */ + void setTaskCmd(String taskCMD); + + /** + * Get Task Timeout in seconds. + * @return value of TaskTimeout + */ + long getTimeout(); + + /** + * Set Task Timeout. + * @param timeout : value of Task Timeout + */ + void setTimeout(long timeout); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/TaskId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/TaskId.java new file mode 100644 index 0000000000000..19f8cbed90402 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/TaskId.java @@ -0,0 +1,149 @@ +/** + * 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.applications.mawo.server.common; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.applications.mawo.server.master.job.JobId; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableUtils; + +/** + * Defines TaskId for MaWo app. + */ +public class TaskId implements Writable { + + /** + * MaWo TaskIds prefix. + */ + static final String TASK_ID_PREFIX = "mawo_task_"; + + /** + * MaWo Job ID. + */ + private JobId jobId = new JobId(); + /** + * Mawo TaskId. + */ + private long taskId; + + /** + * TaskId constructor. + */ + public TaskId() { + } + + /** + * TaskId constructor with jobId and taskId. + * @param localjobId : Job identifier + * @param id : Task identifier + */ + public TaskId(final JobId localjobId, final int id) { + this.jobId = localjobId; + this.taskId = id; + } + + /** + * Getter method for jobId. + * @return JobID: Job identifier + */ + public final int getJobId() { + return jobId.getID(); + } + + /** + * Getter method for TaskID. + * @return TaskId: Task identifier + */ + public final long getId() { + return taskId; + } + + /** + * Print method for TaskId. + * @return : Full TaskId which is TaskId_prefix + jobId + _ + TaskId + */ + public final String toString() { + return TASK_ID_PREFIX + jobId.getID() + "_" + taskId; + } + + @Override + /** + * Hashcode method for TaskId. + */ + public final int hashCode() { + final int prime = 31; + final int bits = 32; + int result = 1; + int jobHash = 0; + if (jobId == null) { + jobHash = 0; + } else { + jobHash = jobId.hashCode(); + } + result = prime * result + jobHash; + result = prime * result + (int) (taskId ^ (taskId >>> bits)); + return result; + } + + @Override + /** + * Equal method override for TaskId. + */ + public final boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + + TaskId other = (TaskId) obj; + if (jobId == null) { + if (other.jobId != null) { + return false; + } + } else if (!jobId.equals(other.jobId)) { + return false; + } + if (taskId != other.taskId) { + return false; + } + return true; + } + + /** {@inheritDoc} */ + public final void write(final DataOutput out) throws IOException { + jobId.write(out); + WritableUtils.writeVLong(out, taskId); + } + + /** {@inheritDoc} */ + public final void readFields(final DataInput in) throws IOException { + jobId = new JobId(); + jobId.readFields(in); + this.taskId = WritableUtils.readVLong(in); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/TaskStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/TaskStatus.java new file mode 100644 index 0000000000000..4f780d8dc3a7d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/TaskStatus.java @@ -0,0 +1,347 @@ +/** + * 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.applications.mawo.server.common; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.applications.mawo.server.worker.WorkerId; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Defines TaskStatus for MaWo app. + */ +public class TaskStatus implements Writable, Cloneable { + + /** + * Set logger. + */ + private static final Logger LOG = LoggerFactory.getLogger(TaskStatus.class); + + /** + * TaskId is a unique task identifier. + */ + private TaskId taskId = new TaskId(); + /** + * epoch time for a task starttime. + */ + private long startTime; + /** + * epoch time for a task endtime. + */ + private long endTime; + /** + * Unique worker identifier. + */ + private WorkerId workerId = new WorkerId(); + /** + * Task exit code. + */ + private int exitCode = -1; + /** + * Task cmd. + */ + private String taskCMD; + /** + * Task type. + */ + private String taskType; + + /** + * Possible Task States. + */ + public enum State { + /** + * INIT State refers to Task initialization. + */ + INIT, + /** + * RUNNING State refers to Task in Running state. + */ + + RUNNING, + /** + * SUCCEEDED State is assigned when task finishes successfully. + */ + SUCCEEDED, + /** + * FAILED State is assigned when task fails. + */ + FAILED, + /** + * KILLED State refers to when a task is killed. + */ + KILLED, + /** + * EXPIRE State refers to when a task is expired. + */ + EXPIRE + } + + /** + * Current Task state. + */ + private volatile State runState; + + /** + * Task status constructor. + */ + public TaskStatus() { + } + + /** + * Task status constructor with workerId, TaskId, TaskCmd, TaskType. + * @param localworkerId : Worker ID + * @param localtaskId : Task ID + * @param localtaskCMD : Task command line + * @param localtaskType : Type of Task + */ + public TaskStatus(final WorkerId localworkerId, final TaskId localtaskId, + final String localtaskCMD, final String localtaskType) { + this(localworkerId, localtaskId, + TaskStatus.State.INIT, localtaskCMD, + localtaskType); + } + + /** + * Task status constructor with workerId, TaskId, + * TaskCmd, TaskType and Run State. + * @param localworkerId : Worker Id + * @param localtaskId : Task Id + * @param localrunState : Task run State + * @param localtaskCMD : Task cmd + * @param localtaskType : Task type + */ + public TaskStatus(final WorkerId localworkerId, final TaskId localtaskId, + final State localrunState, final String localtaskCMD, + final String localtaskType) { + setWorkerId(localworkerId); + setTaskId(localtaskId); + setRunState(localrunState); + setTaskCMD(localtaskCMD); + setTaskType(localtaskType); + } + + /** + * Get status of a Task. + * @return Status of a Task + */ + public final State getRunState() { + return runState; + } + + /** + * Update status of a Task. + * @param localrunState : Status of a Task + */ + public final void setRunState(final State localrunState) { + this.runState = localrunState; + } + + /** + * Set exitcode of a Task. + * @param localexitCode : Exitcode of a Task + */ + public final void setExitCode(final int localexitCode) { + this.exitCode = localexitCode; + } + + /** + * Get exitcode of a Task. + * @return exitCode of Task + */ + public final int getExitCode() { + return exitCode; + } + + /** + * Set Task cmd of a Task. + * @param localcmd : command line which need to be executed + */ + public final void setTaskCMD(final String localcmd) { + this.taskCMD = localcmd; + } + + /** + * Get Task cmd of a Task. + * @return TaskCmd : command line which need to be executed + */ + public final String getTaskCMD() { + return taskCMD; + } + + /** + * Set Task Type. + * @param localtaskType : TaskType such as SimpleTask, NullTask + */ + public final void setTaskType(final String localtaskType) { + this.taskType = localtaskType; + } + + /** + * Get Task Type. + * @return TaskType : TaskType such as SimpleTask, NullTask + */ + public final String getTaskType() { + return taskType; + } + + /** + * Get Task Id. + * @return TaskId : Task identifier + */ + public final TaskId getTaskId() { + return taskId; + } + + /** + * Set TaskId. + * @param localtaskId : Task identifier + */ + public final void setTaskId(final TaskId localtaskId) { + if (localtaskId != null) { + this.taskId = localtaskId; + } + } + + /** + * Set staus of a Task. + * @param localtaskId : TaskId of a task + * @param localrunState : Run state of a task + */ + public final void setTaskState(final TaskId localtaskId, + final State localrunState) { + setTaskId(localtaskId); + setRunState(localrunState); + } + + /** + * Get Task status of a Task. + * @param localtaskId : Task Id + * @return TaskStatus for valid Task otherwise Null + */ + public final State getTaskState(final TaskId localtaskId) { + if (localtaskId.equals(this.taskId)) { + return getRunState(); + } else { + return null; + } + } + + /** + * Get starttime of a Task. + * @return StartTime of Task + */ + public final long getStartTime() { + return startTime; + } + + /** + * Set current time as start time of a Task. + */ + public final void setStartTime() { + this.startTime = getCurrentTime(); + LOG.debug("Start Time for " + this.taskId + " is " + this.startTime); + } + + /** + * Set task start time to a specific time value. + * @param time : epoch timestamp + */ + private void setStartTime(final long time) { + this.startTime = time; + } + + /** + * Get task end time. + * @return End time of task. + */ + public final long getEndTime() { + return endTime; + } + + /** + * Set task end time to current time. + */ + public final void setEndTime() { + this.setEndTime(getCurrentTime()); + } + + /** + * Set task end time to a specific time value. + * @param time : epoch timestamp + */ + private void setEndTime(final long time) { + this.endTime = time; + LOG.debug("End Time for " + this.taskId + " is " + this.endTime); + } + + /** + * Get current time in milliseconds. + * @return Current time in milliseconds + */ + private long getCurrentTime() { + return System.currentTimeMillis(); + } + + /** {@inheritDoc} */ + public final void write(final DataOutput dataOutput) throws IOException { + workerId.write(dataOutput); + taskId.write(dataOutput); + WritableUtils.writeEnum(dataOutput, runState); + WritableUtils.writeVLong(dataOutput, getStartTime()); + WritableUtils.writeVLong(dataOutput, getEndTime()); + WritableUtils.writeString(dataOutput, taskCMD); + WritableUtils.writeString(dataOutput, taskType); + WritableUtils.writeVInt(dataOutput, exitCode); + } + + + /** {@inheritDoc} */ + public final void readFields(final DataInput dataInput) throws IOException { + workerId.readFields(dataInput); + taskId.readFields(dataInput); + setRunState(WritableUtils.readEnum(dataInput, State.class)); + setStartTime(WritableUtils.readVLong(dataInput)); + setEndTime(WritableUtils.readVLong(dataInput)); + setTaskCMD(WritableUtils.readString(dataInput)); + setTaskType(WritableUtils.readString(dataInput)); + setExitCode(WritableUtils.readVInt(dataInput)); + } + + /** + * Get workerId. + * @return workerId : Worker identifier + */ + public final WorkerId getWorkerId() { + return workerId; + } + + /** + * Set WorkerId. + * @param localworkerId : Worker identifier + */ + public final void setWorkerId(final WorkerId localworkerId) { + this.workerId = localworkerId; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/TaskType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/TaskType.java new file mode 100644 index 0000000000000..b7f22ee832b56 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/TaskType.java @@ -0,0 +1,45 @@ +/** + * 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.applications.mawo.server.common; + +/** + * Define MaWo Task Type. + */ +public enum TaskType { + /** + * Its a Simple Task. + */ + SIMPLE, + /** + * Its a composite task which consists of multiple simple tasks. + */ + COMPOSITE, + /** + * Its a null task. + */ + NULL, + /** + * Die Task is to signal for suicide. + */ + DIE, + /** + * Teardown Task is a task which runs after all tasks are finished. + */ + TEARDOWN +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/TeardownTask.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/TeardownTask.java new file mode 100644 index 0000000000000..959e2cb468791 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/TeardownTask.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.applications.mawo.server.common; + +import java.util.Map; + +/** + * Define Teardown Task. + */ +public class TeardownTask extends SimpleTask { + /** + * Teardown task default constructor. + */ + public TeardownTask() { + super(); + setTaskType(TaskType.TEARDOWN); + } + + /** + * Teardown Task constructor. + * @param taskId : Teardown task Id + * @param environment : Environment map for teardown task + * @param taskCMD : Teardown task command + * @param timeout : Timeout for Teardown task + */ + public TeardownTask(final TaskId taskId, + final Map environment, + final String taskCMD, final long timeout) { + super(taskId, environment, taskCMD, timeout); + setTaskType(TaskType.TEARDOWN); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/WorkAssignmentProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/WorkAssignmentProtocol.java new file mode 100644 index 0000000000000..39cb283a4a952 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/WorkAssignmentProtocol.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.applications.mawo.server.common; + +import org.apache.hadoop.applications.mawo.server.worker.WorkerId; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.ipc.ProtocolInfo; + +/** + * Define work assignment protocol. + */ +@ProtocolInfo(protocolName = "WorkAssignmentProtocol", protocolVersion = 1) +public interface WorkAssignmentProtocol { + + /** + * Get next workerId to which new task will be assigned. + * @return return workerId text + */ + Text getNewWorkerId(); + + /** + * Register Worker. + * When worker will be launched first, it needs to be registered with Master. + * @param workerId : Worker Id + * @return Task instance + */ + Task registerWorker(WorkerId workerId); + + /** + * De Register worker. + * When worker is de-registered, no new task will be assigned to this worker. + * @param workerId : Worker identifier + */ + void deRegisterWorker(WorkerId workerId); + + /** + * Worker sends heartbeat to Master. + * @param workerId : Worker Id + * @param taskStatusList : TaskStatus list of all tasks assigned to worker. + * @return Task instance + */ + Task sendHeartbeat(WorkerId workerId, TaskStatus[] taskStatusList); + + /** + * Add Task to the list. + * @param task : Task object + */ + void addTask(Task task); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/package-info.java new file mode 100644 index 0000000000000..f00c547a252c5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/common/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. + */ +package org.apache.hadoop.applications.mawo.server.common; +/** + * Helper classes for the mawo server common operations. + */ diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/master/job/JobId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/master/job/JobId.java new file mode 100644 index 0000000000000..f056f5787fe40 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/master/job/JobId.java @@ -0,0 +1,128 @@ +/** + * 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.applications.mawo.server.master.job; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Random; + +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableUtils; + +/** + * Define MaWo JobId. + */ +public class JobId implements Writable { + + /** + * MaWo job prefix. + */ + private static final String JOB_PREFIX = "mawo_job_"; + + /** + * Create unique random JobId. + * @return unique random JobId + */ + static JobId newJobId() { + Random rn = new Random(); + final int range = 900000; + final int randomadd = 100000; + int randomNum = rn.nextInt(range) + randomadd; + return new JobId(randomNum); + } + + /** + * Unique Id. + */ + private int jobIdentifier; + + /** + * JobId default constructor. + */ + public JobId() { + + } + + /** + * JobId constructor with Id. + * @param id : unique id + */ + public JobId(final int id) { + this.jobIdentifier = id; + } + + /** + * Get JobId. + * @return unique ID + */ + public final int getID() { + return jobIdentifier; + } + + /** + * Print JobId. + * @return JobId + */ + public final String toString() { + return JOB_PREFIX + jobIdentifier; + } + + @Override + /** + * Hashcode for jobId. + */ + public final int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + jobIdentifier; + return result; + } + + @Override + /** + * Implement equals method for jobId. + */ + public final boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + JobId other = (JobId) obj; + if (jobIdentifier != other.jobIdentifier) { + return false; + } + return true; + } + + /** {@inheritDoc} */ + public final void write(final DataOutput out) throws IOException { + WritableUtils.writeVInt(out, jobIdentifier); + } + + /** {@inheritDoc} */ + public final void readFields(final DataInput in) throws IOException { + this.jobIdentifier = WritableUtils.readVInt(in); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/master/job/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/master/job/package-info.java new file mode 100644 index 0000000000000..c9453805d1a8b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/master/job/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. + */ +package org.apache.hadoop.applications.mawo.server.master.job; +/** + * Helper classes for the mawo master job. + */ diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/worker/WorkerId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/worker/WorkerId.java new file mode 100644 index 0000000000000..dfb356b105eab --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/worker/WorkerId.java @@ -0,0 +1,162 @@ +/** + * 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.applications.mawo.server.worker; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.net.InetAddress; +import java.net.UnknownHostException; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; + +/** + * Define WorkerId for Workers. + */ +public class WorkerId implements Writable { + /** + * WorkerId is a unique identifier for workers. + */ + private Text workerId = new Text(); + /** + * Hostname of worker node. + */ + private Text hostname = new Text(); + /** + * Ip address of worker node. + */ + private Text ipAdd = new Text(); + + /** + * Default constructor for workerId. + * Set Hostname and Ip address of the machine where worker is running. + */ + public WorkerId() { + try { + this.hostname = + new Text(InetAddress.getLocalHost().getHostName()); + this.ipAdd = + new Text(InetAddress.getLocalHost().getHostAddress().toString()); + } catch (UnknownHostException e) { + e.printStackTrace(); + } + } + + /** + * Get hostname for Worker. + * @return hostname of worker node + */ + public final Text getHostname() { + return hostname; + } + + /** + * Set hostname for Worker. + * @param wkhostname : Hostname of worker + */ + public final void setHostname(final Text wkhostname) { + this.hostname = wkhostname; + } + + /** + * Get Worker IP address. + * @return IP address of worker node + */ + public final String getIPAddress() { + return this.ipAdd.toString(); + } + + /** + * Print workerId. + * @return workeId in string + */ + public final String toString() { + return workerId.toString(); + } + + /** + * Get workerId. + * @return workerId : Worker identifier + */ + public final String getWorkerId() { + return this.workerId.toString(); + } + + /** + * Set workerId. + * @param localworkerId : Worker identifier + */ + public final void setWorkerId(final String localworkerId) { + this.workerId = new Text(localworkerId); + } + + @Override + /** + * Implememt equals method for WorkerId. + */ + public final boolean equals(final Object o) { + WorkerId x = (WorkerId) o; + return x.getHostname().equals(this.hostname); + } + + /** {@inheritDoc} */ + public final void write(final DataOutput dataOutput) throws IOException { + workerId.write(dataOutput); + hostname.write(dataOutput); + ipAdd.write(dataOutput); + } + + /** {@inheritDoc} */ + public final void readFields(final DataInput dataInput) throws IOException { + workerId.readFields(dataInput); + hostname.readFields(dataInput); + ipAdd.readFields(dataInput); + } + + @Override + /** + * Override hashcode method for WorkerId. + */ + public final int hashCode() { + final int prime = 31; + int result = 1; + int workerHash = 0; + if (workerId == null) { + workerHash = 0; + } else { + workerHash = workerId.hashCode(); + } + int hostHash = 0; + if (hostname == null) { + hostHash = 0; + } else { + hostHash = hostname.hashCode(); + } + int ipHash = 0; + if (ipAdd == null) { + ipHash = 0; + } else { + ipHash = ipAdd.hashCode(); + } + result = prime * result + workerHash; + result = prime * result + hostHash; + result = prime * result + ipHash; + return result; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/worker/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/worker/package-info.java new file mode 100644 index 0000000000000..7fa0d55b6e92d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/java/org/apache/hadoop/applications/mawo/server/worker/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. + */ +package org.apache.hadoop.applications.mawo.server.worker; +/** + * Helper classes for the mawo worker. + */ diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/resources/mawo-default.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/resources/mawo-default.properties new file mode 100644 index 0000000000000..5ebe59eaa1aaa --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/main/resources/mawo-default.properties @@ -0,0 +1,41 @@ +# 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. + +rpc.server.hostname=localhost +rpc.server.port=5120 + +#curator related configurations +zookeeper.parent.path=/mawoRoot +zookeeper.address=localhost:2181 +zookeeper.retry.interval.ms=1000 +zookeeper.session.timeout.ms=10000 +zookeeper.retries.num=1000 +zookeeper.acl=world:anyone:rwcda +worker.num.tasks=10 +mawo.job-queue-storage.enabled=true +mawo.job-builder.class=org.apache.hadoop.applications.mawo.server.master.job.SimpleTaskJsonJobBuilder +worker.workspace=/tmp + +ycloud.url=0.0.0.0:9191 + +task.block-size.min=10 +task.uber-time.min=12 + +master.tasks-status.log.path=/tmp + +master.teardown-worker.validity-interval.ms=120000 +#a comma-separated list of environment variables needed by all the tasks +worker.whitelist.env= diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/test/java/org/apache/hadoop/applications/mawo/server/common/TestMaWoConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/test/java/org/apache/hadoop/applications/mawo/server/common/TestMaWoConfiguration.java new file mode 100644 index 0000000000000..e189bcb8f435c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/test/java/org/apache/hadoop/applications/mawo/server/common/TestMaWoConfiguration.java @@ -0,0 +1,60 @@ +/** + * 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.applications.mawo.server.common; + + +import org.junit.Assert; +import org.junit.Test; + +/** + * Test MaWo configuration. + */ +public class TestMaWoConfiguration { + + /** + * Validate default MaWo Configurations. + */ + @Test + public void testMaWoConfiguration() { + + MawoConfiguration mawoConf = new MawoConfiguration(); + + // validate Rpc server port + Assert.assertEquals(mawoConf.getRpcServerPort(), 5120); + + // validate Rpc hostname + Assert.assertTrue("localhost".equals(mawoConf.getRpcHostName())); + + // validate job queue storage conf + boolean jobQueueStorage = mawoConf.getJobQueueStorageEnabled(); + Assert.assertTrue(jobQueueStorage); + + // validate default teardownWorkerValidity Interval + Assert.assertEquals(mawoConf.getTeardownWorkerValidityInterval(), 120000); + + // validate Zk related configs + Assert.assertTrue("/tmp/mawoRoot".equals(mawoConf.getZKParentPath())); + Assert.assertTrue("localhost:2181".equals(mawoConf.getZKAddress())); + Assert.assertEquals(1000, mawoConf.getZKRetryIntervalMS()); + Assert.assertEquals(10000, mawoConf.getZKSessionTimeoutMS()); + Assert.assertEquals(1000, mawoConf.getZKRetriesNum()); + } + + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/test/resources/mawo.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/test/resources/mawo.properties new file mode 100644 index 0000000000000..0d5ea31589414 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/hadoop-yarn-applications-mawo-core/src/test/resources/mawo.properties @@ -0,0 +1,28 @@ +# 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. + +rpc.server.hostname=localhost +rpc.server.port=5120 + +#curator related configurations +zookeeper.parent.path=/tmp/mawoRoot +zookeeper.address=localhost:2181 +zookeeper.retry.interval.ms=1000 +zookeeper.session.timeout.ms=10000 +zookeeper.retries.num=1000 +zookeeper.acl=world:anyone:rwcda +worker.num.tasks=10 +mawo.job-queue-storage.enabled=true diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/pom.xml new file mode 100644 index 0000000000000..4b710226c2fed --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-mawo/pom.xml @@ -0,0 +1,37 @@ + + + + + hadoop-yarn-applications + org.apache.hadoop + 3.3.0-SNAPSHOT + + 4.0.0 + + org.apache.hadoop.applications.mawo + hadoop-applications-mawo + pom + + Apache Hadoop YARN Application MaWo + http://maven.apache.org + + + UTF-8 + + + + hadoop-yarn-applications-mawo-core + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml index 695d93bc1b930..ff585f944c792 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml @@ -250,6 +250,12 @@ test + + org.assertj + assertj-core + test + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java index 6c38511ed7b55..0512e5845b37a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java @@ -61,6 +61,7 @@ import java.util.*; import java.util.concurrent.TimeoutException; +import static org.assertj.core.api.Assertions.assertThat; import static org.apache.hadoop.yarn.api.records.YarnApplicationState.FINISHED; import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.*; import static org.apache.hadoop.yarn.service.exceptions.LauncherExitCodes.EXIT_COMMAND_ARGUMENT_ERROR; @@ -904,7 +905,7 @@ private void checkEachCompInstancesInOrder(Component component, String int i = 0; for (String s : instances) { - Assert.assertEquals(component.getName() + "-" + i, s); + assertThat(s).isEqualTo(component.getName() + "-" + i); i++; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java index 41be8c7e576a8..f75c0afdbebb6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java @@ -50,6 +50,7 @@ import java.util.Arrays; import java.util.List; +import static org.assertj.core.api.Assertions.assertThat; import static org.apache.hadoop.yarn.client.api.AppAdminClient.YARN_APP_ADMIN_CLIENT_PREFIX; import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.DEPENDENCY_TARBALL_PATH; import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.YARN_SERVICE_BASE_PATH; @@ -172,7 +173,7 @@ public void testInitiateServiceUpgrade() throws Exception { "-initiate", ExampleAppJson.resourceName(ExampleAppJson.APP_JSON), "-appTypes", DUMMY_APP_TYPE}; int result = cli.run(ApplicationCLI.preProcessArgs(args)); - Assert.assertEquals(result, 0); + assertThat(result).isEqualTo(0); } @Test (timeout = 180000) @@ -182,7 +183,7 @@ public void testInitiateAutoFinalizeServiceUpgrade() throws Exception { "-autoFinalize", "-appTypes", DUMMY_APP_TYPE}; int result = cli.run(ApplicationCLI.preProcessArgs(args)); - Assert.assertEquals(result, 0); + assertThat(result).isEqualTo(0); } @Test @@ -194,7 +195,7 @@ public void testUpgradeInstances() throws Exception { "-instances", "comp1-0,comp1-1", "-appTypes", DUMMY_APP_TYPE}; int result = cli.run(ApplicationCLI.preProcessArgs(args)); - Assert.assertEquals(result, 0); + assertThat(result).isEqualTo(0); } @Test @@ -206,7 +207,7 @@ public void testUpgradeComponents() throws Exception { "-components", "comp1,comp2", "-appTypes", DUMMY_APP_TYPE}; int result = cli.run(ApplicationCLI.preProcessArgs(args)); - Assert.assertEquals(result, 0); + assertThat(result).isEqualTo(0); } @Test @@ -218,7 +219,7 @@ public void testGetInstances() throws Exception { "-components", "comp1,comp2", "-appTypes", DUMMY_APP_TYPE}; int result = cli.run(ApplicationCLI.preProcessArgs(args)); - Assert.assertEquals(result, 0); + assertThat(result).isEqualTo(0); } @Test @@ -229,7 +230,7 @@ public void testCancelUpgrade() throws Exception { String[] args = {"app", "-upgrade", "app-1", "-cancel", "-appTypes", DUMMY_APP_TYPE}; int result = cli.run(ApplicationCLI.preProcessArgs(args)); - Assert.assertEquals(result, 0); + assertThat(result).isEqualTo(0); } @Test (timeout = 180000) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/utils/TestServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/utils/TestServiceApiUtil.java index 2bf59b8afc828..3c9b52448db69 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/utils/TestServiceApiUtil.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/utils/TestServiceApiUtil.java @@ -45,6 +45,7 @@ import java.util.Collections; import java.util.List; +import static org.assertj.core.api.Assertions.assertThat; import static org.apache.hadoop.yarn.service.conf.RestApiConstants.DEFAULT_UNLIMITED_LIFETIME; import static org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages.*; import static org.junit.Assert.assertEquals; @@ -268,7 +269,7 @@ public void testArtifacts() throws IOException { Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage()); } - assertEquals(app.getLifetime(), DEFAULT_UNLIMITED_LIFETIME); + assertThat(app.getLifetime()).isEqualTo(DEFAULT_UNLIMITED_LIFETIME); } private static Resource createValidResource() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml index 78b709a45ad84..17a5b495fc60d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml @@ -38,6 +38,7 @@ hadoop-yarn-applications-unmanaged-am-launcher hadoop-yarn-services hadoop-yarn-applications-catalog + hadoop-yarn-applications-mawo diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml index 5de2cb4cb1445..81ff752e98a9d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml @@ -84,6 +84,11 @@ junit test + + org.assertj + assertj-core + test + org.apache.hadoop diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java index f56747e3465d1..3cd4414c9373e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.client.api.async.impl; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyFloat; import static org.mockito.ArgumentMatchers.anyInt; @@ -596,7 +597,7 @@ public float getProgress() { @Override public void onError(Throwable e) { - Assert.assertEquals(e.getMessage(), "Exception from callback handler"); + assertThat(e).hasMessage("Exception from callback handler"); callStopAndNotify(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java index 3dbd1dd4f2d08..dcb8a17d36774 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.client.api.impl; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -84,7 +85,7 @@ public void testGetApplications() throws YarnException, IOException { Assert.assertEquals(reports, expectedReports); reports = client.getApplications(); - Assert.assertEquals(reports.size(), 4); + assertThat(reports).hasSize(4); client.stop(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSv2ClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSv2ClientImpl.java index 1e0a1d6550ac5..f4d7a5629778a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSv2ClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSv2ClientImpl.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.client.api.impl; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -80,14 +81,14 @@ public void testGetContainerReport() throws IOException, YarnException { when(spyTimelineReaderClient.getApplicationEntity(appId, "ALL", null)) .thenReturn(createApplicationTimelineEntity(appId, true, false)); ContainerReport report = client.getContainerReport(containerId); - Assert.assertEquals(report.getContainerId(), containerId); - Assert.assertEquals(report.getAssignedNode().getHost(), "test host"); - Assert.assertEquals(report.getAssignedNode().getPort(), 100); - Assert.assertEquals(report.getAllocatedResource().getVirtualCores(), 8); - Assert.assertEquals(report.getCreationTime(), 123456); - Assert.assertEquals(report.getLogUrl(), - "https://localhost:8188/ahs/logs/test host:100/" - + "container_0_0001_01_000001/container_0_0001_01_000001/user1"); + assertThat(report.getContainerId()).isEqualTo(containerId); + assertThat(report.getAssignedNode().getHost()).isEqualTo("test host"); + assertThat(report.getAssignedNode().getPort()).isEqualTo(100); + assertThat(report.getAllocatedResource().getVirtualCores()).isEqualTo(8); + assertThat(report.getCreationTime()).isEqualTo(123456); + assertThat(report.getLogUrl()).isEqualTo("https://localhost:8188/ahs/logs/" + + "test host:100/container_0_0001_01_000001/" + + "container_0_0001_01_000001/user1"); } @Test @@ -100,10 +101,10 @@ public void testGetAppAttemptReport() throws IOException, YarnException { .thenReturn(createAppAttemptTimelineEntity(appAttemptId)); ApplicationAttemptReport report = client.getApplicationAttemptReport(appAttemptId); - Assert.assertEquals(report.getApplicationAttemptId(), appAttemptId); - Assert.assertEquals(report.getFinishTime(), Integer.MAX_VALUE + 2L); - Assert.assertEquals(report.getOriginalTrackingUrl(), - "test original tracking url"); + assertThat(report.getApplicationAttemptId()).isEqualTo(appAttemptId); + assertThat(report.getFinishTime()).isEqualTo(Integer.MAX_VALUE + 2L); + assertThat(report.getOriginalTrackingUrl()). + isEqualTo("test original tracking url"); } @Test @@ -112,11 +113,12 @@ public void testGetAppReport() throws IOException, YarnException { when(spyTimelineReaderClient.getApplicationEntity(appId, "ALL", null)) .thenReturn(createApplicationTimelineEntity(appId, false, false)); ApplicationReport report = client.getApplicationReport(appId); - Assert.assertEquals(report.getApplicationId(), appId); - Assert.assertEquals(report.getAppNodeLabelExpression(), "test_node_label"); + assertThat(report.getApplicationId()).isEqualTo(appId); + assertThat(report.getAppNodeLabelExpression()). + isEqualTo("test_node_label"); Assert.assertTrue(report.getApplicationTags().contains("Test_APP_TAGS_1")); - Assert.assertEquals(report.getYarnApplicationState(), - YarnApplicationState.FINISHED); + assertThat(report.getYarnApplicationState()). + isEqualTo(YarnApplicationState.FINISHED); } private static TimelineEntity createApplicationTimelineEntity( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java index da4f6ee5c81be..24666f066deec 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.client.api.impl; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; @@ -1616,7 +1617,7 @@ private void waitForContainerCompletion(int numIterations, for(ContainerStatus cStatus :allocResponse .getCompletedContainersStatuses()) { if(releases.contains(cStatus.getContainerId())) { - assertEquals(cStatus.getState(), ContainerState.COMPLETE); + assertThat(cStatus.getState()).isEqualTo(ContainerState.COMPLETE); assertEquals(-100, cStatus.getExitStatus()); releases.remove(cStatus.getContainerId()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocationE2E.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocationE2E.java index 753c00ba6ff62..870862b9cd7af 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocationE2E.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocationE2E.java @@ -81,6 +81,7 @@ import java.util.Set; import java.util.TreeSet; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -657,7 +658,7 @@ public void testMixedAllocationAndRelease() throws YarnException, for (ContainerStatus cStatus : allocResponse .getCompletedContainersStatuses()) { if (releases.contains(cStatus.getContainerId())) { - assertEquals(cStatus.getState(), ContainerState.COMPLETE); + assertThat(cStatus.getState()).isEqualTo(ContainerState.COMPLETE); assertEquals(-100, cStatus.getExitStatus()); releases.remove(cStatus.getContainerId()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java index 3848b10682c42..4af97e4212ae2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java @@ -93,6 +93,7 @@ import java.util.Map; import java.util.Set; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -416,7 +417,7 @@ public void testGetApplications() throws YarnException, IOException { List expectedReports = ((MockYarnClient)client).getReports(); List reports = client.getApplications(); - Assert.assertEquals(reports, expectedReports); + assertThat(reports).isEqualTo(expectedReports); Set appTypes = new HashSet<>(); appTypes.add("YARN"); @@ -424,7 +425,7 @@ public void testGetApplications() throws YarnException, IOException { reports = client.getApplications(appTypes, null); - Assert.assertEquals(reports.size(), 2); + assertThat(reports).hasSize(2); Assert .assertTrue((reports.get(0).getApplicationType().equals("YARN") && reports .get(1).getApplicationType().equals("NON-YARN")) @@ -439,7 +440,7 @@ public void testGetApplications() throws YarnException, IOException { appStates.add(YarnApplicationState.FINISHED); appStates.add(YarnApplicationState.FAILED); reports = client.getApplications(null, appStates); - Assert.assertEquals(reports.size(), 2); + assertThat(reports).hasSize(2); Assert .assertTrue((reports.get(0).getApplicationType().equals("NON-YARN") && reports .get(1).getApplicationType().equals("NON-MAPREDUCE")) @@ -469,9 +470,9 @@ public void testGetApplicationAttempts() throws YarnException, IOException { List reports = client .getApplicationAttempts(applicationId); Assert.assertNotNull(reports); - Assert.assertEquals(reports.get(0).getApplicationAttemptId(), + assertThat(reports.get(0).getApplicationAttemptId()).isEqualTo( ApplicationAttemptId.newInstance(applicationId, 1)); - Assert.assertEquals(reports.get(1).getApplicationAttemptId(), + assertThat(reports.get(1).getApplicationAttemptId()).isEqualTo( ApplicationAttemptId.newInstance(applicationId, 2)); client.stop(); } @@ -492,7 +493,7 @@ public void testGetApplicationAttempt() throws YarnException, IOException { ApplicationAttemptReport report = client .getApplicationAttemptReport(appAttemptId); Assert.assertNotNull(report); - Assert.assertEquals(report.getApplicationAttemptId().toString(), + assertThat(report.getApplicationAttemptId().toString()).isEqualTo( expectedReports.get(0).getCurrentApplicationAttemptId().toString()); client.stop(); } @@ -512,11 +513,11 @@ public void testGetContainers() throws YarnException, IOException { applicationId, 1); List reports = client.getContainers(appAttemptId); Assert.assertNotNull(reports); - Assert.assertEquals(reports.get(0).getContainerId(), + assertThat(reports.get(0).getContainerId()).isEqualTo( (ContainerId.newContainerId(appAttemptId, 1))); - Assert.assertEquals(reports.get(1).getContainerId(), + assertThat(reports.get(1).getContainerId()).isEqualTo( (ContainerId.newContainerId(appAttemptId, 2))); - Assert.assertEquals(reports.get(2).getContainerId(), + assertThat(reports.get(2).getContainerId()).isEqualTo( (ContainerId.newContainerId(appAttemptId, 3))); //First2 containers should come from RM with updated state information and @@ -554,9 +555,9 @@ public List getContainers( List reports = client.getContainers(appAttemptId); Assert.assertNotNull(reports); Assert.assertTrue(reports.size() == 2); - Assert.assertEquals(reports.get(0).getContainerId(), + assertThat(reports.get(0).getContainerId()).isEqualTo( (ContainerId.newContainerId(appAttemptId, 1))); - Assert.assertEquals(reports.get(1).getContainerId(), + assertThat(reports.get(1).getContainerId()).isEqualTo( (ContainerId.newContainerId(appAttemptId, 2))); //Only 2 running containers from RM are present when AHS throws exception @@ -586,13 +587,13 @@ public void testGetContainerReport() throws YarnException, IOException { ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1); ContainerReport report = client.getContainerReport(containerId); Assert.assertNotNull(report); - Assert.assertEquals(report.getContainerId().toString(), + assertThat(report.getContainerId().toString()).isEqualTo( (ContainerId.newContainerId(expectedReports.get(0) .getCurrentApplicationAttemptId(), 1)).toString()); containerId = ContainerId.newContainerId(appAttemptId, 3); report = client.getContainerReport(containerId); Assert.assertNotNull(report); - Assert.assertEquals(report.getContainerId().toString(), + assertThat(report.getContainerId().toString()).isEqualTo( (ContainerId.newContainerId(expectedReports.get(0) .getCurrentApplicationAttemptId(), 3)).toString()); Assert.assertNotNull(report.getExecutionType()); @@ -609,16 +610,16 @@ public void testGetLabelsToNodes() throws YarnException, IOException { Map> expectedLabelsToNodes = ((MockYarnClient)client).getLabelsToNodesMap(); Map> labelsToNodes = client.getLabelsToNodes(); - Assert.assertEquals(labelsToNodes, expectedLabelsToNodes); - Assert.assertEquals(labelsToNodes.size(), 3); + assertThat(labelsToNodes).isEqualTo(expectedLabelsToNodes); + assertThat(labelsToNodes).hasSize(3); // Get labels to nodes for selected labels Set setLabels = new HashSet<>(Arrays.asList("x", "z")); expectedLabelsToNodes = ((MockYarnClient)client).getLabelsToNodesMap(setLabels); labelsToNodes = client.getLabelsToNodes(setLabels); - Assert.assertEquals(labelsToNodes, expectedLabelsToNodes); - Assert.assertEquals(labelsToNodes.size(), 2); + assertThat(labelsToNodes).isEqualTo(expectedLabelsToNodes); + assertThat(labelsToNodes).hasSize(2); client.stop(); client.close(); @@ -634,8 +635,8 @@ public void testGetNodesToLabels() throws YarnException, IOException { Map> expectedNodesToLabels = ((MockYarnClient) client) .getNodeToLabelsMap(); Map> nodesToLabels = client.getNodeToLabels(); - Assert.assertEquals(nodesToLabels, expectedNodesToLabels); - Assert.assertEquals(nodesToLabels.size(), 1); + assertThat(nodesToLabels).isEqualTo(expectedNodesToLabels); + assertThat(nodesToLabels).hasSize(1); client.stop(); client.close(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClientWithReservation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClientWithReservation.java index 0836b7b9c97e6..717647724accf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClientWithReservation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClientWithReservation.java @@ -62,6 +62,8 @@ import java.util.Collections; import java.util.stream.Collectors; +import static org.assertj.core.api.Assertions.assertThat; + /** * This class is to test class {@link YarnClient) and {@link YarnClientImpl} @@ -432,7 +434,7 @@ public void testListReservationsByTimeIntervalContainingNoReservations() // Ensure all reservations are filtered out. Assert.assertNotNull(response); - Assert.assertEquals(response.getReservationAllocationState().size(), 0); + assertThat(response.getReservationAllocationState()).isEmpty(); duration = 30000; deadline = sRequest.getReservationDefinition().getDeadline(); @@ -447,7 +449,7 @@ public void testListReservationsByTimeIntervalContainingNoReservations() // Ensure all reservations are filtered out. Assert.assertNotNull(response); - Assert.assertEquals(response.getReservationAllocationState().size(), 0); + assertThat(response.getReservationAllocationState()).isEmpty(); arrival = clock.getTime(); // List reservations, search by end time before the reservation start @@ -460,7 +462,7 @@ public void testListReservationsByTimeIntervalContainingNoReservations() // Ensure all reservations are filtered out. Assert.assertNotNull(response); - Assert.assertEquals(response.getReservationAllocationState().size(), 0); + assertThat(response.getReservationAllocationState()).isEmpty(); // List reservations, search by very small end time. request = ReservationListRequest @@ -470,7 +472,7 @@ public void testListReservationsByTimeIntervalContainingNoReservations() // Ensure all reservations are filtered out. Assert.assertNotNull(response); - Assert.assertEquals(response.getReservationAllocationState().size(), 0); + assertThat(response.getReservationAllocationState()).isEmpty(); } finally { // clean-up diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java index 17d570f30c3c5..939ae4617139d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java @@ -19,6 +19,7 @@ import org.apache.hadoop.yarn.api.records.NodeAttribute; import org.apache.hadoop.yarn.api.records.NodeAttributeType; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; @@ -1685,26 +1686,26 @@ public void testNodeCLIUsageInfo() throws Exception { public void testMissingArguments() throws Exception { ApplicationCLI cli = createAndGetAppCLI(); int result = cli.run(new String[] { "application", "-status" }); - Assert.assertEquals(result, -1); + assertThat(result).isEqualTo(-1); Assert.assertEquals(String.format("Missing argument for options%n%1s", createApplicationCLIHelpMessage()), sysOutStream.toString()); sysOutStream.reset(); result = cli.run(new String[] { "applicationattempt", "-status" }); - Assert.assertEquals(result, -1); + assertThat(result).isEqualTo(-1); Assert.assertEquals(String.format("Missing argument for options%n%1s", createApplicationAttemptCLIHelpMessage()), sysOutStream.toString()); sysOutStream.reset(); result = cli.run(new String[] { "container", "-status" }); - Assert.assertEquals(result, -1); + assertThat(result).isEqualTo(-1); Assert.assertEquals(String.format("Missing argument for options %1s", createContainerCLIHelpMessage()), normalize(sysOutStream.toString())); sysOutStream.reset(); NodeCLI nodeCLI = createAndGetNodeCLI(); result = nodeCLI.run(new String[] { "-status" }); - Assert.assertEquals(result, -1); + assertThat(result).isEqualTo(-1); Assert.assertEquals(String.format("Missing argument for options%n%1s", createNodeCLIHelpMessage()), sysOutStream.toString()); } @@ -2017,7 +2018,7 @@ public void testUpdateApplicationPriority() throws Exception { cli.run(new String[] { "application", "-appId", applicationId.toString(), "-updatePriority", "1" }); - Assert.assertEquals(result, 0); + assertThat(result).isEqualTo(0); verify(client).updateApplicationPriority(any(ApplicationId.class), any(Priority.class)); @@ -2413,7 +2414,7 @@ public void testUpdateApplicationTimeout() throws Exception { int result = cli.run(new String[] { "application", "-appId", applicationId.toString(), "-updateLifetime", "10" }); - Assert.assertEquals(result, 0); + assertThat(result).isEqualTo(0); verify(client) .updateApplicationTimeouts(any(UpdateApplicationTimeoutsRequest.class)); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml index 14efd2a033212..c28c29792761d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml @@ -114,6 +114,11 @@ mockito-core test + + org.assertj + assertj-core + test + org.apache.hadoop 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 e8659429b65ea..a00b5d6ba4e70 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 @@ -3340,6 +3340,15 @@ false + + + Maximum number of opportunistic containers to be allocated per + Application Master heartbeat. + + yarn.resourcemanager.opportunistic.max.container-allocation.per.am.heartbeat + -1 + + Number of nodes to be used by the Opportunistic Container Allocator for diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestResourcePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestResourcePBImpl.java index f9d296d2cb174..c92e73f44fd40 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestResourcePBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestResourcePBImpl.java @@ -34,6 +34,7 @@ import org.junit.Before; import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; /** @@ -294,7 +295,7 @@ public void testParsingResourceTags() { ResourceUtils.getResourceTypes().get("resource3"); Assert.assertTrue(info.getAttributes().isEmpty()); Assert.assertFalse(info.getTags().isEmpty()); - Assert.assertEquals(info.getTags().size(), 2); + assertThat(info.getTags()).hasSize(2); info.getTags().remove("resource3_tag_1"); info.getTags().remove("resource3_tag_2"); Assert.assertTrue(info.getTags().isEmpty()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexFileController.java index 79226797578a2..64e0b20957cd9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexFileController.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexFileController.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.logaggregation.filecontroller.ifile; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertNotNull; import static org.mockito.Mockito.mock; @@ -288,7 +289,7 @@ public boolean isRollover(final FileContext fc, // We can only get the logs/logmeta from the first write. meta = fileFormat.readAggregatedLogsMeta( logRequest); - Assert.assertEquals(meta.size(), 1); + assertThat(meta.size()).isEqualTo(1); for (ContainerLogMeta log : meta) { Assert.assertTrue(log.getContainerId().equals(containerId.toString())); Assert.assertTrue(log.getNodeId().equals(nodeId.toString())); @@ -319,7 +320,7 @@ public boolean isRollover(final FileContext fc, fileFormat.closeWriter(); meta = fileFormat.readAggregatedLogsMeta( logRequest); - Assert.assertEquals(meta.size(), 2); + assertThat(meta.size()).isEqualTo(2); for (ContainerLogMeta log : meta) { Assert.assertTrue(log.getContainerId().equals(containerId.toString())); Assert.assertTrue(log.getNodeId().equals(nodeId.toString())); @@ -347,7 +348,7 @@ public boolean isRollover(final FileContext fc, Assert.assertTrue(status.length == 2); meta = fileFormat.readAggregatedLogsMeta( logRequest); - Assert.assertEquals(meta.size(), 3); + assertThat(meta.size()).isEqualTo(3); for (ContainerLogMeta log : meta) { Assert.assertTrue(log.getContainerId().equals(containerId.toString())); Assert.assertTrue(log.getNodeId().equals(nodeId.toString())); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestApps.java index 0d65d659f42c3..5db8046c15ccf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestApps.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestApps.java @@ -25,6 +25,7 @@ import java.util.HashMap; import java.util.Map; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -52,7 +53,7 @@ public void testSetEnvFromInputString() { String badEnv = "1,,2=a=b,3=a=,4==,5==a,==,c-3=3,="; environment.clear(); Apps.setEnvFromInputString(environment, badEnv, File.pathSeparator); - assertEquals(environment.size(), 0); + assertThat(environment).isEmpty(); // Test "=" in the value part environment.clear(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestConverterUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestConverterUtils.java index 077558b96a63a..57542218aa216 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestConverterUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestConverterUtils.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.yarn.util; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -93,12 +94,12 @@ public void testNodeIdWithDefaultPort() throws URISyntaxException { NodeId nid; nid = ConverterUtils.toNodeIdWithDefaultPort("node:10"); - assertEquals(nid.getPort(), 10); - assertEquals(nid.getHost(), "node"); + assertThat(nid.getPort()).isEqualTo(10); + assertThat(nid.getHost()).isEqualTo("node"); nid = ConverterUtils.toNodeIdWithDefaultPort("node"); - assertEquals(nid.getPort(), 0); - assertEquals(nid.getHost(), "node"); + assertThat(nid.getPort()).isEqualTo(0); + assertThat(nid.getHost()).isEqualTo("node"); } @Test(expected = IllegalArgumentException.class) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java index d3d815b218b5a..21a27baccd40d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.yarn.util; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -59,7 +61,7 @@ public void testMapCastToHashMap() { HashMap alternateHashMap = TimelineServiceHelper.mapCastToHashMap(firstTreeMap); Assert.assertEquals(firstTreeMap.size(), alternateHashMap.size()); - Assert.assertEquals(alternateHashMap.get(key), value); + assertThat(alternateHashMap.get(key)).isEqualTo(value); // Test complicated hashmap be casted correctly Map> complicatedHashMap = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java index f7ec4f803f282..7a701a4e9031e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java @@ -18,6 +18,8 @@ package org.apache.hadoop.yarn.util.resource; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; @@ -501,27 +503,25 @@ public void testMultipleOpsForResourcesWithTags() throws Exception { ResourceInformation.newInstance("yarn.io/test-volume", "", 3)); Resource addedResource = Resources.add(resourceA, resourceB); - Assert.assertEquals(addedResource.getMemorySize(), 5); - Assert.assertEquals(addedResource.getVirtualCores(), 10); - Assert.assertEquals( - addedResource.getResourceInformation("resource1").getValue(), 8); + assertThat(addedResource.getMemorySize()).isEqualTo(5); + assertThat(addedResource.getVirtualCores()).isEqualTo(10); + assertThat(addedResource.getResourceInformation("resource1").getValue()). + isEqualTo(8); // Verify that value of resourceA and resourceB is not added up for // "yarn.io/test-volume". - Assert.assertEquals( - addedResource.getResourceInformation("yarn.io/test-volume").getValue(), - 2); + assertThat(addedResource.getResourceInformation("yarn.io/test-volume"). + getValue()).isEqualTo(2); Resource mulResource = Resources.multiplyAndRoundDown(resourceA, 3); - Assert.assertEquals(mulResource.getMemorySize(), 6); - Assert.assertEquals(mulResource.getVirtualCores(), 12); - Assert.assertEquals( - mulResource.getResourceInformation("resource1").getValue(), 15); + assertThat(mulResource.getMemorySize()).isEqualTo(6); + assertThat(mulResource.getVirtualCores()).isEqualTo(12); + assertThat(mulResource.getResourceInformation("resource1").getValue()). + isEqualTo(15); // Verify that value of resourceA is not multiplied up for // "yarn.io/test-volume". - Assert.assertEquals( - mulResource.getResourceInformation("yarn.io/test-volume").getValue(), - 2); + assertThat(mulResource.getResourceInformation("yarn.io/test-volume"). + getValue()).isEqualTo(2); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi/pom.xml index 9b2be66e6a841..aa03cbf24a827 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi/pom.xml @@ -73,6 +73,11 @@ junit junit + + org.assertj + assertj-core + test + org.apache.hadoop hadoop-common diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi/src/test/java/org/apache/hadoop/yarn/csi/adaptor/TestCsiAdaptorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi/src/test/java/org/apache/hadoop/yarn/csi/adaptor/TestCsiAdaptorService.java index c415ced748817..2375b06bc4240 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi/src/test/java/org/apache/hadoop/yarn/csi/adaptor/TestCsiAdaptorService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi/src/test/java/org/apache/hadoop/yarn/csi/adaptor/TestCsiAdaptorService.java @@ -52,6 +52,7 @@ import java.net.InetSocketAddress; import java.net.ServerSocket; +import static org.assertj.core.api.Assertions.assertThat; import static org.apache.hadoop.yarn.api.protocolrecords.ValidateVolumeCapabilitiesRequest.AccessMode.MULTI_NODE_MULTI_WRITER; import static org.apache.hadoop.yarn.api.protocolrecords.ValidateVolumeCapabilitiesRequest.VolumeType.FILE_SYSTEM; @@ -333,8 +334,8 @@ public void testCustomizedAdaptor() throws IOException, YarnException { // Test getPluginInfo GetPluginInfoResponse pluginInfo = adaptorClient.getPluginInfo(GetPluginInfoRequest.newInstance()); - Assert.assertEquals(pluginInfo.getDriverName(), "customized-driver"); - Assert.assertEquals(pluginInfo.getVersion(), "1.0"); + assertThat(pluginInfo.getDriverName()).isEqualTo("customized-driver"); + assertThat(pluginInfo.getVersion()).isEqualTo("1.0"); // Test validateVolumeCapacity ValidateVolumeCapabilitiesRequest request = @@ -412,8 +413,8 @@ public void testMultipleCsiAdaptors() throws IOException, YarnException { // Test getPluginInfo GetPluginInfoResponse pluginInfo = client1.getPluginInfo(GetPluginInfoRequest.newInstance()); - Assert.assertEquals(pluginInfo.getDriverName(), "customized-driver-1"); - Assert.assertEquals(pluginInfo.getVersion(), "1.0"); + assertThat(pluginInfo.getDriverName()).isEqualTo("customized-driver-1"); + assertThat(pluginInfo.getVersion()).isEqualTo("1.0"); // Test validateVolumeCapacity ValidateVolumeCapabilitiesRequest request = @@ -440,8 +441,8 @@ public void testMultipleCsiAdaptors() throws IOException, YarnException { driver2Addr.getLocalPort())); GetPluginInfoResponse pluginInfo2 = client2.getPluginInfo(GetPluginInfoRequest.newInstance()); - Assert.assertEquals(pluginInfo2.getDriverName(), "customized-driver-2"); - Assert.assertEquals(pluginInfo2.getVersion(), "1.0"); + assertThat(pluginInfo2.getDriverName()).isEqualTo("customized-driver-2"); + assertThat(pluginInfo2.getVersion()).isEqualTo("1.0"); services.stop(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml index 22f28e2d9c6e5..38e8cf130bb9f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml @@ -76,6 +76,11 @@ junit test + + org.assertj + assertj-core + test + com.google.inject diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java index ab53c48b111ac..65bd0056da000 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp; +import static org.assertj.core.api.Assertions.assertThat; import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -678,7 +679,7 @@ public void testContainerLogsForFinishedApps() throws Exception { .accept(MediaType.TEXT_PLAIN) .get(ClientResponse.class); responseText = response.getEntity(String.class); - assertEquals(responseText.getBytes().length, fullTextSize); + assertThat(responseText.getBytes()).hasSize(fullTextSize); r = resource(); response = r.path("ws").path("v1") @@ -689,7 +690,7 @@ public void testContainerLogsForFinishedApps() throws Exception { .accept(MediaType.TEXT_PLAIN) .get(ClientResponse.class); responseText = response.getEntity(String.class); - assertEquals(responseText.getBytes().length, fullTextSize); + assertThat(responseText.getBytes()).hasSize(fullTextSize); } @Test(timeout = 10000) @@ -880,8 +881,8 @@ public void testContainerLogsMetaForRunningApps() throws Exception { List logMeta = logInfo .getContainerLogsInfo(); assertTrue(logMeta.size() == 1); - assertEquals(logMeta.get(0).getFileName(), fileName); - assertEquals(logMeta.get(0).getFileSize(), String.valueOf( + assertThat(logMeta.get(0).getFileName()).isEqualTo(fileName); + assertThat(logMeta.get(0).getFileSize()).isEqualTo(String.valueOf( content.length())); } else { assertEquals(logInfo.getLogType(), @@ -908,11 +909,11 @@ public void testContainerLogsMetaForRunningApps() throws Exception { List logMeta = logInfo .getContainerLogsInfo(); assertTrue(logMeta.size() == 1); - assertEquals(logMeta.get(0).getFileName(), fileName); - assertEquals(logMeta.get(0).getFileSize(), String.valueOf( + assertThat(logMeta.get(0).getFileName()).isEqualTo(fileName); + assertThat(logMeta.get(0).getFileSize()).isEqualTo(String.valueOf( content.length())); } else { - assertEquals(logInfo.getLogType(), + assertThat(logInfo.getLogType()).isEqualTo( ContainerLogAggregationType.LOCAL.toString()); } } @@ -946,8 +947,8 @@ public void testContainerLogsMetaForFinishedApps() throws Exception { List logMeta = responseText.get(0) .getContainerLogsInfo(); assertTrue(logMeta.size() == 1); - assertEquals(logMeta.get(0).getFileName(), fileName); - assertEquals(logMeta.get(0).getFileSize(), + assertThat(logMeta.get(0).getFileName()).isEqualTo(fileName); + assertThat(logMeta.get(0).getFileSize()).isEqualTo( String.valueOf(content.length())); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml index 62f37470bdbe7..7377cd7cad7bf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml @@ -95,6 +95,11 @@ mockito-core test + + org.assertj + assertj-core + test + org.apache.zookeeper zookeeper diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java index 5600aa80dbb17..10c24022daaea 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.scheduler; +import com.google.common.annotations.VisibleForTesting; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.SecurityUtil; @@ -70,6 +71,8 @@ public class OpportunisticContainerAllocator { private static final int RACK_LOCAL_LOOP = 1; private static final int OFF_SWITCH_LOOP = 2; + private int maxAllocationsPerAMHeartbeat = -1; + /** * This class encapsulates application specific parameters used to build a * Container. @@ -291,6 +294,24 @@ public OpportunisticContainerAllocator( this.tokenSecretManager = tokenSecretManager; } + /** + * Create a new Opportunistic Container Allocator. + * @param tokenSecretManager TokenSecretManager + * @param maxAllocationsPerAMHeartbeat max number of containers to be + * allocated in one AM heartbeat + */ + public OpportunisticContainerAllocator( + BaseContainerTokenSecretManager tokenSecretManager, + int maxAllocationsPerAMHeartbeat) { + this.tokenSecretManager = tokenSecretManager; + this.maxAllocationsPerAMHeartbeat = maxAllocationsPerAMHeartbeat; + } + + @VisibleForTesting + void setMaxAllocationsPerAMHeartbeat(int maxAllocationsPerAMHeartbeat) { + this.maxAllocationsPerAMHeartbeat = maxAllocationsPerAMHeartbeat; + } + /** * Allocate OPPORTUNISTIC containers. * @param blackList Resource BlackList Request @@ -316,8 +337,8 @@ public List allocateContainers(ResourceBlacklistRequest blackList, // Add OPPORTUNISTIC requests to the outstanding ones. opportContext.addToOutstandingReqs(oppResourceReqs); - Set nodeBlackList = new HashSet<>(opportContext.getBlacklist()); + Set allocatedNodes = new HashSet<>(); List allocatedContainers = new ArrayList<>(); // Satisfy the outstanding OPPORTUNISTIC requests. @@ -333,9 +354,21 @@ public List allocateContainers(ResourceBlacklistRequest blackList, // might be different than what is requested, which is why // we need the requested capability (key) to match against // the outstanding reqs) + int remAllocs = -1; + if (maxAllocationsPerAMHeartbeat > 0) { + remAllocs = + maxAllocationsPerAMHeartbeat - allocatedContainers.size() + - getTotalAllocations(allocations); + if (remAllocs <= 0) { + LOG.info("Not allocating more containers as we have reached max " + + "allocations per AM heartbeat {}", + maxAllocationsPerAMHeartbeat); + break; + } + } Map> allocation = allocate( rmIdentifier, opportContext, schedulerKey, applicationAttemptId, - appSubmitter, nodeBlackList); + appSubmitter, nodeBlackList, allocatedNodes, remAllocs); if (allocation.size() > 0) { allocations.add(allocation); continueLoop = true; @@ -355,16 +388,42 @@ public List allocateContainers(ResourceBlacklistRequest blackList, return allocatedContainers; } + private int getTotalAllocations( + List>> allocations) { + int totalAllocs = 0; + for (Map> allocation : allocations) { + for (List allocs : allocation.values()) { + totalAllocs += allocs.size(); + } + } + return totalAllocs; + } + private Map> allocate(long rmIdentifier, OpportunisticContainerContext appContext, SchedulerRequestKey schedKey, - ApplicationAttemptId appAttId, String userName, Set blackList) + ApplicationAttemptId appAttId, String userName, Set blackList, + Set allocatedNodes, int maxAllocations) throws YarnException { Map> containers = new HashMap<>(); for (EnrichedResourceRequest enrichedAsk : appContext.getOutstandingOpReqs().get(schedKey).values()) { + int remainingAllocs = -1; + if (maxAllocations > 0) { + int totalAllocated = 0; + for (List allocs : containers.values()) { + totalAllocated += allocs.size(); + } + remainingAllocs = maxAllocations - totalAllocated; + if (remainingAllocs <= 0) { + LOG.info("Not allocating more containers as max allocations per AM " + + "heartbeat {} has reached", maxAllocationsPerAMHeartbeat); + break; + } + } allocateContainersInternal(rmIdentifier, appContext.getAppParams(), - appContext.getContainerIdGenerator(), blackList, appAttId, - appContext.getNodeMap(), userName, containers, enrichedAsk); + appContext.getContainerIdGenerator(), blackList, allocatedNodes, + appAttId, appContext.getNodeMap(), userName, containers, enrichedAsk, + remainingAllocs); ResourceRequest anyAsk = enrichedAsk.getRequest(); if (!containers.isEmpty()) { LOG.info("Opportunistic allocation requested for [priority={}, " @@ -379,10 +438,10 @@ private Map> allocate(long rmIdentifier, private void allocateContainersInternal(long rmIdentifier, AllocationParams appParams, ContainerIdGenerator idCounter, - Set blacklist, ApplicationAttemptId id, - Map allNodes, String userName, - Map> allocations, - EnrichedResourceRequest enrichedAsk) + Set blacklist, Set allocatedNodes, + ApplicationAttemptId id, Map allNodes, + String userName, Map> allocations, + EnrichedResourceRequest enrichedAsk, int maxAllocations) throws YarnException { if (allNodes.size() == 0) { LOG.info("No nodes currently available to " + @@ -395,6 +454,9 @@ private void allocateContainersInternal(long rmIdentifier, allocations.get(anyAsk.getCapability()).size()); toAllocate = Math.min(toAllocate, appParams.getMaxAllocationsPerSchedulerKeyPerRound()); + if (maxAllocations >= 0) { + toAllocate = Math.min(maxAllocations, toAllocate); + } int numAllocated = 0; // Node Candidates are selected as follows: // * Node local candidates selected in loop == 0 @@ -406,7 +468,8 @@ private void allocateContainersInternal(long rmIdentifier, } while (numAllocated < toAllocate) { Collection nodeCandidates = - findNodeCandidates(loopIndex, allNodes, blacklist, enrichedAsk); + findNodeCandidates(loopIndex, allNodes, blacklist, allocatedNodes, + enrichedAsk); for (RemoteNode rNode : nodeCandidates) { String rNodeHost = rNode.getNodeId().getHost(); // Ignore black list @@ -422,6 +485,10 @@ private void allocateContainersInternal(long rmIdentifier, } else { continue; } + } else if (allocatedNodes.contains(rNodeHost)) { + LOG.info("Opportunistic container has already been allocated on {}.", + rNodeHost); + continue; } if (loopIndex == RACK_LOCAL_LOOP) { if (enrichedAsk.getRackLocations().contains(rNode.getRackName())) { @@ -435,11 +502,7 @@ private void allocateContainersInternal(long rmIdentifier, anyAsk, rNode); numAllocated++; updateMetrics(loopIndex); - // Try to spread the allocations across the nodes. - // But don't add if it is a node local request. - if (loopIndex != NODE_LOCAL_LOOP) { - blacklist.add(rNode.getNodeId().getHost()); - } + allocatedNodes.add(rNodeHost); LOG.info("Allocated [" + container.getId() + "] as opportunistic at " + "location [" + location + "]"); if (numAllocated >= toAllocate) { @@ -475,7 +538,7 @@ private void updateMetrics(int loopIndex) { private Collection findNodeCandidates(int loopIndex, Map allNodes, Set blackList, - EnrichedResourceRequest enrichedRR) { + Set allocatedNodes, EnrichedResourceRequest enrichedRR) { LinkedList retList = new LinkedList<>(); String partition = getRequestPartition(enrichedRR); if (loopIndex > 1) { @@ -495,8 +558,9 @@ private Collection findNodeCandidates(int loopIndex, allNodes, enrichedRR, retList, numContainers); } else { // Rack local candidates - numContainers = collectRackLocalCandidates( - allNodes, enrichedRR, retList, blackList, numContainers); + numContainers = + collectRackLocalCandidates(allNodes, enrichedRR, retList, + blackList, allocatedNodes, numContainers); } if (numContainers == enrichedRR.getRequest().getNumContainers()) { // If there is no change in numContainers, then there is no point @@ -510,12 +574,16 @@ private Collection findNodeCandidates(int loopIndex, private int collectRackLocalCandidates(Map allNodes, EnrichedResourceRequest enrichedRR, LinkedList retList, - Set blackList, int numContainers) { + Set blackList, Set allocatedNodes, int numContainers) { String partition = getRequestPartition(enrichedRR); for (RemoteNode rNode : allNodes.values()) { if (StringUtils.equals(partition, getRemoteNodePartition(rNode)) && enrichedRR.getRackLocations().contains(rNode.getRackName())) { - if (blackList.contains(rNode.getNodeId().getHost())) { + String rHost = rNode.getNodeId().getHost(); + if (blackList.contains(rHost)) { + continue; + } + if (allocatedNodes.contains(rHost)) { retList.addLast(rNode); } else { retList.addFirst(rNode); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/BasePolicyManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/BasePolicyManagerTest.java index bd99cb52ee0e6..55be316e8bfe3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/BasePolicyManagerTest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/BasePolicyManagerTest.java @@ -17,6 +17,8 @@ package org.apache.hadoop.yarn.server.federation.policies.manager; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext; import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy; import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException; @@ -24,7 +26,6 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration; import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil; -import org.junit.Assert; import org.junit.Test; /** @@ -92,10 +93,10 @@ protected static void serializeAndDeserializePolicyManager( FederationRouterPolicy federationRouterPolicy = wfp2.getRouterPolicy(context, null); - Assert.assertEquals(federationAMRMProxyPolicy.getClass(), - expAMRMProxyPolicy); + assertThat(federationAMRMProxyPolicy). + isExactlyInstanceOf(expAMRMProxyPolicy); - Assert.assertEquals(federationRouterPolicy.getClass(), expRouterPolicy); + assertThat(federationRouterPolicy).isExactlyInstanceOf(expRouterPolicy); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/scheduler/TestOpportunisticContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/scheduler/TestOpportunisticContainerAllocator.java index 6f71b36f35740..57e397d0102d7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/scheduler/TestOpportunisticContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/scheduler/TestOpportunisticContainerAllocator.java @@ -197,36 +197,42 @@ public void testNodeLocalAllocation() throws Exception { List reqs = Arrays.asList( ResourceRequest.newBuilder().allocationRequestId(1) + .priority(Priority.newInstance(1)) + .resourceName(ResourceRequest.ANY) + .capability(Resources.createResource(1 * GB)) + .relaxLocality(true) + .executionType(ExecutionType.OPPORTUNISTIC).build(), + ResourceRequest.newBuilder().allocationRequestId(2) .priority(Priority.newInstance(1)) .resourceName("/r1") .capability(Resources.createResource(1 * GB)) .relaxLocality(true) .executionType(ExecutionType.OPPORTUNISTIC).build(), - ResourceRequest.newBuilder().allocationRequestId(1) + ResourceRequest.newBuilder().allocationRequestId(2) .priority(Priority.newInstance(1)) .resourceName("h1") .capability(Resources.createResource(1 * GB)) .relaxLocality(true) .executionType(ExecutionType.OPPORTUNISTIC).build(), - ResourceRequest.newBuilder().allocationRequestId(1) + ResourceRequest.newBuilder().allocationRequestId(2) .priority(Priority.newInstance(1)) .resourceName(ResourceRequest.ANY) .capability(Resources.createResource(1 * GB)) .relaxLocality(true) .executionType(ExecutionType.OPPORTUNISTIC).build(), - ResourceRequest.newBuilder().allocationRequestId(2) + ResourceRequest.newBuilder().allocationRequestId(3) .priority(Priority.newInstance(1)) .resourceName("/r1") .capability(Resources.createResource(1 * GB)) .relaxLocality(true) .executionType(ExecutionType.OPPORTUNISTIC).build(), - ResourceRequest.newBuilder().allocationRequestId(2) + ResourceRequest.newBuilder().allocationRequestId(3) .priority(Priority.newInstance(1)) .resourceName("h1") .capability(Resources.createResource(1 * GB)) .relaxLocality(true) .executionType(ExecutionType.OPPORTUNISTIC).build(), - ResourceRequest.newBuilder().allocationRequestId(2) + ResourceRequest.newBuilder().allocationRequestId(3) .priority(Priority.newInstance(1)) .resourceName(ResourceRequest.ANY) .capability(Resources.createResource(1 * GB)) @@ -247,14 +253,14 @@ public void testNodeLocalAllocation() throws Exception { List containers = allocator.allocateContainers( blacklistRequest, reqs, appAttId, oppCntxt, 1L, "luser"); LOG.info("Containers: {}", containers); - Set allocatedHosts = new HashSet<>(); + // all 3 containers should be allocated. + Assert.assertEquals(3, containers.size()); + // container with allocation id 2 and 3 should be allocated on node h1 for (Container c : containers) { - allocatedHosts.add(c.getNodeHttpAddress()); + if (c.getAllocationRequestId() == 2 || c.getAllocationRequestId() == 3) { + Assert.assertEquals("h1:1234", c.getNodeHttpAddress()); + } } - Assert.assertEquals(2, containers.size()); - Assert.assertTrue(allocatedHosts.contains("h1:1234")); - Assert.assertFalse(allocatedHosts.contains("h2:1234")); - Assert.assertFalse(allocatedHosts.contains("h3:1234")); } @Test @@ -637,4 +643,190 @@ public void testAllocationWithNodeLabels() throws Exception { Assert.assertEquals(1, containers.size()); Assert.assertEquals(0, oppCntxt.getOutstandingOpReqs().size()); } + + /** + * Tests maximum number of opportunistic containers that can be allocated in + * AM heartbeat. + * @throws Exception + */ + @Test + public void testMaxAllocationsPerAMHeartbeat() throws Exception { + ResourceBlacklistRequest blacklistRequest = + ResourceBlacklistRequest.newInstance( + new ArrayList<>(), new ArrayList<>()); + allocator.setMaxAllocationsPerAMHeartbeat(2); + final Priority priority = Priority.newInstance(1); + final ExecutionTypeRequest oppRequest = ExecutionTypeRequest.newInstance( + ExecutionType.OPPORTUNISTIC, true); + final Resource resource = Resources.createResource(1 * GB); + List reqs = + Arrays.asList( + ResourceRequest.newInstance(priority, "*", + resource, 3, true, null, oppRequest), + ResourceRequest.newInstance(priority, "h6", + resource, 3, true, null, oppRequest), + ResourceRequest.newInstance(priority, "/r3", + resource, 3, true, null, oppRequest)); + ApplicationAttemptId appAttId = ApplicationAttemptId.newInstance( + ApplicationId.newInstance(0L, 1), 1); + + oppCntxt.updateNodeList( + Arrays.asList( + RemoteNode.newInstance( + NodeId.newInstance("h3", 1234), "h3:1234", "/r2"), + RemoteNode.newInstance( + NodeId.newInstance("h2", 1234), "h2:1234", "/r1"), + RemoteNode.newInstance( + NodeId.newInstance("h5", 1234), "h5:1234", "/r1"), + RemoteNode.newInstance( + NodeId.newInstance("h4", 1234), "h4:1234", "/r2"))); + + List containers = allocator.allocateContainers( + blacklistRequest, reqs, appAttId, oppCntxt, 1L, "user1"); + LOG.info("Containers: {}", containers); + // Although capacity is present, but only 2 containers should be allocated + // as max allocation per AM heartbeat is set to 2. + Assert.assertEquals(2, containers.size()); + containers = allocator.allocateContainers( + blacklistRequest, new ArrayList<>(), appAttId, oppCntxt, 1L, "user1"); + LOG.info("Containers: {}", containers); + // Remaining 1 container should be allocated. + Assert.assertEquals(1, containers.size()); + } + + /** + * Tests maximum opportunistic container allocation per AM heartbeat for + * allocation requests with different scheduler key. + * @throws Exception + */ + @Test + public void testMaxAllocationsPerAMHeartbeatDifferentSchedKey() + throws Exception { + ResourceBlacklistRequest blacklistRequest = + ResourceBlacklistRequest.newInstance( + new ArrayList<>(), new ArrayList<>()); + allocator.setMaxAllocationsPerAMHeartbeat(2); + final ExecutionTypeRequest oppRequest = ExecutionTypeRequest.newInstance( + ExecutionType.OPPORTUNISTIC, true); + final Resource resource = Resources.createResource(1 * GB); + List reqs = + Arrays.asList( + ResourceRequest.newInstance(Priority.newInstance(1), "*", + resource, 1, true, null, oppRequest), + ResourceRequest.newInstance(Priority.newInstance(2), "h6", + resource, 2, true, null, oppRequest), + ResourceRequest.newInstance(Priority.newInstance(3), "/r3", + resource, 2, true, null, oppRequest)); + ApplicationAttemptId appAttId = ApplicationAttemptId.newInstance( + ApplicationId.newInstance(0L, 1), 1); + + oppCntxt.updateNodeList( + Arrays.asList( + RemoteNode.newInstance( + NodeId.newInstance("h3", 1234), "h3:1234", "/r2"), + RemoteNode.newInstance( + NodeId.newInstance("h2", 1234), "h2:1234", "/r1"), + RemoteNode.newInstance( + NodeId.newInstance("h5", 1234), "h5:1234", "/r1"), + RemoteNode.newInstance( + NodeId.newInstance("h4", 1234), "h4:1234", "/r2"))); + + List containers = allocator.allocateContainers( + blacklistRequest, reqs, appAttId, oppCntxt, 1L, "user1"); + LOG.info("Containers: {}", containers); + // Although capacity is present, but only 2 containers should be allocated + // as max allocation per AM heartbeat is set to 2. + Assert.assertEquals(2, containers.size()); + containers = allocator.allocateContainers( + blacklistRequest, new ArrayList<>(), appAttId, oppCntxt, 1L, "user1"); + LOG.info("Containers: {}", containers); + // 2 more containers should be allocated from pending allocation requests. + Assert.assertEquals(2, containers.size()); + containers = allocator.allocateContainers( + blacklistRequest, new ArrayList<>(), appAttId, oppCntxt, 1L, "user1"); + LOG.info("Containers: {}", containers); + // Remaining 1 container should be allocated. + Assert.assertEquals(1, containers.size()); + } + + /** + * Tests maximum opportunistic container allocation per AM heartbeat when + * limit is set to -1. + * @throws Exception + */ + @Test + public void testMaxAllocationsPerAMHeartbeatWithNoLimit() throws Exception { + ResourceBlacklistRequest blacklistRequest = + ResourceBlacklistRequest.newInstance( + new ArrayList<>(), new ArrayList<>()); + allocator.setMaxAllocationsPerAMHeartbeat(-1); + + Priority priority = Priority.newInstance(1); + Resource capability = Resources.createResource(1 * GB); + List reqs = new ArrayList<>(); + for (int i = 0; i < 20; i++) { + reqs.add(ResourceRequest.newBuilder().allocationRequestId(i + 1) + .priority(priority) + .resourceName("h1") + .capability(capability) + .relaxLocality(true) + .executionType(ExecutionType.OPPORTUNISTIC).build()); + } + ApplicationAttemptId appAttId = ApplicationAttemptId.newInstance( + ApplicationId.newInstance(0L, 1), 1); + + oppCntxt.updateNodeList( + Arrays.asList( + RemoteNode.newInstance( + NodeId.newInstance("h1", 1234), "h1:1234", "/r1"), + RemoteNode.newInstance( + NodeId.newInstance("h2", 1234), "h2:1234", "/r1"))); + + List containers = allocator.allocateContainers( + blacklistRequest, reqs, appAttId, oppCntxt, 1L, "user1"); + + // all containers should be allocated in single heartbeat. + Assert.assertEquals(20, containers.size()); + } + + /** + * Tests maximum opportunistic container allocation per AM heartbeat when + * limit is set to higher value. + * @throws Exception + */ + @Test + public void testMaxAllocationsPerAMHeartbeatWithHighLimit() + throws Exception { + ResourceBlacklistRequest blacklistRequest = + ResourceBlacklistRequest.newInstance( + new ArrayList<>(), new ArrayList<>()); + allocator.setMaxAllocationsPerAMHeartbeat(100); + + Priority priority = Priority.newInstance(1); + Resource capability = Resources.createResource(1 * GB); + List reqs = new ArrayList<>(); + for (int i = 0; i < 20; i++) { + reqs.add(ResourceRequest.newBuilder().allocationRequestId(i + 1) + .priority(priority) + .resourceName("h1") + .capability(capability) + .relaxLocality(true) + .executionType(ExecutionType.OPPORTUNISTIC).build()); + } + ApplicationAttemptId appAttId = ApplicationAttemptId.newInstance( + ApplicationId.newInstance(0L, 1), 1); + + oppCntxt.updateNodeList( + Arrays.asList( + RemoteNode.newInstance( + NodeId.newInstance("h1", 1234), "h1:1234", "/r1"), + RemoteNode.newInstance( + NodeId.newInstance("h2", 1234), "h2:1234", "/r1"))); + + List containers = allocator.allocateContainers( + blacklistRequest, reqs, appAttId, oppCntxt, 1L, "user1"); + + // all containers should be allocated in single heartbeat. + Assert.assertEquals(20, containers.size()); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml index 3c74a2002d72a..1a86a9af12e51 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml @@ -131,6 +131,11 @@ mockito-core test + + org.assertj + assertj-core + test + com.google.inject guice diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt index f0f005d53b5f2..bb7fd06a9228f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt @@ -190,5 +190,10 @@ add_executable(test-oom-listener main/native/oom-listener/impl/oom_listener.h main/native/oom-listener/test/oom_listener_test_main.cc ) -target_link_libraries(test-oom-listener gtest rt) +if(CMAKE_SYSTEM_NAME MATCHES "Darwin") + # macOS doesn't have librt + target_link_libraries(test-oom-listener gtest) +else() + target_link_libraries(test-oom-listener gtest rt) +endif() output_directory(test-oom-listener test) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java index 1ed1fdaabdbaa..89e3b478d1eba 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java @@ -473,10 +473,14 @@ protected void serviceInit(Configuration conf) throws Exception { .getContainersMonitor(), this.aclsManager, dirsHandler); addService(webServer); ((NMContext) context).setWebServer(webServer); - + int maxAllocationsPerAMHeartbeat = conf.getInt( + YarnConfiguration.OPP_CONTAINER_MAX_ALLOCATIONS_PER_AM_HEARTBEAT, + YarnConfiguration. + DEFAULT_OPP_CONTAINER_MAX_ALLOCATIONS_PER_AM_HEARTBEAT); ((NMContext) context).setQueueableContainerAllocator( new OpportunisticContainerAllocator( - context.getContainerTokenSecretManager())); + context.getContainerTokenSecretManager(), + maxAllocationsPerAMHeartbeat)); dispatcher.register(ContainerManagerEventType.class, containerManager); dispatcher.register(NodeManagerEventType.class, this); 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/resourceplugin/com/nec/NECVEPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/com/nec/NECVEPlugin.java new file mode 100644 index 0000000000000..d22623766a2b3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/com/nec/NECVEPlugin.java @@ -0,0 +1,306 @@ +/** + * 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.nodemanager.containermanager.resourceplugin.com.nec; + +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.util.Shell; +import org.apache.hadoop.util.Shell.CommandExecutor; +import org.apache.hadoop.yarn.server.nodemanager.api.deviceplugin.Device; +import org.apache.hadoop.yarn.server.nodemanager.api.deviceplugin.DevicePlugin; +import org.apache.hadoop.yarn.server.nodemanager.api.deviceplugin.DevicePluginScheduler; +import org.apache.hadoop.yarn.server.nodemanager.api.deviceplugin.DeviceRegisterRequest; +import org.apache.hadoop.yarn.server.nodemanager.api.deviceplugin.DeviceRuntimeSpec; +import org.apache.hadoop.yarn.server.nodemanager.api.deviceplugin.YarnRuntimeType; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandlerException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Function; + +/** + * A device framework plugin which supports NEC Vector Engine. + * + */ +public class NECVEPlugin implements DevicePlugin, DevicePluginScheduler { + private static final String HADOOP_COMMON_HOME = "HADOOP_COMMON_HOME"; + private static final String ENV_SCRIPT_PATH = "NEC_VE_GET_SCRIPT_PATH"; + private static final String ENV_SCRIPT_NAME = "NEC_VE_GET_SCRIPT_NAME"; + private static final String DEFAULT_SCRIPT_NAME = "nec-ve-get.py"; + private static final Logger LOG = LoggerFactory.getLogger(NECVEPlugin.class); + private static final String[] DEFAULT_BINARY_SEARCH_DIRS = new String[]{ + "/usr/bin", "/bin", "/opt/nec/ve/bin"}; + + private String binaryPath; + + private Function + commandExecutorProvider = this::createCommandExecutor; + + public NECVEPlugin() throws ResourceHandlerException { + this(System::getenv, DEFAULT_BINARY_SEARCH_DIRS); + } + + @VisibleForTesting + NECVEPlugin(Function envProvider, String[] scriptPaths) + throws ResourceHandlerException { + String binaryName = DEFAULT_SCRIPT_NAME; + + String envScriptName = envProvider.apply(ENV_SCRIPT_NAME); + if (envScriptName != null) { + binaryName = envScriptName; + } + LOG.info("Use {} as script name.", envScriptName); + + // Try to find the script based on an environment variable, if set + boolean found = false; + String envBinaryPath = envProvider.apply(ENV_SCRIPT_PATH); + if (envBinaryPath != null) { + this.binaryPath = getScriptFromEnvSetting(envBinaryPath); + found = binaryPath != null; + } + + // Try $HADOOP_COMMON_HOME + if (!found) { + // print a warning only if the env variable was defined + if (envBinaryPath != null) { + LOG.warn("Script {} does not exist, falling back " + + "to $HADOOP_COMMON_HOME/sbin/DevicePluginScript/", envBinaryPath); + } + + this.binaryPath = getScriptFromHadoopCommon(envProvider, binaryName); + found = binaryPath != null; + } + + // Try the default search directories + if (!found) { + LOG.info("Script not found under" + + " $HADOOP_COMMON_HOME/sbin/DevicePluginScript/," + + " falling back to default search directories"); + + this.binaryPath = getScriptFromSearchDirs(binaryName, scriptPaths); + found = binaryPath != null; + } + + // Script not found + if (!found) { + LOG.error("Script not found in " + + Arrays.toString(scriptPaths)); + throw new ResourceHandlerException( + "No binary found for " + NECVEPlugin.class.getName()); + } + } + + public DeviceRegisterRequest getRegisterRequestInfo() { + return DeviceRegisterRequest.Builder.newInstance() + .setResourceName("nec.com/ve").build(); + } + + public Set getDevices() { + Set devices = null; + + CommandExecutor executor = + commandExecutorProvider.apply(new String[]{this.binaryPath}); + try { + executor.execute(); + String output = executor.getOutput(); + devices = parseOutput(output); + } catch (IOException e) { + LOG.warn(e.toString()); + } + return devices; + } + + public DeviceRuntimeSpec onDevicesAllocated(Set set, + YarnRuntimeType yarnRuntimeType) { + return null; + } + + /** + * Parses the output of the external Python script. + * + * Sample line: + * id=0, dev=/dev/ve0, state=ONLINE, busId=0000:65:00.0, major=243, minor=0 + */ + private Set parseOutput(String output) { + Set devices = new HashSet<>(); + + LOG.info("Parsing output: {}", output); + String[] lines = output.split("\n"); + for (String line : lines) { + Device.Builder builder = Device.Builder.newInstance(); + + // map key --> builder calls + Map> builderInvocations = + getBuilderInvocationsMap(builder); + + String[] keyValues = line.trim().split(","); + for (String keyValue : keyValues) { + String[] tokens = keyValue.trim().split("="); + if (tokens.length != 2) { + LOG.error("Unknown format of script output! Skipping this line"); + continue; + } + + final String key = tokens[0]; + final String value = tokens[1]; + + Consumer builderInvocation = builderInvocations.get(key); + if (builderInvocation != null) { + builderInvocation.accept(value); + } else { + LOG.warn("Unknown key {}, ignored", key); + } + }// for key value pairs + Device device = builder.build(); + if (device.isHealthy()) { + devices.add(device); + } else { + LOG.warn("Skipping device {} because it's not healthy", device); + } + } + + return devices; + } + + @Override + public void onDevicesReleased(Set releasedDevices) { + // nop + } + + @Override + public Set allocateDevices(Set availableDevices, int count, + Map env) { + // Can consider topology, utilization.etc + Set allocated = new HashSet<>(); + int number = 0; + for (Device d : availableDevices) { + allocated.add(d); + number++; + if (number == count) { + break; + } + } + return allocated; + } + + private CommandExecutor createCommandExecutor(String[] command) { + return new Shell.ShellCommandExecutor( + command); + } + + private String getScriptFromEnvSetting(String envBinaryPath) { + LOG.info("Checking script path: {}", envBinaryPath); + File f = new File(envBinaryPath); + + if (!f.exists()) { + LOG.warn("Script {} does not exist", envBinaryPath); + return null; + } + + if (f.isDirectory()) { + LOG.warn("Specified path {} is a directory", envBinaryPath); + return null; + } + + if (!FileUtil.canExecute(f)) { + LOG.warn("Script {} is not executable", envBinaryPath); + return null; + } + + LOG.info("Found script: {}", envBinaryPath); + + return envBinaryPath; + } + + private String getScriptFromHadoopCommon( + Function envProvider, String binaryName) { + String scriptPath = null; + String hadoopCommon = envProvider.apply(HADOOP_COMMON_HOME); + + if (hadoopCommon != null) { + String targetPath = hadoopCommon + + "/sbin/DevicePluginScript/" + binaryName; + LOG.info("Checking script {}: ", targetPath); + if (new File(targetPath).exists()) { + LOG.info("Found script: {}", targetPath); + scriptPath = targetPath; + } + } else { + LOG.info("$HADOOP_COMMON_HOME is not set"); + } + + return scriptPath; + } + + private String getScriptFromSearchDirs(String binaryName, + String[] scriptPaths) { + String scriptPath = null; + + for (String dir : scriptPaths) { + File f = new File(dir, binaryName); + if (f.exists()) { + LOG.info("Found script: {}", dir); + scriptPath = f.getAbsolutePath(); + break; + } + } + + return scriptPath; + } + + private Map> getBuilderInvocationsMap( + Device.Builder builder) { + Map> builderInvocations = new HashMap<>(); + builderInvocations.put("id", v -> builder.setId(Integer.parseInt(v))); + builderInvocations.put("dev", v -> builder.setDevPath(v)); + builderInvocations.put("state", v -> { + if (v.equals("ONLINE")) { + builder.setHealthy(true); + } + builder.setStatus(v); + }); + builderInvocations.put("busId", v -> builder.setBusID(v)); + builderInvocations.put("major", + v -> builder.setMajorNumber(Integer.parseInt(v))); + builderInvocations.put("minor", + v -> builder.setMinorNumber(Integer.parseInt(v))); + + return builderInvocations; + } + + @VisibleForTesting + void setCommandExecutorProvider( + Function provider) { + this.commandExecutorProvider = provider; + } + + @VisibleForTesting + String getBinaryPath() { + return binaryPath; + } +} 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/resourceplugin/com/nec/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/com/nec/package-info.java new file mode 100644 index 0000000000000..8f7fd676a5a97 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/com/nec/package-info.java @@ -0,0 +1,19 @@ +/** + * 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.nodemanager.containermanager.resourceplugin.com.nec; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c index 90d96bb664495..4abee027df66c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c @@ -1462,14 +1462,22 @@ static int check_privileges(const char *user) { exit(INITIALIZE_USER_FAILED); } +#ifdef __linux__ int rc = getgrouplist(user, pw->pw_gid, groups, &ngroups); +#else + int rc = getgrouplist(user, pw->pw_gid, (int *)groups, &ngroups); +#endif if (rc < 0) { groups = (gid_t *) alloc_and_clear_memory(ngroups, sizeof(gid_t)); if (groups == NULL) { fprintf(ERRORFILE, "Failed to allocate buffer for group lookup for user %s.\n", user); exit(OUT_OF_MEMORY); } +#ifdef __linux__ if (getgrouplist(user, pw->pw_gid, groups, &ngroups) == -1) { +#else + if (getgrouplist(user, pw->pw_gid, (int *)groups, &ngroups) == -1) { +#endif fprintf(ERRORFILE, "Fail to lookup groups for user %s.\n", user); ret = 2; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutorWithMocks.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutorWithMocks.java index 51c7d963d7e1f..3d9d33c5a10dd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutorWithMocks.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutorWithMocks.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.nodemanager; +import static org.assertj.core.api.Assertions.assertThat; import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; @@ -306,33 +307,34 @@ public void testStartLocalizer() throws IOException { .build()); List result=readMockParams(); - Assert.assertEquals(result.size(), 26); - Assert.assertEquals(result.get(0), YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER); - Assert.assertEquals(result.get(1), "test"); - Assert.assertEquals(result.get(2), "0" ); - Assert.assertEquals(result.get(3), "application_0"); - Assert.assertEquals(result.get(4), "12345"); - Assert.assertEquals(result.get(5), "/bin/nmPrivateCTokensPath"); - Assert.assertEquals(result.get(9), "-classpath" ); - Assert.assertEquals(result.get(12), "-Xmx256m" ); - Assert.assertEquals(result.get(13), + assertThat(result).hasSize(26); + assertThat(result.get(0)).isEqualTo(YarnConfiguration. + DEFAULT_NM_NONSECURE_MODE_LOCAL_USER); + assertThat(result.get(1)).isEqualTo("test"); + assertThat(result.get(2)).isEqualTo("0"); + assertThat(result.get(3)).isEqualTo("application_0"); + assertThat(result.get(4)).isEqualTo("12345"); + assertThat(result.get(5)).isEqualTo("/bin/nmPrivateCTokensPath"); + assertThat(result.get(9)).isEqualTo("-classpath"); + assertThat(result.get(12)).isEqualTo("-Xmx256m"); + assertThat(result.get(13)).isEqualTo( "-Dlog4j.configuration=container-log4j.properties" ); - Assert.assertEquals(result.get(14), + assertThat(result.get(14)).isEqualTo( String.format("-Dyarn.app.container.log.dir=%s/application_0/12345", mockExec.getConf().get(YarnConfiguration.NM_LOG_DIRS))); - Assert.assertEquals(result.get(15), + assertThat(result.get(15)).isEqualTo( "-Dyarn.app.container.log.filesize=0"); - Assert.assertEquals(result.get(16), "-Dhadoop.root.logger=INFO,CLA"); - Assert.assertEquals(result.get(17), + assertThat(result.get(16)).isEqualTo("-Dhadoop.root.logger=INFO,CLA"); + assertThat(result.get(17)).isEqualTo( "-Dhadoop.root.logfile=container-localizer-syslog"); - Assert.assertEquals(result.get(18), - "org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer"); - Assert.assertEquals(result.get(19), "test"); - Assert.assertEquals(result.get(20), "application_0"); - Assert.assertEquals(result.get(21), "12345"); - Assert.assertEquals(result.get(22), "localhost"); - Assert.assertEquals(result.get(23), "8040"); - Assert.assertEquals(result.get(24), "nmPrivateCTokensPath"); + assertThat(result.get(18)).isEqualTo("org.apache.hadoop.yarn.server." + + "nodemanager.containermanager.localizer.ContainerLocalizer"); + assertThat(result.get(19)).isEqualTo("test"); + assertThat(result.get(20)).isEqualTo("application_0"); + assertThat(result.get(21)).isEqualTo("12345"); + assertThat(result.get(22)).isEqualTo("localhost"); + assertThat(result.get(23)).isEqualTo("8040"); + assertThat(result.get(24)).isEqualTo("nmPrivateCTokensPath"); } catch (InterruptedException e) { LOG.error("Error:"+e.getMessage(),e); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java index e129caab1a640..e920105abf9aa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -302,7 +303,7 @@ public void testApplicationRecovery() throws Exception { // simulate log aggregation completion app.handle(new ApplicationEvent(app.getAppId(), ApplicationEventType.APPLICATION_RESOURCES_CLEANEDUP)); - assertEquals(app.getApplicationState(), ApplicationState.FINISHED); + assertThat(app.getApplicationState()).isEqualTo(ApplicationState.FINISHED); app.handle(new ApplicationEvent(app.getAppId(), ApplicationEventType.APPLICATION_LOG_HANDLING_FINISHED)); @@ -362,7 +363,7 @@ public void testNMRecoveryForAppFinishedWithLogAggregationFailure() app.handle(new ApplicationEvent(app.getAppId(), ApplicationEventType.APPLICATION_RESOURCES_CLEANEDUP)); - assertEquals(app.getApplicationState(), ApplicationState.FINISHED); + assertThat(app.getApplicationState()).isEqualTo(ApplicationState.FINISHED); // application is still in NM context. assertEquals(1, context.getApplications().size()); @@ -386,7 +387,7 @@ public void testNMRecoveryForAppFinishedWithLogAggregationFailure() // is needed. app.handle(new ApplicationEvent(app.getAppId(), ApplicationEventType.APPLICATION_RESOURCES_CLEANEDUP)); - assertEquals(app.getApplicationState(), ApplicationState.FINISHED); + assertThat(app.getApplicationState()).isEqualTo(ApplicationState.FINISHED); // simulate log aggregation failed. app.handle(new ApplicationEvent(app.getAppId(), @@ -528,7 +529,8 @@ public void testContainerSchedulerRecovery() throws Exception { ResourceUtilization utilization = ResourceUtilization.newInstance(1024, 2048, 1.0F); - assertEquals(cm.getContainerScheduler().getNumRunningContainers(), 1); + assertThat(cm.getContainerScheduler().getNumRunningContainers()). + isEqualTo(1); assertEquals(utilization, cm.getContainerScheduler().getCurrentUtilization()); @@ -544,7 +546,8 @@ public void testContainerSchedulerRecovery() throws Exception { assertNotNull(app); waitForNMContainerState(cm, cid, ContainerState.RUNNING); - assertEquals(cm.getContainerScheduler().getNumRunningContainers(), 1); + assertThat(cm.getContainerScheduler().getNumRunningContainers()). + isEqualTo(1); assertEquals(utilization, cm.getContainerScheduler().getCurrentUtilization()); cm.stop(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java index e048577219763..bda78032e16c7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java @@ -18,9 +18,11 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher; +import static org.assertj.core.api.Assertions.assertThat; import static org.apache.hadoop.test.PlatformAssumptions.assumeWindows; import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.*; @@ -124,7 +126,6 @@ import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper; import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin; import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin; -import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Assume; import org.junit.Before; @@ -212,7 +213,7 @@ public void testSpecialCharSymlinks() throws IOException { = new Shell.ShellCommandExecutor(new String[]{tempFile.getAbsolutePath()}, tmpDir); shexc.execute(); - assertEquals(shexc.getExitCode(), 0); + assertThat(shexc.getExitCode()).isEqualTo(0); //Capture output from prelaunch.out List output = Files.readAllLines(Paths.get(localLogDir.getAbsolutePath(), ContainerLaunch.CONTAINER_PRE_LAUNCH_STDOUT), @@ -1485,7 +1486,7 @@ public void testWindowsShellScriptBuilderCommand() throws IOException { "X", Shell.WINDOWS_MAX_SHELL_LENGTH -callCmd.length() + 1))); fail("longCommand was expected to throw"); } catch(IOException e) { - assertThat(e.getMessage(), CoreMatchers.containsString(expectedMessage)); + assertThat(e).hasMessageContaining(expectedMessage); } // Composite tests, from parts: less, exact and + @@ -1507,7 +1508,7 @@ public void testWindowsShellScriptBuilderCommand() throws IOException { org.apache.commons.lang3.StringUtils.repeat("X", 2048 - callCmd.length()))); fail("long commands was expected to throw"); } catch(IOException e) { - assertThat(e.getMessage(), CoreMatchers.containsString(expectedMessage)); + assertThat(e).hasMessageContaining(expectedMessage); } } @@ -1530,7 +1531,7 @@ public void testWindowsShellScriptBuilderEnv() throws IOException { "A", Shell.WINDOWS_MAX_SHELL_LENGTH - ("@set somekey=").length()) + 1); fail("long env was expected to throw"); } catch(IOException e) { - assertThat(e.getMessage(), CoreMatchers.containsString(expectedMessage)); + assertThat(e).hasMessageContaining(expectedMessage); } } @@ -1555,8 +1556,8 @@ public void testWindowsShellScriptBuilderMkdir() throws IOException { "X", (Shell.WINDOWS_MAX_SHELL_LENGTH - mkDirCmd.length())/2 +1))); fail("long mkdir was expected to throw"); } catch(IOException e) { - assertThat(e.getMessage(), CoreMatchers.containsString(expectedMessage)); - } + assertThat(e).hasMessageContaining(expectedMessage); + } } @Test (timeout = 10000) @@ -1586,7 +1587,7 @@ public void testWindowsShellScriptBuilderLink() throws IOException { "Y", (Shell.WINDOWS_MAX_SHELL_LENGTH - linkCmd.length())/2) + 1)); fail("long link was expected to throw"); } catch(IOException e) { - assertThat(e.getMessage(), CoreMatchers.containsString(expectedMessage)); + assertThat(e).hasMessageContaining(expectedMessage); } } @@ -1747,7 +1748,7 @@ public void testDebuggingInformation() throws IOException { new String[] { tempFile.getAbsolutePath() }, tmpDir); shexc.execute(); - assertEquals(shexc.getExitCode(), 0); + assertThat(shexc.getExitCode()).isEqualTo(0); File directorInfo = new File(localLogDir, ContainerExecutor.DIRECTORY_CONTENTS); File scriptCopy = new File(localLogDir, tempFile.getName()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java index 94563035b0145..171bfafc4fe7f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java @@ -20,6 +20,8 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.nodemanager.Context; @@ -71,7 +73,7 @@ public void testOutboundBandwidthHandler() { List resourceHandlers = resourceHandlerChain .getResourceHandlerList(); //Exactly one resource handler in chain - Assert.assertEquals(resourceHandlers.size(), 1); + assertThat(resourceHandlers).hasSize(1); //Same instance is expected to be in the chain. Assert.assertTrue(resourceHandlers.get(0) == resourceHandler); } else { @@ -102,7 +104,7 @@ public void testDiskResourceHandler() throws Exception { List resourceHandlers = resourceHandlerChain.getResourceHandlerList(); // Exactly one resource handler in chain - Assert.assertEquals(resourceHandlers.size(), 1); + assertThat(resourceHandlers).hasSize(1); // Same instance is expected to be in the chain. Assert.assertTrue(resourceHandlers.get(0) == handler); } else { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java index 9f5f608fedba6..8871bf6d1e41a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; @@ -1325,15 +1326,15 @@ private void doLocalization(ResourceLocalizationService spyService, // hence its not removed despite ref cnt being 0. LocalizedResource rsrc1 = tracker.getLocalizedResource(req1); assertNotNull(rsrc1); - assertEquals(rsrc1.getState(), ResourceState.LOCALIZED); - assertEquals(rsrc1.getRefCount(), 0); + assertThat(rsrc1.getState()).isEqualTo(ResourceState.LOCALIZED); + assertThat(rsrc1.getRefCount()).isEqualTo(0); // Container c1 was killed but this resource is referenced by container c2 // as well hence its ref cnt is 1. LocalizedResource rsrc2 = tracker.getLocalizedResource(req2); assertNotNull(rsrc2); - assertEquals(rsrc2.getState(), ResourceState.DOWNLOADING); - assertEquals(rsrc2.getRefCount(), 1); + assertThat(rsrc2.getState()).isEqualTo(ResourceState.DOWNLOADING); + assertThat(rsrc2.getRefCount()).isEqualTo(1); // As container c1 was killed and this resource was not referenced by any // other container, hence its removed. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java index 000b73bf12bbc..796c7e74ce5f4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -870,7 +871,7 @@ public LogAggregationFileController getLogAggregationFileController( }; checkEvents(appEventHandler, expectedEvents, false, "getType", "getApplicationID", "getDiagnostic"); - Assert.assertEquals(logAggregationService.getInvalidTokenApps().size(), 1); + assertThat(logAggregationService.getInvalidTokenApps()).hasSize(1); // verify trying to collect logs for containers/apps we don't know about // doesn't blow up and tear down the NM logAggregationService.handle(new LogHandlerContainerFinishedEvent( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/com/nvidia/TestNvidiaGPUPluginForRuntimeV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/com/nvidia/TestNvidiaGPUPluginForRuntimeV2.java index 595ee08a5eaaf..7b848cfc5c4d4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/com/nvidia/TestNvidiaGPUPluginForRuntimeV2.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/com/nvidia/TestNvidiaGPUPluginForRuntimeV2.java @@ -39,6 +39,7 @@ import java.util.Set; import java.util.TreeSet; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyMap; import static org.mockito.ArgumentMatchers.anySet; @@ -265,7 +266,7 @@ public void testTopologySchedulingWithPackPolicy() throws Exception { reset(spyPlugin); Set allocation = spyPlugin.allocateDevices(copyAvailableDevices, 1, env); - Assert.assertEquals(allocation.size(), 1); + assertThat(allocation).hasSize(1); verify(spyPlugin).basicSchedule(anySet(), anyInt(), anySet()); Assert.assertFalse(spyPlugin.isTopoInitialized()); @@ -273,7 +274,7 @@ public void testTopologySchedulingWithPackPolicy() throws Exception { reset(spyPlugin); allocation = spyPlugin.allocateDevices(allDevices, 1, env); // ensure no topology scheduling needed - Assert.assertEquals(allocation.size(), 1); + assertThat(allocation).hasSize(1); verify(spyPlugin).basicSchedule(anySet(), anyInt(), anySet()); reset(spyPlugin); // Case 2. allocate all available @@ -285,13 +286,13 @@ public void testTopologySchedulingWithPackPolicy() throws Exception { int count = 2; Map pairToWeight = spyPlugin.getDevicePairToWeight(); allocation = spyPlugin.allocateDevices(allDevices, count, env); - Assert.assertEquals(allocation.size(), count); + assertThat(allocation).hasSize(count); // the costTable should be init and used topology scheduling verify(spyPlugin).initCostTable(); Assert.assertTrue(spyPlugin.isTopoInitialized()); verify(spyPlugin).topologyAwareSchedule(anySet(), anyInt(), anyMap(), anySet(), anyMap()); - Assert.assertEquals(allocation.size(), count); + assertThat(allocation).hasSize(count); Device[] allocatedDevices = allocation.toArray(new Device[count]); // Check weights @@ -302,13 +303,13 @@ public void testTopologySchedulingWithPackPolicy() throws Exception { reset(spyPlugin); count = 3; allocation = spyPlugin.allocateDevices(allDevices, count, env); - Assert.assertEquals(allocation.size(), count); + assertThat(allocation).hasSize(count); // the costTable should be init and used topology scheduling verify(spyPlugin, times(0)).initCostTable(); Assert.assertTrue(spyPlugin.isTopoInitialized()); verify(spyPlugin).topologyAwareSchedule(anySet(), anyInt(), anyMap(), anySet(), anyMap()); - Assert.assertEquals(allocation.size(), count); + assertThat(allocation).hasSize(count); allocatedDevices = allocation.toArray(new Device[count]); // check weights @@ -327,13 +328,13 @@ public void testTopologySchedulingWithPackPolicy() throws Exception { iterator.remove(); count = 2; allocation = spyPlugin.allocateDevices(allDevices, count, env); - Assert.assertEquals(allocation.size(), count); + assertThat(allocation).hasSize(count); // the costTable should be init and used topology scheduling verify(spyPlugin, times(0)).initCostTable(); Assert.assertTrue(spyPlugin.isTopoInitialized()); verify(spyPlugin).topologyAwareSchedule(anySet(), anyInt(), anyMap(), anySet(), anyMap()); - Assert.assertEquals(allocation.size(), count); + assertThat(allocation).hasSize(count); allocatedDevices = allocation.toArray(new Device[count]); // check weights @@ -377,13 +378,13 @@ public void testTopologySchedulingWithSpreadPolicy() throws Exception { int count = 2; Map pairToWeight = spyPlugin.getDevicePairToWeight(); allocation = spyPlugin.allocateDevices(allDevices, count, env); - Assert.assertEquals(allocation.size(), count); + assertThat(allocation).hasSize(count); // the costTable should be init and used topology scheduling verify(spyPlugin).initCostTable(); Assert.assertTrue(spyPlugin.isTopoInitialized()); verify(spyPlugin).topologyAwareSchedule(anySet(), anyInt(), anyMap(), anySet(), anyMap()); - Assert.assertEquals(allocation.size(), count); + assertThat(allocation).hasSize(count); Device[] allocatedDevices = allocation.toArray(new Device[count]); // Check weights @@ -394,13 +395,13 @@ public void testTopologySchedulingWithSpreadPolicy() throws Exception { reset(spyPlugin); count = 3; allocation = spyPlugin.allocateDevices(allDevices, count, env); - Assert.assertEquals(allocation.size(), count); + assertThat(allocation).hasSize(count); // the costTable should be init and used topology scheduling verify(spyPlugin, times(0)).initCostTable(); Assert.assertTrue(spyPlugin.isTopoInitialized()); verify(spyPlugin).topologyAwareSchedule(anySet(), anyInt(), anyMap(), anySet(), anyMap()); - Assert.assertEquals(allocation.size(), count); + assertThat(allocation).hasSize(count); allocatedDevices = allocation.toArray(new Device[count]); // check weights @@ -419,13 +420,13 @@ public void testTopologySchedulingWithSpreadPolicy() throws Exception { iterator.remove(); count = 2; allocation = spyPlugin.allocateDevices(allDevices, count, env); - Assert.assertEquals(allocation.size(), count); + assertThat(allocation).hasSize(count); // the costTable should be init and used topology scheduling verify(spyPlugin, times(0)).initCostTable(); Assert.assertTrue(spyPlugin.isTopoInitialized()); verify(spyPlugin).topologyAwareSchedule(anySet(), anyInt(), anyMap(), anySet(), anyMap()); - Assert.assertEquals(allocation.size(), count); + assertThat(allocation).hasSize(count); allocatedDevices = allocation.toArray(new Device[count]); // check weights @@ -574,7 +575,7 @@ public void testTopologySchedulingPerformanceWithPackPolicyWithNVLink() report.readFromFile(); ArrayList dataSet = report.getDataSet(); - Assert.assertEquals(dataSet.size(), 2952); + assertThat(dataSet).hasSize(2952); String[] allModels = {"alexnet", "resnet50", "vgg16", "inception3"}; int[] batchSizes = {32, 64, 128}; int[] gpuCounts = {2, 3, 4, 5, 6, 7}; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java index 157e5d39bed93..bd86f596bc8d6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.nodemanager.recovery; +import static org.assertj.core.api.Assertions.assertThat; import static org.fusesource.leveldbjni.JniDBFactory.bytes; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -36,6 +37,7 @@ import java.io.File; import java.io.IOException; + import java.io.Serializable; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -1350,9 +1352,9 @@ public void testUnexpectedKeyDoesntThrowException() throws IOException { @Test public void testAMRMProxyStorage() throws IOException { RecoveredAMRMProxyState state = stateStore.loadAMRMProxyState(); - assertEquals(state.getCurrentMasterKey(), null); - assertEquals(state.getNextMasterKey(), null); - assertEquals(state.getAppContexts().size(), 0); + assertThat(state.getCurrentMasterKey()).isNull(); + assertThat(state.getNextMasterKey()).isNull(); + assertThat(state.getAppContexts()).isEmpty(); ApplicationId appId1 = ApplicationId.newInstance(1, 1); ApplicationId appId2 = ApplicationId.newInstance(1, 2); @@ -1384,18 +1386,18 @@ public void testAMRMProxyStorage() throws IOException { state = stateStore.loadAMRMProxyState(); assertEquals(state.getCurrentMasterKey(), secretManager.getCurrentMasterKeyData().getMasterKey()); - assertEquals(state.getNextMasterKey(), null); - assertEquals(state.getAppContexts().size(), 2); + assertThat(state.getNextMasterKey()).isNull(); + assertThat(state.getAppContexts()).hasSize(2); // app1 Map map = state.getAppContexts().get(attemptId1); assertNotEquals(map, null); - assertEquals(map.size(), 2); + assertThat(map).hasSize(2); assertTrue(Arrays.equals(map.get(key1), data1)); assertTrue(Arrays.equals(map.get(key2), data2)); // app2 map = state.getAppContexts().get(attemptId2); assertNotEquals(map, null); - assertEquals(map.size(), 2); + assertThat(map).hasSize(2); assertTrue(Arrays.equals(map.get(key1), data1)); assertTrue(Arrays.equals(map.get(key2), data2)); @@ -1414,14 +1416,14 @@ public void testAMRMProxyStorage() throws IOException { assertEquals(state.getAppContexts().size(), 2); // app1 map = state.getAppContexts().get(attemptId1); - assertNotEquals(map, null); - assertEquals(map.size(), 2); + assertThat(map).isNotNull(); + assertThat(map).hasSize(2); assertTrue(Arrays.equals(map.get(key1), data1)); assertTrue(Arrays.equals(map.get(key2), data2)); // app2 map = state.getAppContexts().get(attemptId2); - assertNotEquals(map, null); - assertEquals(map.size(), 1); + assertThat(map).isNotNull(); + assertThat(map).hasSize(1); assertTrue(Arrays.equals(map.get(key2), data2)); // Activate next master key and remove all entries of app1 @@ -1434,12 +1436,12 @@ public void testAMRMProxyStorage() throws IOException { state = stateStore.loadAMRMProxyState(); assertEquals(state.getCurrentMasterKey(), secretManager.getCurrentMasterKeyData().getMasterKey()); - assertEquals(state.getNextMasterKey(), null); - assertEquals(state.getAppContexts().size(), 1); + assertThat(state.getNextMasterKey()).isNull(); + assertThat(state.getAppContexts()).hasSize(1); // app2 only map = state.getAppContexts().get(attemptId2); - assertNotEquals(map, null); - assertEquals(map.size(), 1); + assertThat(map).isNotNull(); + assertThat(map).hasSize(1); assertTrue(Arrays.equals(map.get(key2), data2)); } finally { secretManager.stop(); 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/TestNMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java index 62774f5d4365c..740af8f9748ab 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java @@ -96,6 +96,7 @@ import java.util.List; import java.util.Map; +import static org.assertj.core.api.Assertions.assertThat; import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -704,7 +705,7 @@ private void testContainerLogs(WebResource r, ContainerId containerId) List logMeta = responseList.get(0) .getContainerLogsInfo(); assertTrue(logMeta.size() == 1); - assertEquals(logMeta.get(0).getFileName(), filename); + assertThat(logMeta.get(0).getFileName()).isEqualTo(filename); // now create an aggregated log in Remote File system File tempLogDir = new File("target", @@ -724,19 +725,19 @@ private void testContainerLogs(WebResource r, ContainerId containerId) assertEquals(200, response.getStatus()); responseList = response.getEntity(new GenericType< List>(){}); - assertEquals(responseList.size(), 2); + assertThat(responseList).hasSize(2); for (ContainerLogsInfo logInfo : responseList) { if(logInfo.getLogType().equals( ContainerLogAggregationType.AGGREGATED.toString())) { List meta = logInfo.getContainerLogsInfo(); assertTrue(meta.size() == 1); - assertEquals(meta.get(0).getFileName(), aggregatedLogFile); + assertThat(meta.get(0).getFileName()).isEqualTo(aggregatedLogFile); } else { assertEquals(logInfo.getLogType(), ContainerLogAggregationType.LOCAL.toString()); List meta = logInfo.getContainerLogsInfo(); assertTrue(meta.size() == 1); - assertEquals(meta.get(0).getFileName(), filename); + assertThat(meta.get(0).getFileName()).isEqualTo(filename); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml index f40ca0efe6de7..45eca49c18d1c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml @@ -57,6 +57,11 @@ junit test + + org.assertj + assertj-core + test + org.mockito mockito-core diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java index 9e861bd3a395d..a360ed2b6524d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java @@ -229,8 +229,13 @@ public OpportunisticContainerAllocatorAMService(RMContext rmContext, YarnScheduler scheduler) { super(OpportunisticContainerAllocatorAMService.class.getName(), rmContext, scheduler); + int maxAllocationsPerAMHeartbeat = rmContext.getYarnConfiguration().getInt( + YarnConfiguration.OPP_CONTAINER_MAX_ALLOCATIONS_PER_AM_HEARTBEAT, + YarnConfiguration. + DEFAULT_OPP_CONTAINER_MAX_ALLOCATIONS_PER_AM_HEARTBEAT); this.oppContainerAllocator = new OpportunisticContainerAllocator( - rmContext.getContainerTokenSecretManager()); + rmContext.getContainerTokenSecretManager(), + maxAllocationsPerAMHeartbeat); this.k = rmContext.getYarnConfiguration().getInt( YarnConfiguration.OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED, YarnConfiguration.DEFAULT_OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java index 1438e25c91b91..eeeed48bbac5e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java @@ -473,10 +473,15 @@ private void putEntity(TimelineEntity entity, ApplicationId appId) { } TimelineCollector timelineCollector = rmTimelineCollectorManager.get(appId); - TimelineEntities entities = new TimelineEntities(); - entities.addEntity(entity); - timelineCollector.putEntities(entities, - UserGroupInformation.getCurrentUser()); + if (timelineCollector != null) { + TimelineEntities entities = new TimelineEntities(); + entities.addEntity(entity); + timelineCollector.putEntities(entities, + UserGroupInformation.getCurrentUser()); + } else { + LOG.debug("Cannot find active collector while publishing entity " + + entity); + } } catch (IOException e) { LOG.error("Error when publishing entity " + entity); LOG.debug("Error when publishing entity {}", entity, e); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java index 6049c11dbb3fd..3eed0be25ba97 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -1788,7 +1789,7 @@ public void testListReservationsByTimeIntervalContainingNoReservations() { // Ensure all reservations are filtered out. Assert.assertNotNull(response); - Assert.assertEquals(response.getReservationAllocationState().size(), 0); + assertThat(response.getReservationAllocationState()).isEmpty(); duration = 30000; deadline = sRequest.getReservationDefinition().getDeadline(); @@ -1808,7 +1809,7 @@ public void testListReservationsByTimeIntervalContainingNoReservations() { // Ensure all reservations are filtered out. Assert.assertNotNull(response); - Assert.assertEquals(response.getReservationAllocationState().size(), 0); + assertThat(response.getReservationAllocationState()).isEmpty(); arrival = clock.getTime(); // List reservations, search by end time before the reservation start @@ -1826,7 +1827,7 @@ public void testListReservationsByTimeIntervalContainingNoReservations() { // Ensure all reservations are filtered out. Assert.assertNotNull(response); - Assert.assertEquals(response.getReservationAllocationState().size(), 0); + assertThat(response.getReservationAllocationState()).isEmpty(); // List reservations, search by very small end time. request = ReservationListRequest @@ -1841,7 +1842,7 @@ public void testListReservationsByTimeIntervalContainingNoReservations() { // Ensure all reservations are filtered out. Assert.assertNotNull(response); - Assert.assertEquals(response.getReservationAllocationState().size(), 0); + assertThat(response.getReservationAllocationState()).isEmpty(); rm.stop(); } @@ -2012,7 +2013,7 @@ protected ClientRMService createClientRMService() { Arrays.asList(node1A))); Assert.assertTrue(labelsToNodes.get(labelZ.getName()).containsAll( Arrays.asList(node1B, node3B))); - Assert.assertEquals(labelsToNodes.get(labelY.getName()), null); + assertThat(labelsToNodes.get(labelY.getName())).isNull(); rpc.stopProxy(client, conf); rm.close(); @@ -2113,10 +2114,10 @@ protected ClientRMService createClientRMService() { client.getAttributesToNodes(request); Map> attrs = response.getAttributesToNodes(); - Assert.assertEquals(response.getAttributesToNodes().size(), 4); - Assert.assertEquals(attrs.get(dist.getAttributeKey()).size(), 2); - Assert.assertEquals(attrs.get(os.getAttributeKey()).size(), 1); - Assert.assertEquals(attrs.get(gpu.getAttributeKey()).size(), 1); + assertThat(response.getAttributesToNodes()).hasSize(4); + assertThat(attrs.get(dist.getAttributeKey())).hasSize(2); + assertThat(attrs.get(os.getAttributeKey())).hasSize(1); + assertThat(attrs.get(gpu.getAttributeKey())).hasSize(1); Assert.assertTrue(findHostnameAndValInMapping(node1, "3_0_2", attrs.get(dist.getAttributeKey()))); Assert.assertTrue(findHostnameAndValInMapping(node2, "3_0_2", @@ -2130,7 +2131,7 @@ protected ClientRMService createClientRMService() { client.getAttributesToNodes(request2); Map> attrs2 = response2.getAttributesToNodes(); - Assert.assertEquals(attrs2.size(), 1); + assertThat(attrs2).hasSize(1); Assert.assertTrue(findHostnameAndValInMapping(node2, "docker0", attrs2.get(docker.getAttributeKey()))); @@ -2141,7 +2142,7 @@ protected ClientRMService createClientRMService() { client.getAttributesToNodes(request3); Map> attrs3 = response3.getAttributesToNodes(); - Assert.assertEquals(attrs3.size(), 2); + assertThat(attrs3).hasSize(2); Assert.assertTrue(findHostnameAndValInMapping(node1, "windows64", attrs3.get(os.getAttributeKey()))); Assert.assertTrue(findHostnameAndValInMapping(node2, "docker0", diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestKillApplicationWithRMHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestKillApplicationWithRMHA.java index 30443d7506392..979d8c7b974c9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestKillApplicationWithRMHA.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestKillApplicationWithRMHA.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.fail; import java.io.IOException; @@ -150,7 +151,7 @@ public void testKillAppWhenFailOverHappensDuringApplicationKill() MockAM am0 = launchAM(app0, rm1, nm1); // ensure that the app is in running state - Assert.assertEquals(app0.getState(), RMAppState.RUNNING); + assertThat(app0.getState()).isEqualTo(RMAppState.RUNNING); // kill the app. rm1.killApp(app0.getApplicationId()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java index 01c89d389b78f..6659f107effcb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -608,7 +609,7 @@ private void verifyServiceACLsRefresh(ServiceAuthorizationManager manager, Assert.assertEquals(accessList.getAclString(), aclString); } else { - Assert.assertEquals(accessList.getAclString(), "*"); + assertThat(accessList.getAclString()).isEqualTo("*"); } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java index 3b3bd4b424cb0..4b11296efbdee 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java @@ -22,6 +22,7 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -381,14 +382,14 @@ public void testHAIDLookup() { rm = new MockRM(conf); rm.init(conf); - assertEquals(conf.get(YarnConfiguration.RM_HA_ID), RM2_NODE_ID); + assertThat(conf.get(YarnConfiguration.RM_HA_ID)).isEqualTo(RM2_NODE_ID); //test explicitly lookup HA-ID configuration.set(YarnConfiguration.RM_HA_ID, RM1_NODE_ID); conf = new YarnConfiguration(configuration); rm = new MockRM(conf); rm.init(conf); - assertEquals(conf.get(YarnConfiguration.RM_HA_ID), RM1_NODE_ID); + assertThat(conf.get(YarnConfiguration.RM_HA_ID)).isEqualTo(RM1_NODE_ID); //test if RM_HA_ID can not be found configuration diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java index f09850afaf9b6..87a9423522662 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.isA; @@ -609,7 +610,7 @@ public void testRMRestartWaitForPreviousAMToFinish() throws Exception { MockAM am2 = launchAM(app1, rm1, nm1); Assert.assertEquals(1, rmAppState.size()); - Assert.assertEquals(app1.getState(), RMAppState.RUNNING); + assertThat(app1.getState()).isEqualTo(RMAppState.RUNNING); Assert.assertEquals(app1.getAppAttempts() .get(app1.getCurrentAppAttempt().getAppAttemptId()) .getAppAttemptState(), RMAppAttemptState.RUNNING); @@ -665,7 +666,7 @@ public void testRMRestartWaitForPreviousAMToFinish() throws Exception { rmApp = rm3.getRMContext().getRMApps().get(app1.getApplicationId()); // application should be in ACCEPTED state rm3.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); - Assert.assertEquals(rmApp.getState(), RMAppState.ACCEPTED); + assertThat(rmApp.getState()).isEqualTo(RMAppState.ACCEPTED); // new attempt should not be started Assert.assertEquals(3, rmApp.getAppAttempts().size()); // am1 and am2 attempts should be in FAILED state where as am3 should be diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMServerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMServerUtils.java index 9c233b9cf9e85..78fa90ded2fa4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMServerUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMServerUtils.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.assertj.core.api.Assertions.assertThat; import static org.apache.hadoop.yarn.api.records.ContainerUpdateType.INCREASE_RESOURCE; import static org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils.RESOURCE_OUTSIDE_ALLOWED_RANGE; @@ -140,22 +141,22 @@ public void testQueryRMNodes() throws Exception { List result = RMServerUtils.queryRMNodes(rmContext, EnumSet.of(NodeState.SHUTDOWN)); Assert.assertTrue(result.size() != 0); - Assert.assertEquals(result.get(0), rmNode1); + assertThat(result.get(0)).isEqualTo(rmNode1); when(rmNode1.getState()).thenReturn(NodeState.DECOMMISSIONED); result = RMServerUtils.queryRMNodes(rmContext, EnumSet.of(NodeState.DECOMMISSIONED)); Assert.assertTrue(result.size() != 0); - Assert.assertEquals(result.get(0), rmNode1); + assertThat(result.get(0)).isEqualTo(rmNode1); when(rmNode1.getState()).thenReturn(NodeState.LOST); result = RMServerUtils.queryRMNodes(rmContext, EnumSet.of(NodeState.LOST)); Assert.assertTrue(result.size() != 0); - Assert.assertEquals(result.get(0), rmNode1); + assertThat(result.get(0)).isEqualTo(rmNode1); when(rmNode1.getState()).thenReturn(NodeState.REBOOTED); result = RMServerUtils.queryRMNodes(rmContext, EnumSet.of(NodeState.REBOOTED)); Assert.assertTrue(result.size() != 0); - Assert.assertEquals(result.get(0), rmNode1); + assertThat(result.get(0)).isEqualTo(rmNode1); } @Test diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java index fa5f135153b81..d503cb44fb895 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java @@ -100,6 +100,8 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; + +import static org.assertj.core.api.Assertions.assertThat; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler @@ -263,7 +265,8 @@ public void testSchedulerRecovery() throws Exception { scheduler.getRMContainer(amContainer.getContainerId()))); assertTrue(schedulerAttempt.getLiveContainers().contains( scheduler.getRMContainer(runningContainer.getContainerId()))); - assertEquals(schedulerAttempt.getCurrentConsumption(), usedResources); + assertThat(schedulerAttempt.getCurrentConsumption()). + isEqualTo(usedResources); // *********** check appSchedulingInfo state *********** assertEquals((1L << 40) + 1L, schedulerAttempt.getNewContainerId()); @@ -421,7 +424,8 @@ public void testDynamicQueueRecovery() throws Exception { .contains(scheduler.getRMContainer(amContainer.getContainerId()))); assertTrue(schedulerAttempt.getLiveContainers() .contains(scheduler.getRMContainer(runningContainer.getContainerId()))); - assertEquals(schedulerAttempt.getCurrentConsumption(), usedResources); + assertThat(schedulerAttempt.getCurrentConsumption()). + isEqualTo(usedResources); // *********** check appSchedulingInfo state *********** assertEquals((1L << 40) + 1L, schedulerAttempt.getNewContainerId()); @@ -775,8 +779,9 @@ private void verifyAppRecoveryWithWrongQueueConfig( ApplicationReport report = rm2.getApplicationReport(app.getApplicationId()); assertEquals(report.getFinalApplicationStatus(), FinalApplicationStatus.KILLED); - assertEquals(report.getYarnApplicationState(), YarnApplicationState.KILLED); - assertEquals(report.getDiagnostics(), diagnostics); + assertThat(report.getYarnApplicationState()). + isEqualTo(YarnApplicationState.KILLED); + assertThat(report.getDiagnostics()).isEqualTo(diagnostics); //Reload previous state with cloned app sub context object RMState newState = memStore2.reloadStateWithClonedAppSubCtxt(state); @@ -1730,7 +1735,8 @@ public void testDynamicAutoCreatedQueueRecovery(String user, String queueName) .contains(scheduler.getRMContainer(amContainer.getContainerId()))); assertTrue(schedulerAttempt.getLiveContainers() .contains(scheduler.getRMContainer(runningContainer.getContainerId()))); - assertEquals(schedulerAttempt.getCurrentConsumption(), usedResources); + assertThat(schedulerAttempt.getCurrentConsumption()). + isEqualTo(usedResources); // *********** check appSchedulingInfo state *********** assertEquals((1L << 40) + 1L, schedulerAttempt.getNewContainerId()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java index 64c90637f2f37..35f55c6d476c0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.metrics; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -290,7 +291,7 @@ public void testPublishApplicationMetrics() throws Exception { } else if (event.getEventType().equals( ApplicationMetricsConstants.STATE_UPDATED_EVENT_TYPE)) { hasStateUpdateEvent = true; - Assert.assertEquals(event.getTimestamp(), stateUpdateTimeStamp); + assertThat(event.getTimestamp()).isEqualTo(stateUpdateTimeStamp); Assert.assertEquals(YarnApplicationState.RUNNING.toString(), event .getEventInfo().get( ApplicationMetricsConstants.STATE_EVENT_INFO)); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java index 5e1a224961ceb..76e8f0ee8cf93 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java @@ -28,12 +28,17 @@ import java.io.File; import java.io.FileReader; import java.io.IOException; +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.ConcurrentMap; +import org.apache.log4j.AppenderSkeleton; +import org.apache.log4j.Logger; +import org.apache.log4j.spi.LoggingEvent; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.Path; @@ -293,6 +298,48 @@ public void testPublishContainerMetrics() throws Exception { TimelineServiceHelper.invertLong(containerId.getContainerId())); } + @Test(timeout = 10000) + public void testPutEntityWhenNoCollector() throws Exception { + // Validating the logs as DrainDispatcher won't throw exception + class TestAppender extends AppenderSkeleton { + private final List log = new ArrayList<>(); + + @Override + public boolean requiresLayout() { + return false; + } + + @Override + protected void append(final LoggingEvent loggingEvent) { + log.add(loggingEvent); + } + + @Override + public void close() { + } + + public List getLog() { + return new ArrayList<>(log); + } + } + + TestAppender appender = new TestAppender(); + final Logger logger = Logger.getRootLogger(); + logger.addAppender(appender); + + try { + RMApp app = createRMApp(ApplicationId.newInstance(0, 1)); + metricsPublisher.appCreated(app, app.getStartTime()); + dispatcher.await(); + for (LoggingEvent event : appender.getLog()) { + assertFalse("Dispatcher Crashed", + event.getRenderedMessage().contains("Error in dispatcher thread")); + } + } finally { + logger.removeAppender(appender); + } + } + private RMApp createAppAndRegister(ApplicationId appId) { RMApp app = createRMApp(appId); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyPreemptToBalance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyPreemptToBalance.java index 6f67e4b13cada..8c0eee6865e6c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyPreemptToBalance.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyPreemptToBalance.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.util.Map; import java.util.Set; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.Mockito.times; @@ -230,7 +231,8 @@ public void testPreemptionToBalanceWithConfiguredTimeout() throws IOException { FifoCandidatesSelector pcs = (FifoCandidatesSelector) pc.getKey(); if (pcs.getAllowQueuesBalanceAfterAllQueuesSatisfied() == true) { hasFifoSelector = true; - assertEquals(pcs.getMaximumKillWaitTimeMs(), FB_MAX_BEFORE_KILL); + assertThat(pcs.getMaximumKillWaitTimeMs()). + isEqualTo(FB_MAX_BEFORE_KILL); } } } 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/TestFileSystemNodeAttributeStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java index a53b88e9629b8..a05cf3eb5cc5e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.yarn.server.resourcemanager.nodelabels; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -139,8 +140,8 @@ public void testRecoverWithMirror() throws Exception { toAddAttributes); Map attrs = mgr.getAttributesForNode("host0"); - Assert.assertEquals(attrs.size(), 1); - Assert.assertEquals(attrs.keySet().toArray()[0], docker); + assertThat(attrs).hasSize(1); + assertThat(attrs.keySet().toArray()[0]).isEqualTo(docker); mgr.stop(); // Start new attribute manager with same path @@ -154,8 +155,8 @@ public void testRecoverWithMirror() throws Exception { Assert.assertEquals("host1 size", 1, mgr.getAttributesForNode("host1").size()); attrs = mgr.getAttributesForNode("host0"); - Assert.assertEquals(attrs.size(), 1); - Assert.assertEquals(attrs.keySet().toArray()[0], docker); + assertThat(attrs).hasSize(1); + assertThat(attrs.keySet().toArray()[0]).isEqualTo(docker); //------host0---- // current - docker // replace - gpu @@ -181,8 +182,8 @@ public void testRecoverWithMirror() throws Exception { Assert.assertEquals("host1 size", 2, mgr.getAttributesForNode("host1").size()); attrs = mgr.getAttributesForNode("host0"); - Assert.assertEquals(attrs.size(), 1); - Assert.assertEquals(attrs.keySet().toArray()[0], gpu); + assertThat(attrs).hasSize(1); + assertThat(attrs.keySet().toArray()[0]).isEqualTo(gpu); attrs = mgr.getAttributesForNode("host1"); Assert.assertTrue(attrs.keySet().contains(docker)); Assert.assertTrue(attrs.keySet().contains(gpu)); 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/TestRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java index 1da6f93f664e1..5c9b073defac2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.yarn.server.resourcemanager.nodelabels; + +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -88,37 +90,37 @@ public void testGetLabelResourceWhenNodeActiveDeactive() throws Exception { mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("p1"), toNodeId("n2"), toSet("p2"), toNodeId("n3"), toSet("p3"))); - Assert.assertEquals(mgr.getResourceByLabel("p1", null), EMPTY_RESOURCE); - Assert.assertEquals(mgr.getResourceByLabel("p2", null), EMPTY_RESOURCE); - Assert.assertEquals(mgr.getResourceByLabel("p3", null), EMPTY_RESOURCE); - Assert.assertEquals(mgr.getResourceByLabel(RMNodeLabelsManager.NO_LABEL, null), - EMPTY_RESOURCE); + assertThat(mgr.getResourceByLabel("p1", null)).isEqualTo(EMPTY_RESOURCE); + assertThat(mgr.getResourceByLabel("p2", null)).isEqualTo(EMPTY_RESOURCE); + assertThat(mgr.getResourceByLabel("p3", null)).isEqualTo(EMPTY_RESOURCE); + assertThat(mgr.getResourceByLabel(RMNodeLabelsManager.NO_LABEL, null)). + isEqualTo(EMPTY_RESOURCE); // active two NM to n1, one large and one small mgr.activateNode(NodeId.newInstance("n1", 1), SMALL_RESOURCE); mgr.activateNode(NodeId.newInstance("n1", 2), LARGE_NODE); - Assert.assertEquals(mgr.getResourceByLabel("p1", null), + assertThat(mgr.getResourceByLabel("p1", null)).isEqualTo( Resources.add(SMALL_RESOURCE, LARGE_NODE)); // check add labels multiple times shouldn't overwrite // original attributes on labels like resource mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p1", "p4")); - Assert.assertEquals(mgr.getResourceByLabel("p1", null), + assertThat(mgr.getResourceByLabel("p1", null)).isEqualTo( Resources.add(SMALL_RESOURCE, LARGE_NODE)); Assert.assertEquals(mgr.getResourceByLabel("p4", null), EMPTY_RESOURCE); // change the large NM to small, check if resource updated mgr.updateNodeResource(NodeId.newInstance("n1", 2), SMALL_RESOURCE); - Assert.assertEquals(mgr.getResourceByLabel("p1", null), + assertThat(mgr.getResourceByLabel("p1", null)).isEqualTo( Resources.multiply(SMALL_RESOURCE, 2)); // deactive one NM, and check if resource updated mgr.deactivateNode(NodeId.newInstance("n1", 1)); - Assert.assertEquals(mgr.getResourceByLabel("p1", null), SMALL_RESOURCE); + assertThat(mgr.getResourceByLabel("p1", null)).isEqualTo(SMALL_RESOURCE); // continus deactive, check if resource updated mgr.deactivateNode(NodeId.newInstance("n1", 2)); - Assert.assertEquals(mgr.getResourceByLabel("p1", null), EMPTY_RESOURCE); + assertThat(mgr.getResourceByLabel("p1", null)).isEqualTo(EMPTY_RESOURCE); // Add two NM to n1 back mgr.activateNode(NodeId.newInstance("n1", 1), SMALL_RESOURCE); @@ -126,8 +128,8 @@ public void testGetLabelResourceWhenNodeActiveDeactive() throws Exception { // And remove p1, now the two NM should come to default label, mgr.removeFromClusterNodeLabels(ImmutableSet.of("p1")); - Assert.assertEquals(mgr.getResourceByLabel(RMNodeLabelsManager.NO_LABEL, null), - Resources.add(SMALL_RESOURCE, LARGE_NODE)); + assertThat(mgr.getResourceByLabel(RMNodeLabelsManager.NO_LABEL, null)). + isEqualTo(Resources.add(SMALL_RESOURCE, LARGE_NODE)); } @Test(timeout = 5000) @@ -152,10 +154,10 @@ public void testGetLabelResource() throws Exception { // change label of n1 to p2 mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("p2"))); - Assert.assertEquals(mgr.getResourceByLabel("p1", null), EMPTY_RESOURCE); - Assert.assertEquals(mgr.getResourceByLabel("p2", null), + assertThat(mgr.getResourceByLabel("p1", null)).isEqualTo(EMPTY_RESOURCE); + assertThat(mgr.getResourceByLabel("p2", null)).isEqualTo( Resources.multiply(SMALL_RESOURCE, 2)); - Assert.assertEquals(mgr.getResourceByLabel("p3", null), SMALL_RESOURCE); + assertThat(mgr.getResourceByLabel("p3", null)).isEqualTo(SMALL_RESOURCE); // add more labels mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p4", "p5", "p6")); @@ -180,17 +182,17 @@ public void testGetLabelResource() throws Exception { mgr.activateNode(NodeId.newInstance("n9", 1), SMALL_RESOURCE); // check varibles - Assert.assertEquals(mgr.getResourceByLabel("p1", null), SMALL_RESOURCE); - Assert.assertEquals(mgr.getResourceByLabel("p2", null), + assertThat(mgr.getResourceByLabel("p1", null)).isEqualTo(SMALL_RESOURCE); + assertThat(mgr.getResourceByLabel("p2", null)).isEqualTo( Resources.multiply(SMALL_RESOURCE, 3)); - Assert.assertEquals(mgr.getResourceByLabel("p3", null), + assertThat(mgr.getResourceByLabel("p3", null)).isEqualTo( Resources.multiply(SMALL_RESOURCE, 2)); - Assert.assertEquals(mgr.getResourceByLabel("p4", null), + assertThat(mgr.getResourceByLabel("p4", null)).isEqualTo( Resources.multiply(SMALL_RESOURCE, 1)); - Assert.assertEquals(mgr.getResourceByLabel("p5", null), - Resources.multiply(SMALL_RESOURCE, 1)); - Assert.assertEquals(mgr.getResourceByLabel(RMNodeLabelsManager.NO_LABEL, null), + assertThat(mgr.getResourceByLabel("p5", null)).isEqualTo( Resources.multiply(SMALL_RESOURCE, 1)); + assertThat(mgr.getResourceByLabel(RMNodeLabelsManager.NO_LABEL, null)). + isEqualTo(Resources.multiply(SMALL_RESOURCE, 1)); // change a bunch of nodes -> labels // n4 -> p2 @@ -212,17 +214,17 @@ public void testGetLabelResource() throws Exception { toNodeId("n9"), toSet("p1"))); // check varibles - Assert.assertEquals(mgr.getResourceByLabel("p1", null), + assertThat(mgr.getResourceByLabel("p1", null)).isEqualTo( Resources.multiply(SMALL_RESOURCE, 2)); - Assert.assertEquals(mgr.getResourceByLabel("p2", null), + assertThat(mgr.getResourceByLabel("p2", null)).isEqualTo( Resources.multiply(SMALL_RESOURCE, 3)); - Assert.assertEquals(mgr.getResourceByLabel("p3", null), + assertThat(mgr.getResourceByLabel("p3", null)).isEqualTo( Resources.multiply(SMALL_RESOURCE, 2)); - Assert.assertEquals(mgr.getResourceByLabel("p4", null), + assertThat(mgr.getResourceByLabel("p4", null)).isEqualTo( Resources.multiply(SMALL_RESOURCE, 0)); - Assert.assertEquals(mgr.getResourceByLabel("p5", null), + assertThat(mgr.getResourceByLabel("p5", null)).isEqualTo( Resources.multiply(SMALL_RESOURCE, 0)); - Assert.assertEquals(mgr.getResourceByLabel("", null), + assertThat(mgr.getResourceByLabel("", null)).isEqualTo( Resources.multiply(SMALL_RESOURCE, 2)); } @@ -413,9 +415,9 @@ public void testGetLabelResourceWhenMultipleNMsExistingInSameHost() throws IOExc mgr.activateNode(NodeId.newInstance("n1", 4), SMALL_RESOURCE); // check resource of no label, it should be small * 4 - Assert.assertEquals( - mgr.getResourceByLabel(CommonNodeLabelsManager.NO_LABEL, null), - Resources.multiply(SMALL_RESOURCE, 4)); + assertThat( + mgr.getResourceByLabel(CommonNodeLabelsManager.NO_LABEL, null)). + isEqualTo(Resources.multiply(SMALL_RESOURCE, 4)); // change two of these nodes to p1, check resource of no_label and P1 mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p1")); @@ -423,12 +425,12 @@ public void testGetLabelResourceWhenMultipleNMsExistingInSameHost() throws IOExc toNodeId("n1:2"), toSet("p1"))); // check resource - Assert.assertEquals( - mgr.getResourceByLabel(CommonNodeLabelsManager.NO_LABEL, null), - Resources.multiply(SMALL_RESOURCE, 2)); - Assert.assertEquals( - mgr.getResourceByLabel("p1", null), - Resources.multiply(SMALL_RESOURCE, 2)); + assertThat( + mgr.getResourceByLabel(CommonNodeLabelsManager.NO_LABEL, null)). + isEqualTo(Resources.multiply(SMALL_RESOURCE, 2)); + assertThat( + mgr.getResourceByLabel("p1", null)).isEqualTo( + Resources.multiply(SMALL_RESOURCE, 2)); } @Test(timeout = 5000) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java index cd2a0fa208673..13faff0fc8ac1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.recovery; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertTrue; import java.io.IOException; @@ -125,8 +126,8 @@ public RMStateStore getRMStateStore() throws Exception { YarnConfiguration.YARN_INTERMEDIATE_DATA_ENCRYPTION, true); } this.store = new TestFileSystemRMStore(conf); - Assert.assertEquals(store.getNumRetries(), 8); - Assert.assertEquals(store.getRetryInterval(), 900L); + assertThat(store.getNumRetries()).isEqualTo(8); + assertThat(store.getRetryInterval()).isEqualTo(900L); Assert.assertTrue(store.fs.getConf() == store.fsConf); FileSystem previousFs = store.fs; store.startInternal(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java index 23c2b9a4ff2f4..f21ff96209786 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java @@ -76,6 +76,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -435,14 +436,14 @@ public void testZKRootPathAcls() throws Exception { rm.getRMContext().getRMAdminService().transitionToActive(req); ZKRMStateStore stateStore = (ZKRMStateStore) rm.getRMContext().getStateStore(); List acls = stateStore.getACL(rootPath); - assertEquals(acls.size(), 2); + assertThat(acls).hasSize(2); // CREATE and DELETE permissions for root node based on RM ID verifyZKACL("digest", "localhost", Perms.CREATE | Perms.DELETE, acls); verifyZKACL( "world", "anyone", Perms.ALL ^ (Perms.CREATE | Perms.DELETE), acls); acls = stateStore.getACL(parentPath); - assertEquals(1, acls.size()); + assertThat(acls).hasSize(1); assertEquals(perm, acls.get(0).getPerms()); rm.close(); @@ -463,7 +464,7 @@ public void testZKRootPathAcls() throws Exception { rm.start(); rm.getRMContext().getRMAdminService().transitionToActive(req); acls = stateStore.getACL(rootPath); - assertEquals(acls.size(), 2); + assertThat(acls).hasSize(2); verifyZKACL("digest", "localhost", Perms.CREATE | Perms.DELETE, acls); verifyZKACL( "world", "anyone", Perms.ALL ^ (Perms.CREATE | Perms.DELETE), acls); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationSystemUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationSystemUtil.java index d383bac2189a5..a092e0d527a30 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationSystemUtil.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationSystemUtil.java @@ -17,6 +17,7 @@ *******************************************************************************/ package org.apache.hadoop.yarn.server.resourcemanager.reservation; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.mock; import org.apache.hadoop.yarn.api.records.ReservationAllocationState; @@ -49,9 +50,9 @@ public void testConvertAllocationsToReservationInfo() { .convertAllocationsToReservationInfo( Collections.singleton(allocation), true); - Assert.assertEquals(infoList.size(), 1); - Assert.assertEquals(infoList.get(0).getReservationId().toString(), - id.toString()); + assertThat(infoList).hasSize(1); + assertThat(infoList.get(0).getReservationId().toString()).isEqualTo( + id.toString()); Assert.assertFalse(infoList.get(0).getResourceAllocationRequests() .isEmpty()); } @@ -104,7 +105,7 @@ public void testConvertAllocationsToReservationInfoEmptySet() { .convertAllocationsToReservationInfo( Collections.emptySet(), false); - Assert.assertEquals(infoList.size(), 0); + assertThat(infoList).isEmpty(); } private ReservationAllocation createReservationAllocation(long startTime, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestAlignedPlanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestAlignedPlanner.java index 9fa85595dbbc5..e26f9458682f6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestAlignedPlanner.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestAlignedPlanner.java @@ -18,7 +18,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation.planning; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.within; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -990,8 +991,9 @@ public void testGetDurationInterval() throws PlanningException { StageAllocatorLowCostAligned.getDurationInterval(10*step, 30*step, planLoads, planModifications, clusterCapacity, netRLERes, res, step, requestedResources); - assertEquals(durationInterval.numCanFit(), 4); - assertEquals(durationInterval.getTotalCost(), 0.55, 0.00001); + assertThat(durationInterval.numCanFit()).isEqualTo(4); + assertThat(durationInterval.getTotalCost()). + isCloseTo(0.55, within(0.00001)); // 2. // currLoad: should start at 20*step, end at 31*step with a null value @@ -1003,8 +1005,9 @@ public void testGetDurationInterval() throws PlanningException { planLoads, planModifications, clusterCapacity, netRLERes, res, step, requestedResources); System.out.println(durationInterval); - assertEquals(durationInterval.numCanFit(), 3); - assertEquals(durationInterval.getTotalCost(), 0.56, 0.00001); + assertThat(durationInterval.numCanFit()).isEqualTo(3); + assertThat(durationInterval.getTotalCost()). + isCloseTo(0.56, within(0.00001)); // 3. // currLoad: should start at 20*step, end at 30*step with a null value @@ -1015,8 +1018,9 @@ public void testGetDurationInterval() throws PlanningException { StageAllocatorLowCostAligned.getDurationInterval(15*step, 30*step, planLoads, planModifications, clusterCapacity, netRLERes, res, step, requestedResources); - assertEquals(durationInterval.numCanFit(), 4); - assertEquals(durationInterval.getTotalCost(), 0.55, 0.00001); + assertThat(durationInterval.numCanFit()).isEqualTo(4); + assertThat(durationInterval.getTotalCost()). + isCloseTo(0.55, within(0.00001)); // 4. // currLoad: should start at 20*step, end at 31*step with a null value @@ -1028,8 +1032,9 @@ public void testGetDurationInterval() throws PlanningException { planLoads, planModifications, clusterCapacity, netRLERes, res, step, requestedResources); System.out.println(durationInterval); - assertEquals(durationInterval.numCanFit(), 3); - assertEquals(durationInterval.getTotalCost(), 0.56, 0.00001); + assertThat(durationInterval.numCanFit()).isEqualTo(3); + assertThat(durationInterval.getTotalCost()). + isCloseTo(0.56, within(0.00001)); // 5. // currLoad: should only contain one entry at startTime @@ -1042,8 +1047,9 @@ public void testGetDurationInterval() throws PlanningException { planLoads, planModifications, clusterCapacity, netRLERes, res, step, requestedResources); System.out.println(durationInterval); - assertEquals(durationInterval.numCanFit(), 8); - assertEquals(durationInterval.getTotalCost(), 0.05, 0.00001); + assertThat(durationInterval.numCanFit()).isEqualTo(8); + assertThat(durationInterval.getTotalCost()). + isCloseTo(0.05, within(0.00001)); // 6. // currLoad: should start at 39*step, end at 41*step with a null value @@ -1055,8 +1061,9 @@ public void testGetDurationInterval() throws PlanningException { planLoads, planModifications, clusterCapacity, netRLERes, res, step, requestedResources); System.out.println(durationInterval); - assertEquals(durationInterval.numCanFit(), 0); - assertEquals(durationInterval.getTotalCost(), 0, 0.00001); + assertThat(durationInterval.numCanFit()).isEqualTo(0); + assertThat(durationInterval.getTotalCost()). + isCloseTo(0, within(0.00001)); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java index dc189c1bea8a4..e5d7e3fb637b1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java @@ -105,6 +105,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import static org.assertj.core.api.Assertions.assertThat; import static junit.framework.TestCase.assertTrue; import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; @@ -1278,7 +1279,8 @@ public void testGetAppReport() throws IOException { assertAppState(RMAppState.NEW, app); ApplicationReport report = app.createAndGetApplicationReport(null, true); Assert.assertNotNull(report.getApplicationResourceUsageReport()); - Assert.assertEquals(report.getApplicationResourceUsageReport(),RMServerUtils.DUMMY_APPLICATION_RESOURCE_USAGE_REPORT); + assertThat(report.getApplicationResourceUsageReport()). + isEqualTo(RMServerUtils.DUMMY_APPLICATION_RESOURCE_USAGE_REPORT); report = app.createAndGetApplicationReport("clientuser", true); Assert.assertNotNull(report.getApplicationResourceUsageReport()); Assert.assertTrue("bad proxy url for app", 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/TestApplicationLimits.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/TestApplicationLimits.java index 6958182028402..172db0a3d6f63 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/TestApplicationLimits.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/TestApplicationLimits.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -308,14 +309,15 @@ public void testLimitsComputation() throws Exception { queue.getUserAMResourceLimit()); Resource amResourceLimit = Resource.newInstance(160 * GB, 1); - assertEquals(queue.calculateAndGetAMResourceLimit(), amResourceLimit); - assertEquals(queue.getUserAMResourceLimit(), + assertThat(queue.calculateAndGetAMResourceLimit()). + isEqualTo(amResourceLimit); + assertThat(queue.getUserAMResourceLimit()).isEqualTo( Resource.newInstance(80*GB, 1)); // Assert in metrics - assertEquals(queue.getMetrics().getAMResourceLimitMB(), + assertThat(queue.getMetrics().getAMResourceLimitMB()).isEqualTo( amResourceLimit.getMemorySize()); - assertEquals(queue.getMetrics().getAMResourceLimitVCores(), + assertThat(queue.getMetrics().getAMResourceLimitVCores()).isEqualTo( amResourceLimit.getVirtualCores()); assertEquals( @@ -327,11 +329,11 @@ public void testLimitsComputation() throws Exception { clusterResource = Resources.createResource(120 * 16 * GB); root.updateClusterResource(clusterResource, new ResourceLimits( clusterResource)); - - assertEquals(queue.calculateAndGetAMResourceLimit(), + + assertThat(queue.calculateAndGetAMResourceLimit()).isEqualTo( Resource.newInstance(192 * GB, 1)); - assertEquals(queue.getUserAMResourceLimit(), - Resource.newInstance(96*GB, 1)); + assertThat(queue.getUserAMResourceLimit()).isEqualTo( + Resource.newInstance(96*GB, 1)); assertEquals( (int)(clusterResource.getMemorySize() * queue.getAbsoluteCapacity()), @@ -378,11 +380,11 @@ public void testLimitsComputation() throws Exception { (long) csConf.getMaximumApplicationMasterResourcePerQueuePercent( queue.getQueuePath()) ); - - assertEquals(queue.calculateAndGetAMResourceLimit(), + + assertThat(queue.calculateAndGetAMResourceLimit()).isEqualTo( Resource.newInstance(800 * GB, 1)); - assertEquals(queue.getUserAMResourceLimit(), - Resource.newInstance(400*GB, 1)); + assertThat(queue.getUserAMResourceLimit()).isEqualTo( + Resource.newInstance(400*GB, 1)); // Change the per-queue max applications. csConf.setInt(PREFIX + queue.getQueuePath() + ".maximum-applications", 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/TestApplicationPriority.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/TestApplicationPriority.java index cad0151cf0556..fea82e538876d 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/TestApplicationPriority.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/TestApplicationPriority.java @@ -18,7 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -140,9 +140,9 @@ public void testApplicationOrderingWithPriority() throws Exception { // Now, the first assignment will be for app2 since app2 is of highest // priority - assertEquals(q.getApplications().size(), 2); - assertEquals(q.getApplications().iterator().next() - .getApplicationAttemptId(), appAttemptId2); + assertThat(q.getApplications()).hasSize(2); + assertThat(q.getApplications().iterator().next().getApplicationAttemptId()) + .isEqualTo(appAttemptId2); rm.stop(); } 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/TestCapacityScheduler.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/TestCapacityScheduler.java index fd8fa0535c974..ae4009ab8ff21 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/TestCapacityScheduler.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/TestCapacityScheduler.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; +import static org.assertj.core.api.Assertions.assertThat; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.MAXIMUM_ALLOCATION; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.MAXIMUM_ALLOCATION_MB; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.MAXIMUM_ALLOCATION_VCORES; @@ -5251,7 +5252,8 @@ public void testContainerAllocationLocalitySkipped() throws Exception { RMNode node1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId()); cs.handle(new NodeUpdateSchedulerEvent(node1)); ContainerId cid = ContainerId.newContainerId(am.getApplicationAttemptId(), 1l); - Assert.assertEquals(cs.getRMContainer(cid).getState(), RMContainerState.ACQUIRED); + assertThat(cs.getRMContainer(cid).getState()). + isEqualTo(RMContainerState.ACQUIRED); cid = ContainerId.newContainerId(am.getApplicationAttemptId(), 2l); Assert.assertNull(cs.getRMContainer(cid)); 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/TestQueueManagementDynamicEditPolicy.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/TestQueueManagementDynamicEditPolicy.java index 66b488db27505..733f04171577f 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/TestQueueManagementDynamicEditPolicy.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/TestQueueManagementDynamicEditPolicy.java @@ -118,7 +118,7 @@ private void waitForPolicyState(float expectedVal, nodeLabel, int timesec) throws InterruptedException { long start = System.currentTimeMillis(); while (System.currentTimeMillis() - start < timesec * 1000) { - if (Float.compare(expectedVal, queueManagementPolicy + if (Math.abs(expectedVal - queueManagementPolicy .getAbsoluteActivatedChildQueueCapacity(nodeLabel)) > EPSILON) { Thread.sleep(100); } else { 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/fair/TestConfigurableResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestConfigurableResource.java index 249d1f77ca6b3..debd8ae7ebe53 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestConfigurableResource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestConfigurableResource.java @@ -22,6 +22,7 @@ import org.apache.hadoop.yarn.util.resource.Resources; import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -52,8 +53,10 @@ public void testGetResourceWithPercentage() { public void testGetResourceWithAbsolute() { ConfigurableResource configurableResource = new ConfigurableResource(Resources.createResource(3072, 3)); - assertEquals(configurableResource.getResource().getMemorySize(), 3072); - assertEquals(configurableResource.getResource().getVirtualCores(), 3); + assertThat(configurableResource.getResource().getMemorySize()). + isEqualTo(3072); + assertThat(configurableResource.getResource().getVirtualCores()). + isEqualTo(3); assertEquals( configurableResource.getResource(clusterResource).getMemorySize(), @@ -62,7 +65,9 @@ public void testGetResourceWithAbsolute() { configurableResource.getResource(clusterResource).getVirtualCores(), 3); - assertEquals(configurableResource.getResource(null).getMemorySize(), 3072); - assertEquals(configurableResource.getResource(null).getVirtualCores(), 3); + assertThat(configurableResource.getResource(null).getMemorySize()). + isEqualTo(3072); + assertThat(configurableResource.getResource(null).getVirtualCores()). + isEqualTo(3); } } 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/fair/TestContinuousScheduling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java index 81500dbfcd7e7..5dac8622164d1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java @@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.util.resource.Resources; import org.junit.After; import org.junit.Assert; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; @@ -153,9 +154,10 @@ public void testSortedNodes() throws Exception { scheduler.handle(nodeEvent2); // available resource - Assert.assertEquals(scheduler.getClusterResource().getMemorySize(), - 16 * 1024); - Assert.assertEquals(scheduler.getClusterResource().getVirtualCores(), 16); + assertThat(scheduler.getClusterResource().getMemorySize()). + isEqualTo(16 * 1024); + assertThat(scheduler.getClusterResource().getVirtualCores()). + isEqualTo(16); // send application request ApplicationAttemptId appAttemptId = 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/fair/TestFSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java index 0cf1a7b4164da..9c47b6be0f99a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; @@ -90,8 +91,9 @@ public void testUpdateDemand() { String queueName = "root.queue1"; FSLeafQueue schedulable = new FSLeafQueue(queueName, scheduler, null); schedulable.setMaxShare(new ConfigurableResource(maxResource)); - assertEquals(schedulable.getMetrics().getMaxApps(), Integer.MAX_VALUE); - assertEquals(schedulable.getMetrics().getSchedulingPolicy(), + assertThat(schedulable.getMetrics().getMaxApps()). + isEqualTo(Integer.MAX_VALUE); + assertThat(schedulable.getMetrics().getSchedulingPolicy()).isEqualTo( SchedulingPolicy.DEFAULT_POLICY.getName()); FSAppAttempt app = mock(FSAppAttempt.class); @@ -124,8 +126,8 @@ public void test() throws Exception { resourceManager.start(); scheduler = (FairScheduler) resourceManager.getResourceScheduler(); for(FSQueue queue: scheduler.getQueueManager().getQueues()) { - assertEquals(queue.getMetrics().getMaxApps(), Integer.MAX_VALUE); - assertEquals(queue.getMetrics().getSchedulingPolicy(), + assertThat(queue.getMetrics().getMaxApps()).isEqualTo(Integer.MAX_VALUE); + assertThat(queue.getMetrics().getSchedulingPolicy()).isEqualTo( SchedulingPolicy.DEFAULT_POLICY.getName()); } 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/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java index e76e7e3637f4d..d9f29f1be118e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java @@ -114,6 +114,7 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.assertj.core.api.Assertions.assertThat; import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -4623,17 +4624,17 @@ public void testResourceUsageByMoveApp() throws Exception { FSQueue queue2 = queueMgr.getLeafQueue("parent2.queue2", true); FSQueue queue1 = queueMgr.getLeafQueue("parent1.queue1", true); - Assert.assertEquals(parent2.getResourceUsage().getMemorySize(), 0); - Assert.assertEquals(queue2.getResourceUsage().getMemorySize(), 0); - Assert.assertEquals(parent1.getResourceUsage().getMemorySize(), 1 * GB); - Assert.assertEquals(queue1.getResourceUsage().getMemorySize(), 1 * GB); + assertThat(parent2.getResourceUsage().getMemorySize()).isEqualTo(0); + assertThat(queue2.getResourceUsage().getMemorySize()).isEqualTo(0); + assertThat(parent1.getResourceUsage().getMemorySize()).isEqualTo(1 * GB); + assertThat(queue1.getResourceUsage().getMemorySize()).isEqualTo(1 * GB); scheduler.moveApplication(appAttId.getApplicationId(), "parent2.queue2"); - Assert.assertEquals(parent2.getResourceUsage().getMemorySize(), 1 * GB); - Assert.assertEquals(queue2.getResourceUsage().getMemorySize(), 1 * GB); - Assert.assertEquals(parent1.getResourceUsage().getMemorySize(), 0); - Assert.assertEquals(queue1.getResourceUsage().getMemorySize(), 0); + assertThat(parent2.getResourceUsage().getMemorySize()).isEqualTo(1 * GB); + assertThat(queue2.getResourceUsage().getMemorySize()).isEqualTo(1 * GB); + assertThat(parent1.getResourceUsage().getMemorySize()).isEqualTo(0); + assertThat(queue1.getResourceUsage().getMemorySize()).isEqualTo(0); } @Test (expected = YarnException.class) @@ -5070,20 +5071,20 @@ public void handle(Event event) { Resource usedResource = resourceManager.getResourceScheduler() .getSchedulerNode(nm_0.getNodeId()).getAllocatedResource(); - Assert.assertEquals(usedResource.getMemorySize(), 0); - Assert.assertEquals(usedResource.getVirtualCores(), 0); + assertThat(usedResource.getMemorySize()).isEqualTo(0); + assertThat(usedResource.getVirtualCores()).isEqualTo(0); // Check total resource of scheduler node is also changed to 0 GB 0 core Resource totalResource = resourceManager.getResourceScheduler() .getSchedulerNode(nm_0.getNodeId()).getTotalResource(); - Assert.assertEquals(totalResource.getMemorySize(), 0 * GB); - Assert.assertEquals(totalResource.getVirtualCores(), 0); + assertThat(totalResource.getMemorySize()).isEqualTo(0 * GB); + assertThat(totalResource.getVirtualCores()).isEqualTo(0); // Check the available resource is 0/0 Resource availableResource = resourceManager.getResourceScheduler() .getSchedulerNode(nm_0.getNodeId()).getUnallocatedResource(); - Assert.assertEquals(availableResource.getMemorySize(), 0); - Assert.assertEquals(availableResource.getVirtualCores(), 0); + assertThat(availableResource.getMemorySize()).isEqualTo(0); + assertThat(availableResource.getVirtualCores()).isEqualTo(0); } private NodeManager registerNode(String hostName, int containerManagerPort, @@ -5159,8 +5160,7 @@ public void testContainerAllocationWithContainerIdLeap() throws Exception { // container will be allocated at node2 scheduler.handle(new NodeUpdateSchedulerEvent(node2)); - assertEquals(scheduler.getSchedulerApp(app2). - getLiveContainers().size(), 1); + assertThat(scheduler.getSchedulerApp(app2).getLiveContainers()).hasSize(1); long maxId = 0; for (RMContainer container : 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/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java index f9093c18d4e5c..dc94e5e6d74b4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -333,7 +334,7 @@ public void testUpdateResourceOnNode() throws Exception { NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node0); scheduler.handle(nodeEvent1); - assertEquals(scheduler.getNumClusterNodes(), 1); + assertThat(scheduler.getNumClusterNodes()).isEqualTo(1); Resource newResource = Resources.createResource(1024, 4); @@ -1286,20 +1287,20 @@ public void handle(Event event) { Resource usedResource = resourceManager.getResourceScheduler() .getSchedulerNode(nm_0.getNodeId()).getAllocatedResource(); - Assert.assertEquals(usedResource.getMemorySize(), 1 * GB); - Assert.assertEquals(usedResource.getVirtualCores(), 1); + assertThat(usedResource.getMemorySize()).isEqualTo(1 * GB); + assertThat(usedResource.getVirtualCores()).isEqualTo(1); // Check total resource of scheduler node is also changed to 1 GB 1 core Resource totalResource = resourceManager.getResourceScheduler() .getSchedulerNode(nm_0.getNodeId()).getTotalResource(); - Assert.assertEquals(totalResource.getMemorySize(), 1 * GB); - Assert.assertEquals(totalResource.getVirtualCores(), 1); + assertThat(totalResource.getMemorySize()).isEqualTo(1 * GB); + assertThat(totalResource.getVirtualCores()).isEqualTo(1); // Check the available resource is 0/0 Resource availableResource = resourceManager.getResourceScheduler() .getSchedulerNode(nm_0.getNodeId()).getUnallocatedResource(); - Assert.assertEquals(availableResource.getMemorySize(), 0); - Assert.assertEquals(availableResource.getVirtualCores(), 0); + assertThat(availableResource.getMemorySize()).isEqualTo(0); + assertThat(availableResource.getVirtualCores()).isEqualTo(0); } private void checkApplicationResourceUsage(int expected, 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/policy/TestFifoOrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicy.java index 447618f4f242a..776f6c6962b16 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicy.java @@ -25,6 +25,8 @@ import org.apache.hadoop.yarn.api.records.Priority; +import static org.assertj.core.api.Assertions.assertThat; + public class TestFifoOrderingPolicy { @Test @@ -33,14 +35,14 @@ public void testFifoOrderingPolicy() { new FifoOrderingPolicy(); MockSchedulableEntity r1 = new MockSchedulableEntity(); MockSchedulableEntity r2 = new MockSchedulableEntity(); - - Assert.assertEquals(policy.getComparator().compare(r1, r2), 0); + + assertThat(policy.getComparator().compare(r1, r2)).isEqualTo(0); r1.setSerial(1); - Assert.assertEquals(policy.getComparator().compare(r1, r2), 1); + assertThat(policy.getComparator().compare(r1, r2)).isEqualTo(1); r2.setSerial(2); - Assert.assertEquals(policy.getComparator().compare(r1, r2), -1); + assertThat(policy.getComparator().compare(r1, r2)).isEqualTo(-1); } @Test 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/policy/TestFifoOrderingPolicyForPendingApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicyForPendingApps.java index befa8e6c3210c..5fa9a1d0a91c7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicyForPendingApps.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicyForPendingApps.java @@ -23,6 +23,8 @@ import org.junit.Assert; import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; + public class TestFifoOrderingPolicyForPendingApps { @Test @@ -33,16 +35,16 @@ public void testFifoOrderingPolicyForPendingApps() { MockSchedulableEntity r1 = new MockSchedulableEntity(); MockSchedulableEntity r2 = new MockSchedulableEntity(); - Assert.assertEquals(policy.getComparator().compare(r1, r2), 0); + assertThat(policy.getComparator().compare(r1, r2)).isEqualTo(0); r1.setSerial(1); r1.setRecovering(true); - Assert.assertEquals(policy.getComparator().compare(r1, r2), -1); + assertThat(policy.getComparator().compare(r1, r2)).isEqualTo(-1); r1.setRecovering(false); r2.setSerial(2); r2.setRecovering(true); - Assert.assertEquals(policy.getComparator().compare(r1, r2), 1); + assertThat(policy.getComparator().compare(r1, r2)).isEqualTo(1); } /** diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/volume/csi/TestVolumeProcessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/volume/csi/TestVolumeProcessor.java index 5113ba532d700..370c527cecbb7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/volume/csi/TestVolumeProcessor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/volume/csi/TestVolumeProcessor.java @@ -64,6 +64,7 @@ import java.util.Arrays; import java.util.List; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; @@ -316,13 +317,13 @@ public void testVolumeResourceAllocate() throws Exception { Assert.assertEquals(1, allocated.size()); Container alloc = allocated.get(0); - Assert.assertEquals(alloc.getResource().getMemorySize(), 1024); - Assert.assertEquals(alloc.getResource().getVirtualCores(), 1); + assertThat(alloc.getResource().getMemorySize()).isEqualTo(1024); + assertThat(alloc.getResource().getVirtualCores()).isEqualTo(1); ResourceInformation allocatedVolume = alloc.getResource().getResourceInformation(VOLUME_RESOURCE_NAME); Assert.assertNotNull(allocatedVolume); - Assert.assertEquals(allocatedVolume.getValue(), 1024); - Assert.assertEquals(allocatedVolume.getUnits(), "Mi"); + assertThat(allocatedVolume.getValue()).isEqualTo(1024); + assertThat(allocatedVolume.getUnits()).isEqualTo("Mi"); rm.stop(); } } \ 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/webapp/TestRMWebServicesNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java index e8f62d03ea4c3..c3f41f62f6f4c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp; +import static org.assertj.core.api.Assertions.assertThat; import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -1088,8 +1089,8 @@ private void verifyNodeAllocationTag(JSONObject json, for (int j=0; jmockito-core test + + org.assertj + assertj-core + test + com.fasterxml.jackson.core jackson-databind diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java index 61da3c8930825..64354166358c6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java @@ -60,6 +60,7 @@ import java.util.Map; import java.util.Set; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; @@ -414,7 +415,7 @@ store.new AppLogs(mainTestAppId, mainTestAppDirPath, TimelineEntities entities = tdm.getEntities("type_1", null, null, null, null, null, null, null, EnumSet.allOf(TimelineReader.Field.class), UserGroupInformation.getLoginUser()); - assertEquals(entities.getEntities().size(), 1); + assertThat(entities.getEntities()).hasSize(1); for (TimelineEntity entity : entities.getEntities()) { assertEquals((Long) 123L, entity.getStartTime()); } 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 01eeec93822ce..43021d38ad562 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 @@ -324,6 +324,12 @@ test + + org.assertj + assertj-core + test + + org.apache.hbase hbase-testing-util diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java index e0a58da98057a..475cb7a306ffd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.flow; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; @@ -119,7 +120,8 @@ public void testWriteNonNumericData() throws Exception { Cell actualValue = r.getColumnLatestCell( FlowRunColumnFamily.INFO.getBytes(), columnNameBytes); assertNotNull(CellUtil.cloneValue(actualValue)); - assertEquals(Bytes.toString(CellUtil.cloneValue(actualValue)), value); + assertThat(Bytes.toString(CellUtil.cloneValue(actualValue))). + isEqualTo(value); } @Test diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml index 81ee6e274bd4a..7293a0699d234 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml @@ -131,6 +131,12 @@ test + + org.assertj + assertj-core + test + + org.mockito mockito-core diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java index 80517922f875d..766c2cd4bc61d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java @@ -42,6 +42,7 @@ import java.util.Map; import java.util.Set; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -109,7 +110,7 @@ public void testAggregation() throws Exception { TimelineEntities testEntities = generateTestEntities(groups, n); TimelineEntity resultEntity = TimelineCollector.aggregateEntities( testEntities, "test_result", "TEST_AGGR", true); - assertEquals(resultEntity.getMetrics().size(), groups * 3); + assertThat(resultEntity.getMetrics()).hasSize(groups * 3); for (int i = 0; i < groups; i++) { Set metrics = resultEntity.getMetrics(); @@ -130,7 +131,7 @@ public void testAggregation() throws Exception { TimelineEntities testEntities1 = generateTestEntities(1, n); TimelineEntity resultEntity1 = TimelineCollector.aggregateEntities( testEntities1, "test_result", "TEST_AGGR", false); - assertEquals(resultEntity1.getMetrics().size(), 3); + assertThat(resultEntity1.getMetrics()).hasSize(3); Set metrics = resultEntity1.getMetrics(); for (TimelineMetric m : metrics) { 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 8eaa24b11d176..17715a55748c2 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 @@ -74,6 +74,11 @@ junit test + + org.assertj + assertj-core + test + org.apache.hadoop 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/TestWebAppProxyServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java index f04bc9a39fe4e..6cc1d22d5825f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.webproxy; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -334,7 +335,7 @@ public void testWebAppProxyPassThroughHeaders() throws Exception { "Access-Control-Request-Headers", "Authorization"); proxyConn.addRequestProperty(UNKNOWN_HEADER, "unknown"); // Verify if four headers mentioned above have been added - assertEquals(proxyConn.getRequestProperties().size(), 4); + assertThat(proxyConn.getRequestProperties()).hasSize(4); proxyConn.connect(); assertEquals(HttpURLConnection.HTTP_OK, proxyConn.getResponseCode()); // Verify if number of headers received by end server is 9. 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 8b0c1a4c2f204..641c2bf40613e 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 @@ -44,6 +44,7 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertEquals; @@ -163,7 +164,7 @@ public void testFindRedirectUrl() throws Exception { spy.proxyUriBases.put(rm2, rm2Url); spy.rmUrls = new String[] { rm1, rm2 }; - assertEquals(spy.findRedirectUrl(), rm1Url); + assertThat(spy.findRedirectUrl()).isEqualTo(rm1Url); } private String startHttpServer() throws Exception { 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/TestSecureAmFilter.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/TestSecureAmFilter.java index e87b76541e575..0a88243823aaf 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/TestSecureAmFilter.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/TestSecureAmFilter.java @@ -25,7 +25,7 @@ import java.util.HashSet; import java.util.HashMap; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -137,7 +137,7 @@ public void testFindRedirectUrl() throws Exception { assertTrue(spy.isValidUrl(rm1Url)); assertFalse(spy.isValidUrl(rm2Url)); - assertEquals(spy.findRedirectUrl(), rm1Url); + assertThat(spy.findRedirectUrl()).isEqualTo(rm1Url); } private String startSecureHttpServer() throws Exception { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md index cf98c573b06c6..b75a9d4b3d6de 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md @@ -330,14 +330,6 @@ Following are the basic configurations to start Timeline service v.2: yarn.system-metrics-publisher.enabled true - - - The setting that controls whether yarn container events are - published to the timeline service or not by RM. This configuration setting - is for ATS V2. - yarn.rm.system-metrics-publisher.emit-container-events - true - ``` If using an aux services manifest instead of setting aux services through the Configuration, ensure that the manifest services array includes the timeline\_collector service as follows: diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Examples.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Examples.md index 7aae830f58eac..bf9d1f56f6530 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Examples.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Examples.md @@ -188,6 +188,60 @@ For secure cluster, Kerberos settings for application catalog can be configured | KEYTAB | /etc/security/keytabs/yarn.service.ketab | Path to keytab file, used by YARN service application master. | | PRINCIPAL | yarn/_HOST@EXAMPLE.COM | Service principal used by YARN service application master. | +Application Catalog environment options: + +| Environment variables | Description | +| KEYTAB | Service user keytab file for accessing HDFS. | +| PRINCIPAL | Service user Kerboers principal. | +| SOLR_DATA_DIR | Location to store Solr data. | +| SOLR_STORAGE_TYPE | Storage type for Solr data, supported type are: hdfs, local | +| SPNEGO_KEYTAB | Location of the keytab file used for authenticating HTTP endpoint. | +| SPNEGO_PRINCIPAL | The Kerberos principal to be used for HTTP endpoint. The principal MUST start with 'HTTP'/ as per Kerberos HTTP SPNEGO specification. | + +Secure application catalog Yarnfile example: +``` +{ + "name": "catalog", + "kerberos_principal" : { + "principal_name" : "catalog/_HOST@EXAMPLE.COM", + "keytab" : "file:///etc/security/keytabs/catalog.service.keytab" + }, + "version": "1", + "components" : + [ + { + "name": "appcatalog", + "number_of_containers": 1, + "artifact": { + "id": "apache/hadoop-yarn-applications-catalog-docker:3.3.0-SNAPSHOT", + "type": "DOCKER" + }, + "resource": { + "cpus": 1, + "memory": "256" + }, + "configuration": { + "env": { + "YARN_CONTAINER_RUNTIME_DOCKER_RUN_OVERRIDE_DISABLE":"true", + "YARN_CONTAINER_RUNTIME_DOCKER_MOUNTS":"/etc/hadoop/conf:/etc/hadoop/conf:ro,/etc/krb5.conf:/etc/krb5.conf:ro,/etc/security/keytabs/catalog.service.keytab:/etc/security/keytabs/hbase.service.keytab:ro,/etc/security/keytabs/spnego.service.keytab:/etc/security/keytabs/spnego.service.keytab:ro", + "SPNEGO_KEYTAB":"/etc/security/keytabs/spnego.service.keytab", + "SPNEGO_PRINCIPAL":"HTTP/host-3.example.com@EXAMPLE.COM", + "KEYTAB":"/etc/security/keytabs/catalog.service.keytab", + "PRINCIPAL":"catalog/host3.example.com@EXAMPLE.COM", + "SOLR_DATA_DIR":"hdfs://host-1.example.com:9000/tmp/solr", + "SOLR_UPDATE_LOG":"hdfs://host-1.example.com:9000/tmp/solr", + "SOLR_STORAGE_TYPE":"hdfs" + }, + "properties": { + "docker.network": "host" + } + } + } + ] +} + +``` + ## Docker image ENTRYPOINT support Docker images may have built with ENTRYPOINT to enable start up of docker image without any parameters. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/MasterWorker.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/MasterWorker.md new file mode 100644 index 0000000000000..ca49d302663bf --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/MasterWorker.md @@ -0,0 +1,36 @@ + + +# MaWo: A Master Worker framework on YARN Services + +## Overview + +MaWo is a YARN service based framework which handles Master Worker based workload. +This is an app which can take an input job specification with tasks, their expected durations and have a Master dish the tasks off to a predetermined set of workers. +The components will be responsible to finish the job within specific time duration. + +## MaWo Components + +MaWo app is a YARN Service Application. It has mainly two components. + +* Master + - Read MaWo-Payload file and create a queue of Tasks + - Register Worker + - Assign tasks to worker nodes + - Monitor status of Tasks + - Log Task status + +* Worker + - Send heartbeat to Worker + - Execute Task \ No newline at end of file