You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2021/01/05 14:58:42 UTC

[GitHub] [hudi] lw309637554 commented on a change in pull request #2379: [HUDI-1399] support a independent clustering spark job to asynchronously clustering

lw309637554 commented on a change in pull request #2379:
URL: https://github.com/apache/hudi/pull/2379#discussion_r551984380



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.hudi.utilities;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.TableSchemaResolver;
+import org.apache.hudi.common.util.Option;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class HoodieClusteringJob {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieClusteringJob.class);
+  private final Config cfg;
+  private transient FileSystem fs;
+  private TypedProperties props;
+  private final JavaSparkContext jsc;
+
+  public HoodieClusteringJob(JavaSparkContext jsc, Config cfg) {
+    this.cfg = cfg;
+    this.jsc = jsc;
+    this.props = cfg.propsFilePath == null
+        ? UtilHelpers.buildProperties(cfg.configs)
+        : readConfigFromFileSystem(jsc, cfg);
+  }
+
+  private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cfg) {
+    final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
+
+    return UtilHelpers
+        .readConfig(fs, new Path(cfg.propsFilePath), cfg.configs)
+        .getConfig();
+  }
+
+  public static class Config implements Serializable {
+    @Parameter(names = {"--base-path", "-sp"}, description = "Base path for the table", required = true)
+    public String basePath = null;
+    @Parameter(names = {"--table-name", "-tn"}, description = "Table name", required = true)
+    public String tableName = null;
+    @Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time", required = true)
+    public String clusteringInstantTime = null;
+    @Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert", required = false)
+    public int parallelism = 1;
+    @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false)
+    public String sparkMaster = null;
+    @Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = true)
+    public String sparkMemory = null;
+    @Parameter(names = {"--retry", "-rt"}, description = "number of retries", required = false)
+    public int retry = 0;
+
+    @Parameter(names = {"--schedule", "-sc"}, description = "Schedule clustering")
+    public Boolean runSchedule = false;
+    @Parameter(names = {"--help", "-h"}, help = true)
+    public Boolean help = false;
+
+    @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for "
+        + "hoodie client for clustering")
+    public String propsFilePath = null;
+
+    @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
+        + "(using the CLI parameter \"--props\") can also be passed command line using this parameter. This can be repeated",
+            splitter = IdentitySplitter.class)
+    public List<String> configs = new ArrayList<>();
+  }
+
+  public static void main(String[] args) {
+    final Config cfg = new Config();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    final JavaSparkContext jsc = UtilHelpers.buildSparkContext("clustering-" + cfg.tableName, cfg.sparkMaster, cfg.sparkMemory);
+    HoodieClusteringJob clusteringJob = new HoodieClusteringJob(jsc, cfg);
+    int result = clusteringJob.cluster(cfg.retry);
+    String resultMsg = String.format("Clustering with basePath: %s, tableName: %s, runSchedule: %s, clusteringInstantTime: %s",
+        cfg.basePath, cfg.tableName, cfg.runSchedule, cfg.clusteringInstantTime);
+    if (result == -1) {
+      LOG.error(resultMsg + " failed");
+    } else {
+      LOG.info(resultMsg + " success");
+    }
+    jsc.stop();
+  }
+
+  public int cluster(int retry) {
+    this.fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
+    int ret = -1;
+    try {
+      do {
+        if (cfg.runSchedule) {
+          LOG.info("Do schedule");
+          ret = doSchedule(jsc);
+        } else {
+          LOG.info("Do cluster");
+          ret = doCluster(jsc);
+        }
+      } while (ret != 0 && retry-- > 0);
+    } catch (Throwable t) {
+      LOG.error("Cluster failed", t);
+    }
+    return ret;
+  }
+
+  private String getSchemaFromLatestInstant() throws Exception {
+    HoodieTableMetaClient metaClient =  new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.basePath, true);
+    TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient);
+    Schema schema = schemaUtil.getTableAvroSchema(false);
+    return schema.toString();
+  }
+
+  private int doCluster(JavaSparkContext jsc) throws Exception {
+    String schemaStr = getSchemaFromLatestInstant();
+    SparkRDDWriteClient client =
+        UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props);
+    JavaRDD<WriteStatus> writeResponse =
+        (JavaRDD<WriteStatus>) client.cluster(cfg.clusteringInstantTime, true).getWriteStatuses();
+    return UtilHelpers.handleErrors(jsc, cfg.clusteringInstantTime, writeResponse);
+  }
+
+  private int doSchedule(JavaSparkContext jsc) throws Exception {
+    String schemaStr = getSchemaFromLatestInstant();
+    SparkRDDWriteClient client =
+        UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props);
+    return client.scheduleClusteringAtInstant(cfg.clusteringInstantTime, Option.empty()) ? 0 : -1;

Review comment:
       @satishkotha  make sense , have use client.scheduleClustering() to auto generate instantTime. Also have some questions for you. Why clustering instant time should greater than latest commit?  I see BaseScheduleCompactionActionExecutor.execute() have check  that should not have earliest write inflight instant time than compaction instant time. But BaseClusteringPlanActionExecutor.execute()  do not have similarity check. Thanks.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org