You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by vanzin <gi...@git.apache.org> on 2018/08/21 19:40:08 UTC

[GitHub] spark pull request #20761: [SPARK-20327][CORE][YARN] Add CLI support for YAR...

Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20761#discussion_r211733592
  
    --- Diff: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceTypeValidator.scala ---
    @@ -0,0 +1,185 @@
    +/*
    + * 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.spark.deploy.yarn
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.yarn.ProcessType.{am, driver, executor, ProcessType}
    +import org.apache.spark.deploy.yarn.ResourceType.{cores, memory, ResourceType}
    +import org.apache.spark.deploy.yarn.RunMode.{client, cluster, RunMode}
    +import org.apache.spark.deploy.yarn.config._
    +import org.apache.spark.internal.config._
    +
    +private[spark] object ProcessType extends Enumeration {
    +  type ProcessType = Value
    +  val driver, executor, am = Value
    +}
    +
    +private[spark] object RunMode extends Enumeration {
    +  type RunMode = Value
    +  val client, cluster = Value
    +}
    +
    +private[spark] object ResourceType extends Enumeration {
    +  type ResourceType = Value
    +  val cores, memory = Value
    +}
    +
    +private object ResourceTypeValidator {
    +  private val ERROR_PREFIX: String = "Error: "
    +  private val POSSIBLE_RESOURCE_DEFINITIONS = Seq[ResourceConfigProperties](
    +    new ResourceConfigProperties(am, client, memory),
    +    new ResourceConfigProperties(am, client, cores),
    +    new ResourceConfigProperties(driver, cluster, memory),
    +    new ResourceConfigProperties(driver, cluster, cores),
    +    new ResourceConfigProperties(processType = executor, resourceType = memory),
    +    new ResourceConfigProperties(processType = executor, resourceType = cores))
    +
    +  /**
    +   * Validates sparkConf and throws a SparkException if a standard resource (memory or cores)
    +   * is defined with the property spark.yarn.x.resource.y<br>
    +   *
    +   * Example of an invalid config:<br>
    +   * - spark.yarn.driver.resource.memory=2g<br>
    +   *
    +   * Please note that if multiple resources are defined like described above,
    +   * the error messages will be concatenated.<br>
    +   * Example of such a config:<br>
    +   * - spark.yarn.driver.resource.memory=2g<br>
    +   * - spark.yarn.executor.resource.cores=2<br>
    +   * Then the following two error messages will be printed:<br>
    +   * - "memory cannot be requested with config spark.yarn.driver.resource.memory,
    +   * please use config spark.driver.memory instead!<br>
    +   * - "cores cannot be requested with config spark.yarn.executor.resource.cores,
    +   * please use config spark.executor.cores instead!<br>
    +   *
    +   * @param sparkConf
    +   */
    +  def validateResources(sparkConf: SparkConf): Unit = {
    --- End diff --
    
    You said: 
    
    >  the error message points you to the right direction, explicitly printing the right config to use instead.
    
    So does my code above, it just need to be changed a little to be more parameterized. And it's an order of magnitude smaller and simpler than your validation code.
    
    So, same question: what else is your validator checking that those two asserts (properly parameterized to check for driver, am, or executor) do not cover?



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org