 
                        
        
                        
        org.apache.hudi.utilities.HoodieIndexer Maven / Gradle / Ivy
/*
 * 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 org.apache.hudi.avro.model.HoodieIndexCommitMetadata;
import org.apache.hudi.avro.model.HoodieIndexPartitionInfo;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIndexException;
import org.apache.hudi.metadata.MetadataPartitionType;
import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter;
import org.apache.hadoop.fs.Path;
import org.apache.spark.api.java.JavaSparkContext;
import org.jetbrains.annotations.TestOnly;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
import java.util.Set;
import java.util.stream.Collectors;
import static org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE_METADATA_INDEX_BLOOM_FILTER;
import static org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS;
import static org.apache.hudi.common.config.HoodieMetadataConfig.RECORD_INDEX_ENABLE_PROP;
import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty;
import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_BLOOM_FILTERS;
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS;
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_RECORD_INDEX;
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightAndCompletedMetadataPartitions;
import static org.apache.hudi.utilities.UtilHelpers.EXECUTE;
import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE;
import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE_AND_EXECUTE;
/**
 * TODO: [HUDI-8294]
 * A tool to run metadata indexing asynchronously.
 * 
 * Example command (assuming indexer.properties contains related index configs, see {@link org.apache.hudi.common.config.HoodieMetadataConfig} for configs):
 * 
 * spark-submit \
 * --class org.apache.hudi.utilities.HoodieIndexer \
 * /path/to/hudi/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.11-0.11.0-SNAPSHOT.jar \
 * --props /path/to/indexer.properties \
 * --mode scheduleAndExecute \
 * --base-path /tmp/hudi_trips_cow \
 * --table-name hudi_trips_cow \
 * --index-types COLUMN_STATS \
 * --parallelism 1 \
 * --spark-memory 1g
 * 
 * A sample indexer.properties file:
 * 
 * hoodie.metadata.index.async=true
 * hoodie.metadata.index.column.stats.enable=true
 * hoodie.metadata.index.check.timeout.seconds=60
 * hoodie.write.concurrency.mode=optimistic_concurrency_control
 * hoodie.write.lock.provider=org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider
 */
