-
Notifications
You must be signed in to change notification settings - Fork 744
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
* Add generic topic validation support
* Add the first validator TopicNameValidator into the validator chain, as a refactor of existing codes Add generic topic validation support Add OrcSchemaConversionValidator
- Loading branch information
Tao Qin
committed
Oct 11, 2023
1 parent
9b254b6
commit 03dc7bc
Showing
9 changed files
with
534 additions
and
6 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
61 changes: 61 additions & 0 deletions
61
...apache/gobblin/source/extractor/extract/kafka/validator/OrcSchemaConversionValidator.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,61 @@ | ||
/* | ||
* 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.gobblin.source.extractor.extract.kafka.validator; | ||
|
||
import java.io.IOException; | ||
import org.apache.avro.Schema; | ||
import org.apache.gobblin.configuration.SourceState; | ||
import org.apache.gobblin.kafka.schemareg.KafkaSchemaRegistry; | ||
import org.apache.gobblin.kafka.schemareg.KafkaSchemaRegistryFactory; | ||
import org.apache.gobblin.kafka.schemareg.SchemaRegistryException; | ||
import org.apache.gobblin.source.extractor.extract.kafka.KafkaTopic; | ||
import org.apache.gobblin.util.orc.AvroOrcSchemaConverter; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
|
||
public class OrcSchemaConversionValidator extends TopicValidatorBase { | ||
private static final Logger LOGGER = LoggerFactory.getLogger(OrcSchemaConversionValidator.class); | ||
|
||
public static final String MAX_RECURSIVE_DEPTH_KEY = "gobblin.kafka.topicValidators.orcSchemaConversionValidator.maxRecursiveDepth"; | ||
public static final int DEFAULT_MAX_RECURSIVE_DEPTH = 200; | ||
|
||
private final KafkaSchemaRegistry schemaRegistry; | ||
|
||
public OrcSchemaConversionValidator(SourceState sourceState) { | ||
super(sourceState); | ||
this.schemaRegistry = KafkaSchemaRegistryFactory.getSchemaRegistry(sourceState.getProperties()); | ||
} | ||
|
||
@Override | ||
public boolean validate(KafkaTopic topic) { | ||
LOGGER.debug("Validating ORC schema conversion for topic {}", topic.getName()); | ||
try { | ||
Schema schema = (Schema) this.schemaRegistry.getLatestSchema(topic.getName()); | ||
// Try converting the avro schema to orc schema to check if any errors. | ||
int maxRecursiveDepth = this.sourceState.getPropAsInt(MAX_RECURSIVE_DEPTH_KEY, DEFAULT_MAX_RECURSIVE_DEPTH); | ||
AvroOrcSchemaConverter.tryGetOrcSchema(schema, 0, maxRecursiveDepth); | ||
} catch (StackOverflowError e) { | ||
LOGGER.warn("Failed to covert latest schema to ORC schema for topic: {}", topic.getName()); | ||
return false; | ||
} catch (IOException | SchemaRegistryException e) { | ||
LOGGER.warn("Failed to get latest schema for topic: {}, validation is skipped, exception: ", topic.getName(), e); | ||
return true; | ||
} | ||
return true; | ||
} | ||
} |
43 changes: 43 additions & 0 deletions
43
.../java/org/apache/gobblin/source/extractor/extract/kafka/validator/TopicNameValidator.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,43 @@ | ||
/* | ||
* 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.gobblin.source.extractor.extract.kafka.validator; | ||
|
||
import org.apache.gobblin.configuration.SourceState; | ||
import org.apache.gobblin.source.extractor.extract.kafka.KafkaTopic; | ||
|
||
|
||
/** | ||
* A topic validator that validates the topic name | ||
*/ | ||
public class TopicNameValidator extends TopicValidatorBase { | ||
private static final String DOT = "."; | ||
|
||
public TopicNameValidator(SourceState sourceState) { | ||
super(sourceState); | ||
} | ||
|
||
/** | ||
* Check if a topic name is valid, current rules are: | ||
* 1. must not contain "." | ||
* @param topic the topic to be validated | ||
* @return true if the topic name is valid (aka. doesn't contain ".") | ||
*/ | ||
@Override | ||
public boolean validate(KafkaTopic topic) { | ||
return !topic.getName().contains(DOT); | ||
} | ||
} |
39 changes: 39 additions & 0 deletions
39
.../java/org/apache/gobblin/source/extractor/extract/kafka/validator/TopicValidatorBase.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,39 @@ | ||
/* | ||
* 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.gobblin.source.extractor.extract.kafka.validator; | ||
|
||
import org.apache.gobblin.configuration.SourceState; | ||
import org.apache.gobblin.source.extractor.extract.kafka.KafkaTopic; | ||
|
||
/** | ||
* The base class of a topic validator | ||
*/ | ||
public abstract class TopicValidatorBase { | ||
protected SourceState sourceState; | ||
|
||
public TopicValidatorBase(SourceState sourceState) { | ||
this.sourceState = sourceState; | ||
} | ||
|
||
/** | ||
* Validate a KafkaTopic. | ||
* This method must be thread-safe. | ||
* @param topic The topic to validate | ||
* @return Whether the topic passes the validation | ||
*/ | ||
public abstract boolean validate(KafkaTopic topic); | ||
} |
83 changes: 83 additions & 0 deletions
83
...ain/java/org/apache/gobblin/source/extractor/extract/kafka/validator/TopicValidators.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,83 @@ | ||
/* | ||
* 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.gobblin.source.extractor.extract.kafka.validator; | ||
|
||
import com.google.common.base.Strings; | ||
import java.util.ArrayList; | ||
import java.util.Arrays; | ||
import java.util.List; | ||
import java.util.stream.Collectors; | ||
import lombok.extern.slf4j.Slf4j; | ||
import org.apache.gobblin.configuration.SourceState; | ||
import org.apache.gobblin.source.extractor.extract.kafka.KafkaTopic; | ||
import org.apache.gobblin.util.reflection.GobblinConstructorUtils; | ||
|
||
/** | ||
* The TopicValidators contains a list of {@link TopicValidatorBase} that validate topics. | ||
* gobblin.kafka.topicValidators=validator1_class_name,validator2_class_name... | ||
*/ | ||
@Slf4j | ||
public class TopicValidators { | ||
public static final String VALIDATOR_CLASSES_KEY = "gobblin.kafka.topicValidators"; | ||
|
||
public static final String VALIDATOR_CLASS_DELIMITER = ","; | ||
|
||
private final List<TopicValidatorBase> validators = new ArrayList<>(); | ||
|
||
public TopicValidators(SourceState state) { | ||
String validatorClasses = state.getProp(VALIDATOR_CLASSES_KEY); | ||
if (Strings.isNullOrEmpty(validatorClasses)) { | ||
return; | ||
} | ||
|
||
String[] validatorClassNames = validatorClasses.split(VALIDATOR_CLASS_DELIMITER); | ||
Arrays.stream(validatorClassNames).forEach(validator -> { | ||
try { | ||
this.validators.add(GobblinConstructorUtils.invokeConstructor(TopicValidatorBase.class, validator, state)); | ||
} catch (Exception e) { | ||
log.error("Failed to create topic validator: {}, due to {}", validator, e); | ||
} | ||
}); | ||
} | ||
|
||
/** | ||
* Validate topics with all the internal validators. | ||
* Note: the validations for every topic run in parallel. | ||
* @param topics the topics to be validated | ||
* @return the topics that pass all the validators | ||
*/ | ||
public List<KafkaTopic> validate(List<KafkaTopic> topics) { | ||
// Validate the topics in parallel | ||
return topics.parallelStream() | ||
.filter(this::validate) | ||
.collect(Collectors.toList()); | ||
} | ||
|
||
/** | ||
* Validates a single topic with all the internal validators | ||
*/ | ||
private boolean validate(KafkaTopic topic) { | ||
log.debug("Validating topic {} in thread: {}", topic, Thread.currentThread().getName()); | ||
for (TopicValidatorBase validator : this.validators) { | ||
if (!validator.validate(topic)) { | ||
log.info("Skip KafkaTopic: {}, by validator: {}", topic, validator.getClass().getName()); | ||
return false; | ||
} | ||
} | ||
return true; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.