You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by zj...@apache.org on 2020/07/10 15:57:23 UTC

[zeppelin] branch master updated: [ZEPPELIN-4940]. yarn config doesn't work for flink 1.11

This is an automated email from the ASF dual-hosted git repository.

zjffdu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/zeppelin.git


The following commit(s) were added to refs/heads/master by this push:
     new b235a19  [ZEPPELIN-4940]. yarn config doesn't work for flink 1.11
b235a19 is described below

commit b235a19afafe7c8c54175bc3e12010be7083ae56
Author: Jeff Zhang <zj...@apache.org>
AuthorDate: Tue Jul 7 15:05:13 2020 +0800

    [ZEPPELIN-4940]. yarn config doesn't work for flink 1.11
    
    ### What is this PR for?
    
    Because the bug of FLINK-17788, zeppelin is unable to work with flink 1.10.1, although we use workaround before, but this workaround introduce new issues. So this PR just copy FlinkShell from flink, and customize it in zeppelin side. After this PR, we are not only able to use flink yarn mode, but also make the yarn config work
    
    ### What type of PR is it?
    [Bug Fix]
    
    ### Todos
    * [ ] - Task
    
    ### What is the Jira issue?
    * https://issues.apache.org/jira/browse/ZEPPELIN-4940
    
    ### How should this be tested?
    * CI pass, manually tested
    
    ### Screenshots (if appropriate)
    
    ### Questions:
    * Does the licenses files need update? NO
    * Is there breaking changes for older versions? No
    * Does this needs documentation? No
    
    Author: Jeff Zhang <zj...@apache.org>
    
    Closes #3843 from zjffdu/ZEPPELIN-4940 and squashes the following commits:
    
    38761befe [Jeff Zhang] [ZEPPELIN-4940]. yarn config doesn't work for flink 1.11
---
 .../java/org/apache/zeppelin/flink/FlinkShims.java |   2 +
 .../org/apache/zeppelin/flink/Flink110Shims.java   |   7 +
 .../org/apache/zeppelin/flink/Flink111Shims.java   |   6 +
 .../zeppelin/flink/FlinkScalaInterpreter.scala     |  10 +-
 .../org/apache/zeppelin/flink/FlinkShell.scala     | 251 +++++++++++++++++++++
 5 files changed, 268 insertions(+), 8 deletions(-)

diff --git a/flink/flink-shims/src/main/java/org/apache/zeppelin/flink/FlinkShims.java b/flink/flink-shims/src/main/java/org/apache/zeppelin/flink/FlinkShims.java
index 274bf2c..9eb600d 100644
--- a/flink/flink-shims/src/main/java/org/apache/zeppelin/flink/FlinkShims.java
+++ b/flink/flink-shims/src/main/java/org/apache/zeppelin/flink/FlinkShims.java
@@ -104,4 +104,6 @@ public abstract class FlinkShims {
   public abstract void registerTableAggregateFunction(Object btenv, String name, Object tableAggregateFunction);
 
   public abstract void registerTableSink(Object stenv, String tableName, Object collectTableSink);
+
+  public abstract Object getCustomCli(Object cliFrontend, Object commandLine);
 }
diff --git a/flink/flink1.10-shims/src/main/java/org/apache/zeppelin/flink/Flink110Shims.java b/flink/flink1.10-shims/src/main/java/org/apache/zeppelin/flink/Flink110Shims.java
index f6d506a..1f7dd55 100644
--- a/flink/flink1.10-shims/src/main/java/org/apache/zeppelin/flink/Flink110Shims.java
+++ b/flink/flink1.10-shims/src/main/java/org/apache/zeppelin/flink/Flink110Shims.java
@@ -18,9 +18,11 @@
 
 package org.apache.zeppelin.flink;
 
