1
0

[HUDI-2488][HUDI-3175] Implement async metadata indexing (#4693)

- Add a new action called INDEX, whose state transition is described in the RFC.
- Changes in timeline to support the new action.
- Add an index planner in ScheduleIndexActionExecutor.
- Add index plan executor in RunIndexActionExecutor.
- Add 3 APIs in HoodieTableMetadataWriter; a) scheduleIndex: will generate an index plan based on latest completed instant, initialize file groups and add a requested INDEX instant, b) index: executes the index plan and also takes care of writes that happened after indexing was requested, c) dropIndex: will drop index by removing the given metadata partition.
- Add 2 new table configs to serve as the source of truth for inflight and completed indexes.
- Support upgrade/downgrade taking care of the newly added configs.
- Add tool to trigger indexing in HoodieIndexer.
- Handle corner cases related to partial failures.
- Abort gracefully after deleting partition and instant.
- Handle other actions in timeline to consider before catching up
This commit is contained in:
Sagar Sumit
2022-04-01 01:33:12 +05:30
committed by GitHub
parent 1da196c1e8
commit 28dafa774e
44 changed files with 2123 additions and 150 deletions

View File

@@ -20,24 +20,19 @@ package org.apache.hudi.utilities;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.exception.HoodieClusteringException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.table.HoodieSparkTable;
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.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -49,15 +44,15 @@ import java.util.ArrayList;
import java.util.Date;
import java.util.List;
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;
public class HoodieClusteringJob {
public static final String EXECUTE = "execute";
public static final String SCHEDULE = "schedule";
public static final String SCHEDULE_AND_EXECUTE = "scheduleandexecute";
private static final Logger LOG = LogManager.getLogger(HoodieClusteringJob.class);
private final Config cfg;
private transient FileSystem fs;
private TypedProperties props;
private final TypedProperties props;
private final JavaSparkContext jsc;
private final HoodieTableMetaClient metaClient;
@@ -83,34 +78,34 @@ public class HoodieClusteringJob {
@Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time, only used when set --mode execute. "
+ "If the instant time is not provided with --mode execute, "
+ "the earliest scheduled clustering instant time is used by default. "
+ "When set \"--mode scheduleAndExecute\" this instant-time will be ignored.", required = false)
+ "When set \"--mode scheduleAndExecute\" this instant-time will be ignored.")
public String clusteringInstantTime = null;
@Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert", required = false)
@Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert")
public int parallelism = 1;
@Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false)
@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", required = false)
@Parameter(names = {"--retry", "-rt"}, description = "number of retries")
public int retry = 0;
@Parameter(names = {"--schedule", "-sc"}, description = "Schedule clustering @desperate soon please use \"--mode schedule\" instead")
public Boolean runSchedule = false;
@Parameter(names = {"--retry-last-failed-clustering-job", "-rc"}, description = "Take effect when using --mode/-m scheduleAndExecute. Set true means "
+ "check, rollback and execute last failed clustering plan instead of planing a new clustering job directly.", required = false)
+ "check, rollback and execute last failed clustering plan instead of planing a new clustering job directly.")
public Boolean retryLastFailedClusteringJob = false;
@Parameter(names = {"--mode", "-m"}, description = "Set job mode: Set \"schedule\" means make a cluster plan; "
+ "Set \"execute\" means execute a cluster plan at given instant which means --instant-time is needed here; "
+ "Set \"scheduleAndExecute\" means make a cluster plan first and execute that plan immediately", required = false)
+ "Set \"execute\" means execute a cluster plan at given instant which means --instant-time is needed here; "
+ "Set \"scheduleAndExecute\" means make a cluster plan first and execute that plan immediately")
public String runningMode = null;
@Parameter(names = {"--help", "-h"}, help = true)
public Boolean help = false;
@Parameter(names = {"--job-max-processing-time-ms", "-jt"}, description = "Take effect when using --mode/-m scheduleAndExecute and --retry-last-failed-clustering-job/-rc true. "
+ "If maxProcessingTimeMs passed but clustering job is still unfinished, hoodie would consider this job as failed and relaunch.", required = false)
+ "If maxProcessingTimeMs passed but clustering job is still unfinished, hoodie would consider this job as failed and relaunch.")
public long maxProcessingTimeMs = 0;
@Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for "
@@ -119,7 +114,7 @@ public class HoodieClusteringJob {
@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)
splitter = IdentitySplitter.class)
public List<String> configs = new ArrayList<>();
}
@@ -155,10 +150,9 @@ public class HoodieClusteringJob {
}
public int cluster(int retry) {
this.fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
// need to do validate in case that users call cluster() directly without setting cfg.runningMode
validateRunningMode(cfg);
int ret = UtilHelpers.retry(retry, () -> {
return UtilHelpers.retry(retry, () -> {
switch (cfg.runningMode.toLowerCase()) {
case SCHEDULE: {
LOG.info("Running Mode: [" + SCHEDULE + "]; Do schedule");
@@ -183,20 +177,10 @@ public class HoodieClusteringJob {
}
}
}, "Cluster failed");
return ret;
}
private String getSchemaFromLatestInstant() throws Exception {
TableSchemaResolver schemaResolver = new TableSchemaResolver(metaClient);
if (metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 0) {
throw new HoodieException("Cannot run clustering without any completed commits");
}
Schema schema = schemaResolver.getTableAvroSchema(false);
return schema.toString();
}
private int doCluster(JavaSparkContext jsc) throws Exception {
String schemaStr = getSchemaFromLatestInstant();
String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient);
try (SparkRDDWriteClient<HoodieRecordPayload> client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) {
if (StringUtils.isNullOrEmpty(cfg.clusteringInstantTime)) {
// Instant time is not specified
@@ -224,7 +208,7 @@ public class HoodieClusteringJob {
}
private Option<String> doSchedule(JavaSparkContext jsc) throws Exception {
String schemaStr = getSchemaFromLatestInstant();
String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient);
try (SparkRDDWriteClient<HoodieRecordPayload> client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) {
return doSchedule(client);
}
@@ -240,7 +224,7 @@ public class HoodieClusteringJob {
private int doScheduleAndCluster(JavaSparkContext jsc) throws Exception {
LOG.info("Step 1: Do schedule");
String schemaStr = getSchemaFromLatestInstant();
String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient);
try (SparkRDDWriteClient<HoodieRecordPayload> client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) {
Option<String> instantTime = Option.empty();

View File

@@ -0,0 +1,307 @@
/*
* 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.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.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.jetbrains.annotations.TestOnly;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import java.util.Set;
import java.util.stream.Collectors;
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.getCompletedMetadataPartitions;
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;
/**
* A tool to run metadata indexing asynchronously.
* <p>
* Example command (assuming indexer.properties contains related index configs, see {@link org.apache.hudi.common.config.HoodieMetadataConfig} for configs):
* <p>
* 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
* <p>
* A sample indexer.properties file:
* <p>
* 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 = LogManager.getLogger(HoodieIndexer.class);
private static final String DROP_INDEX = "dropindex";
private final HoodieIndexer.Config cfg;
private 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<String> 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();
System.exit(1);
}
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 == -1) {
LOG.error(resultMsg + " failed");
} else {
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;
}
return UtilHelpers.retry(retry, () -> {
switch (cfg.runningMode.toLowerCase()) {
case SCHEDULE: {
LOG.info("Running Mode: [" + SCHEDULE + "]; Do schedule");
Option<String> 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<String> doSchedule() throws Exception {
return this.scheduleIndexing(jsc);
}
private Option<String> scheduleIndexing(JavaSparkContext jsc) throws Exception {
String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient);
try (SparkRDDWriteClient<HoodieRecordPayload> client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) {
return doSchedule(client);
}
}
private Option<String> doSchedule(SparkRDDWriteClient<HoodieRecordPayload> client) {
List<MetadataPartitionType> partitionTypes = getRequestedPartitionTypes(cfg.indexTypes);
checkArgument(partitionTypes.size() == 1, "Currently, only one index type can be scheduled at a time.");
if (indexExists(partitionTypes)) {
return Option.empty();
}
Option<String> indexingInstant = client.scheduleIndexing(partitionTypes);
if (!indexingInstant.isPresent()) {
LOG.error("Scheduling of index action did not return any instant.");
}
return indexingInstant;
}
private boolean indexExists(List<MetadataPartitionType> partitionTypes) {
Set<String> indexedMetadataPartitions = getCompletedMetadataPartitions(metaClient.getTableConfig());
Set<String> 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 int runIndexing(JavaSparkContext jsc) throws Exception {
String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient);
try (SparkRDDWriteClient<HoodieRecordPayload> 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<HoodieInstant> earliestPendingIndexInstant = metaClient.getActiveTimeline()
.filterPendingIndexTimeline()
.firstInstant();
if (earliestPendingIndexInstant.isPresent()) {
cfg.indexInstantTime = earliestPendingIndexInstant.get().getTimestamp();
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<HoodieRecordPayload> client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) {
Option<String> 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<MetadataPartitionType> partitionTypes = getRequestedPartitionTypes(cfg.indexTypes);
String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient);
try (SparkRDDWriteClient<HoodieRecordPayload> 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<HoodieIndexCommitMetadata> commitMetadata) {
if (!commitMetadata.isPresent()) {
LOG.error("Indexing failed as no commit metadata present.");
return false;
}
List<HoodieIndexPartitionInfo> indexPartitionInfos = commitMetadata.get().getIndexPartitionInfos();
LOG.info(String.format("Indexing complete for partitions: %s",
indexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toList())));
return isIndexBuiltForAllRequestedTypes(indexPartitionInfos);
}
boolean isIndexBuiltForAllRequestedTypes(List<HoodieIndexPartitionInfo> indexPartitionInfos) {
Set<String> indexedPartitions = indexPartitionInfos.stream()
.map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet());
Set<String> requestedPartitions = getRequestedPartitionTypes(cfg.indexTypes).stream()
.map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet());
requestedPartitions.removeAll(indexedPartitions);
return requestedPartitions.isEmpty();
}
List<MetadataPartitionType> getRequestedPartitionTypes(String indexTypes) {
List<String> requestedIndexTypes = Arrays.asList(indexTypes.split(","));
return requestedIndexTypes.stream()
.map(p -> MetadataPartitionType.valueOf(p.toUpperCase(Locale.ROOT)))
// FILES partition is initialized synchronously while getting metadata writer
.filter(p -> !MetadataPartitionType.FILES.equals(p))
.collect(Collectors.toList());
}
}

View File

@@ -104,21 +104,26 @@ import java.util.Properties;
* Bunch of helper methods.
*/
public class UtilHelpers {
public static final String EXECUTE = "execute";
public static final String SCHEDULE = "schedule";
public static final String SCHEDULE_AND_EXECUTE = "scheduleandexecute";
private static final Logger LOG = LogManager.getLogger(UtilHelpers.class);
public static Source createSource(String sourceClass, TypedProperties cfg, JavaSparkContext jssc,
SparkSession sparkSession, SchemaProvider schemaProvider,
HoodieDeltaStreamerMetrics metrics) throws IOException {
SparkSession sparkSession, SchemaProvider schemaProvider,
HoodieDeltaStreamerMetrics metrics) throws IOException {
try {
try {
return (Source) ReflectionUtils.loadClass(sourceClass,
new Class<?>[]{TypedProperties.class, JavaSparkContext.class,
new Class<?>[] {TypedProperties.class, JavaSparkContext.class,
SparkSession.class, SchemaProvider.class,
HoodieDeltaStreamerMetrics.class},
cfg, jssc, sparkSession, schemaProvider, metrics);
} catch (HoodieException e) {
return (Source) ReflectionUtils.loadClass(sourceClass,
new Class<?>[]{TypedProperties.class, JavaSparkContext.class,
new Class<?>[] {TypedProperties.class, JavaSparkContext.class,
SparkSession.class, SchemaProvider.class},
cfg, jssc, sparkSession, schemaProvider);
}
@@ -238,7 +243,7 @@ public class UtilHelpers {
/**
* Parse Schema from file.
*
* @param fs File System
* @param fs File System
* @param schemaFile Schema File
*/
public static String parseSchema(FileSystem fs, String schemaFile) throws Exception {
@@ -300,13 +305,13 @@ public class UtilHelpers {
/**
* Build Hoodie write client.
*
* @param jsc Java Spark Context
* @param basePath Base Path
* @param schemaStr Schema
* @param jsc Java Spark Context
* @param basePath Base Path
* @param schemaStr Schema
* @param parallelism Parallelism
*/
public static SparkRDDWriteClient<HoodieRecordPayload> createHoodieClient(JavaSparkContext jsc, String basePath, String schemaStr,
int parallelism, Option<String> compactionStrategyClass, TypedProperties properties) {
int parallelism, Option<String> compactionStrategyClass, TypedProperties properties) {
HoodieCompactionConfig compactionConfig = compactionStrategyClass
.map(strategy -> HoodieCompactionConfig.newBuilder().withInlineCompaction(false)
.withCompactionStrategy(ReflectionUtils.loadClass(strategy)).build())
@@ -466,8 +471,7 @@ public class UtilHelpers {
Option.ofNullable(createSchemaPostProcessor(schemaPostProcessorClass, cfg, jssc)));
}
public static SchemaProvider createRowBasedSchemaProvider(StructType structType,
TypedProperties cfg, JavaSparkContext jssc) {
public static SchemaProvider createRowBasedSchemaProvider(StructType structType, TypedProperties cfg, JavaSparkContext jssc) {
SchemaProvider rowSchemaProvider = new RowBasedSchemaProvider(structType);
return wrapSchemaProviderWithPostProcessor(rowSchemaProvider, cfg, jssc, null);
}
@@ -476,13 +480,13 @@ public class UtilHelpers {
* Create latest schema provider for Target schema.
*
* @param structType spark data type of incoming batch.
* @param jssc instance of {@link JavaSparkContext}.
* @param fs instance of {@link FileSystem}.
* @param basePath base path of the table.
* @param jssc instance of {@link JavaSparkContext}.
* @param fs instance of {@link FileSystem}.
* @param basePath base path of the table.
* @return the schema provider where target schema refers to latest schema(either incoming schema or table schema).
*/
public static SchemaProvider createLatestSchemaProvider(StructType structType,
JavaSparkContext jssc, FileSystem fs, String basePath) {
JavaSparkContext jssc, FileSystem fs, String basePath) {
SchemaProvider rowSchemaProvider = new RowBasedSchemaProvider(structType);
Schema writeSchema = rowSchemaProvider.getTargetSchema();
Schema latestTableSchema = writeSchema;
@@ -540,4 +544,12 @@ public class UtilHelpers {
return ret;
}
public static String getSchemaFromLatestInstant(HoodieTableMetaClient metaClient) throws Exception {
TableSchemaResolver schemaResolver = new TableSchemaResolver(metaClient);
if (metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 0) {
throw new HoodieException("Cannot run clustering without any completed commits");
}
Schema schema = schemaResolver.getTableAvroSchema(false);
return schema.toString();
}
}

View File

@@ -0,0 +1,122 @@
/*
* 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.HoodieReadClient;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.metadata.MetadataPartitionType;
import org.apache.hudi.testutils.providers.SparkProvider;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SQLContext;
import org.apache.spark.sql.SparkSession;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestHoodieIndexer extends HoodieCommonTestHarness implements SparkProvider {
private static transient SparkSession spark;
private static transient SQLContext sqlContext;
private static transient JavaSparkContext jsc;
private static transient HoodieSparkEngineContext context;
@BeforeEach
public void init() throws IOException {
boolean initialized = spark != null;
if (!initialized) {
SparkConf sparkConf = conf();
SparkRDDWriteClient.registerClasses(sparkConf);
HoodieReadClient.addHoodieSupport(sparkConf);
spark = SparkSession.builder().config(sparkConf).getOrCreate();
sqlContext = spark.sqlContext();
jsc = new JavaSparkContext(spark.sparkContext());
context = new HoodieSparkEngineContext(jsc);
}
initPath();
metaClient = HoodieTestUtils.init(basePath, getTableType());
}
@Test
public void testGetRequestedPartitionTypes() {
HoodieIndexer.Config config = new HoodieIndexer.Config();
config.basePath = basePath;
config.tableName = "indexer_test";
config.indexTypes = "FILES,BLOOM_FILTERS,COLUMN_STATS";
HoodieIndexer indexer = new HoodieIndexer(jsc, config);
List<MetadataPartitionType> partitionTypes = indexer.getRequestedPartitionTypes(config.indexTypes);
assertFalse(partitionTypes.contains(MetadataPartitionType.FILES));
assertTrue(partitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS));
assertTrue(partitionTypes.contains(MetadataPartitionType.COLUMN_STATS));
}
@Test
public void testIsIndexBuiltForAllRequestedTypes() {
HoodieIndexer.Config config = new HoodieIndexer.Config();
config.basePath = basePath;
config.tableName = "indexer_test";
config.indexTypes = "BLOOM_FILTERS,COLUMN_STATS";
HoodieIndexer indexer = new HoodieIndexer(jsc, config);
HoodieIndexCommitMetadata commitMetadata = HoodieIndexCommitMetadata.newBuilder()
.setIndexPartitionInfos(Arrays.asList(new HoodieIndexPartitionInfo(
1,
MetadataPartitionType.COLUMN_STATS.getPartitionPath(),
"0000")))
.build();
assertFalse(indexer.isIndexBuiltForAllRequestedTypes(commitMetadata.getIndexPartitionInfos()));
config.indexTypes = "COLUMN_STATS";
indexer = new HoodieIndexer(jsc, config);
assertTrue(indexer.isIndexBuiltForAllRequestedTypes(commitMetadata.getIndexPartitionInfos()));
}
@Override
public HoodieEngineContext context() {
return context;
}
@Override
public SparkSession spark() {
return spark;
}
@Override
public SQLContext sqlContext() {
return sqlContext;
}
@Override
public JavaSparkContext jsc() {
return jsc;
}
}

View File

@@ -129,6 +129,7 @@ public class HoodieDeltaStreamerTestBase extends UtilitiesTestBase {
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/uber_config.properties", dfs, dfsBasePath + "/config/uber_config.properties");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/short_trip_uber_config.properties", dfs, dfsBasePath + "/config/short_trip_uber_config.properties");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/clusteringjob.properties", dfs, dfsBasePath + "/clusteringjob.properties");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/indexer.properties", dfs, dfsBasePath + "/indexer.properties");
writeCommonPropsToFile(dfs, dfsBasePath);

View File

@@ -57,6 +57,7 @@ import org.apache.hudi.hive.HoodieHiveClient;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.utilities.DummySchemaProvider;
import org.apache.hudi.utilities.HoodieClusteringJob;
import org.apache.hudi.utilities.HoodieIndexer;
import org.apache.hudi.utilities.deltastreamer.DeltaSync;
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
@@ -129,6 +130,9 @@ import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
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;
import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY;
import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE;
import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME;
@@ -397,6 +401,22 @@ public class TestHoodieDeltaStreamer extends HoodieDeltaStreamerTestBase {
assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected);
}
static void assertPendingIndexCommit(String tablePath, FileSystem fs) {
HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).setLoadActiveTimelineOnLoad(true).build();
HoodieTimeline timeline = meta.getActiveTimeline().getAllCommitsTimeline().filterPendingIndexTimeline();
LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList()));
int numIndexCommits = (int) timeline.getInstants().count();
assertEquals(1, numIndexCommits, "Got=" + numIndexCommits + ", exp=1");
}
static void assertCompletedIndexCommit(String tablePath, FileSystem fs) {
HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).setLoadActiveTimelineOnLoad(true).build();
HoodieTimeline timeline = meta.getActiveTimeline().getAllCommitsTimeline().filterCompletedIndexTimeline();
LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList()));
int numIndexCommits = (int) timeline.getInstants().count();
assertEquals(1, numIndexCommits, "Got=" + numIndexCommits + ", exp=1");
}
static void assertNoReplaceCommits(String tablePath, FileSystem fs) {
HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).setLoadActiveTimelineOnLoad(true).build();
HoodieTimeline timeline = meta.getActiveTimeline().getCompletedReplaceTimeline();
@@ -961,6 +981,53 @@ public class TestHoodieDeltaStreamer extends HoodieDeltaStreamerTestBase {
return config;
}
private HoodieIndexer.Config buildIndexerConfig(String basePath,
String tableName,
String indexInstantTime,
String runningMode,
String indexTypes) {
HoodieIndexer.Config config = new HoodieIndexer.Config();
config.basePath = basePath;
config.tableName = tableName;
config.indexInstantTime = indexInstantTime;
config.propsFilePath = dfsBasePath + "/indexer.properties";
config.runningMode = runningMode;
config.indexTypes = indexTypes;
return config;
}
@Test
public void testHoodieIndexer() throws Exception {
String tableBasePath = dfsBasePath + "/asyncindexer";
HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 1000, "false");
deltaStreamerTestRunner(ds, (r) -> {
TestHelpers.assertAtLeastNCommits(2, tableBasePath, dfs);
Option<String> scheduleIndexInstantTime = Option.empty();
try {
HoodieIndexer scheduleIndexingJob = new HoodieIndexer(jsc,
buildIndexerConfig(tableBasePath, ds.getConfig().targetTableName, null, SCHEDULE, "COLUMN_STATS"));
scheduleIndexInstantTime = scheduleIndexingJob.doSchedule();
} catch (Exception e) {
LOG.info("Schedule indexing failed", e);
return false;
}
if (scheduleIndexInstantTime.isPresent()) {
TestHelpers.assertPendingIndexCommit(tableBasePath, dfs);
LOG.info("Schedule indexing success, now build index with instant time " + scheduleIndexInstantTime.get());
HoodieIndexer runIndexingJob = new HoodieIndexer(jsc,
buildIndexerConfig(tableBasePath, ds.getConfig().targetTableName, scheduleIndexInstantTime.get(), EXECUTE, "COLUMN_STATS"));
runIndexingJob.start(0);
LOG.info("Metadata indexing success");
TestHelpers.assertCompletedIndexCommit(tableBasePath, dfs);
} else {
LOG.warn("Metadata indexing failed");
}
return true;
});
}
@Disabled("HUDI-3710 to fix the ConcurrentModificationException")
@ParameterizedTest
@ValueSource(booleans = {true, false})
@@ -1131,28 +1198,28 @@ public class TestHoodieDeltaStreamer extends HoodieDeltaStreamerTestBase {
LOG.info("Cluster success");
} else {
LOG.warn("Import failed");
if (!runningMode.toLowerCase().equals(HoodieClusteringJob.EXECUTE)) {
if (!runningMode.toLowerCase().equals(EXECUTE)) {
return false;
}
}
} catch (Exception e) {
LOG.warn("ScheduleAndExecute clustering failed", e);
exception = e;
if (!runningMode.equalsIgnoreCase(HoodieClusteringJob.EXECUTE)) {
if (!runningMode.equalsIgnoreCase(EXECUTE)) {
return false;
}
}
switch (runningMode.toLowerCase()) {
case HoodieClusteringJob.SCHEDULE_AND_EXECUTE: {
case SCHEDULE_AND_EXECUTE: {
TestHelpers.assertAtLeastNReplaceCommits(2, tableBasePath, dfs);
return true;
}
case HoodieClusteringJob.SCHEDULE: {
case SCHEDULE: {
TestHelpers.assertAtLeastNReplaceRequests(2, tableBasePath, dfs);
TestHelpers.assertNoReplaceCommits(tableBasePath, dfs);
return true;
}
case HoodieClusteringJob.EXECUTE: {
case EXECUTE: {
TestHelpers.assertNoReplaceCommits(tableBasePath, dfs);
return true;
}

View File

@@ -0,0 +1,25 @@
#
# 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.
#
hoodie.metadata.enable=true
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.InProcessLockProvider