diff --git a/build.gradle b/build.gradle index d1a3aa8d48..006e641354 100644 --- a/build.gradle +++ b/build.gradle @@ -177,6 +177,8 @@ subprojects { } } else if (details.requested.group == 'log4j' && details.requested.name == 'log4j') { details.useTarget group: 'org.apache.logging.log4j', name: 'log4j-1.2-api', version: '2.17.1' + } else if (details.requested.group == 'org.xerial.snappy' && details.requested.name == 'snappy-java') { + details.useTarget group: 'org.xerial.snappy', name: 'snappy-java', version: '1.1.10.1' } } } diff --git a/data-prepper-plugins/common/build.gradle b/data-prepper-plugins/common/build.gradle index 24a4fca124..87b15dcd31 100644 --- a/data-prepper-plugins/common/build.gradle +++ b/data-prepper-plugins/common/build.gradle @@ -21,7 +21,7 @@ dependencies { implementation 'io.micrometer:micrometer-core' testImplementation testLibs.junit.vintage implementation 'org.apache.parquet:parquet-common:1.12.3' - implementation 'org.xerial.snappy:snappy-java:1.1.9.1' + implementation 'org.xerial.snappy:snappy-java:1.1.10.1' testImplementation project(':data-prepper-plugins:blocking-buffer') testImplementation 'commons-io:commons-io:2.12.0' testImplementation testLibs.mockito.inline diff --git a/data-prepper-plugins/kafka-plugins/build.gradle b/data-prepper-plugins/kafka-plugins/build.gradle index 1772778dee..a20777fa88 100644 --- a/data-prepper-plugins/kafka-plugins/build.gradle +++ b/data-prepper-plugins/kafka-plugins/build.gradle @@ -9,6 +9,7 @@ plugins { dependencies { implementation project(':data-prepper-api') + implementation project(':data-prepper-plugins:buffer-common') implementation 'org.apache.kafka:kafka-clients:3.4.0' implementation 'org.apache.avro:avro:1.11.0' implementation 'com.fasterxml.jackson.core:jackson-databind' diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/JSONConsumerIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/JSONConsumerIT.java index 1fab0d7ac1..cc777b25df 100644 --- a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/JSONConsumerIT.java +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/JSONConsumerIT.java @@ -25,6 +25,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; @@ -49,7 +50,7 @@ public class JSONConsumerIT { private KafkaSourceConfig kafkaSourceConfig; private KafkaSource kafkaSource; - private Buffer> buffer; + private Buffer> buffer; @ClassRule public static final EmbeddedKafkaClusterSingleNode CLUSTER = new EmbeddedKafkaClusterSingleNode(); @@ -111,4 +112,4 @@ private void produceTestMessages() throws JsonProcessingException { throw new RuntimeException(e); } } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/PlainTextConsumerIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/PlainTextConsumerIT.java index a2f10eeba7..a5118e64c5 100644 --- a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/PlainTextConsumerIT.java +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/PlainTextConsumerIT.java @@ -20,6 +20,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; @@ -44,7 +45,7 @@ public class PlainTextConsumerIT { private KafkaSourceConfig kafkaSourceConfig; private KafkaSource kafkaSource; - private Buffer> buffer; + private Buffer> buffer; @ClassRule public static final EmbeddedKafkaClusterSingleNode CLUSTER = new EmbeddedKafkaClusterSingleNode(); @@ -101,4 +102,4 @@ private void produceTestMessages() { throw new RuntimeException(e); } } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java index 4a42649fae..b2536170cb 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java @@ -21,7 +21,7 @@ public class TopicConfig { private static final Duration SESSION_TIMEOUT = Duration.ofSeconds(45); private static final int MAX_RETRY_ATTEMPT = Integer.MAX_VALUE; private static final String AUTO_OFFSET_RESET = "earliest"; - private static final Duration THREAD_WAITING_TIME = Duration.ofSeconds(1); + static final Duration THREAD_WAITING_TIME = Duration.ofSeconds(5); private static final Duration MAX_RECORD_FETCH_TIME = Duration.ofSeconds(4); private static final Duration BUFFER_DEFAULT_TIMEOUT = Duration.ofSeconds(5); private static final Duration MAX_RETRY_DELAY = Duration.ofSeconds(1); @@ -30,7 +30,7 @@ public class TopicConfig { private static final Long FETCH_MIN_BYTES = 1L; private static final Duration RETRY_BACKOFF = Duration.ofSeconds(100); private static final Duration MAX_POLL_INTERVAL = Duration.ofSeconds(300000); - private static final Long CONSUMER_MAX_POLL_RECORDS = 500L; + private static final Integer CONSUMER_MAX_POLL_RECORDS = 500; private static final Integer NUM_OF_WORKERS = 10; private static final Duration HEART_BEAT_INTERVAL_DURATION = Duration.ofSeconds(3); @@ -54,10 +54,10 @@ public class TopicConfig { @Size(min = 1) private Duration maxRetryDelay = MAX_RETRY_DELAY; - @JsonProperty("autocommit") - private String autoCommit = AUTO_COMMIT; + @JsonProperty("auto_commit") + private Boolean autoCommit = false; - @JsonProperty("autocommit_interval") + @JsonProperty("auto_commit_interval") @Size(min = 1) private Duration autoCommitInterval = AUTOCOMMIT_INTERVAL; @@ -100,7 +100,7 @@ public class TopicConfig { private Duration maxPollInterval = MAX_POLL_INTERVAL; @JsonProperty("consumer_max_poll_records") - private Long consumerMaxPollRecords = CONSUMER_MAX_POLL_RECORDS; + private Integer consumerMaxPollRecords = CONSUMER_MAX_POLL_RECORDS; @JsonProperty("heart_beat_interval") @Size(min = 1) @@ -118,7 +118,7 @@ public void setMaxRetryAttempts(Integer maxRetryAttempts) { this.maxRetryAttempts = maxRetryAttempts; } - public String getAutoCommit() { + public Boolean getAutoCommit() { return autoCommit; } @@ -186,7 +186,7 @@ public void setFetchMaxBytes(Long fetchMaxBytes) { this.fetchMaxBytes = fetchMaxBytes; } - public void setAutoCommit(String autoCommit) { + public void setAutoCommit(Boolean autoCommit) { this.autoCommit = autoCommit; } @@ -222,11 +222,11 @@ public void setMaxPollInterval(Duration maxPollInterval) { this.maxPollInterval = maxPollInterval; } - public Long getConsumerMaxPollRecords() { + public Integer getConsumerMaxPollRecords() { return consumerMaxPollRecords; } - public void setConsumerMaxPollRecords(Long consumerMaxPollRecords) { + public void setConsumerMaxPollRecords(Integer consumerMaxPollRecords) { this.consumerMaxPollRecords = consumerMaxPollRecords; } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java index f409193d1f..e5129e030e 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java @@ -4,6 +4,9 @@ */ package org.opensearch.dataprepper.plugins.kafka.consumer; +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -11,12 +14,13 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.CommitFailedException; import org.apache.kafka.common.TopicPartition; +import org.opensearch.dataprepper.model.log.JacksonLog; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; +import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; -import org.opensearch.dataprepper.plugins.kafka.source.KafkaSourceBufferAccumulator; +import org.opensearch.dataprepper.buffer.common.BufferAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -28,84 +32,119 @@ import java.util.Collection; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; /** * * A utility class which will handle the core Kafka consumer operation. */ -public class KafkaSourceCustomConsumer implements ConsumerRebalanceListener { +public class KafkaSourceCustomConsumer implements Runnable, ConsumerRebalanceListener { private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceCustomConsumer.class); - private Map offsetsToCommit = new HashMap<>(); - private long lastReadOffset = 0L; - private volatile long lastCommitTime = System.currentTimeMillis(); - private KafkaConsumer consumer= null; - private AtomicBoolean status = new AtomicBoolean(false); - private Buffer> buffer= null; - private TopicConfig topicConfig = null; - private KafkaSourceConfig kafkaSourceConfig= null; + private static final Long COMMIT_OFFSET_INTERVAL_MS = 300000L; + private static final int DEFAULT_NUMBER_OF_RECORDS_TO_ACCUMULATE = 1; + private volatile long lastCommitTime; + private KafkaConsumer consumer= null; + private AtomicBoolean shutdownInProgress; + private final String topicName; + private final TopicConfig topicConfig; private PluginMetrics pluginMetrics= null; - private String schemaType= null; + private MessageFormat schema; + private final BufferAccumulator> bufferAccumulator; + private final Buffer> buffer; + private static final ObjectMapper objectMapper = new ObjectMapper(); + private final JsonFactory jsonFactory = new JsonFactory(); + private Map offsetsToCommit; - public KafkaSourceCustomConsumer() { - } - - private KafkaSourceBufferAccumulator kafkaSourceBufferAccumulator= null; - public KafkaSourceCustomConsumer(KafkaConsumer consumer, - AtomicBoolean status, - Buffer> buffer, - TopicConfig topicConfig, - KafkaSourceConfig kafkaSourceConfig, - String schemaType, - PluginMetrics pluginMetrics) { + public KafkaSourceCustomConsumer(final KafkaConsumer consumer, + final AtomicBoolean shutdownInProgress, + final Buffer> buffer, + final TopicConfig topicConfig, + final String schemaType, + final PluginMetrics pluginMetrics) { + this.topicName = topicConfig.getName(); + this.topicConfig = topicConfig; + this.shutdownInProgress = shutdownInProgress; this.consumer = consumer; - this.status = status; this.buffer = buffer; - this.topicConfig = topicConfig; - this.kafkaSourceConfig = kafkaSourceConfig; - this.schemaType = schemaType; + this.offsetsToCommit = new HashMap<>(); this.pluginMetrics = pluginMetrics; - kafkaSourceBufferAccumulator= new KafkaSourceBufferAccumulator(topicConfig, kafkaSourceConfig, - schemaType, pluginMetrics); + schema = MessageFormat.getByMessageFormatByName(schemaType); + Duration bufferTimeout = Duration.ofSeconds(1); + bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_NUMBER_OF_RECORDS_TO_ACCUMULATE, bufferTimeout); + lastCommitTime = System.currentTimeMillis(); } + public void consumeRecords() throws Exception { + ConsumerRecords records = + consumer.poll(topicConfig.getThreadWaitingTime().toMillis()/2); + if (!records.isEmpty() && records.count() > 0) { + Map offsets = iterateRecordPartitions(records); + offsets.forEach((partition, offset) -> + offsetsToCommit.put(partition, offset)); + } + } + + Map getOffsetsToCommit() { + return offsetsToCommit; + } - @SuppressWarnings({"rawtypes", "unchecked"}) - public void consumeRecords() { + @Override + public void run() { try { - consumer.subscribe(Arrays.asList(topicConfig.getName())); - do { - offsetsToCommit.clear(); - ConsumerRecords records = poll(consumer); - if (!records.isEmpty() && records.count() > 0) { - iterateRecordPartitions(records); + consumer.subscribe(Arrays.asList(topicName)); + while (!shutdownInProgress.get()) { + consumeRecords(); + long currentTimeMillis = System.currentTimeMillis(); + if (!topicConfig.getAutoCommit() && !offsetsToCommit.isEmpty() && + (currentTimeMillis - lastCommitTime) >= COMMIT_OFFSET_INTERVAL_MS) { + try { + consumer.commitSync(offsetsToCommit); + offsetsToCommit.clear(); + lastCommitTime = currentTimeMillis; + } catch (CommitFailedException e) { + LOG.error("Failed to commit offsets in topic "+topicName); + } } - }while (!status.get()); + } } catch (Exception exp) { LOG.error("Error while reading the records from the topic...", exp); } } - private void iterateRecordPartitions(ConsumerRecords records) throws Exception { - for (TopicPartition partition : records.partitions()) { - List> kafkaRecords = new ArrayList<>(); - List> partitionRecords = records.records(partition); - iterateConsumerRecords(kafkaRecords, partitionRecords); - if (!kafkaRecords.isEmpty()) { - kafkaSourceBufferAccumulator.writeAllRecordToBuffer(kafkaRecords, buffer, topicConfig); + private Record getRecord(ConsumerRecord consumerRecord) { + Map data = new HashMap<>(); + Event event; + if (schema == MessageFormat.JSON || schema == MessageFormat.AVRO) { + Map message = new HashMap<>(); + try { + final JsonParser jsonParser = jsonFactory.createParser((String)consumerRecord.value().toString()); + message = objectMapper.readValue(jsonParser, Map.class); + } catch (Exception e){ + LOG.error("Failed to parse JSON or AVRO record"); + return null; } + data.put(consumerRecord.key(), message); + } else { + data.put(consumerRecord.key(), (String)consumerRecord.value()); } - if (!offsetsToCommit.isEmpty() && topicConfig.getAutoCommit().equalsIgnoreCase("false")) { - lastCommitTime = kafkaSourceBufferAccumulator.commitOffsets(consumer, lastCommitTime, offsetsToCommit); - } + event = JacksonLog.builder().withData(data).build(); + return new Record(event); } - private void iterateConsumerRecords(List> kafkaRecords, List> partitionRecords) { - for (ConsumerRecord consumerRecord : partitionRecords) { - lastReadOffset = kafkaSourceBufferAccumulator.processConsumerRecords(offsetsToCommit, kafkaRecords, lastReadOffset, consumerRecord, partitionRecords); + private Map iterateRecordPartitions(ConsumerRecords records) throws Exception { + Map offsets = new HashMap<>(); + for (TopicPartition topicPartition : records.partitions()) { + List> kafkaRecords = new ArrayList<>(); + List> partitionRecords = records.records(topicPartition); + for (ConsumerRecord consumerRecord : partitionRecords) { + Record record = getRecord(consumerRecord); + if (record != null) { + bufferAccumulator.add(record); + } + } + long lastOffset = partitionRecords.get(partitionRecords.size() - 1).offset(); + offsets.put(topicPartition, new OffsetAndMetadata(lastOffset + 1)); } - } - - private ConsumerRecords poll(final KafkaConsumer consumer) { - return consumer.poll(Duration.ofMillis(1)); + return offsets; } public void closeConsumer(){ @@ -117,17 +156,13 @@ public void shutdownConsumer(){ } @Override public void onPartitionsAssigned(Collection partitions) { - for (TopicPartition partition : partitions) { - consumer.seek(partition, lastReadOffset); + for (TopicPartition topicPartition : partitions) { + Long committedOffset = consumer.committed(topicPartition).offset(); + consumer.seek(topicPartition, committedOffset); } } @Override public void onPartitionsRevoked(Collection partitions) { - try { - consumer.commitSync(offsetsToCommit); - } catch (CommitFailedException e) { - LOG.error("Failed to commit the record for the Json consumer...", e); - } } } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/MultithreadedConsumer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/MultithreadedConsumer.java deleted file mode 100644 index cff5286a91..0000000000 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/MultithreadedConsumer.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.kafka.consumer; - -import com.fasterxml.jackson.databind.JsonNode; -import org.apache.avro.generic.GenericRecord; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.errors.WakeupException; -import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.buffer.Buffer; -import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; -import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.time.LocalDateTime; -import java.util.Objects; -import java.util.Properties; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * * A Multithreaded helper class which helps to process the records from multiple topics in an - * asynchronous way. - */ -@SuppressWarnings("deprecation") -public class MultithreadedConsumer implements Runnable { - private static final Logger LOG = LoggerFactory.getLogger(MultithreadedConsumer.class); - private final AtomicBoolean status = new AtomicBoolean(false); - private final KafkaSourceConfig sourceConfig; - private final TopicConfig topicConfig; - private final Buffer> buffer; - private final KafkaSourceCustomConsumer customConsumer = new KafkaSourceCustomConsumer(); - private String consumerId; - private String consumerGroupId; - private String schemaType; - private Properties consumerProperties; - private PluginMetrics pluginMetrics; - - public MultithreadedConsumer(String consumerId, - String consumerGroupId, - Properties properties, - TopicConfig topicConfig, - KafkaSourceConfig sourceConfig, - Buffer> buffer, - PluginMetrics pluginMetric, - String schemaType) { - this.consumerProperties = Objects.requireNonNull(properties); - this.consumerId = consumerId; - this.consumerGroupId = consumerGroupId; - this.sourceConfig = sourceConfig; - this.topicConfig = topicConfig; - this.buffer = buffer; - this.schemaType = schemaType; - this.pluginMetrics = pluginMetric; - } - - @SuppressWarnings({"unchecked"}) - @Override - public void run() { - LOG.info("Consumer group : {} and Consumer : {} executed on : {}", consumerGroupId, consumerId, LocalDateTime.now()); - try { - MessageFormat schema = MessageFormat.getByMessageFormatByName(schemaType); - switch (schema) { - case JSON: - new KafkaSourceCustomConsumer(new KafkaConsumer(consumerProperties), status, buffer, topicConfig, sourceConfig, schemaType, pluginMetrics).consumeRecords(); - break; - case AVRO: - new KafkaSourceCustomConsumer(new KafkaConsumer(consumerProperties), status, buffer, topicConfig, sourceConfig, schemaType, pluginMetrics).consumeRecords(); - break; - case PLAINTEXT: - default: - new KafkaSourceCustomConsumer(new KafkaConsumer(consumerProperties), status, buffer, topicConfig, sourceConfig, schemaType, pluginMetrics).consumeRecords(); - break; - } - - } catch (Exception exp) { - if (exp.getCause() instanceof WakeupException && !status.get()) { - LOG.error("Error reading records from the topic...{}", exp.getMessage()); - } - } finally { - LOG.info("Closing the consumer... {}", consumerId); - closeConsumers(); - } - } - - private void closeConsumers() { - customConsumer.closeConsumer(); - } - - public void shutdownConsumer() { - status.set(false); - customConsumer.shutdownConsumer(); - } -} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSource.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSource.java index 51b7362eee..9a1d96023f 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSource.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSource.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.kafka.source; +import org.apache.avro.generic.GenericRecord; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; @@ -12,16 +13,18 @@ import io.micrometer.core.instrument.Counter; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.clients.consumer.KafkaConsumer; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.configuration.PipelineDescription; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.source.Source; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; -import org.opensearch.dataprepper.plugins.kafka.consumer.MultithreadedConsumer; +import org.opensearch.dataprepper.plugins.kafka.consumer.KafkaSourceCustomConsumer; import org.opensearch.dataprepper.plugins.kafka.util.KafkaSourceJsonDeserializer; import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; import org.slf4j.Logger; @@ -43,6 +46,7 @@ import java.util.concurrent.TimeUnit; import java.util.stream.IntStream; +import java.util.concurrent.atomic.AtomicBoolean; /** * The starting point of the Kafka-source plugin and the Kafka consumer * properties and kafka multithreaded consumers are being handled here. @@ -50,18 +54,17 @@ @SuppressWarnings("deprecation") @DataPrepperPlugin(name = "kafka", pluginType = Source.class, pluginConfigurationType = KafkaSourceConfig.class) -public class KafkaSource implements Source> { +public class KafkaSource implements Source> { private static final Logger LOG = LoggerFactory.getLogger(KafkaSource.class); private final KafkaSourceConfig sourceConfig; + private AtomicBoolean shutdownInProgress; private ExecutorService executorService; private static final String KAFKA_WORKER_THREAD_PROCESSING_ERRORS = "kafkaWorkerThreadProcessingErrors"; private final Counter kafkaWorkerThreadProcessingErrors; private final PluginMetrics pluginMetrics; - private String consumerGroupID; - private MultithreadedConsumer multithreadedConsumer; - private int totalWorkers; + private KafkaSourceCustomConsumer consumer; private String pipelineName; - private static String schemaType = MessageFormat.PLAINTEXT.toString(); + private String schemaType = MessageFormat.PLAINTEXT.toString(); private static final String SCHEMA_TYPE= "schemaType"; @DataPrepperPluginConstructor @@ -71,37 +74,52 @@ public KafkaSource(final KafkaSourceConfig sourceConfig, final PluginMetrics plu this.pluginMetrics = pluginMetrics; this.pipelineName = pipelineDescription.getPipelineName(); this.kafkaWorkerThreadProcessingErrors = pluginMetrics.counter(KAFKA_WORKER_THREAD_PROCESSING_ERRORS); + shutdownInProgress = new AtomicBoolean(false); } @Override - public void start(Buffer> buffer) { + public void start(Buffer> buffer) { sourceConfig.getTopics().forEach(topic -> { - totalWorkers = 0; + Properties consumerProperties = getConsumerProperties(topic); + MessageFormat schema = MessageFormat.getByMessageFormatByName(schemaType); + KafkaConsumer kafkaConsumer; + switch (schema) { + case JSON: + kafkaConsumer = new KafkaConsumer(consumerProperties); + break; + case AVRO: + kafkaConsumer = new KafkaConsumer(consumerProperties); + break; + case PLAINTEXT: + default: + kafkaConsumer = new KafkaConsumer(consumerProperties); + break; + } + try { - Properties consumerProperties = getConsumerProperties(topic); - totalWorkers = topic.getWorkers(); - consumerGroupID = getGroupId(topic.getName()); - executorService = Executors.newFixedThreadPool(totalWorkers); - IntStream.range(0, totalWorkers + 1).forEach(index -> { - String consumerId = consumerGroupID + "::" + Integer.toString(index + 1); - multithreadedConsumer = new MultithreadedConsumer(consumerId, - consumerGroupID, consumerProperties, topic, sourceConfig, buffer, pluginMetrics, schemaType); - executorService.submit(multithreadedConsumer); + int numWorkers = topic.getWorkers(); + executorService = Executors.newFixedThreadPool(numWorkers); + IntStream.range(0, numWorkers + 1).forEach(index -> { + consumer = new KafkaSourceCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, topic, schemaType, pluginMetrics); + + executorService.submit(consumer); }); } catch (Exception e) { LOG.error("Failed to setup the Kafka Source Plugin.", e); throw new RuntimeException(); } + LOG.info("Started Kafka source for topic " + topic.getName()); }); } @Override public void stop() { LOG.info("Shutting down Consumers..."); + shutdownInProgress.set(true); executorService.shutdown(); try { if (!executorService.awaitTermination( - calculateLongestThreadWaitingTime(), TimeUnit.MILLISECONDS)) { + calculateLongestThreadWaitingTime(), TimeUnit.SECONDS)) { LOG.info("Consumer threads are waiting for shutting down..."); executorService.shutdownNow(); } @@ -115,10 +133,6 @@ public void stop() { LOG.info("Consumer shutdown successfully..."); } - private String getGroupId(String name) { - return pipelineName + "::" + name; - } - private long calculateLongestThreadWaitingTime() { List topicsList = sourceConfig.getTopics(); return topicsList.stream(). @@ -138,8 +152,12 @@ private Properties getConsumerProperties(TopicConfig topicConfig) { properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, sourceConfig.getBootStrapServers()); properties.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, topicConfig.getAutoCommit()); + properties.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, + topicConfig.getConsumerMaxPollRecords()); properties.put(ConsumerConfig.GROUP_ID_CONFIG, topicConfig.getGroupId()); - schemaType = getSchemaType(sourceConfig.getSchemaConfig().getRegistryURL(), topicConfig.getName(), sourceConfig.getSchemaConfig().getVersion()); + if (sourceConfig.getSchemaConfig() != null) { + schemaType = getSchemaType(sourceConfig.getSchemaConfig().getRegistryURL(), topicConfig.getName(), sourceConfig.getSchemaConfig().getVersion()); + } if (schemaType.isEmpty()) { schemaType = MessageFormat.PLAINTEXT.toString(); } diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java index 3e60bb2771..6c26666adf 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java @@ -69,11 +69,11 @@ void testConfigValues_default() { assertEquals("my-topic-2", topicConfig.getName()); assertEquals("DPKafkaProj-2", topicConfig.getGroupId()); assertEquals("kafka-consumer-group-2", topicConfig.getGroupName()); - assertEquals("false", topicConfig.getAutoCommit()); + assertEquals(false, topicConfig.getAutoCommit()); assertEquals(Duration.ofSeconds(5), topicConfig.getAutoCommitInterval()); assertEquals(Duration.ofSeconds(45), topicConfig.getSessionTimeOut()); assertEquals("earliest", topicConfig.getAutoOffsetReset()); - assertEquals(Duration.ofSeconds(1), topicConfig.getThreadWaitingTime()); + assertEquals(TopicConfig.THREAD_WAITING_TIME, topicConfig.getThreadWaitingTime()); assertEquals(Duration.ofSeconds(4), topicConfig.getMaxRecordFetchTime()); assertEquals(Duration.ofSeconds(5), topicConfig.getBufferDefaultTimeout()); assertEquals(52428800L, topicConfig.getFetchMaxBytes().longValue()); @@ -93,7 +93,7 @@ void testConfigValues_from_yaml() { assertEquals("my-topic-1", topicConfig.getName()); assertEquals("DPKafkaProj-2", topicConfig.getGroupId()); assertEquals("kafka-consumer-group-2", topicConfig.getGroupName()); - assertEquals("false", topicConfig.getAutoCommit()); + assertEquals(false, topicConfig.getAutoCommit()); assertEquals(Duration.ofSeconds(5), topicConfig.getAutoCommitInterval()); assertEquals(Duration.ofSeconds(45), topicConfig.getSessionTimeOut()); assertEquals("earliest", topicConfig.getAutoOffsetReset()); diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumerTest.java index 8567658004..22172aece5 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumerTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumerTest.java @@ -1,99 +1,101 @@ package org.opensearch.dataprepper.plugins.kafka.consumer; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import com.fasterxml.jackson.databind.JsonNode; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.Mock; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.CheckpointState; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.plugins.buffer.blockingbuffer.BlockingBuffer; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; -import org.yaml.snakeyaml.Yaml; -import java.io.FileReader; -import java.io.IOException; -import java.io.Reader; -import java.io.StringReader; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.hamcrest.Matchers.hasEntry; +import static org.hamcrest.MatcherAssert.assertThat; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; +import org.mockito.Mock; + import java.util.Map; import java.util.HashMap; import java.util.Arrays; import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.LinkedHashMap; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; - -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.doCallRealMethod; - +import java.time.Duration; +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) public class KafkaSourceCustomConsumerTest { + @Mock private KafkaConsumer kafkaConsumer; private AtomicBoolean status; - private Buffer> buffer; + private Buffer> buffer; @Mock private KafkaSourceConfig sourceConfig; + @Mock private TopicConfig topicConfig; @Mock private PluginMetrics pluginMetrics; - private String schemaType; - private KafkaSourceCustomConsumer consumer; - private final String TEST_PIPELINE_NAME = "test_pipeline"; - - private Map>> records = new LinkedHashMap>>(); + private ConsumerRecords consumerRecords; + private final String TEST_PIPELINE_NAME = "test_pipeline"; + private AtomicBoolean shutdownInProgress; + private final String testKey1 = "testkey1"; + private final String testKey2 = "testkey2"; + private final String testValue1 = "testValue1"; + private final String testValue2 = "testValue2"; + private final Map testMap1 = Map.of("key1", "value1", "key2", 2); + private final Map testMap2 = Map.of("key3", "value3", "key4", false); + private final String testJsonValue1 = "{ \"key1\": \"value1\", \"key2\": 2}"; + private final String testJsonValue2 = "{ \"key3\": \"value3\", \"key4\": false}"; + private final int testPartition = 0; + private final int testJsonPartition = 1; @BeforeEach - public void setUp() throws IOException { - Yaml yaml = new Yaml(); - FileReader fileReader = new FileReader(getClass().getClassLoader().getResource("sample-pipelines.yaml").getFile()); - Object data = yaml.load(fileReader); - if (data instanceof Map) { - Map propertyMap = (Map) data; - Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); - Map sourceMap = (Map) logPipelineMap.get("source"); - Map kafkaConfigMap = (Map) sourceMap.get("kafka"); - ObjectMapper mapper = new ObjectMapper(); - mapper.registerModule(new JavaTimeModule()); - String json = mapper.writeValueAsString(kafkaConfigMap); - Reader reader = new StringReader(json); - sourceConfig = mapper.readValue(reader, KafkaSourceConfig.class); - topicConfig = sourceConfig.getTopics().get(0); - } + public void setUp() { + kafkaConsumer = mock(KafkaConsumer.class); pluginMetrics = mock(PluginMetrics.class); + topicConfig = mock(TopicConfig.class); + when(topicConfig.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(1)); + when(topicConfig.getAutoCommit()).thenReturn(false); + sourceConfig = mock(KafkaSourceConfig.class); buffer = getBuffer(); - status = new AtomicBoolean(true); - kafkaConsumer = mock(KafkaConsumer.class); - schemaType = "plaintext"; - consumer = new KafkaSourceCustomConsumer(kafkaConsumer, status, buffer, topicConfig, sourceConfig, schemaType, pluginMetrics); + shutdownInProgress = new AtomicBoolean(false); + when(topicConfig.getName()).thenReturn("topic1"); } - private BlockingBuffer> getBuffer() { + public KafkaSourceCustomConsumer createObjectUnderTest(String schemaType) { + return new KafkaSourceCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, topicConfig, schemaType, pluginMetrics); + } + + private BlockingBuffer> getBuffer() { final HashMap integerHashMap = new HashMap<>(); integerHashMap.put("buffer_size", 10); integerHashMap.put("batch_size", 10); @@ -103,55 +105,86 @@ private BlockingBuffer> getBuffer() { } @Test - public void testConsumeRecords() throws InterruptedException { - + public void testPlainTextConsumeRecords() throws InterruptedException { String topic = topicConfig.getName(); - - Thread producerThread = new Thread(() -> { - setTopicData(topic); + consumerRecords = createPlainTextRecords(topic); + when(kafkaConsumer.poll(anyLong())).thenReturn(consumerRecords); + consumer = createObjectUnderTest("plaintext"); + + try { + consumer.consumeRecords(); + } catch (Exception e){} + final Map.Entry>, CheckpointState> bufferRecords = buffer.read(1000); + ArrayList> bufferedRecords = new ArrayList<>(bufferRecords.getKey()); + Assertions.assertEquals(consumerRecords.count(), bufferedRecords.size()); + Map offsetsToCommit = consumer.getOffsetsToCommit(); + offsetsToCommit.forEach((topicPartition, offsetAndMetadata) -> { + Assertions.assertEquals(topicPartition.partition(), testPartition); + Assertions.assertEquals(topicPartition.topic(), topic); + Assertions.assertEquals(offsetAndMetadata.offset(), 2L); }); - producerThread.start(); - TimeUnit.SECONDS.sleep(1); - ConsumerRecords consumerRecords = new ConsumerRecords(records); - when(kafkaConsumer.poll(any())).thenReturn(consumerRecords); - KafkaSourceCustomConsumer spyConsumer = spy(consumer); - spyConsumer.consumeRecords(); - verify(spyConsumer).consumeRecords(); - final Map.Entry>, CheckpointState> bufferRecords = buffer.read(1000); - Assertions.assertEquals(2, new ArrayList<>(bufferRecords.getKey()).size()); - } - private void setTopicData(String topic) { - ConsumerRecord record1 = new ConsumerRecord<>(topic, 0, 0L, "mykey-1", "myvalue-1"); - ConsumerRecord record2 = new ConsumerRecord<>(topic, 0, 0L, "mykey-2", "myvalue-2"); - records.put(new TopicPartition(topic, 1), Arrays.asList(record1, record2)); + for (Record record: bufferedRecords) { + Event event = record.getData(); + String value1 = event.get(testKey1, String.class); + String value2 = event.get(testKey2, String.class); + assertTrue(value1 != null || value2 != null); + if (value1 != null) { + Assertions.assertEquals(value1, testValue1); + } + if (value2 != null) { + Assertions.assertEquals(value2, testValue2); + } + } } @Test - void testOnPartitionsRevoked() { - KafkaSourceCustomConsumer spyConsumer = spy(consumer); - setTopicData(topicConfig.getName()); - final List topicPartitions = records.keySet().stream().collect(Collectors.toList()); - spyConsumer.onPartitionsRevoked(topicPartitions); - verify(spyConsumer).onPartitionsRevoked(topicPartitions); + public void testJsonConsumeRecords() throws InterruptedException, Exception { + String topic = topicConfig.getName(); + consumerRecords = createJsonRecords(topic); + when(kafkaConsumer.poll(anyLong())).thenReturn(consumerRecords); + consumer = createObjectUnderTest("json"); + + consumer.consumeRecords(); + final Map.Entry>, CheckpointState> bufferRecords = buffer.read(1000); + ArrayList> bufferedRecords = new ArrayList<>(bufferRecords.getKey()); + Assertions.assertEquals(consumerRecords.count(), bufferedRecords.size()); + Map offsetsToCommit = consumer.getOffsetsToCommit(); + offsetsToCommit.forEach((topicPartition, offsetAndMetadata) -> { + Assertions.assertEquals(topicPartition.partition(), testJsonPartition); + Assertions.assertEquals(topicPartition.topic(), topic); + Assertions.assertEquals(offsetAndMetadata.offset(), 102L); + }); + + for (Record record: bufferedRecords) { + Event event = record.getData(); + Map value1 = event.get(testKey1, Map.class); + Map value2 = event.get(testKey2, Map.class); + assertTrue(value1 != null || value2 != null); + if (value1 != null) { + testMap1.forEach((k, v) -> assertThat(value1, hasEntry(k,v))); + } + if (value2 != null) { + testMap2.forEach((k, v) -> assertThat(value2, hasEntry(k,v))); + } + } } - @Test - void testOnPartitionsAssigned() { - //Map>> records = new LinkedHashMap<>(); - ConsumerRecord record1 = new ConsumerRecord<>("my-topic-1", 0, 0L, "mykey-1", "myvalue-1"); - ConsumerRecord record2 = new ConsumerRecord<>("my-topic-1", 0, 0L, "mykey-2", "myvalue-2"); - //records.put(new TopicPartition("my-topic-1", 1), Arrays.asList(record1, record2)); - TopicPartition partition = new TopicPartition("my-topic-1", 1); - //records.put(partition, Arrays.asList(record1, record2)); - - KafkaSourceCustomConsumer spyConsumer = spy(consumer); - doCallRealMethod().when(spyConsumer).onPartitionsAssigned(Arrays.asList(partition)); - - spyConsumer.onPartitionsAssigned(Arrays.asList(partition)); - verify(spyConsumer).onPartitionsAssigned(Arrays.asList(partition)); - - /*spyConsumer.onPartitionsRevoked(anyList()); - verify(spyConsumer).onPartitionsRevoked(anyList());*/ + private ConsumerRecords createPlainTextRecords(String topic) { + Map> records = new HashMap<>(); + ConsumerRecord record1 = new ConsumerRecord<>(topic, testPartition, 0L, testKey1, testValue1); + ConsumerRecord record2 = new ConsumerRecord<>(topic, testPartition, 1L, testKey2, testValue2); + records.put(new TopicPartition(topic, testPartition), Arrays.asList(record1, record2)); + return new ConsumerRecords(records); + } + + private ConsumerRecords createJsonRecords(String topic) throws Exception { + final ObjectMapper mapper = new ObjectMapper(); + Map> records = new HashMap<>(); + ConsumerRecord record1 = new ConsumerRecord<>(topic, testJsonPartition, 100L, testKey1, mapper.convertValue(testMap1, JsonNode.class)); + ConsumerRecord record2 = new ConsumerRecord<>(topic, testJsonPartition, 101L, testKey2, mapper.convertValue(testMap2, JsonNode.class)); + records.put(new TopicPartition(topic, testJsonPartition), Arrays.asList(record1, record2)); + return new ConsumerRecords(records); } + } diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/MultithreadedConsumerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/MultithreadedConsumerTest.java deleted file mode 100644 index da66a26531..0000000000 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/MultithreadedConsumerTest.java +++ /dev/null @@ -1,126 +0,0 @@ -package org.opensearch.dataprepper.plugins.kafka.consumer; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.Mock; -import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.buffer.Buffer; -import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; -import org.yaml.snakeyaml.Yaml; - -import java.io.FileReader; -import java.io.IOException; -import java.io.Reader; -import java.io.StringReader; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Arrays; -import java.util.Properties; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.doCallRealMethod; - - -class MultithreadedConsumerTest { - - @Mock - MultithreadedConsumer multithreadedConsumer; - @Mock - Properties properties; - @Mock - KafkaSourceConfig sourceConfig; - @Mock - TopicConfig topicConfig; - @Mock - Buffer> buffer; - @Mock - PluginMetrics pluginMetrics; - private static final String BOOTSTRAP_SERVERS = "localhost:9092"; - @BeforeEach - void setUp()throws IOException { - Yaml yaml = new Yaml(); - FileReader fileReader = new FileReader(getClass().getClassLoader().getResource("sample-pipelines.yaml").getFile()); - Object data = yaml.load(fileReader); - if (data instanceof Map) { - Map propertyMap = (Map) data; - Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); - Map sourceMap = (Map) logPipelineMap.get("source"); - Map kafkaConfigMap = (Map) sourceMap.get("kafka"); - ObjectMapper mapper = new ObjectMapper(); - mapper.registerModule(new JavaTimeModule()); - String json = mapper.writeValueAsString(kafkaConfigMap); - Reader reader = new StringReader(json); - sourceConfig = mapper.readValue(reader, KafkaSourceConfig.class); - topicConfig = sourceConfig.getTopics().get(0); - } - pluginMetrics = mock(PluginMetrics.class); - buffer = mock(Buffer.class); - } - - private MultithreadedConsumer createObjectUnderTest(String consumerId, - String consumerGroupId, - String schema){ - properties = new Properties(); - properties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, BOOTSTRAP_SERVERS); - properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - return new MultithreadedConsumer(consumerId, - consumerGroupId, - properties, - topicConfig, - sourceConfig, - buffer, - pluginMetrics, - schema); - } - - // @ParameterizedTest - // @ValueSource(strings = "plaintext") - @Test - void testRunWithPlainText() throws InterruptedException { - String topic = topicConfig.getName(); - //schemaType = "plaintext"; - Map>> records = new LinkedHashMap<>(); - Thread producerThread = new Thread(() -> { - ConsumerRecord record1 = new ConsumerRecord<>(topic, 0, 0L, "mykey-1", "myvalue-1"); - ConsumerRecord record2 = new ConsumerRecord<>(topic, 0, 0L, "mykey-2", "myvalue-2"); - records.put(new TopicPartition(topic, 1), Arrays.asList(record1, record2)); - }); - producerThread.start(); - TimeUnit.SECONDS.sleep(1); - producerThread.join(); - multithreadedConsumer = createObjectUnderTest("DPKafkaProj-1", - "DPKafkaProj-1","plaintext"); - MultithreadedConsumer spySource = spy(multithreadedConsumer); - doCallRealMethod().when(spySource).run(); - // spySource.run(); - //verify(spySource).run(); - } - - - /* @ParameterizedTest - @ValueSource(strings = {"plaintext", "json", "avro"}) - @Test - void testRunWithParameters(String schemaType) { - multithreadedConsumer = createObjectUnderTest("DPKafkaProj-1", - "DPKafkaProj-1", - schemaType); - MultithreadedConsumer spySource = spy(multithreadedConsumer); - doCallRealMethod().when(spySource).run(); - spySource.run(); - verify(spySource).run(); - }*/ - -} \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java index ec8d15e9b1..ba08ec5dbd 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java @@ -5,11 +5,16 @@ package org.opensearch.dataprepper.plugins.kafka.source; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig; + import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -18,37 +23,17 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.quality.Strictness; -import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.configuration.PipelineDescription; -import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.OAuthConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig; -import org.opensearch.dataprepper.plugins.kafka.consumer.MultithreadedConsumer; -import org.springframework.test.util.ReflectionTestUtils; -import org.yaml.snakeyaml.Yaml; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; -import java.io.FileReader; -import java.io.Reader; -import java.io.StringReader; -import java.util.ArrayList; +import java.util.Arrays; import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.ExecutorService; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.doCallRealMethod; -import static org.mockito.Mockito.verify; - +import java.time.Duration; @ExtendWith(MockitoExtension.class) @MockitoSettings(strictness = Strictness.LENIENT) class KafkaSourceTest { - @Mock - private KafkaSource source; + private KafkaSource kafkaSource; @Mock private KafkaSourceConfig sourceConfig; @@ -56,9 +41,6 @@ class KafkaSourceTest { @Mock private PluginMetrics pluginMetrics; - @Mock - private ExecutorService executorService; - @Mock private SchemaConfig schemaConfig; @@ -67,63 +49,59 @@ class KafkaSourceTest { @Mock private PipelineDescription pipelineDescription; @Mock - OAuthConfig oAuthConfig; - @Mock PlainTextAuthConfig plainTextAuthConfig; + @Mock + TopicConfig topic1, topic2; + @Mock + private Buffer> buffer; + private static final String BOOTSTRAP_SERVERS = "localhost:9092"; private static final String TOPIC = "my-topic"; + private static final String TEST_GROUP_ID = "testGroupId"; + + + public KafkaSource createObjectUnderTest() { + return new KafkaSource(sourceConfig, pluginMetrics, pipelineDescription); + } @BeforeEach void setUp() throws Exception { - Yaml yaml = new Yaml(); - FileReader fileReader = new FileReader(getClass().getClassLoader().getResource("sample-pipelines.yaml").getFile()); - Object data = yaml.load(fileReader); - if (data instanceof Map) { - Map propertyMap = (Map) data; - Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); - Map sourceMap = (Map) logPipelineMap.get("source"); - Map kafkaConfigMap = (Map) sourceMap.get("kafka"); - ObjectMapper mapper = new ObjectMapper(); - mapper.registerModule(new JavaTimeModule()); - String json = mapper.writeValueAsString(kafkaConfigMap); - Reader reader = new StringReader(json); - sourceConfig = mapper.readValue(reader, KafkaSourceConfig.class); - topicConfig = sourceConfig.getTopics().get(0); - oAuthConfig = sourceConfig.getAuthConfig().getoAuthConfig(); - plainTextAuthConfig = sourceConfig.getAuthConfig().getPlainTextAuthConfig(); - schemaConfig = sourceConfig.getSchemaConfig(); - } + sourceConfig = mock(KafkaSourceConfig.class); + pipelineDescription = mock(PipelineDescription.class); + pluginMetrics = mock(PluginMetrics.class); + when(topic1.getName()).thenReturn("topic1"); + when(topic2.getName()).thenReturn("topic2"); + when(topic1.getWorkers()).thenReturn(2); + when(topic2.getWorkers()).thenReturn(3); + when(topic1.getAutoCommitInterval()).thenReturn(Duration.ofSeconds(1)); + when(topic2.getAutoCommitInterval()).thenReturn(Duration.ofSeconds(1)); + when(topic1.getAutoOffsetReset()).thenReturn("earliest"); + when(topic2.getAutoOffsetReset()).thenReturn("earliest"); + when(topic1.getConsumerMaxPollRecords()).thenReturn(1); + when(topic2.getConsumerMaxPollRecords()).thenReturn(1); + when(topic1.getGroupId()).thenReturn(TEST_GROUP_ID); + when(topic2.getGroupId()).thenReturn(TEST_GROUP_ID); + when(topic1.getAutoCommit()).thenReturn(false); + when(topic2.getAutoCommit()).thenReturn(false); + when(topic1.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(10)); + when(topic2.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(10)); + when(sourceConfig.getBootStrapServers()).thenReturn(List.of("http://localhost:1234")); + when(sourceConfig.getTopics()).thenReturn(Arrays.asList(topic1, topic2)); } @Test - void test_kafkaSource_start_execution_catch_block() { - source = new KafkaSource(null, pluginMetrics, pipelineDescription); - KafkaSource spySource = spy(source); - Assertions.assertThrows(Exception.class, () -> spySource.start(any())); + void test_kafkaSource_start_stop() { + kafkaSource = createObjectUnderTest(); + kafkaSource.start(buffer); + try { + Thread.sleep(10); + } catch (Exception e){} + kafkaSource.stop(); } @Test - void test_kafkaSource_stop_execution() throws Exception { - List consumers = buildKafkaSourceConsumer(); - source = new KafkaSource(sourceConfig, pluginMetrics,pipelineDescription); - KafkaSource spySource = spy(source); - ReflectionTestUtils.setField(spySource, "executorService", executorService); - doCallRealMethod().when(spySource).stop(); - spySource.stop(); - verify(spySource).stop(); - } - - private List buildKafkaSourceConsumer() { - List consumers = new ArrayList<>(); - Properties prop = new Properties(); - prop.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, BOOTSTRAP_SERVERS); - prop.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - prop.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - MultithreadedConsumer kafkaSourceConsumer = new MultithreadedConsumer( - topicConfig.getGroupId(), - topicConfig.getGroupId(), - prop, null,sourceConfig, null, pluginMetrics,null); - consumers.add(kafkaSourceConsumer); - return consumers; + void test_kafkaSource_start_execution_catch_block() { + kafkaSource = createObjectUnderTest(); + Assertions.assertThrows(Exception.class, () -> kafkaSource.start(null)); } } diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml index 2c398a99dc..efd860a8a9 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml @@ -8,8 +8,8 @@ log-pipeline: group_name: kafka-consumer-group-2 group_id: DPKafkaProj-2 workers: 10 #optional and default is 10 - autocommit: false #optional and dafault is false - autocommit_interval: 5 #optional and dafault is 5s + auto_commit: false #optional and dafault is false + auto_commit_interval: 5 #optional and dafault is 5s session_timeout: 45 #optional and dafault is 45s max_retry_attempts: 1000 #optional and dafault is 5 max_retry_delay: 1 #optional and dafault is 5 @@ -48,4 +48,4 @@ log-pipeline: oauth_sasl_login_callback_handler_class: org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler oauth_jwks_endpoint_url: https://dev-13650048.okta.com/oauth2/default/v1/keys sink: - - stdout: \ No newline at end of file + - stdout: diff --git a/data-prepper-plugins/opensearch-source/README.md b/data-prepper-plugins/opensearch-source/README.md index 4894ba646d..b904d67378 100644 --- a/data-prepper-plugins/opensearch-source/README.md +++ b/data-prepper-plugins/opensearch-source/README.md @@ -114,6 +114,9 @@ opensearch-source-pipeline: - `password` (Optional) : A String of password used in the internal users of OpenSearch cluster. Default is null. +- `disable_authentication` (Optional) : A boolean that can disable authentication if the cluster supports it. Defaults to false. + + - `aws` (Optional) : AWS configurations. See [AWS Configuration](#aws_configuration) for details. SigV4 is enabled by default when this option is used. diff --git a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSource.java b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSource.java index 9a91db3ac1..22455dec3f 100644 --- a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSource.java +++ b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSource.java @@ -31,6 +31,8 @@ public OpenSearchSource(final OpenSearchSourceConfiguration openSearchSourceConf final AwsCredentialsSupplier awsCredentialsSupplier) { this.openSearchSourceConfiguration = openSearchSourceConfiguration; this.awsCredentialsSupplier = awsCredentialsSupplier; + + openSearchSourceConfiguration.validateAwsConfigWithUsernameAndPassword(); } @Override diff --git a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfiguration.java b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfiguration.java index 8e2fd42384..a100e17b20 100644 --- a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfiguration.java +++ b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfiguration.java @@ -6,9 +6,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.Valid; -import jakarta.validation.constraints.AssertTrue; -import jakarta.validation.constraints.Min; import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.plugins.source.opensearch.configuration.AwsAuthenticationConfiguration; import org.opensearch.dataprepper.plugins.source.opensearch.configuration.ConnectionConfiguration; import org.opensearch.dataprepper.plugins.source.opensearch.configuration.IndexParametersConfiguration; @@ -20,13 +19,6 @@ public class OpenSearchSourceConfiguration { - /** - * 0 indicates infinite retries - */ - @JsonProperty("max_retries") - @Min(0) - private Integer maxRetries = 0; - @NotNull @JsonProperty("hosts") private List hosts; @@ -37,6 +29,9 @@ public class OpenSearchSourceConfiguration { @JsonProperty("password") private String password; + @JsonProperty("disable_authentication") + private Boolean disableAuthentication = false; + @JsonProperty("connection") @Valid private ConnectionConfiguration connectionConfiguration = new ConnectionConfiguration(); @@ -57,10 +52,6 @@ public class OpenSearchSourceConfiguration { @Valid private SearchConfiguration searchConfiguration = new SearchConfiguration(); - public Integer getMaxRetries() { - return maxRetries; - } - public List getHosts() { return hosts; } @@ -73,6 +64,8 @@ public String getPassword() { return password; } + public Boolean isAuthenticationDisabled() { return disableAuthentication; } + public ConnectionConfiguration getConnectionConfiguration() { return connectionConfiguration; } @@ -93,10 +86,13 @@ public SearchConfiguration getSearchConfiguration() { return searchConfiguration; } - @AssertTrue(message = "Either username and password, or aws options must be specified. Both cannot be set at once.") - boolean validateAwsConfigWithUsernameAndPassword() { - return !((Objects.nonNull(awsAuthenticationOptions) && (Objects.nonNull(username) || Objects.nonNull(password))) || - (Objects.isNull(awsAuthenticationOptions) && (Objects.isNull(username) || Objects.isNull(password)))); + void validateAwsConfigWithUsernameAndPassword() { + + if (((Objects.nonNull(awsAuthenticationOptions) && ((Objects.nonNull(username) || Objects.nonNull(password)) || disableAuthentication)) || + (Objects.nonNull(username) || Objects.nonNull(password)) && disableAuthentication) || + (Objects.isNull(awsAuthenticationOptions) && (Objects.isNull(username) || Objects.isNull(password)) && !disableAuthentication)) { + throw new InvalidPluginConfigurationException("Either username and password, or aws options must be specified. Both cannot be set at once. Authentication can be disabled by setting the disable_authentication flag to true."); + } } } diff --git a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactory.java b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactory.java index d9e3a2f739..e588e1f711 100644 --- a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactory.java +++ b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactory.java @@ -133,8 +133,7 @@ private RestClient createOpenSearchRestClient(final OpenSearchSourceConfiguratio final RestClientBuilder restClientBuilder = RestClient.builder(httpHosts); - LOG.info("Using username and password for auth for the OpenSearch source"); - attachUsernamePassword(restClientBuilder, openSearchSourceConfiguration); + attachBasicAuth(restClientBuilder, openSearchSourceConfiguration); setConnectAndSocketTimeout(restClientBuilder, openSearchSourceConfiguration); @@ -161,33 +160,36 @@ private org.elasticsearch.client.RestClient createElasticSearchRestClient(final new BasicHeader("Content-type", "application/json") }); - LOG.info("Using username and password for auth for the OpenSearch source"); - attachUsernamePassword(restClientBuilder, openSearchSourceConfiguration); - + attachBasicAuth(restClientBuilder, openSearchSourceConfiguration); setConnectAndSocketTimeout(restClientBuilder, openSearchSourceConfiguration); return restClientBuilder.build(); } - private void attachUsernamePassword(final RestClientBuilder restClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { - final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); - credentialsProvider.setCredentials(AuthScope.ANY, - new UsernamePasswordCredentials(openSearchSourceConfiguration.getUsername(), openSearchSourceConfiguration.getPassword())); + private void attachBasicAuth(final RestClientBuilder restClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { restClientBuilder.setHttpClientConfigCallback(httpClientBuilder -> { - httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider); + if (!openSearchSourceConfiguration.isAuthenticationDisabled()) { + attachUsernameAndPassword(httpClientBuilder, openSearchSourceConfiguration); + } else { + LOG.warn("Authentication was explicitly disabled for the OpenSearch source"); + } + attachSSLContext(httpClientBuilder, openSearchSourceConfiguration); return httpClientBuilder; }); } - private void attachUsernamePassword(final org.elasticsearch.client.RestClientBuilder restClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { - final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); - credentialsProvider.setCredentials(AuthScope.ANY, - new UsernamePasswordCredentials(openSearchSourceConfiguration.getUsername(), openSearchSourceConfiguration.getPassword())); + private void attachBasicAuth(final org.elasticsearch.client.RestClientBuilder restClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { restClientBuilder.setHttpClientConfigCallback(httpClientBuilder -> { - httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider); + + if (!openSearchSourceConfiguration.isAuthenticationDisabled()) { + attachUsernameAndPassword(httpClientBuilder, openSearchSourceConfiguration); + } else { + LOG.warn("Authentication was explicitly disabled for the OpenSearch source"); + } + attachSSLContext(httpClientBuilder, openSearchSourceConfiguration); httpClientBuilder.addInterceptorLast( (HttpResponseInterceptor) @@ -211,6 +213,15 @@ private void setConnectAndSocketTimeout(final RestClientBuilder restClientBuilde }); } + private void attachUsernameAndPassword(final HttpAsyncClientBuilder httpClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { + LOG.info("Using username and password for auth for the OpenSearch source"); + + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials(AuthScope.ANY, + new UsernamePasswordCredentials(openSearchSourceConfiguration.getUsername(), openSearchSourceConfiguration.getPassword())); + httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider); + } + private void setConnectAndSocketTimeout(final org.elasticsearch.client.RestClientBuilder restClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { restClientBuilder.setRequestConfigCallback(requestConfigBuilder -> { if (Objects.nonNull(openSearchSourceConfiguration.getConnectionConfiguration().getConnectTimeout())) { diff --git a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfigurationTest.java b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfigurationTest.java index 7c9f8dbd19..950533c145 100644 --- a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfigurationTest.java +++ b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfigurationTest.java @@ -9,10 +9,12 @@ import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; +import static org.junit.jupiter.api.Assertions.assertThrows; public class OpenSearchSourceConfigurationTest { @@ -21,7 +23,7 @@ public class OpenSearchSourceConfigurationTest { @Test void open_search_source_username_password_only() throws JsonProcessingException { - final String sourceConfigurationYaml = "max_retries: 5\n" + + final String sourceConfigurationYaml = "hosts: [\"http://localhost:9200\"]\n" + "username: test\n" + "password: test\n" + @@ -44,18 +46,49 @@ void open_search_source_username_password_only() throws JsonProcessingException assertThat(sourceConfiguration.getIndexParametersConfiguration(), notNullValue()); assertThat(sourceConfiguration.getSchedulingParameterConfiguration(), notNullValue()); assertThat(sourceConfiguration.getHosts(), notNullValue()); - assertThat(sourceConfiguration.getMaxRetries(), equalTo(5)); - assertThat(sourceConfiguration.validateAwsConfigWithUsernameAndPassword(), equalTo(true)); + sourceConfiguration.validateAwsConfigWithUsernameAndPassword(); assertThat(sourceConfiguration.getPassword(), equalTo("test")); assertThat(sourceConfiguration.getUsername(), equalTo("test")); assertThat(sourceConfiguration.getAwsAuthenticationOptions(), equalTo(null)); } @Test - void opensearch_source_aws_only() throws JsonProcessingException { - final String sourceConfigurationYaml = "max_retries: 5\n" + + void open_search_disabled_authentication() throws JsonProcessingException { + + final String sourceConfigurationYaml = "hosts: [\"http://localhost:9200\"]\n" + + "disable_authentication: true\n" + + "connection:\n" + + " insecure: true\n" + + " cert: \"cert\"\n" + + "indices:\n" + + " include:\n" + + " - index_name_regex: \"regex\"\n" + + " - index_name_regex: \"regex-two\"\n" + + "scheduling:\n" + + " job_count: 3\n" + + "search_options:\n" + + " batch_size: 1000\n" + + " query: \"test\"\n"; + final OpenSearchSourceConfiguration sourceConfiguration = objectMapper.readValue(sourceConfigurationYaml, OpenSearchSourceConfiguration.class); + + assertThat(sourceConfiguration.getSearchConfiguration(), notNullValue()); + assertThat(sourceConfiguration.getConnectionConfiguration(), notNullValue()); + assertThat(sourceConfiguration.getIndexParametersConfiguration(), notNullValue()); + assertThat(sourceConfiguration.getSchedulingParameterConfiguration(), notNullValue()); + assertThat(sourceConfiguration.getHosts(), notNullValue()); + + sourceConfiguration.validateAwsConfigWithUsernameAndPassword(); + assertThat(sourceConfiguration.isAuthenticationDisabled(), equalTo(true)); + assertThat(sourceConfiguration.getPassword(), equalTo(null)); + assertThat(sourceConfiguration.getUsername(), equalTo(null)); + assertThat(sourceConfiguration.getAwsAuthenticationOptions(), equalTo(null)); + } + + @Test + void opensearch_source_aws_only() throws JsonProcessingException { + final String sourceConfigurationYaml = "hosts: [\"http://localhost:9200\"]\n" + "connection:\n" + " insecure: true\n" + " cert: \"cert\"\n" + @@ -74,7 +107,7 @@ void opensearch_source_aws_only() throws JsonProcessingException { final OpenSearchSourceConfiguration sourceConfiguration = objectMapper.readValue(sourceConfigurationYaml, OpenSearchSourceConfiguration.class); - assertThat(sourceConfiguration.validateAwsConfigWithUsernameAndPassword(), equalTo(true)); + sourceConfiguration.validateAwsConfigWithUsernameAndPassword(); assertThat(sourceConfiguration.getPassword(), equalTo(null)); assertThat(sourceConfiguration.getUsername(), equalTo(null)); assertThat(sourceConfiguration.getAwsAuthenticationOptions(), notNullValue()); @@ -85,8 +118,7 @@ void opensearch_source_aws_only() throws JsonProcessingException { @Test void opensearch_source_aws_sts_external_id() throws JsonProcessingException { - final String sourceConfigurationYaml = "max_retries: 5\n" + - "hosts: [\"http://localhost:9200\"]\n" + + final String sourceConfigurationYaml = "hosts: [\"http://localhost:9200\"]\n" + "connection:\n" + " insecure: true\n" + " cert: \"cert\"\n" + @@ -106,7 +138,7 @@ void opensearch_source_aws_sts_external_id() throws JsonProcessingException { final OpenSearchSourceConfiguration sourceConfiguration = objectMapper.readValue(sourceConfigurationYaml, OpenSearchSourceConfiguration.class); - assertThat(sourceConfiguration.validateAwsConfigWithUsernameAndPassword(), equalTo(true)); + sourceConfiguration.validateAwsConfigWithUsernameAndPassword(); assertThat(sourceConfiguration.getPassword(), equalTo(null)); assertThat(sourceConfiguration.getUsername(), equalTo(null)); assertThat(sourceConfiguration.getAwsAuthenticationOptions(), notNullValue()); @@ -141,14 +173,12 @@ void using_both_aws_config_and_username_password_is_invalid() throws JsonProcess final OpenSearchSourceConfiguration sourceConfiguration = objectMapper.readValue(sourceConfigurationYaml, OpenSearchSourceConfiguration.class); - assertThat(sourceConfiguration.validateAwsConfigWithUsernameAndPassword(), equalTo(false)); - assertThat(sourceConfiguration.getMaxRetries(), equalTo(0)); + assertThrows(InvalidPluginConfigurationException.class, sourceConfiguration::validateAwsConfigWithUsernameAndPassword); } @Test - void one_of_username_password_or_aws_config_is_required() throws JsonProcessingException { + void one_of_username_password_or_aws_config_or_authDisabled_is_required() throws JsonProcessingException { final String sourceConfigurationYaml = - "max_retries: 5\n" + "hosts: [\"http://localhost:9200\"]\n" + "connection:\n" + " insecure: true\n" + @@ -165,6 +195,6 @@ void one_of_username_password_or_aws_config_is_required() throws JsonProcessingE final OpenSearchSourceConfiguration sourceConfiguration = objectMapper.readValue(sourceConfigurationYaml, OpenSearchSourceConfiguration.class); - assertThat(sourceConfiguration.validateAwsConfigWithUsernameAndPassword(), equalTo(false)); + assertThrows(InvalidPluginConfigurationException.class, sourceConfiguration::validateAwsConfigWithUsernameAndPassword); } } diff --git a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactoryTest.java b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactoryTest.java index cc811625d1..1cd2ad551c 100644 --- a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactoryTest.java +++ b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactoryTest.java @@ -29,6 +29,8 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; @@ -119,4 +121,38 @@ void provideOpenSearchClient_with_aws_auth() { assertThat(awsCredentialsOptions.getStsHeaderOverrides(), equalTo(Collections.emptyMap())); assertThat(awsCredentialsOptions.getStsRoleArn(), equalTo(stsRoleArn)); } + + @Test + void provideElasticSearchClient_with_auth_disabled() { + when(openSearchSourceConfiguration.isAuthenticationDisabled()).thenReturn(true); + + when(connectionConfiguration.getCertPath()).thenReturn(null); + when(connectionConfiguration.getSocketTimeout()).thenReturn(null); + when(connectionConfiguration.getConnectTimeout()).thenReturn(null); + when(connectionConfiguration.isInsecure()).thenReturn(true); + + final ElasticsearchClient elasticsearchClient = createObjectUnderTest().provideElasticSearchClient(openSearchSourceConfiguration); + assertThat(elasticsearchClient, notNullValue()); + + verifyNoInteractions(awsCredentialsSupplier); + verify(openSearchSourceConfiguration, never()).getUsername(); + verify(openSearchSourceConfiguration, never()).getPassword(); + } + + @Test + void provideOpenSearchClient_with_auth_disabled() { + when(openSearchSourceConfiguration.isAuthenticationDisabled()).thenReturn(true); + + when(connectionConfiguration.getCertPath()).thenReturn(null); + when(connectionConfiguration.getSocketTimeout()).thenReturn(null); + when(connectionConfiguration.getConnectTimeout()).thenReturn(null); + when(connectionConfiguration.isInsecure()).thenReturn(true); + + final OpenSearchClient openSearchClient = createObjectUnderTest().provideOpenSearchClient(openSearchSourceConfiguration); + assertThat(openSearchClient, notNullValue()); + + verifyNoInteractions(awsCredentialsSupplier); + verify(openSearchSourceConfiguration, never()).getUsername(); + verify(openSearchSourceConfiguration, never()).getPassword(); + } } diff --git a/data-prepper-plugins/s3-source/build.gradle b/data-prepper-plugins/s3-source/build.gradle index 343f578b07..70c27e342a 100644 --- a/data-prepper-plugins/s3-source/build.gradle +++ b/data-prepper-plugins/s3-source/build.gradle @@ -30,7 +30,7 @@ dependencies { implementation 'org.hibernate.validator:hibernate-validator:7.0.5.Final' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-csv' implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.14.2' - implementation 'org.xerial.snappy:snappy-java:1.1.9.1' + implementation 'org.xerial.snappy:snappy-java:1.1.10.1' implementation 'org.apache.parquet:parquet-common:1.12.3' testImplementation 'org.apache.commons:commons-lang3:3.12.0' testImplementation 'com.github.tomakehurst:wiremock:3.0.0-beta-8' diff --git a/data-prepper-plugins/translate-processor/build.gradle b/data-prepper-plugins/translate-processor/build.gradle new file mode 100644 index 0000000000..6b6526fda4 --- /dev/null +++ b/data-prepper-plugins/translate-processor/build.gradle @@ -0,0 +1,23 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +plugins { + id 'java' +} + + +dependencies { + implementation project(':data-prepper-api') + implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.15.0' + implementation 'io.micrometer:micrometer-core' + implementation project(path: ':data-prepper-api') + testImplementation project(':data-prepper-plugins:log-generator-source') + testImplementation project(':data-prepper-test-common') + implementation 'org.apache.commons:commons-lang3:3.12.0' +} + +test { + useJUnitPlatform() +} \ No newline at end of file diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/RegexParameterConfiguration.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/RegexParameterConfiguration.java new file mode 100644 index 0000000000..e8e49bd83f --- /dev/null +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/RegexParameterConfiguration.java @@ -0,0 +1,25 @@ +package org.opensearch.dataprepper.plugins.processor.translate; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.NotNull; + +import java.util.Map; + + +public class RegexParameterConfiguration { + + private static final boolean DEFAULT_EXACT = true; + @NotNull + @JsonProperty("patterns") + private Map patterns; + + @JsonProperty("exact") + private Boolean exact = DEFAULT_EXACT; + + public Map getPatterns() { + return patterns; + } + + public Boolean getExact() { return exact; } + +} diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessor.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessor.java new file mode 100644 index 0000000000..be965029a1 --- /dev/null +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessor.java @@ -0,0 +1,213 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.translate; + +import org.apache.commons.lang3.Range; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.math.NumberUtils; +import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; +import org.opensearch.dataprepper.model.processor.AbstractProcessor; +import org.opensearch.dataprepper.model.processor.Processor; +import org.opensearch.dataprepper.model.record.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.LinkedHashMap; +import java.util.HashMap; +import java.util.Objects; +import java.util.regex.Pattern; + +import static org.opensearch.dataprepper.logging.DataPrepperMarkers.EVENT; + + +@DataPrepperPlugin(name = "translate", pluginType = Processor.class, pluginConfigurationType = TranslateProcessorConfig.class) +public class TranslateProcessor extends AbstractProcessor, Record> { + + private static final Logger LOG = LoggerFactory.getLogger(TranslateProcessor.class); + private final ExpressionEvaluator expressionEvaluator; + private final TranslateProcessorConfig translateProcessorConfig; + private final LinkedHashMap, String> rangeMappings; + private final Map individualMappings; + private final Map patternMappings; + + @DataPrepperPluginConstructor + public TranslateProcessor(PluginMetrics pluginMetrics, final TranslateProcessorConfig translateProcessorConfig, final ExpressionEvaluator expressionEvaluator) { + super(pluginMetrics); + this.translateProcessorConfig = translateProcessorConfig; + this.expressionEvaluator = expressionEvaluator; + individualMappings = new HashMap<>(); + rangeMappings = new LinkedHashMap<>(); + if(this.translateProcessorConfig.getRegexParameterConfiguration()!=null) { + patternMappings = translateProcessorConfig.getRegexParameterConfiguration().getPatterns(); + } + else{ + patternMappings = Collections.emptyMap(); + } + + processMapField(translateProcessorConfig.getMap()); + parseFile(translateProcessorConfig.getFilePath()); + checkOverlappingKeys(); + } + + private void processMapField(Map map){ + if(Objects.nonNull(map)) { + for (Map.Entry mapEntry : map.entrySet()) { + parseIndividualKeys(mapEntry); + } + } + } + + private void parseIndividualKeys(Map.Entry mapEntry){ + String[] commaSeparatedKeys = mapEntry.getKey().split(","); + for(String individualKey : commaSeparatedKeys){ + if(individualKey.contains("-")){ + addRangeMapping(Map.entry(individualKey, mapEntry.getValue())); + } + else { + addIndividualMapping(individualKey, mapEntry.getValue()); + } + } + } + + private void addRangeMapping(Map.Entry mapEntry){ + String[] rangeKeys = mapEntry.getKey().split("-"); + if(rangeKeys.length!=2 || !StringUtils.isNumericSpace(rangeKeys[0]) || !StringUtils.isNumericSpace(rangeKeys[1])){ + addIndividualMapping(mapEntry.getKey(), mapEntry.getValue()); + } + else { + Float lowKey = Float.parseFloat(rangeKeys[0]); + Float highKey = Float.parseFloat(rangeKeys[1]); + Range rangeEntry = Range.between(lowKey, highKey); + if (isRangeOverlapping(rangeEntry)) { + String exceptionMsg = "map option contains key "+mapEntry.getKey()+" that overlaps with other range entries"; + throw new InvalidPluginConfigurationException(exceptionMsg); + } else { + rangeMappings.put(Range.between(lowKey, highKey), mapEntry.getValue()); + } + } + } + + private void addIndividualMapping(String key, String value){ + if(individualMappings.containsKey(key)){ + String exceptionMsg = "map option contains duplicate entries of "+key; + throw new InvalidPluginConfigurationException(exceptionMsg); + } + else{ + individualMappings.put(key.strip(), value); + } + } + + private boolean isRangeOverlapping(Range rangeEntry){ + for(Range range : rangeMappings.keySet()){ + if(range.isOverlappedBy(rangeEntry)){ + return true; + } + } + return false; + } + + private void checkOverlappingKeys(){ + for(String individualKey : individualMappings.keySet()){ + if(NumberUtils.isParsable(individualKey)){ + Float floatKey = Float.parseFloat(individualKey); + Range range = Range.between(floatKey, floatKey); + if(isRangeOverlapping(range)){ + String exceptionMsg = "map option contains key "+individualKey+" that overlaps with other range entries"; + throw new InvalidPluginConfigurationException(exceptionMsg); + } + } + } + } + + private void parseFile(String filePath){ + //todo + } + + @Override + public Collection> doExecute(Collection> records) { + //todo + for(final Record record : records) { + final Event recordEvent = record.getData(); + if (Objects.nonNull(translateProcessorConfig.getMapWhen()) && !expressionEvaluator.evaluateConditional(translateProcessorConfig.getMapWhen(), recordEvent)) { + continue; + } + try { + String matchKey = record.getData().get(translateProcessorConfig.getSource(), String.class); + if(matchesIndividualEntry(record, matchKey) || matchesRangeEntry(record, matchKey) || matchesPatternEntry(record, matchKey)){ + continue; + } + else{ + + // todo : add default, increment metrics, and/or add_tags + + } + } catch (Exception ex){ + LOG.error(EVENT, "Error mapping the source [{}] of entry [{}]", + translateProcessorConfig.getSource(), record.getData(), ex); + } + } + return records; + } + + public boolean matchesIndividualEntry(Record record, String matchKey){ + if(individualMappings.containsKey(matchKey)){ + record.getData().put(translateProcessorConfig.getTarget(), individualMappings.get(matchKey)); + return true; + } + return false; + } + + public boolean matchesRangeEntry(Record record, String matchKey){ + if(!NumberUtils.isParsable(matchKey)){ + return false; + } + Float floatKey = Float.parseFloat(matchKey); + for(Map.Entry, String> rangeEntry : rangeMappings.entrySet()) { + Range range = rangeEntry.getKey(); + if (range.contains(floatKey)) { + record.getData().put(translateProcessorConfig.getTarget(), rangeEntry.getValue()); + return true; + } + } + return false; + } + + public boolean matchesPatternEntry(Record record, String matchKey){ + //todo + if(!Objects.nonNull(patternMappings)){ + return false; + } + for(String pattern : patternMappings.keySet()){ + if(Pattern.matches(pattern, matchKey)){ + record.getData().put(translateProcessorConfig.getTarget(), patternMappings.get(pattern)); + return true; + } + } + return false; + } + @Override + public void prepareForShutdown() { + + } + + @Override + public boolean isReadyForShutdown() { + return true; + } + + @Override + public void shutdown() { + + } +} diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java new file mode 100644 index 0000000000..906778d9f2 --- /dev/null +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java @@ -0,0 +1,66 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.translate; + + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.AssertTrue; +import jakarta.validation.constraints.NotEmpty; +import jakarta.validation.constraints.NotNull; +import java.util.Map; +import java.util.stream.Stream; + + +public class TranslateProcessorConfig { + + @JsonProperty("source") + @NotNull + @NotEmpty + private String source; + + @JsonProperty("target") + @NotNull + @NotEmpty + private String target; + + @JsonProperty("map") + private Map map; + @JsonProperty("file_path") + private String filePath; + + @JsonProperty("map_when") + private String mapWhen; + + @JsonProperty("regex") + private RegexParameterConfiguration regexParameterConfiguration; + + + public String getSource() { return source; } + + public String getTarget() { return target; } + + public Map getMap() { return map; } + + public String getFilePath() { return filePath; } + + public String getMapWhen() { return mapWhen; } + + public RegexParameterConfiguration getRegexParameterConfiguration(){ return regexParameterConfiguration; } + + + @AssertTrue(message = "Either of map / patterns / file_path options need to be configured. (pattern option is mandatory while configuring regex option)") + public boolean hasMappings() { + return (Stream.of(map, filePath, regexParameterConfiguration).filter(n -> n!=null).count() != 0) && checkPatternUnderRegex(); + } + + public boolean checkPatternUnderRegex(){ + if(regexParameterConfiguration!=null && regexParameterConfiguration.getPatterns()==null){ + return false; + } + return true; + } + +} diff --git a/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/RegexParameterConfigurationTest.java b/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/RegexParameterConfigurationTest.java new file mode 100644 index 0000000000..a20b8ff98a --- /dev/null +++ b/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/RegexParameterConfigurationTest.java @@ -0,0 +1,46 @@ +package org.opensearch.dataprepper.plugins.processor.translate; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.opensearch.dataprepper.test.helper.ReflectivelySetField.setField; +import static org.hamcrest.CoreMatchers.is; + +class RegexParameterConfigurationTest { + + private RegexParameterConfiguration regexParameterConfiguration; + + @BeforeEach + void setup(){ + regexParameterConfiguration = createObjectUnderTest(); + } + + @Test + public void test_get_patterns() throws NoSuchFieldException, IllegalAccessException{ + final Map patternMap = Collections.singletonMap("key1", "val1"); + setField(RegexParameterConfiguration.class, regexParameterConfiguration, "patterns", patternMap); + assertThat(regexParameterConfiguration.getPatterns(), is(patternMap)); + } + + @Test + void test_get_exact() throws NoSuchFieldException, IllegalAccessException{ + setField(RegexParameterConfiguration.class, regexParameterConfiguration, "exact", false); + assertFalse(regexParameterConfiguration.getExact()); + } + + @Test + void test_default_exact_option(){ + assertTrue(regexParameterConfiguration.getExact()); + } + + private RegexParameterConfiguration createObjectUnderTest() { + return new RegexParameterConfiguration(); + } + +} \ No newline at end of file diff --git a/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfigTest.java b/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfigTest.java new file mode 100644 index 0000000000..32534a44b7 --- /dev/null +++ b/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfigTest.java @@ -0,0 +1,59 @@ +package org.opensearch.dataprepper.plugins.processor.translate; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.opensearch.dataprepper.test.helper.ReflectivelySetField.setField; + + +class TranslateProcessorConfigTest { + private TranslateProcessorConfig translateProcessorConfig; + private RegexParameterConfiguration regexParameterConfiguration; + private TranslateProcessorConfig createObjectUnderTest() { + return new TranslateProcessorConfig(); + } + + @BeforeEach + void setup() throws NoSuchFieldException, IllegalAccessException{ + translateProcessorConfig = createObjectUnderTest(); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "source", "sourceKey"); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "target", "targetKey"); + } + + @Test + void test_no_map_patterns_filepath_options_present(){ + assertFalse(translateProcessorConfig.hasMappings()); + } + + @Test + void test_only_map_option_present() throws NoSuchFieldException, IllegalAccessException{ + setField(TranslateProcessorConfig.class, translateProcessorConfig, "map", Collections.singletonMap("key1", "val1")); + assertTrue(translateProcessorConfig.hasMappings()); + } + + @Test + void test_only_filepath_option_present() throws NoSuchFieldException, IllegalAccessException{ + setField(TranslateProcessorConfig.class, translateProcessorConfig, "filePath", "/path/to/file.yaml"); + assertTrue(translateProcessorConfig.hasMappings()); + } + + @Test + void test_only_patterns_option_present() throws NoSuchFieldException, IllegalAccessException{ + regexParameterConfiguration = new RegexParameterConfiguration(); + setField(RegexParameterConfiguration.class, regexParameterConfiguration, "patterns", Collections.singletonMap("patternKey1", "patternVal1")); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "regexParameterConfiguration", regexParameterConfiguration); + assertTrue(translateProcessorConfig.hasMappings()); + } + + @Test + void test_no_patterns_under_regex() throws NoSuchFieldException, IllegalAccessException{ + regexParameterConfiguration = new RegexParameterConfiguration(); + setField(RegexParameterConfiguration.class, regexParameterConfiguration, "exact", true); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "map", Collections.singletonMap("key1", "val1")); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "regexParameterConfiguration", regexParameterConfiguration); + assertFalse(translateProcessorConfig.hasMappings()); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorTest.java b/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorTest.java new file mode 100644 index 0000000000..52688cfe8d --- /dev/null +++ b/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorTest.java @@ -0,0 +1,300 @@ +package org.opensearch.dataprepper.plugins.processor.translate; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; +import org.opensearch.dataprepper.model.record.Record; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Collections; +import java.util.AbstractMap; +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.when; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.CoreMatchers.is; + +@ExtendWith(MockitoExtension.class) +class TranslateProcessorTest { + + @Mock + private PluginMetrics pluginMetrics; + @Mock + private TranslateProcessorConfig mockConfig; + + @Mock + private RegexParameterConfiguration mockRegexConfig; + + @Mock + private ExpressionEvaluator expressionEvaluator; + + @BeforeEach + void setup(){ + lenient().when(mockConfig.getSource()).thenReturn("sourceField"); + lenient().when(mockConfig.getTarget()).thenReturn("targetField"); + } + + @Test + public void test_string_keys_in_map(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("key1","mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("key1"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + } + + @Test + public void test_integer_keys_in_map(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("123","mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("123"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + } + + @Test + public void test_integer_range_keys_in_map(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("1-10","mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("5"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + + } + + @Test + public void test_comma_separated_keys_in_map(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("key1,key2, key3","mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + + for(String key : Arrays.asList("key1","key2","key3")){ + final Record record = getEvent(key); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + } + + final Record failureRecord = getEvent("key4"); + final List> failingTranslatedRecords = (List>) processor.doExecute(Collections.singletonList(failureRecord)); + + assertFalse(failingTranslatedRecords.get(0).getData().containsKey("targetField")); + } + + @Test + public void test_comma_separated_range_keys_in_map(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("1-10,11-20, 21-30","mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + + for(String key : Arrays.asList("5","15","25")){ + final Record record = getEvent(key); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + } + + final Record failureRecord = getEvent("35"); + final List> failingTranslatedRecords = (List>) processor.doExecute(Collections.singletonList(failureRecord)); + + assertFalse(failingTranslatedRecords.get(0).getData().containsKey("targetField")); + } + + @Test + public void test_float_source(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("1-10,11-20, 21-30","mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("11.1"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + + final Record failureRecord = getEvent("20.5"); + final List> failingTranslatedRecords = (List>) processor.doExecute(Collections.singletonList(failureRecord)); + + assertFalse(failingTranslatedRecords.get(0).getData().containsKey("targetField")); + } + + @Test + public void test_comma_separated_integer_ranges_and_string_keys(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("1-10,key1","mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("5.2"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + + final Record recordStringKey = getEvent("key1"); + final List> translatedStringKeyRecords = (List>) processor.doExecute(Collections.singletonList(recordStringKey)); + + assertTrue(translatedStringKeyRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedStringKeyRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + } + + @Test + public void test_multiple_dashes_in_keys_should_be_treated_as_string_literal(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("1-10-20","mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + final Record failureRecord = getEvent("1-10-20"); + final List> failingTranslatedRecords = (List>) processor.doExecute(Collections.singletonList(failureRecord)); + + assertTrue(failingTranslatedRecords.get(0).getData().containsKey("targetField")); + + final Record record = getEvent("10"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertFalse(translatedRecords.get(0).getData().containsKey("targetField")); + + } + + @Test + public void test_overlapping_ranges_should_fail_when_overlapping(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("1-10","mappedValue1"), createMapping("10-20", "mappedValue2"))); + + assertThrows(InvalidPluginConfigurationException.class,() -> createObjectUnderTest()); + } + + @Test + public void test_overlapping_key_and_range_in_map_option(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("1-10","mappedValue1"), createMapping("5.3", "mappedValue2"))); + + assertThrows(InvalidPluginConfigurationException.class,() -> createObjectUnderTest()); + } + + @Test + public void test_string_literal_in_pattern_option(){ + when(mockConfig.getRegexParameterConfiguration()).thenReturn(mockRegexConfig); + when(mockRegexConfig.getPatterns()).thenReturn(createMapEntries(createMapping("key1","mappedValue1"))); + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("key1"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + + final Record failureRecord = getEvent("key2"); + final List> failingTranslatedRecords = (List>) processor.doExecute(Collections.singletonList(failureRecord)); + + assertFalse(failingTranslatedRecords.get(0).getData().containsKey("targetField")); + } + + @Test + public void test_matching_of_regex_pattern_in_pattern_option(){ + when(mockConfig.getRegexParameterConfiguration()).thenReturn(mockRegexConfig); + when(mockRegexConfig.getPatterns()).thenReturn(createMapEntries(createMapping("^(1[0-9]|20)$", "patternValue1"))); //Range between 10-20 + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("15"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("patternValue1")); + + final Record failureRecord = getEvent("1"); + final List> failingTranslatedRecords = (List>) processor.doExecute(Collections.singletonList(failureRecord)); + + assertFalse(failingTranslatedRecords.get(0).getData().containsKey("targetField")); + } + + @Test + public void test_pattern_matching_when_no_match_in_map(){ + when(mockConfig.getRegexParameterConfiguration()).thenReturn(mockRegexConfig); + when(mockConfig.getMap()).thenReturn((createMapEntries(createMapping("key1", "mappedValue1"), createMapping("key2", "mappedValue2")))); + when(mockRegexConfig.getPatterns()).thenReturn(createMapEntries(createMapping("patternKey1", "patternValue1"))); + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("patternKey1"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("patternValue1")); + + final Record recordMapKey = getEvent("key1"); + final List> translatedMapKeyRecords = (List>) processor.doExecute(Collections.singletonList(recordMapKey)); + + assertTrue(translatedMapKeyRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedMapKeyRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + } + + @Test + public void test_map_matching_when_overlapping_ranges_in_map_and_pattern(){ + when(mockConfig.getRegexParameterConfiguration()).thenReturn(mockRegexConfig); + when(mockConfig.getMap()).thenReturn((createMapEntries(createMapping("400", "mappedValue1")))); + when(mockRegexConfig.getPatterns()).thenReturn(createMapEntries(createMapping("^(400|404)$", "patternValue1"))); // Matches 400 or 404 + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("400"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + + final Record recordPatternKey = getEvent("404"); + final List> translatedPatternKeyRecords = (List>) processor.doExecute(Collections.singletonList(recordPatternKey)); + + assertTrue(translatedPatternKeyRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedPatternKeyRecords.get(0).getData().get("targetField", String.class), is("patternValue1")); + } + + + + + private TranslateProcessor createObjectUnderTest() { + return new TranslateProcessor(pluginMetrics, mockConfig, expressionEvaluator); + } + + private Record sourceAndTargetFields(Object sourceValue, Object targetValue) { + final Map testData = new HashMap<>(); + testData.put("sourceField", sourceValue); + testData.put("targetField", targetValue); + return buildRecordWithEvent(testData); + } + private Record getEvent(Object sourceField) { + final Map testData = new HashMap<>(); + testData.put("sourceField", sourceField); + return buildRecordWithEvent(testData); + } + + private static Record buildRecordWithEvent(final Map data) { + return new Record<>(JacksonEvent.builder() + .withData(data) + .withEventType("event") + .build()); + } + + public Map.Entry createMapping(String key, String value){ + return new AbstractMap.SimpleEntry<>(key, value); + } + + public Map createMapEntries(Map.Entry... mappings){ + final Map finalMap = new HashMap<>(); + for(Map.Entry mapping : mappings){ + finalMap.put(mapping.getKey(), mapping.getValue()); + } + + return finalMap; + } +} \ No newline at end of file diff --git a/docs/simple_pipelines.md b/docs/simple_pipelines.md index b05953e575..0f6aea94aa 100644 --- a/docs/simple_pipelines.md +++ b/docs/simple_pipelines.md @@ -40,7 +40,7 @@ built from source, you will need to make some modifications to the example comma For Data Prepper 2.0 or above, use this command: ``` -docker run --name data-prepper -p 4900:4900 -v ${PWD}/pipelines.yaml:/usr/share/data-prepper/pipelines/pipelines.yaml -v ${PWD} /data-prepper-config.yaml:/usr/share/data-prepper/config/data-prepper-config.yaml opensearchproject/data-prepper:latest +docker run --name data-prepper -p 4900:4900 -v ${PWD}/pipelines.yaml:/usr/share/data-prepper/pipelines/pipelines.yaml -v ${PWD}/data-prepper-config.yaml:/usr/share/data-prepper/config/data-prepper-config.yaml opensearchproject/data-prepper:latest ``` For Data Prepper before version 2.0, use this command: diff --git a/release/staging-resources-cdk/package-lock.json b/release/staging-resources-cdk/package-lock.json index a6fdae5930..8b77d7582f 100644 --- a/release/staging-resources-cdk/package-lock.json +++ b/release/staging-resources-cdk/package-lock.json @@ -8,7 +8,7 @@ "name": "staging-resources-cdk", "version": "0.1.0", "dependencies": { - "aws-cdk-lib": "2.13.0", + "aws-cdk-lib": "2.80.0", "constructs": "^10.0.0", "source-map-support": "^0.5.16" }, @@ -41,43 +41,58 @@ "node": ">=6.0.0" } }, + "node_modules/@aws-cdk/asset-awscli-v1": { + "version": "2.2.199", + "resolved": "https://registry.npmjs.org/@aws-cdk/asset-awscli-v1/-/asset-awscli-v1-2.2.199.tgz", + "integrity": "sha512-zNdD2OxALdsdQaRZBpTfMTuudxV+4jLMznJIvVj6O+OqCru4m5UtgVQmyApW1z2H9s4/06ovVt20aXe2G8Ta+w==" + }, + "node_modules/@aws-cdk/asset-kubectl-v20": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/@aws-cdk/asset-kubectl-v20/-/asset-kubectl-v20-2.1.2.tgz", + "integrity": "sha512-3M2tELJOxQv0apCIiuKQ4pAbncz9GuLwnKFqxifWfe77wuMxyTRPmxssYHs42ePqzap1LT6GDcPygGs+hHstLg==" + }, + "node_modules/@aws-cdk/asset-node-proxy-agent-v5": { + "version": "2.0.165", + "resolved": "https://registry.npmjs.org/@aws-cdk/asset-node-proxy-agent-v5/-/asset-node-proxy-agent-v5-2.0.165.tgz", + "integrity": "sha512-bsyLQD/vqXQcc9RDmlM1XqiFNO/yewgVFXmkMcQkndJbmE/jgYkzewwYGrBlfL725hGLQipXq19+jwWwdsXQqg==" + }, "node_modules/@babel/code-frame": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.18.6.tgz", - "integrity": "sha512-TDCmlK5eOvH+eH7cdAFlNXeVJqWIQ7gW9tY1GJIpUtFb6CmjVyq2VM3u71bOyR8CRihcCgMUYoDNyLXao3+70Q==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.22.5.tgz", + "integrity": "sha512-Xmwn266vad+6DAqEB2A6V/CcZVp62BbwVmcOJc2RPuwih1kw02TjQvWVWlcKGbBPd+8/0V5DEkOcizRGYsspYQ==", "dev": true, "dependencies": { - "@babel/highlight": "^7.18.6" + "@babel/highlight": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/compat-data": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.21.0.tgz", - "integrity": "sha512-gMuZsmsgxk/ENC3O/fRw5QY8A9/uxQbbCEypnLIiYYc/qVJtEV7ouxC3EllIIwNzMqAQee5tanFabWsUOutS7g==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.22.5.tgz", + "integrity": "sha512-4Jc/YuIaYqKnDDz892kPIledykKg12Aw1PYX5i/TY28anJtacvM1Rrr8wbieB9GfEJwlzqT0hUEao0CxEebiDA==", "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/core": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.21.0.tgz", - "integrity": "sha512-PuxUbxcW6ZYe656yL3EAhpy7qXKq0DmYsrJLpbB8XrsCP9Nm+XCg9XFMb5vIDliPD7+U/+M+QJlH17XOcB7eXA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.22.5.tgz", + "integrity": "sha512-SBuTAjg91A3eKOvD+bPEz3LlhHZRNu1nFOVts9lzDJTXshHTjII0BAtDS3Y2DAkdZdDKWVZGVwkDfc4Clxn1dg==", "dev": true, "dependencies": { "@ampproject/remapping": "^2.2.0", - "@babel/code-frame": "^7.18.6", - "@babel/generator": "^7.21.0", - "@babel/helper-compilation-targets": "^7.20.7", - "@babel/helper-module-transforms": "^7.21.0", - "@babel/helpers": "^7.21.0", - "@babel/parser": "^7.21.0", - "@babel/template": "^7.20.7", - "@babel/traverse": "^7.21.0", - "@babel/types": "^7.21.0", + "@babel/code-frame": "^7.22.5", + "@babel/generator": "^7.22.5", + "@babel/helper-compilation-targets": "^7.22.5", + "@babel/helper-module-transforms": "^7.22.5", + "@babel/helpers": "^7.22.5", + "@babel/parser": "^7.22.5", + "@babel/template": "^7.22.5", + "@babel/traverse": "^7.22.5", + "@babel/types": "^7.22.5", "convert-source-map": "^1.7.0", "debug": "^4.1.0", "gensync": "^1.0.0-beta.2", @@ -102,12 +117,12 @@ } }, "node_modules/@babel/generator": { - "version": "7.21.1", - "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.21.1.tgz", - "integrity": "sha512-1lT45bAYlQhFn/BHivJs43AiW2rg3/UbLyShGfF3C0KmHvO5fSghWd5kBJy30kpRRucGzXStvnnCFniCR2kXAA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.22.5.tgz", + "integrity": "sha512-+lcUbnTRhd0jOewtFSedLyiPsD5tswKkbgcezOqqWFUVNEwoUTlpPOBmvhG7OXWLR4jMdv0czPGH5XbflnD1EA==", "dev": true, "dependencies": { - "@babel/types": "^7.21.0", + "@babel/types": "^7.22.5", "@jridgewell/gen-mapping": "^0.3.2", "@jridgewell/trace-mapping": "^0.3.17", "jsesc": "^2.5.1" @@ -131,13 +146,13 @@ } }, "node_modules/@babel/helper-compilation-targets": { - "version": "7.20.7", - "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.20.7.tgz", - "integrity": "sha512-4tGORmfQcrc+bvrjb5y3dG9Mx1IOZjsHqQVUz7XCNHO+iTmqxWnVg3KRygjGmpRLJGdQSKuvFinbIb0CnZwHAQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.22.5.tgz", + "integrity": "sha512-Ji+ywpHeuqxB8WDxraCiqR0xfhYjiDE/e6k7FuIaANnoOFxAHskHChz4vA1mJC9Lbm01s1PVAGhQY4FUKSkGZw==", "dev": true, "dependencies": { - "@babel/compat-data": "^7.20.5", - "@babel/helper-validator-option": "^7.18.6", + "@babel/compat-data": "^7.22.5", + "@babel/helper-validator-option": "^7.22.5", "browserslist": "^4.21.3", "lru-cache": "^5.1.1", "semver": "^6.3.0" @@ -159,65 +174,65 @@ } }, "node_modules/@babel/helper-environment-visitor": { - "version": "7.18.9", - "resolved": "https://registry.npmjs.org/@babel/helper-environment-visitor/-/helper-environment-visitor-7.18.9.tgz", - "integrity": "sha512-3r/aACDJ3fhQ/EVgFy0hpj8oHyHpQc+LPtJoY9SzTThAsStm4Ptegq92vqKoE3vD706ZVFWITnMnxucw+S9Ipg==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-environment-visitor/-/helper-environment-visitor-7.22.5.tgz", + "integrity": "sha512-XGmhECfVA/5sAt+H+xpSg0mfrHq6FzNr9Oxh7PSEBBRUb/mL7Kz3NICXb194rCqAEdxkhPT1a88teizAFyvk8Q==", "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-function-name": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/helper-function-name/-/helper-function-name-7.21.0.tgz", - "integrity": "sha512-HfK1aMRanKHpxemaY2gqBmL04iAPOPRj7DxtNbiDOrJK+gdwkiNRVpCpUJYbUT+aZyemKN8brqTOxzCaG6ExRg==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-function-name/-/helper-function-name-7.22.5.tgz", + "integrity": "sha512-wtHSq6jMRE3uF2otvfuD3DIvVhOsSNshQl0Qrd7qC9oQJzHvOL4qQXlQn2916+CXGywIjpGuIkoyZRRxHPiNQQ==", "dev": true, "dependencies": { - "@babel/template": "^7.20.7", - "@babel/types": "^7.21.0" + "@babel/template": "^7.22.5", + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-hoist-variables": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/helper-hoist-variables/-/helper-hoist-variables-7.18.6.tgz", - "integrity": "sha512-UlJQPkFqFULIcyW5sbzgbkxn2FKRgwWiRexcuaR8RNJRy8+LLveqPjwZV/bwrLZCN0eUHD/x8D0heK1ozuoo6Q==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-hoist-variables/-/helper-hoist-variables-7.22.5.tgz", + "integrity": "sha512-wGjk9QZVzvknA6yKIUURb8zY3grXCcOZt+/7Wcy8O2uctxhplmUPkOdlgoNhmdVee2c92JXbf1xpMtVNbfoxRw==", "dev": true, "dependencies": { - "@babel/types": "^7.18.6" + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-module-imports": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.18.6.tgz", - "integrity": "sha512-0NFvs3VkuSYbFi1x2Vd6tKrywq+z/cLeYC/RJNFrIX/30Bf5aiGYbtvGXolEktzJH8o5E5KJ3tT+nkxuuZFVlA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.22.5.tgz", + "integrity": "sha512-8Dl6+HD/cKifutF5qGd/8ZJi84QeAKh+CEe1sBzz8UayBBGg1dAIJrdHOcOM5b2MpzWL2yuotJTtGjETq0qjXg==", "dev": true, "dependencies": { - "@babel/types": "^7.18.6" + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-module-transforms": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.21.2.tgz", - "integrity": "sha512-79yj2AR4U/Oqq/WOV7Lx6hUjau1Zfo4cI+JLAVYeMV5XIlbOhmjEk5ulbTc9fMpmlojzZHkUUxAiK+UKn+hNQQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.22.5.tgz", + "integrity": "sha512-+hGKDt/Ze8GFExiVHno/2dvG5IdstpzCq0y4Qc9OJ25D4q3pKfiIP/4Vp3/JvhDkLKsDK2api3q3fpIgiIF5bw==", "dev": true, "dependencies": { - "@babel/helper-environment-visitor": "^7.18.9", - "@babel/helper-module-imports": "^7.18.6", - "@babel/helper-simple-access": "^7.20.2", - "@babel/helper-split-export-declaration": "^7.18.6", - "@babel/helper-validator-identifier": "^7.19.1", - "@babel/template": "^7.20.7", - "@babel/traverse": "^7.21.2", - "@babel/types": "^7.21.2" + "@babel/helper-environment-visitor": "^7.22.5", + "@babel/helper-module-imports": "^7.22.5", + "@babel/helper-simple-access": "^7.22.5", + "@babel/helper-split-export-declaration": "^7.22.5", + "@babel/helper-validator-identifier": "^7.22.5", + "@babel/template": "^7.22.5", + "@babel/traverse": "^7.22.5", + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" @@ -233,77 +248,77 @@ } }, "node_modules/@babel/helper-simple-access": { - "version": "7.20.2", - "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.20.2.tgz", - "integrity": "sha512-+0woI/WPq59IrqDYbVGfshjT5Dmk/nnbdpcF8SnMhhXObpTq2KNBdLFRFrkVdbDOyUmHBCxzm5FHV1rACIkIbA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.22.5.tgz", + "integrity": "sha512-n0H99E/K+Bika3++WNL17POvo4rKWZ7lZEp1Q+fStVbUi8nxPQEBOlTmCOxW/0JsS56SKKQ+ojAe2pHKJHN35w==", "dev": true, "dependencies": { - "@babel/types": "^7.20.2" + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-split-export-declaration": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.18.6.tgz", - "integrity": "sha512-bde1etTx6ZyTmobl9LLMMQsaizFVZrquTEHOqKeQESMKo4PlObf+8+JA25ZsIpZhT/WEd39+vOdLXAFG/nELpA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.22.5.tgz", + "integrity": "sha512-thqK5QFghPKWLhAV321lxF95yCg2K3Ob5yw+M3VHWfdia0IkPXUtoLH8x/6Fh486QUvzhb8YOWHChTVen2/PoQ==", "dev": true, "dependencies": { - "@babel/types": "^7.18.6" + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-string-parser": { - "version": "7.19.4", - "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.19.4.tgz", - "integrity": "sha512-nHtDoQcuqFmwYNYPz3Rah5ph2p8PFeFCsZk9A/48dPc/rGocJ5J3hAAZ7pb76VWX3fZKu+uEr/FhH5jLx7umrw==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.22.5.tgz", + "integrity": "sha512-mM4COjgZox8U+JcXQwPijIZLElkgEpO5rsERVDJTc2qfCDfERyob6k5WegS14SX18IIjv+XD+GrqNumY5JRCDw==", "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-validator-identifier": { - "version": "7.19.1", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.19.1.tgz", - "integrity": "sha512-awrNfaMtnHUr653GgGEs++LlAvW6w+DcPrOliSMXWCKo597CwL5Acf/wWdNkf/tfEQE3mjkeD1YOVZOUV/od1w==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.22.5.tgz", + "integrity": "sha512-aJXu+6lErq8ltp+JhkJUfk1MTGyuA4v7f3pA+BJ5HLfNC6nAQ0Cpi9uOquUj8Hehg0aUiHzWQbOVJGao6ztBAQ==", "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-validator-option": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.21.0.tgz", - "integrity": "sha512-rmL/B8/f0mKS2baE9ZpyTcTavvEuWhTTW8amjzXNvYG4AwBsqTLikfXsEofsJEfKHf+HQVQbFOHy6o+4cnC/fQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.22.5.tgz", + "integrity": "sha512-R3oB6xlIVKUnxNUxbmgq7pKjxpru24zlimpE8WK47fACIlM0II/Hm1RS8IaOI7NgCr6LNS+jl5l75m20npAziw==", "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helpers": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.21.0.tgz", - "integrity": "sha512-XXve0CBtOW0pd7MRzzmoyuSj0e3SEzj8pgyFxnTT1NJZL38BD1MK7yYrm8yefRPIDvNNe14xR4FdbHwpInD4rA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.22.5.tgz", + "integrity": "sha512-pSXRmfE1vzcUIDFQcSGA5Mr+GxBV9oiRKDuDxXvWQQBCh8HoIjs/2DlDB7H8smac1IVrB9/xdXj2N3Wol9Cr+Q==", "dev": true, "dependencies": { - "@babel/template": "^7.20.7", - "@babel/traverse": "^7.21.0", - "@babel/types": "^7.21.0" + "@babel/template": "^7.22.5", + "@babel/traverse": "^7.22.5", + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/highlight": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.18.6.tgz", - "integrity": "sha512-u7stbOuYjaPezCuLj29hNW1v64M2Md2qupEKP1fHc7WdOA3DgLh37suiSrZYY7haUB7iBeQZ9P1uiRF359do3g==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.22.5.tgz", + "integrity": "sha512-BSKlD1hgnedS5XRnGOljZawtag7H1yPfQp0tdNJCHoH6AZ+Pcm9VvkrK59/Yy593Ypg0zMxH2BxD1VPYUQ7UIw==", "dev": true, "dependencies": { - "@babel/helper-validator-identifier": "^7.18.6", + "@babel/helper-validator-identifier": "^7.22.5", "chalk": "^2.0.0", "js-tokens": "^4.0.0" }, @@ -383,9 +398,9 @@ } }, "node_modules/@babel/parser": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.21.2.tgz", - "integrity": "sha512-URpaIJQwEkEC2T9Kn+Ai6Xe/02iNaVCuT/PtoRz3GPVJVDpPd7mLo+VddTbhCRU9TXqW5mSrQfXZyi8kDKOVpQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.22.5.tgz", + "integrity": "sha512-DFZMC9LJUG9PLOclRC32G63UXwzqS2koQC8dkx+PLdmt1xSePYpbT/NbsrJy8Q/muXz7o/h/d4A7Fuyixm559Q==", "dev": true, "bin": { "parser": "bin/babel-parser.js" @@ -542,33 +557,33 @@ } }, "node_modules/@babel/template": { - "version": "7.20.7", - "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.20.7.tgz", - "integrity": "sha512-8SegXApWe6VoNw0r9JHpSteLKTpTiLZ4rMlGIm9JQ18KiCtyQiAMEazujAHrUS5flrcqYZa75ukev3P6QmUwUw==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.22.5.tgz", + "integrity": "sha512-X7yV7eiwAxdj9k94NEylvbVHLiVG1nvzCV2EAowhxLTwODV1jl9UzZ48leOC0sH7OnuHrIkllaBgneUykIcZaw==", "dev": true, "dependencies": { - "@babel/code-frame": "^7.18.6", - "@babel/parser": "^7.20.7", - "@babel/types": "^7.20.7" + "@babel/code-frame": "^7.22.5", + "@babel/parser": "^7.22.5", + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/traverse": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.21.2.tgz", - "integrity": "sha512-ts5FFU/dSUPS13tv8XiEObDu9K+iagEKME9kAbaP7r0Y9KtZJZ+NGndDvWoRAYNpeWafbpFeki3q9QoMD6gxyw==", - "dev": true, - "dependencies": { - "@babel/code-frame": "^7.18.6", - "@babel/generator": "^7.21.1", - "@babel/helper-environment-visitor": "^7.18.9", - "@babel/helper-function-name": "^7.21.0", - "@babel/helper-hoist-variables": "^7.18.6", - "@babel/helper-split-export-declaration": "^7.18.6", - "@babel/parser": "^7.21.2", - "@babel/types": "^7.21.2", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.22.5.tgz", + "integrity": "sha512-7DuIjPgERaNo6r+PZwItpjCZEa5vyw4eJGufeLxrPdBXBoLcCJCIasvK6pK/9DVNrLZTLFhUGqaC6X/PA007TQ==", + "dev": true, + "dependencies": { + "@babel/code-frame": "^7.22.5", + "@babel/generator": "^7.22.5", + "@babel/helper-environment-visitor": "^7.22.5", + "@babel/helper-function-name": "^7.22.5", + "@babel/helper-hoist-variables": "^7.22.5", + "@babel/helper-split-export-declaration": "^7.22.5", + "@babel/parser": "^7.22.5", + "@babel/types": "^7.22.5", "debug": "^4.1.0", "globals": "^11.1.0" }, @@ -586,13 +601,13 @@ } }, "node_modules/@babel/types": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.21.2.tgz", - "integrity": "sha512-3wRZSs7jiFaB8AjxiiD+VqN5DTG2iRvJGQ+qYFrs/654lg6kGTQWIOFjlBo5RaXuAZjBmP3+OQH4dmhqiiyYxw==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.22.5.tgz", + "integrity": "sha512-zo3MIHGOkPOfoRXitsgHLjEXmlDaD/5KU1Uzuc9GNiZPhSqVxVRtxuPaSBZDsYZ9qV88AjtMtWW7ww98loJ9KA==", "dev": true, "dependencies": { - "@babel/helper-string-parser": "^7.19.4", - "@babel/helper-validator-identifier": "^7.19.1", + "@babel/helper-string-parser": "^7.22.5", + "@babel/helper-validator-identifier": "^7.22.5", "to-fast-properties": "^2.0.0" }, "engines": { @@ -1703,9 +1718,9 @@ } }, "node_modules/aws-cdk-lib": { - "version": "2.13.0", - "resolved": "https://registry.npmjs.org/aws-cdk-lib/-/aws-cdk-lib-2.13.0.tgz", - "integrity": "sha512-nKpQk+9H7T128gpzl+7XTu+19Yzj6kmCMrvSwTXLa/qr4/soEpXI68/+19ymEAHOYEL4Dd3eyk490P+y0wzi6A==", + "version": "2.80.0", + "resolved": "https://registry.npmjs.org/aws-cdk-lib/-/aws-cdk-lib-2.80.0.tgz", + "integrity": "sha512-PoqD3Yms5I0ajuTi071nTW/hpkH3XsdyZzn5gYsPv0qD7mqP3h6Qr+6RiGx+yQ1KcVFyxWdX15uK+DsC0KwvcQ==", "bundleDependencies": [ "@balena/dockerignore", "case", @@ -1715,17 +1730,22 @@ "minimatch", "punycode", "semver", + "table", "yaml" ], "dependencies": { + "@aws-cdk/asset-awscli-v1": "^2.2.177", + "@aws-cdk/asset-kubectl-v20": "^2.1.1", + "@aws-cdk/asset-node-proxy-agent-v5": "^2.0.148", "@balena/dockerignore": "^1.0.2", "case": "1.6.3", - "fs-extra": "^9.1.0", - "ignore": "^5.2.0", - "jsonschema": "^1.4.0", + "fs-extra": "^11.1.1", + "ignore": "^5.2.4", + "jsonschema": "^1.4.1", "minimatch": "^3.1.2", - "punycode": "^2.1.1", - "semver": "^7.3.5", + "punycode": "^2.3.0", + "semver": "^7.5.1", + "table": "^6.8.1", "yaml": "1.10.2" }, "engines": { @@ -1740,12 +1760,49 @@ "inBundle": true, "license": "Apache-2.0" }, - "node_modules/aws-cdk-lib/node_modules/at-least-node": { - "version": "1.0.0", + "node_modules/aws-cdk-lib/node_modules/ajv": { + "version": "8.12.0", "inBundle": true, - "license": "ISC", + "license": "MIT", + "dependencies": { + "fast-deep-equal": "^3.1.1", + "json-schema-traverse": "^1.0.0", + "require-from-string": "^2.0.2", + "uri-js": "^4.2.2" + }, + "funding": { + "type": "github", + "url": "https://github.com/sponsors/epoberezkin" + } + }, + "node_modules/aws-cdk-lib/node_modules/ansi-regex": { + "version": "5.0.1", + "inBundle": true, + "license": "MIT", "engines": { - "node": ">= 4.0.0" + "node": ">=8" + } + }, + "node_modules/aws-cdk-lib/node_modules/ansi-styles": { + "version": "4.3.0", + "inBundle": true, + "license": "MIT", + "dependencies": { + "color-convert": "^2.0.1" + }, + "engines": { + "node": ">=8" + }, + "funding": { + "url": "https://github.com/chalk/ansi-styles?sponsor=1" + } + }, + "node_modules/aws-cdk-lib/node_modules/astral-regex": { + "version": "2.0.0", + "inBundle": true, + "license": "MIT", + "engines": { + "node": ">=8" } }, "node_modules/aws-cdk-lib/node_modules/balanced-match": { @@ -1770,38 +1827,76 @@ "node": ">= 0.8.0" } }, + "node_modules/aws-cdk-lib/node_modules/color-convert": { + "version": "2.0.1", + "inBundle": true, + "license": "MIT", + "dependencies": { + "color-name": "~1.1.4" + }, + "engines": { + "node": ">=7.0.0" + } + }, + "node_modules/aws-cdk-lib/node_modules/color-name": { + "version": "1.1.4", + "inBundle": true, + "license": "MIT" + }, "node_modules/aws-cdk-lib/node_modules/concat-map": { "version": "0.0.1", "inBundle": true, "license": "MIT" }, + "node_modules/aws-cdk-lib/node_modules/emoji-regex": { + "version": "8.0.0", + "inBundle": true, + "license": "MIT" + }, + "node_modules/aws-cdk-lib/node_modules/fast-deep-equal": { + "version": "3.1.3", + "inBundle": true, + "license": "MIT" + }, "node_modules/aws-cdk-lib/node_modules/fs-extra": { - "version": "9.1.0", + "version": "11.1.1", "inBundle": true, "license": "MIT", "dependencies": { - "at-least-node": "^1.0.0", "graceful-fs": "^4.2.0", "jsonfile": "^6.0.1", "universalify": "^2.0.0" }, "engines": { - "node": ">=10" + "node": ">=14.14" } }, "node_modules/aws-cdk-lib/node_modules/graceful-fs": { - "version": "4.2.9", + "version": "4.2.11", "inBundle": true, "license": "ISC" }, "node_modules/aws-cdk-lib/node_modules/ignore": { - "version": "5.2.0", + "version": "5.2.4", "inBundle": true, "license": "MIT", "engines": { "node": ">= 4" } }, + "node_modules/aws-cdk-lib/node_modules/is-fullwidth-code-point": { + "version": "3.0.0", + "inBundle": true, + "license": "MIT", + "engines": { + "node": ">=8" + } + }, + "node_modules/aws-cdk-lib/node_modules/json-schema-traverse": { + "version": "1.0.0", + "inBundle": true, + "license": "MIT" + }, "node_modules/aws-cdk-lib/node_modules/jsonfile": { "version": "6.1.0", "inBundle": true, @@ -1814,13 +1909,18 @@ } }, "node_modules/aws-cdk-lib/node_modules/jsonschema": { - "version": "1.4.0", + "version": "1.4.1", "inBundle": true, "license": "MIT", "engines": { "node": "*" } }, + "node_modules/aws-cdk-lib/node_modules/lodash.truncate": { + "version": "4.4.2", + "inBundle": true, + "license": "MIT" + }, "node_modules/aws-cdk-lib/node_modules/lru-cache": { "version": "6.0.0", "inBundle": true, @@ -1844,15 +1944,23 @@ } }, "node_modules/aws-cdk-lib/node_modules/punycode": { - "version": "2.1.1", + "version": "2.3.0", "inBundle": true, "license": "MIT", "engines": { "node": ">=6" } }, + "node_modules/aws-cdk-lib/node_modules/require-from-string": { + "version": "2.0.2", + "inBundle": true, + "license": "MIT", + "engines": { + "node": ">=0.10.0" + } + }, "node_modules/aws-cdk-lib/node_modules/semver": { - "version": "7.3.5", + "version": "7.5.1", "inBundle": true, "license": "ISC", "dependencies": { @@ -1865,6 +1973,61 @@ "node": ">=10" } }, + "node_modules/aws-cdk-lib/node_modules/slice-ansi": { + "version": "4.0.0", + "inBundle": true, + "license": "MIT", + "dependencies": { + "ansi-styles": "^4.0.0", + "astral-regex": "^2.0.0", + "is-fullwidth-code-point": "^3.0.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/chalk/slice-ansi?sponsor=1" + } + }, + "node_modules/aws-cdk-lib/node_modules/string-width": { + "version": "4.2.3", + "inBundle": true, + "license": "MIT", + "dependencies": { + "emoji-regex": "^8.0.0", + "is-fullwidth-code-point": "^3.0.0", + "strip-ansi": "^6.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/aws-cdk-lib/node_modules/strip-ansi": { + "version": "6.0.1", + "inBundle": true, + "license": "MIT", + "dependencies": { + "ansi-regex": "^5.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/aws-cdk-lib/node_modules/table": { + "version": "6.8.1", + "inBundle": true, + "license": "BSD-3-Clause", + "dependencies": { + "ajv": "^8.0.1", + "lodash.truncate": "^4.4.2", + "slice-ansi": "^4.0.0", + "string-width": "^4.2.3", + "strip-ansi": "^6.0.1" + }, + "engines": { + "node": ">=10.0.0" + } + }, "node_modules/aws-cdk-lib/node_modules/universalify": { "version": "2.0.0", "inBundle": true, @@ -1873,6 +2036,14 @@ "node": ">= 10.0.0" } }, + "node_modules/aws-cdk-lib/node_modules/uri-js": { + "version": "4.4.1", + "inBundle": true, + "license": "BSD-2-Clause", + "dependencies": { + "punycode": "^2.1.0" + } + }, "node_modules/aws-cdk-lib/node_modules/yallist": { "version": "4.0.0", "inBundle": true, @@ -2006,7 +2177,8 @@ "node_modules/balanced-match": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.2.tgz", - "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==" + "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==", + "dev": true }, "node_modules/base": { "version": "0.11.2", @@ -2042,6 +2214,7 @@ "version": "1.1.11", "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.11.tgz", "integrity": "sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA==", + "dev": true, "dependencies": { "balanced-match": "^1.0.0", "concat-map": "0.0.1" @@ -2399,7 +2572,8 @@ "node_modules/concat-map": { "version": "0.0.1", "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", - "integrity": "sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==" + "integrity": "sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==", + "dev": true }, "node_modules/constructs": { "version": "10.1.264", @@ -4977,6 +5151,7 @@ "version": "3.1.2", "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.1.2.tgz", "integrity": "sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw==", + "dev": true, "dependencies": { "brace-expansion": "^1.1.7" }, @@ -6243,9 +6418,9 @@ } }, "node_modules/semver": { - "version": "7.3.8", - "resolved": "https://registry.npmjs.org/semver/-/semver-7.3.8.tgz", - "integrity": "sha512-NB1ctGL5rlHrPJtFDVIVzTyQylMLu9N9VICA6HSFJo8MCGVTMW6gfpicwKmmK/dAjTOrqu5l63JJOpDSrAis3A==", + "version": "7.5.3", + "resolved": "https://registry.npmjs.org/semver/-/semver-7.5.3.tgz", + "integrity": "sha512-QBlUtyVk/5EeHbi7X0fw6liDZc7BBmEaSYn01fMU1OUYbf6GPsbTtd8WmnqbI20SeycoHSeiybkE/q1Q+qlThQ==", "dependencies": { "lru-cache": "^6.0.0" }, @@ -7641,37 +7816,52 @@ "@jridgewell/trace-mapping": "^0.3.9" } }, + "@aws-cdk/asset-awscli-v1": { + "version": "2.2.199", + "resolved": "https://registry.npmjs.org/@aws-cdk/asset-awscli-v1/-/asset-awscli-v1-2.2.199.tgz", + "integrity": "sha512-zNdD2OxALdsdQaRZBpTfMTuudxV+4jLMznJIvVj6O+OqCru4m5UtgVQmyApW1z2H9s4/06ovVt20aXe2G8Ta+w==" + }, + "@aws-cdk/asset-kubectl-v20": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/@aws-cdk/asset-kubectl-v20/-/asset-kubectl-v20-2.1.2.tgz", + "integrity": "sha512-3M2tELJOxQv0apCIiuKQ4pAbncz9GuLwnKFqxifWfe77wuMxyTRPmxssYHs42ePqzap1LT6GDcPygGs+hHstLg==" + }, + "@aws-cdk/asset-node-proxy-agent-v5": { + "version": "2.0.165", + "resolved": "https://registry.npmjs.org/@aws-cdk/asset-node-proxy-agent-v5/-/asset-node-proxy-agent-v5-2.0.165.tgz", + "integrity": "sha512-bsyLQD/vqXQcc9RDmlM1XqiFNO/yewgVFXmkMcQkndJbmE/jgYkzewwYGrBlfL725hGLQipXq19+jwWwdsXQqg==" + }, "@babel/code-frame": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.18.6.tgz", - "integrity": "sha512-TDCmlK5eOvH+eH7cdAFlNXeVJqWIQ7gW9tY1GJIpUtFb6CmjVyq2VM3u71bOyR8CRihcCgMUYoDNyLXao3+70Q==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.22.5.tgz", + "integrity": "sha512-Xmwn266vad+6DAqEB2A6V/CcZVp62BbwVmcOJc2RPuwih1kw02TjQvWVWlcKGbBPd+8/0V5DEkOcizRGYsspYQ==", "dev": true, "requires": { - "@babel/highlight": "^7.18.6" + "@babel/highlight": "^7.22.5" } }, "@babel/compat-data": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.21.0.tgz", - "integrity": "sha512-gMuZsmsgxk/ENC3O/fRw5QY8A9/uxQbbCEypnLIiYYc/qVJtEV7ouxC3EllIIwNzMqAQee5tanFabWsUOutS7g==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.22.5.tgz", + "integrity": "sha512-4Jc/YuIaYqKnDDz892kPIledykKg12Aw1PYX5i/TY28anJtacvM1Rrr8wbieB9GfEJwlzqT0hUEao0CxEebiDA==", "dev": true }, "@babel/core": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.21.0.tgz", - "integrity": "sha512-PuxUbxcW6ZYe656yL3EAhpy7qXKq0DmYsrJLpbB8XrsCP9Nm+XCg9XFMb5vIDliPD7+U/+M+QJlH17XOcB7eXA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.22.5.tgz", + "integrity": "sha512-SBuTAjg91A3eKOvD+bPEz3LlhHZRNu1nFOVts9lzDJTXshHTjII0BAtDS3Y2DAkdZdDKWVZGVwkDfc4Clxn1dg==", "dev": true, "requires": { "@ampproject/remapping": "^2.2.0", - "@babel/code-frame": "^7.18.6", - "@babel/generator": "^7.21.0", - "@babel/helper-compilation-targets": "^7.20.7", - "@babel/helper-module-transforms": "^7.21.0", - "@babel/helpers": "^7.21.0", - "@babel/parser": "^7.21.0", - "@babel/template": "^7.20.7", - "@babel/traverse": "^7.21.0", - "@babel/types": "^7.21.0", + "@babel/code-frame": "^7.22.5", + "@babel/generator": "^7.22.5", + "@babel/helper-compilation-targets": "^7.22.5", + "@babel/helper-module-transforms": "^7.22.5", + "@babel/helpers": "^7.22.5", + "@babel/parser": "^7.22.5", + "@babel/template": "^7.22.5", + "@babel/traverse": "^7.22.5", + "@babel/types": "^7.22.5", "convert-source-map": "^1.7.0", "debug": "^4.1.0", "gensync": "^1.0.0-beta.2", @@ -7688,12 +7878,12 @@ } }, "@babel/generator": { - "version": "7.21.1", - "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.21.1.tgz", - "integrity": "sha512-1lT45bAYlQhFn/BHivJs43AiW2rg3/UbLyShGfF3C0KmHvO5fSghWd5kBJy30kpRRucGzXStvnnCFniCR2kXAA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.22.5.tgz", + "integrity": "sha512-+lcUbnTRhd0jOewtFSedLyiPsD5tswKkbgcezOqqWFUVNEwoUTlpPOBmvhG7OXWLR4jMdv0czPGH5XbflnD1EA==", "dev": true, "requires": { - "@babel/types": "^7.21.0", + "@babel/types": "^7.22.5", "@jridgewell/gen-mapping": "^0.3.2", "@jridgewell/trace-mapping": "^0.3.17", "jsesc": "^2.5.1" @@ -7713,13 +7903,13 @@ } }, "@babel/helper-compilation-targets": { - "version": "7.20.7", - "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.20.7.tgz", - "integrity": "sha512-4tGORmfQcrc+bvrjb5y3dG9Mx1IOZjsHqQVUz7XCNHO+iTmqxWnVg3KRygjGmpRLJGdQSKuvFinbIb0CnZwHAQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.22.5.tgz", + "integrity": "sha512-Ji+ywpHeuqxB8WDxraCiqR0xfhYjiDE/e6k7FuIaANnoOFxAHskHChz4vA1mJC9Lbm01s1PVAGhQY4FUKSkGZw==", "dev": true, "requires": { - "@babel/compat-data": "^7.20.5", - "@babel/helper-validator-option": "^7.18.6", + "@babel/compat-data": "^7.22.5", + "@babel/helper-validator-option": "^7.22.5", "browserslist": "^4.21.3", "lru-cache": "^5.1.1", "semver": "^6.3.0" @@ -7734,53 +7924,53 @@ } }, "@babel/helper-environment-visitor": { - "version": "7.18.9", - "resolved": "https://registry.npmjs.org/@babel/helper-environment-visitor/-/helper-environment-visitor-7.18.9.tgz", - "integrity": "sha512-3r/aACDJ3fhQ/EVgFy0hpj8oHyHpQc+LPtJoY9SzTThAsStm4Ptegq92vqKoE3vD706ZVFWITnMnxucw+S9Ipg==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-environment-visitor/-/helper-environment-visitor-7.22.5.tgz", + "integrity": "sha512-XGmhECfVA/5sAt+H+xpSg0mfrHq6FzNr9Oxh7PSEBBRUb/mL7Kz3NICXb194rCqAEdxkhPT1a88teizAFyvk8Q==", "dev": true }, "@babel/helper-function-name": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/helper-function-name/-/helper-function-name-7.21.0.tgz", - "integrity": "sha512-HfK1aMRanKHpxemaY2gqBmL04iAPOPRj7DxtNbiDOrJK+gdwkiNRVpCpUJYbUT+aZyemKN8brqTOxzCaG6ExRg==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-function-name/-/helper-function-name-7.22.5.tgz", + "integrity": "sha512-wtHSq6jMRE3uF2otvfuD3DIvVhOsSNshQl0Qrd7qC9oQJzHvOL4qQXlQn2916+CXGywIjpGuIkoyZRRxHPiNQQ==", "dev": true, "requires": { - "@babel/template": "^7.20.7", - "@babel/types": "^7.21.0" + "@babel/template": "^7.22.5", + "@babel/types": "^7.22.5" } }, "@babel/helper-hoist-variables": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/helper-hoist-variables/-/helper-hoist-variables-7.18.6.tgz", - "integrity": "sha512-UlJQPkFqFULIcyW5sbzgbkxn2FKRgwWiRexcuaR8RNJRy8+LLveqPjwZV/bwrLZCN0eUHD/x8D0heK1ozuoo6Q==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-hoist-variables/-/helper-hoist-variables-7.22.5.tgz", + "integrity": "sha512-wGjk9QZVzvknA6yKIUURb8zY3grXCcOZt+/7Wcy8O2uctxhplmUPkOdlgoNhmdVee2c92JXbf1xpMtVNbfoxRw==", "dev": true, "requires": { - "@babel/types": "^7.18.6" + "@babel/types": "^7.22.5" } }, "@babel/helper-module-imports": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.18.6.tgz", - "integrity": "sha512-0NFvs3VkuSYbFi1x2Vd6tKrywq+z/cLeYC/RJNFrIX/30Bf5aiGYbtvGXolEktzJH8o5E5KJ3tT+nkxuuZFVlA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.22.5.tgz", + "integrity": "sha512-8Dl6+HD/cKifutF5qGd/8ZJi84QeAKh+CEe1sBzz8UayBBGg1dAIJrdHOcOM5b2MpzWL2yuotJTtGjETq0qjXg==", "dev": true, "requires": { - "@babel/types": "^7.18.6" + "@babel/types": "^7.22.5" } }, "@babel/helper-module-transforms": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.21.2.tgz", - "integrity": "sha512-79yj2AR4U/Oqq/WOV7Lx6hUjau1Zfo4cI+JLAVYeMV5XIlbOhmjEk5ulbTc9fMpmlojzZHkUUxAiK+UKn+hNQQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.22.5.tgz", + "integrity": "sha512-+hGKDt/Ze8GFExiVHno/2dvG5IdstpzCq0y4Qc9OJ25D4q3pKfiIP/4Vp3/JvhDkLKsDK2api3q3fpIgiIF5bw==", "dev": true, "requires": { - "@babel/helper-environment-visitor": "^7.18.9", - "@babel/helper-module-imports": "^7.18.6", - "@babel/helper-simple-access": "^7.20.2", - "@babel/helper-split-export-declaration": "^7.18.6", - "@babel/helper-validator-identifier": "^7.19.1", - "@babel/template": "^7.20.7", - "@babel/traverse": "^7.21.2", - "@babel/types": "^7.21.2" + "@babel/helper-environment-visitor": "^7.22.5", + "@babel/helper-module-imports": "^7.22.5", + "@babel/helper-simple-access": "^7.22.5", + "@babel/helper-split-export-declaration": "^7.22.5", + "@babel/helper-validator-identifier": "^7.22.5", + "@babel/template": "^7.22.5", + "@babel/traverse": "^7.22.5", + "@babel/types": "^7.22.5" } }, "@babel/helper-plugin-utils": { @@ -7790,59 +7980,59 @@ "dev": true }, "@babel/helper-simple-access": { - "version": "7.20.2", - "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.20.2.tgz", - "integrity": "sha512-+0woI/WPq59IrqDYbVGfshjT5Dmk/nnbdpcF8SnMhhXObpTq2KNBdLFRFrkVdbDOyUmHBCxzm5FHV1rACIkIbA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.22.5.tgz", + "integrity": "sha512-n0H99E/K+Bika3++WNL17POvo4rKWZ7lZEp1Q+fStVbUi8nxPQEBOlTmCOxW/0JsS56SKKQ+ojAe2pHKJHN35w==", "dev": true, "requires": { - "@babel/types": "^7.20.2" + "@babel/types": "^7.22.5" } }, "@babel/helper-split-export-declaration": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.18.6.tgz", - "integrity": "sha512-bde1etTx6ZyTmobl9LLMMQsaizFVZrquTEHOqKeQESMKo4PlObf+8+JA25ZsIpZhT/WEd39+vOdLXAFG/nELpA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.22.5.tgz", + "integrity": "sha512-thqK5QFghPKWLhAV321lxF95yCg2K3Ob5yw+M3VHWfdia0IkPXUtoLH8x/6Fh486QUvzhb8YOWHChTVen2/PoQ==", "dev": true, "requires": { - "@babel/types": "^7.18.6" + "@babel/types": "^7.22.5" } }, "@babel/helper-string-parser": { - "version": "7.19.4", - "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.19.4.tgz", - "integrity": "sha512-nHtDoQcuqFmwYNYPz3Rah5ph2p8PFeFCsZk9A/48dPc/rGocJ5J3hAAZ7pb76VWX3fZKu+uEr/FhH5jLx7umrw==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.22.5.tgz", + "integrity": "sha512-mM4COjgZox8U+JcXQwPijIZLElkgEpO5rsERVDJTc2qfCDfERyob6k5WegS14SX18IIjv+XD+GrqNumY5JRCDw==", "dev": true }, "@babel/helper-validator-identifier": { - "version": "7.19.1", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.19.1.tgz", - "integrity": "sha512-awrNfaMtnHUr653GgGEs++LlAvW6w+DcPrOliSMXWCKo597CwL5Acf/wWdNkf/tfEQE3mjkeD1YOVZOUV/od1w==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.22.5.tgz", + "integrity": "sha512-aJXu+6lErq8ltp+JhkJUfk1MTGyuA4v7f3pA+BJ5HLfNC6nAQ0Cpi9uOquUj8Hehg0aUiHzWQbOVJGao6ztBAQ==", "dev": true }, "@babel/helper-validator-option": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.21.0.tgz", - "integrity": "sha512-rmL/B8/f0mKS2baE9ZpyTcTavvEuWhTTW8amjzXNvYG4AwBsqTLikfXsEofsJEfKHf+HQVQbFOHy6o+4cnC/fQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.22.5.tgz", + "integrity": "sha512-R3oB6xlIVKUnxNUxbmgq7pKjxpru24zlimpE8WK47fACIlM0II/Hm1RS8IaOI7NgCr6LNS+jl5l75m20npAziw==", "dev": true }, "@babel/helpers": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.21.0.tgz", - "integrity": "sha512-XXve0CBtOW0pd7MRzzmoyuSj0e3SEzj8pgyFxnTT1NJZL38BD1MK7yYrm8yefRPIDvNNe14xR4FdbHwpInD4rA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.22.5.tgz", + "integrity": "sha512-pSXRmfE1vzcUIDFQcSGA5Mr+GxBV9oiRKDuDxXvWQQBCh8HoIjs/2DlDB7H8smac1IVrB9/xdXj2N3Wol9Cr+Q==", "dev": true, "requires": { - "@babel/template": "^7.20.7", - "@babel/traverse": "^7.21.0", - "@babel/types": "^7.21.0" + "@babel/template": "^7.22.5", + "@babel/traverse": "^7.22.5", + "@babel/types": "^7.22.5" } }, "@babel/highlight": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.18.6.tgz", - "integrity": "sha512-u7stbOuYjaPezCuLj29hNW1v64M2Md2qupEKP1fHc7WdOA3DgLh37suiSrZYY7haUB7iBeQZ9P1uiRF359do3g==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.22.5.tgz", + "integrity": "sha512-BSKlD1hgnedS5XRnGOljZawtag7H1yPfQp0tdNJCHoH6AZ+Pcm9VvkrK59/Yy593Ypg0zMxH2BxD1VPYUQ7UIw==", "dev": true, "requires": { - "@babel/helper-validator-identifier": "^7.18.6", + "@babel/helper-validator-identifier": "^7.22.5", "chalk": "^2.0.0", "js-tokens": "^4.0.0" }, @@ -7906,9 +8096,9 @@ } }, "@babel/parser": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.21.2.tgz", - "integrity": "sha512-URpaIJQwEkEC2T9Kn+Ai6Xe/02iNaVCuT/PtoRz3GPVJVDpPd7mLo+VddTbhCRU9TXqW5mSrQfXZyi8kDKOVpQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.22.5.tgz", + "integrity": "sha512-DFZMC9LJUG9PLOclRC32G63UXwzqS2koQC8dkx+PLdmt1xSePYpbT/NbsrJy8Q/muXz7o/h/d4A7Fuyixm559Q==", "dev": true }, "@babel/plugin-syntax-async-generators": { @@ -8020,30 +8210,30 @@ } }, "@babel/template": { - "version": "7.20.7", - "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.20.7.tgz", - "integrity": "sha512-8SegXApWe6VoNw0r9JHpSteLKTpTiLZ4rMlGIm9JQ18KiCtyQiAMEazujAHrUS5flrcqYZa75ukev3P6QmUwUw==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.22.5.tgz", + "integrity": "sha512-X7yV7eiwAxdj9k94NEylvbVHLiVG1nvzCV2EAowhxLTwODV1jl9UzZ48leOC0sH7OnuHrIkllaBgneUykIcZaw==", "dev": true, "requires": { - "@babel/code-frame": "^7.18.6", - "@babel/parser": "^7.20.7", - "@babel/types": "^7.20.7" + "@babel/code-frame": "^7.22.5", + "@babel/parser": "^7.22.5", + "@babel/types": "^7.22.5" } }, "@babel/traverse": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.21.2.tgz", - "integrity": "sha512-ts5FFU/dSUPS13tv8XiEObDu9K+iagEKME9kAbaP7r0Y9KtZJZ+NGndDvWoRAYNpeWafbpFeki3q9QoMD6gxyw==", - "dev": true, - "requires": { - "@babel/code-frame": "^7.18.6", - "@babel/generator": "^7.21.1", - "@babel/helper-environment-visitor": "^7.18.9", - "@babel/helper-function-name": "^7.21.0", - "@babel/helper-hoist-variables": "^7.18.6", - "@babel/helper-split-export-declaration": "^7.18.6", - "@babel/parser": "^7.21.2", - "@babel/types": "^7.21.2", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.22.5.tgz", + "integrity": "sha512-7DuIjPgERaNo6r+PZwItpjCZEa5vyw4eJGufeLxrPdBXBoLcCJCIasvK6pK/9DVNrLZTLFhUGqaC6X/PA007TQ==", + "dev": true, + "requires": { + "@babel/code-frame": "^7.22.5", + "@babel/generator": "^7.22.5", + "@babel/helper-environment-visitor": "^7.22.5", + "@babel/helper-function-name": "^7.22.5", + "@babel/helper-hoist-variables": "^7.22.5", + "@babel/helper-split-export-declaration": "^7.22.5", + "@babel/parser": "^7.22.5", + "@babel/types": "^7.22.5", "debug": "^4.1.0", "globals": "^11.1.0" }, @@ -8057,13 +8247,13 @@ } }, "@babel/types": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.21.2.tgz", - "integrity": "sha512-3wRZSs7jiFaB8AjxiiD+VqN5DTG2iRvJGQ+qYFrs/654lg6kGTQWIOFjlBo5RaXuAZjBmP3+OQH4dmhqiiyYxw==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.22.5.tgz", + "integrity": "sha512-zo3MIHGOkPOfoRXitsgHLjEXmlDaD/5KU1Uzuc9GNiZPhSqVxVRtxuPaSBZDsYZ9qV88AjtMtWW7ww98loJ9KA==", "dev": true, "requires": { - "@babel/helper-string-parser": "^7.19.4", - "@babel/helper-validator-identifier": "^7.19.1", + "@babel/helper-string-parser": "^7.22.5", + "@babel/helper-validator-identifier": "^7.22.5", "to-fast-properties": "^2.0.0" } }, @@ -8902,18 +9092,22 @@ } }, "aws-cdk-lib": { - "version": "2.13.0", - "resolved": "https://registry.npmjs.org/aws-cdk-lib/-/aws-cdk-lib-2.13.0.tgz", - "integrity": "sha512-nKpQk+9H7T128gpzl+7XTu+19Yzj6kmCMrvSwTXLa/qr4/soEpXI68/+19ymEAHOYEL4Dd3eyk490P+y0wzi6A==", + "version": "2.80.0", + "resolved": "https://registry.npmjs.org/aws-cdk-lib/-/aws-cdk-lib-2.80.0.tgz", + "integrity": "sha512-PoqD3Yms5I0ajuTi071nTW/hpkH3XsdyZzn5gYsPv0qD7mqP3h6Qr+6RiGx+yQ1KcVFyxWdX15uK+DsC0KwvcQ==", "requires": { + "@aws-cdk/asset-awscli-v1": "^2.2.177", + "@aws-cdk/asset-kubectl-v20": "^2.1.1", + "@aws-cdk/asset-node-proxy-agent-v5": "^2.0.148", "@balena/dockerignore": "^1.0.2", "case": "1.6.3", - "fs-extra": "^9.1.0", - "ignore": "^5.2.0", - "jsonschema": "^1.4.0", + "fs-extra": "^11.1.1", + "ignore": "^5.2.4", + "jsonschema": "^1.4.1", "minimatch": "^3.1.2", - "punycode": "^2.1.1", - "semver": "^7.3.5", + "punycode": "^2.3.0", + "semver": "^7.5.1", + "table": "^6.8.1", "yaml": "1.10.2" }, "dependencies": { @@ -8921,8 +9115,29 @@ "version": "1.0.2", "bundled": true }, - "at-least-node": { - "version": "1.0.0", + "ajv": { + "version": "8.12.0", + "bundled": true, + "requires": { + "fast-deep-equal": "^3.1.1", + "json-schema-traverse": "^1.0.0", + "require-from-string": "^2.0.2", + "uri-js": "^4.2.2" + } + }, + "ansi-regex": { + "version": "5.0.1", + "bundled": true + }, + "ansi-styles": { + "version": "4.3.0", + "bundled": true, + "requires": { + "color-convert": "^2.0.1" + } + }, + "astral-regex": { + "version": "2.0.0", "bundled": true }, "balanced-match": { @@ -8941,26 +9156,52 @@ "version": "1.6.3", "bundled": true }, + "color-convert": { + "version": "2.0.1", + "bundled": true, + "requires": { + "color-name": "~1.1.4" + } + }, + "color-name": { + "version": "1.1.4", + "bundled": true + }, "concat-map": { "version": "0.0.1", "bundled": true }, + "emoji-regex": { + "version": "8.0.0", + "bundled": true + }, + "fast-deep-equal": { + "version": "3.1.3", + "bundled": true + }, "fs-extra": { - "version": "9.1.0", + "version": "11.1.1", "bundled": true, "requires": { - "at-least-node": "^1.0.0", "graceful-fs": "^4.2.0", "jsonfile": "^6.0.1", "universalify": "^2.0.0" } }, "graceful-fs": { - "version": "4.2.9", + "version": "4.2.11", "bundled": true }, "ignore": { - "version": "5.2.0", + "version": "5.2.4", + "bundled": true + }, + "is-fullwidth-code-point": { + "version": "3.0.0", + "bundled": true + }, + "json-schema-traverse": { + "version": "1.0.0", "bundled": true }, "jsonfile": { @@ -8972,7 +9213,11 @@ } }, "jsonschema": { - "version": "1.4.0", + "version": "1.4.1", + "bundled": true + }, + "lodash.truncate": { + "version": "4.4.2", "bundled": true }, "lru-cache": { @@ -8990,20 +9235,67 @@ } }, "punycode": { - "version": "2.1.1", + "version": "2.3.0", + "bundled": true + }, + "require-from-string": { + "version": "2.0.2", "bundled": true }, "semver": { - "version": "7.3.5", + "version": "7.5.1", "bundled": true, "requires": { "lru-cache": "^6.0.0" } }, + "slice-ansi": { + "version": "4.0.0", + "bundled": true, + "requires": { + "ansi-styles": "^4.0.0", + "astral-regex": "^2.0.0", + "is-fullwidth-code-point": "^3.0.0" + } + }, + "string-width": { + "version": "4.2.3", + "bundled": true, + "requires": { + "emoji-regex": "^8.0.0", + "is-fullwidth-code-point": "^3.0.0", + "strip-ansi": "^6.0.1" + } + }, + "strip-ansi": { + "version": "6.0.1", + "bundled": true, + "requires": { + "ansi-regex": "^5.0.1" + } + }, + "table": { + "version": "6.8.1", + "bundled": true, + "requires": { + "ajv": "^8.0.1", + "lodash.truncate": "^4.4.2", + "slice-ansi": "^4.0.0", + "string-width": "^4.2.3", + "strip-ansi": "^6.0.1" + } + }, "universalify": { "version": "2.0.0", "bundled": true }, + "uri-js": { + "version": "4.4.1", + "bundled": true, + "requires": { + "punycode": "^2.1.0" + } + }, "yallist": { "version": "4.0.0", "bundled": true @@ -9109,7 +9401,8 @@ "balanced-match": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.2.tgz", - "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==" + "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==", + "dev": true }, "base": { "version": "0.11.2", @@ -9141,6 +9434,7 @@ "version": "1.1.11", "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.11.tgz", "integrity": "sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA==", + "dev": true, "requires": { "balanced-match": "^1.0.0", "concat-map": "0.0.1" @@ -9414,7 +9708,8 @@ "concat-map": { "version": "0.0.1", "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", - "integrity": "sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==" + "integrity": "sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==", + "dev": true }, "constructs": { "version": "10.1.264", @@ -11399,6 +11694,7 @@ "version": "3.1.2", "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.1.2.tgz", "integrity": "sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw==", + "dev": true, "requires": { "brace-expansion": "^1.1.7" } @@ -12356,9 +12652,9 @@ } }, "semver": { - "version": "7.3.8", - "resolved": "https://registry.npmjs.org/semver/-/semver-7.3.8.tgz", - "integrity": "sha512-NB1ctGL5rlHrPJtFDVIVzTyQylMLu9N9VICA6HSFJo8MCGVTMW6gfpicwKmmK/dAjTOrqu5l63JJOpDSrAis3A==", + "version": "7.5.3", + "resolved": "https://registry.npmjs.org/semver/-/semver-7.5.3.tgz", + "integrity": "sha512-QBlUtyVk/5EeHbi7X0fw6liDZc7BBmEaSYn01fMU1OUYbf6GPsbTtd8WmnqbI20SeycoHSeiybkE/q1Q+qlThQ==", "requires": { "lru-cache": "^6.0.0" }, diff --git a/release/staging-resources-cdk/package.json b/release/staging-resources-cdk/package.json index 9b16af0014..b293856d63 100644 --- a/release/staging-resources-cdk/package.json +++ b/release/staging-resources-cdk/package.json @@ -25,7 +25,7 @@ "typescript": "~3.9.7" }, "dependencies": { - "aws-cdk-lib": "2.13.0", + "aws-cdk-lib": "2.80.0", "constructs": "^10.0.0", "source-map-support": "^0.5.16" } diff --git a/settings.gradle b/settings.gradle index 0760851409..d668b7d12c 100644 --- a/settings.gradle +++ b/settings.gradle @@ -31,10 +31,10 @@ dependencyResolutionManagement { version('opensearch', '1.3.8') library('opensearch-client', 'org.opensearch.client', 'opensearch-rest-client').versionRef('opensearch') library('opensearch-rhlc', 'org.opensearch.client', 'opensearch-rest-high-level-client').versionRef('opensearch') - version('spring', '5.3.27') + version('spring', '5.3.28') library('spring-core', 'org.springframework', 'spring-core').versionRef('spring') library('spring-context', 'org.springframework', 'spring-context').versionRef('spring') - version('guava', '31.1-jre') + version('guava', '32.0.1-jre') library('guava-core', 'com.google.guava', 'guava').versionRef('guava') } testLibs { @@ -42,7 +42,7 @@ dependencyResolutionManagement { version('mockito', '3.11.2') version('hamcrest', '2.2') version('awaitility', '4.2.0') - version('spring', '5.3.26') + version('spring', '5.3.28') version('slf4j', '2.0.6') library('junit-core', 'org.junit.jupiter', 'junit-jupiter').versionRef('junit') library('junit-params', 'org.junit.jupiter', 'junit-jupiter-params').versionRef('junit') @@ -103,6 +103,7 @@ include 'data-prepper-plugins:rss-source' include 'data-prepper-plugins:csv-processor' include 'data-prepper-plugins:parse-json-processor' include 'data-prepper-plugins:trace-peer-forwarder-processor' +include 'data-prepper-plugins:translate-processor' include 'data-prepper-plugins:dynamodb-source-coordination-store' include 'release' include 'release:archives'