[
https://issues.apache.org/jira/browse/KYLIN-5141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17475918#comment-17475918
]
ASF GitHub Bot commented on KYLIN-5141:
---------------------------------------
zhangayqian commented on a change in pull request #1780:
URL: https://github.com/apache/kylin/pull/1780#discussion_r784476015
##########
File path:
source-hive/src/main/java/org/apache/kylin/source/hive/SparkHiveDictReplaceStep.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.kylin.source.hive;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.AbstractApplication;
+import org.apache.kylin.common.util.OptionsHelper;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.EngineFactory;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.common.SerializableConfiguration;
+import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.Map;
+
+public class SparkHiveDictReplaceStep extends AbstractApplication implements
Serializable {
+
+ private static final Logger logger =
LoggerFactory.getLogger(SparkHiveDictReplaceStep.class);
+
+ public static final Option OPTION_CUBE_NAME =
OptionBuilder.withArgName(BatchConstants.ARG_CUBE_NAME).hasArg().isRequired(true).withDescription("Cube
Name").create(BatchConstants.ARG_CUBE_NAME);
+ public static final Option OPTION_META_URL =
OptionBuilder.withArgName("metaUrl").hasArg().isRequired(true).withDescription("Meta
Url").create("metaUrl");
+ public static final Option OPTION_SEGMENT_ID =
OptionBuilder.withArgName("segmentId").hasArg().isRequired(true).withDescription("Segment
Id").create("segmentId");
+ public static final Option OPTION_FLOW_JOB_ID =
OptionBuilder.withArgName("flowJobId").hasArg().isRequired(true).withDescription("Flow
Job Id").create("flowJobId");
+ public static final Option OPTION_WAREHOUSE_DIR =
OptionBuilder.withArgName("wareHouseDir").hasArg().isRequired(true).withDescription("WareHouse
Dir").create("wareHouseDir");
+ public static final Option OPTION_DICT_COLUMNS =
OptionBuilder.withArgName("dictColumns").hasArg().isRequired(true).withDescription("Dict
Columns").create("dictColumns");
+ public static final Option OPTION_GLOBAL_DICT_TABLE =
OptionBuilder.withArgName("globalDictTable").hasArg().isRequired(true).withDescription("Global
Dict Table").create("globalDictTable");
+ public static final Option OPTION_DICT_SUFFIX =
OptionBuilder.withArgName("dictSuffix").hasArg().isRequired(true).withDescription("Dict
Suffix").create("dictSuffix");
+
+ private String cubeName;
+ private String segmentId;
+ private String metaUrl;
+ private String wareHouseDir;
+ private String dictColumns;
+ private String globalDictTable;
+ private String dictSuffix;
+ private String flowJobId;
+
+ private Options options;
+
+ public SparkHiveDictReplaceStep() {
+ options = new Options();
+ options.addOption(OPTION_CUBE_NAME);
+ options.addOption(OPTION_DICT_COLUMNS);
+ options.addOption(OPTION_META_URL);
+ options.addOption(OPTION_SEGMENT_ID);
+ options.addOption(OPTION_FLOW_JOB_ID);
+ options.addOption(OPTION_WAREHOUSE_DIR);
+ options.addOption(OPTION_GLOBAL_DICT_TABLE);
+ options.addOption(OPTION_DICT_SUFFIX);
+ }
+
+ @Override
+ protected Options getOptions() {
+ return options;
+ }
+
+ @Override
+ protected void execute(OptionsHelper optionsHelper) throws Exception {
+ initParam(optionsHelper);
+
+ try {
+ SparkSession spark = SparkSession.builder().appName("Spark Hive
Global Dict").master("yarn").config("spark.sql.warehouse.dir",
wareHouseDir).enableHiveSupport().getOrCreate();
+
+ SerializableConfiguration sConf = new
SerializableConfiguration(spark.sparkContext().hadoopConfiguration());
+ KylinConfig envConfig =
AbstractHadoopJob.loadKylinConfigFromHdfs(sConf, metaUrl);
+
+ CubeInstance cubeInstance =
CubeManager.getInstance(envConfig).getCube(cubeName);
+ CubeSegment seg = cubeInstance.getSegmentById(segmentId);
+ IJoinedFlatTableDesc flatDesc =
EngineFactory.getJoinedFlatTableDesc(seg);
+
+ Map<String, String> dictRef =
seg.getConfig().getMrHiveDictRefColumns();
+ for (String dictColumn : dictColumns.split(",")) {
+ dictRef.put(dictColumn, "");
+ }
+
+ Map<String, String> replaceSqlMap =
MRHiveDictUtil.generateReplaceSql(flatDesc, dictRef,
seg.getConfig().getHiveDatabaseForIntermediateTable(),
seg.getConfig().getMrHiveDictDB(), globalDictTable, dictSuffix);
+ for (Map.Entry<String, String> entry : replaceSqlMap.entrySet()) {
+ logger.info("field:" + entry.getKey() + ":\n" +
entry.getValue());
+ String replaceSql = entry.getValue();
+ replaceSql = replaceSql.substring(0,
replaceSql.lastIndexOf(";"));
+ spark.sql(replaceSql);
+ }
+ } catch (Exception e) {
+ logger.error("", e);
Review comment:
Please improve this error reporting information.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
> hive global dict replace with spark
> -----------------------------------
>
> Key: KYLIN-5141
> URL: https://issues.apache.org/jira/browse/KYLIN-5141
> Project: Kylin
> Issue Type: New Feature
> Affects Versions: v3.1.1, v3.1.2, v3.1.3
> Reporter: fengpod
> Priority: Minor
> Attachments: image-2021-12-09-18-28-25-428.png
>
>
> use spark engine in build-hive-global-dict-replace-intermediate-table step
> !image-2021-12-09-18-28-25-428.png!
--
This message was sent by Atlassian Jira
(v8.20.1#820001)