wsarecv commented on code in PR #3793: URL: https://github.com/apache/gobblin/pull/3793#discussion_r1355794417
########## gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/validator/TopicValidators.java: ########## @@ -0,0 +1,85 @@ +/* + * 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 org.apache.gobblin.configuration.SourceState; +import org.apache.gobblin.source.extractor.extract.kafka.KafkaTopic; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The TopicValidators contains a list of {@link TopicValidatorBase} that validate topics. + * To enable it, add below settings in the config: + * gobblin.kafka.topicValidators=validator1_class_name;validator2_class_name... + */ +public class TopicValidators { + public static final String VALIDATOR_CLASSES_KEY = "gobblin.kafka.topicValidators"; + + public static final String VALIDATOR_CLASS_DELIMITER = ";"; + private static final Logger LOGGER = LoggerFactory.getLogger(TopicValidators.class); + + 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( + (TopicValidatorBase) Class.forName(validator).getConstructor(SourceState.class).newInstance(state)); + } catch (Exception e) { + LOGGER.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() Review Comment: Thanks for the reference of that PR. Yes, the parallelism is on purpose. Some validator could be slow, validating the topics sequentially would significantly slow down the startup. For example, there will be [another PR to add an OrcSchemaConversionValidator](https://github.com/apache/gobblin/pull/3794/files#diff-c480e84c1e7ab69ed31e521c5615de4b270ce6da72333a29ec779b47f04e3d0aR45-R48) that queries the remote schema registry for every topic. If the `parallelStream` is not suggested, we would have to do it in a separate thread pool while trying to avoid the context class loader issue. Any comment on this approach? ########## gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/validator/TopicValidators.java: ########## @@ -0,0 +1,85 @@ +/* + * 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 org.apache.gobblin.configuration.SourceState; +import org.apache.gobblin.source.extractor.extract.kafka.KafkaTopic; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The TopicValidators contains a list of {@link TopicValidatorBase} that validate topics. + * To enable it, add below settings in the config: + * gobblin.kafka.topicValidators=validator1_class_name;validator2_class_name... + */ +public class TopicValidators { + public static final String VALIDATOR_CLASSES_KEY = "gobblin.kafka.topicValidators"; + + public static final String VALIDATOR_CLASS_DELIMITER = ";"; Review Comment: Fixed. ########## gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/validator/TopicValidators.java: ########## @@ -0,0 +1,85 @@ +/* + * 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 org.apache.gobblin.configuration.SourceState; +import org.apache.gobblin.source.extractor.extract.kafka.KafkaTopic; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The TopicValidators contains a list of {@link TopicValidatorBase} that validate topics. + * To enable it, add below settings in the config: + * gobblin.kafka.topicValidators=validator1_class_name;validator2_class_name... + */ +public class TopicValidators { + public static final String VALIDATOR_CLASSES_KEY = "gobblin.kafka.topicValidators"; + + public static final String VALIDATOR_CLASS_DELIMITER = ";"; + private static final Logger LOGGER = LoggerFactory.getLogger(TopicValidators.class); Review Comment: Fixed. -- 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. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