public class HoodieIndexer {
  private static final Logger LOG = LoggerFactory.getLogger(HoodieIndexer.class);
  static final String DROP_INDEX = "dropindex";
  private final HoodieIndexer.Config cfg;
  private final TypedProperties props;
  private final JavaSparkContext jsc;
  private final HoodieTableMetaClient metaClient;
  public HoodieIndexer(JavaSparkContext jsc, HoodieIndexer.Config cfg) {
    this.cfg = cfg;
    this.jsc = jsc;
    this.props = isNullOrEmpty(cfg.propsFilePath)
        ? UtilHelpers.buildProperties(cfg.configs)
        : readConfigFromFileSystem(jsc, cfg);
    this.metaClient = UtilHelpers.createMetaClient(jsc, cfg.basePath, true);
  }
  private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, HoodieIndexer.Config cfg) {
    return UtilHelpers.readConfig(jsc.hadoopConfiguration(), new Path(cfg.propsFilePath), cfg.configs)
        .getProps(true);
  }
  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 = "Indexing Instant time")
    public String indexInstantTime = null;
    @Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert", required = true)
    public int parallelism = 1;
    @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master")
    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")
    public int retry = 0;
    @Parameter(names = {"--index-types", "-ixt"}, description = "Comma-separated index types to be built, e.g. BLOOM_FILTERS,COLUMN_STATS", required = true)
    public String indexTypes = null;
    @Parameter(names = {"--mode", "-m"}, description = "Set job mode: Set \"schedule\" to generate an indexing plan; "
        + "Set \"execute\" to execute the indexing plan at the given instant, which means --instant-time is required here; "
        + "Set \"scheduleAndExecute\" to generate an indexing plan first and execute that plan immediately;"
        + "Set \"dropIndex\" to drop the index types specified in --index-types;")
    public String runningMode = null;
    @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 indexing")
    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 configs = new ArrayList<>();
  }
  public static void main(String[] args) {
    final HoodieIndexer.Config cfg = new HoodieIndexer.Config();
    JCommander cmd = new JCommander(cfg, null, args);
    if (cfg.help || args.length == 0) {
      cmd.usage();
      throw new HoodieException("Indexing failed for basePath : " + cfg.basePath);
    }
    final JavaSparkContext jsc = UtilHelpers.buildSparkContext("indexing-" + cfg.tableName, cfg.sparkMaster, cfg.sparkMemory);
    HoodieIndexer indexer = new HoodieIndexer(jsc, cfg);
    int result = indexer.start(cfg.retry);
    String resultMsg = String.format("Indexing with basePath: %s, tableName: %s, runningMode: %s",
        cfg.basePath, cfg.tableName, cfg.runningMode);
    if (result != 0) {
      throw new HoodieException(resultMsg + " failed");
    }
    LOG.info(resultMsg + " success");
    jsc.stop();
  }
  public int start(int retry) {
    // indexing should be done only if metadata is enabled
    if (!props.getBoolean(HoodieMetadataConfig.ENABLE.key())) {
      LOG.error(String.format("Metadata is not enabled. Please set %s to true.", HoodieMetadataConfig.ENABLE.key()));
      return -1;
    }
    // all inflight or completed metadata partitions have already been initialized
    // so enable corresponding indexes in the props so that they're not deleted
    Set initializedMetadataPartitions = getInflightAndCompletedMetadataPartitions(metaClient.getTableConfig());
    LOG.info("Setting props for: " + initializedMetadataPartitions);
    initializedMetadataPartitions.forEach(p -> {
      if (PARTITION_NAME_COLUMN_STATS.equals(p)) {
        props.setProperty(ENABLE_METADATA_INDEX_COLUMN_STATS.key(), "true");
      }
      if (PARTITION_NAME_BLOOM_FILTERS.equals(p)) {
        props.setProperty(ENABLE_METADATA_INDEX_BLOOM_FILTER.key(), "true");
      }
      if (PARTITION_NAME_RECORD_INDEX.equals(p)) {
        props.setProperty(RECORD_INDEX_ENABLE_PROP.key(), "true");
      }
    });
    return UtilHelpers.retry(retry, () -> {
      switch (cfg.runningMode.toLowerCase()) {
        case SCHEDULE: {
          LOG.info("Running Mode: [" + SCHEDULE + "]; Do schedule");
          Option instantTime = scheduleIndexing(jsc);
          int result = instantTime.isPresent() ? 0 : -1;
          if (result == 0) {
            LOG.info("The schedule instant time is " + instantTime.get());
          }
          return result;
        }
        case SCHEDULE_AND_EXECUTE: {
          LOG.info("Running Mode: [" + SCHEDULE_AND_EXECUTE + "]");
          return scheduleAndRunIndexing(jsc);
        }
        case EXECUTE: {
          LOG.info("Running Mode: [" + EXECUTE + "];");
          return runIndexing(jsc);
        }
        case DROP_INDEX: {
          LOG.info("Running Mode: [" + DROP_INDEX + "];");
          return dropIndex(jsc);
        }
        default: {
          LOG.info("Unsupported running mode [" + cfg.runningMode + "], quit the job directly");
          return -1;
        }
      }
    }, "Indexer failed");
  }
  @TestOnly
  public Option doSchedule() throws Exception {
    return this.scheduleIndexing(jsc);
  }
  private Option scheduleIndexing(JavaSparkContext jsc) throws Exception {
    String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient);
    try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) {
      return doSchedule(client);
    }
  }
  private Option doSchedule(SparkRDDWriteClient client) {
    HoodieMetadataConfig metadataConfig = getHoodieMetadataConfig();
    List partitionTypes = getRequestedPartitionTypes(cfg.indexTypes, Option.of(metadataConfig));
    checkArgument(partitionTypes.size() == 1, "Currently, only one index type can be scheduled at a time.");
    if (!isMetadataInitialized() && !partitionTypes.contains(MetadataPartitionType.FILES)) {
      throw new HoodieException("Metadata table is not yet initialized. Initialize FILES partition before any other partition " + Arrays.toString(partitionTypes.toArray()));
    }
    if (indexExists(partitionTypes)) {
      return Option.empty();
    }
    Option indexingInstant = client.scheduleIndexing(partitionTypes, Collections.emptyList());
    if (!indexingInstant.isPresent()) {
      LOG.error("Scheduling of index action did not return any instant.");
    }
    return indexingInstant;
  }
  private HoodieMetadataConfig getHoodieMetadataConfig() {
    props.setProperty(HoodieWriteConfig.BASE_PATH.key(), cfg.basePath);
    HoodieWriteConfig dataTableWriteConfig = HoodieWriteConfig.newBuilder().withProps(props).build();
    return dataTableWriteConfig.getMetadataConfig();
  }
  private boolean indexExists(List partitionTypes) {
    Set indexedMetadataPartitions = metaClient.getTableConfig().getMetadataPartitions();
    Set requestedIndexPartitionPaths = partitionTypes.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet());
    requestedIndexPartitionPaths.retainAll(indexedMetadataPartitions);
    if (!requestedIndexPartitionPaths.isEmpty()) {
      LOG.error("Following indexes already built: " + requestedIndexPartitionPaths);
      return true;
    }
    return false;
  }
  private boolean isMetadataInitialized() {
    Set indexedMetadataPartitions = metaClient.getTableConfig().getMetadataPartitions();
    return !indexedMetadataPartitions.isEmpty();
  }
  private int runIndexing(JavaSparkContext jsc) throws Exception {
    String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient);
    try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) {
      if (isNullOrEmpty(cfg.indexInstantTime)) {
        // Instant time is not specified
        // Find the earliest scheduled indexing instant for execution
        Option earliestPendingIndexInstant = metaClient.getActiveTimeline()
            .filterPendingIndexTimeline()
            .firstInstant();
        if (earliestPendingIndexInstant.isPresent()) {
          cfg.indexInstantTime = earliestPendingIndexInstant.get().requestedTime();
          LOG.info("Found the earliest scheduled indexing instant which will be executed: "
              + cfg.indexInstantTime);
        } else {
          throw new HoodieIndexException("There is no scheduled indexing in the table.");
        }
      }
      return handleResponse(client.index(cfg.indexInstantTime)) ? 0 : 1;
    }
  }
  private int scheduleAndRunIndexing(JavaSparkContext jsc) throws Exception {
    String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient);
    try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) {
      Option indexingInstantTime = doSchedule(client);
      if (indexingInstantTime.isPresent()) {
        return handleResponse(client.index(indexingInstantTime.get())) ? 0 : 1;
      } else {
        return -1;
      }
    }
  }
  private int dropIndex(JavaSparkContext jsc) throws Exception {
    List partitionTypes = getRequestedPartitionTypes(cfg.indexTypes, Option.empty())
        .stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toList());
    String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient);
    try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) {
      client.dropIndex(partitionTypes);
      return 0;
    } catch (Exception e) {
      LOG.error("Failed to drop index. ", e);
      return -1;
    }
  }
  private boolean handleResponse(Option commitMetadata) {
    if (!commitMetadata.isPresent()) {
      LOG.error("Indexing failed as no commit metadata present.");
      return false;
    }
    List indexPartitionInfos = commitMetadata.get().getIndexPartitionInfos();
    LOG.info("Indexing complete for partitions: {}", indexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toList()));
    return isIndexBuiltForAllRequestedTypes(indexPartitionInfos);
  }
  boolean isIndexBuiltForAllRequestedTypes(List indexPartitionInfos) {
    Set indexedPartitions = indexPartitionInfos.stream()
        .map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet());
    Set requestedPartitions = getRequestedPartitionTypes(cfg.indexTypes, Option.empty()).stream()
        .map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet());
    requestedPartitions.removeAll(indexedPartitions);
    if (requestedPartitions.isEmpty()) {
      return true;
    }
    metaClient.reloadTableConfig();
    return metaClient.getTableConfig().getMetadataPartitions().containsAll(indexedPartitions);
  }
  List getRequestedPartitionTypes(String indexTypes, Option metadataConfig) {
    List requestedIndexTypes = Arrays.asList(indexTypes.split(","));
    return requestedIndexTypes.stream()
        .map(p -> MetadataPartitionType.valueOf(p.toUpperCase(Locale.ROOT)))
        .collect(Collectors.toList());
  }
}