-
Notifications
You must be signed in to change notification settings - Fork 22
Changed exception calls to use FailureCollector #67
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,7 +29,9 @@ | |
| import io.cdap.cdap.api.dataset.lib.KeyValue; | ||
| import io.cdap.cdap.common.io.ByteBuffers; | ||
| import io.cdap.cdap.etl.api.Emitter; | ||
| import io.cdap.cdap.etl.api.FailureCollector; | ||
| import io.cdap.cdap.etl.api.PipelineConfigurer; | ||
| import io.cdap.cdap.etl.api.StageConfigurer; | ||
| import io.cdap.cdap.etl.api.batch.BatchRuntimeContext; | ||
| import io.cdap.cdap.etl.api.batch.BatchSource; | ||
| import io.cdap.cdap.etl.api.batch.BatchSourceContext; | ||
|
|
@@ -53,6 +55,7 @@ | |
| import java.util.HashMap; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Set; | ||
| import java.util.stream.Collectors; | ||
| import javax.annotation.Nullable; | ||
|
|
||
|
|
@@ -122,9 +125,9 @@ public Map<String, String> getKafkaProperties() { | |
| return conf; | ||
| } | ||
|
|
||
| public void validate() { | ||
| super.validate(); | ||
| KafkaHelpers.validateKerberosSetting(principal, keytabLocation); | ||
| public void validate(FailureCollector collector) { | ||
| super.validate(collector); | ||
| KafkaHelpers.validateKerberosSetting(principal, keytabLocation, collector); | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -134,16 +137,23 @@ public KafkaBatchSource(Kafka10BatchConfig config) { | |
|
|
||
| @Override | ||
| public void configurePipeline(PipelineConfigurer pipelineConfigurer) { | ||
| config.validate(); | ||
| pipelineConfigurer.getStageConfigurer().setOutputSchema(config.getSchema()); | ||
| StageConfigurer stageConfigurer = pipelineConfigurer.getStageConfigurer(); | ||
| FailureCollector failureCollector = stageConfigurer.getFailureCollector(); | ||
| config.validate(failureCollector); | ||
| stageConfigurer.setOutputSchema(config.getSchema(failureCollector)); | ||
| failureCollector.getOrThrowException(); | ||
| } | ||
|
|
||
| @Override | ||
| public void prepareRun(BatchSourceContext context) throws Exception { | ||
| Job job = JobUtils.createInstance(); | ||
| Configuration conf = job.getConfiguration(); | ||
|
|
||
| KafkaPartitionOffsets partitionOffsets = config.getInitialPartitionOffsets(); | ||
| FailureCollector failureCollector = context.getFailureCollector(); | ||
| KafkaPartitionOffsets partitionOffsets = config.getInitialPartitionOffsets(failureCollector); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. as mentioned before, do all the validation here before anything else happens. This should be:
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ok, but I previously moved FailureCollector later in the function, since I didn't think the if block or the saveKafkaRequests call needed to be done after getOrThrowException. |
||
| Schema schema = config.getSchema(failureCollector); | ||
| Set<Integer> partitions = config.getPartitions(failureCollector); | ||
| failureCollector.getOrThrowException(); | ||
|
|
||
| // If the offset directory is provided, try to load the file | ||
| if (!context.isPreviewEnabled() && config.getOffsetDir() != null) { | ||
|
|
@@ -167,10 +177,11 @@ public void prepareRun(BatchSourceContext context) throws Exception { | |
| KafkaHelpers.setupKerberosLogin(kafkaConf, config.getPrincipal(), config.getKeytabLocation()); | ||
| kafkaConf.putAll(config.getKafkaProperties()); | ||
| kafkaRequests = KafkaInputFormat.saveKafkaRequests(conf, config.getTopic(), kafkaConf, | ||
| config.getPartitions(), config.getMaxNumberRecords(), | ||
| partitions, | ||
| config.getMaxNumberRecords(), | ||
| partitionOffsets); | ||
| LineageRecorder lineageRecorder = new LineageRecorder(context, config.referenceName); | ||
| Schema schema = config.getSchema(); | ||
|
|
||
| if (schema != null) { | ||
| lineageRecorder.createExternalDataset(schema); | ||
| if (schema.getFields() != null && !schema.getFields().isEmpty()) { | ||
|
|
@@ -203,8 +214,11 @@ public void onRunFinish(boolean succeeded, BatchSourceContext context) { | |
| public void initialize(BatchRuntimeContext context) throws Exception { | ||
| super.initialize(context); | ||
|
|
||
| schema = config.getSchema(); | ||
| Schema messageSchema = config.getMessageSchema(); | ||
| schema = config.getSchema(context.getFailureCollector()); | ||
| Schema messageSchema = config.getMessageSchema(context.getFailureCollector()); | ||
| if (schema == null || messageSchema == null) { | ||
| return; | ||
| } | ||
| for (Schema.Field field : schema.getFields()) { | ||
| String name = field.getName(); | ||
| if (!name.equals(config.getKeyField()) && !name.equals(config.getPartitionField()) && | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,13 +27,14 @@ | |
| import io.cdap.cdap.etl.api.Alert; | ||
| import io.cdap.cdap.etl.api.AlertPublisher; | ||
| import io.cdap.cdap.etl.api.AlertPublisherContext; | ||
| import io.cdap.cdap.etl.api.FailureCollector; | ||
| import io.cdap.cdap.etl.api.PipelineConfigurer; | ||
| import io.cdap.plugin.common.KeyValueListParser; | ||
| import kafka.common.InvalidTopicException; | ||
| import kafka.common.Topic; | ||
| import org.apache.kafka.clients.producer.KafkaProducer; | ||
| import org.apache.kafka.clients.producer.ProducerConfig; | ||
| import org.apache.kafka.clients.producer.ProducerRecord; | ||
| import org.apache.kafka.common.errors.InvalidTopicException; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
|
|
@@ -61,13 +62,13 @@ public KafkaAlertPublisher(Config config) { | |
|
|
||
| @Override | ||
| public void configurePipeline(PipelineConfigurer pipelineConfigurer) throws IllegalArgumentException { | ||
| config.validate(); | ||
| config.validate(pipelineConfigurer.getStageConfigurer().getFailureCollector()); | ||
| } | ||
|
|
||
| @Override | ||
| public void initialize(AlertPublisherContext context) throws Exception { | ||
| super.initialize(context); | ||
| config.validate(); | ||
| config.validate(context.getFailureCollector()); | ||
| Properties props = new Properties(); | ||
| // Add client id property with stage name as value. | ||
| props.put(ProducerConfig.CLIENT_ID_CONFIG, context.getStageName()); | ||
|
|
@@ -111,18 +112,22 @@ public void destroy() { | |
| */ | ||
| public static class Config extends PluginConfig { | ||
|
|
||
| @Name("brokers") | ||
| private static final String TOPIC = "topic"; | ||
| private static final String BROKERS = "brokers"; | ||
| private static final String PRODUCER_PROPERTIES = "producerProperties"; | ||
|
|
||
| @Name(BROKERS) | ||
| @Description("Specifies the connection string where Producer can find one or more brokers to " + | ||
| "determine the leader for each topic.") | ||
| @Macro | ||
| private String brokers; | ||
|
|
||
| @Name("topic") | ||
| @Name(TOPIC) | ||
| @Description("Topic to which message needs to be published. The topic should already exist on kafka.") | ||
| @Macro | ||
| private String topic; | ||
|
|
||
| @Name("producerProperties") | ||
| @Name(PRODUCER_PROPERTIES) | ||
| @Nullable | ||
| @Description("Additional kafka producer properties to set.") | ||
| private String producerProperties; | ||
|
|
@@ -146,7 +151,7 @@ private Map<String, String> getProducerProperties() { | |
| return producerProps; | ||
| } | ||
|
|
||
| private void validate() { | ||
| private void validate(FailureCollector collector) { | ||
| // If the topic or brokers are macros they would not be available at config time. So do not perform | ||
| // validations yet. | ||
| if (Strings.isNullOrEmpty(topic) || Strings.isNullOrEmpty(brokers)) { | ||
|
|
@@ -156,8 +161,9 @@ private void validate() { | |
| try { | ||
| Topic.validate(topic); | ||
| } catch (InvalidTopicException e) { | ||
| throw new IllegalArgumentException(String.format("Topic name %s is not a valid kafka topic. Please provide " + | ||
| "valid kafka topic name. %s", topic, e.getMessage())); | ||
| collector.addFailure(String.format("Topic name %s is not a valid kafka topic. Please provide valid kafka" + | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. missing space between 'kafka' and 'topic' on the next line |
||
| "topic name. %s", topic, e.getMessage()), null) | ||
| .withConfigProperty(TOPIC); | ||
| } | ||
| } | ||
| } | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.