1
0

[HUDI-2072] Add pre-commit validator framework (#3153)

* [HUDI-2072] Add pre-commit validator framework

* trigger Travis rebuild
This commit is contained in:
satishkotha
2021-08-03 12:07:45 -07:00
committed by GitHub
parent bec23bda50
commit 826a04d142
14 changed files with 1130 additions and 32 deletions

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.config;
import org.apache.hudi.common.config.ConfigProperty;
import org.apache.hudi.common.config.HoodieConfig;
import javax.annotation.concurrent.Immutable;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.util.Properties;
/**
* Storage related config.
*/
@Immutable
public class HoodiePreCommitValidatorConfig extends HoodieConfig {
public static final ConfigProperty<String> PRE_COMMIT_VALIDATORS = ConfigProperty
.key("hoodie.precommit.validators")
.defaultValue("")
.withDocumentation("Comma separated list of class names that can be invoked to validate commit");
public static final String VALIDATOR_TABLE_VARIABLE = "<TABLE_NAME>";
public static final ConfigProperty<String> PRE_COMMIT_VALIDATORS_EQUALITY_SQL_QUERIES = ConfigProperty
.key("hoodie.precommit.validators.equality.sql.queries")
.defaultValue("")
.withDocumentation("Spark SQL queries to run on table before committing new data to validate state before and after commit."
+ " Multiple queries separated by ';' delimiter are supported."
+ " Example: \"select count(*) from \\<TABLE_NAME\\>"
+ " Note \\<TABLE_NAME\\> is replaced by table state before and after commit.");
public static final ConfigProperty<String> PRE_COMMIT_VALIDATORS_SINGLE_VALUE_SQL_QUERIES = ConfigProperty
.key("hoodie.precommit.validators.single.value.sql.queries")
.defaultValue("")
.withDocumentation("Spark SQL queries to run on table before committing new data to validate state after commit."
+ "Multiple queries separated by ';' delimiter are supported."
+ "Expected result is included as part of query separated by '#'. Example query: 'query1#result1:query2#result2'"
+ "Note \\<TABLE_NAME\\> variable is expected to be present in query.");
/**
* Spark SQL queries to run on table before committing new data to validate state before and after commit.
* Multiple queries separated by ';' delimiter are supported.
* Example query: 'select count(*) from \<TABLE_NAME\> where col=null'
* Note \<TABLE_NAME\> variable is expected to be present in query.
*/
public static final ConfigProperty<String> PRE_COMMIT_VALIDATORS_INEQUALITY_SQL_QUERIES = ConfigProperty
.key("hoodie.precommit.validators.inequality.sql.queries")
.defaultValue("")
.withDocumentation("Spark SQL queries to run on table before committing new data to validate state before and after commit."
+ "Multiple queries separated by ';' delimiter are supported."
+ "Example query: 'select count(*) from \\<TABLE_NAME\\> where col=null'"
+ "Note \\<TABLE_NAME\\> variable is expected to be present in query.");
private HoodiePreCommitValidatorConfig() {
super();
}
public static HoodiePreCommitValidatorConfig.Builder newBuilder() {
return new Builder();
}
public static class Builder {
private final HoodiePreCommitValidatorConfig preCommitValidatorConfig = new HoodiePreCommitValidatorConfig();
public Builder fromFile(File propertiesFile) throws IOException {
try (FileReader reader = new FileReader(propertiesFile)) {
this.preCommitValidatorConfig.getProps().load(reader);
return this;
}
}
public Builder fromProperties(Properties props) {
this.preCommitValidatorConfig.getProps().putAll(props);
return this;
}
public Builder withPreCommitValidator(String preCommitValidators) {
preCommitValidatorConfig.setValue(PRE_COMMIT_VALIDATORS, preCommitValidators);
return this;
}
public Builder withPrecommitValidatorEqualitySqlQueries(String preCommitValidators) {
preCommitValidatorConfig.setValue(PRE_COMMIT_VALIDATORS_EQUALITY_SQL_QUERIES, preCommitValidators);
return this;
}
public Builder withPrecommitValidatorSingleResultSqlQueries(String preCommitValidators) {
preCommitValidatorConfig.setValue(PRE_COMMIT_VALIDATORS_SINGLE_VALUE_SQL_QUERIES, preCommitValidators);
return this;
}
public Builder withPrecommitValidatorInequalitySqlQueries(String preCommitValidators) {
preCommitValidatorConfig.setValue(PRE_COMMIT_VALIDATORS_INEQUALITY_SQL_QUERIES, preCommitValidators);
return this;
}
public HoodiePreCommitValidatorConfig build() {
preCommitValidatorConfig.setDefaults(HoodiePreCommitValidatorConfig.class.getName());
return preCommitValidatorConfig;
}
}
}

View File

@@ -1280,6 +1280,22 @@ public class HoodieWriteConfig extends HoodieConfig {
public String getWriteMetaKeyPrefixes() {
return getString(WRITE_META_KEY_PREFIXES_PROP);
}
public String getPreCommitValidators() {
return getString(HoodiePreCommitValidatorConfig.PRE_COMMIT_VALIDATORS);
}
public String getPreCommitValidatorEqualitySqlQueries() {
return getString(HoodiePreCommitValidatorConfig.PRE_COMMIT_VALIDATORS_EQUALITY_SQL_QUERIES);
}
public String getPreCommitValidatorSingleResultSqlQueries() {
return getString(HoodiePreCommitValidatorConfig.PRE_COMMIT_VALIDATORS_SINGLE_VALUE_SQL_QUERIES);
}
public String getPreCommitValidatorInequalitySqlQueries() {
return getString(HoodiePreCommitValidatorConfig.PRE_COMMIT_VALIDATORS_INEQUALITY_SQL_QUERIES);
}
public boolean allowEmptyCommit() {
return getBooleanOrDefault(ALLOW_EMPTY_COMMIT);
@@ -1302,6 +1318,7 @@ public class HoodieWriteConfig extends HoodieConfig {
private boolean isPayloadConfigSet = false;
private boolean isMetadataConfigSet = false;
private boolean isLockConfigSet = false;
private boolean isPreCommitValidationConfigSet = false;
public Builder withEngineType(EngineType engineType) {
this.engineType = engineType;
@@ -1451,6 +1468,12 @@ public class HoodieWriteConfig extends HoodieConfig {
return this;
}
public Builder withPreCommitValidatorConfig(HoodiePreCommitValidatorConfig validatorConfig) {
writeConfig.getProps().putAll(validatorConfig.getProps());
isPreCommitValidationConfigSet = true;
return this;
}
public Builder withMetricsConfig(HoodieMetricsConfig metricsConfig) {
writeConfig.getProps().putAll(metricsConfig.getProps());
isMetricsConfigSet = true;
@@ -1620,7 +1643,8 @@ public class HoodieWriteConfig extends HoodieConfig {
HoodieMetadataConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
writeConfig.setDefaultOnCondition(!isLockConfigSet,
HoodieLockConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
writeConfig.setDefaultOnCondition(!isPreCommitValidationConfigSet,
HoodiePreCommitValidatorConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
writeConfig.setDefaultValue(TIMELINE_LAYOUT_VERSION, String.valueOf(TimelineLayoutVersion.CURR_VERSION));
}

View File

@@ -33,6 +33,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
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.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCommitException;
@@ -123,7 +124,20 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload, I,
return table.getMetaClient().getCommitActionType();
}
/**
* Check if any validators are configured and run those validations. If any of the validations fail, throws HoodieValidationException.
*/
protected void runPrecommitValidators(HoodieWriteMetadata<O> writeMetadata) {
if (StringUtils.isNullOrEmpty(config.getPreCommitValidators())) {
return;
}
throw new HoodieIOException("Precommit validation not implemented for all engines yet");
}
protected void commitOnAutoCommit(HoodieWriteMetadata result) {
// validate commit action before committing result
runPrecommitValidators(result);
if (config.shouldAutoCommit()) {
LOG.info("Auto commit enabled: Committing " + instantTime);
autoCommit(extraMetadata, result);