+import org.apache.commons.cli.CommandLine;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.scala.DataSet;
+import org.apache.flink.client.cli.CliFrontend;
 import org.apache.flink.python.util.ResourceUtil;
 import org.apache.flink.table.api.Table;
 import org.apache.flink.table.api.TableEnvironment;
@@ -147,4 +149,9 @@ public class Flink110Shims extends FlinkShims {
   public void registerTableAggregateFunction(Object btenv, String name, Object tableAggregateFunction) {
     ((StreamTableEnvironmentImpl)(btenv)).registerFunction(name, (TableAggregateFunction) tableAggregateFunction);
   }
+
+  @Override
+  public Object getCustomCli(Object cliFrontend, Object commandLine) {
+    return ((CliFrontend)cliFrontend).getActiveCustomCommandLine((CommandLine) commandLine);
+  }
 }
diff --git a/flink/flink1.11-shims/src/main/java/org/apache/zeppelin/flink/Flink111Shims.java b/flink/flink1.11-shims/src/main/java/org/apache/zeppelin/flink/Flink111Shims.java
index 2480c69..f6d929a 100644
--- a/flink/flink1.11-shims/src/main/java/org/apache/zeppelin/flink/Flink111Shims.java
+++ b/flink/flink1.11-shims/src/main/java/org/apache/zeppelin/flink/Flink111Shims.java
@@ -18,11 +18,13 @@
 
 package org.apache.zeppelin.flink;
 
+import org.apache.commons.cli.CommandLine;
 import org.apache.commons.compress.utils.Lists;
 import org.apache.flink.api.common.JobStatus;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.scala.DataSet;
+import org.apache.flink.client.cli.CliFrontend;
 import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.core.execution.JobClient;
 import org.apache.flink.table.api.StatementSet;
@@ -169,4 +171,8 @@ public class Flink111Shims extends FlinkShims {
     ((StreamTableEnvironmentImpl)(btenv)).registerFunction(name, (TableAggregateFunction) tableAggregateFunction);
   }
 
+  @Override
+  public Object getCustomCli(Object cliFrontend, Object commandLine) {
+    return ((CliFrontend)cliFrontend).validateAndGetActiveCommandLine((CommandLine) commandLine);
+  }
 }
diff --git a/flink/interpreter/src/main/scala/org/apache/zeppelin/flink/FlinkScalaInterpreter.scala b/flink/interpreter/src/main/scala/org/apache/zeppelin/flink/FlinkScalaInterpreter.scala
index 2f544b4..329722e 100644
--- a/flink/interpreter/src/main/scala/org/apache/zeppelin/flink/FlinkScalaInterpreter.scala
+++ b/flink/interpreter/src/main/scala/org/apache/zeppelin/flink/FlinkScalaInterpreter.scala
@@ -24,12 +24,10 @@ import java.nio.file.Files
 import java.util.Properties
 import java.util.concurrent.TimeUnit
 import java.util.jar.JarFile
-import java.util.regex.Pattern
 
 import org.apache.commons.lang3.StringUtils
 import org.apache.commons.lang3.exception.ExceptionUtils
 import org.apache.flink.api.common.JobExecutionResult
-import org.apache.flink.api.scala.FlinkShell.{ExecutionMode, _}
 import org.apache.flink.api.scala.{ExecutionEnvironment, FlinkILoop}
 import org.apache.flink.client.program.ClusterClient
 import org.apache.flink.configuration._
@@ -48,8 +46,8 @@ import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, Tabl
 import org.apache.flink.table.module.ModuleManager
 import org.apache.flink.table.module.hive.HiveModule
 import org.apache.flink.yarn.cli.FlinkYarnSessionCli
-import org.apache.flink.yarn.executors.YarnSessionClusterExecutor
 import org.apache.zeppelin.flink.util.DependencyUtils
+import org.apache.zeppelin.flink.FlinkShell._
 import org.apache.zeppelin.interpreter.thrift.InterpreterCompletion
 import org.apache.zeppelin.interpreter.util.InterpreterOutputStream
 import org.apache.zeppelin.interpreter.{InterpreterContext, InterpreterException, InterpreterHookRegistry, InterpreterResult}
