ZihanLi58 commented on code in PR #3794: URL: https://github.com/apache/gobblin/pull/3794#discussion_r1366284783
########## gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/validator/OrcSchemaConversionValidator.java: ########## @@ -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.State; +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(State sourceState) { + super(sourceState); + this.schemaRegistry = KafkaSchemaRegistryFactory.getSchemaRegistry(sourceState.getProperties()); + } + + @Override + public boolean validate(KafkaTopic topic) throws Exception { + 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.state.getPropAsInt(MAX_RECURSIVE_DEPTH_KEY, DEFAULT_MAX_RECURSIVE_DEPTH); + AvroOrcSchemaConverter.tryGetOrcSchema(schema, 0, maxRecursiveDepth); + } catch (StackOverflowError e) { Review Comment: If you plan to catch the StackOverflowError anyway, why do we even need to introduce the depth here? ########## gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/validator/OrcSchemaConversionValidator.java: ########## @@ -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.State; +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(State sourceState) { + super(sourceState); + this.schemaRegistry = KafkaSchemaRegistryFactory.getSchemaRegistry(sourceState.getProperties()); + } + + @Override + public boolean validate(KafkaTopic topic) throws Exception { + 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.state.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; Review Comment: I believe we should emit some event to indicate this kind of error to avoid silent failure here. Same for other validator. ########## gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/validator/OrcSchemaConversionValidator.java: ########## @@ -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.State; +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(State sourceState) { + super(sourceState); + this.schemaRegistry = KafkaSchemaRegistryFactory.getSchemaRegistry(sourceState.getProperties()); + } + + @Override + public boolean validate(KafkaTopic topic) throws Exception { + 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.state.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()); Review Comment: If it's possible to detect which filed is recursive and include in this error message? so it will be easier for us to find the problematic filed and re-enable the ingestion when necessary ########## gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/validator/OrcSchemaConversionValidator.java: ########## @@ -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.State; +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(State sourceState) { + super(sourceState); + this.schemaRegistry = KafkaSchemaRegistryFactory.getSchemaRegistry(sourceState.getProperties()); + } + + @Override + public boolean validate(KafkaTopic topic) throws Exception { + LOGGER.debug("Validating ORC schema conversion for topic {}", topic.getName()); + try { + Schema schema = (Schema) this.schemaRegistry.getLatestSchema(topic.getName()); Review Comment: The schema from schema registry is not the final schema, especially we do have converter to change the schema there and some times remove the recursive schema to enable ingestion, so I believe we should use converted schema to do validation here -- 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]
