[
https://issues.apache.org/jira/browse/GOBBLIN-1927?focusedWorklogId=885162&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-885162
]
ASF GitHub Bot logged work on GOBBLIN-1927:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 14/Oct/23 05:56
Start Date: 14/Oct/23 05:56
Worklog Time Spent: 10m
Work Description: wsarecv commented on code in PR #3793:
URL: https://github.com/apache/gobblin/pull/3793#discussion_r1359177739
##########
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:
Ideally, the data from a bad topic should not come to the incremental
compaction stage, because:
1. With static validators, `KafkaSource` can skip the bad topics during the
work unit creation.
2. With dynamic validators, bad topics are detected on the fly and then
FastIngest can be restarted to skip those bad topics.
However, the incremental compaction may potentially use a different set of
validators, so it still make sense to provide the validation support beyond
FastIngest. So let's go for it.
To set the context class loader correctly, in the new iteration the
`ExecutorsUtils` is enhanced to help create thread pools where the running
tasks can have the same access control and class loader settings as the thread
that submits the tasks. Please help take a look.
Issue Time Tracking
-------------------
Worklog Id: (was: 885162)
Time Spent: 1.5h (was: 1h 20m)
> Add topic validation support during work unit creation
> ------------------------------------------------------
>
> Key: GOBBLIN-1927
> URL: https://issues.apache.org/jira/browse/GOBBLIN-1927
> Project: Apache Gobblin
> Issue Type: Bug
> Reporter: Tao Qin
> Priority: Major
> Time Spent: 1.5h
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)