@@ -226,10 +224,6 @@ class FlinkScalaInterpreter(val properties: Properties) {
         .copy(port = Some(Integer.parseInt(port)))
     }
 
-    if (config.executionMode == ExecutionMode.YARN) {
-      // workaround for FLINK-17788, otherwise it won't work with flink 1.10.1 which has been released.
-      configuration.set(DeploymentOptions.TARGET, YarnSessionClusterExecutor.NAME)
-    }
     config
   }
 
@@ -254,7 +248,7 @@ class FlinkScalaInterpreter(val properties: Properties) {
         }
       }
 
-      val (effectiveConfig, cluster) = fetchConnectionInfo(config, configuration)
+      val (effectiveConfig, cluster) = fetchConnectionInfo(config, configuration, flinkShims)
       this.configuration = effectiveConfig
       cluster match {
         case Some(clusterClient) =>
diff --git a/flink/interpreter/src/main/scala/org/apache/zeppelin/flink/FlinkShell.scala b/flink/interpreter/src/main/scala/org/apache/zeppelin/flink/FlinkShell.scala
new file mode 100644
index 0000000..3f814b9
--- /dev/null
+++ b/flink/interpreter/src/main/scala/org/apache/zeppelin/flink/FlinkShell.scala
@@ -0,0 +1,251 @@
+/*
+ * 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.zeppelin.flink
+
+import java.io._
+
+import org.apache.flink.annotation.Internal
+import org.apache.flink.client.cli.{CliFrontend, CliFrontendParser, CustomCommandLine}
+import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader
+import org.apache.flink.client.deployment.executors.RemoteExecutor
+import org.apache.flink.client.program.{ClusterClient, MiniClusterClient}
+import org.apache.flink.configuration._
+import org.apache.flink.runtime.minicluster.{MiniCluster, MiniClusterConfiguration}
+import org.apache.flink.yarn.executors.YarnSessionClusterExecutor
+
+import scala.collection.mutable.ArrayBuffer
+
+/**
+ * Copy from flink, because we need to customize it to make sure
+ * it work with multiple versions of flink.
+ */
+object FlinkShell {
+
+  object ExecutionMode extends Enumeration {
+    val UNDEFINED, LOCAL, REMOTE, YARN = Value
+  }
+
+  /** Configuration object */
+  case class Config(
+    host: Option[String] = None,
+    port: Option[Int] = None,
+    externalJars: Option[Array[String]] = None,
+    executionMode: ExecutionMode.Value = ExecutionMode.UNDEFINED,
+    yarnConfig: Option[YarnConfig] = None,
+    configDir: Option[String] = None
+  )
+
+  /** YARN configuration object */
+  case class YarnConfig(
+    jobManagerMemory: Option[String] = None,
+    name: Option[String] = None,
+    queue: Option[String] = None,
+    slots: Option[Int] = None,
+    taskManagerMemory: Option[String] = None
+  )
+
+  /** Buffered reader to substitute input in test */
+  var bufferedReader: Option[BufferedReader] = None
+
+  @Internal def ensureYarnConfig(config: Config) = config.yarnConfig match {
+    case Some(yarnConfig) => yarnConfig
+    case None => YarnConfig()
+  }
+
+  private def getConfigDir(config: Config) = {
+    config.configDir.getOrElse(CliFrontend.getConfigurationDirectoryFromEnv)
+  }
+
+  @Internal def fetchConnectionInfo(
+      config: Config,
+      flinkConfig: Configuration,
+      flinkShims: FlinkShims): (Configuration, Option[ClusterClient[_]]) = {
+
+    config.executionMode match {
+      case ExecutionMode.LOCAL => createLocalClusterAndConfig(flinkConfig)
+      case ExecutionMode.REMOTE => createRemoteConfig(config, flinkConfig)
+      case ExecutionMode.YARN => createYarnClusterIfNeededAndGetConfig(config, flinkConfig, flinkShims)
+      case ExecutionMode.UNDEFINED => // Wrong input
+        throw new IllegalArgumentException("please specify execution mode:\n" +
+          "[local | remote <host> <port> | yarn]")
+    }
+  }
+
+  private def createYarnClusterIfNeededAndGetConfig(config: Config, flinkConfig: Configuration, flinkShims: FlinkShims) = {
+    flinkConfig.setBoolean(DeploymentOptions.ATTACHED, true)
+
+    val (clusterConfig, clusterClient) = config.yarnConfig match {
+      case Some(_) => deployNewYarnCluster(config, flinkConfig, flinkShims)
+      case None => (flinkConfig, None)
+    }
+
+    // workaround for FLINK-17788, otherwise it won't work with flink 1.10.1 which has been released.
+    flinkConfig.set(DeploymentOptions.TARGET, YarnSessionClusterExecutor.NAME)
+
+    val (effectiveConfig, _) = clusterClient match {
+      case Some(_) => fetchDeployedYarnClusterInfo(config, clusterConfig, "yarn-cluster", flinkShims)
+      case None => fetchDeployedYarnClusterInfo(config, clusterConfig, "default", flinkShims)
+    }
+
+    println("Configuration: " + effectiveConfig)
+
+    (effectiveConfig, clusterClient)
+  }
+
+  private def deployNewYarnCluster(config: Config, flinkConfig: Configuration, flinkShims: FlinkShims) = {
+    val effectiveConfig = new Configuration(flinkConfig)
+    val args = parseArgList(config, "yarn-cluster")
+
+    val configurationDirectory = getConfigDir(config)
+
+    val frontend = new CliFrontend(
+      effectiveConfig,
+      CliFrontend.loadCustomCommandLines(effectiveConfig, configurationDirectory))
+
+    val commandOptions = CliFrontendParser.getRunCommandOptions
+    val commandLineOptions = CliFrontendParser.mergeOptions(commandOptions,
+      frontend.getCustomCommandLineOptions)
+    val commandLine = CliFrontendParser.parse(commandLineOptions, args, true)
+
+    val customCLI = flinkShims.getCustomCli(frontend, commandLine).asInstanceOf[CustomCommandLine]
+    val executorConfig = customCLI.applyCommandLineOptionsToConfiguration(commandLine)
+
+    val serviceLoader = new DefaultClusterClientServiceLoader
+    val clientFactory = serviceLoader.getClusterClientFactory(executorConfig)
+    val clusterDescriptor = clientFactory.createClusterDescriptor(executorConfig)
+    val clusterSpecification = clientFactory.getClusterSpecification(executorConfig)
+
+    val clusterClient = try {
+      clusterDescriptor
+        .deploySessionCluster(clusterSpecification)
+        .getClusterClient
+    } finally {
+      executorConfig.set(DeploymentOptions.TARGET, "yarn-session")
+      clusterDescriptor.close()
+    }
+
+    (executorConfig, Some(clusterClient))
+  }
+
+  private def fetchDeployedYarnClusterInfo(
+      config: Config,
+      flinkConfig: Configuration,
+      mode: String,
+      flinkShims: FlinkShims) = {
+
+    val effectiveConfig = new Configuration(flinkConfig)
+    val args = parseArgList(config, mode)
+
+    val configurationDirectory = getConfigDir(config)
+
+    val frontend = new CliFrontend(
+      effectiveConfig,
+      CliFrontend.loadCustomCommandLines(effectiveConfig, configurationDirectory))
+
+    val commandOptions = CliFrontendParser.getRunCommandOptions
+    val commandLineOptions = CliFrontendParser.mergeOptions(commandOptions,
+      frontend.getCustomCommandLineOptions)
+    val commandLine = CliFrontendParser.parse(commandLineOptions, args, true)
+
+    val customCLI = flinkShims.getCustomCli(frontend, commandLine).asInstanceOf[CustomCommandLine]
+    val executorConfig = customCLI.applyCommandLineOptionsToConfiguration(commandLine);
+
+    (executorConfig, None)
+  }
+
+  def parseArgList(config: Config, mode: String): Array[String] = {
+    val args = if (mode == "default") {
+      ArrayBuffer[String]()
+    } else {
+      ArrayBuffer[String]("-m", mode)
+    }
+
+    config.yarnConfig match {
+      case Some(yarnConfig) =>
+        yarnConfig.jobManagerMemory.foreach((jmMem) => args ++= Seq("-yjm", jmMem.toString))
+        yarnConfig.taskManagerMemory.foreach((tmMem) => args ++= Seq("-ytm", tmMem.toString))
+        yarnConfig.name.foreach((name) => args ++= Seq("-ynm", name.toString))
+        yarnConfig.queue.foreach((queue) => args ++= Seq("-yqu", queue.toString))
+        yarnConfig.slots.foreach((slots) => args ++= Seq("-ys", slots.toString))
+        args.toArray
+      case None => args.toArray
+    }
+  }
+
+  private def createRemoteConfig(
+      config: Config,
+      flinkConfig: Configuration): (Configuration, None.type) = {
+
+    if (config.host.isEmpty || config.port.isEmpty) {
+      throw new IllegalArgumentException("<host> or <port> is not specified!")
+    }
+
+    val effectiveConfig = new Configuration(flinkConfig)
+    setJobManagerInfoToConfig(effectiveConfig, config.host.get, config.port.get)
+    effectiveConfig.set(DeploymentOptions.TARGET, RemoteExecutor.NAME)
+    effectiveConfig.setBoolean(DeploymentOptions.ATTACHED, true)
+
+    (effectiveConfig, None)
+  }
+
+  private def createLocalClusterAndConfig(flinkConfig: Configuration) = {
+    val config = new Configuration(flinkConfig)
+    config.setInteger(JobManagerOptions.PORT, 0)
+
+    val cluster = createLocalCluster(config)
+    val port = cluster.getRestAddress.get.getPort
+
+    setJobManagerInfoToConfig(config, "localhost", port)
+    config.set(DeploymentOptions.TARGET, RemoteExecutor.NAME)
+    config.setBoolean(DeploymentOptions.ATTACHED, true)
+
+    println(s"\nStarting local Flink cluster (host: localhost, port: ${port}).\n")
+
+    val clusterClient = new MiniClusterClient(config, cluster)
+    (config, Some(clusterClient))
+  }
+
+  private def createLocalCluster(flinkConfig: Configuration) = {
+
+    val numTaskManagers = flinkConfig.getInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER,
+      ConfigConstants.DEFAULT_LOCAL_NUMBER_TASK_MANAGER)
+    val numSlotsPerTaskManager = flinkConfig.getInteger(TaskManagerOptions.NUM_TASK_SLOTS)
+
+    val miniClusterConfig = new MiniClusterConfiguration.Builder()
+      .setConfiguration(flinkConfig)
+      .setNumSlotsPerTaskManager(numSlotsPerTaskManager)
+      .setNumTaskManagers(numTaskManagers)
+      .build()
+
+    val cluster = new MiniCluster(miniClusterConfig)
+    cluster.start()
+    cluster
+  }
+
+  private def setJobManagerInfoToConfig(
+      config: Configuration,
+      host: String, port: Integer): Unit = {
+
+    config.setString(JobManagerOptions.ADDRESS, host)
+    config.setInteger(JobManagerOptions.PORT, port)
+
+    config.setString(RestOptions.ADDRESS, host)
+    config.setInteger(RestOptions.PORT, port)
+  }
+}