This is an automated email from the ASF dual-hosted git repository. xxyu pushed a commit to branch kylin-on-parquet-v2 in repository https://gitbox.apache.org/repos/asf/kylin.git
The following commit(s) were added to refs/heads/kylin-on-parquet-v2 by this push: new 9b0cc69 KYLIN-4741 Support to config the sparder application name 9b0cc69 is described below commit 9b0cc69e58029209242b210e23782b0cdb244ee9 Author: Zhichao Zhang <441586...@qq.com> AuthorDate: Tue Dec 22 10:43:02 2020 +0800 KYLIN-4741 Support to config the sparder application name --- .../org/apache/kylin/common/KylinConfigBase.java | 12 ++++ .../org/apache/spark/sql/SparderContext.scala | 4 +- .../org/apache/spark/sql/SparderAppNameTest.java | 78 ++++++++++++++++++++++ 3 files changed, 92 insertions(+), 2 deletions(-) diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java index 000ad80..f57f899 100644 --- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java +++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java @@ -1653,6 +1653,18 @@ public abstract class KylinConfigBase implements Serializable { return getOptional("kylin.env.hadoop-conf-dir", ""); } + /** + * Get the sparder app name, default value is: 'sparder_on_localhost-7070' + */ + public String getSparderAppName() { + String customSparderAppName = getOptional("kylin.query.sparder-context.app-name", ""); + if (StringUtils.isEmpty(customSparderAppName)) { + customSparderAppName = + "sparder_on_" + getServerRestAddress().replaceAll(":", "-"); + } + return customSparderAppName; + } + public String getSparkAdditionalJars() { return getOptional("kylin.engine.spark.additional-jars", ""); } diff --git a/kylin-spark-project/kylin-spark-query/src/main/scala/org/apache/spark/sql/SparderContext.scala b/kylin-spark-project/kylin-spark-query/src/main/scala/org/apache/spark/sql/SparderContext.scala index ba4c7b7..9e89a62 100644 --- a/kylin-spark-project/kylin-spark-query/src/main/scala/org/apache/spark/sql/SparderContext.scala +++ b/kylin-spark-project/kylin-spark-query/src/main/scala/org/apache/spark/sql/SparderContext.scala @@ -134,7 +134,7 @@ object SparderContext extends Logging { case "true" => SparkSession.builder .master("local") - .appName("sparder-test-sql-context") + .appName(kylinConf.getSparderAppName) .withExtensions { ext => ext.injectPlannerStrategy(_ => KylinSourceStrategy) } @@ -142,7 +142,7 @@ object SparderContext extends Logging { .getOrCreateKylinSession() case _ => SparkSession.builder - .appName("sparder-sql-context") + .appName(kylinConf.getSparderAppName) .master("yarn-client") .withExtensions { ext => ext.injectPlannerStrategy(_ => KylinSourceStrategy) diff --git a/kylin-spark-project/kylin-spark-query/src/test/java/org/apache/spark/sql/SparderAppNameTest.java b/kylin-spark-project/kylin-spark-query/src/test/java/org/apache/spark/sql/SparderAppNameTest.java new file mode 100644 index 0000000..e5c6c8c --- /dev/null +++ b/kylin-spark-project/kylin-spark-query/src/test/java/org/apache/spark/sql/SparderAppNameTest.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.util.Shell; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.engine.spark.LocalWithSparkSessionTest; +import org.apache.kylin.job.exception.SchedulerException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SparderAppNameTest extends LocalWithSparkSessionTest { + + private static final Logger logger = LoggerFactory.getLogger(SparderAppNameTest.class); + + @BeforeClass + public static void beforeClass() { + } + + @Override + @Before + public void setup() throws SchedulerException { + super.setup(); + if (Shell.MAC) + System.setProperty("org.xerial.snappy.lib.name", "libsnappyjava.jnilib");//for snappy + KylinConfig conf = KylinConfig.getInstanceFromEnv(); + conf.setProperty("kylin.query.spark-conf.spark.master", "local"); + SparderContext.getOriginalSparkSession(); + } + + @After + public void after() { + SparderContext.stopSpark(); + super.after(); + } + + @Test + public void testThreadSparkSession() { + Assert.assertTrue(StringUtils.isNotBlank( + SparderContext.getOriginalSparkSession().sparkContext().getConf() + .get("spark.app.name"))); + Assert.assertTrue(SparderContext.getOriginalSparkSession().sparkContext().getConf() + .get("spark.app.name").equals("sparder_on_localhost-7070")); + + KylinConfig config = KylinConfig.getInstanceFromEnv(); + config.setProperty("kylin.query.sparder-context.app-name", "test-sparder-app-name"); + SparderContext.restartSpark(); + + Assert.assertTrue(StringUtils.isNotBlank( + SparderContext.getOriginalSparkSession().sparkContext().getConf() + .get("spark.app.name"))); + Assert.assertTrue(SparderContext.getOriginalSparkSession().sparkContext().getConf() + .get("spark.app.name").equals("test-sparder-app-name")); + config.setProperty("kylin.query.sparder-context.app-name", ""); + } +}