From f4dc9dad5b6e5d0f6576ea4ae46fd2289c18d552 Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Wed, 21 Jun 2023 13:59:00 -0500 Subject: [PATCH 01/43] Elasticsearch client implementation with pit and no context search (#2910) Create Elasticsearch client, implement search and pit apis for ElasticsearchAccessor Signed-off-by: Taylor Gray Signed-off-by: Marcos Gonzalez Mayedo --- .../opensearch-source/build.gradle | 1 + .../source/opensearch/OpenSearchSource.java | 5 +- .../worker/NoSearchContextWorker.java | 2 +- ...nSearchIndexPartitionCreationSupplier.java | 41 ++- .../source/opensearch/worker/PitWorker.java | 12 +- .../worker/client/ElasticsearchAccessor.java | 156 +++++++++- .../client/OpenSearchClientFactory.java | 293 ++++++++++++++++++ .../worker/client/SearchAccessorStrategy.java | 265 ++++------------ .../opensearch/OpenSearchSourceTest.java | 8 +- .../client/ElasticsearchAccessorTest.java | 281 +++++++++++++++++ .../client/OpenSearchClientFactoryTest.java | 122 ++++++++ ...rchIndexPartitionCreationSupplierTest.java | 97 ++++++ .../client/SearchAccessStrategyTest.java | 232 ++++++-------- 13 files changed, 1138 insertions(+), 377 deletions(-) create mode 100644 data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactory.java create mode 100644 data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/ElasticsearchAccessorTest.java create mode 100644 data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactoryTest.java diff --git a/data-prepper-plugins/opensearch-source/build.gradle b/data-prepper-plugins/opensearch-source/build.gradle index b974a9a32b..22a02beeda 100644 --- a/data-prepper-plugins/opensearch-source/build.gradle +++ b/data-prepper-plugins/opensearch-source/build.gradle @@ -14,6 +14,7 @@ dependencies { testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' implementation 'org.opensearch.client:opensearch-java:2.5.0' implementation 'org.opensearch.client:opensearch-rest-client:2.7.0' + implementation 'co.elastic.clients:elasticsearch-java:7.17.0' implementation "org.apache.commons:commons-lang3:3.12.0" implementation('org.apache.maven:maven-artifact:3.0.3') { exclude group: 'org.codehaus.plexus' 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 779b8a3cbd..9a91db3ac1 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 @@ -13,6 +13,7 @@ import org.opensearch.dataprepper.model.source.coordinator.SourceCoordinator; import org.opensearch.dataprepper.model.source.coordinator.UsesSourceCoordination; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.OpenSearchClientFactory; import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.SearchAccessor; import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.SearchAccessorStrategy; @@ -41,7 +42,9 @@ public void start(final Buffer> buffer) { } private void startProcess(final OpenSearchSourceConfiguration openSearchSourceConfiguration, final Buffer> buffer) { - final SearchAccessorStrategy searchAccessorStrategy = SearchAccessorStrategy.create(openSearchSourceConfiguration, awsCredentialsSupplier); + + final OpenSearchClientFactory openSearchClientFactory = OpenSearchClientFactory.create(awsCredentialsSupplier); + final SearchAccessorStrategy searchAccessorStrategy = SearchAccessorStrategy.create(openSearchSourceConfiguration, openSearchClientFactory); final SearchAccessor searchAccessor = searchAccessorStrategy.getSearchAccessor(); diff --git a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/NoSearchContextWorker.java b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/NoSearchContextWorker.java index 352d39588e..f63f99552c 100644 --- a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/NoSearchContextWorker.java +++ b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/NoSearchContextWorker.java @@ -117,7 +117,7 @@ private void processIndex(final SourcePartition op } }); } catch (final Exception e) { - LOG.error("Received an exception while searching with PIT for index '{}'", indexName); + LOG.error("Received an exception while searching with no search context for index '{}'", indexName); throw new RuntimeException(e); } diff --git a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/OpenSearchIndexPartitionCreationSupplier.java b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/OpenSearchIndexPartitionCreationSupplier.java index e4ca82a3bc..94f72efa33 100644 --- a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/OpenSearchIndexPartitionCreationSupplier.java +++ b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/OpenSearchIndexPartitionCreationSupplier.java @@ -5,10 +5,11 @@ package org.opensearch.dataprepper.plugins.source.opensearch.worker; +import co.elastic.clients.elasticsearch.ElasticsearchClient; +import co.elastic.clients.elasticsearch._types.ElasticsearchException; import org.opensearch.client.opensearch.OpenSearchClient; import org.opensearch.client.opensearch._types.OpenSearchException; import org.opensearch.client.opensearch.cat.IndicesResponse; -import org.opensearch.client.opensearch.cat.indices.IndicesRecord; import org.opensearch.dataprepper.model.source.coordinator.PartitionIdentifier; import org.opensearch.dataprepper.plugins.source.opensearch.OpenSearchSourceConfiguration; import org.opensearch.dataprepper.plugins.source.opensearch.configuration.IndexParametersConfiguration; @@ -32,7 +33,9 @@ public class OpenSearchIndexPartitionCreationSupplier implements Function apply(final Map globalStateMap) if (Objects.nonNull(openSearchClient)) { return applyForOpenSearchClient(globalStateMap); + } else if (Objects.nonNull(elasticsearchClient)) { + return applyForElasticSearchClient(globalStateMap); } return Collections.emptyList(); @@ -70,13 +77,29 @@ private List applyForOpenSearchClient(final Map shouldIndexBeProcessed(osIndicesRecord.index())) + .map(indexRecord -> PartitionIdentifier.builder().withPartitionKey(indexRecord.index()).build()) + .collect(Collectors.toList()); + } + + private List applyForElasticSearchClient(final Map globalStateMap) { + co.elastic.clients.elasticsearch.cat.IndicesResponse indicesResponse; + try { + indicesResponse = elasticsearchClient.cat().indices(); + } catch (IOException | ElasticsearchException e) { + LOG.error("There was an exception when calling /_cat/indices to create new index partitions", e); + return Collections.emptyList(); + } + + return indicesResponse.valueBody().stream() + .filter(esIndicesRecord -> shouldIndexBeProcessed(esIndicesRecord.index())) .map(indexRecord -> PartitionIdentifier.builder().withPartitionKey(indexRecord.index()).build()) .collect(Collectors.toList()); } - private boolean shouldIndexBeProcessed(final IndicesRecord indicesRecord) { - if (Objects.isNull(indicesRecord.index())) { + private boolean shouldIndexBeProcessed(final String indexName) { + + if (Objects.isNull(indexName)) { return false; } @@ -87,16 +110,16 @@ private boolean shouldIndexBeProcessed(final IndicesRecord indicesRecord) { final List includedIndices = indexParametersConfiguration.getIncludedIndices(); final List excludedIndices = indexParametersConfiguration.getExcludedIndices(); - final boolean matchesIncludedPattern = includedIndices.isEmpty() || doesIndexMatchPattern(includedIndices, indicesRecord); - final boolean matchesExcludePattern = doesIndexMatchPattern(excludedIndices, indicesRecord); + final boolean matchesIncludedPattern = includedIndices.isEmpty() || doesIndexMatchPattern(includedIndices, indexName); + final boolean matchesExcludePattern = doesIndexMatchPattern(excludedIndices, indexName); return matchesIncludedPattern && !matchesExcludePattern; } - private boolean doesIndexMatchPattern(final List indices, final IndicesRecord indicesRecord) { + private boolean doesIndexMatchPattern(final List indices, final String indexName) { for (final OpenSearchIndex index : indices) { - final Matcher matcher = index.getIndexNamePattern().matcher(indicesRecord.index()); + final Matcher matcher = index.getIndexNamePattern().matcher(indexName); if (matcher.matches()) { return true; diff --git a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/PitWorker.java b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/PitWorker.java index f4a2510322..73040240ed 100644 --- a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/PitWorker.java +++ b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/PitWorker.java @@ -179,12 +179,12 @@ private OpenSearchIndexProgressState initializeProgressState() { } private List getSearchAfter(final OpenSearchIndexProgressState openSearchIndexProgressState, final SearchWithSearchAfterResults searchWithSearchAfterResults) { - if (Objects.isNull(searchWithSearchAfterResults) && Objects.isNull(openSearchIndexProgressState.getSearchAfter())) { - return null; - } - - if (Objects.isNull(searchWithSearchAfterResults) && Objects.nonNull(openSearchIndexProgressState.getSearchAfter())) { - return openSearchIndexProgressState.getSearchAfter(); + if (Objects.isNull(searchWithSearchAfterResults)) { + if (Objects.isNull(openSearchIndexProgressState.getSearchAfter())) { + return null; + } else { + return openSearchIndexProgressState.getSearchAfter(); + } } return searchWithSearchAfterResults.getNextSearchAfter(); diff --git a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/ElasticsearchAccessor.java b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/ElasticsearchAccessor.java index 3e3b8b6794..97185081d9 100644 --- a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/ElasticsearchAccessor.java +++ b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/ElasticsearchAccessor.java @@ -4,6 +4,26 @@ */ package org.opensearch.dataprepper.plugins.source.opensearch.worker.client; +import co.elastic.clients.elasticsearch.ElasticsearchClient; +import co.elastic.clients.elasticsearch._types.ElasticsearchException; +import co.elastic.clients.elasticsearch._types.ScoreSort; +import co.elastic.clients.elasticsearch._types.SortOptions; +import co.elastic.clients.elasticsearch._types.SortOrder; +import co.elastic.clients.elasticsearch._types.Time; +import co.elastic.clients.elasticsearch._types.query_dsl.MatchAllQuery; +import co.elastic.clients.elasticsearch._types.query_dsl.Query; +import co.elastic.clients.elasticsearch.core.ClosePointInTimeRequest; +import co.elastic.clients.elasticsearch.core.ClosePointInTimeResponse; +import co.elastic.clients.elasticsearch.core.OpenPointInTimeRequest; +import co.elastic.clients.elasticsearch.core.OpenPointInTimeResponse; +import co.elastic.clients.elasticsearch.core.SearchRequest; +import co.elastic.clients.elasticsearch.core.SearchResponse; +import co.elastic.clients.elasticsearch.core.search.PointInTimeReference; +import com.fasterxml.jackson.databind.node.ObjectNode; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventType; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.exceptions.SearchContextLimitException; import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.CreatePointInTimeRequest; import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.CreatePointInTimeResponse; import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.CreateScrollRequest; @@ -13,32 +33,100 @@ import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.NoSearchContextSearchRequest; import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.SearchContextType; import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.SearchPointInTimeRequest; -import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.SearchWithSearchAfterResults; import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.SearchScrollRequest; import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.SearchScrollResponse; +import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.SearchWithSearchAfterResults; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.Instant; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +import static org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.MetadataKeyAttributes.DOCUMENT_ID_METADATA_ATTRIBUTE_NAME; +import static org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.MetadataKeyAttributes.INDEX_METADATA_ATTRIBUTE_NAME; public class ElasticsearchAccessor implements SearchAccessor, ClusterClientFactory { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchAccessor.class); + + static final String PIT_RESOURCE_LIMIT_ERROR_TYPE = "rejected_execution_exception"; + + private final ElasticsearchClient elasticsearchClient; + private final SearchContextType searchContextType; + + public ElasticsearchAccessor(final ElasticsearchClient elasticsearchClient, final SearchContextType searchContextType) { + this.elasticsearchClient = elasticsearchClient; + this.searchContextType = searchContextType; + } + @Override public SearchContextType getSearchContextType() { - // todo: implement - return null; + return searchContextType; } @Override public CreatePointInTimeResponse createPit(final CreatePointInTimeRequest createPointInTimeRequest) { - //todo: implement - return null; + + OpenPointInTimeResponse openPointInTimeResponse; + try { + openPointInTimeResponse = elasticsearchClient.openPointInTime(OpenPointInTimeRequest.of(request -> request + .keepAlive(Time.of(time -> time.time(createPointInTimeRequest.getKeepAlive()))) + .index(createPointInTimeRequest.getIndex()))); + } catch (final ElasticsearchException e) { + if (isDueToPitLimitExceeded(e)) { + throw new SearchContextLimitException(String.format("There was an error creating a new point in time for index '%s': %s", createPointInTimeRequest.getIndex(), + e.error().causedBy().reason())); + } + LOG.error("There was an error creating a point in time for Elasticsearch: ", e); + throw e; + } catch (final IOException e) { + LOG.error("There was an error creating a point in time for Elasticsearch: ", e); + throw new RuntimeException(e); + } + + return CreatePointInTimeResponse.builder() + .withPitId(openPointInTimeResponse.id()) + .withCreationTime(Instant.now().toEpochMilli()) + .build(); } @Override - public SearchWithSearchAfterResults searchWithPit(SearchPointInTimeRequest searchPointInTimeRequest) { - //todo: implement - return null; + public SearchWithSearchAfterResults searchWithPit(final SearchPointInTimeRequest searchPointInTimeRequest) { + final SearchRequest searchRequest = SearchRequest.of(builder -> { builder + .pit(PointInTimeReference.of(pit -> pit + .id(searchPointInTimeRequest.getPitId()) + .keepAlive(Time.of(time -> time.time(searchPointInTimeRequest.getKeepAlive()))))) + .size(searchPointInTimeRequest.getPaginationSize()) + .sort(SortOptions.of(sortOptionsBuilder -> sortOptionsBuilder.doc(ScoreSort.of(scoreSort -> scoreSort.order(SortOrder.Asc))))) + .query(Query.of(query -> query.matchAll(MatchAllQuery.of(matchAllQuery -> matchAllQuery)))); + + if (Objects.nonNull(searchPointInTimeRequest.getSearchAfter())) { + builder.searchAfter(searchPointInTimeRequest.getSearchAfter()); + } + return builder; + }); + + + return searchWithSearchAfter(searchRequest); } @Override public void deletePit(final DeletePointInTimeRequest deletePointInTimeRequest) { - //todo: implement + try { + final ClosePointInTimeResponse closePointInTimeResponse = elasticsearchClient.closePointInTime(ClosePointInTimeRequest.of(request -> request + .id(deletePointInTimeRequest.getPitId()))); + if (closePointInTimeResponse.succeeded()) { + LOG.debug("Successfully deleted point in time id {}", deletePointInTimeRequest.getPitId()); + } else { + LOG.warn("Point in time id {} was not deleted successfully. It will expire from keep-alive", deletePointInTimeRequest.getPitId()); + } + } catch (final IOException | RuntimeException e) { + LOG.error("There was an error deleting the point in time with id {} for Elasticsearch. It will expire from keep-alive: ", deletePointInTimeRequest.getPitId(), e); + } } @Override @@ -59,12 +147,56 @@ public void deleteScroll(DeleteScrollRequest deleteScrollRequest) { } @Override - public SearchWithSearchAfterResults searchWithoutSearchContext(NoSearchContextSearchRequest noSearchContextSearchRequest) { - return null; + public SearchWithSearchAfterResults searchWithoutSearchContext(final NoSearchContextSearchRequest noSearchContextSearchRequest) { + final SearchRequest searchRequest = SearchRequest.of(builder -> { + builder + .index(noSearchContextSearchRequest.getIndex()) + .size(noSearchContextSearchRequest.getPaginationSize()) + .sort(SortOptions.of(sortOptionsBuilder -> sortOptionsBuilder.doc(ScoreSort.of(scoreSort -> scoreSort.order(SortOrder.Asc))))) + .query(Query.of(query -> query.matchAll(MatchAllQuery.of(matchAllQuery -> matchAllQuery)))); + + if (Objects.nonNull(noSearchContextSearchRequest.getSearchAfter())) { + builder.searchAfter(noSearchContextSearchRequest.getSearchAfter()); + } + + return builder; + }); + + return searchWithSearchAfter(searchRequest); } @Override public Object getClient() { - return null; + return elasticsearchClient; + } + + private SearchWithSearchAfterResults searchWithSearchAfter(final SearchRequest searchRequest) { + + try { + final SearchResponse searchResponse = elasticsearchClient.search(searchRequest, ObjectNode.class); + + final List documents = searchResponse.hits().hits().stream() + .map(hit -> JacksonEvent.builder() + .withData(hit.source()) + .withEventMetadataAttributes(Map.of(DOCUMENT_ID_METADATA_ATTRIBUTE_NAME, hit.id(), INDEX_METADATA_ATTRIBUTE_NAME, hit.index())) + .withEventType(EventType.DOCUMENT.toString()).build()) + .collect(Collectors.toList()); + + final List nextSearchAfter = Objects.nonNull(searchResponse.hits().hits()) && !searchResponse.hits().hits().isEmpty() ? + searchResponse.hits().hits().get(searchResponse.hits().hits().size() - 1).sort() : + null; + + return SearchWithSearchAfterResults.builder() + .withDocuments(documents) + .withNextSearchAfter(nextSearchAfter) + .build(); + } catch (final IOException e) { + throw new RuntimeException(e); + } + } + + private boolean isDueToPitLimitExceeded(final ElasticsearchException e) { + return Objects.nonNull(e.error()) && Objects.nonNull(e.error().causedBy()) && Objects.nonNull(e.error().causedBy().type()) + && PIT_RESOURCE_LIMIT_ERROR_TYPE.equals(e.error().causedBy().type()); } } 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 new file mode 100644 index 0000000000..d9e3a2f739 --- /dev/null +++ b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactory.java @@ -0,0 +1,293 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.opensearch.worker.client; + +import co.elastic.clients.elasticsearch.ElasticsearchClient; +import co.elastic.clients.transport.ElasticsearchTransport; +import org.apache.http.Header; +import org.apache.http.HttpHost; +import org.apache.http.HttpResponseInterceptor; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.conn.ssl.NoopHostnameVerifier; +import org.apache.http.conn.ssl.TrustAllStrategy; +import org.apache.http.conn.ssl.TrustStrategy; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; +import org.apache.http.message.BasicHeader; +import org.apache.http.ssl.SSLContextBuilder; +import org.apache.http.ssl.SSLContexts; +import org.opensearch.client.RestClient; +import org.opensearch.client.RestClientBuilder; +import org.opensearch.client.json.jackson.JacksonJsonpMapper; +import org.opensearch.client.opensearch.OpenSearchClient; +import org.opensearch.client.transport.OpenSearchTransport; +import org.opensearch.client.transport.aws.AwsSdk2Transport; +import org.opensearch.client.transport.aws.AwsSdk2TransportOptions; +import org.opensearch.client.transport.rest_client.RestClientTransport; +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.source.opensearch.OpenSearchSourceConfiguration; +import org.opensearch.dataprepper.plugins.source.opensearch.configuration.ConnectionConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.http.SdkHttpClient; +import software.amazon.awssdk.http.apache.ApacheHttpClient; + +import javax.net.ssl.SSLContext; +import javax.net.ssl.TrustManager; +import javax.net.ssl.TrustManagerFactory; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.security.KeyStore; +import java.security.cert.Certificate; +import java.security.cert.CertificateFactory; +import java.util.List; +import java.util.Objects; + +public class OpenSearchClientFactory { + + private static final Logger LOG = LoggerFactory.getLogger(OpenSearchClientFactory.class); + + private static final String AOS_SERVICE_NAME = "es"; + + private final AwsCredentialsSupplier awsCredentialsSupplier; + + public static OpenSearchClientFactory create(final AwsCredentialsSupplier awsCredentialsSupplier) { + return new OpenSearchClientFactory(awsCredentialsSupplier); + } + + private OpenSearchClientFactory(final AwsCredentialsSupplier awsCredentialsSupplier) { + this.awsCredentialsSupplier = awsCredentialsSupplier; + } + + public OpenSearchClient provideOpenSearchClient(final OpenSearchSourceConfiguration openSearchSourceConfiguration) { + OpenSearchTransport transport; + if (Objects.nonNull(openSearchSourceConfiguration.getAwsAuthenticationOptions())) { + transport = createOpenSearchTransportForAws(openSearchSourceConfiguration); + } else { + final RestClient restClient = createOpenSearchRestClient(openSearchSourceConfiguration); + transport = createOpenSearchTransport(restClient); + } + return new OpenSearchClient(transport); + } + + public ElasticsearchClient provideElasticSearchClient(final OpenSearchSourceConfiguration openSearchSourceConfiguration) { + final org.elasticsearch.client.RestClient restClientElasticsearch = createElasticSearchRestClient(openSearchSourceConfiguration); + final ElasticsearchTransport elasticsearchTransport = createElasticSearchTransport(restClientElasticsearch); + return new ElasticsearchClient(elasticsearchTransport); + } + + private OpenSearchTransport createOpenSearchTransport(final RestClient restClient) { + return new RestClientTransport(restClient, new JacksonJsonpMapper()); + } + + private OpenSearchTransport createOpenSearchTransportForAws(final OpenSearchSourceConfiguration openSearchSourceConfiguration) { + final AwsCredentialsProvider awsCredentialsProvider = awsCredentialsSupplier.getProvider(AwsCredentialsOptions.builder() + .withRegion(openSearchSourceConfiguration.getAwsAuthenticationOptions().getAwsRegion()) + .withStsRoleArn(openSearchSourceConfiguration.getAwsAuthenticationOptions().getAwsStsRoleArn()) + .withStsExternalId(openSearchSourceConfiguration.getAwsAuthenticationOptions().getAwsStsExternalId()) + .withStsHeaderOverrides(openSearchSourceConfiguration.getAwsAuthenticationOptions().getAwsStsHeaderOverrides()) + .build()); + + return new AwsSdk2Transport(createSdkHttpClient(openSearchSourceConfiguration), + HttpHost.create(openSearchSourceConfiguration.getHosts().get(0)).getHostName(), + AOS_SERVICE_NAME, openSearchSourceConfiguration.getAwsAuthenticationOptions().getAwsRegion(), + AwsSdk2TransportOptions.builder() + .setCredentials(awsCredentialsProvider) + .setMapper(new JacksonJsonpMapper()) + .build()); + } + + private SdkHttpClient createSdkHttpClient(final OpenSearchSourceConfiguration openSearchSourceConfiguration) { + final ApacheHttpClient.Builder apacheHttpClientBuilder = ApacheHttpClient.builder(); + + if (Objects.nonNull(openSearchSourceConfiguration.getConnectionConfiguration().getConnectTimeout())) { + apacheHttpClientBuilder.connectionTimeout(openSearchSourceConfiguration.getConnectionConfiguration().getConnectTimeout()); + } + + if (Objects.nonNull(openSearchSourceConfiguration.getConnectionConfiguration().getSocketTimeout())) { + apacheHttpClientBuilder.socketTimeout(openSearchSourceConfiguration.getConnectionConfiguration().getSocketTimeout()); + } + + attachSSLContext(apacheHttpClientBuilder, openSearchSourceConfiguration); + + return apacheHttpClientBuilder.build(); + } + + private RestClient createOpenSearchRestClient(final OpenSearchSourceConfiguration openSearchSourceConfiguration) { + final List hosts = openSearchSourceConfiguration.getHosts(); + final HttpHost[] httpHosts = new HttpHost[hosts.size()]; + + int i = 0; + for (final String host : hosts) { + httpHosts[i] = HttpHost.create(host); + i++; + } + + final RestClientBuilder restClientBuilder = RestClient.builder(httpHosts); + + LOG.info("Using username and password for auth for the OpenSearch source"); + attachUsernamePassword(restClientBuilder, openSearchSourceConfiguration); + + setConnectAndSocketTimeout(restClientBuilder, openSearchSourceConfiguration); + + return restClientBuilder.build(); + } + + private ElasticsearchTransport createElasticSearchTransport(final org.elasticsearch.client.RestClient restClient) { + return new co.elastic.clients.transport.rest_client.RestClientTransport(restClient, new co.elastic.clients.json.jackson.JacksonJsonpMapper()); + } + + private org.elasticsearch.client.RestClient createElasticSearchRestClient(final OpenSearchSourceConfiguration openSearchSourceConfiguration) { + final List hosts = openSearchSourceConfiguration.getHosts(); + final HttpHost[] httpHosts = new HttpHost[hosts.size()]; + + int i = 0; + for (final String host : hosts) { + httpHosts[i] = HttpHost.create(host); + i++; + } + + final org.elasticsearch.client.RestClientBuilder restClientBuilder = org.elasticsearch.client.RestClient.builder(httpHosts); + + restClientBuilder.setDefaultHeaders(new Header[] { + new BasicHeader("Content-type", "application/json") + }); + + LOG.info("Using username and password for auth for the OpenSearch source"); + attachUsernamePassword(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())); + + restClientBuilder.setHttpClientConfigCallback(httpClientBuilder -> { + httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider); + 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())); + + restClientBuilder.setHttpClientConfigCallback(httpClientBuilder -> { + httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider); + attachSSLContext(httpClientBuilder, openSearchSourceConfiguration); + httpClientBuilder.addInterceptorLast( + (HttpResponseInterceptor) + (response, context) -> + response.addHeader("X-Elastic-Product", "Elasticsearch")); + return httpClientBuilder; + }); + } + + private void setConnectAndSocketTimeout(final RestClientBuilder restClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { + restClientBuilder.setRequestConfigCallback(requestConfigBuilder -> { + if (Objects.nonNull(openSearchSourceConfiguration.getConnectionConfiguration().getConnectTimeout())) { + requestConfigBuilder.setConnectTimeout((int) openSearchSourceConfiguration.getConnectionConfiguration().getConnectTimeout().toMillis()); + } + + if (Objects.nonNull(openSearchSourceConfiguration.getConnectionConfiguration().getSocketTimeout())) { + requestConfigBuilder.setSocketTimeout((int) openSearchSourceConfiguration.getConnectionConfiguration().getSocketTimeout().toMillis()); + } + + return requestConfigBuilder; + }); + } + + private void setConnectAndSocketTimeout(final org.elasticsearch.client.RestClientBuilder restClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { + restClientBuilder.setRequestConfigCallback(requestConfigBuilder -> { + if (Objects.nonNull(openSearchSourceConfiguration.getConnectionConfiguration().getConnectTimeout())) { + requestConfigBuilder.setConnectTimeout((int) openSearchSourceConfiguration.getConnectionConfiguration().getConnectTimeout().toMillis()); + } + + if (Objects.nonNull(openSearchSourceConfiguration.getConnectionConfiguration().getSocketTimeout())) { + requestConfigBuilder.setSocketTimeout((int) openSearchSourceConfiguration.getConnectionConfiguration().getSocketTimeout().toMillis()); + } + + return requestConfigBuilder; + }); + } + + private void attachSSLContext(final ApacheHttpClient.Builder apacheHttpClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { + TrustManager[] trustManagers = createTrustManagers(openSearchSourceConfiguration.getConnectionConfiguration().getCertPath()); + apacheHttpClientBuilder.tlsTrustManagersProvider(() -> trustManagers); + } + + private void attachSSLContext(final HttpAsyncClientBuilder httpClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { + + final ConnectionConfiguration connectionConfiguration = openSearchSourceConfiguration.getConnectionConfiguration(); + final SSLContext sslContext = Objects.nonNull(connectionConfiguration.getCertPath()) ? getCAStrategy(connectionConfiguration.getCertPath()) : getTrustAllStrategy(); + httpClientBuilder.setSSLContext(sslContext); + + if (connectionConfiguration.isInsecure()) { + httpClientBuilder.setSSLHostnameVerifier(NoopHostnameVerifier.INSTANCE); + } + } + + private static TrustManager[] createTrustManagers(final Path certPath) { + if (certPath != null) { + LOG.info("Using the cert provided in the config."); + try (InputStream certificateInputStream = Files.newInputStream(certPath)) { + final CertificateFactory factory = CertificateFactory.getInstance("X.509"); + final Certificate trustedCa = factory.generateCertificate(certificateInputStream); + final KeyStore trustStore = KeyStore.getInstance("pkcs12"); + trustStore.load(null, null); + trustStore.setCertificateEntry("ca", trustedCa); + + final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509"); + trustManagerFactory.init(trustStore); + return trustManagerFactory.getTrustManagers(); + } catch (Exception ex) { + throw new RuntimeException(ex.getMessage(), ex); + } + } else { + return new TrustManager[] { new X509TrustAllManager() }; + } + } + + private SSLContext getCAStrategy(final Path certPath) { + LOG.info("Using the cert provided in the config."); + try { + CertificateFactory factory = CertificateFactory.getInstance("X.509"); + Certificate trustedCa; + try (InputStream is = Files.newInputStream(certPath)) { + trustedCa = factory.generateCertificate(is); + } + KeyStore trustStore = KeyStore.getInstance("pkcs12"); + trustStore.load(null, null); + trustStore.setCertificateEntry("ca", trustedCa); + SSLContextBuilder sslContextBuilder = SSLContexts.custom() + .loadTrustMaterial(trustStore, null); + return sslContextBuilder.build(); + } catch (Exception ex) { + throw new RuntimeException(ex.getMessage(), ex); + } + } + + private SSLContext getTrustAllStrategy() { + LOG.info("Using the trust all strategy"); + final TrustStrategy trustStrategy = new TrustAllStrategy(); + try { + return SSLContexts.custom().loadTrustMaterial(null, trustStrategy).build(); + } catch (Exception ex) { + throw new RuntimeException(ex.getMessage(), ex); + } + } +} diff --git a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/SearchAccessorStrategy.java b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/SearchAccessorStrategy.java index f25c59b8f2..50e7414703 100644 --- a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/SearchAccessorStrategy.java +++ b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/SearchAccessorStrategy.java @@ -4,50 +4,19 @@ */ package org.opensearch.dataprepper.plugins.source.opensearch.worker.client; -import org.apache.http.HttpHost; -import org.apache.http.auth.AuthScope; -import org.apache.http.auth.UsernamePasswordCredentials; -import org.apache.http.client.CredentialsProvider; -import org.apache.http.conn.ssl.NoopHostnameVerifier; -import org.apache.http.conn.ssl.TrustAllStrategy; -import org.apache.http.conn.ssl.TrustStrategy; -import org.apache.http.impl.client.BasicCredentialsProvider; -import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; -import org.apache.http.ssl.SSLContextBuilder; -import org.apache.http.ssl.SSLContexts; +import co.elastic.clients.elasticsearch.ElasticsearchClient; +import org.apache.commons.lang3.tuple.Pair; import org.apache.maven.artifact.versioning.DefaultArtifactVersion; -import org.opensearch.client.RestClient; -import org.opensearch.client.RestClientBuilder; -import org.opensearch.client.json.jackson.JacksonJsonpMapper; import org.opensearch.client.opensearch.OpenSearchClient; import org.opensearch.client.opensearch._types.OpenSearchException; import org.opensearch.client.opensearch.core.InfoResponse; -import org.opensearch.client.transport.OpenSearchTransport; -import org.opensearch.client.transport.aws.AwsSdk2Transport; -import org.opensearch.client.transport.aws.AwsSdk2TransportOptions; -import org.opensearch.client.transport.rest_client.RestClientTransport; -import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; -import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.client.util.MissingRequiredPropertyException; import org.opensearch.dataprepper.plugins.source.opensearch.OpenSearchSourceConfiguration; -import org.opensearch.dataprepper.plugins.source.opensearch.configuration.ConnectionConfiguration; import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.SearchContextType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; -import software.amazon.awssdk.http.SdkHttpClient; -import software.amazon.awssdk.http.apache.ApacheHttpClient; -import javax.net.ssl.SSLContext; -import javax.net.ssl.TrustManager; -import javax.net.ssl.TrustManagerFactory; import java.io.IOException; -import java.io.InputStream; -import java.nio.file.Files; -import java.nio.file.Path; -import java.security.KeyStore; -import java.security.cert.Certificate; -import java.security.cert.CertificateFactory; -import java.util.List; import java.util.Objects; /** @@ -59,22 +28,27 @@ public class SearchAccessorStrategy { private static final Logger LOG = LoggerFactory.getLogger(SearchAccessorStrategy.class); - private static final String AOS_SERVICE_NAME = "es"; static final String OPENSEARCH_DISTRIBUTION = "opensearch"; + static final String ELASTICSEARCH_DISTRIBUTION = "elasticsearch"; + static final String ELASTICSEARCH_OSS_BUILD_FLAVOR = "oss"; + static final String OPENDISTRO_DISTRIUBTION = "opendistro"; + private static final String OPENSEARCH_POINT_IN_TIME_SUPPORT_VERSION_CUTOFF = "2.5.0"; + private static final String ELASTICSEARCH_POINT_IN_TIME_SUPPORT_VERSION_CUTOFF = "7.10.0"; + - private final AwsCredentialsSupplier awsCredentialsSupplier; + private final OpenSearchClientFactory openSearchClientFactory; private final OpenSearchSourceConfiguration openSearchSourceConfiguration; public static SearchAccessorStrategy create(final OpenSearchSourceConfiguration openSearchSourceConfiguration, - final AwsCredentialsSupplier awsCredentialsSupplier) { - return new SearchAccessorStrategy(openSearchSourceConfiguration, awsCredentialsSupplier); + final OpenSearchClientFactory openSearchClientFactory) { + return new SearchAccessorStrategy(openSearchSourceConfiguration, openSearchClientFactory); } private SearchAccessorStrategy(final OpenSearchSourceConfiguration openSearchSourceConfiguration, - final AwsCredentialsSupplier awsCredentialsSupplier) { - this.awsCredentialsSupplier = awsCredentialsSupplier; + final OpenSearchClientFactory openSearchClientFactory) { this.openSearchSourceConfiguration = openSearchSourceConfiguration; + this.openSearchClientFactory = openSearchClientFactory; } /** @@ -84,208 +58,93 @@ private SearchAccessorStrategy(final OpenSearchSourceConfiguration openSearchSou */ public SearchAccessor getSearchAccessor() { - OpenSearchTransport transport; - if (Objects.nonNull(openSearchSourceConfiguration.getAwsAuthenticationOptions())) { - transport = createOpenSearchTransportForAws(); - } else { - final RestClient restClient = createOpenSearchRestClient(); - transport = createOpenSearchTransport(restClient); - } - final OpenSearchClient openSearchClient = new OpenSearchClient(transport); + final OpenSearchClient openSearchClient = openSearchClientFactory.provideOpenSearchClient(openSearchSourceConfiguration); - InfoResponse infoResponse; + InfoResponse infoResponse = null; + + ElasticsearchClient elasticsearchClient = null; try { infoResponse = openSearchClient.info(); + } catch (final MissingRequiredPropertyException e) { + LOG.info("Detected Elasticsearch cluster. Constructing Elasticsearch client"); + elasticsearchClient = openSearchClientFactory.provideElasticSearchClient(openSearchSourceConfiguration); } catch (final IOException | OpenSearchException e) { throw new RuntimeException("There was an error looking up the OpenSearch cluster info: ", e); } - final String distribution = infoResponse.version().distribution(); - final String versionNumber = infoResponse.version().number(); + final Pair distributionAndVersion = getDistributionAndVersionNumber(infoResponse, elasticsearchClient); - if (!distribution.equals(OPENSEARCH_DISTRIBUTION)) { - throw new IllegalArgumentException(String.format("Only opensearch distributions are supported at this time. The cluster distribution being used is '%s'", distribution)); - } + final String distribution = distributionAndVersion.getLeft(); + final String versionNumber = distributionAndVersion.getRight(); + + validateDistribution(distribution); SearchContextType searchContextType; if (Objects.nonNull(openSearchSourceConfiguration.getSearchConfiguration().getSearchContextType())) { LOG.info("Using search_context_type set in the config: '{}'", openSearchSourceConfiguration.getSearchConfiguration().getSearchContextType().toString().toLowerCase()); - validateSearchContextTypeOverride(openSearchSourceConfiguration.getSearchConfiguration().getSearchContextType(), versionNumber); + validateSearchContextTypeOverride(openSearchSourceConfiguration.getSearchConfiguration().getSearchContextType(), distribution, versionNumber); searchContextType = openSearchSourceConfiguration.getSearchConfiguration().getSearchContextType(); - } else if (versionSupportsPointInTimeForOpenSearch(versionNumber)) { - LOG.info("OpenSearch version {} detected. Point in time APIs will be used to search documents", versionNumber); + } else if (versionSupportsPointInTime(distribution, versionNumber)) { + LOG.info("{} distribution and version {} detected. Point in time APIs will be used to search documents", distribution, versionNumber); searchContextType = SearchContextType.POINT_IN_TIME; } else { - LOG.info("OpenSearch version {} detected. Scroll contexts will be used to search documents. " + - "Upgrade your cluster to at least version {} to use Point in Time APIs instead of scroll.", versionNumber, OPENSEARCH_POINT_IN_TIME_SUPPORT_VERSION_CUTOFF); + LOG.info("{} distribution, version {} detected. Scroll contexts will be used to search documents. " + + "Upgrade your cluster to at least version {} to use Point in Time APIs instead of scroll.", distribution, versionNumber, + distribution.equals(ELASTICSEARCH_DISTRIBUTION) ? ELASTICSEARCH_POINT_IN_TIME_SUPPORT_VERSION_CUTOFF : OPENSEARCH_POINT_IN_TIME_SUPPORT_VERSION_CUTOFF); searchContextType = SearchContextType.SCROLL; } - return new OpenSearchAccessor(openSearchClient, searchContextType); - } - - private RestClient createOpenSearchRestClient() { - final List hosts = openSearchSourceConfiguration.getHosts(); - final HttpHost[] httpHosts = new HttpHost[hosts.size()]; - - int i = 0; - for (final String host : hosts) { - httpHosts[i] = HttpHost.create(host); - i++; + if (Objects.isNull(elasticsearchClient)) { + return new OpenSearchAccessor(openSearchClient, searchContextType); } - final RestClientBuilder restClientBuilder = RestClient.builder(httpHosts); - - LOG.info("Using username and password for auth for the OpenSearch source"); - attachUsernamePassword(restClientBuilder); - - setConnectAndSocketTimeout(restClientBuilder); - - return restClientBuilder.build(); + return new ElasticsearchAccessor(elasticsearchClient, searchContextType); } - private void attachSSLContext(final ApacheHttpClient.Builder apacheHttpClientBuilder) { - TrustManager[] trustManagers = createTrustManagers(openSearchSourceConfiguration.getConnectionConfiguration().getCertPath()); - apacheHttpClientBuilder.tlsTrustManagersProvider(() -> trustManagers); - } - - private void attachSSLContext(final HttpAsyncClientBuilder httpClientBuilder) { + private void validateSearchContextTypeOverride(final SearchContextType searchContextType, final String distribution, final String version) { - final ConnectionConfiguration connectionConfiguration = openSearchSourceConfiguration.getConnectionConfiguration(); - final SSLContext sslContext = Objects.nonNull(connectionConfiguration.getCertPath()) ? getCAStrategy(connectionConfiguration.getCertPath()) : getTrustAllStrategy(); - httpClientBuilder.setSSLContext(sslContext); - - if (connectionConfiguration.isInsecure()) { - httpClientBuilder.setSSLHostnameVerifier(NoopHostnameVerifier.INSTANCE); + if (searchContextType.equals(SearchContextType.POINT_IN_TIME) && !versionSupportsPointInTime(distribution, version)) { + throw new IllegalArgumentException( + String.format("A search_context_type of point_in_time is only supported on OpenSearch versions %s and above. " + + "The version of the OpenSearch cluster passed is %s. Elasticsearch clusters with build-flavor %s do not support point in time", + distribution.startsWith(ELASTICSEARCH_DISTRIBUTION) ? ELASTICSEARCH_POINT_IN_TIME_SUPPORT_VERSION_CUTOFF : OPENSEARCH_POINT_IN_TIME_SUPPORT_VERSION_CUTOFF, + version, ELASTICSEARCH_OSS_BUILD_FLAVOR)); } } - private static TrustManager[] createTrustManagers(final Path certPath) { - if (certPath != null) { - LOG.info("Using the cert provided in the config."); - try (InputStream certificateInputStream = Files.newInputStream(certPath)) { - final CertificateFactory factory = CertificateFactory.getInstance("X.509"); - final Certificate trustedCa = factory.generateCertificate(certificateInputStream); - final KeyStore trustStore = KeyStore.getInstance("pkcs12"); - trustStore.load(null, null); - trustStore.setCertificateEntry("ca", trustedCa); + private boolean versionSupportsPointInTime(final String distribution, final String version) { + final DefaultArtifactVersion actualVersion = new DefaultArtifactVersion(version); - final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509"); - trustManagerFactory.init(trustStore); - return trustManagerFactory.getTrustManagers(); - } catch (Exception ex) { - throw new RuntimeException(ex.getMessage(), ex); + DefaultArtifactVersion cutoffVersion; + if (distribution.startsWith(ELASTICSEARCH_DISTRIBUTION)) { + if (distribution.endsWith(ELASTICSEARCH_OSS_BUILD_FLAVOR)) { + return false; } + cutoffVersion = new DefaultArtifactVersion(ELASTICSEARCH_POINT_IN_TIME_SUPPORT_VERSION_CUTOFF); } else { - return new TrustManager[] { new X509TrustAllManager() }; - } - } - - private void attachUsernamePassword(final RestClientBuilder restClientBuilder) { - final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); - credentialsProvider.setCredentials(AuthScope.ANY, - new UsernamePasswordCredentials(openSearchSourceConfiguration.getUsername(), openSearchSourceConfiguration.getPassword())); - - restClientBuilder.setHttpClientConfigCallback(httpClientBuilder -> { - httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider); - attachSSLContext(httpClientBuilder); - return httpClientBuilder; - }); - } - - private void setConnectAndSocketTimeout(final RestClientBuilder restClientBuilder) { - restClientBuilder.setRequestConfigCallback(requestConfigBuilder -> { - if (Objects.nonNull(openSearchSourceConfiguration.getConnectionConfiguration().getConnectTimeout())) { - requestConfigBuilder.setConnectTimeout((int) openSearchSourceConfiguration.getConnectionConfiguration().getConnectTimeout().toMillis()); - } - - if (Objects.nonNull(openSearchSourceConfiguration.getConnectionConfiguration().getSocketTimeout())) { - requestConfigBuilder.setSocketTimeout((int) openSearchSourceConfiguration.getConnectionConfiguration().getSocketTimeout().toMillis()); - } - - return requestConfigBuilder; - }); - } - - private OpenSearchTransport createOpenSearchTransport(final RestClient restClient) { - return new RestClientTransport(restClient, new JacksonJsonpMapper()); - } - - private OpenSearchTransport createOpenSearchTransportForAws() { - final AwsCredentialsProvider awsCredentialsProvider = awsCredentialsSupplier.getProvider(AwsCredentialsOptions.builder() - .withRegion(openSearchSourceConfiguration.getAwsAuthenticationOptions().getAwsRegion()) - .withStsRoleArn(openSearchSourceConfiguration.getAwsAuthenticationOptions().getAwsStsRoleArn()) - .withStsExternalId(openSearchSourceConfiguration.getAwsAuthenticationOptions().getAwsStsExternalId()) - .withStsHeaderOverrides(openSearchSourceConfiguration.getAwsAuthenticationOptions().getAwsStsHeaderOverrides()) - .build()); - - return new AwsSdk2Transport(createSdkHttpClient(), - HttpHost.create(openSearchSourceConfiguration.getHosts().get(0)).getHostName(), - AOS_SERVICE_NAME, openSearchSourceConfiguration.getAwsAuthenticationOptions().getAwsRegion(), - AwsSdk2TransportOptions.builder() - .setCredentials(awsCredentialsProvider) - .setMapper(new JacksonJsonpMapper()) - .build()); - } - - private SdkHttpClient createSdkHttpClient() { - final ApacheHttpClient.Builder apacheHttpClientBuilder = ApacheHttpClient.builder(); - - if (Objects.nonNull(openSearchSourceConfiguration.getConnectionConfiguration().getConnectTimeout())) { - apacheHttpClientBuilder.connectionTimeout(openSearchSourceConfiguration.getConnectionConfiguration().getConnectTimeout()); - } - - if (Objects.nonNull(openSearchSourceConfiguration.getConnectionConfiguration().getSocketTimeout())) { - apacheHttpClientBuilder.socketTimeout(openSearchSourceConfiguration.getConnectionConfiguration().getSocketTimeout()); + cutoffVersion = new DefaultArtifactVersion(OPENSEARCH_POINT_IN_TIME_SUPPORT_VERSION_CUTOFF); } - - attachSSLContext(apacheHttpClientBuilder); - - return apacheHttpClientBuilder.build(); + return actualVersion.compareTo(cutoffVersion) >= 0; } - private SSLContext getCAStrategy(final Path certPath) { - LOG.info("Using the cert provided in the config."); - try { - CertificateFactory factory = CertificateFactory.getInstance("X.509"); - Certificate trustedCa; - try (InputStream is = Files.newInputStream(certPath)) { - trustedCa = factory.generateCertificate(is); - } - KeyStore trustStore = KeyStore.getInstance("pkcs12"); - trustStore.load(null, null); - trustStore.setCertificateEntry("ca", trustedCa); - SSLContextBuilder sslContextBuilder = SSLContexts.custom() - .loadTrustMaterial(trustStore, null); - return sslContextBuilder.build(); - } catch (Exception ex) { - throw new RuntimeException(ex.getMessage(), ex); + private Pair getDistributionAndVersionNumber(final InfoResponse infoResponseOpenSearch, final ElasticsearchClient elasticsearchClient) { + if (Objects.nonNull(infoResponseOpenSearch)) { + return Pair.of(infoResponseOpenSearch.version().distribution(), infoResponseOpenSearch.version().number()); } - } - private SSLContext getTrustAllStrategy() { - LOG.info("Using the trust all strategy"); - final TrustStrategy trustStrategy = new TrustAllStrategy(); try { - return SSLContexts.custom().loadTrustMaterial(null, trustStrategy).build(); - } catch (Exception ex) { - throw new RuntimeException(ex.getMessage(), ex); + final co.elastic.clients.elasticsearch.core.InfoResponse infoResponseElasticsearch = elasticsearchClient.info(); + return Pair.of(ELASTICSEARCH_DISTRIBUTION + "-" + infoResponseElasticsearch.version().buildFlavor(), infoResponseElasticsearch.version().number()); + } catch (final Exception e) { + throw new RuntimeException("Unable to call info API using the elasticsearch client", e); } } - private void validateSearchContextTypeOverride(final SearchContextType searchContextType, final String version) { - - if (searchContextType.equals(SearchContextType.POINT_IN_TIME) && !versionSupportsPointInTimeForOpenSearch(version)) { - throw new IllegalArgumentException( - String.format("A search_context_type of point_in_time is only supported on OpenSearch versions %s and above. " + - "The version of the OpenSearch cluster passed is %s", OPENSEARCH_POINT_IN_TIME_SUPPORT_VERSION_CUTOFF, version)); + private void validateDistribution(final String distribution) { + if (!distribution.equals(OPENSEARCH_DISTRIBUTION) && !distribution.startsWith(ELASTICSEARCH_DISTRIBUTION) && !distribution.equals(OPENDISTRO_DISTRIUBTION)) { + throw new IllegalArgumentException(String.format("Only %s, %s, or %s distributions are supported at this time. The cluster distribution being used is '%s'", + OPENSEARCH_DISTRIBUTION, OPENDISTRO_DISTRIUBTION, ELASTICSEARCH_DISTRIBUTION, distribution)); } } - - private boolean versionSupportsPointInTimeForOpenSearch(final String version) { - final DefaultArtifactVersion cutoffVersion = new DefaultArtifactVersion(OPENSEARCH_POINT_IN_TIME_SUPPORT_VERSION_CUTOFF); - final DefaultArtifactVersion actualVersion = new DefaultArtifactVersion(version); - return actualVersion.compareTo(cutoffVersion) >= 0; - } } diff --git a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceTest.java b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceTest.java index 68affcfc64..e5f19cffeb 100644 --- a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceTest.java +++ b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceTest.java @@ -15,6 +15,7 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.coordinator.SourceCoordinator; +import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.OpenSearchClientFactory; import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.SearchAccessor; import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.SearchAccessorStrategy; @@ -35,6 +36,9 @@ public class OpenSearchSourceTest { @Mock private OpenSearchService openSearchService; + @Mock + private OpenSearchClientFactory openSearchClientFactory; + @Mock private SearchAccessorStrategy searchAccessorStrategy; @@ -66,8 +70,10 @@ void start_with_non_null_buffer_does_not_throw() { objectUnderTest.setSourceCoordinator(sourceCoordinator); try (final MockedStatic searchAccessorStrategyMockedStatic = mockStatic(SearchAccessorStrategy.class); + final MockedStatic openSearchClientFactoryMockedStatic = mockStatic(OpenSearchClientFactory.class); final MockedStatic openSearchServiceMockedStatic = mockStatic(OpenSearchService.class)) { - searchAccessorStrategyMockedStatic.when(() -> SearchAccessorStrategy.create(openSearchSourceConfiguration, awsCredentialsSupplier)).thenReturn(searchAccessorStrategy); + openSearchClientFactoryMockedStatic.when(() -> OpenSearchClientFactory.create(awsCredentialsSupplier)).thenReturn(openSearchClientFactory); + searchAccessorStrategyMockedStatic.when(() -> SearchAccessorStrategy.create(openSearchSourceConfiguration, openSearchClientFactory)).thenReturn(searchAccessorStrategy); openSearchServiceMockedStatic.when(() -> OpenSearchService.createOpenSearchService(searchAccessor, sourceCoordinator, openSearchSourceConfiguration, buffer)) .thenReturn(openSearchService); diff --git a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/ElasticsearchAccessorTest.java b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/ElasticsearchAccessorTest.java new file mode 100644 index 0000000000..20cbd617a1 --- /dev/null +++ b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/ElasticsearchAccessorTest.java @@ -0,0 +1,281 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.opensearch.worker.client; + +import co.elastic.clients.elasticsearch.ElasticsearchClient; +import co.elastic.clients.elasticsearch._types.ElasticsearchException; +import co.elastic.clients.elasticsearch._types.ErrorCause; +import co.elastic.clients.elasticsearch.core.ClosePointInTimeRequest; +import co.elastic.clients.elasticsearch.core.ClosePointInTimeResponse; +import co.elastic.clients.elasticsearch.core.OpenPointInTimeRequest; +import co.elastic.clients.elasticsearch.core.OpenPointInTimeResponse; +import co.elastic.clients.elasticsearch.core.SearchRequest; +import co.elastic.clients.elasticsearch.core.SearchResponse; +import co.elastic.clients.elasticsearch.core.search.Hit; +import co.elastic.clients.elasticsearch.core.search.HitsMetadata; +import com.fasterxml.jackson.databind.node.ObjectNode; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.exceptions.SearchContextLimitException; +import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.CreatePointInTimeRequest; +import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.CreatePointInTimeResponse; +import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.DeletePointInTimeRequest; +import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.NoSearchContextSearchRequest; +import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.SearchContextType; +import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.SearchPointInTimeRequest; +import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.SearchWithSearchAfterResults; + +import java.io.IOException; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.opensearch.worker.client.OpenSearchAccessor.PIT_RESOURCE_LIMIT_ERROR_TYPE; + +@ExtendWith(MockitoExtension.class) +public class ElasticsearchAccessorTest { + + @Mock + private ElasticsearchClient elasticSearchClient; + + private SearchAccessor createObjectUnderTest() { + return new ElasticsearchAccessor(elasticSearchClient, SearchContextType.POINT_IN_TIME); + } + + @Test + void create_pit_returns_expected_create_point_in_time_response() throws IOException { + final String indexName = UUID.randomUUID().toString(); + final String keepAlive = UUID.randomUUID().toString(); + + final CreatePointInTimeRequest createPointInTimeRequest = mock(CreatePointInTimeRequest.class); + when(createPointInTimeRequest.getIndex()).thenReturn(indexName); + when(createPointInTimeRequest.getKeepAlive()).thenReturn(keepAlive); + + final String pitId = UUID.randomUUID().toString(); + final OpenPointInTimeResponse createPitResponse = mock(OpenPointInTimeResponse.class); + when(createPitResponse.id()).thenReturn(pitId); + + when(elasticSearchClient.openPointInTime(any(OpenPointInTimeRequest.class))).thenReturn(createPitResponse); + + final CreatePointInTimeResponse createPointInTimeResponse = createObjectUnderTest().createPit(createPointInTimeRequest); + assertThat(createPointInTimeResponse, notNullValue()); + assertThat(createPointInTimeResponse.getPitCreationTime(), lessThanOrEqualTo(Instant.now().toEpochMilli())); + assertThat(createPointInTimeResponse.getPitId(), equalTo(pitId)); + } + + @Test + void create_pit_with_exception_for_pit_limit_throws_SearchContextLimitException() throws IOException { + final String indexName = UUID.randomUUID().toString(); + final String keepAlive = UUID.randomUUID().toString(); + + final CreatePointInTimeRequest createPointInTimeRequest = mock(CreatePointInTimeRequest.class); + when(createPointInTimeRequest.getIndex()).thenReturn(indexName); + when(createPointInTimeRequest.getKeepAlive()).thenReturn(keepAlive); + + final ElasticsearchException elasticsearchException = mock(ElasticsearchException.class); + final ErrorCause errorCause = mock(ErrorCause.class); + final ErrorCause rootCause = mock(ErrorCause.class); + when(rootCause.type()).thenReturn(PIT_RESOURCE_LIMIT_ERROR_TYPE); + when(rootCause.reason()).thenReturn(UUID.randomUUID().toString()); + when(errorCause.causedBy()).thenReturn(rootCause); + when(elasticsearchException.error()).thenReturn(errorCause); + + when(elasticSearchClient.openPointInTime(any(OpenPointInTimeRequest.class))).thenThrow(elasticsearchException); + + assertThrows(SearchContextLimitException.class, () -> createObjectUnderTest().createPit(createPointInTimeRequest)); + } + + @Test + void createPit_throws_Elasticsearch_exception_throws_that_exception() throws IOException { + final String indexName = UUID.randomUUID().toString(); + final String keepAlive = UUID.randomUUID().toString(); + + final CreatePointInTimeRequest createPointInTimeRequest = mock(CreatePointInTimeRequest.class); + when(createPointInTimeRequest.getIndex()).thenReturn(indexName); + when(createPointInTimeRequest.getKeepAlive()).thenReturn(keepAlive); + + final ElasticsearchException openSearchException = mock(ElasticsearchException.class); + final ErrorCause errorCause = mock(ErrorCause.class); + when(errorCause.causedBy()).thenReturn(null); + when(openSearchException.error()).thenReturn(errorCause); + + when(elasticSearchClient.openPointInTime(any(OpenPointInTimeRequest.class))).thenThrow(openSearchException); + + assertThrows(ElasticsearchException.class, () -> createObjectUnderTest().createPit(createPointInTimeRequest)); + } + + @Test + void createPit_throws_runtime_exception_throws_IO_Exception() throws IOException { + final String indexName = UUID.randomUUID().toString(); + final String keepAlive = UUID.randomUUID().toString(); + + final CreatePointInTimeRequest createPointInTimeRequest = mock(CreatePointInTimeRequest.class); + when(createPointInTimeRequest.getIndex()).thenReturn(indexName); + when(createPointInTimeRequest.getKeepAlive()).thenReturn(keepAlive); + + when(elasticSearchClient.openPointInTime(any(OpenPointInTimeRequest.class))).thenThrow(IOException.class); + + assertThrows(RuntimeException.class, () -> createObjectUnderTest().createPit(createPointInTimeRequest)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void delete_pit_with_no_exception_does_not_throw(final boolean successful) throws IOException { + final String pitId = UUID.randomUUID().toString(); + + final DeletePointInTimeRequest deletePointInTimeRequest = mock(DeletePointInTimeRequest.class); + when(deletePointInTimeRequest.getPitId()).thenReturn(pitId); + + final ClosePointInTimeResponse deletePitResponse = mock(ClosePointInTimeResponse.class); + when(deletePitResponse.succeeded()).thenReturn(successful); + + when(elasticSearchClient.closePointInTime(any(ClosePointInTimeRequest.class))).thenReturn(deletePitResponse); + + createObjectUnderTest().deletePit(deletePointInTimeRequest); + } + + @Test + void delete_pit_does_not_throw_during_opensearch_exception() throws IOException { + final String pitId = UUID.randomUUID().toString(); + + final DeletePointInTimeRequest deletePointInTimeRequest = mock(DeletePointInTimeRequest.class); + when(deletePointInTimeRequest.getPitId()).thenReturn(pitId); + + when(elasticSearchClient.closePointInTime(any(ClosePointInTimeRequest.class))).thenThrow(ElasticsearchException.class); + + createObjectUnderTest().deletePit(deletePointInTimeRequest); + } + + @Test + void delete_pit_does_not_throw_exception_when_client_throws_IOException() throws IOException { + final String pitId = UUID.randomUUID().toString(); + + final DeletePointInTimeRequest deletePointInTimeRequest = mock(DeletePointInTimeRequest.class); + when(deletePointInTimeRequest.getPitId()).thenReturn(pitId); + + when(elasticSearchClient.closePointInTime(any(ClosePointInTimeRequest.class))).thenThrow(IOException.class); + + createObjectUnderTest().deletePit(deletePointInTimeRequest); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void search_with_pit_returns_expected_SearchPointInTimeResponse(final boolean hasSearchAfter) throws IOException { + final String pitId = UUID.randomUUID().toString(); + final Integer paginationSize = new Random().nextInt(); + final List searchAfter = Collections.singletonList(UUID.randomUUID().toString()); + + final SearchPointInTimeRequest searchPointInTimeRequest = mock(SearchPointInTimeRequest.class); + when(searchPointInTimeRequest.getPitId()).thenReturn(pitId); + when(searchPointInTimeRequest.getKeepAlive()).thenReturn("1m"); + when(searchPointInTimeRequest.getPaginationSize()).thenReturn(paginationSize); + + if (hasSearchAfter) { + when(searchPointInTimeRequest.getSearchAfter()).thenReturn(searchAfter); + } else { + when(searchPointInTimeRequest.getSearchAfter()).thenReturn(null); + } + + final SearchResponse searchResponse = mock(SearchResponse.class); + final HitsMetadata hitsMetadata = mock(HitsMetadata.class); + final List> hits = new ArrayList<>(); + final Hit firstHit = mock(Hit.class); + when(firstHit.id()).thenReturn(UUID.randomUUID().toString()); + when(firstHit.index()).thenReturn(UUID.randomUUID().toString()); + when(firstHit.source()).thenReturn(mock(ObjectNode.class)); + + final Hit secondHit = mock(Hit.class); + when(secondHit.id()).thenReturn(UUID.randomUUID().toString()); + when(secondHit.index()).thenReturn(UUID.randomUUID().toString()); + when(secondHit.source()).thenReturn(mock(ObjectNode.class)); + when(secondHit.sort()).thenReturn(searchAfter); + + hits.add(firstHit); + hits.add(secondHit); + + when(hitsMetadata.hits()).thenReturn(hits); + when(searchResponse.hits()).thenReturn(hitsMetadata); + + final ArgumentCaptor searchRequestArgumentCaptor = ArgumentCaptor.forClass(SearchRequest.class); + + when(elasticSearchClient.search(searchRequestArgumentCaptor.capture(), eq(ObjectNode.class))).thenReturn(searchResponse); + + final SearchWithSearchAfterResults searchWithSearchAfterResults = createObjectUnderTest().searchWithPit(searchPointInTimeRequest); + + assertThat(searchWithSearchAfterResults, notNullValue()); + assertThat(searchWithSearchAfterResults.getDocuments(), notNullValue()); + assertThat(searchWithSearchAfterResults.getDocuments().size(), equalTo(2)); + + assertThat(searchWithSearchAfterResults.getNextSearchAfter(), equalTo(secondHit.sort())); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void search_without_search_context_returns_expected_SearchPointInTimeResponse(final boolean hasSearchAfter) throws IOException { + final Integer paginationSize = new Random().nextInt(); + final String index = UUID.randomUUID().toString(); + final List searchAfter = Collections.singletonList(UUID.randomUUID().toString()); + + final NoSearchContextSearchRequest noSearchContextSearchRequest = mock(NoSearchContextSearchRequest.class); + when(noSearchContextSearchRequest.getPaginationSize()).thenReturn(paginationSize); + when(noSearchContextSearchRequest.getIndex()).thenReturn(index); + + if (hasSearchAfter) { + when(noSearchContextSearchRequest.getSearchAfter()).thenReturn(searchAfter); + } else { + when(noSearchContextSearchRequest.getSearchAfter()).thenReturn(null); + } + + final SearchResponse searchResponse = mock(SearchResponse.class); + final HitsMetadata hitsMetadata = mock(HitsMetadata.class); + final List> hits = new ArrayList<>(); + final Hit firstHit = mock(Hit.class); + when(firstHit.id()).thenReturn(UUID.randomUUID().toString()); + when(firstHit.index()).thenReturn(index); + when(firstHit.source()).thenReturn(mock(ObjectNode.class)); + + final Hit secondHit = mock(Hit.class); + when(secondHit.id()).thenReturn(UUID.randomUUID().toString()); + when(secondHit.index()).thenReturn(index); + when(secondHit.source()).thenReturn(mock(ObjectNode.class)); + when(secondHit.sort()).thenReturn(searchAfter); + + hits.add(firstHit); + hits.add(secondHit); + + when(hitsMetadata.hits()).thenReturn(hits); + when(searchResponse.hits()).thenReturn(hitsMetadata); + + final ArgumentCaptor searchRequestArgumentCaptor = ArgumentCaptor.forClass(SearchRequest.class); + + when(elasticSearchClient.search(searchRequestArgumentCaptor.capture(), eq(ObjectNode.class))).thenReturn(searchResponse); + + final SearchWithSearchAfterResults searchWithSearchAfterResults = createObjectUnderTest().searchWithoutSearchContext(noSearchContextSearchRequest); + + assertThat(searchWithSearchAfterResults, notNullValue()); + assertThat(searchWithSearchAfterResults.getDocuments(), notNullValue()); + assertThat(searchWithSearchAfterResults.getDocuments().size(), equalTo(2)); + + assertThat(searchWithSearchAfterResults.getNextSearchAfter(), equalTo(secondHit.sort())); + } +} 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 new file mode 100644 index 0000000000..cc811625d1 --- /dev/null +++ b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactoryTest.java @@ -0,0 +1,122 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.opensearch.worker.client; + +import co.elastic.clients.elasticsearch.ElasticsearchClient; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.client.opensearch.OpenSearchClient; +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.source.opensearch.OpenSearchSourceConfiguration; +import org.opensearch.dataprepper.plugins.source.opensearch.configuration.AwsAuthenticationConfiguration; +import org.opensearch.dataprepper.plugins.source.opensearch.configuration.ConnectionConfiguration; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.regions.Region; + +import java.util.Collections; +import java.util.List; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class OpenSearchClientFactoryTest { + + @Mock + private AwsCredentialsSupplier awsCredentialsSupplier; + + @Mock + private OpenSearchSourceConfiguration openSearchSourceConfiguration; + + @Mock + private ConnectionConfiguration connectionConfiguration; + + @BeforeEach + void setup() { + when(openSearchSourceConfiguration.getHosts()).thenReturn(List.of("http://localhost:9200")); + when(openSearchSourceConfiguration.getConnectionConfiguration()).thenReturn(connectionConfiguration); + } + + private OpenSearchClientFactory createObjectUnderTest() { + return OpenSearchClientFactory.create(awsCredentialsSupplier); + } + + @Test + void provideOpenSearchClient_with_username_and_password() { + final String username = UUID.randomUUID().toString(); + final String password = UUID.randomUUID().toString(); + when(openSearchSourceConfiguration.getUsername()).thenReturn(username); + when(openSearchSourceConfiguration.getPassword()).thenReturn(password); + + when(connectionConfiguration.getCertPath()).thenReturn(null); + when(connectionConfiguration.getSocketTimeout()).thenReturn(null); + when(connectionConfiguration.getConnectTimeout()).thenReturn(null); + when(connectionConfiguration.isInsecure()).thenReturn(true); + + when(openSearchSourceConfiguration.getAwsAuthenticationOptions()).thenReturn(null); + + final OpenSearchClient openSearchClient = createObjectUnderTest().provideOpenSearchClient(openSearchSourceConfiguration); + assertThat(openSearchClient, notNullValue()); + + verifyNoInteractions(awsCredentialsSupplier); + + } + + @Test + void provideElasticSearchClient_with_username_and_password() { + final String username = UUID.randomUUID().toString(); + final String password = UUID.randomUUID().toString(); + when(openSearchSourceConfiguration.getUsername()).thenReturn(username); + when(openSearchSourceConfiguration.getPassword()).thenReturn(password); + + 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); + } + + @Test + void provideOpenSearchClient_with_aws_auth() { + when(connectionConfiguration.getCertPath()).thenReturn(null); + when(connectionConfiguration.getSocketTimeout()).thenReturn(null); + when(connectionConfiguration.getConnectTimeout()).thenReturn(null); + + final AwsAuthenticationConfiguration awsAuthenticationConfiguration = mock(AwsAuthenticationConfiguration.class); + when(awsAuthenticationConfiguration.getAwsRegion()).thenReturn(Region.US_EAST_1); + final String stsRoleArn = "arn:aws:iam::123456789012:role/my-role"; + when(awsAuthenticationConfiguration.getAwsStsRoleArn()).thenReturn(stsRoleArn); + when(awsAuthenticationConfiguration.getAwsStsHeaderOverrides()).thenReturn(Collections.emptyMap()); + when(openSearchSourceConfiguration.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationConfiguration); + + final ArgumentCaptor awsCredentialsOptionsArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsOptions.class); + final AwsCredentialsProvider awsCredentialsProvider = mock(AwsCredentialsProvider.class); + when(awsCredentialsSupplier.getProvider(awsCredentialsOptionsArgumentCaptor.capture())).thenReturn(awsCredentialsProvider); + + final OpenSearchClient openSearchClient = createObjectUnderTest().provideOpenSearchClient(openSearchSourceConfiguration); + assertThat(openSearchClient, notNullValue()); + + final AwsCredentialsOptions awsCredentialsOptions = awsCredentialsOptionsArgumentCaptor.getValue(); + assertThat(awsCredentialsOptions, notNullValue()); + assertThat(awsCredentialsOptions.getRegion(), equalTo(Region.US_EAST_1)); + assertThat(awsCredentialsOptions.getStsHeaderOverrides(), equalTo(Collections.emptyMap())); + assertThat(awsCredentialsOptions.getStsRoleArn(), equalTo(stsRoleArn)); + } +} diff --git a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchIndexPartitionCreationSupplierTest.java b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchIndexPartitionCreationSupplierTest.java index 0a424b0610..3c0ed9628a 100644 --- a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchIndexPartitionCreationSupplierTest.java +++ b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchIndexPartitionCreationSupplierTest.java @@ -5,6 +5,9 @@ package org.opensearch.dataprepper.plugins.source.opensearch.worker.client; +import co.elastic.clients.elasticsearch.ElasticsearchClient; +import co.elastic.clients.elasticsearch._types.ElasticsearchException; +import co.elastic.clients.elasticsearch.cat.ElasticsearchCatClient; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; @@ -49,6 +52,9 @@ public class OpenSearchIndexPartitionCreationSupplierTest { @Mock private OpenSearchClient openSearchClient; + @Mock + private ElasticsearchClient elasticsearchClient; + private OpenSearchIndexPartitionCreationSupplier createObjectUnderTest() { return new OpenSearchIndexPartitionCreationSupplier(openSearchSourceConfiguration, clusterClientFactory); } @@ -75,6 +81,21 @@ void apply_with_opensearch_client_cat_indices_throws_exception_returns_empty_lis assertThat(partitionIdentifierList.isEmpty(), equalTo(true)); } + @ParameterizedTest + @MethodSource("elasticsearchCatIndicesExceptions") + void apply_with_elasticsearch_client_cat_indices_throws_exception_returns_empty_list(final Class exception) throws IOException { + when(clusterClientFactory.getClient()).thenReturn(elasticsearchClient); + + final ElasticsearchCatClient elasticsearchCatClient = mock(ElasticsearchCatClient.class); + when(elasticsearchCatClient.indices()).thenThrow(exception); + when(elasticsearchClient.cat()).thenReturn(elasticsearchCatClient); + + final List partitionIdentifierList = createObjectUnderTest().apply(Collections.emptyMap()); + + assertThat(partitionIdentifierList, notNullValue()); + assertThat(partitionIdentifierList.isEmpty(), equalTo(true)); + } + @Test void apply_with_opensearch_client_with_no_indices_return_empty_list() throws IOException { when(clusterClientFactory.getClient()).thenReturn(openSearchClient); @@ -91,6 +112,22 @@ void apply_with_opensearch_client_with_no_indices_return_empty_list() throws IOE assertThat(partitionIdentifierList.isEmpty(), equalTo(true)); } + @Test + void apply_with_elasticsearch_client_with_no_indices_return_empty_list() throws IOException { + when(clusterClientFactory.getClient()).thenReturn(elasticsearchClient); + + final ElasticsearchCatClient elasticsearchCatClient = mock(ElasticsearchCatClient.class); + final co.elastic.clients.elasticsearch.cat.IndicesResponse indicesResponse = mock(co.elastic.clients.elasticsearch.cat.IndicesResponse.class); + when(indicesResponse.valueBody()).thenReturn(Collections.emptyList()); + when(elasticsearchCatClient.indices()).thenReturn(indicesResponse); + when(elasticsearchClient.cat()).thenReturn(elasticsearchCatClient); + + final List partitionIdentifierList = createObjectUnderTest().apply(Collections.emptyMap()); + + assertThat(partitionIdentifierList, notNullValue()); + assertThat(partitionIdentifierList.isEmpty(), equalTo(true)); + } + @Test void apply_with_opensearch_client_with_indices_filters_them_correctly() throws IOException { when(clusterClientFactory.getClient()).thenReturn(openSearchClient); @@ -146,8 +183,68 @@ void apply_with_opensearch_client_with_indices_filters_them_correctly() throws I assertThat(partitionIdentifierList, notNullValue()); } + @Test + void apply_with_elasticsearch_client_with_indices_filters_them_correctly() throws IOException { + when(clusterClientFactory.getClient()).thenReturn(elasticsearchClient); + + final ElasticsearchCatClient elasticsearchCatClient = mock(ElasticsearchCatClient.class); + final co.elastic.clients.elasticsearch.cat.IndicesResponse indicesResponse = mock(co.elastic.clients.elasticsearch.cat.IndicesResponse.class); + + final IndexParametersConfiguration indexParametersConfiguration = mock(IndexParametersConfiguration.class); + + final List includedIndices = new ArrayList<>(); + final OpenSearchIndex includeIndex = mock(OpenSearchIndex.class); + final String includePattern = "my-pattern-[a-c].*"; + when(includeIndex.getIndexNamePattern()).thenReturn(Pattern.compile(includePattern)); + includedIndices.add(includeIndex); + + final List excludedIndices = new ArrayList<>(); + final OpenSearchIndex excludeIndex = mock(OpenSearchIndex.class); + final String excludePattern = "my-pattern-[a-c]-exclude"; + when(excludeIndex.getIndexNamePattern()).thenReturn(Pattern.compile(excludePattern)); + excludedIndices.add(excludeIndex); + + final OpenSearchIndex secondExcludeIndex = mock(OpenSearchIndex.class); + final String secondExcludePattern = "second-exclude-.*"; + when(secondExcludeIndex.getIndexNamePattern()).thenReturn(Pattern.compile(secondExcludePattern)); + excludedIndices.add(secondExcludeIndex); + + when(indexParametersConfiguration.getIncludedIndices()).thenReturn(includedIndices); + when(indexParametersConfiguration.getExcludedIndices()).thenReturn(excludedIndices); + when(openSearchSourceConfiguration.getIndexParametersConfiguration()).thenReturn(indexParametersConfiguration); + + final List indicesRecords = new ArrayList<>(); + final co.elastic.clients.elasticsearch.cat.indices.IndicesRecord includedIndex = mock(co.elastic.clients.elasticsearch.cat.indices.IndicesRecord.class); + when(includedIndex.index()).thenReturn("my-pattern-a-include"); + final co.elastic.clients.elasticsearch.cat.indices.IndicesRecord excludedIndex = mock(co.elastic.clients.elasticsearch.cat.indices.IndicesRecord.class); + when(excludedIndex.index()).thenReturn("second-exclude-test"); + final co.elastic.clients.elasticsearch.cat.indices.IndicesRecord includedAndThenExcluded = mock(co.elastic.clients.elasticsearch.cat.indices.IndicesRecord.class); + when(includedAndThenExcluded.index()).thenReturn("my-pattern-a-exclude"); + final co.elastic.clients.elasticsearch.cat.indices.IndicesRecord neitherIncludedOrExcluded = mock(co.elastic.clients.elasticsearch.cat.indices.IndicesRecord.class); + when(neitherIncludedOrExcluded.index()).thenReturn("random-index"); + + indicesRecords.add(includedIndex); + indicesRecords.add(excludedIndex); + indicesRecords.add(includedAndThenExcluded); + indicesRecords.add(neitherIncludedOrExcluded); + + when(indicesResponse.valueBody()).thenReturn(indicesRecords); + + when(elasticsearchCatClient.indices()).thenReturn(indicesResponse); + when(elasticsearchClient.cat()).thenReturn(elasticsearchCatClient); + + final List partitionIdentifierList = createObjectUnderTest().apply(Collections.emptyMap()); + + assertThat(partitionIdentifierList, notNullValue()); + } + private static Stream opensearchCatIndicesExceptions() { return Stream.of(Arguments.of(IOException.class), Arguments.of(OpenSearchException.class)); } + + private static Stream elasticsearchCatIndicesExceptions() { + return Stream.of(Arguments.of(IOException.class), + Arguments.of(ElasticsearchException.class)); + } } diff --git a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/SearchAccessStrategyTest.java b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/SearchAccessStrategyTest.java index 582c9f4652..e30ec3559e 100644 --- a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/SearchAccessStrategyTest.java +++ b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/SearchAccessStrategyTest.java @@ -5,38 +5,29 @@ package org.opensearch.dataprepper.plugins.source.opensearch.worker.client; -import org.junit.jupiter.api.BeforeEach; +import co.elastic.clients.elasticsearch.ElasticsearchClient; +import co.elastic.clients.elasticsearch._types.ElasticsearchVersionInfo; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; -import org.mockito.ArgumentCaptor; import org.mockito.Mock; -import org.mockito.MockedConstruction; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.client.opensearch.OpenSearchClient; import org.opensearch.client.opensearch._types.OpenSearchVersionInfo; import org.opensearch.client.opensearch.core.InfoResponse; -import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; -import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.client.util.MissingRequiredPropertyException; import org.opensearch.dataprepper.plugins.source.opensearch.OpenSearchSourceConfiguration; -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.SearchConfiguration; import org.opensearch.dataprepper.plugins.source.opensearch.worker.client.model.SearchContextType; -import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; -import software.amazon.awssdk.regions.Region; -import java.util.Collections; -import java.util.List; -import java.util.UUID; +import java.io.IOException; 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; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.mockConstruction; -import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; import static org.opensearch.dataprepper.plugins.source.opensearch.worker.client.SearchAccessorStrategy.OPENSEARCH_DISTRIBUTION; @@ -44,79 +35,97 @@ public class SearchAccessStrategyTest { @Mock - private AwsCredentialsSupplier awsCredentialsSupplier; + private OpenSearchClientFactory openSearchClientFactory; @Mock private OpenSearchSourceConfiguration openSearchSourceConfiguration; - @Mock - private ConnectionConfiguration connectionConfiguration; - - @BeforeEach - void setup() { - when(openSearchSourceConfiguration.getHosts()).thenReturn(List.of("http://localhost:9200")); - when(openSearchSourceConfiguration.getConnectionConfiguration()).thenReturn(connectionConfiguration); - } - private SearchAccessorStrategy createObjectUnderTest() { - return SearchAccessorStrategy.create(openSearchSourceConfiguration, awsCredentialsSupplier); + return SearchAccessorStrategy.create(openSearchSourceConfiguration, openSearchClientFactory); } @ParameterizedTest @ValueSource(strings = {"2.5.0", "2.6.1", "3.0.0"}) - void testHappyPath_with_username_and_password_and_insecure_for_different_point_in_time_versions_for_opensearch(final String osVersion) { - final String username = UUID.randomUUID().toString(); - final String password = UUID.randomUUID().toString(); - when(openSearchSourceConfiguration.getUsername()).thenReturn(username); - when(openSearchSourceConfiguration.getPassword()).thenReturn(password); - - when(connectionConfiguration.getCertPath()).thenReturn(null); - when(connectionConfiguration.getSocketTimeout()).thenReturn(null); - when(connectionConfiguration.getConnectTimeout()).thenReturn(null); - when(connectionConfiguration.isInsecure()).thenReturn(true); + void testHappyPath_for_different_point_in_time_versions_for_opensearch(final String osVersion) throws IOException { final SearchConfiguration searchConfiguration = mock(SearchConfiguration.class); when(searchConfiguration.getSearchContextType()).thenReturn(null); when(openSearchSourceConfiguration.getSearchConfiguration()).thenReturn(searchConfiguration); - when(openSearchSourceConfiguration.getAwsAuthenticationOptions()).thenReturn(null); - final InfoResponse infoResponse = mock(InfoResponse.class); final OpenSearchVersionInfo openSearchVersionInfo = mock(OpenSearchVersionInfo.class); when(openSearchVersionInfo.distribution()).thenReturn(OPENSEARCH_DISTRIBUTION); when(openSearchVersionInfo.number()).thenReturn(osVersion); when(infoResponse.version()).thenReturn(openSearchVersionInfo); - try (MockedConstruction openSearchClientMockedConstruction = mockConstruction(OpenSearchClient.class, - (clientMock, context) -> { - when(clientMock.info()).thenReturn(infoResponse); - })) { + final OpenSearchClient openSearchClient = mock(OpenSearchClient.class); + when(openSearchClient.info()).thenReturn(infoResponse); + when(openSearchClientFactory.provideOpenSearchClient(openSearchSourceConfiguration)).thenReturn(openSearchClient); + + final SearchAccessor searchAccessor = createObjectUnderTest().getSearchAccessor(); + assertThat(searchAccessor, notNullValue()); + assertThat(searchAccessor.getSearchContextType(), equalTo(SearchContextType.POINT_IN_TIME)); + } + + @ParameterizedTest + @ValueSource(strings = {"7.10.2", "8.1.1", "7.10.0"}) + void testHappyPath_for_different_point_in_time_versions_for_elasticsearch(final String esVersion) throws IOException { + + final SearchConfiguration searchConfiguration = mock(SearchConfiguration.class); + when(searchConfiguration.getSearchContextType()).thenReturn(null); + when(openSearchSourceConfiguration.getSearchConfiguration()).thenReturn(searchConfiguration); + + final OpenSearchClient openSearchClient = mock(OpenSearchClient.class); + when(openSearchClient.info()).thenThrow(MissingRequiredPropertyException.class); + when(openSearchClientFactory.provideOpenSearchClient(openSearchSourceConfiguration)).thenReturn(openSearchClient); + + final ElasticsearchClient elasticsearchClient = mock(ElasticsearchClient.class); + + final co.elastic.clients.elasticsearch.core.InfoResponse infoResponse = mock(co.elastic.clients.elasticsearch.core.InfoResponse.class); + final ElasticsearchVersionInfo elasticsearchVersionInfo = mock(ElasticsearchVersionInfo.class); + when(elasticsearchVersionInfo.buildFlavor()).thenReturn("default"); + when(elasticsearchVersionInfo.number()).thenReturn(esVersion); + when(infoResponse.version()).thenReturn(elasticsearchVersionInfo); + + when(elasticsearchClient.info()).thenReturn(infoResponse); + when(openSearchClientFactory.provideElasticSearchClient(openSearchSourceConfiguration)).thenReturn(elasticsearchClient); + + final SearchAccessor searchAccessor = createObjectUnderTest().getSearchAccessor(); + assertThat(searchAccessor, notNullValue()); + assertThat(searchAccessor.getSearchContextType(), equalTo(SearchContextType.POINT_IN_TIME)); + + } + + @ParameterizedTest + @CsvSource(value = {"6.3.0,default", "7.9.0,default", "0.3.2,default", "7.10.2,oss"}) + void search_context_type_set_to_point_in_time_with_invalid_version_throws_IllegalArgumentException_for_elasticsearch(final String esVersion, final String esBuildFlavor) throws IOException { + + final OpenSearchClient openSearchClient = mock(OpenSearchClient.class); + when(openSearchClient.info()).thenThrow(MissingRequiredPropertyException.class); + when(openSearchClientFactory.provideOpenSearchClient(openSearchSourceConfiguration)).thenReturn(openSearchClient); - final SearchAccessor searchAccessor = createObjectUnderTest().getSearchAccessor(); - assertThat(searchAccessor, notNullValue()); - assertThat(searchAccessor.getSearchContextType(), equalTo(SearchContextType.POINT_IN_TIME)); + final ElasticsearchClient elasticsearchClient = mock(ElasticsearchClient.class); - final List constructedClients = openSearchClientMockedConstruction.constructed(); - assertThat(constructedClients.size(), equalTo(1)); - } + final co.elastic.clients.elasticsearch.core.InfoResponse infoResponse = mock(co.elastic.clients.elasticsearch.core.InfoResponse.class); + final ElasticsearchVersionInfo elasticsearchVersionInfo = mock(ElasticsearchVersionInfo.class); + when(elasticsearchVersionInfo.buildFlavor()).thenReturn(esBuildFlavor); + when(elasticsearchVersionInfo.number()).thenReturn(esVersion); + when(infoResponse.version()).thenReturn(elasticsearchVersionInfo); - verifyNoInteractions(awsCredentialsSupplier); + final SearchConfiguration searchConfiguration = mock(SearchConfiguration.class); + when(searchConfiguration.getSearchContextType()).thenReturn(SearchContextType.POINT_IN_TIME); + when(openSearchSourceConfiguration.getSearchConfiguration()).thenReturn(searchConfiguration); + when(elasticsearchClient.info()).thenReturn(infoResponse); + when(openSearchClientFactory.provideElasticSearchClient(openSearchSourceConfiguration)).thenReturn(elasticsearchClient); + + + assertThrows(IllegalArgumentException.class, () -> createObjectUnderTest().getSearchAccessor()); } @ParameterizedTest @ValueSource(strings = {"1.3.0", "2.4.9", "0.3.2"}) - void testHappyPath_with_aws_credentials_for_different_scroll_versions_for_opensearch(final String osVersion) { - when(connectionConfiguration.getCertPath()).thenReturn(null); - when(connectionConfiguration.getSocketTimeout()).thenReturn(null); - when(connectionConfiguration.getConnectTimeout()).thenReturn(null); - - final AwsAuthenticationConfiguration awsAuthenticationConfiguration = mock(AwsAuthenticationConfiguration.class); - when(awsAuthenticationConfiguration.getAwsRegion()).thenReturn(Region.US_EAST_1); - final String stsRoleArn = "arn:aws:iam::123456789012:role/my-role"; - when(awsAuthenticationConfiguration.getAwsStsRoleArn()).thenReturn(stsRoleArn); - when(awsAuthenticationConfiguration.getAwsStsHeaderOverrides()).thenReturn(Collections.emptyMap()); - when(openSearchSourceConfiguration.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationConfiguration); + void testHappyPath_with_for_different_scroll_versions_for_opensearch(final String osVersion) throws IOException { final SearchConfiguration searchConfiguration = mock(SearchConfiguration.class); when(searchConfiguration.getSearchContextType()).thenReturn(null); @@ -128,43 +137,18 @@ void testHappyPath_with_aws_credentials_for_different_scroll_versions_for_opense when(openSearchVersionInfo.number()).thenReturn(osVersion); when(infoResponse.version()).thenReturn(openSearchVersionInfo); - final ArgumentCaptor awsCredentialsOptionsArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsOptions.class); - final AwsCredentialsProvider awsCredentialsProvider = mock(AwsCredentialsProvider.class); - when(awsCredentialsSupplier.getProvider(awsCredentialsOptionsArgumentCaptor.capture())).thenReturn(awsCredentialsProvider); - - try (MockedConstruction openSearchClientMockedConstruction = mockConstruction(OpenSearchClient.class, - (clientMock, context) -> { - when(clientMock.info()).thenReturn(infoResponse); - })) { - - final SearchAccessor searchAccessor = createObjectUnderTest().getSearchAccessor(); - assertThat(searchAccessor, notNullValue()); - assertThat(searchAccessor.getSearchContextType(), equalTo(SearchContextType.SCROLL)); - - final List constructedClients = openSearchClientMockedConstruction.constructed(); - assertThat(constructedClients.size(), equalTo(1)); - } + final OpenSearchClient openSearchClient = mock(OpenSearchClient.class); + when(openSearchClient.info()).thenReturn(infoResponse); + when(openSearchClientFactory.provideOpenSearchClient(openSearchSourceConfiguration)).thenReturn(openSearchClient); - final AwsCredentialsOptions awsCredentialsOptions = awsCredentialsOptionsArgumentCaptor.getValue(); - assertThat(awsCredentialsOptions, notNullValue()); - assertThat(awsCredentialsOptions.getRegion(), equalTo(Region.US_EAST_1)); - assertThat(awsCredentialsOptions.getStsHeaderOverrides(), equalTo(Collections.emptyMap())); - assertThat(awsCredentialsOptions.getStsRoleArn(), equalTo(stsRoleArn)); + final SearchAccessor searchAccessor = createObjectUnderTest().getSearchAccessor(); + assertThat(searchAccessor, notNullValue()); + assertThat(searchAccessor.getSearchContextType(), equalTo(SearchContextType.SCROLL)); } @ParameterizedTest @ValueSource(strings = {"1.3.0", "2.4.9", "0.3.2"}) - void search_context_type_set_to_point_in_time_with_invalid_version_throws_IllegalArgumentException(final String osVersion) { - when(connectionConfiguration.getCertPath()).thenReturn(null); - when(connectionConfiguration.getSocketTimeout()).thenReturn(null); - when(connectionConfiguration.getConnectTimeout()).thenReturn(null); - - final AwsAuthenticationConfiguration awsAuthenticationConfiguration = mock(AwsAuthenticationConfiguration.class); - when(awsAuthenticationConfiguration.getAwsRegion()).thenReturn(Region.US_EAST_1); - final String stsRoleArn = "arn:aws:iam::123456789012:role/my-role"; - when(awsAuthenticationConfiguration.getAwsStsRoleArn()).thenReturn(stsRoleArn); - when(awsAuthenticationConfiguration.getAwsStsHeaderOverrides()).thenReturn(Collections.emptyMap()); - when(openSearchSourceConfiguration.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationConfiguration); + void search_context_type_set_to_point_in_time_with_invalid_version_throws_IllegalArgumentException_for_opensearch(final String osVersion) throws IOException { final InfoResponse infoResponse = mock(InfoResponse.class); final OpenSearchVersionInfo openSearchVersionInfo = mock(OpenSearchVersionInfo.class); @@ -172,45 +156,20 @@ void search_context_type_set_to_point_in_time_with_invalid_version_throws_Illega when(openSearchVersionInfo.number()).thenReturn(osVersion); when(infoResponse.version()).thenReturn(openSearchVersionInfo); + final OpenSearchClient openSearchClient = mock(OpenSearchClient.class); + when(openSearchClient.info()).thenReturn(infoResponse); + when(openSearchClientFactory.provideOpenSearchClient(openSearchSourceConfiguration)).thenReturn(openSearchClient); + final SearchConfiguration searchConfiguration = mock(SearchConfiguration.class); when(searchConfiguration.getSearchContextType()).thenReturn(SearchContextType.POINT_IN_TIME); when(openSearchSourceConfiguration.getSearchConfiguration()).thenReturn(searchConfiguration); - final ArgumentCaptor awsCredentialsOptionsArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsOptions.class); - final AwsCredentialsProvider awsCredentialsProvider = mock(AwsCredentialsProvider.class); - when(awsCredentialsSupplier.getProvider(awsCredentialsOptionsArgumentCaptor.capture())).thenReturn(awsCredentialsProvider); - - try (MockedConstruction openSearchClientMockedConstruction = mockConstruction(OpenSearchClient.class, - (clientMock, context) -> { - when(clientMock.info()).thenReturn(infoResponse); - })) { - - assertThrows(IllegalArgumentException.class, () -> createObjectUnderTest().getSearchAccessor()); - - final List constructedClients = openSearchClientMockedConstruction.constructed(); - assertThat(constructedClients.size(), equalTo(1)); - } - - final AwsCredentialsOptions awsCredentialsOptions = awsCredentialsOptionsArgumentCaptor.getValue(); - assertThat(awsCredentialsOptions, notNullValue()); - assertThat(awsCredentialsOptions.getRegion(), equalTo(Region.US_EAST_1)); - assertThat(awsCredentialsOptions.getStsHeaderOverrides(), equalTo(Collections.emptyMap())); - assertThat(awsCredentialsOptions.getStsRoleArn(), equalTo(stsRoleArn)); + assertThrows(IllegalArgumentException.class, () -> createObjectUnderTest().getSearchAccessor()); } @ParameterizedTest @ValueSource(strings = {"1.3.0", "2.4.9", "2.5.0"}) - void search_context_type_set_to_none_uses_that_search_context_regardless_of_version(final String osVersion) { - when(connectionConfiguration.getCertPath()).thenReturn(null); - when(connectionConfiguration.getSocketTimeout()).thenReturn(null); - when(connectionConfiguration.getConnectTimeout()).thenReturn(null); - - final AwsAuthenticationConfiguration awsAuthenticationConfiguration = mock(AwsAuthenticationConfiguration.class); - when(awsAuthenticationConfiguration.getAwsRegion()).thenReturn(Region.US_EAST_1); - final String stsRoleArn = "arn:aws:iam::123456789012:role/my-role"; - when(awsAuthenticationConfiguration.getAwsStsRoleArn()).thenReturn(stsRoleArn); - when(awsAuthenticationConfiguration.getAwsStsHeaderOverrides()).thenReturn(Collections.emptyMap()); - when(openSearchSourceConfiguration.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationConfiguration); + void search_context_type_set_to_none_uses_that_search_context_regardless_of_version(final String osVersion) throws IOException { final InfoResponse infoResponse = mock(InfoResponse.class); final OpenSearchVersionInfo openSearchVersionInfo = mock(OpenSearchVersionInfo.class); @@ -218,31 +177,16 @@ void search_context_type_set_to_none_uses_that_search_context_regardless_of_vers when(openSearchVersionInfo.number()).thenReturn(osVersion); when(infoResponse.version()).thenReturn(openSearchVersionInfo); + final OpenSearchClient openSearchClient = mock(OpenSearchClient.class); + when(openSearchClient.info()).thenReturn(infoResponse); + when(openSearchClientFactory.provideOpenSearchClient(openSearchSourceConfiguration)).thenReturn(openSearchClient); + final SearchConfiguration searchConfiguration = mock(SearchConfiguration.class); when(searchConfiguration.getSearchContextType()).thenReturn(SearchContextType.NONE); when(openSearchSourceConfiguration.getSearchConfiguration()).thenReturn(searchConfiguration); - final ArgumentCaptor awsCredentialsOptionsArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsOptions.class); - final AwsCredentialsProvider awsCredentialsProvider = mock(AwsCredentialsProvider.class); - when(awsCredentialsSupplier.getProvider(awsCredentialsOptionsArgumentCaptor.capture())).thenReturn(awsCredentialsProvider); - - try (MockedConstruction openSearchClientMockedConstruction = mockConstruction(OpenSearchClient.class, - (clientMock, context) -> { - when(clientMock.info()).thenReturn(infoResponse); - })) { - - final SearchAccessor searchAccessor = createObjectUnderTest().getSearchAccessor(); - assertThat(searchAccessor, notNullValue()); - assertThat(searchAccessor.getSearchContextType(), equalTo(SearchContextType.NONE)); - - final List constructedClients = openSearchClientMockedConstruction.constructed(); - assertThat(constructedClients.size(), equalTo(1)); - } - - final AwsCredentialsOptions awsCredentialsOptions = awsCredentialsOptionsArgumentCaptor.getValue(); - assertThat(awsCredentialsOptions, notNullValue()); - assertThat(awsCredentialsOptions.getRegion(), equalTo(Region.US_EAST_1)); - assertThat(awsCredentialsOptions.getStsHeaderOverrides(), equalTo(Collections.emptyMap())); - assertThat(awsCredentialsOptions.getStsRoleArn(), equalTo(stsRoleArn)); + final SearchAccessor searchAccessor = createObjectUnderTest().getSearchAccessor(); + assertThat(searchAccessor, notNullValue()); + assertThat(searchAccessor.getSearchContextType(), equalTo(SearchContextType.NONE)); } } From 4387b294fcec3c80f2bba3280a64a987564f00ee Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo <95880281+MaGonzalMayedo@users.noreply.github.com> Date: Wed, 21 Jun 2023 15:03:43 -0700 Subject: [PATCH 02/43] GitHub-Issue#2778: Refactoring config files for CloudWatchLogs Sink (#4) Added Config Files for CloudWatchLogs Sink. Signed-off-by: Marcos Gonzalez Mayedo --- data-prepper-plugins/cwl-sink/.gitignore | 42 ++++++++++++++ data-prepper-plugins/cwl-sink/build.gradle | 42 ++++++++++++++ .../java/org/opensearch/dataprepper/Main.java | 8 +++ .../plugins/sink/config/CwlSinkConfig.java | 54 ++++++++++++++++++ .../plugins/sink/config/ThresholdConfig.java | 57 +++++++++++++++++++ .../sink/configuration/CwlSinkConfigTest.java | 44 ++++++++++++++ .../configuration/ThresholdConfigTest.java | 38 +++++++++++++ settings.gradle | 1 + 8 files changed, 286 insertions(+) create mode 100644 data-prepper-plugins/cwl-sink/.gitignore create mode 100644 data-prepper-plugins/cwl-sink/build.gradle create mode 100644 data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/Main.java create mode 100644 data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java create mode 100644 data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java create mode 100644 data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java create mode 100644 data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java diff --git a/data-prepper-plugins/cwl-sink/.gitignore b/data-prepper-plugins/cwl-sink/.gitignore new file mode 100644 index 0000000000..b63da4551b --- /dev/null +++ b/data-prepper-plugins/cwl-sink/.gitignore @@ -0,0 +1,42 @@ +.gradle +build/ +!gradle/wrapper/gradle-wrapper.jar +!**/src/main/**/build/ +!**/src/test/**/build/ + +### IntelliJ IDEA ### +.idea/modules.xml +.idea/jarRepositories.xml +.idea/compiler.xml +.idea/libraries/ +*.iws +*.iml +*.ipr +out/ +!**/src/main/**/out/ +!**/src/test/**/out/ + +### Eclipse ### +.apt_generated +.classpath +.factorypath +.project +.settings +.springBeans +.sts4-cache +bin/ +!**/src/main/**/bin/ +!**/src/test/**/bin/ + +### NetBeans ### +/nbproject/private/ +/nbbuild/ +/dist/ +/nbdist/ +/.nb-gradle/ + +### VS Code ### +.vscode/ + +### Mac OS ### +.DS_Store \ No newline at end of file diff --git a/data-prepper-plugins/cwl-sink/build.gradle b/data-prepper-plugins/cwl-sink/build.gradle new file mode 100644 index 0000000000..bcea1ce923 --- /dev/null +++ b/data-prepper-plugins/cwl-sink/build.gradle @@ -0,0 +1,42 @@ +plugins { + id 'java' + id 'java-library' +} + +group = 'org.opensearch.dataprepper' +version = '2.3.0-SNAPSHOT' + +repositories { + mavenCentral() +} + +dependencies { + api project(':data-prepper-api') + implementation project(':data-prepper-plugins:aws-plugin-api') + implementation project(path: ':data-prepper-plugins:s3-sink') + testImplementation platform('org.junit:junit-bom:5.9.1') + implementation platform('software.amazon.awssdk:bom:2.20.56') + implementation project(path: ':data-prepper-plugins:common') + testImplementation 'org.junit.jupiter:junit-jupiter' + implementation 'com.fasterxml.jackson.core:jackson-core' + implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' + implementation 'software.amazon.awssdk:cloudwatch' + implementation 'software.amazon.awssdk:cloudwatchlogs' + implementation 'org.apache.commons:commons-lang3:3.12.0' +} + +jacocoTestCoverageVerification { + dependsOn jacocoTestReport + violationRules { + rule { //in addition to core projects rule + limit { + minimum = 0.90 + } + } + } +} + +test { + useJUnitPlatform() +} \ No newline at end of file diff --git a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/Main.java b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/Main.java new file mode 100644 index 0000000000..cee3b572c6 --- /dev/null +++ b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/Main.java @@ -0,0 +1,8 @@ +package org.opensearch.dataprepper; + +public class Main { + private String path; + public static void main(String[] args) { + System.out.println("Hello world!"); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java new file mode 100644 index 0000000000..22a00c631c --- /dev/null +++ b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java @@ -0,0 +1,54 @@ +package org.opensearch.dataprepper.plugins.sink.config; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotEmpty; +import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.plugins.sink.configuration.AwsAuthenticationOptions; + +public class CwlSinkConfig { + public static final String DEFAULT_BUFFER_TYPE = "in_memory"; + + //Class was utilized from the + @JsonProperty("aws_config") + @NotNull + @Valid + private AwsAuthenticationOptions awsConfig; + + @JsonProperty("threshold_config") + @NotNull + private ThresholdConfig thresholdConfig; + + @JsonProperty("buffer_type") + private String bufferType = DEFAULT_BUFFER_TYPE; + + @JsonProperty("log_group") + @NotEmpty + @NotNull + private String logGroup; + + @JsonProperty("log_stream") + @NotEmpty + @NotNull + private String logStream; + + public AwsAuthenticationOptions getAwsConfig() { + return awsConfig; + } + + public ThresholdConfig getThresholdConfig() { + return thresholdConfig; + } + + public String getBufferType() { + return bufferType; + } + + public String getLogGroup() { + return logGroup; + } + + public String getLogStream() { + return logStream; + } +} diff --git a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java new file mode 100644 index 0000000000..e0eacadda6 --- /dev/null +++ b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java @@ -0,0 +1,57 @@ +package org.opensearch.dataprepper.plugins.sink.config; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; + +/** + * The threshold config holds the different configurations for + * buffer restrictions, retransmission restrictions and timeout + * restrictions. + */ +public class ThresholdConfig { + public static final int DEFAULT_BATCH_SIZE = 10; + public static final int DEFAULT_EVENT_SIZE = 50; + public static final int DEFAULT_NUMBER_OF_EVENTS = 10; + public static final int DEFAULT_RETRY_COUNT = 5; + public static final int DEFAULT_BACKOFF_TIME = 1000; + + @JsonProperty("batch_size") + @Size(min = 1, max = 10000, message = "batch_size amount should be between 1 to 10000") + private int batchSize = DEFAULT_BATCH_SIZE; + + @JsonProperty("max_event_size") + @Size(min = 1, max = 256, message = "max_event_size amount should be between 1 to 256 bytes") + private int maxEventSize = DEFAULT_EVENT_SIZE; + + @JsonProperty("max_batch_request_size") + @Size(min = 1, max = 1048576, message = "max_batch_request_size amount should be between 1 and 1048576 bytes") + private int maxEvents = DEFAULT_NUMBER_OF_EVENTS; + + @JsonProperty("retry_count") + @Size(min = 1, max = 15, message = "retry_count amount should be between 1 and 15") + private int retryCount = DEFAULT_RETRY_COUNT; + + @JsonProperty("backoff_time") + @Size(min = 0, max = 20000, message = "backoff_time amount should be between 0 and 20000 milliseconds") + private int backOffTime = DEFAULT_BACKOFF_TIME; + + public int getBatchSize() { + return batchSize; + } + + public int getMaxEventSize() { + return maxEventSize; + } + + public int getMaxEvents() { + return maxEvents; + } + + public int getRetryCount() { + return retryCount; + } + + public int getBackOffTime() { + return backOffTime; + } +} diff --git a/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java b/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java new file mode 100644 index 0000000000..1091ad708e --- /dev/null +++ b/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java @@ -0,0 +1,44 @@ +package org.opensearch.dataprepper.plugins.sink.configuration; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.sink.config.CwlSinkConfig; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +public class CwlSinkConfigTest { + //Client Config: + public static final String LOG_GROUP = "testGroup"; + public static final String LOG_STREAM = "testStream"; + public static final String BUFFER_TYPE = "in_memory"; + public static final int BATCH_SIZE = 10; + public static final int MAX_RETRIES = 10; + //Auth Config: + public static final String REGION = "us-east-1"; + + @BeforeEach + void setUp() { + + } + + @Test + void check_null_auth_config_test() { + assertThat(new CwlSinkConfig().getAwsConfig(), equalTo(null)); + } + + @Test + void check_default_buffer_type_test() { + assertThat(new CwlSinkConfig().getBufferType(), equalTo(CwlSinkConfig.DEFAULT_BUFFER_TYPE)); + } + + @Test + void check_null_log_group_test() { + assertThat(new CwlSinkConfig().getLogGroup(), equalTo(null)); + } + @Test + void check_null_log_stream_test() { + assertThat(new CwlSinkConfig().getLogStream(), equalTo(null)); + } + +} diff --git a/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java b/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java new file mode 100644 index 0000000000..883b5548aa --- /dev/null +++ b/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java @@ -0,0 +1,38 @@ +package org.opensearch.dataprepper.plugins.sink.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; + +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +public class ThresholdConfigTest { + + private ObjectMapper objectMapper; + + @BeforeEach + void setUp() { + objectMapper = new ObjectMapper(); + } + + @ParameterizedTest + @ValueSource(ints = {1, 10, 10000}) + void check_valid_batch_size(final int batchSize) { + final Map jsonMap = Map.of("batch_size", batchSize); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getBatchSize(), equalTo(batchSize)); + } + + @ParameterizedTest + @ValueSource(ints = {1, 10, 256}) + void check_valid_max_event_size(final int max_event_size) { + final Map jsonMap = Map.of("max_event_size", max_event_size); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getMaxEventSize(), equalTo(max_event_size)); + } +} diff --git a/settings.gradle b/settings.gradle index c14df4205f..01eece37be 100644 --- a/settings.gradle +++ b/settings.gradle @@ -124,4 +124,5 @@ include 'data-prepper-plugins:obfuscate-processor' include 'data-prepper-plugins:parquet-codecs' include 'data-prepper-plugins:buffer-common' include 'data-prepper-plugins:sqs-source' +include 'data-prepper-plugins:cwl-sink' include 'data-prepper-plugins:http-sink' From f1c25bbf8273989616aa66459a6b7ce35d32f136 Mon Sep 17 00:00:00 2001 From: Marcos Date: Thu, 22 Jun 2023 11:49:55 -0700 Subject: [PATCH 03/43] Added fixes from comments to code (including pathing and nomenclature syntax) Signed-off-by: Marcos Gonzalez Mayedo --- data-prepper-plugins/cwl-sink/.gitignore | 42 ------------------- .../java/org/opensearch/dataprepper/Main.java | 8 ---- .../plugins/sink/config/CwlSinkConfig.java | 2 +- 3 files changed, 1 insertion(+), 51 deletions(-) delete mode 100644 data-prepper-plugins/cwl-sink/.gitignore delete mode 100644 data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/Main.java diff --git a/data-prepper-plugins/cwl-sink/.gitignore b/data-prepper-plugins/cwl-sink/.gitignore deleted file mode 100644 index b63da4551b..0000000000 --- a/data-prepper-plugins/cwl-sink/.gitignore +++ /dev/null @@ -1,42 +0,0 @@ -.gradle -build/ -!gradle/wrapper/gradle-wrapper.jar -!**/src/main/**/build/ -!**/src/test/**/build/ - -### IntelliJ IDEA ### -.idea/modules.xml -.idea/jarRepositories.xml -.idea/compiler.xml -.idea/libraries/ -*.iws -*.iml -*.ipr -out/ -!**/src/main/**/out/ -!**/src/test/**/out/ - -### Eclipse ### -.apt_generated -.classpath -.factorypath -.project -.settings -.springBeans -.sts4-cache -bin/ -!**/src/main/**/bin/ -!**/src/test/**/bin/ - -### NetBeans ### -/nbproject/private/ -/nbbuild/ -/dist/ -/nbdist/ -/.nb-gradle/ - -### VS Code ### -.vscode/ - -### Mac OS ### -.DS_Store \ No newline at end of file diff --git a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/Main.java b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/Main.java deleted file mode 100644 index cee3b572c6..0000000000 --- a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/Main.java +++ /dev/null @@ -1,8 +0,0 @@ -package org.opensearch.dataprepper; - -public class Main { - private String path; - public static void main(String[] args) { - System.out.println("Hello world!"); - } -} \ No newline at end of file diff --git a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java index 22a00c631c..44f74f6e96 100644 --- a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java +++ b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java @@ -10,7 +10,7 @@ public class CwlSinkConfig { public static final String DEFAULT_BUFFER_TYPE = "in_memory"; //Class was utilized from the - @JsonProperty("aws_config") + @JsonProperty("aws") @NotNull @Valid private AwsAuthenticationOptions awsConfig; From 9d640e576703eed05ba34e8bcdc66ebec55270f5 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo <95880281+MaGonzalMayedo@users.noreply.github.com> Date: Wed, 21 Jun 2023 18:07:04 -0700 Subject: [PATCH 04/43] Refactoring config (#5) Added default params for back_off and log_send_interval alongside test cases for ThresholdConfig. Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/config/AwsConfig.java | 35 +++++++++++++++++++ .../plugins/sink/config/CwlSinkConfig.java | 6 ++-- .../plugins/sink/config/ThresholdConfig.java | 23 +++++++----- .../sink/configuration/CwlSinkConfigTest.java | 6 ---- .../configuration/ThresholdConfigTest.java | 25 ++++++++++++- 5 files changed, 76 insertions(+), 19 deletions(-) create mode 100644 data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java diff --git a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java new file mode 100644 index 0000000000..ba77c0df72 --- /dev/null +++ b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java @@ -0,0 +1,35 @@ +package org.opensearch.dataprepper.plugins.sink.config; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; +import software.amazon.awssdk.regions.Region; + +/** + * AwsConfig is based on the S3-Sink AwsAuthenticationOptions + * where the configuration allows the sink to fetch Aws credentials + * and resources. + */ +public class AwsConfig { + @JsonProperty("region") + @Size(min = 1, message = "Region cannot be empty string") + private String awsRegion; + + @JsonProperty("sts_role_arn") + @Size(min = 20, max = 2048, message = "awsStsRoleArn length should be between 1 and 2048 characters") + private String awsStsRoleArn; + + @JsonProperty("path_to_credentials") + private String pathToCredentials; + + public Region getAwsRegion() { + return awsRegion != null ? Region.of(awsRegion) : null; + } + + public String getAwsStsRoleArn() { + return awsStsRoleArn; + } + + public String getPathToCredentials() { + return pathToCredentials; + } +} diff --git a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java index 44f74f6e96..e8696e6486 100644 --- a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java +++ b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java @@ -4,16 +4,14 @@ import jakarta.validation.Valid; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; -import org.opensearch.dataprepper.plugins.sink.configuration.AwsAuthenticationOptions; public class CwlSinkConfig { public static final String DEFAULT_BUFFER_TYPE = "in_memory"; - //Class was utilized from the @JsonProperty("aws") @NotNull @Valid - private AwsAuthenticationOptions awsConfig; + private AwsConfig awsConfig; @JsonProperty("threshold_config") @NotNull @@ -32,7 +30,7 @@ public class CwlSinkConfig { @NotNull private String logStream; - public AwsAuthenticationOptions getAwsConfig() { + public AwsConfig getAwsConfig() { return awsConfig; } diff --git a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java index e0eacadda6..a88a7d9c40 100644 --- a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java +++ b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java @@ -11,9 +11,10 @@ public class ThresholdConfig { public static final int DEFAULT_BATCH_SIZE = 10; public static final int DEFAULT_EVENT_SIZE = 50; - public static final int DEFAULT_NUMBER_OF_EVENTS = 10; + public static final int DEFAULT_SIZE_OF_REQUEST = 25000; public static final int DEFAULT_RETRY_COUNT = 5; - public static final int DEFAULT_BACKOFF_TIME = 1000; + public static final int DEFAULT_LOG_SEND_INTERVAL_TIME = 60; + public static final int DEFAULT_BACKOFF_TIME = 5000; @JsonProperty("batch_size") @Size(min = 1, max = 10000, message = "batch_size amount should be between 1 to 10000") @@ -23,16 +24,18 @@ public class ThresholdConfig { @Size(min = 1, max = 256, message = "max_event_size amount should be between 1 to 256 bytes") private int maxEventSize = DEFAULT_EVENT_SIZE; - @JsonProperty("max_batch_request_size") + @JsonProperty("max_request_size") @Size(min = 1, max = 1048576, message = "max_batch_request_size amount should be between 1 and 1048576 bytes") - private int maxEvents = DEFAULT_NUMBER_OF_EVENTS; + private int maxRequestSize = DEFAULT_SIZE_OF_REQUEST; @JsonProperty("retry_count") @Size(min = 1, max = 15, message = "retry_count amount should be between 1 and 15") private int retryCount = DEFAULT_RETRY_COUNT; - @JsonProperty("backoff_time") - @Size(min = 0, max = 20000, message = "backoff_time amount should be between 0 and 20000 milliseconds") + @JsonProperty("log_send_interval") + @Size(min = 5, max = 300, message = "log_send_interval amount should be between 5 and 300 seconds") + private int logSendInterval = DEFAULT_LOG_SEND_INTERVAL_TIME; + private int backOffTime = DEFAULT_BACKOFF_TIME; public int getBatchSize() { @@ -43,14 +46,18 @@ public int getMaxEventSize() { return maxEventSize; } - public int getMaxEvents() { - return maxEvents; + public int getMaxBatchSize() { + return maxRequestSize; } public int getRetryCount() { return retryCount; } + public int getLogSendInterval() { + return logSendInterval; + } + public int getBackOffTime() { return backOffTime; } diff --git a/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java b/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java index 1091ad708e..3d946c2eb8 100644 --- a/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java +++ b/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java @@ -8,7 +8,6 @@ import static org.hamcrest.Matchers.equalTo; public class CwlSinkConfigTest { - //Client Config: public static final String LOG_GROUP = "testGroup"; public static final String LOG_STREAM = "testStream"; public static final String BUFFER_TYPE = "in_memory"; @@ -17,11 +16,6 @@ public class CwlSinkConfigTest { //Auth Config: public static final String REGION = "us-east-1"; - @BeforeEach - void setUp() { - - } - @Test void check_null_auth_config_test() { assertThat(new CwlSinkConfig().getAwsConfig(), equalTo(null)); diff --git a/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java b/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java index 883b5548aa..b5cf4c895f 100644 --- a/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java +++ b/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java @@ -12,7 +12,6 @@ import static org.hamcrest.Matchers.equalTo; public class ThresholdConfigTest { - private ObjectMapper objectMapper; @BeforeEach @@ -35,4 +34,28 @@ void check_valid_max_event_size(final int max_event_size) { final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); assertThat(thresholdConfigTest.getMaxEventSize(), equalTo(max_event_size)); } + + @ParameterizedTest + @ValueSource(ints = {1, 100, 1048576}) + void check_valid_request_size(final int max_batch_request_size) { + final Map jsonMap = Map.of("max_request_size", max_batch_request_size); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getMaxBatchSize(), equalTo(max_batch_request_size)); + } + + @ParameterizedTest + @ValueSource(ints = {1, 10, 15}) + void check_valid_retry_count(final int retryCount) { + final Map jsonMap = Map.of("retry_count", retryCount); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getRetryCount(), equalTo(retryCount)); + } + + @ParameterizedTest + @ValueSource(ints = {5, 10, 300}) + void check_valid_log_send_interval(final int logSendInterval) { + final Map jsonMap = Map.of("log_send_interval", logSendInterval); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getLogSendInterval(), equalTo(logSendInterval)); + } } From b100ee36c2eb4a106b6244022011255c34b0f4fa Mon Sep 17 00:00:00 2001 From: Marcos Date: Thu, 22 Jun 2023 15:22:44 -0700 Subject: [PATCH 05/43] Fixed deleted AwsConfig file Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/config/AwsConfig.java | 0 data-prepper-plugins/cwl-sink/build.gradle | 42 ------------- .../sink/configuration/CwlSinkConfigTest.java | 38 ------------ .../configuration/ThresholdConfigTest.java | 61 ------------------- 4 files changed, 141 deletions(-) rename data-prepper-plugins/{cwl-sink => cloudwatch-logs}/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java (100%) delete mode 100644 data-prepper-plugins/cwl-sink/build.gradle delete mode 100644 data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java delete mode 100644 data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java diff --git a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java similarity index 100% rename from data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java rename to data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java diff --git a/data-prepper-plugins/cwl-sink/build.gradle b/data-prepper-plugins/cwl-sink/build.gradle deleted file mode 100644 index bcea1ce923..0000000000 --- a/data-prepper-plugins/cwl-sink/build.gradle +++ /dev/null @@ -1,42 +0,0 @@ -plugins { - id 'java' - id 'java-library' -} - -group = 'org.opensearch.dataprepper' -version = '2.3.0-SNAPSHOT' - -repositories { - mavenCentral() -} - -dependencies { - api project(':data-prepper-api') - implementation project(':data-prepper-plugins:aws-plugin-api') - implementation project(path: ':data-prepper-plugins:s3-sink') - testImplementation platform('org.junit:junit-bom:5.9.1') - implementation platform('software.amazon.awssdk:bom:2.20.56') - implementation project(path: ':data-prepper-plugins:common') - testImplementation 'org.junit.jupiter:junit-jupiter' - implementation 'com.fasterxml.jackson.core:jackson-core' - implementation 'com.fasterxml.jackson.core:jackson-databind' - implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' - implementation 'software.amazon.awssdk:cloudwatch' - implementation 'software.amazon.awssdk:cloudwatchlogs' - implementation 'org.apache.commons:commons-lang3:3.12.0' -} - -jacocoTestCoverageVerification { - dependsOn jacocoTestReport - violationRules { - rule { //in addition to core projects rule - limit { - minimum = 0.90 - } - } - } -} - -test { - useJUnitPlatform() -} \ No newline at end of file diff --git a/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java b/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java deleted file mode 100644 index 3d946c2eb8..0000000000 --- a/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java +++ /dev/null @@ -1,38 +0,0 @@ -package org.opensearch.dataprepper.plugins.sink.configuration; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.opensearch.dataprepper.plugins.sink.config.CwlSinkConfig; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; - -public class CwlSinkConfigTest { - public static final String LOG_GROUP = "testGroup"; - public static final String LOG_STREAM = "testStream"; - public static final String BUFFER_TYPE = "in_memory"; - public static final int BATCH_SIZE = 10; - public static final int MAX_RETRIES = 10; - //Auth Config: - public static final String REGION = "us-east-1"; - - @Test - void check_null_auth_config_test() { - assertThat(new CwlSinkConfig().getAwsConfig(), equalTo(null)); - } - - @Test - void check_default_buffer_type_test() { - assertThat(new CwlSinkConfig().getBufferType(), equalTo(CwlSinkConfig.DEFAULT_BUFFER_TYPE)); - } - - @Test - void check_null_log_group_test() { - assertThat(new CwlSinkConfig().getLogGroup(), equalTo(null)); - } - @Test - void check_null_log_stream_test() { - assertThat(new CwlSinkConfig().getLogStream(), equalTo(null)); - } - -} diff --git a/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java b/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java deleted file mode 100644 index b5cf4c895f..0000000000 --- a/data-prepper-plugins/cwl-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java +++ /dev/null @@ -1,61 +0,0 @@ -package org.opensearch.dataprepper.plugins.sink.configuration; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; -import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; - -import java.util.Map; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; - -public class ThresholdConfigTest { - private ObjectMapper objectMapper; - - @BeforeEach - void setUp() { - objectMapper = new ObjectMapper(); - } - - @ParameterizedTest - @ValueSource(ints = {1, 10, 10000}) - void check_valid_batch_size(final int batchSize) { - final Map jsonMap = Map.of("batch_size", batchSize); - final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); - assertThat(thresholdConfigTest.getBatchSize(), equalTo(batchSize)); - } - - @ParameterizedTest - @ValueSource(ints = {1, 10, 256}) - void check_valid_max_event_size(final int max_event_size) { - final Map jsonMap = Map.of("max_event_size", max_event_size); - final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); - assertThat(thresholdConfigTest.getMaxEventSize(), equalTo(max_event_size)); - } - - @ParameterizedTest - @ValueSource(ints = {1, 100, 1048576}) - void check_valid_request_size(final int max_batch_request_size) { - final Map jsonMap = Map.of("max_request_size", max_batch_request_size); - final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); - assertThat(thresholdConfigTest.getMaxBatchSize(), equalTo(max_batch_request_size)); - } - - @ParameterizedTest - @ValueSource(ints = {1, 10, 15}) - void check_valid_retry_count(final int retryCount) { - final Map jsonMap = Map.of("retry_count", retryCount); - final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); - assertThat(thresholdConfigTest.getRetryCount(), equalTo(retryCount)); - } - - @ParameterizedTest - @ValueSource(ints = {5, 10, 300}) - void check_valid_log_send_interval(final int logSendInterval) { - final Map jsonMap = Map.of("log_send_interval", logSendInterval); - final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); - assertThat(thresholdConfigTest.getLogSendInterval(), equalTo(logSendInterval)); - } -} From 35859b0b2cb960d92c8f0335da1383243a522ea4 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Mon, 26 Jun 2023 14:34:44 -0700 Subject: [PATCH 06/43] Removed the s3 dependency from build.gradle, replaced the AwsAuth.. with AwsConfig. Signed-off-by: Marcos Gonzalez Mayedo --- .../cloudwatch-logs/build.gradle | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 data-prepper-plugins/cloudwatch-logs/build.gradle diff --git a/data-prepper-plugins/cloudwatch-logs/build.gradle b/data-prepper-plugins/cloudwatch-logs/build.gradle new file mode 100644 index 0000000000..5bdf7b01be --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/build.gradle @@ -0,0 +1,38 @@ +plugins { + id 'java' + id 'java-library' +} + +repositories { + mavenCentral() +} + +dependencies { + api project(':data-prepper-api') + implementation project(':data-prepper-plugins:aws-plugin-api') + testImplementation platform('org.junit:junit-bom:5.9.1') + implementation platform('software.amazon.awssdk:bom:2.20.56') + implementation project(path: ':data-prepper-plugins:common') + testImplementation 'org.junit.jupiter:junit-jupiter' + implementation 'com.fasterxml.jackson.core:jackson-core' + implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' + implementation 'software.amazon.awssdk:cloudwatch' + implementation 'software.amazon.awssdk:cloudwatchlogs' + implementation 'org.apache.commons:commons-lang3:3.12.0' +} + +jacocoTestCoverageVerification { + dependsOn jacocoTestReport + violationRules { + rule { //in addition to core projects rule + limit { + minimum = 0.90 + } + } + } +} + +test { + useJUnitPlatform() +} \ No newline at end of file From 7040186ba8cf947913f4834f6ddb2c0f34f9063e Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Tue, 27 Jun 2023 14:14:07 -0700 Subject: [PATCH 07/43] Added modifiable back_off_timer, added threshold test for back_off_timer and params to AwsConfig Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/config/AwsConfig.java | 25 +++++-- .../sink/configuration/CwlSinkConfigTest.java | 30 ++++++++ .../configuration/ThresholdConfigTest.java | 69 +++++++++++++++++++ .../plugins/sink/config/ThresholdConfig.java | 8 ++- 4 files changed, 125 insertions(+), 7 deletions(-) create mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java index ba77c0df72..8ba82fad87 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java @@ -4,12 +4,16 @@ import jakarta.validation.constraints.Size; import software.amazon.awssdk.regions.Region; +import java.util.Map; + /** * AwsConfig is based on the S3-Sink AwsAuthenticationOptions * where the configuration allows the sink to fetch Aws credentials * and resources. */ public class AwsConfig { + private int DEFAULT_CONNECTION_ATTEMPTS = 5; + @JsonProperty("region") @Size(min = 1, message = "Region cannot be empty string") private String awsRegion; @@ -18,8 +22,17 @@ public class AwsConfig { @Size(min = 20, max = 2048, message = "awsStsRoleArn length should be between 1 and 2048 characters") private String awsStsRoleArn; - @JsonProperty("path_to_credentials") - private String pathToCredentials; + @JsonProperty("sts_header_overrides") + @Size(max = 5, message = "sts_header_overrides supports a maximum of 5 headers to override") + private Map awsStsHeaderOverrides; + + @JsonProperty("sts_external_id") + @Size(min = 2, max = 1224, message = "awsStsExternalId length should be between 2 and 1224 characters") + private String awsStsExternalId; + + public int getDEFAULT_CONNECTION_ATTEMPTS() { + return DEFAULT_CONNECTION_ATTEMPTS; + } public Region getAwsRegion() { return awsRegion != null ? Region.of(awsRegion) : null; @@ -29,7 +42,11 @@ public String getAwsStsRoleArn() { return awsStsRoleArn; } - public String getPathToCredentials() { - return pathToCredentials; + public String getAwsStsExternalId() { + return awsStsExternalId; + } + + public Map getAwsStsHeaderOverrides() { + return awsStsHeaderOverrides; } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java new file mode 100644 index 0000000000..fd1aef8b2d --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java @@ -0,0 +1,30 @@ +package org.opensearch.dataprepper.plugins.sink.configuration; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.sink.config.CwlSinkConfig; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +public class CwlSinkConfigTest { + @Test + void check_null_auth_config_test() { + assertThat(new CwlSinkConfig().getAwsConfig(), equalTo(null)); + } + + @Test + void check_default_buffer_type_test() { + assertThat(new CwlSinkConfig().getBufferType(), equalTo(CwlSinkConfig.DEFAULT_BUFFER_TYPE)); + } + + @Test + void check_null_log_group_test() { + assertThat(new CwlSinkConfig().getLogGroup(), equalTo(null)); + } + @Test + void check_null_log_stream_test() { + assertThat(new CwlSinkConfig().getLogStream(), equalTo(null)); + } + +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java new file mode 100644 index 0000000000..cbe7ea606f --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java @@ -0,0 +1,69 @@ +package org.opensearch.dataprepper.plugins.sink.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; + +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +public class ThresholdConfigTest { + private ObjectMapper objectMapper; + + @BeforeEach + void setUp() { + objectMapper = new ObjectMapper(); + } + + @ParameterizedTest + @ValueSource(ints = {1, 10, 10000}) + void check_valid_batch_size(final int batchSize) { + final Map jsonMap = Map.of("batch_size", batchSize); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getBatchSize(), equalTo(batchSize)); + } + + @ParameterizedTest + @ValueSource(ints = {1, 10, 256}) + void check_valid_max_event_size(final int max_event_size) { + final Map jsonMap = Map.of("max_event_size", max_event_size); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getMaxEventSize(), equalTo(max_event_size)); + } + + @ParameterizedTest + @ValueSource(ints = {1, 100, 1048576}) + void check_valid_request_size(final int max_batch_request_size) { + final Map jsonMap = Map.of("max_request_size", max_batch_request_size); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getMaxBatchSize(), equalTo(max_batch_request_size)); + } + + @ParameterizedTest + @ValueSource(ints = {1, 10, 15}) + void check_valid_retry_count(final int retry_count) { + final Map jsonMap = Map.of("retry_count", retry_count); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getRetryCount(), equalTo(retry_count)); + } + + @ParameterizedTest + @ValueSource(ints = {5, 10, 300}) + void check_valid_log_send_interval(final int log_send_interval) { + final Map jsonMap = Map.of("log_send_interval", log_send_interval); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getLogSendInterval(), equalTo(log_send_interval)); + } + + @ParameterizedTest + @ValueSource(ints = {0, 100, 5000}) + void check_valid_back_off_time(final int back_off_time) { + final Map jsonMap = Map.of("back_off_time", back_off_time); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getBackOffTime(), equalTo(back_off_time)); + } +} diff --git a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java index a88a7d9c40..211e3db1bc 100644 --- a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java +++ b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java @@ -9,9 +9,9 @@ * restrictions. */ public class ThresholdConfig { - public static final int DEFAULT_BATCH_SIZE = 10; + public static final int DEFAULT_BATCH_SIZE = 100; public static final int DEFAULT_EVENT_SIZE = 50; - public static final int DEFAULT_SIZE_OF_REQUEST = 25000; + public static final int DEFAULT_SIZE_OF_REQUEST = 524288; public static final int DEFAULT_RETRY_COUNT = 5; public static final int DEFAULT_LOG_SEND_INTERVAL_TIME = 60; public static final int DEFAULT_BACKOFF_TIME = 5000; @@ -21,7 +21,7 @@ public class ThresholdConfig { private int batchSize = DEFAULT_BATCH_SIZE; @JsonProperty("max_event_size") - @Size(min = 1, max = 256, message = "max_event_size amount should be between 1 to 256 bytes") + @Size(min = 1, max = 256, message = "max_event_size amount should be between 1 to 256 kilobytes") private int maxEventSize = DEFAULT_EVENT_SIZE; @JsonProperty("max_request_size") @@ -36,6 +36,8 @@ public class ThresholdConfig { @Size(min = 5, max = 300, message = "log_send_interval amount should be between 5 and 300 seconds") private int logSendInterval = DEFAULT_LOG_SEND_INTERVAL_TIME; + @JsonProperty("back_off_time") + @Size(min = 0, max = 10000, message = "back_off_time amount should be between 0 and 10000 milliseconds") private int backOffTime = DEFAULT_BACKOFF_TIME; public int getBatchSize() { From 6b13e21287b1867e3b9e0ae07be6443ec943b0de Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Wed, 28 Jun 2023 10:12:04 -0700 Subject: [PATCH 08/43] Added fixes to gradle file, added tests to AwsConfig, and used Reflective mapping to tests CwlSink Signed-off-by: Marcos Gonzalez Mayedo --- .../cloudwatch-logs/build.gradle | 2 +- .../plugins/sink/config/AwsConfig.java | 6 +- .../sink/configuration/AwsConfigTest.java | 87 +++++++++++++++++++ .../sink/configuration/CwlSinkConfigTest.java | 36 ++++++++ .../plugins/sink/config/CwlSinkConfig.java | 2 +- 5 files changed, 126 insertions(+), 7 deletions(-) create mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/AwsConfigTest.java diff --git a/data-prepper-plugins/cloudwatch-logs/build.gradle b/data-prepper-plugins/cloudwatch-logs/build.gradle index 5bdf7b01be..f2c0495085 100644 --- a/data-prepper-plugins/cloudwatch-logs/build.gradle +++ b/data-prepper-plugins/cloudwatch-logs/build.gradle @@ -11,7 +11,6 @@ dependencies { api project(':data-prepper-api') implementation project(':data-prepper-plugins:aws-plugin-api') testImplementation platform('org.junit:junit-bom:5.9.1') - implementation platform('software.amazon.awssdk:bom:2.20.56') implementation project(path: ':data-prepper-plugins:common') testImplementation 'org.junit.jupiter:junit-jupiter' implementation 'com.fasterxml.jackson.core:jackson-core' @@ -20,6 +19,7 @@ dependencies { implementation 'software.amazon.awssdk:cloudwatch' implementation 'software.amazon.awssdk:cloudwatchlogs' implementation 'org.apache.commons:commons-lang3:3.12.0' + testImplementation project(path: ':data-prepper-test-common') } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java index 8ba82fad87..6a2a85557e 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java @@ -12,7 +12,7 @@ * and resources. */ public class AwsConfig { - private int DEFAULT_CONNECTION_ATTEMPTS = 5; + public static int DEFAULT_CONNECTION_ATTEMPTS = 5; @JsonProperty("region") @Size(min = 1, message = "Region cannot be empty string") @@ -30,10 +30,6 @@ public class AwsConfig { @Size(min = 2, max = 1224, message = "awsStsExternalId length should be between 2 and 1224 characters") private String awsStsExternalId; - public int getDEFAULT_CONNECTION_ATTEMPTS() { - return DEFAULT_CONNECTION_ATTEMPTS; - } - public Region getAwsRegion() { return awsRegion != null ? Region.of(awsRegion) : null; } diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/AwsConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/AwsConfigTest.java new file mode 100644 index 0000000000..8ab03d575d --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/AwsConfigTest.java @@ -0,0 +1,87 @@ +package org.opensearch.dataprepper.plugins.sink.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; +import software.amazon.awssdk.regions.Region; + +import java.util.Collections; +import java.util.Map; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; + +public class AwsConfigTest { + private ObjectMapper objectMapper; + + @BeforeEach + void setUp() { + objectMapper = new ObjectMapper(); + } + + @ParameterizedTest + @ValueSource(strings = {"us-east-1", "us-west-2", "eu-central-1"}) + void getAwsRegion_returns_Region_of(final String regionString) { + final Region expectedRegionObject = Region.of(regionString); + final Map jsonMap = Map.of("region", regionString); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsRegion(), equalTo(expectedRegionObject)); + } + + @Test + void getAwsRegion_returns_null_when_region_is_null() { + final Map jsonMap = Collections.emptyMap(); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsRegion(), nullValue()); + } + + @Test + void getAwsStsRoleArn_returns_value_from_deserialized_JSON() { + final String stsRoleArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsRoleArn(), equalTo(stsRoleArn)); + } + + @Test + void getAwsStsRoleArn_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsRoleArn(), nullValue()); + } + + @Test + void getAwsStsExternalId_returns_value_from_deserialized_JSON() { + final String stsExternalId = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("sts_external_id", stsExternalId); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsExternalId(), equalTo(stsExternalId)); + } + + @Test + void getAwsStsExternalId_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsExternalId(), nullValue()); + } + + @Test + void getAwsStsHeaderOverrides_returns_value_from_deserialized_JSON() { + final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + final Map jsonMap = Map.of("sts_header_overrides", stsHeaderOverrides); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsHeaderOverrides(), equalTo(stsHeaderOverrides)); + } + + @Test + void getAwsStsHeaderOverrides_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsHeaderOverrides(), nullValue()); + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java index fd1aef8b2d..804fce39d3 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java @@ -2,12 +2,31 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; import org.opensearch.dataprepper.plugins.sink.config.CwlSinkConfig; +import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; +import org.opensearch.dataprepper.test.helper.ReflectivelySetField; + +import java.util.Map; +import java.util.UUID; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; public class CwlSinkConfigTest { + private CwlSinkConfig cwlSinkConfig; + private AwsConfig awsConfig; + private ThresholdConfig thresholdConfig; + private final String LOG_GROUP = "testLogGroup"; + private final String LOG_STREAM = "testLogStream"; + + @BeforeEach + void setUp() { + cwlSinkConfig = new CwlSinkConfig(); + awsConfig = new AwsConfig(); + thresholdConfig = new ThresholdConfig(); + } + @Test void check_null_auth_config_test() { assertThat(new CwlSinkConfig().getAwsConfig(), equalTo(null)); @@ -27,4 +46,21 @@ void check_null_log_stream_test() { assertThat(new CwlSinkConfig().getLogStream(), equalTo(null)); } + @Test + void check_valid_log_group_and_log_stream_test() throws NoSuchFieldException, IllegalAccessException { + ReflectivelySetField.setField(cwlSinkConfig.getClass(), cwlSinkConfig, "logGroup", LOG_GROUP); + ReflectivelySetField.setField(cwlSinkConfig.getClass(), cwlSinkConfig, "logStream", LOG_STREAM); + + assertThat(cwlSinkConfig.getLogGroup(), equalTo(LOG_GROUP)); + assertThat(cwlSinkConfig.getLogStream(), equalTo(LOG_STREAM)); + } + + @Test + void check_valid_sub_config_test() throws NoSuchFieldException, IllegalAccessException { + ReflectivelySetField.setField(cwlSinkConfig.getClass(), cwlSinkConfig, "thresholdConfig", thresholdConfig); + ReflectivelySetField.setField(cwlSinkConfig.getClass(), cwlSinkConfig, "awsConfig", awsConfig); + + assertThat(cwlSinkConfig.getAwsConfig(), equalTo(awsConfig)); + assertThat(cwlSinkConfig.getThresholdConfig(), equalTo(thresholdConfig)); + } } diff --git a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java index e8696e6486..230512bde5 100644 --- a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java +++ b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java @@ -13,7 +13,7 @@ public class CwlSinkConfig { @Valid private AwsConfig awsConfig; - @JsonProperty("threshold_config") + @JsonProperty("threshold") @NotNull private ThresholdConfig thresholdConfig; From 3bb125a1f3b063e6985b6558aa8a0a3253529106 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Wed, 28 Jun 2023 14:37:50 -0700 Subject: [PATCH 09/43] Added default value test to ThresholdConfig and renamed getter for maxRequestSize Signed-off-by: Marcos Gonzalez Mayedo --- data-prepper-plugins/cloudwatch-logs/build.gradle | 2 +- .../sink/configuration/ThresholdConfigTest.java | 15 ++++++++++++++- .../plugins/sink/config/ThresholdConfig.java | 2 +- 3 files changed, 16 insertions(+), 3 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/build.gradle b/data-prepper-plugins/cloudwatch-logs/build.gradle index f2c0495085..2d00cbd2d0 100644 --- a/data-prepper-plugins/cloudwatch-logs/build.gradle +++ b/data-prepper-plugins/cloudwatch-logs/build.gradle @@ -10,7 +10,6 @@ repositories { dependencies { api project(':data-prepper-api') implementation project(':data-prepper-plugins:aws-plugin-api') - testImplementation platform('org.junit:junit-bom:5.9.1') implementation project(path: ':data-prepper-plugins:common') testImplementation 'org.junit.jupiter:junit-jupiter' implementation 'com.fasterxml.jackson.core:jackson-core' @@ -20,6 +19,7 @@ dependencies { implementation 'software.amazon.awssdk:cloudwatchlogs' implementation 'org.apache.commons:commons-lang3:3.12.0' testImplementation project(path: ':data-prepper-test-common') + testImplementation project(path: ':data-prepper-test-common') } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java index cbe7ea606f..e6af96e08d 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java @@ -2,6 +2,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; @@ -19,6 +20,18 @@ void setUp() { objectMapper = new ObjectMapper(); } + @Test + void check_default_values() { + final ThresholdConfig thresholdConfig = new ThresholdConfig(); + + assertThat(thresholdConfig.getBackOffTime(), equalTo(ThresholdConfig.DEFAULT_BACKOFF_TIME)); + assertThat(thresholdConfig.getRetryCount(), equalTo(ThresholdConfig.DEFAULT_RETRY_COUNT)); + assertThat(thresholdConfig.getBatchSize(), equalTo(ThresholdConfig.DEFAULT_BATCH_SIZE)); + assertThat(thresholdConfig.getMaxEventSize(), equalTo(ThresholdConfig.DEFAULT_EVENT_SIZE)); + assertThat(thresholdConfig.getMaxRequestSize(), equalTo(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST)); + assertThat(thresholdConfig.getLogSendInterval(), equalTo(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME)); + } + @ParameterizedTest @ValueSource(ints = {1, 10, 10000}) void check_valid_batch_size(final int batchSize) { @@ -40,7 +53,7 @@ void check_valid_max_event_size(final int max_event_size) { void check_valid_request_size(final int max_batch_request_size) { final Map jsonMap = Map.of("max_request_size", max_batch_request_size); final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); - assertThat(thresholdConfigTest.getMaxBatchSize(), equalTo(max_batch_request_size)); + assertThat(thresholdConfigTest.getMaxRequestSize(), equalTo(max_batch_request_size)); } @ParameterizedTest diff --git a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java index 211e3db1bc..13ba1e4dbf 100644 --- a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java +++ b/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java @@ -48,7 +48,7 @@ public int getMaxEventSize() { return maxEventSize; } - public int getMaxBatchSize() { + public int getMaxRequestSize() { return maxRequestSize; } From e5ee1e56efbb7125cac9809e953c208168cdcd5f Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Wed, 28 Jun 2023 14:40:05 -0700 Subject: [PATCH 10/43] Removed unnecessary imports Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/configuration/CwlSinkConfigTest.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java index 804fce39d3..9842a333ee 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java @@ -7,9 +7,6 @@ import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; import org.opensearch.dataprepper.test.helper.ReflectivelySetField; -import java.util.Map; -import java.util.UUID; - import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; From e90b05a27b87abed00fc6ab533e0b2045a89208e Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Mon, 3 Jul 2023 13:01:54 -0700 Subject: [PATCH 11/43] Added cloudwatch-logs to settings.gradle Signed-off-by: Marcos Gonzalez Mayedo --- .../dataprepper/plugins/sink/config/CwlSinkConfig.java | 0 .../dataprepper/plugins/sink/config/ThresholdConfig.java | 0 settings.gradle | 2 +- 3 files changed, 1 insertion(+), 1 deletion(-) rename data-prepper-plugins/{cwl-sink => cloudwatch-logs}/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java (100%) rename data-prepper-plugins/{cwl-sink => cloudwatch-logs}/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java (100%) diff --git a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java similarity index 100% rename from data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java rename to data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java diff --git a/data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java similarity index 100% rename from data-prepper-plugins/cwl-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java rename to data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java diff --git a/settings.gradle b/settings.gradle index 01eece37be..06c188ae11 100644 --- a/settings.gradle +++ b/settings.gradle @@ -124,5 +124,5 @@ include 'data-prepper-plugins:obfuscate-processor' include 'data-prepper-plugins:parquet-codecs' include 'data-prepper-plugins:buffer-common' include 'data-prepper-plugins:sqs-source' -include 'data-prepper-plugins:cwl-sink' +include 'data-prepper-plugins:cloudwatch-logs' include 'data-prepper-plugins:http-sink' From 40cb280779d377b5482f45987f835a11b61fc450 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Mon, 3 Jul 2023 17:08:39 -0700 Subject: [PATCH 12/43] Added a quick fix to the back_off_time range Signed-off-by: Marcos Gonzalez Mayedo --- .../dataprepper/plugins/sink/config/ThresholdConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java index 13ba1e4dbf..77571a2c29 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java @@ -37,7 +37,7 @@ public class ThresholdConfig { private int logSendInterval = DEFAULT_LOG_SEND_INTERVAL_TIME; @JsonProperty("back_off_time") - @Size(min = 0, max = 10000, message = "back_off_time amount should be between 0 and 10000 milliseconds") + @Size(min = 500, max = 1000, message = "back_off_time amount should be between 500 and 1000 milliseconds") private int backOffTime = DEFAULT_BACKOFF_TIME; public int getBatchSize() { From b56a845bba6ad0fe525a00dba9c507f4d6bf3b4c Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Wed, 5 Jul 2023 09:32:29 -0700 Subject: [PATCH 13/43] Added Buffer classes, ClientFactory similar to S3, and ThresholdCheck Signed-off-by: Marcos Gonzalez Mayedo --- .../cloudwatch-logs/build.gradle | 2 + .../plugins/sink/client/CwlClientFactory.java | 66 +++++++++++++ .../sink/threshold/ThresholdCheck.java | 2 +- .../sink/client/CwlClientFactoryTest.java | 97 +++++++++++++++++++ 4 files changed, 166 insertions(+), 1 deletion(-) create mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactory.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactoryTest.java diff --git a/data-prepper-plugins/cloudwatch-logs/build.gradle b/data-prepper-plugins/cloudwatch-logs/build.gradle index bd387d69ef..ddb00bd7cd 100644 --- a/data-prepper-plugins/cloudwatch-logs/build.gradle +++ b/data-prepper-plugins/cloudwatch-logs/build.gradle @@ -9,6 +9,8 @@ repositories { dependencies { implementation project(':data-prepper-plugins:aws-plugin-api') + testImplementation platform('org.junit:junit-bom:5.9.1') + testImplementation('org.mockito:mockito-inline:3.8.0') implementation project(path: ':data-prepper-plugins:common') implementation 'io.micrometer:micrometer-core' implementation 'com.fasterxml.jackson.core:jackson-core' diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactory.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactory.java new file mode 100644 index 0000000000..8770875ec3 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactory.java @@ -0,0 +1,66 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.client; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.ProfileCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; + +/** + * CwlClientFactory is in charge of reading in + * aws config parameters to return a working + * client for interfacing with + * CloudWatchLogs services. + */ +public final class CwlClientFactory { + + /** + * Generates a CloudWatchLogs Client based on STS role ARN system credentials. + * @return CloudWatchLogsClient -> used to interact with CloudWatch Logs services. + */ + public static CloudWatchLogsClient createCwlClient(final AwsConfig awsConfig, final AwsCredentialsSupplier awsCredentialsSupplier) { + final AwsCredentialsOptions awsCredentialsOptions = convertToCredentialOptions(awsConfig); + final AwsCredentialsProvider awsCredentialsProvider = awsCredentialsSupplier.getProvider(awsCredentialsOptions); + + return CloudWatchLogsClient.builder() + .region(awsConfig.getAwsRegion()) + .credentialsProvider(awsCredentialsProvider) + .overrideConfiguration(createOverrideConfiguration(awsConfig)).build(); + } + + /** + * Generates a CloudWatchLogs Client based on default system credentials. + * @return CloudWatchLogsClient -> used to interact with CloudWatch Logs services. + * //TODO: Might not be needed, remove if this is the case. + */ + public static CloudWatchLogsClient createCwlClient() { + return CloudWatchLogsClient.builder() + .credentialsProvider(ProfileCredentialsProvider.create()) + .build(); + } + + private static ClientOverrideConfiguration createOverrideConfiguration(final AwsConfig awsConfig) { + final RetryPolicy retryPolicy = RetryPolicy.builder().numRetries(AwsConfig.DEFAULT_CONNECTION_ATTEMPTS).build(); + + return ClientOverrideConfiguration.builder() + .retryPolicy(retryPolicy) + .build(); + } + + private static AwsCredentialsOptions convertToCredentialOptions(final AwsConfig awsConfig) { + return AwsCredentialsOptions.builder() + .withRegion(awsConfig.getAwsRegion()) + .withStsRoleArn(awsConfig.getAwsStsRoleArn()) + .withStsExternalId(awsConfig.getAwsStsExternalId()) + .withStsHeaderOverrides(awsConfig.getAwsStsHeaderOverrides()) + .build(); + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java index 7916efeca8..32e50834e9 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java @@ -15,7 +15,7 @@ public class ThresholdCheck { private final int maxRequestSizeBytes; private final long logSendInterval; - public ThresholdCheck(final int batchSize, final int maxEventSizeBytes, final int maxRequestSizeBytes, final int logSendInterval) { + ThresholdCheck (int batchSize, int maxEventSizeBytes, int maxRequestSizeBytes, int logSendInterval) { this.batchSize = batchSize; this.maxEventSizeBytes = maxEventSizeBytes; this.maxRequestSizeBytes = maxRequestSizeBytes; diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactoryTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactoryTest.java new file mode 100644 index 0000000000..bb79ea419a --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactoryTest.java @@ -0,0 +1,97 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.client; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.ArgumentMatchers; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; +import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClientBuilder; + +import java.util.Map; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.*; +import static org.mockito.Mockito.*; + +public class CwlClientFactoryTest { + private AwsConfig awsConfig; + private AwsCredentialsSupplier awsCredentialsSupplier; + private AwsCredentialsOptions awsCredentialsOptions; + + @BeforeEach + void setUp() { + awsConfig = mock(AwsConfig.class); + awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); + awsCredentialsOptions = mock(AwsCredentialsOptions.class); + when(awsConfig.getAwsRegion()).thenReturn(Region.US_EAST_1); + } + + @Test + void check_created_real_default_client_test() { + final CloudWatchLogsClient cloudWatchLogsClientToTest = CwlClientFactory.createCwlClient(); + + assertThat(cloudWatchLogsClientToTest, notNullValue()); + } + + @Test + void check_created_client_with_no_params() { + final CloudWatchLogsClient cloudWatchLogsClient = CwlClientFactory.createCwlClient(awsConfig, awsCredentialsSupplier); + + assertThat(cloudWatchLogsClient, notNullValue()); + } + + @Test + void check_CwlClient_with_correct_inputs() { + final String stsRoleArn = UUID.randomUUID().toString(); + final String externalId = UUID.randomUUID().toString(); + final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + when(awsConfig.getAwsStsRoleArn()).thenReturn(stsRoleArn); + when(awsConfig.getAwsStsExternalId()).thenReturn(externalId); + when(awsConfig.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); + + final AwsCredentialsProvider expectedCredentialsProvider = mock(AwsCredentialsProvider.class); + when(awsCredentialsSupplier.getProvider(ArgumentMatchers.any())).thenReturn(expectedCredentialsProvider); + + final CloudWatchLogsClientBuilder cloudWatchLogsClientBuilder = mock(CloudWatchLogsClientBuilder.class); + when(cloudWatchLogsClientBuilder.region(awsConfig.getAwsRegion())).thenReturn(cloudWatchLogsClientBuilder); + when(cloudWatchLogsClientBuilder.credentialsProvider(ArgumentMatchers.any())).thenReturn(cloudWatchLogsClientBuilder); + when(cloudWatchLogsClientBuilder.overrideConfiguration(ArgumentMatchers.any(ClientOverrideConfiguration.class))).thenReturn(cloudWatchLogsClientBuilder); + try(final MockedStatic cloudWatchLogsClientMockedStatic = mockStatic(CloudWatchLogsClient.class)) { + cloudWatchLogsClientMockedStatic.when(CloudWatchLogsClient::builder) + .thenReturn(cloudWatchLogsClientBuilder); + CwlClientFactory.createCwlClient(awsConfig, awsCredentialsSupplier); + } + + final ArgumentCaptor credentialsProviderArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsProvider.class); + verify(cloudWatchLogsClientBuilder).credentialsProvider(credentialsProviderArgumentCaptor.capture()); + + final AwsCredentialsProvider actualProvider = credentialsProviderArgumentCaptor.getValue(); + + assertThat(actualProvider, equalTo(expectedCredentialsProvider)); + + final ArgumentCaptor credentialsOptionsArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsOptions.class); + verify(awsCredentialsSupplier).getProvider(credentialsOptionsArgumentCaptor.capture()); + + final AwsCredentialsOptions actualOptions = credentialsOptionsArgumentCaptor.getValue(); + assertThat(actualOptions.getRegion(), equalTo(awsConfig.getAwsRegion())); + assertThat(actualOptions.getStsRoleArn(), equalTo(awsConfig.getAwsStsRoleArn())); + assertThat(actualOptions.getStsExternalId(), equalTo(awsConfig.getAwsStsExternalId())); + assertThat(actualOptions.getStsHeaderOverrides(), equalTo(awsConfig.getAwsStsHeaderOverrides())); + } +} From b06ed0b67e665487edfd8bae7ddf9053f855168a Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Wed, 5 Jul 2023 09:37:12 -0700 Subject: [PATCH 14/43] Removed unnecessary default method from ClientFactory Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/client/CwlClientFactory.java | 11 ----------- .../plugins/sink/client/CwlClientFactoryTest.java | 2 +- 2 files changed, 1 insertion(+), 12 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactory.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactory.java index 8770875ec3..c84cfe4177 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactory.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactory.java @@ -36,17 +36,6 @@ public static CloudWatchLogsClient createCwlClient(final AwsConfig awsConfig, fi .overrideConfiguration(createOverrideConfiguration(awsConfig)).build(); } - /** - * Generates a CloudWatchLogs Client based on default system credentials. - * @return CloudWatchLogsClient -> used to interact with CloudWatch Logs services. - * //TODO: Might not be needed, remove if this is the case. - */ - public static CloudWatchLogsClient createCwlClient() { - return CloudWatchLogsClient.builder() - .credentialsProvider(ProfileCredentialsProvider.create()) - .build(); - } - private static ClientOverrideConfiguration createOverrideConfiguration(final AwsConfig awsConfig) { final RetryPolicy retryPolicy = RetryPolicy.builder().numRetries(AwsConfig.DEFAULT_CONNECTION_ATTEMPTS).build(); diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactoryTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactoryTest.java index bb79ea419a..211c60487b 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactoryTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactoryTest.java @@ -44,7 +44,7 @@ void setUp() { @Test void check_created_real_default_client_test() { - final CloudWatchLogsClient cloudWatchLogsClientToTest = CwlClientFactory.createCwlClient(); + final CloudWatchLogsClient cloudWatchLogsClientToTest = CwlClientFactory.createCwlClient(awsConfig, awsCredentialsSupplier); assertThat(cloudWatchLogsClientToTest, notNullValue()); } From 7f5a432230796e05a9b054785837aeb37557374b Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Wed, 5 Jul 2023 09:44:26 -0700 Subject: [PATCH 15/43] Added comments in Buffer Interface, change some default values to suit the plugin use case more Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/client/CwlClientFactory.java | 55 ----------- .../sink/client/CwlClientFactoryTest.java | 97 ------------------- 2 files changed, 152 deletions(-) delete mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactory.java delete mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactoryTest.java diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactory.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactory.java deleted file mode 100644 index c84cfe4177..0000000000 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactory.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.sink.client; - -import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; -import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; -import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; -import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; -import software.amazon.awssdk.auth.credentials.ProfileCredentialsProvider; -import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; -import software.amazon.awssdk.core.retry.RetryPolicy; -import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; - -/** - * CwlClientFactory is in charge of reading in - * aws config parameters to return a working - * client for interfacing with - * CloudWatchLogs services. - */ -public final class CwlClientFactory { - - /** - * Generates a CloudWatchLogs Client based on STS role ARN system credentials. - * @return CloudWatchLogsClient -> used to interact with CloudWatch Logs services. - */ - public static CloudWatchLogsClient createCwlClient(final AwsConfig awsConfig, final AwsCredentialsSupplier awsCredentialsSupplier) { - final AwsCredentialsOptions awsCredentialsOptions = convertToCredentialOptions(awsConfig); - final AwsCredentialsProvider awsCredentialsProvider = awsCredentialsSupplier.getProvider(awsCredentialsOptions); - - return CloudWatchLogsClient.builder() - .region(awsConfig.getAwsRegion()) - .credentialsProvider(awsCredentialsProvider) - .overrideConfiguration(createOverrideConfiguration(awsConfig)).build(); - } - - private static ClientOverrideConfiguration createOverrideConfiguration(final AwsConfig awsConfig) { - final RetryPolicy retryPolicy = RetryPolicy.builder().numRetries(AwsConfig.DEFAULT_CONNECTION_ATTEMPTS).build(); - - return ClientOverrideConfiguration.builder() - .retryPolicy(retryPolicy) - .build(); - } - - private static AwsCredentialsOptions convertToCredentialOptions(final AwsConfig awsConfig) { - return AwsCredentialsOptions.builder() - .withRegion(awsConfig.getAwsRegion()) - .withStsRoleArn(awsConfig.getAwsStsRoleArn()) - .withStsExternalId(awsConfig.getAwsStsExternalId()) - .withStsHeaderOverrides(awsConfig.getAwsStsHeaderOverrides()) - .build(); - } -} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactoryTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactoryTest.java deleted file mode 100644 index 211c60487b..0000000000 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CwlClientFactoryTest.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.sink.client; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.ArgumentCaptor; -import org.mockito.ArgumentMatchers; -import org.mockito.Mock; -import org.mockito.MockedStatic; -import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; -import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; -import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; -import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; -import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; -import software.amazon.awssdk.regions.Region; -import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; -import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClientBuilder; - -import java.util.Map; -import java.util.UUID; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.*; -import static org.mockito.Mockito.*; - -public class CwlClientFactoryTest { - private AwsConfig awsConfig; - private AwsCredentialsSupplier awsCredentialsSupplier; - private AwsCredentialsOptions awsCredentialsOptions; - - @BeforeEach - void setUp() { - awsConfig = mock(AwsConfig.class); - awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); - awsCredentialsOptions = mock(AwsCredentialsOptions.class); - when(awsConfig.getAwsRegion()).thenReturn(Region.US_EAST_1); - } - - @Test - void check_created_real_default_client_test() { - final CloudWatchLogsClient cloudWatchLogsClientToTest = CwlClientFactory.createCwlClient(awsConfig, awsCredentialsSupplier); - - assertThat(cloudWatchLogsClientToTest, notNullValue()); - } - - @Test - void check_created_client_with_no_params() { - final CloudWatchLogsClient cloudWatchLogsClient = CwlClientFactory.createCwlClient(awsConfig, awsCredentialsSupplier); - - assertThat(cloudWatchLogsClient, notNullValue()); - } - - @Test - void check_CwlClient_with_correct_inputs() { - final String stsRoleArn = UUID.randomUUID().toString(); - final String externalId = UUID.randomUUID().toString(); - final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); - when(awsConfig.getAwsStsRoleArn()).thenReturn(stsRoleArn); - when(awsConfig.getAwsStsExternalId()).thenReturn(externalId); - when(awsConfig.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); - - final AwsCredentialsProvider expectedCredentialsProvider = mock(AwsCredentialsProvider.class); - when(awsCredentialsSupplier.getProvider(ArgumentMatchers.any())).thenReturn(expectedCredentialsProvider); - - final CloudWatchLogsClientBuilder cloudWatchLogsClientBuilder = mock(CloudWatchLogsClientBuilder.class); - when(cloudWatchLogsClientBuilder.region(awsConfig.getAwsRegion())).thenReturn(cloudWatchLogsClientBuilder); - when(cloudWatchLogsClientBuilder.credentialsProvider(ArgumentMatchers.any())).thenReturn(cloudWatchLogsClientBuilder); - when(cloudWatchLogsClientBuilder.overrideConfiguration(ArgumentMatchers.any(ClientOverrideConfiguration.class))).thenReturn(cloudWatchLogsClientBuilder); - try(final MockedStatic cloudWatchLogsClientMockedStatic = mockStatic(CloudWatchLogsClient.class)) { - cloudWatchLogsClientMockedStatic.when(CloudWatchLogsClient::builder) - .thenReturn(cloudWatchLogsClientBuilder); - CwlClientFactory.createCwlClient(awsConfig, awsCredentialsSupplier); - } - - final ArgumentCaptor credentialsProviderArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsProvider.class); - verify(cloudWatchLogsClientBuilder).credentialsProvider(credentialsProviderArgumentCaptor.capture()); - - final AwsCredentialsProvider actualProvider = credentialsProviderArgumentCaptor.getValue(); - - assertThat(actualProvider, equalTo(expectedCredentialsProvider)); - - final ArgumentCaptor credentialsOptionsArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsOptions.class); - verify(awsCredentialsSupplier).getProvider(credentialsOptionsArgumentCaptor.capture()); - - final AwsCredentialsOptions actualOptions = credentialsOptionsArgumentCaptor.getValue(); - assertThat(actualOptions.getRegion(), equalTo(awsConfig.getAwsRegion())); - assertThat(actualOptions.getStsRoleArn(), equalTo(awsConfig.getAwsStsRoleArn())); - assertThat(actualOptions.getStsExternalId(), equalTo(awsConfig.getAwsStsExternalId())); - assertThat(actualOptions.getStsHeaderOverrides(), equalTo(awsConfig.getAwsStsHeaderOverrides())); - } -} From 45768998b67c40c6785b753ddca2312190239f25 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Fri, 7 Jul 2023 11:17:43 -0600 Subject: [PATCH 16/43] Removed unused imports Signed-off-by: Marcos Gonzalez Mayedo --- data-prepper-plugins/cloudwatch-logs/build.gradle | 2 -- 1 file changed, 2 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/build.gradle b/data-prepper-plugins/cloudwatch-logs/build.gradle index ddb00bd7cd..bd387d69ef 100644 --- a/data-prepper-plugins/cloudwatch-logs/build.gradle +++ b/data-prepper-plugins/cloudwatch-logs/build.gradle @@ -9,8 +9,6 @@ repositories { dependencies { implementation project(':data-prepper-plugins:aws-plugin-api') - testImplementation platform('org.junit:junit-bom:5.9.1') - testImplementation('org.mockito:mockito-inline:3.8.0') implementation project(path: ':data-prepper-plugins:common') implementation 'io.micrometer:micrometer-core' implementation 'com.fasterxml.jackson.core:jackson-core' From a539833afcb8b812872ff33742c8c6ee6023387b Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Fri, 7 Jul 2023 12:05:41 -0600 Subject: [PATCH 17/43] Changed the unused imports, made parameters final in the ThresholdCheck Signed-off-by: Marcos Gonzalez Mayedo --- .../dataprepper/plugins/sink/threshold/ThresholdCheck.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java index 32e50834e9..253611f626 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java @@ -15,7 +15,7 @@ public class ThresholdCheck { private final int maxRequestSizeBytes; private final long logSendInterval; - ThresholdCheck (int batchSize, int maxEventSizeBytes, int maxRequestSizeBytes, int logSendInterval) { + ThresholdCheck (final int batchSize, final int maxEventSizeBytes, final int maxRequestSizeBytes, final int logSendInterval) { this.batchSize = batchSize; this.maxEventSizeBytes = maxEventSizeBytes; this.maxRequestSizeBytes = maxRequestSizeBytes; From c89ea17f6790e53334af0574d018403b281bdad4 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Mon, 10 Jul 2023 11:08:40 -0700 Subject: [PATCH 18/43] Made changes to the tests and the method signatures in ThresholdCheck, made fixes to gradle file to include catalog Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/threshold/ThresholdCheckTest.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheckTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheckTest.java index 7afd69ade7..370288eb42 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheckTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheckTest.java @@ -37,6 +37,18 @@ void check_max_event_size_valid(final int event_size) { assertFalse(isEventGreater); } +// @ParameterizedTest +// @ValueSource(ints = {10000, 250000, 500000}) +// void check_max_request_size_invalid(final int request_size) { +// assertThat(thresholdCheck.checkGreaterThanMaxRequestSize(request_size), is(false)); +// } + +// @ParameterizedTest +// @ValueSource(ints = {550000, 750000, 1000000}) +// void check_max_request_size_valid(final int request_size) { +// assertThat(thresholdCheck.checkGreaterThanMaxRequestSize(request_size), is(true)); +// } + @ParameterizedTest @ValueSource(ints = {60, 80, 100}) void check_greater_than_threshold_conditions_time_true(final int send_interval) { From 063e1d33e054244251c090c9daf2d5a26d219ca6 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Mon, 10 Jul 2023 12:45:57 -0700 Subject: [PATCH 19/43] Removed unused methods/comments Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/threshold/ThresholdCheckTest.java | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheckTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheckTest.java index 370288eb42..7afd69ade7 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheckTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheckTest.java @@ -37,18 +37,6 @@ void check_max_event_size_valid(final int event_size) { assertFalse(isEventGreater); } -// @ParameterizedTest -// @ValueSource(ints = {10000, 250000, 500000}) -// void check_max_request_size_invalid(final int request_size) { -// assertThat(thresholdCheck.checkGreaterThanMaxRequestSize(request_size), is(false)); -// } - -// @ParameterizedTest -// @ValueSource(ints = {550000, 750000, 1000000}) -// void check_max_request_size_valid(final int request_size) { -// assertThat(thresholdCheck.checkGreaterThanMaxRequestSize(request_size), is(true)); -// } - @ParameterizedTest @ValueSource(ints = {60, 80, 100}) void check_greater_than_threshold_conditions_time_true(final int send_interval) { From 1aad0b50a61d0db27bb68e70f6a3b9b4d80506ea Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Wed, 12 Jul 2023 11:14:37 -0700 Subject: [PATCH 20/43] Added CloudWatchLogsService, CloudWatchLogsServiceTest and RetransmissionLimitException Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/buffer/Buffer.java | 1 + .../sink/client/CloudWatchLogsService.java | 223 ++++++++++++ .../RetransmissionLimitException.java | 12 + .../client/CloudWatchLogsServiceTest.java | 317 ++++++++++++++++++ 4 files changed, 553 insertions(+) create mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/exception/RetransmissionLimitException.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java index bfdfb0d825..26e09f97a0 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.sink.buffer; +import java.io.IOException; import java.util.ArrayList; /** diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java new file mode 100644 index 0000000000..0ffa5c9d66 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -0,0 +1,223 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.client; + +import io.micrometer.core.instrument.Counter; +import org.apache.commons.lang3.time.StopWatch; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.sink.buffer.Buffer; +import org.opensearch.dataprepper.plugins.sink.config.CloudWatchLogsSinkConfig; +import org.opensearch.dataprepper.plugins.sink.exception.RetransmissionLimitException; +import org.opensearch.dataprepper.plugins.sink.threshold.ThresholdCheck; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; +import software.amazon.awssdk.services.cloudwatchlogs.model.InputLogEvent; +import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantLock; + + /*TODO: Can add DLQ logic here for sending these logs to a particular DLQ for error checking. (Explicitly for bad formatted logs). + as currently the logs that are able to be published but rejected by CloudWatch Logs will simply be deleted if not deferred to + a backup storage. + */ +//TODO: Must also consider if the customer makes the logEvent size bigger than the send request size. +//TODO: Can inject another class for the stopWatch functionality. + +public class CloudWatchLogsService { + public static final int LOG_EVENT_OVERHEAD_SIZE = 26; //Size of overhead for each log event message. + public static final String NUMBER_OF_RECORDS_PUSHED_TO_CWL_SUCCESS = "cloudWatchLogsEventsSucceeded"; + public static final String NUMBER_OF_RECORDS_PUSHED_TO_CWL_FAIL = "cloudWatchLogsEventsFailed"; + public static final String REQUESTS_SUCCEEDED = "cloudWatchLogsRequestsSucceeded"; + public static final String REQUESTS_FAILED = "cloudWatchLogsRequestsFailed"; + private static final Logger LOG = LoggerFactory.getLogger(CloudWatchLogsService.class); + private final CloudWatchLogsClient cloudWatchLogsClient; + private final Buffer buffer; + private final ThresholdCheck thresholdCheck; + private final List bufferedEventHandles; + private final String logGroup; + private final String logStream; + private final int retryCount; + private final long backOffTimeBase; + private final io.micrometer.core.instrument.Counter logEventSuccessCounter; //Counter to be used on the fly for counting successful transmissions. (Success per single event successfully published). + private final Counter requestSuccessCount; + private final io.micrometer.core.instrument.Counter logEventFailCounter; + private final io.micrometer.core.instrument.Counter requestFailCount; //Counter to be used on the fly during error handling. + private int failCounter = 0; + private boolean failedPost; + private final StopWatch stopWatch; + private boolean stopWatchOn; + private final ReentrantLock reentrantLock; + + public CloudWatchLogsService(final CloudWatchLogsClient cloudWatchLogsClient, final CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig, final Buffer buffer, + final PluginMetrics pluginMetrics, final ThresholdCheck thresholdCheck, final int retryCount, final long backOffTimeBase) { + + this.cloudWatchLogsClient = cloudWatchLogsClient; + this.buffer = buffer; + this.logGroup = cloudWatchLogsSinkConfig.getLogGroup(); + this.logStream = cloudWatchLogsSinkConfig.getLogStream(); + this.thresholdCheck = thresholdCheck; + + this.retryCount = retryCount; + this.backOffTimeBase = backOffTimeBase; + + this.bufferedEventHandles = new ArrayList<>(); + this.logEventSuccessCounter = pluginMetrics.counter(NUMBER_OF_RECORDS_PUSHED_TO_CWL_SUCCESS); + this.requestFailCount = pluginMetrics.counter(REQUESTS_FAILED); + this.logEventFailCounter = pluginMetrics.counter(NUMBER_OF_RECORDS_PUSHED_TO_CWL_FAIL); + this.requestSuccessCount = pluginMetrics.counter(REQUESTS_SUCCEEDED); + + reentrantLock = new ReentrantLock(); + + stopWatch = StopWatch.create(); + stopWatchOn = false; + } + + /** + * Function handles the packaging of events into log events before sending a bulk request to CloudWatchLogs. + * Implements simple conditional buffer. (Sends once batch size, request size in bytes, or time limit is reached) + * @param logs - Collection of Record events which hold log data. + */ + public void output(final Collection> logs) { + reentrantLock.lock(); + + try { + if (!stopWatchOn) { + startStopWatch(); + } + + for (Record singleLog: logs) { + String logJsonString = singleLog.getData().toJsonString(); + int logLength = logJsonString.length(); + + if (thresholdCheck.isGreaterThanMaxEventSize(logLength + LOG_EVENT_OVERHEAD_SIZE)) { + LOG.warn("Event blocked due to Max Size restriction! {Event Size: " + (logLength + LOG_EVENT_OVERHEAD_SIZE) + " bytes}"); + continue; + } + + int bufferSizeWithOverHead = (buffer.getBufferSize() + (buffer.getEventCount() * LOG_EVENT_OVERHEAD_SIZE)); + if ((thresholdCheck.isGreaterThanThresholdReached(getStopWatchTime(), bufferSizeWithOverHead + logLength + LOG_EVENT_OVERHEAD_SIZE, buffer.getEventCount() + 1) && (buffer.getEventCount() > 0))) { + pushLogs(); + } + + if (singleLog.getData().getEventHandle() != null) { + bufferedEventHandles.add(singleLog.getData().getEventHandle()); + } + buffer.writeEvent(logJsonString.getBytes()); + } + + runExitCheck(); + + } catch (InterruptedException e) { + LOG.error("Caught InterruptedException while attempting to publish logs!"); + reentrantLock.unlock(); + } + } + + private void pushLogs() throws InterruptedException { + LOG.info("Attempting to push logs! {Batch size: " + buffer.getEventCount() + "}"); + stopAndResetStopWatch(); + startStopWatch(); + + ArrayList logEventList = new ArrayList<>(); + failedPost = true; + + for (byte[] data: buffer.getBufferedData()) { + InputLogEvent tempLogEvent = InputLogEvent.builder() + .message(new String(data)) + .timestamp(System.currentTimeMillis()) + .build(); + logEventList.add(tempLogEvent); + } + + while (failedPost && (failCounter < retryCount)) { + try { + PutLogEventsRequest putLogEventsRequest = PutLogEventsRequest.builder() + .logEvents(logEventList) + .logGroupName(logGroup) + .logStreamName(logStream) + .build(); + + cloudWatchLogsClient.putLogEvents(putLogEventsRequest); + + requestSuccessCount.increment(); + failedPost = false; + + //TODO: When a log is rejected by the service, we cannot send it, can probably push to a DLQ here. + + } catch (AwsServiceException | SdkClientException e) { + LOG.error("Failed to push logs with error: {}", e.getMessage()); + + Thread.sleep(calculateBackOffTime(backOffTimeBase)); + + LOG.warn("Trying to retransmit request... {Attempt: " + retryCount + "}"); + requestFailCount.increment(); + failCounter += 1; + } + } + + buffer.clearBuffer(); + + if (failedPost) { + logEventFailCounter.increment(logEventList.size()); + releaseEventHandles(false); + LOG.error("Error, timed out trying to push logs!"); + throw new RetransmissionLimitException("Error, timed out trying to push logs! (Max retry_count reached: {" + retryCount + "})"); + } else { + logEventSuccessCounter.increment(logEventList.size()); + releaseEventHandles(true); + failCounter = 0; + } + } + + private long calculateBackOffTime(long backOffTimeBase) { + return failCounter * backOffTimeBase; + } + + private void runExitCheck() throws InterruptedException { + int bufferSizeWithOverHead = (buffer.getBufferSize() + (buffer.getEventCount() * LOG_EVENT_OVERHEAD_SIZE)); + if (thresholdCheck.isEqualToThresholdReached(bufferSizeWithOverHead, buffer.getEventCount())) { + pushLogs(); + } + } + + private void releaseEventHandles(final boolean result) { + if (bufferedEventHandles.size() == 0) { + return; + } + + for (EventHandle eventHandle : bufferedEventHandles) { + eventHandle.release(result); + } + + bufferedEventHandles.clear(); + } + + private void startStopWatch() { + stopWatchOn = true; + stopWatch.start(); + } + + private void stopAndResetStopWatch() { + stopWatchOn = false; + stopWatch.stop(); + stopWatch.reset(); + } + + private long getStopWatchTime() { + return stopWatch.getTime(TimeUnit.SECONDS); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/exception/RetransmissionLimitException.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/exception/RetransmissionLimitException.java new file mode 100644 index 0000000000..c476dabc55 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/exception/RetransmissionLimitException.java @@ -0,0 +1,12 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.exception; + +public class RetransmissionLimitException extends RuntimeException{ + public RetransmissionLimitException(String message) { + super(message); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java new file mode 100644 index 0000000000..7f73d73e89 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java @@ -0,0 +1,317 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.client; + +import io.micrometer.core.instrument.Counter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.sink.buffer.Buffer; +import org.opensearch.dataprepper.plugins.sink.buffer.BufferFactory; +import org.opensearch.dataprepper.plugins.sink.buffer.InMemoryBufferFactory; +import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; +import org.opensearch.dataprepper.plugins.sink.config.CloudWatchLogsSinkConfig; +import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; +import org.opensearch.dataprepper.plugins.sink.exception.RetransmissionLimitException; +import org.opensearch.dataprepper.plugins.sink.threshold.ThresholdCheck; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; +import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; +import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsResponse; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.Mockito.*; + +//TODO: Add Codec session. +//TODO: Finish adding feature for ARN reading. + +public class CloudWatchLogsServiceTest { + private CloudWatchLogsClient mockClient; + private PutLogEventsResponse putLogEventsResponse; + private CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig; + private ThresholdConfig thresholdConfig; + private ThresholdCheck thresholdCheck; + private AwsConfig awsConfig; + private AwsCredentialsSupplier awsCredentialsSupplier; + private BufferFactory bufferFactory; + private Buffer buffer; + private PluginMetrics pluginMetrics; + private Counter requestSuccessCounter; + private Counter requestFailCounter; + private Counter successEventCounter; + private Counter failedEventCounter; + private final String TEST_LOG_GROUP = "TESTGROUP"; + private final String TEST_LOG_STREAM = "TESTSTREAM"; + private static final int messageKeyByteSize = 14; + private static final int convertToBytesFromKiloBytes = 1024; + + @BeforeEach + void setUp() { + cloudWatchLogsSinkConfig = mock(CloudWatchLogsSinkConfig.class); + + thresholdConfig = new ThresholdConfig(); //Class can stay as is. + thresholdCheck = new ThresholdCheck(thresholdConfig.getBatchSize(), thresholdConfig.getMaxEventSize() * convertToBytesFromKiloBytes, + thresholdConfig.getMaxRequestSize(), thresholdConfig.getLogSendInterval()); + + awsConfig = mock(AwsConfig.class); + bufferFactory = new InMemoryBufferFactory(); + buffer = bufferFactory.getBuffer(); + awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); + + pluginMetrics = mock(PluginMetrics.class); + requestSuccessCounter = mock(Counter.class); + requestFailCounter = mock(Counter.class); + successEventCounter = mock(Counter.class); + failedEventCounter = mock(Counter.class); + + final String stsRoleArn = UUID.randomUUID().toString(); + final String externalId = UUID.randomUUID().toString(); + final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + + when(cloudWatchLogsSinkConfig.getLogGroup()).thenReturn(TEST_LOG_GROUP); + when(cloudWatchLogsSinkConfig.getLogStream()).thenReturn(TEST_LOG_STREAM); + when(cloudWatchLogsSinkConfig.getBufferType()).thenReturn("in_memory"); + when(cloudWatchLogsSinkConfig.getAwsConfig()).thenReturn(awsConfig); + when(cloudWatchLogsSinkConfig.getThresholdConfig()).thenReturn(thresholdConfig); + + when(awsConfig.getAwsRegion()).thenReturn(Region.US_EAST_1); + when(awsConfig.getAwsStsRoleArn()).thenReturn(stsRoleArn); + when(awsConfig.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); + when(awsConfig.getAwsStsExternalId()).thenReturn(externalId); + + lenient().when(pluginMetrics.counter(CloudWatchLogsService.NUMBER_OF_RECORDS_PUSHED_TO_CWL_SUCCESS)).thenReturn(successEventCounter); + lenient().when(pluginMetrics.counter(CloudWatchLogsService.REQUESTS_SUCCEEDED)).thenReturn(requestSuccessCounter); + lenient().when(pluginMetrics.counter(CloudWatchLogsService.NUMBER_OF_RECORDS_PUSHED_TO_CWL_FAIL)).thenReturn(failedEventCounter); + lenient().when(pluginMetrics.counter(CloudWatchLogsService.REQUESTS_FAILED)).thenReturn(requestFailCounter); + } + + void setThresholdForTestingRequestSize(int size) { + thresholdCheck = new ThresholdCheck(10000, size, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME); + } + + void setThresholdForTestingMaxRequestRequestSize() { + thresholdCheck = new ThresholdCheck(10000, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST * 2, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST * 2, ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME); + } + + CloudWatchLogsService getCwlClientWithMemoryBuffer() { + return new CloudWatchLogsService(mockClient, cloudWatchLogsSinkConfig, buffer, pluginMetrics, + thresholdCheck, thresholdConfig.getRetryCount(), ThresholdConfig.DEFAULT_BACKOFF_TIME); + } + + void setMockClientNoErrors() { + mockClient = mock(CloudWatchLogsClient.class); + putLogEventsResponse = mock(PutLogEventsResponse.class); + when(mockClient.putLogEvents(any(PutLogEventsRequest.class))).thenReturn(putLogEventsResponse); + when(putLogEventsResponse.rejectedLogEventsInfo()).thenReturn(null); + } + + void setMockClientThrowCWLException() { + mockClient = mock(CloudWatchLogsClient.class); + doThrow(AwsServiceException.class).when(mockClient).putLogEvents(any(PutLogEventsRequest.class)); + } + + Collection> getSampleRecords(int numberOfRecords) { + final ArrayList> returnCollection = new ArrayList<>(); + for (int i = 0; i < numberOfRecords; i++) { + JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("testMessage"); + final EventHandle mockEventHandle = mock(EventHandle.class); + mockJacksonEvent.setEventHandle(mockEventHandle); + returnCollection.add(new Record<>(mockJacksonEvent)); + } + + return returnCollection; + } + + Collection> getSampleRecordsLarge(int numberOfRecords, int sizeOfRecordsBytes) { + final ArrayList> returnCollection = new ArrayList<>(); + final String testMessage = "a"; + for (int i = 0; i < numberOfRecords; i++) { + JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage(testMessage.repeat(sizeOfRecordsBytes)); + final EventHandle mockEventHandle = mock(EventHandle.class); + mockJacksonEvent.setEventHandle(mockEventHandle); + returnCollection.add(new Record<>(mockJacksonEvent)); + } + + return returnCollection; + } + + @Test + void client_creation_test() { + CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); + } + + @Test + void retry_count_limit_reached_test() { + setMockClientThrowCWLException(); + CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); + + try { + cloudWatchLogsService.output(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 2)); + } catch (RetransmissionLimitException e) { //TODO: Create a dedicated RuntimeException for this. + assertThat(e, notNullValue()); + } + } + + @Test + void check_failed_event_transmission_test() { + setMockClientThrowCWLException(); + CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); + + try { + cloudWatchLogsService.output(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE)); + } catch (RetransmissionLimitException e) { + verify(failedEventCounter).increment(ThresholdConfig.DEFAULT_BATCH_SIZE); + } + } + + @Test + void check_successful_event_transmission_test() { + setMockClientNoErrors(); + CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); + + cloudWatchLogsService.output(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 2)); + + verify(successEventCounter, atLeast(2)).increment(anyDouble()); + } + + @Test + void check_failed_event_test() { + setMockClientThrowCWLException(); + CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); + + try { + cloudWatchLogsService.output(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 4)); + } catch (RetransmissionLimitException e) { + verify(requestFailCounter, atLeast(ThresholdConfig.DEFAULT_RETRY_COUNT)).increment(); + } + } + + @Test + void check_successful_event_test() { + setMockClientNoErrors(); + CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); + + cloudWatchLogsService.output(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 4)); + + verify(requestSuccessCounter, atLeast(4)).increment(); + } + + @Test + void check_event_handles_successfully_released_test() { + setMockClientNoErrors(); + CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); + + final Collection> sampleEvents = getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 2); + final Collection sampleEventHandles = sampleEvents.stream().map(Record::getData).map(Event::getEventHandle).collect(Collectors.toList()); + + cloudWatchLogsService.output(sampleEvents); + + for (EventHandle sampleEventHandle: sampleEventHandles) { + verify(sampleEventHandle).release(true); + } + } + + @Test + void check_event_handles_failed_released_test() { + setMockClientThrowCWLException(); + CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); + + final Collection> sampleEvents = getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE); + final Collection sampleEventHandles = sampleEvents.stream().map(Record::getData).map(Event::getEventHandle).collect(Collectors.toList()); + + try { + cloudWatchLogsService.output(sampleEvents); + } catch (RetransmissionLimitException e) { + for (EventHandle sampleEventHandle: sampleEventHandles) { + verify(sampleEventHandle).release(false); + } + } + } + + /** + * Tests if our json string is equal to the default event size in bytes. + * 14 accounts for the "message": byte size. + */ + @Test + void check_event_size_correct_test() { + ArrayList> sampleEvents = (ArrayList>) getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - messageKeyByteSize); //Accounts for the key string value. + + assertThat(sampleEvents.get(0).getData().toJsonString().length(), equalTo(ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes)); + } + + @Test + void check_max_size_threshold_fail_test() { + setMockClientNoErrors(); + CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); + + final Collection> sampleEvents = getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - CloudWatchLogsService.LOG_EVENT_OVERHEAD_SIZE - messageKeyByteSize + 1); + + cloudWatchLogsService.output(sampleEvents); + + verify(successEventCounter, never()).increment(anyDouble()); + verify(requestSuccessCounter, never()).increment(); + } + + @Test + void check_max_size_threshold_success_test() { + setMockClientNoErrors(); + CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); + + final Collection> sampleEvents = getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, (ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - messageKeyByteSize) - CloudWatchLogsService.LOG_EVENT_OVERHEAD_SIZE); + + cloudWatchLogsService.output(sampleEvents); + + verify(successEventCounter, atLeastOnce()).increment(anyDouble()); + verify(requestSuccessCounter, atLeastOnce()).increment(); + } + + @Test + void check_max_request_size_threshold_fail_test() { + setThresholdForTestingRequestSize(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST); + setMockClientNoErrors(); + CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); + + cloudWatchLogsService.output(getSampleRecordsLarge(1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - messageKeyByteSize - CloudWatchLogsService.LOG_EVENT_OVERHEAD_SIZE + 1)); + + verify(requestSuccessCounter, never()).increment(); + } + + @Test + void check_max_request_size_threshold_success_test() { + setThresholdForTestingRequestSize(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST); + setMockClientNoErrors(); + CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); + + cloudWatchLogsService.output(getSampleRecordsLarge(1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - messageKeyByteSize - CloudWatchLogsService.LOG_EVENT_OVERHEAD_SIZE)); + + verify(requestSuccessCounter, atLeast(1)).increment(); + } + + @Test + void check_max_api_request_size_threshold_success_test() { + setThresholdForTestingMaxRequestRequestSize(); + setMockClientNoErrors(); + CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); + + cloudWatchLogsService.output(getSampleRecordsLarge(1, (ThresholdConfig.DEFAULT_SIZE_OF_REQUEST * 2) - messageKeyByteSize - CloudWatchLogsService.LOG_EVENT_OVERHEAD_SIZE)); + + verify(requestSuccessCounter, atLeast(1)).increment(); + } +} \ No newline at end of file From cf1f8e1b564eacf31d2799075e759376d89c1287 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Wed, 12 Jul 2023 15:04:50 -0700 Subject: [PATCH 21/43] Fixed retransmission logging fixed value Signed-off-by: Marcos Gonzalez Mayedo --- .../dataprepper/plugins/sink/client/CloudWatchLogsService.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index 0ffa5c9d66..40f2a267a7 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -163,9 +163,8 @@ private void pushLogs() throws InterruptedException { Thread.sleep(calculateBackOffTime(backOffTimeBase)); - LOG.warn("Trying to retransmit request... {Attempt: " + retryCount + "}"); + LOG.warn("Trying to retransmit request... {Attempt: " + (++failCounter) + "}"); requestFailCount.increment(); - failCounter += 1; } } From 77f6d0f6f4b7f0dbb3d310504425bba5453b098f Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Wed, 12 Jul 2023 15:08:09 -0700 Subject: [PATCH 22/43] Fixed unused imports Signed-off-by: Marcos Gonzalez Mayedo --- .../org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java | 1 - .../dataprepper/plugins/sink/client/CloudWatchLogsService.java | 1 - 2 files changed, 2 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java index 26e09f97a0..bfdfb0d825 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java @@ -5,7 +5,6 @@ package org.opensearch.dataprepper.plugins.sink.buffer; -import java.io.IOException; import java.util.ArrayList; /** diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index 40f2a267a7..5556146ec7 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -23,7 +23,6 @@ import software.amazon.awssdk.services.cloudwatchlogs.model.InputLogEvent; import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; -import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.List; From 75d90fec15b7aeefe356e2fcc2e79dc9fcfa84cf Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Wed, 12 Jul 2023 15:57:10 -0700 Subject: [PATCH 23/43] Fixed making ThresholdCheck public Signed-off-by: Marcos Gonzalez Mayedo --- .../dataprepper/plugins/sink/threshold/ThresholdCheck.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java index 253611f626..7916efeca8 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java @@ -15,7 +15,7 @@ public class ThresholdCheck { private final int maxRequestSizeBytes; private final long logSendInterval; - ThresholdCheck (final int batchSize, final int maxEventSizeBytes, final int maxRequestSizeBytes, final int logSendInterval) { + public ThresholdCheck(final int batchSize, final int maxEventSizeBytes, final int maxRequestSizeBytes, final int logSendInterval) { this.batchSize = batchSize; this.maxEventSizeBytes = maxEventSizeBytes; this.maxRequestSizeBytes = maxRequestSizeBytes; From 5f2f5112767dc8aa94475b52497e4ce3047be853 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Wed, 12 Jul 2023 16:04:01 -0700 Subject: [PATCH 24/43] Added fixes to ThresholdCheck and CloudWatchLogsService to decouple methods Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/client/CloudWatchLogsService.java | 2 +- .../dataprepper/plugins/sink/threshold/ThresholdCheck.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index 5556146ec7..958cd8a80b 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -187,7 +187,7 @@ private long calculateBackOffTime(long backOffTimeBase) { private void runExitCheck() throws InterruptedException { int bufferSizeWithOverHead = (buffer.getBufferSize() + (buffer.getEventCount() * LOG_EVENT_OVERHEAD_SIZE)); - if (thresholdCheck.isEqualToThresholdReached(bufferSizeWithOverHead, buffer.getEventCount())) { + if ((thresholdCheck.isEqualToThresholdReached(bufferSizeWithOverHead, buffer.getEventCount()) && (buffer.getEventCount() > 0))) { pushLogs(); } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java index 7916efeca8..969f4096d7 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java @@ -31,7 +31,7 @@ public ThresholdCheck(final int batchSize, final int maxEventSizeBytes, final in */ public boolean isGreaterThanThresholdReached(final long currentTime, final int currentRequestSize, final int batchSize) { return ((isGreaterThanBatchSize(batchSize) || isGreaterEqualToLogSendInterval(currentTime) - || isGreaterThanMaxRequestSize(currentRequestSize)) && (batchSize > 0)); + || isGreaterThanMaxRequestSize(currentRequestSize))); } /** @@ -41,7 +41,7 @@ public boolean isGreaterThanThresholdReached(final long currentTime, final int c * @return boolean - true if we equal the threshold events or false otherwise. */ public boolean isEqualToThresholdReached(final int currentRequestSize, final int batchSize) { - return ((isEqualBatchSize(batchSize) || isEqualMaxRequestSize(currentRequestSize)) && (batchSize > 0)); + return ((isEqualBatchSize(batchSize) || isEqualMaxRequestSize(currentRequestSize))); } /** From fdc5b0065ef64f57b07a5ea7a76aa795a2252faf Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Wed, 12 Jul 2023 16:59:52 -0700 Subject: [PATCH 25/43] Fixed syntax start import in CloudWatchLogsServiceTest Signed-off-by: Marcos Gonzalez Mayedo --- .../sink/client/CloudWatchLogsServiceTest.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java index 7f73d73e89..c80b431eeb 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java @@ -37,7 +37,16 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; -import static org.mockito.Mockito.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.anyDouble; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.atLeastOnce; //TODO: Add Codec session. //TODO: Finish adding feature for ARN reading. From 039969467a7d425c7af1ee5f0617809cf155db66 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Thu, 13 Jul 2023 23:18:51 -0700 Subject: [PATCH 26/43] Extracted LogPusher and SinkStopWatch classes for code cleanup. Addded fixes to variables and retry logic for InterruptExceptions Signed-off-by: Marcos Gonzalez Mayedo --- .../sink/client/CloudWatchLogsService.java | 174 +++++++----------- .../sink/threshold/ThresholdCheck.java | 4 +- .../plugins/sink/utils/LogPusher.java | 89 +++++++++ .../plugins/sink/utils/SinkStopWatch.java | 40 ++++ .../client/CloudWatchLogsServiceTest.java | 22 +-- 5 files changed, 209 insertions(+), 120 deletions(-) create mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/LogPusher.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index 958cd8a80b..9e422d4ce5 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -6,7 +6,6 @@ package org.opensearch.dataprepper.plugins.sink.client; import io.micrometer.core.instrument.Counter; -import org.apache.commons.lang3.time.StopWatch; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventHandle; @@ -15,34 +14,33 @@ import org.opensearch.dataprepper.plugins.sink.config.CloudWatchLogsSinkConfig; import org.opensearch.dataprepper.plugins.sink.exception.RetransmissionLimitException; import org.opensearch.dataprepper.plugins.sink.threshold.ThresholdCheck; +import org.opensearch.dataprepper.plugins.sink.utils.LogPusher; +import org.opensearch.dataprepper.plugins.sink.utils.SinkStopWatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import software.amazon.awssdk.awscore.exception.AwsServiceException; -import software.amazon.awssdk.core.exception.SdkClientException; import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; -import software.amazon.awssdk.services.cloudwatchlogs.model.InputLogEvent; -import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantLock; - /*TODO: Can add DLQ logic here for sending these logs to a particular DLQ for error checking. (Explicitly for bad formatted logs). - as currently the logs that are able to be published but rejected by CloudWatch Logs will simply be deleted if not deferred to - a backup storage. - */ +/* + TODO: Can add DLQ logic here for sending these logs to a particular DLQ for error checking. (Explicitly for bad formatted logs). + as currently the logs that are able to be published but rejected by CloudWatch Logs will simply be deleted if not deferred to + a backup storage. +*/ //TODO: Must also consider if the customer makes the logEvent size bigger than the send request size. //TODO: Can inject another class for the stopWatch functionality. public class CloudWatchLogsService { - public static final int LOG_EVENT_OVERHEAD_SIZE = 26; //Size of overhead for each log event message. - public static final String NUMBER_OF_RECORDS_PUSHED_TO_CWL_SUCCESS = "cloudWatchLogsEventsSucceeded"; - public static final String NUMBER_OF_RECORDS_PUSHED_TO_CWL_FAIL = "cloudWatchLogsEventsFailed"; - public static final String REQUESTS_SUCCEEDED = "cloudWatchLogsRequestsSucceeded"; - public static final String REQUESTS_FAILED = "cloudWatchLogsRequestsFailed"; + public static final int APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE = 26; //Size of overhead for each log event message. + public static final String CLOUDWATCH_LOGS_EVENTS_SUCCEEDED = "cloudWatchLogsEventsSucceeded"; + public static final String CLOUDWATCH_LOGS_EVENTS_FAILED = "cloudWatchLogsEventsFailed"; + public static final String CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED = "cloudWatchLogsRequestsSucceeded"; + public static final String CLOUDWATCH_LOGS_REQUESTS_FAILED = "cloudWatchLogsRequestsFailed"; private static final Logger LOG = LoggerFactory.getLogger(CloudWatchLogsService.class); + private static final int RETRY_THREAD_ERROR_CAP = 3; private final CloudWatchLogsClient cloudWatchLogsClient; private final Buffer buffer; private final ThresholdCheck thresholdCheck; @@ -51,15 +49,13 @@ public class CloudWatchLogsService { private final String logStream; private final int retryCount; private final long backOffTimeBase; - private final io.micrometer.core.instrument.Counter logEventSuccessCounter; //Counter to be used on the fly for counting successful transmissions. (Success per single event successfully published). + private final Counter logEventSuccessCounter; //Counter to be used on the fly for counting successful transmissions. (Success per single event successfully published). private final Counter requestSuccessCount; - private final io.micrometer.core.instrument.Counter logEventFailCounter; - private final io.micrometer.core.instrument.Counter requestFailCount; //Counter to be used on the fly during error handling. - private int failCounter = 0; - private boolean failedPost; - private final StopWatch stopWatch; - private boolean stopWatchOn; + private final Counter logEventFailCounter; + private final Counter requestFailCount; //Counter to be used on the fly during error handling. + private final SinkStopWatch sinkStopWatch; private final ReentrantLock reentrantLock; + private final LogPusher logPusher; public CloudWatchLogsService(final CloudWatchLogsClient cloudWatchLogsClient, final CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig, final Buffer buffer, final PluginMetrics pluginMetrics, final ThresholdCheck thresholdCheck, final int retryCount, final long backOffTimeBase) { @@ -74,15 +70,16 @@ public CloudWatchLogsService(final CloudWatchLogsClient cloudWatchLogsClient, fi this.backOffTimeBase = backOffTimeBase; this.bufferedEventHandles = new ArrayList<>(); - this.logEventSuccessCounter = pluginMetrics.counter(NUMBER_OF_RECORDS_PUSHED_TO_CWL_SUCCESS); - this.requestFailCount = pluginMetrics.counter(REQUESTS_FAILED); - this.logEventFailCounter = pluginMetrics.counter(NUMBER_OF_RECORDS_PUSHED_TO_CWL_FAIL); - this.requestSuccessCount = pluginMetrics.counter(REQUESTS_SUCCEEDED); + this.logEventSuccessCounter = pluginMetrics.counter(CLOUDWATCH_LOGS_EVENTS_SUCCEEDED); + this.requestFailCount = pluginMetrics.counter(CLOUDWATCH_LOGS_REQUESTS_FAILED); + this.logEventFailCounter = pluginMetrics.counter(CLOUDWATCH_LOGS_EVENTS_FAILED); + this.requestSuccessCount = pluginMetrics.counter(CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED); reentrantLock = new ReentrantLock(); - stopWatch = StopWatch.create(); - stopWatchOn = false; + sinkStopWatch = new SinkStopWatch(); + + this.logPusher = new LogPusher(logEventSuccessCounter, logEventFailCounter, requestSuccessCount, requestFailCount, retryCount, backOffTimeBase); } /** @@ -93,100 +90,78 @@ public CloudWatchLogsService(final CloudWatchLogsClient cloudWatchLogsClient, fi public void output(final Collection> logs) { reentrantLock.lock(); - try { - if (!stopWatchOn) { - startStopWatch(); + int threadRetries = 0; + boolean processedLogsSuccessfully = false; + + while (threadRetries < RETRY_THREAD_ERROR_CAP) { + processedLogsSuccessfully = processLogEvents(logs); + + if (processedLogsSuccessfully) { + threadRetries = RETRY_THREAD_ERROR_CAP; + } else { + LOG.error("Thread threw InterruptedException!"); + threadRetries++; } + } + + if (processedLogsSuccessfully) { + LOG.info("Successfully processed logs."); + } else { + LOG.warn("Failed to process logs."); + //TODO: Insert DLQ logic as a last resort if we cannot manage to process logs prior to this point. + } + + reentrantLock.unlock(); + } + + private boolean processLogEvents(final Collection> logs) { + try { + sinkStopWatch.startIfNotRunning(); - for (Record singleLog: logs) { - String logJsonString = singleLog.getData().toJsonString(); + for (Record log: logs) { + String logJsonString = log.getData().toJsonString(); int logLength = logJsonString.length(); - if (thresholdCheck.isGreaterThanMaxEventSize(logLength + LOG_EVENT_OVERHEAD_SIZE)) { - LOG.warn("Event blocked due to Max Size restriction! {Event Size: " + (logLength + LOG_EVENT_OVERHEAD_SIZE) + " bytes}"); + if (thresholdCheck.isGreaterThanMaxEventSize(logLength + APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)) { + LOG.warn("Event blocked due to Max Size restriction! {Event Size: " + (logLength + APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE) + " bytes}"); continue; } - int bufferSizeWithOverHead = (buffer.getBufferSize() + (buffer.getEventCount() * LOG_EVENT_OVERHEAD_SIZE)); - if ((thresholdCheck.isGreaterThanThresholdReached(getStopWatchTime(), bufferSizeWithOverHead + logLength + LOG_EVENT_OVERHEAD_SIZE, buffer.getEventCount() + 1) && (buffer.getEventCount() > 0))) { + int bufferSizeWithOverhead = (buffer.getBufferSize() + (buffer.getEventCount() * APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + if ((thresholdCheck.isGreaterThanThresholdReached(sinkStopWatch.getStopWatchTimeSeconds(), bufferSizeWithOverhead + logLength + APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE, buffer.getEventCount() + 1) && (buffer.getEventCount() > 0))) { pushLogs(); } - if (singleLog.getData().getEventHandle() != null) { - bufferedEventHandles.add(singleLog.getData().getEventHandle()); + if (log.getData().getEventHandle() != null) { + bufferedEventHandles.add(log.getData().getEventHandle()); } buffer.writeEvent(logJsonString.getBytes()); } runExitCheck(); + return true; } catch (InterruptedException e) { LOG.error("Caught InterruptedException while attempting to publish logs!"); - reentrantLock.unlock(); + return false; } } private void pushLogs() throws InterruptedException { LOG.info("Attempting to push logs! {Batch size: " + buffer.getEventCount() + "}"); - stopAndResetStopWatch(); - startStopWatch(); - - ArrayList logEventList = new ArrayList<>(); - failedPost = true; - - for (byte[] data: buffer.getBufferedData()) { - InputLogEvent tempLogEvent = InputLogEvent.builder() - .message(new String(data)) - .timestamp(System.currentTimeMillis()) - .build(); - logEventList.add(tempLogEvent); - } - - while (failedPost && (failCounter < retryCount)) { - try { - PutLogEventsRequest putLogEventsRequest = PutLogEventsRequest.builder() - .logEvents(logEventList) - .logGroupName(logGroup) - .logStreamName(logStream) - .build(); - - cloudWatchLogsClient.putLogEvents(putLogEventsRequest); + sinkStopWatch.stopAndResetStopWatch(); + sinkStopWatch.startStopWatch(); - requestSuccessCount.increment(); - failedPost = false; + boolean succeededTransmission = logPusher.pushLogs(buffer, cloudWatchLogsClient, logGroup, logStream); + releaseEventHandles(succeededTransmission); - //TODO: When a log is rejected by the service, we cannot send it, can probably push to a DLQ here. - - } catch (AwsServiceException | SdkClientException e) { - LOG.error("Failed to push logs with error: {}", e.getMessage()); - - Thread.sleep(calculateBackOffTime(backOffTimeBase)); - - LOG.warn("Trying to retransmit request... {Attempt: " + (++failCounter) + "}"); - requestFailCount.increment(); - } - } - - buffer.clearBuffer(); - - if (failedPost) { - logEventFailCounter.increment(logEventList.size()); - releaseEventHandles(false); - LOG.error("Error, timed out trying to push logs!"); + if (!succeededTransmission) { throw new RetransmissionLimitException("Error, timed out trying to push logs! (Max retry_count reached: {" + retryCount + "})"); - } else { - logEventSuccessCounter.increment(logEventList.size()); - releaseEventHandles(true); - failCounter = 0; } } - private long calculateBackOffTime(long backOffTimeBase) { - return failCounter * backOffTimeBase; - } - private void runExitCheck() throws InterruptedException { - int bufferSizeWithOverHead = (buffer.getBufferSize() + (buffer.getEventCount() * LOG_EVENT_OVERHEAD_SIZE)); + int bufferSizeWithOverHead = (buffer.getBufferSize() + (buffer.getEventCount() * APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); if ((thresholdCheck.isEqualToThresholdReached(bufferSizeWithOverHead, buffer.getEventCount()) && (buffer.getEventCount() > 0))) { pushLogs(); } @@ -203,19 +178,4 @@ private void releaseEventHandles(final boolean result) { bufferedEventHandles.clear(); } - - private void startStopWatch() { - stopWatchOn = true; - stopWatch.start(); - } - - private void stopAndResetStopWatch() { - stopWatchOn = false; - stopWatch.stop(); - stopWatch.reset(); - } - - private long getStopWatchTime() { - return stopWatch.getTime(TimeUnit.SECONDS); - } } \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java index 969f4096d7..b5c7ad1245 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java @@ -30,8 +30,8 @@ public ThresholdCheck(final int batchSize, final int maxEventSizeBytes, final in * @return boolean - true if we exceed the threshold events or false otherwise. */ public boolean isGreaterThanThresholdReached(final long currentTime, final int currentRequestSize, final int batchSize) { - return ((isGreaterThanBatchSize(batchSize) || isGreaterEqualToLogSendInterval(currentTime) - || isGreaterThanMaxRequestSize(currentRequestSize))); + return (isGreaterThanBatchSize(batchSize) || isGreaterEqualToLogSendInterval(currentTime) + || isGreaterThanMaxRequestSize(currentRequestSize)); } /** diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/LogPusher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/LogPusher.java new file mode 100644 index 0000000000..bd34061b3f --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/LogPusher.java @@ -0,0 +1,89 @@ +package org.opensearch.dataprepper.plugins.sink.utils; + +import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.plugins.sink.buffer.Buffer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; +import software.amazon.awssdk.services.cloudwatchlogs.model.CloudWatchLogsException; +import software.amazon.awssdk.services.cloudwatchlogs.model.InputLogEvent; +import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; + +import java.util.ArrayList; + +public class LogPusher { + private final Counter logEventSuccessCounter; + private final Counter logEventFailCounter; + private final Counter requestSuccessCount; + private final Counter requestFailCount; + final int retryCount; + final long backOffTimeBase; + + static final Logger LOG = LoggerFactory.getLogger(LogPusher.class); + public LogPusher(Counter logEventSuccessCounter, Counter logEventFailCounter, Counter requestSuccessCount, Counter requestFailCount, final int retryCount, final long backOffTimeBase) { + this.logEventSuccessCounter = logEventSuccessCounter; + this.logEventFailCounter = logEventFailCounter; + this.requestSuccessCount = requestSuccessCount; + this.requestFailCount = requestFailCount; + this.retryCount = retryCount; + this.backOffTimeBase = backOffTimeBase; + } + + public boolean pushLogs(final Buffer buffer, final CloudWatchLogsClient cloudWatchLogsClient, final String logGroup, final String logStream) throws InterruptedException { + boolean failedPost = true; + int failCounter = 0; + + ArrayList logEventList = new ArrayList<>(); + + for (byte[] data: buffer.getBufferedData()) { + InputLogEvent tempLogEvent = InputLogEvent.builder() + .message(new String(data)) + .timestamp(System.currentTimeMillis()) + .build(); + logEventList.add(tempLogEvent); + } + + PutLogEventsRequest putLogEventsRequest = PutLogEventsRequest.builder() + .logEvents(logEventList) + .logGroupName(logGroup) + .logStreamName(logStream) + .build(); + + while (failedPost && (failCounter < retryCount)) { + try { + cloudWatchLogsClient.putLogEvents(putLogEventsRequest); + + requestSuccessCount.increment(); + failedPost = false; + + //TODO: When a log is rejected by the service, we cannot send it, can probably push to a DLQ here. + + } catch (CloudWatchLogsException | SdkClientException e) { + LOG.error("Failed to push logs with error: {}", e.getMessage()); + + requestFailCount.increment(); + + Thread.sleep(calculateBackOffTime(backOffTimeBase, failCounter)); + + LOG.warn("Trying to retransmit request... {Attempt: " + (++failCounter) + "}"); + } + } + + buffer.clearBuffer(); + + if (failedPost) { + logEventFailCounter.increment(logEventList.size()); + LOG.error("Error, timed out trying to push logs!"); + } else { + logEventSuccessCounter.increment(logEventList.size()); + return true; + } + + return false; + } + + private long calculateBackOffTime(final long backOffTimeBase, final int failCounter) { + return failCounter * backOffTimeBase; + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java new file mode 100644 index 0000000000..d26da22a47 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java @@ -0,0 +1,40 @@ +package org.opensearch.dataprepper.plugins.sink.utils; + +import org.apache.commons.lang3.time.StopWatch; + +import java.util.concurrent.TimeUnit; + +public class SinkStopWatch { + private final StopWatch stopWatch; + private boolean stopWatchOn; + + public SinkStopWatch() { + stopWatch = StopWatch.create(); + stopWatchOn = false; + } + + public void startIfNotRunning() { + if (!stopWatchOn) { + startStopWatch(); + } + } + + public void startStopWatch() { + stopWatchOn = true; + stopWatch.start(); + } + + public void stopAndResetStopWatch() { + stopWatchOn = false; + stopWatch.stop(); + stopWatch.reset(); + } + + public long getStopWatchTimeSeconds() { + return stopWatch.getTime(TimeUnit.SECONDS); + } + + public boolean isStopWatchOn() { + return stopWatchOn; + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java index c80b431eeb..9592b5037c 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java @@ -22,9 +22,9 @@ import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; import org.opensearch.dataprepper.plugins.sink.exception.RetransmissionLimitException; import org.opensearch.dataprepper.plugins.sink.threshold.ThresholdCheck; -import software.amazon.awssdk.awscore.exception.AwsServiceException; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; +import software.amazon.awssdk.services.cloudwatchlogs.model.CloudWatchLogsException; import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsResponse; @@ -105,10 +105,10 @@ void setUp() { when(awsConfig.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); when(awsConfig.getAwsStsExternalId()).thenReturn(externalId); - lenient().when(pluginMetrics.counter(CloudWatchLogsService.NUMBER_OF_RECORDS_PUSHED_TO_CWL_SUCCESS)).thenReturn(successEventCounter); - lenient().when(pluginMetrics.counter(CloudWatchLogsService.REQUESTS_SUCCEEDED)).thenReturn(requestSuccessCounter); - lenient().when(pluginMetrics.counter(CloudWatchLogsService.NUMBER_OF_RECORDS_PUSHED_TO_CWL_FAIL)).thenReturn(failedEventCounter); - lenient().when(pluginMetrics.counter(CloudWatchLogsService.REQUESTS_FAILED)).thenReturn(requestFailCounter); + lenient().when(pluginMetrics.counter(CloudWatchLogsService.CLOUDWATCH_LOGS_EVENTS_SUCCEEDED)).thenReturn(successEventCounter); + lenient().when(pluginMetrics.counter(CloudWatchLogsService.CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED)).thenReturn(requestSuccessCounter); + lenient().when(pluginMetrics.counter(CloudWatchLogsService.CLOUDWATCH_LOGS_EVENTS_FAILED)).thenReturn(failedEventCounter); + lenient().when(pluginMetrics.counter(CloudWatchLogsService.CLOUDWATCH_LOGS_REQUESTS_FAILED)).thenReturn(requestFailCounter); } void setThresholdForTestingRequestSize(int size) { @@ -133,7 +133,7 @@ void setMockClientNoErrors() { void setMockClientThrowCWLException() { mockClient = mock(CloudWatchLogsClient.class); - doThrow(AwsServiceException.class).when(mockClient).putLogEvents(any(PutLogEventsRequest.class)); + doThrow(CloudWatchLogsException.class).when(mockClient).putLogEvents(any(PutLogEventsRequest.class)); } Collection> getSampleRecords(int numberOfRecords) { @@ -270,7 +270,7 @@ void check_max_size_threshold_fail_test() { setMockClientNoErrors(); CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - final Collection> sampleEvents = getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - CloudWatchLogsService.LOG_EVENT_OVERHEAD_SIZE - messageKeyByteSize + 1); + final Collection> sampleEvents = getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - CloudWatchLogsService.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE - messageKeyByteSize + 1); cloudWatchLogsService.output(sampleEvents); @@ -283,7 +283,7 @@ void check_max_size_threshold_success_test() { setMockClientNoErrors(); CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - final Collection> sampleEvents = getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, (ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - messageKeyByteSize) - CloudWatchLogsService.LOG_EVENT_OVERHEAD_SIZE); + final Collection> sampleEvents = getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, (ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - messageKeyByteSize) - CloudWatchLogsService.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); cloudWatchLogsService.output(sampleEvents); @@ -297,7 +297,7 @@ void check_max_request_size_threshold_fail_test() { setMockClientNoErrors(); CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - cloudWatchLogsService.output(getSampleRecordsLarge(1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - messageKeyByteSize - CloudWatchLogsService.LOG_EVENT_OVERHEAD_SIZE + 1)); + cloudWatchLogsService.output(getSampleRecordsLarge(1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - messageKeyByteSize - CloudWatchLogsService.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE + 1)); verify(requestSuccessCounter, never()).increment(); } @@ -308,7 +308,7 @@ void check_max_request_size_threshold_success_test() { setMockClientNoErrors(); CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - cloudWatchLogsService.output(getSampleRecordsLarge(1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - messageKeyByteSize - CloudWatchLogsService.LOG_EVENT_OVERHEAD_SIZE)); + cloudWatchLogsService.output(getSampleRecordsLarge(1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - messageKeyByteSize - CloudWatchLogsService.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); verify(requestSuccessCounter, atLeast(1)).increment(); } @@ -319,7 +319,7 @@ void check_max_api_request_size_threshold_success_test() { setMockClientNoErrors(); CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - cloudWatchLogsService.output(getSampleRecordsLarge(1, (ThresholdConfig.DEFAULT_SIZE_OF_REQUEST * 2) - messageKeyByteSize - CloudWatchLogsService.LOG_EVENT_OVERHEAD_SIZE)); + cloudWatchLogsService.output(getSampleRecordsLarge(1, (ThresholdConfig.DEFAULT_SIZE_OF_REQUEST * 2) - messageKeyByteSize - CloudWatchLogsService.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); verify(requestSuccessCounter, atLeast(1)).increment(); } From 3c02e1d0de0f9aab27d552e79fcce0d3577d87be Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Fri, 14 Jul 2023 09:20:22 -0700 Subject: [PATCH 27/43] Changed method uses in CloudWatchLogsService and removed logging the batch size in LogPusher Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/client/CloudWatchLogsService.java | 9 ++++----- .../dataprepper/plugins/sink/utils/LogPusher.java | 3 +-- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index 9e422d4ce5..a640172cd1 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -49,10 +49,10 @@ public class CloudWatchLogsService { private final String logStream; private final int retryCount; private final long backOffTimeBase; - private final Counter logEventSuccessCounter; //Counter to be used on the fly for counting successful transmissions. (Success per single event successfully published). + private final Counter logEventSuccessCounter; private final Counter requestSuccessCount; private final Counter logEventFailCounter; - private final Counter requestFailCount; //Counter to be used on the fly during error handling. + private final Counter requestFailCount; private final SinkStopWatch sinkStopWatch; private final ReentrantLock reentrantLock; private final LogPusher logPusher; @@ -99,7 +99,6 @@ public void output(final Collection> logs) { if (processedLogsSuccessfully) { threadRetries = RETRY_THREAD_ERROR_CAP; } else { - LOG.error("Thread threw InterruptedException!"); threadRetries++; } } @@ -109,6 +108,7 @@ public void output(final Collection> logs) { } else { LOG.warn("Failed to process logs."); //TODO: Insert DLQ logic as a last resort if we cannot manage to process logs prior to this point. + buffer.clearBuffer(); } reentrantLock.unlock(); @@ -148,7 +148,6 @@ private boolean processLogEvents(final Collection> logs) { } private void pushLogs() throws InterruptedException { - LOG.info("Attempting to push logs! {Batch size: " + buffer.getEventCount() + "}"); sinkStopWatch.stopAndResetStopWatch(); sinkStopWatch.startStopWatch(); @@ -168,7 +167,7 @@ private void runExitCheck() throws InterruptedException { } private void releaseEventHandles(final boolean result) { - if (bufferedEventHandles.size() == 0) { + if (bufferedEventHandles.isEmpty()) { return; } diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/LogPusher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/LogPusher.java index bd34061b3f..e2d5e485bd 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/LogPusher.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/LogPusher.java @@ -19,9 +19,8 @@ public class LogPusher { private final Counter requestFailCount; final int retryCount; final long backOffTimeBase; - static final Logger LOG = LoggerFactory.getLogger(LogPusher.class); - public LogPusher(Counter logEventSuccessCounter, Counter logEventFailCounter, Counter requestSuccessCount, Counter requestFailCount, final int retryCount, final long backOffTimeBase) { + public LogPusher(final Counter logEventSuccessCounter, final Counter logEventFailCounter, final Counter requestSuccessCount, final Counter requestFailCount, final int retryCount, final long backOffTimeBase) { this.logEventSuccessCounter = logEventSuccessCounter; this.logEventFailCounter = logEventFailCounter; this.requestSuccessCount = requestSuccessCount; From c2a02eca889bf503c9182503a32162fb88d45ed1 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Wed, 19 Jul 2023 14:13:56 -0700 Subject: [PATCH 28/43] Added Multithreaded CloudWatchLogsDispatcher for handling various async calls to perform PLE's and added tests Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/buffer/Buffer.java | 3 +- .../plugins/sink/buffer/InMemoryBuffer.java | 9 +- .../sink/client/CloudWatchLogsDispatcher.java | 134 +++++++ .../sink/client/CloudWatchLogsMetrics.java | 48 +++ .../sink/client/CloudWatchLogsService.java | 181 +++------- .../plugins/sink/config/ThresholdConfig.java | 5 +- .../sink/packaging/ThreadTaskEvents.java | 25 ++ .../CloudWatchLogsLimits.java} | 19 +- .../plugins/sink/time/SinkStopWatch.java | 45 +++ .../plugins/sink/utils/LogPusher.java | 88 ----- .../plugins/sink/utils/SinkStopWatch.java | 40 --- .../client/CloudWatchLogsDispatcherTest.java | 114 ++++++ .../client/CloudWatchLogsServiceTest.java | 297 ++-------------- .../client/CloudWatchLogsServiceTestOld.java | 326 ++++++++++++++++++ .../sink/config/ThresholdConfigTest.java | 4 +- .../CloudWatchLogsLimitsTest.java} | 33 +- 16 files changed, 826 insertions(+), 545 deletions(-) create mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsMetrics.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/packaging/ThreadTaskEvents.java rename data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/{threshold/ThresholdCheck.java => push_condition/CloudWatchLogsLimits.java} (76%) create mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/time/SinkStopWatch.java delete mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/LogPusher.java delete mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTestOld.java rename data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/{threshold/ThresholdCheckTest.java => something/CloudWatchLogsLimitsTest.java} (55%) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java index bfdfb0d825..a0c20c13ab 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.sink.buffer; import java.util.ArrayList; +import java.util.List; /** * Buffer that handles the temporary storage of @@ -31,7 +32,7 @@ public interface Buffer { byte[] popEvent(); - ArrayList getBufferedData(); + List getBufferedData(); void clearBuffer(); } \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java index 25de30ea98..a93cb99ad7 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java @@ -6,9 +6,11 @@ package org.opensearch.dataprepper.plugins.sink.buffer; import java.util.ArrayList; +import java.util.Collections; +import java.util.List; public class InMemoryBuffer implements Buffer { - private final ArrayList eventsBuffered; + private final List eventsBuffered; private int bufferSize = 0; InMemoryBuffer() { @@ -38,8 +40,9 @@ public byte[] popEvent() { } @Override - public ArrayList getBufferedData() { - return eventsBuffered; + public List getBufferedData() { + Collections.unmodifiableList(eventsBuffered); + return Collections.unmodifiableList(eventsBuffered); } @Override diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java new file mode 100644 index 0000000000..d035aba0e1 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java @@ -0,0 +1,134 @@ +package org.opensearch.dataprepper.plugins.sink.client; + +import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.plugins.sink.packaging.ThreadTaskEvents; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; +import software.amazon.awssdk.services.cloudwatchlogs.model.CloudWatchLogsException; +import software.amazon.awssdk.services.cloudwatchlogs.model.InputLogEvent; +import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.BlockingQueue; + +public class CloudWatchLogsDispatcher implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(CloudWatchLogsDispatcher.class); + private final BlockingQueue taskQueue; + private final CloudWatchLogsClient cloudWatchLogsClient; + private final CloudWatchLogsMetrics cloudWatchLogsMetrics; + private final String logGroup; + private final String logStream; + final int retryCount; + final long backOffTimeBase; + public CloudWatchLogsDispatcher(final BlockingQueue taskQueue, + final CloudWatchLogsClient cloudWatchLogsClient, + final CloudWatchLogsMetrics cloudWatchLogsMetrics, + final String logGroup, final String logStream, + final int retryCount, final int backOffTimeBase) { + this.taskQueue = taskQueue; + this.cloudWatchLogsClient = cloudWatchLogsClient; + this.cloudWatchLogsMetrics = cloudWatchLogsMetrics; + this.logGroup = logGroup; + this.logStream = logStream; + this.retryCount = retryCount; + this.backOffTimeBase = backOffTimeBase; + } + + private List prepareInputLogEvents(final ThreadTaskEvents eventData) { + List logEventList = new ArrayList<>(); + + for (byte[] data: eventData.getEventMessages()) { + InputLogEvent tempLogEvent = InputLogEvent.builder() + .message(new String(data)) + .timestamp(System.currentTimeMillis()) + .build(); + logEventList.add(tempLogEvent); + } + + return logEventList; + } + + /** + * Flush function to handle the flushing of logs to CloudWatchLogs services; + * @param inputLogEvents Collection of inputLogEvents to be flushed + * @return true if successful, false otherwise + * @throws InterruptedException + */ + public boolean dispatchLogs(List inputLogEvents, Collection eventHandles) { + boolean failedPost = true; + int failCounter = 0; + + PutLogEventsRequest putLogEventsRequest = PutLogEventsRequest.builder() + .logEvents(inputLogEvents) + .logGroupName(logGroup) + .logStreamName(logStream) + .build(); + + //TODO: Could also continue to retry even with InterruptedException instead of directly pushing to DLQ. + try { + while (failedPost && (failCounter < retryCount)) { + try { + cloudWatchLogsClient.putLogEvents(putLogEventsRequest); + + cloudWatchLogsMetrics.increaseRequestSuccessCounter(1); + failedPost = false; + + //TODO: When a log is rejected by the service, we cannot send it, can probably push to a DLQ here. + + } catch (CloudWatchLogsException | SdkClientException e) { + LOG.error("Failed to push logs with error: {}", e.getMessage()); + cloudWatchLogsMetrics.increaseRequestFailCounter(1); + Thread.sleep(calculateBackOffTime(backOffTimeBase, failCounter)); + LOG.warn("Trying to retransmit request... {Attempt: {} }", (++failCounter)); + } + } + } catch (InterruptedException e) { + LOG.warn("Got interrupted while waiting!"); + //TODO: Push to DLQ. + } + + + if (failedPost) { + cloudWatchLogsMetrics.increaseLogEventFailCounter(inputLogEvents.size()); + LOG.error("Error, timed out trying to push logs!"); + releaseEventHandles(false, eventHandles); + return false; + } else { + cloudWatchLogsMetrics.increaseLogEventSuccessCounter(inputLogEvents.size()); + LOG.info("Succeeded in publishing logs!"); + releaseEventHandles(true, eventHandles); + return true; + } + } + + //TODO: Can abstract this if clients want more choice. + private long calculateBackOffTime(final long backOffTimeBase, final int failCounter) { + return failCounter * backOffTimeBase; + } + + @Override + public void run() { + try { + ThreadTaskEvents taskData = taskQueue.take(); + List inputLogEvents = prepareInputLogEvents(taskData); + dispatchLogs(inputLogEvents, taskData.getEventHandles()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + //TODO: Implement back up to taskQueue read failure. + } + } + + private void releaseEventHandles(final boolean result, final Collection eventHandles) { + if (eventHandles.isEmpty()) { + return; + } + + for (EventHandle eventHandle : eventHandles) { + eventHandle.release(result); + } + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsMetrics.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsMetrics.java new file mode 100644 index 0000000000..1c2624f935 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsMetrics.java @@ -0,0 +1,48 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.client; + +import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.metrics.PluginMetrics; + +/** + * Class is meant to abstract the metric book-keeping of + * CloudWatchLogs metrics so that multiple instances + * may refer to it. + */ +public class CloudWatchLogsMetrics { + public static final String CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED = "cloudWatchLogsRequestsSucceeded"; + public static final String CLOUDWATCH_LOGS_EVENTS_SUCCEEDED = "cloudWatchLogsEventsSucceeded"; + public static final String CLOUDWATCH_LOGS_EVENTS_FAILED = "cloudWatchLogsEventsFailed"; + public static final String CLOUDWATCH_LOGS_REQUESTS_FAILED = "cloudWatchLogsRequestsFailed"; + private final Counter logEventSuccessCounter; + private final Counter logEventFailCounter; + private final Counter requestSuccessCount; + private final Counter requestFailCount; + + public CloudWatchLogsMetrics(final PluginMetrics pluginMetrics) { + this.logEventSuccessCounter = pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_EVENTS_SUCCEEDED); + this.requestFailCount = pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_REQUESTS_FAILED); + this.logEventFailCounter = pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_EVENTS_FAILED); + this.requestSuccessCount = pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED); + } + + public void increaseLogEventSuccessCounter(int value) { + logEventSuccessCounter.increment(value); + } + + public void increaseRequestSuccessCounter(int value) { + requestSuccessCount.increment(value); + } + + public void increaseLogEventFailCounter(int value) { + logEventFailCounter.increment(value); + } + + public void increaseRequestFailCounter(int value) { + requestFailCount.increment(value); + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index a640172cd1..b148a6fcf4 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -5,81 +5,51 @@ package org.opensearch.dataprepper.plugins.sink.client; -import io.micrometer.core.instrument.Counter; -import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventHandle; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.sink.buffer.Buffer; -import org.opensearch.dataprepper.plugins.sink.config.CloudWatchLogsSinkConfig; -import org.opensearch.dataprepper.plugins.sink.exception.RetransmissionLimitException; -import org.opensearch.dataprepper.plugins.sink.threshold.ThresholdCheck; -import org.opensearch.dataprepper.plugins.sink.utils.LogPusher; -import org.opensearch.dataprepper.plugins.sink.utils.SinkStopWatch; +import org.opensearch.dataprepper.plugins.sink.packaging.ThreadTaskEvents; +import org.opensearch.dataprepper.plugins.sink.push_condition.CloudWatchLogsLimits; +import org.opensearch.dataprepper.plugins.sink.time.SinkStopWatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Executor; import java.util.concurrent.locks.ReentrantLock; -/* - TODO: Can add DLQ logic here for sending these logs to a particular DLQ for error checking. (Explicitly for bad formatted logs). - as currently the logs that are able to be published but rejected by CloudWatch Logs will simply be deleted if not deferred to - a backup storage. -*/ -//TODO: Must also consider if the customer makes the logEvent size bigger than the send request size. -//TODO: Can inject another class for the stopWatch functionality. +import static java.util.concurrent.Executors.newCachedThreadPool; //TODO: Can implement a more strict pooling method if needed. public class CloudWatchLogsService { - public static final int APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE = 26; //Size of overhead for each log event message. - public static final String CLOUDWATCH_LOGS_EVENTS_SUCCEEDED = "cloudWatchLogsEventsSucceeded"; - public static final String CLOUDWATCH_LOGS_EVENTS_FAILED = "cloudWatchLogsEventsFailed"; - public static final String CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED = "cloudWatchLogsRequestsSucceeded"; - public static final String CLOUDWATCH_LOGS_REQUESTS_FAILED = "cloudWatchLogsRequestsFailed"; private static final Logger LOG = LoggerFactory.getLogger(CloudWatchLogsService.class); - private static final int RETRY_THREAD_ERROR_CAP = 3; - private final CloudWatchLogsClient cloudWatchLogsClient; + private static final int MAX_BLOCKING_QUEUE_SIZE = 10; private final Buffer buffer; - private final ThresholdCheck thresholdCheck; - private final List bufferedEventHandles; - private final String logGroup; - private final String logStream; - private final int retryCount; - private final long backOffTimeBase; - private final Counter logEventSuccessCounter; - private final Counter requestSuccessCount; - private final Counter logEventFailCounter; - private final Counter requestFailCount; + private final CloudWatchLogsLimits cloudWatchLogsLimits; + private List bufferedEventHandles; + private final BlockingQueue taskQueue; private final SinkStopWatch sinkStopWatch; - private final ReentrantLock reentrantLock; - private final LogPusher logPusher; + private final ReentrantLock bufferLock; + private final Executor sinkThreadManager; + private final CloudWatchLogsDispatcher dispatcher; - public CloudWatchLogsService(final CloudWatchLogsClient cloudWatchLogsClient, final CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig, final Buffer buffer, - final PluginMetrics pluginMetrics, final ThresholdCheck thresholdCheck, final int retryCount, final long backOffTimeBase) { + public CloudWatchLogsService(final Buffer buffer, + final CloudWatchLogsLimits cloudWatchLogsLimits, + final CloudWatchLogsDispatcher dispatcher) { - this.cloudWatchLogsClient = cloudWatchLogsClient; this.buffer = buffer; - this.logGroup = cloudWatchLogsSinkConfig.getLogGroup(); - this.logStream = cloudWatchLogsSinkConfig.getLogStream(); - this.thresholdCheck = thresholdCheck; - - this.retryCount = retryCount; - this.backOffTimeBase = backOffTimeBase; - + this.dispatcher = dispatcher; this.bufferedEventHandles = new ArrayList<>(); - this.logEventSuccessCounter = pluginMetrics.counter(CLOUDWATCH_LOGS_EVENTS_SUCCEEDED); - this.requestFailCount = pluginMetrics.counter(CLOUDWATCH_LOGS_REQUESTS_FAILED); - this.logEventFailCounter = pluginMetrics.counter(CLOUDWATCH_LOGS_EVENTS_FAILED); - this.requestSuccessCount = pluginMetrics.counter(CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED); - - reentrantLock = new ReentrantLock(); + this.cloudWatchLogsLimits = cloudWatchLogsLimits; + this.taskQueue = new ArrayBlockingQueue<>(MAX_BLOCKING_QUEUE_SIZE); + bufferLock = new ReentrantLock(); sinkStopWatch = new SinkStopWatch(); - - this.logPusher = new LogPusher(logEventSuccessCounter, logEventFailCounter, requestSuccessCount, requestFailCount, retryCount, backOffTimeBase); + sinkThreadManager = newCachedThreadPool(); } /** @@ -87,94 +57,55 @@ public CloudWatchLogsService(final CloudWatchLogsClient cloudWatchLogsClient, fi * Implements simple conditional buffer. (Sends once batch size, request size in bytes, or time limit is reached) * @param logs - Collection of Record events which hold log data. */ - public void output(final Collection> logs) { - reentrantLock.lock(); - - int threadRetries = 0; - boolean processedLogsSuccessfully = false; - - while (threadRetries < RETRY_THREAD_ERROR_CAP) { - processedLogsSuccessfully = processLogEvents(logs); - - if (processedLogsSuccessfully) { - threadRetries = RETRY_THREAD_ERROR_CAP; - } else { - threadRetries++; + public void processLogEvents(final Collection> logs) { + sinkStopWatch.startIfNotRunning(); + for (Record log: logs) { + int logLength = log.getData().toJsonString().length(); + + if (cloudWatchLogsLimits.isGreaterThanMaxEventSize(logLength)) { + LOG.warn("Event blocked due to Max Size restriction! {Event Size: " + (logLength + CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE) + " bytes}"); + continue; } - } - if (processedLogsSuccessfully) { - LOG.info("Successfully processed logs."); - } else { - LOG.warn("Failed to process logs."); - //TODO: Insert DLQ logic as a last resort if we cannot manage to process logs prior to this point. - buffer.clearBuffer(); - } + long time = sinkStopWatch.getStopWatchTimeSeconds(); - reentrantLock.unlock(); - } + bufferLock.lock(); - private boolean processLogEvents(final Collection> logs) { - try { - sinkStopWatch.startIfNotRunning(); + int bufferSize = buffer.getBufferSize(); + int bufferEventCount = buffer.getEventCount(); + int bufferEventCountWithEvent = bufferEventCount + 1; + int bufferSizeWithAddedEvent = bufferSize + logLength; - for (Record log: logs) { - String logJsonString = log.getData().toJsonString(); - int logLength = logJsonString.length(); - - if (thresholdCheck.isGreaterThanMaxEventSize(logLength + APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)) { - LOG.warn("Event blocked due to Max Size restriction! {Event Size: " + (logLength + APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE) + " bytes}"); - continue; - } - - int bufferSizeWithOverhead = (buffer.getBufferSize() + (buffer.getEventCount() * APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); - if ((thresholdCheck.isGreaterThanThresholdReached(sinkStopWatch.getStopWatchTimeSeconds(), bufferSizeWithOverhead + logLength + APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE, buffer.getEventCount() + 1) && (buffer.getEventCount() > 0))) { - pushLogs(); - } - - if (log.getData().getEventHandle() != null) { - bufferedEventHandles.add(log.getData().getEventHandle()); - } - buffer.writeEvent(logJsonString.getBytes()); + if ((cloudWatchLogsLimits.isGreaterThanLimitReached(time, bufferSizeWithAddedEvent, bufferEventCountWithEvent) && (bufferEventCount > 0))) { + stageLogEvents(); + addToBuffer(log); + } else if (cloudWatchLogsLimits.isEqualToLimitReached(bufferSizeWithAddedEvent, bufferEventCountWithEvent)) { + addToBuffer(log); + stageLogEvents(); + } else { + addToBuffer(log); } - runExitCheck(); - - return true; - } catch (InterruptedException e) { - LOG.error("Caught InterruptedException while attempting to publish logs!"); - return false; + bufferLock.unlock(); } } - private void pushLogs() throws InterruptedException { + private void stageLogEvents() { sinkStopWatch.stopAndResetStopWatch(); - sinkStopWatch.startStopWatch(); - boolean succeededTransmission = logPusher.pushLogs(buffer, cloudWatchLogsClient, logGroup, logStream); - releaseEventHandles(succeededTransmission); + ThreadTaskEvents dataToPush = new ThreadTaskEvents(buffer.getBufferedData(), bufferedEventHandles); + taskQueue.add(dataToPush); - if (!succeededTransmission) { - throw new RetransmissionLimitException("Error, timed out trying to push logs! (Max retry_count reached: {" + retryCount + "})"); - } - } + bufferedEventHandles = new ArrayList<>(); + buffer.clearBuffer(); - private void runExitCheck() throws InterruptedException { - int bufferSizeWithOverHead = (buffer.getBufferSize() + (buffer.getEventCount() * APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); - if ((thresholdCheck.isEqualToThresholdReached(bufferSizeWithOverHead, buffer.getEventCount()) && (buffer.getEventCount() > 0))) { - pushLogs(); - } + sinkThreadManager.execute(dispatcher); } - private void releaseEventHandles(final boolean result) { - if (bufferedEventHandles.isEmpty()) { - return; + private void addToBuffer(final Record log) { + if (log.getData().getEventHandle() != null) { + bufferedEventHandles.add(log.getData().getEventHandle()); } - - for (EventHandle eventHandle : bufferedEventHandles) { - eventHandle.release(result); - } - - bufferedEventHandles.clear(); + buffer.writeEvent(log.getData().toString().getBytes()); } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java index a84cdf4a89..55c6f6e0f5 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java @@ -20,6 +20,7 @@ public class ThresholdConfig { public static final int DEFAULT_RETRY_COUNT = 5; public static final int DEFAULT_LOG_SEND_INTERVAL_TIME = 60; public static final int DEFAULT_BACKOFF_TIME = 500; + public static final int CONVERT_TO_BYTES_FROM_KB = 1024; @JsonProperty("batch_size") @Size(min = 1, max = 10000, message = "batch_size amount should be between 1 to 10000") @@ -49,8 +50,8 @@ public int getBatchSize() { return batchSize; } - public int getMaxEventSize() { - return maxEventSize; + public int getMaxEventSizeBytes() { + return maxEventSize * CONVERT_TO_BYTES_FROM_KB; } public int getMaxRequestSize() { diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/packaging/ThreadTaskEvents.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/packaging/ThreadTaskEvents.java new file mode 100644 index 0000000000..89fa32c207 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/packaging/ThreadTaskEvents.java @@ -0,0 +1,25 @@ +package org.opensearch.dataprepper.plugins.sink.packaging; + +import org.opensearch.dataprepper.model.event.EventHandle; + +import java.util.Collection; + +/** + * Simple data class for packaging event messages and their handles into a queue. + */ +public class ThreadTaskEvents { + Collection eventMessages; + Collection eventHandles; + public ThreadTaskEvents(Collection eventMessages, Collection eventHandles) { + this.eventMessages = eventMessages; + this.eventHandles = eventHandles; + } + + public Collection getEventMessages() { + return eventMessages; + } + + public Collection getEventHandles() { + return eventHandles; + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimits.java similarity index 76% rename from data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java rename to data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimits.java index b5c7ad1245..d1fc837da9 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheck.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimits.java @@ -3,19 +3,20 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.threshold; +package org.opensearch.dataprepper.plugins.sink.push_condition; /** * ThresholdCheck receives parameters for which to reference the * limits of a buffer and CloudWatchLogsClient before making a * PutLogEvent request to AWS. */ -public class ThresholdCheck { +public class CloudWatchLogsLimits { + public static final int APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE = 26; //Size of overhead for each log event message. private final int batchSize; private final int maxEventSizeBytes; private final int maxRequestSizeBytes; private final long logSendInterval; - public ThresholdCheck(final int batchSize, final int maxEventSizeBytes, final int maxRequestSizeBytes, final int logSendInterval) { + public CloudWatchLogsLimits(final int batchSize, final int maxEventSizeBytes, final int maxRequestSizeBytes, final int logSendInterval) { this.batchSize = batchSize; this.maxEventSizeBytes = maxEventSizeBytes; this.maxRequestSizeBytes = maxRequestSizeBytes; @@ -29,9 +30,10 @@ public ThresholdCheck(final int batchSize, final int maxEventSizeBytes, final in * @param batchSize - size of batch in events. * @return boolean - true if we exceed the threshold events or false otherwise. */ - public boolean isGreaterThanThresholdReached(final long currentTime, final int currentRequestSize, final int batchSize) { + public boolean isGreaterThanLimitReached(final long currentTime, final int currentRequestSize, final int batchSize) { + int bufferSizeWithOverhead = (currentRequestSize + ((batchSize) * APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); return (isGreaterThanBatchSize(batchSize) || isGreaterEqualToLogSendInterval(currentTime) - || isGreaterThanMaxRequestSize(currentRequestSize)); + || isGreaterThanMaxRequestSize(bufferSizeWithOverhead)); } /** @@ -40,8 +42,9 @@ public boolean isGreaterThanThresholdReached(final long currentTime, final int c * @param batchSize - size of batch in events. * @return boolean - true if we equal the threshold events or false otherwise. */ - public boolean isEqualToThresholdReached(final int currentRequestSize, final int batchSize) { - return ((isEqualBatchSize(batchSize) || isEqualMaxRequestSize(currentRequestSize))); + public boolean isEqualToLimitReached(final int currentRequestSize, final int batchSize) { + int bufferSizeWithOverhead = (currentRequestSize + ((batchSize) * APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + return (isEqualBatchSize(batchSize) || isEqualMaxRequestSize(bufferSizeWithOverhead)); } /** @@ -60,7 +63,7 @@ private boolean isGreaterEqualToLogSendInterval(final long currentTimeSeconds) { * @return boolean - true if greater than MaxEventSize, false otherwise. */ public boolean isGreaterThanMaxEventSize(final int eventSize) { - return eventSize > maxEventSizeBytes; + return (eventSize + APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE) > maxEventSizeBytes; } /** diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/time/SinkStopWatch.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/time/SinkStopWatch.java new file mode 100644 index 0000000000..d973f5d3a0 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/time/SinkStopWatch.java @@ -0,0 +1,45 @@ +package org.opensearch.dataprepper.plugins.sink.time; + +import org.apache.commons.lang3.time.StopWatch; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantLock; + +public class SinkStopWatch { + private final StopWatch stopWatch; + private final ReentrantLock stopWatchLock; + + public SinkStopWatch() { + stopWatch = StopWatch.create(); + stopWatchLock = new ReentrantLock(); + } + + public void startIfNotRunning() { + stopWatchLock.lock(); + if (!stopWatch.isStarted()) { + startStopWatch(); + } + stopWatchLock.unlock(); + } + + public void startStopWatch() { + stopWatchLock.lock(); + stopWatch.start(); + stopWatchLock.unlock(); + } + + public void stopAndResetStopWatch() { + stopWatchLock.lock(); + stopWatch.stop(); + stopWatch.reset(); + stopWatch.start(); + stopWatchLock.unlock(); + } + + public long getStopWatchTimeSeconds() { + stopWatchLock.lock(); + long time = stopWatch.getTime(TimeUnit.SECONDS); + stopWatchLock.unlock(); + return time; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/LogPusher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/LogPusher.java deleted file mode 100644 index e2d5e485bd..0000000000 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/LogPusher.java +++ /dev/null @@ -1,88 +0,0 @@ -package org.opensearch.dataprepper.plugins.sink.utils; - -import io.micrometer.core.instrument.Counter; -import org.opensearch.dataprepper.plugins.sink.buffer.Buffer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import software.amazon.awssdk.core.exception.SdkClientException; -import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; -import software.amazon.awssdk.services.cloudwatchlogs.model.CloudWatchLogsException; -import software.amazon.awssdk.services.cloudwatchlogs.model.InputLogEvent; -import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; - -import java.util.ArrayList; - -public class LogPusher { - private final Counter logEventSuccessCounter; - private final Counter logEventFailCounter; - private final Counter requestSuccessCount; - private final Counter requestFailCount; - final int retryCount; - final long backOffTimeBase; - static final Logger LOG = LoggerFactory.getLogger(LogPusher.class); - public LogPusher(final Counter logEventSuccessCounter, final Counter logEventFailCounter, final Counter requestSuccessCount, final Counter requestFailCount, final int retryCount, final long backOffTimeBase) { - this.logEventSuccessCounter = logEventSuccessCounter; - this.logEventFailCounter = logEventFailCounter; - this.requestSuccessCount = requestSuccessCount; - this.requestFailCount = requestFailCount; - this.retryCount = retryCount; - this.backOffTimeBase = backOffTimeBase; - } - - public boolean pushLogs(final Buffer buffer, final CloudWatchLogsClient cloudWatchLogsClient, final String logGroup, final String logStream) throws InterruptedException { - boolean failedPost = true; - int failCounter = 0; - - ArrayList logEventList = new ArrayList<>(); - - for (byte[] data: buffer.getBufferedData()) { - InputLogEvent tempLogEvent = InputLogEvent.builder() - .message(new String(data)) - .timestamp(System.currentTimeMillis()) - .build(); - logEventList.add(tempLogEvent); - } - - PutLogEventsRequest putLogEventsRequest = PutLogEventsRequest.builder() - .logEvents(logEventList) - .logGroupName(logGroup) - .logStreamName(logStream) - .build(); - - while (failedPost && (failCounter < retryCount)) { - try { - cloudWatchLogsClient.putLogEvents(putLogEventsRequest); - - requestSuccessCount.increment(); - failedPost = false; - - //TODO: When a log is rejected by the service, we cannot send it, can probably push to a DLQ here. - - } catch (CloudWatchLogsException | SdkClientException e) { - LOG.error("Failed to push logs with error: {}", e.getMessage()); - - requestFailCount.increment(); - - Thread.sleep(calculateBackOffTime(backOffTimeBase, failCounter)); - - LOG.warn("Trying to retransmit request... {Attempt: " + (++failCounter) + "}"); - } - } - - buffer.clearBuffer(); - - if (failedPost) { - logEventFailCounter.increment(logEventList.size()); - LOG.error("Error, timed out trying to push logs!"); - } else { - logEventSuccessCounter.increment(logEventList.size()); - return true; - } - - return false; - } - - private long calculateBackOffTime(final long backOffTimeBase, final int failCounter) { - return failCounter * backOffTimeBase; - } -} diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java deleted file mode 100644 index d26da22a47..0000000000 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java +++ /dev/null @@ -1,40 +0,0 @@ -package org.opensearch.dataprepper.plugins.sink.utils; - -import org.apache.commons.lang3.time.StopWatch; - -import java.util.concurrent.TimeUnit; - -public class SinkStopWatch { - private final StopWatch stopWatch; - private boolean stopWatchOn; - - public SinkStopWatch() { - stopWatch = StopWatch.create(); - stopWatchOn = false; - } - - public void startIfNotRunning() { - if (!stopWatchOn) { - startStopWatch(); - } - } - - public void startStopWatch() { - stopWatchOn = true; - stopWatch.start(); - } - - public void stopAndResetStopWatch() { - stopWatchOn = false; - stopWatch.stop(); - stopWatch.reset(); - } - - public long getStopWatchTimeSeconds() { - return stopWatch.getTime(TimeUnit.SECONDS); - } - - public boolean isStopWatchOn() { - return stopWatchOn; - } -} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java new file mode 100644 index 0000000000..b2d8856e6e --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java @@ -0,0 +1,114 @@ +package org.opensearch.dataprepper.plugins.sink.client; + +import io.micrometer.core.instrument.Counter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; +import org.opensearch.dataprepper.plugins.sink.packaging.ThreadTaskEvents; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; +import software.amazon.awssdk.services.cloudwatchlogs.model.CloudWatchLogsException; +import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; + +import java.util.ArrayList; +import java.util.concurrent.BlockingQueue; + +import static org.mockito.Mockito.*; + +public class CloudWatchLogsDispatcherTest { + private CloudWatchLogsDispatcher cloudWatchLogsDispatcher; + private BlockingQueue mockTaskQueue; + private CloudWatchLogsClient cloudWatchLogsClient; + private CloudWatchLogsMetrics cloudWatchLogsMetrics; + private PluginMetrics pluginMetrics; + private Counter requestSuccessCounter; + private Counter requestFailCounter; + private Counter successEventCounter; + private Counter failedEventCounter; + private static final String LOG_GROUP = "testGroup"; + private static final String LOG_STREAM = "testStream"; + private static final String TEST_STRING = "testMessage"; + + @BeforeEach + void setUp() throws InterruptedException { + mockTaskQueue = mock(BlockingQueue.class); + cloudWatchLogsClient = mock(CloudWatchLogsClient.class); + + pluginMetrics = mock(PluginMetrics.class); + requestSuccessCounter = mock(Counter.class); + requestFailCounter = mock(Counter.class); + successEventCounter = mock(Counter.class); + failedEventCounter = mock(Counter.class); + + when(mockTaskQueue.take()).thenReturn(getSampleBufferedData()); + + cloudWatchLogsMetrics = mock(CloudWatchLogsMetrics.class); + + when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_EVENTS_SUCCEEDED)).thenReturn(successEventCounter); + when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED)).thenReturn(requestSuccessCounter); + when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_EVENTS_FAILED)).thenReturn(failedEventCounter); + when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_REQUESTS_FAILED)).thenReturn(requestFailCounter); + } + + ThreadTaskEvents getSampleBufferedData() { + final ArrayList returnCollection = new ArrayList<>(); + final ArrayList eventHandles = new ArrayList<>(); + + for (int i = 0; i < ThresholdConfig.DEFAULT_BATCH_SIZE; i++) { + returnCollection.add(new String(TEST_STRING).getBytes()); + final EventHandle mockEventHandle = mock(EventHandle.class); + eventHandles.add(mockEventHandle); + } + + return new ThreadTaskEvents(returnCollection, eventHandles); + } + + CloudWatchLogsDispatcher getCloudWatchLogsDispatcher() { + return new CloudWatchLogsDispatcher(mockTaskQueue, cloudWatchLogsClient, + cloudWatchLogsMetrics, LOG_GROUP, LOG_STREAM, ThresholdConfig.DEFAULT_RETRY_COUNT, + ThresholdConfig.DEFAULT_BACKOFF_TIME); + } + + void establishFailingClientWithCloudWatchLogsExcept() { + when(cloudWatchLogsClient.putLogEvents(any(PutLogEventsRequest.class))).thenThrow(CloudWatchLogsException.class); + } + + void establishFailingClientWithSdkClientExcept() { + when(cloudWatchLogsClient.putLogEvents(any(PutLogEventsRequest.class))).thenThrow(SdkClientException.class); + } + + void setUpInterruptedQueueException() throws InterruptedException { + when(mockTaskQueue.take()).thenThrow(InterruptedException.class); + } + + @Test + void check_successful_transmission_test() { + cloudWatchLogsDispatcher = getCloudWatchLogsDispatcher(); + cloudWatchLogsDispatcher.run(); + + verify(cloudWatchLogsMetrics, atLeastOnce()).increaseRequestSuccessCounter(1); + verify(cloudWatchLogsMetrics, atLeastOnce()).increaseLogEventSuccessCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); + } + + @Test + void check_unsuccesful_transmission_with_cloudwatchlogsexcept_test() { + establishFailingClientWithCloudWatchLogsExcept(); + cloudWatchLogsDispatcher = getCloudWatchLogsDispatcher(); + cloudWatchLogsDispatcher.run(); + + verify(cloudWatchLogsMetrics, times(ThresholdConfig.DEFAULT_RETRY_COUNT)).increaseRequestFailCounter(1); + verify(cloudWatchLogsMetrics, atLeastOnce()).increaseLogEventFailCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); + } + + @Test + void check_unsuccesful_transmission_with_sdkexcept_test() { + establishFailingClientWithSdkClientExcept(); + cloudWatchLogsDispatcher = getCloudWatchLogsDispatcher(); + cloudWatchLogsDispatcher.run(); + + verify(cloudWatchLogsMetrics, times(ThresholdConfig.DEFAULT_RETRY_COUNT)).increaseRequestFailCounter(1); + verify(cloudWatchLogsMetrics, atLeastOnce()).increaseLogEventFailCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java index 9592b5037c..bc393364fc 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java @@ -1,144 +1,53 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - package org.opensearch.dataprepper.plugins.sink.client; -import io.micrometer.core.instrument.Counter; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; -import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventHandle; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.sink.buffer.Buffer; -import org.opensearch.dataprepper.plugins.sink.buffer.BufferFactory; import org.opensearch.dataprepper.plugins.sink.buffer.InMemoryBufferFactory; -import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; import org.opensearch.dataprepper.plugins.sink.config.CloudWatchLogsSinkConfig; import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; -import org.opensearch.dataprepper.plugins.sink.exception.RetransmissionLimitException; -import org.opensearch.dataprepper.plugins.sink.threshold.ThresholdCheck; -import software.amazon.awssdk.regions.Region; -import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; -import software.amazon.awssdk.services.cloudwatchlogs.model.CloudWatchLogsException; -import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; -import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsResponse; +import org.opensearch.dataprepper.plugins.sink.push_condition.CloudWatchLogsLimits; import java.util.ArrayList; import java.util.Collection; -import java.util.Map; -import java.util.UUID; -import java.util.stream.Collectors; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.notNullValue; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import static org.mockito.Mockito.lenient; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.anyDouble; -import static org.mockito.Mockito.atLeast; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.atLeastOnce; -//TODO: Add Codec session. -//TODO: Finish adding feature for ARN reading. +import static org.mockito.Mockito.*; public class CloudWatchLogsServiceTest { - private CloudWatchLogsClient mockClient; - private PutLogEventsResponse putLogEventsResponse; + private CloudWatchLogsService cloudWatchLogsService; private CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig; private ThresholdConfig thresholdConfig; - private ThresholdCheck thresholdCheck; - private AwsConfig awsConfig; - private AwsCredentialsSupplier awsCredentialsSupplier; - private BufferFactory bufferFactory; + private CloudWatchLogsLimits cloudWatchLogsLimits; + private InMemoryBufferFactory inMemoryBufferFactory; private Buffer buffer; - private PluginMetrics pluginMetrics; - private Counter requestSuccessCounter; - private Counter requestFailCounter; - private Counter successEventCounter; - private Counter failedEventCounter; - private final String TEST_LOG_GROUP = "TESTGROUP"; - private final String TEST_LOG_STREAM = "TESTSTREAM"; + private CloudWatchLogsDispatcher dispatcher; private static final int messageKeyByteSize = 14; - private static final int convertToBytesFromKiloBytes = 1024; + private volatile int testCounter; @BeforeEach void setUp() { cloudWatchLogsSinkConfig = mock(CloudWatchLogsSinkConfig.class); thresholdConfig = new ThresholdConfig(); //Class can stay as is. - thresholdCheck = new ThresholdCheck(thresholdConfig.getBatchSize(), thresholdConfig.getMaxEventSize() * convertToBytesFromKiloBytes, + cloudWatchLogsLimits = new CloudWatchLogsLimits(thresholdConfig.getBatchSize(), thresholdConfig.getMaxEventSizeBytes(), thresholdConfig.getMaxRequestSize(), thresholdConfig.getLogSendInterval()); - awsConfig = mock(AwsConfig.class); - bufferFactory = new InMemoryBufferFactory(); - buffer = bufferFactory.getBuffer(); - awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); - - pluginMetrics = mock(PluginMetrics.class); - requestSuccessCounter = mock(Counter.class); - requestFailCounter = mock(Counter.class); - successEventCounter = mock(Counter.class); - failedEventCounter = mock(Counter.class); - - final String stsRoleArn = UUID.randomUUID().toString(); - final String externalId = UUID.randomUUID().toString(); - final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); - - when(cloudWatchLogsSinkConfig.getLogGroup()).thenReturn(TEST_LOG_GROUP); - when(cloudWatchLogsSinkConfig.getLogStream()).thenReturn(TEST_LOG_STREAM); - when(cloudWatchLogsSinkConfig.getBufferType()).thenReturn("in_memory"); - when(cloudWatchLogsSinkConfig.getAwsConfig()).thenReturn(awsConfig); - when(cloudWatchLogsSinkConfig.getThresholdConfig()).thenReturn(thresholdConfig); - - when(awsConfig.getAwsRegion()).thenReturn(Region.US_EAST_1); - when(awsConfig.getAwsStsRoleArn()).thenReturn(stsRoleArn); - when(awsConfig.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); - when(awsConfig.getAwsStsExternalId()).thenReturn(externalId); - - lenient().when(pluginMetrics.counter(CloudWatchLogsService.CLOUDWATCH_LOGS_EVENTS_SUCCEEDED)).thenReturn(successEventCounter); - lenient().when(pluginMetrics.counter(CloudWatchLogsService.CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED)).thenReturn(requestSuccessCounter); - lenient().when(pluginMetrics.counter(CloudWatchLogsService.CLOUDWATCH_LOGS_EVENTS_FAILED)).thenReturn(failedEventCounter); - lenient().when(pluginMetrics.counter(CloudWatchLogsService.CLOUDWATCH_LOGS_REQUESTS_FAILED)).thenReturn(requestFailCounter); - } - - void setThresholdForTestingRequestSize(int size) { - thresholdCheck = new ThresholdCheck(10000, size, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME); - } - - void setThresholdForTestingMaxRequestRequestSize() { - thresholdCheck = new ThresholdCheck(10000, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST * 2, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST * 2, ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME); - } + inMemoryBufferFactory = new InMemoryBufferFactory(); + buffer = inMemoryBufferFactory.getBuffer(); + dispatcher = mock(CloudWatchLogsDispatcher.class); - CloudWatchLogsService getCwlClientWithMemoryBuffer() { - return new CloudWatchLogsService(mockClient, cloudWatchLogsSinkConfig, buffer, pluginMetrics, - thresholdCheck, thresholdConfig.getRetryCount(), ThresholdConfig.DEFAULT_BACKOFF_TIME); - } - - void setMockClientNoErrors() { - mockClient = mock(CloudWatchLogsClient.class); - putLogEventsResponse = mock(PutLogEventsResponse.class); - when(mockClient.putLogEvents(any(PutLogEventsRequest.class))).thenReturn(putLogEventsResponse); - when(putLogEventsResponse.rejectedLogEventsInfo()).thenReturn(null); - } + cloudWatchLogsService = new CloudWatchLogsService(buffer, cloudWatchLogsLimits, dispatcher); - void setMockClientThrowCWLException() { - mockClient = mock(CloudWatchLogsClient.class); - doThrow(CloudWatchLogsException.class).when(mockClient).putLogEvents(any(PutLogEventsRequest.class)); + testCounter = 0; } - Collection> getSampleRecords(int numberOfRecords) { + Collection> getSampleRecordsLess() { final ArrayList> returnCollection = new ArrayList<>(); - for (int i = 0; i < numberOfRecords; i++) { + for (int i = 0; i < 5; i++) { JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("testMessage"); final EventHandle mockEventHandle = mock(EventHandle.class); mockJacksonEvent.setEventHandle(mockEventHandle); @@ -148,11 +57,10 @@ Collection> getSampleRecords(int numberOfRecords) { return returnCollection; } - Collection> getSampleRecordsLarge(int numberOfRecords, int sizeOfRecordsBytes) { + Collection> getSampleRecords() { final ArrayList> returnCollection = new ArrayList<>(); - final String testMessage = "a"; - for (int i = 0; i < numberOfRecords; i++) { - JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage(testMessage.repeat(sizeOfRecordsBytes)); + for (int i = 0; i < thresholdConfig.getBatchSize(); i++) { + JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("testMessage"); final EventHandle mockEventHandle = mock(EventHandle.class); mockJacksonEvent.setEventHandle(mockEventHandle); returnCollection.add(new Record<>(mockJacksonEvent)); @@ -161,166 +69,35 @@ Collection> getSampleRecordsLarge(int numberOfRecords, int sizeOfR return returnCollection; } - @Test - void client_creation_test() { - CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - } - - @Test - void retry_count_limit_reached_test() { - setMockClientThrowCWLException(); - CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - - try { - cloudWatchLogsService.output(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 2)); - } catch (RetransmissionLimitException e) { //TODO: Create a dedicated RuntimeException for this. - assertThat(e, notNullValue()); - } - } - - @Test - void check_failed_event_transmission_test() { - setMockClientThrowCWLException(); - CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - - try { - cloudWatchLogsService.output(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE)); - } catch (RetransmissionLimitException e) { - verify(failedEventCounter).increment(ThresholdConfig.DEFAULT_BATCH_SIZE); - } - } - - @Test - void check_successful_event_transmission_test() { - setMockClientNoErrors(); - CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - - cloudWatchLogsService.output(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 2)); - - verify(successEventCounter, atLeast(2)).increment(anyDouble()); - } - - @Test - void check_failed_event_test() { - setMockClientThrowCWLException(); - CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - - try { - cloudWatchLogsService.output(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 4)); - } catch (RetransmissionLimitException e) { - verify(requestFailCounter, atLeast(ThresholdConfig.DEFAULT_RETRY_COUNT)).increment(); - } - } - - @Test - void check_successful_event_test() { - setMockClientNoErrors(); - CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - - cloudWatchLogsService.output(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 4)); - - verify(requestSuccessCounter, atLeast(4)).increment(); - } - - @Test - void check_event_handles_successfully_released_test() { - setMockClientNoErrors(); - CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - - final Collection> sampleEvents = getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 2); - final Collection sampleEventHandles = sampleEvents.stream().map(Record::getData).map(Event::getEventHandle).collect(Collectors.toList()); - - cloudWatchLogsService.output(sampleEvents); - - for (EventHandle sampleEventHandle: sampleEventHandles) { - verify(sampleEventHandle).release(true); - } - } - - @Test - void check_event_handles_failed_released_test() { - setMockClientThrowCWLException(); - CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - - final Collection> sampleEvents = getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE); - final Collection sampleEventHandles = sampleEvents.stream().map(Record::getData).map(Event::getEventHandle).collect(Collectors.toList()); - - try { - cloudWatchLogsService.output(sampleEvents); - } catch (RetransmissionLimitException e) { - for (EventHandle sampleEventHandle: sampleEventHandles) { - verify(sampleEventHandle).release(false); - } + Collection> getSampleRecordsLarge() { + final ArrayList> returnCollection = new ArrayList<>(); + for (int i = 0; i < (thresholdConfig.getBatchSize() * 4); i++) { + JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("testMessage"); + final EventHandle mockEventHandle = mock(EventHandle.class); + mockJacksonEvent.setEventHandle(mockEventHandle); + returnCollection.add(new Record<>(mockJacksonEvent)); } - } - /** - * Tests if our json string is equal to the default event size in bytes. - * 14 accounts for the "message": byte size. - */ - @Test - void check_event_size_correct_test() { - ArrayList> sampleEvents = (ArrayList>) getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - messageKeyByteSize); //Accounts for the key string value. - - assertThat(sampleEvents.get(0).getData().toJsonString().length(), equalTo(ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes)); - } - - @Test - void check_max_size_threshold_fail_test() { - setMockClientNoErrors(); - CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - - final Collection> sampleEvents = getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - CloudWatchLogsService.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE - messageKeyByteSize + 1); - - cloudWatchLogsService.output(sampleEvents); - - verify(successEventCounter, never()).increment(anyDouble()); - verify(requestSuccessCounter, never()).increment(); + return returnCollection; } @Test - void check_max_size_threshold_success_test() { - setMockClientNoErrors(); - CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - - final Collection> sampleEvents = getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, (ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - messageKeyByteSize) - CloudWatchLogsService.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); - - cloudWatchLogsService.output(sampleEvents); - - verify(successEventCounter, atLeastOnce()).increment(anyDouble()); - verify(requestSuccessCounter, atLeastOnce()).increment(); + void check_dispatcher_run_was_not_called() { + cloudWatchLogsService.processLogEvents(getSampleRecordsLess()); + verify(dispatcher, never()).run(); } @Test - void check_max_request_size_threshold_fail_test() { - setThresholdForTestingRequestSize(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST); - setMockClientNoErrors(); - CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - - cloudWatchLogsService.output(getSampleRecordsLarge(1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - messageKeyByteSize - CloudWatchLogsService.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE + 1)); - - verify(requestSuccessCounter, never()).increment(); + void check_dispatcher_run_was_called_test() { + cloudWatchLogsService.processLogEvents(getSampleRecords()); + verify(dispatcher, atLeastOnce()).run(); } @Test - void check_max_request_size_threshold_success_test() { - setThresholdForTestingRequestSize(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST); - setMockClientNoErrors(); - CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - - cloudWatchLogsService.output(getSampleRecordsLarge(1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - messageKeyByteSize - CloudWatchLogsService.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); - - verify(requestSuccessCounter, atLeast(1)).increment(); + void check_dispatcher_run_called_heavy_load() { + cloudWatchLogsService.processLogEvents(getSampleRecordsLarge()); + verify(dispatcher, atLeast(4)).run(); } - @Test - void check_max_api_request_size_threshold_success_test() { - setThresholdForTestingMaxRequestRequestSize(); - setMockClientNoErrors(); - CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); - - cloudWatchLogsService.output(getSampleRecordsLarge(1, (ThresholdConfig.DEFAULT_SIZE_OF_REQUEST * 2) - messageKeyByteSize - CloudWatchLogsService.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); - - verify(requestSuccessCounter, atLeast(1)).increment(); - } -} \ No newline at end of file + //TODO: Add multithreaded testing to ensure that the proper methods (run) gets called. +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTestOld.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTestOld.java new file mode 100644 index 0000000000..d0a0fb33e8 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTestOld.java @@ -0,0 +1,326 @@ +///* +// * Copyright OpenSearch Contributors +// * SPDX-License-Identifier: Apache-2.0 +// */ +// +//package org.opensearch.dataprepper.plugins.sink.client; +// +//import io.micrometer.core.instrument.Counter; +//import org.junit.jupiter.api.BeforeEach; +//import org.junit.jupiter.api.Test; +//import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +//import org.opensearch.dataprepper.metrics.PluginMetrics; +//import org.opensearch.dataprepper.model.event.Event; +//import org.opensearch.dataprepper.model.event.EventHandle; +//import org.opensearch.dataprepper.model.event.JacksonEvent; +//import org.opensearch.dataprepper.model.record.Record; +//import org.opensearch.dataprepper.plugins.sink.buffer.Buffer; +//import org.opensearch.dataprepper.plugins.sink.buffer.BufferFactory; +//import org.opensearch.dataprepper.plugins.sink.buffer.InMemoryBufferFactory; +//import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; +//import org.opensearch.dataprepper.plugins.sink.config.CloudWatchLogsSinkConfig; +//import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; +//import org.opensearch.dataprepper.plugins.sink.exception.RetransmissionLimitException; +//import org.opensearch.dataprepper.plugins.sink.push_condition.CloudWatchLogsLimits; +//import software.amazon.awssdk.regions.Region; +//import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; +//import software.amazon.awssdk.services.cloudwatchlogs.model.CloudWatchLogsException; +//import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; +//import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsResponse; +// +//import java.util.ArrayList; +//import java.util.Collection; +//import java.util.Map; +//import java.util.UUID; +//import java.util.stream.Collectors; +// +//import static org.hamcrest.MatcherAssert.assertThat; +//import static org.hamcrest.Matchers.equalTo; +//import static org.hamcrest.Matchers.notNullValue; +//import static org.mockito.ArgumentMatchers.any; +//import static org.mockito.Mockito.mock; +//import static org.mockito.Mockito.when; +//import static org.mockito.Mockito.lenient; +//import static org.mockito.Mockito.doThrow; +//import static org.mockito.Mockito.verify; +//import static org.mockito.Mockito.anyDouble; +//import static org.mockito.Mockito.atLeast; +//import static org.mockito.Mockito.never; +//import static org.mockito.Mockito.atLeastOnce; +// +////TODO: Add Codec session. +////TODO: Finish adding feature for ARN reading. +// +//public class CloudWatchLogsServiceTestOld { +// private CloudWatchLogsClient mockClient; +// private PutLogEventsResponse putLogEventsResponse; +// private CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig; +// private ThresholdConfig thresholdConfig; +// private CloudWatchLogsLimits cloudWatchLogsLimits; +// private AwsConfig awsConfig; +// private AwsCredentialsSupplier awsCredentialsSupplier; +// private BufferFactory bufferFactory; +// private Buffer buffer; +// private PluginMetrics pluginMetrics; +// private Counter requestSuccessCounter; +// private Counter requestFailCounter; +// private Counter successEventCounter; +// private Counter failedEventCounter; +// private final String TEST_LOG_GROUP = "TESTGROUP"; +// private final String TEST_LOG_STREAM = "TESTSTREAM"; +// private static final int messageKeyByteSize = 14; +// private static final int convertToBytesFromKiloBytes = 1024; +// +// @BeforeEach +// void setUp() { +// cloudWatchLogsSinkConfig = mock(CloudWatchLogsSinkConfig.class); +// +// thresholdConfig = new ThresholdConfig(); //Class can stay as is. +// cloudWatchLogsLimits = new CloudWatchLogsLimits(thresholdConfig.getBatchSize(), thresholdConfig.getMaxEventSizeBytes() * convertToBytesFromKiloBytes, +// thresholdConfig.getMaxRequestSize(), thresholdConfig.getLogSendInterval()); +// +// awsConfig = mock(AwsConfig.class); +// bufferFactory = new InMemoryBufferFactory(); +// buffer = bufferFactory.getBuffer(); +// awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); +// +// pluginMetrics = mock(PluginMetrics.class); +// requestSuccessCounter = mock(Counter.class); +// requestFailCounter = mock(Counter.class); +// successEventCounter = mock(Counter.class); +// failedEventCounter = mock(Counter.class); +// +// final String stsRoleArn = UUID.randomUUID().toString(); +// final String externalId = UUID.randomUUID().toString(); +// final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); +// +// when(cloudWatchLogsSinkConfig.getLogGroup()).thenReturn(TEST_LOG_GROUP); +// when(cloudWatchLogsSinkConfig.getLogStream()).thenReturn(TEST_LOG_STREAM); +// when(cloudWatchLogsSinkConfig.getBufferType()).thenReturn("in_memory"); +// when(cloudWatchLogsSinkConfig.getAwsConfig()).thenReturn(awsConfig); +// when(cloudWatchLogsSinkConfig.getThresholdConfig()).thenReturn(thresholdConfig); +// +// when(awsConfig.getAwsRegion()).thenReturn(Region.US_EAST_1); +// when(awsConfig.getAwsStsRoleArn()).thenReturn(stsRoleArn); +// when(awsConfig.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); +// when(awsConfig.getAwsStsExternalId()).thenReturn(externalId); +// +// lenient().when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_EVENTS_SUCCEEDED)).thenReturn(successEventCounter); +// lenient().when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED)).thenReturn(requestSuccessCounter); +// lenient().when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_EVENTS_FAILED)).thenReturn(failedEventCounter); +// lenient().when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_REQUESTS_FAILED)).thenReturn(requestFailCounter); +// } +// +// void setThresholdForTestingRequestSize(int size) { +// cloudWatchLogsLimits = new CloudWatchLogsLimits(10000, size, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME); +// } +// +// void setThresholdForTestingMaxRequestRequestSize() { +// cloudWatchLogsLimits = new CloudWatchLogsLimits(10000, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST * 2, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST * 2, ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME); +// } +// +// CloudWatchLogsService getCwlClientWithMemoryBuffer() { +// return new CloudWatchLogsService(mockClient, cloudWatchLogsSinkConfig, buffer, pluginMetrics, +// cloudWatchLogsLimits, thresholdConfig.getRetryCount(), ThresholdConfig.DEFAULT_BACKOFF_TIME); +// } +// +// void setMockClientNoErrors() { +// mockClient = mock(CloudWatchLogsClient.class); +// putLogEventsResponse = mock(PutLogEventsResponse.class); +// when(mockClient.putLogEvents(any(PutLogEventsRequest.class))).thenReturn(putLogEventsResponse); +// when(putLogEventsResponse.rejectedLogEventsInfo()).thenReturn(null); +// } +// +// void setMockClientThrowCWLException() { +// mockClient = mock(CloudWatchLogsClient.class); +// doThrow(CloudWatchLogsException.class).when(mockClient).putLogEvents(any(PutLogEventsRequest.class)); +// } +// +// Collection> getSampleRecords(int numberOfRecords) { +// final ArrayList> returnCollection = new ArrayList<>(); +// for (int i = 0; i < numberOfRecords; i++) { +// JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("testMessage"); +// final EventHandle mockEventHandle = mock(EventHandle.class); +// mockJacksonEvent.setEventHandle(mockEventHandle); +// returnCollection.add(new Record<>(mockJacksonEvent)); +// } +// +// return returnCollection; +// } +// +// Collection> getSampleRecordsLarge(int numberOfRecords, int sizeOfRecordsBytes) { +// final ArrayList> returnCollection = new ArrayList<>(); +// final String testMessage = "a"; +// for (int i = 0; i < numberOfRecords; i++) { +// JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage(testMessage.repeat(sizeOfRecordsBytes)); +// final EventHandle mockEventHandle = mock(EventHandle.class); +// mockJacksonEvent.setEventHandle(mockEventHandle); +// returnCollection.add(new Record<>(mockJacksonEvent)); +// } +// +// return returnCollection; +// } +// +// @Test +// void client_creation_test() { +// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); +// } +// +// @Test +// void retry_count_limit_reached_test() { +// setMockClientThrowCWLException(); +// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); +// +// try { +// cloudWatchLogsService.processLogEvents(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 2)); +// } catch (RetransmissionLimitException e) { //TODO: Create a dedicated RuntimeException for this. +// assertThat(e, notNullValue()); +// } +// } +// +// @Test +// void check_failed_event_transmission_test() { +// setMockClientThrowCWLException(); +// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); +// +// try { +// cloudWatchLogsService.processLogEvents(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE)); +// } catch (RetransmissionLimitException e) { +// verify(failedEventCounter).increment(ThresholdConfig.DEFAULT_BATCH_SIZE); +// } +// } +// +// @Test +// void check_successful_event_transmission_test() { +// setMockClientNoErrors(); +// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); +// +// cloudWatchLogsService.processLogEvents(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 2)); +// +// verify(successEventCounter, atLeast(2)).increment(anyDouble()); +// } +// +// @Test +// void check_failed_event_test() { +// setMockClientThrowCWLException(); +// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); +// +// try { +// cloudWatchLogsService.processLogEvents(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 4)); +// } catch (RetransmissionLimitException e) { +// verify(requestFailCounter, atLeast(ThresholdConfig.DEFAULT_RETRY_COUNT)).increment(); +// } +// } +// +// @Test +// void check_successful_event_test() { +// setMockClientNoErrors(); +// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); +// +// cloudWatchLogsService.processLogEvents(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 4)); +// +// verify(requestSuccessCounter, atLeast(4)).increment(); +// } +// +// @Test +// void check_event_handles_successfully_released_test() { +// setMockClientNoErrors(); +// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); +// +// final Collection> sampleEvents = getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 2); +// final Collection sampleEventHandles = sampleEvents.stream().map(Record::getData).map(Event::getEventHandle).collect(Collectors.toList()); +// +// cloudWatchLogsService.processLogEvents(sampleEvents); +// +// for (EventHandle sampleEventHandle: sampleEventHandles) { +// verify(sampleEventHandle).release(true); +// } +// } +// +// @Test +// void check_event_handles_failed_released_test() { +// setMockClientThrowCWLException(); +// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); +// +// final Collection> sampleEvents = getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE); +// final Collection sampleEventHandles = sampleEvents.stream().map(Record::getData).map(Event::getEventHandle).collect(Collectors.toList()); +// +// try { +// cloudWatchLogsService.processLogEvents(sampleEvents); +// } catch (RetransmissionLimitException e) { +// for (EventHandle sampleEventHandle: sampleEventHandles) { +// verify(sampleEventHandle).release(false); +// } +// } +// } +// +// /** +// * Tests if our json string is equal to the default event size in bytes. +// * 14 accounts for the "message": byte size. +// */ +// @Test +// void check_event_size_correct_test() { +// ArrayList> sampleEvents = (ArrayList>) getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - messageKeyByteSize); //Accounts for the key string value. +// +// assertThat(sampleEvents.get(0).getData().toJsonString().length(), equalTo(ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes)); +// } +// +// @Test +// void check_max_size_threshold_fail_test() { +// setMockClientNoErrors(); +// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); +// +// final Collection> sampleEvents = getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE - messageKeyByteSize + 1); +// +// cloudWatchLogsService.processLogEvents(sampleEvents); +// +// verify(successEventCounter, never()).increment(anyDouble()); +// verify(requestSuccessCounter, never()).increment(); +// } +// +// @Test +// void check_max_size_threshold_success_test() { +// setMockClientNoErrors(); +// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); +// +// final Collection> sampleEvents = getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, (ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - messageKeyByteSize) - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); +// +// cloudWatchLogsService.processLogEvents(sampleEvents); +// +// verify(successEventCounter, atLeastOnce()).increment(anyDouble()); +// verify(requestSuccessCounter, atLeastOnce()).increment(); +// } +// +// @Test +// void check_max_request_size_threshold_fail_test() { +// setThresholdForTestingRequestSize(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST); +// setMockClientNoErrors(); +// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); +// +// cloudWatchLogsService.processLogEvents(getSampleRecordsLarge(1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - messageKeyByteSize - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE + 1)); +// +// verify(requestSuccessCounter, never()).increment(); +// } +// +// @Test +// void check_max_request_size_threshold_success_test() { +// setThresholdForTestingRequestSize(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST); +// setMockClientNoErrors(); +// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); +// +// cloudWatchLogsService.processLogEvents(getSampleRecordsLarge(1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - messageKeyByteSize - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); +// +// verify(requestSuccessCounter, atLeast(1)).increment(); +// } +// +// @Test +// void check_max_api_request_size_threshold_success_test() { +// setThresholdForTestingMaxRequestRequestSize(); +// setMockClientNoErrors(); +// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); +// +// cloudWatchLogsService.processLogEvents(getSampleRecordsLarge(1, (ThresholdConfig.DEFAULT_SIZE_OF_REQUEST * 2) - messageKeyByteSize - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); +// +// verify(requestSuccessCounter, atLeast(1)).increment(); +// } +//} \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfigTest.java index caee95ecc4..cd43b3ee0f 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfigTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfigTest.java @@ -31,7 +31,7 @@ void check_default_values() { assertThat(thresholdConfig.getBackOffTime(), equalTo(ThresholdConfig.DEFAULT_BACKOFF_TIME)); assertThat(thresholdConfig.getRetryCount(), equalTo(ThresholdConfig.DEFAULT_RETRY_COUNT)); assertThat(thresholdConfig.getBatchSize(), equalTo(ThresholdConfig.DEFAULT_BATCH_SIZE)); - assertThat(thresholdConfig.getMaxEventSize(), equalTo(ThresholdConfig.DEFAULT_EVENT_SIZE)); + assertThat(thresholdConfig.getMaxEventSizeBytes(), equalTo(ThresholdConfig.DEFAULT_EVENT_SIZE)); assertThat(thresholdConfig.getMaxRequestSize(), equalTo(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST)); assertThat(thresholdConfig.getLogSendInterval(), equalTo(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME)); } @@ -49,7 +49,7 @@ void check_valid_batch_size(final int batchSize) { void check_valid_max_event_size(final int max_event_size) { final Map jsonMap = Map.of("max_event_size", max_event_size); final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); - assertThat(thresholdConfigTest.getMaxEventSize(), equalTo(max_event_size)); + assertThat(thresholdConfigTest.getMaxEventSizeBytes(), equalTo(max_event_size)); } @ParameterizedTest diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheckTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/something/CloudWatchLogsLimitsTest.java similarity index 55% rename from data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheckTest.java rename to data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/something/CloudWatchLogsLimitsTest.java index 7afd69ade7..2e283b1006 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/threshold/ThresholdCheckTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/something/CloudWatchLogsLimitsTest.java @@ -3,103 +3,104 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.threshold; +package org.opensearch.dataprepper.plugins.sink.something; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; +import org.opensearch.dataprepper.plugins.sink.push_condition.CloudWatchLogsLimits; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; -public class ThresholdCheckTest { - private ThresholdCheck thresholdCheck; +public class CloudWatchLogsLimitsTest { + private CloudWatchLogsLimits cloudWatchLogsLimits; @BeforeEach void setUp() { - thresholdCheck = new ThresholdCheck(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE, + cloudWatchLogsLimits = new CloudWatchLogsLimits(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME); } @ParameterizedTest @ValueSource(ints = {55, 80, 100}) void check_max_event_size_invalid(final int event_size) { - boolean isEventGreater = thresholdCheck.isGreaterThanMaxEventSize(event_size); + boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize(event_size); assertTrue(isEventGreater); } @ParameterizedTest @ValueSource(ints = {10, 30, 50}) void check_max_event_size_valid(final int event_size) { - boolean isEventGreater = thresholdCheck.isGreaterThanMaxEventSize(event_size); + boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize(event_size); assertFalse(isEventGreater); } @ParameterizedTest @ValueSource(ints = {60, 80, 100}) void check_greater_than_threshold_conditions_time_true(final int send_interval) { - boolean thresholdMetTime = thresholdCheck.isGreaterThanThresholdReached(send_interval, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST,ThresholdConfig.DEFAULT_BATCH_SIZE); + boolean thresholdMetTime = cloudWatchLogsLimits.isGreaterThanLimitReached(send_interval, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST,ThresholdConfig.DEFAULT_BATCH_SIZE); assertTrue(thresholdMetTime); } @ParameterizedTest @ValueSource(ints = {1, 3, 59}) void check_greater_than_threshold_conditions_time_false(final int send_interval) { - boolean thresholdMetTime = thresholdCheck.isGreaterThanThresholdReached(send_interval, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST,ThresholdConfig.DEFAULT_BATCH_SIZE); + boolean thresholdMetTime = cloudWatchLogsLimits.isGreaterThanLimitReached(send_interval, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST,ThresholdConfig.DEFAULT_BATCH_SIZE); assertFalse(thresholdMetTime); } @ParameterizedTest @ValueSource(ints = {550000, 750000, 1000000}) void check_greater_than_threshold_conditions_request_size_true(final int request_size) { - boolean thresholdMetRequestSize = thresholdCheck.isGreaterThanThresholdReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, request_size, ThresholdConfig.DEFAULT_BATCH_SIZE); + boolean thresholdMetRequestSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, request_size, ThresholdConfig.DEFAULT_BATCH_SIZE); assertTrue(thresholdMetRequestSize); } @ParameterizedTest @ValueSource(ints = {10000, 250000, 500000}) void check_greater_than_threshold_conditions_request_size_false(final int request_size) { - boolean thresholdMetRequestSize = thresholdCheck.isGreaterThanThresholdReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, request_size, ThresholdConfig.DEFAULT_BATCH_SIZE); + boolean thresholdMetRequestSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, request_size, ThresholdConfig.DEFAULT_BATCH_SIZE); assertFalse(thresholdMetRequestSize); } @ParameterizedTest @ValueSource(ints = {26, 50, 100}) void check_greater_than_threshold_conditions_batch_size_true(final int batch_size) { - boolean thresholdMetBatchSize = thresholdCheck.isGreaterThanThresholdReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME -1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, batch_size); + boolean thresholdMetBatchSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME -1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, batch_size); assertTrue(thresholdMetBatchSize); } @ParameterizedTest @ValueSource(ints = {1, 10, 25}) void check_greater_than_threshold_conditions_batch_size_false(final int batch_size) { - boolean thresholdMetBatchSize = thresholdCheck.isGreaterThanThresholdReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME -1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, batch_size); + boolean thresholdMetBatchSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME -1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, batch_size); assertFalse(thresholdMetBatchSize); } @Test void check_equal_than_threshold_conditions_request_size_true() { - boolean thresholdMetRequestSize = thresholdCheck.isEqualToThresholdReached(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); + boolean thresholdMetRequestSize = cloudWatchLogsLimits.isEqualToLimitReached(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); assertTrue(thresholdMetRequestSize); } @Test void check_equal_than_threshold_conditions_request_size_false() { - boolean thresholdMetRequestSize = thresholdCheck.isEqualToThresholdReached(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); + boolean thresholdMetRequestSize = cloudWatchLogsLimits.isEqualToLimitReached(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); assertFalse(thresholdMetRequestSize); } @Test void check_equal_than_threshold_conditions_batch_size_true() { - boolean thresholdMetBatchSize = thresholdCheck.isEqualToThresholdReached(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1, ThresholdConfig.DEFAULT_BATCH_SIZE); + boolean thresholdMetBatchSize = cloudWatchLogsLimits.isEqualToLimitReached(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1, ThresholdConfig.DEFAULT_BATCH_SIZE); assertTrue(thresholdMetBatchSize); } @Test void check_equal_than_threshold_conditions_batch_size_false() { - boolean thresholdMetBatchSize = thresholdCheck.isEqualToThresholdReached(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); + boolean thresholdMetBatchSize = cloudWatchLogsLimits.isEqualToLimitReached(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); assertFalse(thresholdMetBatchSize); } } \ No newline at end of file From abec5e3be2b1e60a67247d559a3f69a01657bd6b Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Thu, 20 Jul 2023 11:58:35 -0700 Subject: [PATCH 29/43] Added fixesto test and defaulted the parameters in the config to CloudWatchLogs limits, customer can change this in config file Signed-off-by: Marcos Gonzalez Mayedo --- .../sink/client/CloudWatchLogsDispatcher.java | 6 +- .../sink/client/CloudWatchLogsService.java | 5 +- .../plugins/sink/config/ThresholdConfig.java | 4 +- .../sink/packaging/ThreadTaskEvents.java | 5 + .../plugins/sink/time/SinkStopWatch.java | 5 + .../client/CloudWatchLogsDispatcherTest.java | 5 + .../client/CloudWatchLogsServiceTest.java | 179 +++++++++- .../client/CloudWatchLogsServiceTestOld.java | 326 ------------------ .../sink/config/ThresholdConfigTest.java | 4 +- .../CloudWatchLogsLimitsTest.java | 22 +- 10 files changed, 214 insertions(+), 347 deletions(-) delete mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTestOld.java rename data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/{something => push_condition}/CloudWatchLogsLimitsTest.java (78%) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java index d035aba0e1..f661949086 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java @@ -1,3 +1,8 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + package org.opensearch.dataprepper.plugins.sink.client; import org.opensearch.dataprepper.model.event.EventHandle; @@ -56,7 +61,6 @@ private List prepareInputLogEvents(final ThreadTaskEvents eventDa * Flush function to handle the flushing of logs to CloudWatchLogs services; * @param inputLogEvents Collection of inputLogEvents to be flushed * @return true if successful, false otherwise - * @throws InterruptedException */ public boolean dispatchLogs(List inputLogEvents, Collection eventHandles) { boolean failedPost = true; diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index b148a6fcf4..3cf12f4cbb 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -27,7 +27,6 @@ public class CloudWatchLogsService { private static final Logger LOG = LoggerFactory.getLogger(CloudWatchLogsService.class); - private static final int MAX_BLOCKING_QUEUE_SIZE = 10; private final Buffer buffer; private final CloudWatchLogsLimits cloudWatchLogsLimits; private List bufferedEventHandles; @@ -39,13 +38,13 @@ public class CloudWatchLogsService { public CloudWatchLogsService(final Buffer buffer, final CloudWatchLogsLimits cloudWatchLogsLimits, - final CloudWatchLogsDispatcher dispatcher) { + final CloudWatchLogsDispatcher dispatcher, BlockingQueue blockingQueue) { this.buffer = buffer; this.dispatcher = dispatcher; this.bufferedEventHandles = new ArrayList<>(); this.cloudWatchLogsLimits = cloudWatchLogsLimits; - this.taskQueue = new ArrayBlockingQueue<>(MAX_BLOCKING_QUEUE_SIZE); + this.taskQueue = blockingQueue; bufferLock = new ReentrantLock(); sinkStopWatch = new SinkStopWatch(); diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java index 55c6f6e0f5..69839f6937 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java @@ -15,8 +15,8 @@ */ public class ThresholdConfig { public static final int DEFAULT_BATCH_SIZE = 25; - public static final int DEFAULT_EVENT_SIZE = 50; - public static final int DEFAULT_SIZE_OF_REQUEST = 524288; + public static final int DEFAULT_EVENT_SIZE = 256; + public static final int DEFAULT_SIZE_OF_REQUEST = 1048576; public static final int DEFAULT_RETRY_COUNT = 5; public static final int DEFAULT_LOG_SEND_INTERVAL_TIME = 60; public static final int DEFAULT_BACKOFF_TIME = 500; diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/packaging/ThreadTaskEvents.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/packaging/ThreadTaskEvents.java index 89fa32c207..fb9653af92 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/packaging/ThreadTaskEvents.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/packaging/ThreadTaskEvents.java @@ -1,3 +1,8 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + package org.opensearch.dataprepper.plugins.sink.packaging; import org.opensearch.dataprepper.model.event.EventHandle; diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/time/SinkStopWatch.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/time/SinkStopWatch.java index d973f5d3a0..52352d6cd1 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/time/SinkStopWatch.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/time/SinkStopWatch.java @@ -1,3 +1,8 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + package org.opensearch.dataprepper.plugins.sink.time; import org.apache.commons.lang3.time.StopWatch; diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java index b2d8856e6e..a16d21ef18 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java @@ -1,3 +1,8 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + package org.opensearch.dataprepper.plugins.sink.client; import io.micrometer.core.instrument.Counter; diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java index bc393364fc..da548a55a1 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java @@ -1,3 +1,8 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + package org.opensearch.dataprepper.plugins.sink.client; import org.junit.jupiter.api.BeforeEach; @@ -10,14 +15,24 @@ import org.opensearch.dataprepper.plugins.sink.buffer.InMemoryBufferFactory; import org.opensearch.dataprepper.plugins.sink.config.CloudWatchLogsSinkConfig; import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; +import org.opensearch.dataprepper.plugins.sink.packaging.ThreadTaskEvents; import org.opensearch.dataprepper.plugins.sink.push_condition.CloudWatchLogsLimits; import java.util.ArrayList; import java.util.Collection; +import java.util.concurrent.BlockingQueue; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.atLeast; public class CloudWatchLogsServiceTest { + private static final int NUMBER_THREADS_SMALL = 5; + private static final int NUMBER_THREADS_BIG = 10; + private static final int NUMBER_THREADS_LARGE = 20; + private BlockingQueue mockQueue; private CloudWatchLogsService cloudWatchLogsService; private CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig; private ThresholdConfig thresholdConfig; @@ -25,7 +40,6 @@ public class CloudWatchLogsServiceTest { private InMemoryBufferFactory inMemoryBufferFactory; private Buffer buffer; private CloudWatchLogsDispatcher dispatcher; - private static final int messageKeyByteSize = 14; private volatile int testCounter; @BeforeEach @@ -39,8 +53,9 @@ void setUp() { inMemoryBufferFactory = new InMemoryBufferFactory(); buffer = inMemoryBufferFactory.getBuffer(); dispatcher = mock(CloudWatchLogsDispatcher.class); + mockQueue = mock(BlockingQueue.class); - cloudWatchLogsService = new CloudWatchLogsService(buffer, cloudWatchLogsLimits, dispatcher); + cloudWatchLogsService = new CloudWatchLogsService(buffer, cloudWatchLogsLimits, dispatcher, mockQueue); testCounter = 0; } @@ -100,4 +115,162 @@ void check_dispatcher_run_called_heavy_load() { } //TODO: Add multithreaded testing to ensure that the proper methods (run) gets called. + + @Test + void test_less_threads_normal_load() { + Collection threadsToRun = new ArrayList<>(); + + for (int i = 0; i < NUMBER_THREADS_SMALL; i++) { + Thread testingThread = new Thread(new CloudWatchLogsServiceTester(getSampleRecords(), cloudWatchLogsService)); + threadsToRun.add(testingThread); + } + + for (Thread serviceTester: threadsToRun) { + serviceTester.start(); + } + + for (Thread serviceTester: threadsToRun) { + try { + serviceTester.join(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + verify(dispatcher, atLeast(NUMBER_THREADS_SMALL)).run(); + } + + @Test + void test_less_threads_heavy_load() { + Collection threadsToRun = new ArrayList<>(); + + for (int i = 0; i < NUMBER_THREADS_SMALL; i++) { + Thread testingThread = new Thread(new CloudWatchLogsServiceTester(getSampleRecordsLarge(), cloudWatchLogsService)); + threadsToRun.add(testingThread); + } + + for (Thread serviceTester: threadsToRun) { + serviceTester.start(); + } + + for (Thread serviceTester: threadsToRun) { + try { + serviceTester.join(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + verify(dispatcher, atLeast(NUMBER_THREADS_SMALL * 4)).run(); + } + + @Test + void test_more_threads_normal_load() { + Collection threadsToRun = new ArrayList<>(); + + for (int i = 0; i < NUMBER_THREADS_BIG; i++) { + Thread testingThread = new Thread(new CloudWatchLogsServiceTester(getSampleRecords(), cloudWatchLogsService)); + threadsToRun.add(testingThread); + } + + for (Thread serviceTester: threadsToRun) { + serviceTester.start(); + } + + for (Thread serviceTester: threadsToRun) { + try { + serviceTester.join(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + verify(dispatcher, atLeast(NUMBER_THREADS_BIG)).run(); + } + + @Test + void test_more_threads_heavy_load() { + Collection threadsToRun = new ArrayList<>(); + + for (int i = 0; i < NUMBER_THREADS_BIG; i++) { + Thread testingThread = new Thread(new CloudWatchLogsServiceTester(getSampleRecordsLarge(), cloudWatchLogsService)); + threadsToRun.add(testingThread); + } + + for (Thread serviceTester: threadsToRun) { + serviceTester.start(); + } + + for (Thread serviceTester: threadsToRun) { + try { + serviceTester.join(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + verify(dispatcher, atLeast(NUMBER_THREADS_BIG * 4)).run(); + } + + @Test + void test_large_threads_normal_load() { + Collection threadsToRun = new ArrayList<>(); + + for (int i = 0; i < NUMBER_THREADS_LARGE; i++) { + Thread testingThread = new Thread(new CloudWatchLogsServiceTester(getSampleRecords(), cloudWatchLogsService)); + threadsToRun.add(testingThread); + } + + for (Thread serviceTester: threadsToRun) { + serviceTester.start(); + } + + for (Thread serviceTester: threadsToRun) { + try { + serviceTester.join(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + verify(dispatcher, atLeast(NUMBER_THREADS_LARGE)).run(); + } + + @Test + void test_large_threads_heavy_load() { + Collection threadsToRun = new ArrayList<>(); + + for (int i = 0; i < NUMBER_THREADS_LARGE; i++) { + Thread testingThread = new Thread(new CloudWatchLogsServiceTester(getSampleRecordsLarge(), cloudWatchLogsService)); + threadsToRun.add(testingThread); + } + + for (Thread serviceTester: threadsToRun) { + serviceTester.start(); + } + + for (Thread serviceTester: threadsToRun) { + try { + serviceTester.join(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + verify(dispatcher, atLeast(NUMBER_THREADS_LARGE * 4)).run(); + } + + static class CloudWatchLogsServiceTester implements Runnable { + Collection> testEvents; + CloudWatchLogsService testCloudWatchLogsService; + CloudWatchLogsServiceTester(Collection> events, CloudWatchLogsService cloudWatchLogsService) { + testEvents = events; + testCloudWatchLogsService = cloudWatchLogsService; + } + + @Override + public void run() { + testCloudWatchLogsService.processLogEvents(testEvents); + } + } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTestOld.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTestOld.java deleted file mode 100644 index d0a0fb33e8..0000000000 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTestOld.java +++ /dev/null @@ -1,326 +0,0 @@ -///* -// * Copyright OpenSearch Contributors -// * SPDX-License-Identifier: Apache-2.0 -// */ -// -//package org.opensearch.dataprepper.plugins.sink.client; -// -//import io.micrometer.core.instrument.Counter; -//import org.junit.jupiter.api.BeforeEach; -//import org.junit.jupiter.api.Test; -//import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; -//import org.opensearch.dataprepper.metrics.PluginMetrics; -//import org.opensearch.dataprepper.model.event.Event; -//import org.opensearch.dataprepper.model.event.EventHandle; -//import org.opensearch.dataprepper.model.event.JacksonEvent; -//import org.opensearch.dataprepper.model.record.Record; -//import org.opensearch.dataprepper.plugins.sink.buffer.Buffer; -//import org.opensearch.dataprepper.plugins.sink.buffer.BufferFactory; -//import org.opensearch.dataprepper.plugins.sink.buffer.InMemoryBufferFactory; -//import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; -//import org.opensearch.dataprepper.plugins.sink.config.CloudWatchLogsSinkConfig; -//import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; -//import org.opensearch.dataprepper.plugins.sink.exception.RetransmissionLimitException; -//import org.opensearch.dataprepper.plugins.sink.push_condition.CloudWatchLogsLimits; -//import software.amazon.awssdk.regions.Region; -//import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; -//import software.amazon.awssdk.services.cloudwatchlogs.model.CloudWatchLogsException; -//import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; -//import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsResponse; -// -//import java.util.ArrayList; -//import java.util.Collection; -//import java.util.Map; -//import java.util.UUID; -//import java.util.stream.Collectors; -// -//import static org.hamcrest.MatcherAssert.assertThat; -//import static org.hamcrest.Matchers.equalTo; -//import static org.hamcrest.Matchers.notNullValue; -//import static org.mockito.ArgumentMatchers.any; -//import static org.mockito.Mockito.mock; -//import static org.mockito.Mockito.when; -//import static org.mockito.Mockito.lenient; -//import static org.mockito.Mockito.doThrow; -//import static org.mockito.Mockito.verify; -//import static org.mockito.Mockito.anyDouble; -//import static org.mockito.Mockito.atLeast; -//import static org.mockito.Mockito.never; -//import static org.mockito.Mockito.atLeastOnce; -// -////TODO: Add Codec session. -////TODO: Finish adding feature for ARN reading. -// -//public class CloudWatchLogsServiceTestOld { -// private CloudWatchLogsClient mockClient; -// private PutLogEventsResponse putLogEventsResponse; -// private CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig; -// private ThresholdConfig thresholdConfig; -// private CloudWatchLogsLimits cloudWatchLogsLimits; -// private AwsConfig awsConfig; -// private AwsCredentialsSupplier awsCredentialsSupplier; -// private BufferFactory bufferFactory; -// private Buffer buffer; -// private PluginMetrics pluginMetrics; -// private Counter requestSuccessCounter; -// private Counter requestFailCounter; -// private Counter successEventCounter; -// private Counter failedEventCounter; -// private final String TEST_LOG_GROUP = "TESTGROUP"; -// private final String TEST_LOG_STREAM = "TESTSTREAM"; -// private static final int messageKeyByteSize = 14; -// private static final int convertToBytesFromKiloBytes = 1024; -// -// @BeforeEach -// void setUp() { -// cloudWatchLogsSinkConfig = mock(CloudWatchLogsSinkConfig.class); -// -// thresholdConfig = new ThresholdConfig(); //Class can stay as is. -// cloudWatchLogsLimits = new CloudWatchLogsLimits(thresholdConfig.getBatchSize(), thresholdConfig.getMaxEventSizeBytes() * convertToBytesFromKiloBytes, -// thresholdConfig.getMaxRequestSize(), thresholdConfig.getLogSendInterval()); -// -// awsConfig = mock(AwsConfig.class); -// bufferFactory = new InMemoryBufferFactory(); -// buffer = bufferFactory.getBuffer(); -// awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); -// -// pluginMetrics = mock(PluginMetrics.class); -// requestSuccessCounter = mock(Counter.class); -// requestFailCounter = mock(Counter.class); -// successEventCounter = mock(Counter.class); -// failedEventCounter = mock(Counter.class); -// -// final String stsRoleArn = UUID.randomUUID().toString(); -// final String externalId = UUID.randomUUID().toString(); -// final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); -// -// when(cloudWatchLogsSinkConfig.getLogGroup()).thenReturn(TEST_LOG_GROUP); -// when(cloudWatchLogsSinkConfig.getLogStream()).thenReturn(TEST_LOG_STREAM); -// when(cloudWatchLogsSinkConfig.getBufferType()).thenReturn("in_memory"); -// when(cloudWatchLogsSinkConfig.getAwsConfig()).thenReturn(awsConfig); -// when(cloudWatchLogsSinkConfig.getThresholdConfig()).thenReturn(thresholdConfig); -// -// when(awsConfig.getAwsRegion()).thenReturn(Region.US_EAST_1); -// when(awsConfig.getAwsStsRoleArn()).thenReturn(stsRoleArn); -// when(awsConfig.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); -// when(awsConfig.getAwsStsExternalId()).thenReturn(externalId); -// -// lenient().when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_EVENTS_SUCCEEDED)).thenReturn(successEventCounter); -// lenient().when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED)).thenReturn(requestSuccessCounter); -// lenient().when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_EVENTS_FAILED)).thenReturn(failedEventCounter); -// lenient().when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_REQUESTS_FAILED)).thenReturn(requestFailCounter); -// } -// -// void setThresholdForTestingRequestSize(int size) { -// cloudWatchLogsLimits = new CloudWatchLogsLimits(10000, size, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME); -// } -// -// void setThresholdForTestingMaxRequestRequestSize() { -// cloudWatchLogsLimits = new CloudWatchLogsLimits(10000, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST * 2, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST * 2, ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME); -// } -// -// CloudWatchLogsService getCwlClientWithMemoryBuffer() { -// return new CloudWatchLogsService(mockClient, cloudWatchLogsSinkConfig, buffer, pluginMetrics, -// cloudWatchLogsLimits, thresholdConfig.getRetryCount(), ThresholdConfig.DEFAULT_BACKOFF_TIME); -// } -// -// void setMockClientNoErrors() { -// mockClient = mock(CloudWatchLogsClient.class); -// putLogEventsResponse = mock(PutLogEventsResponse.class); -// when(mockClient.putLogEvents(any(PutLogEventsRequest.class))).thenReturn(putLogEventsResponse); -// when(putLogEventsResponse.rejectedLogEventsInfo()).thenReturn(null); -// } -// -// void setMockClientThrowCWLException() { -// mockClient = mock(CloudWatchLogsClient.class); -// doThrow(CloudWatchLogsException.class).when(mockClient).putLogEvents(any(PutLogEventsRequest.class)); -// } -// -// Collection> getSampleRecords(int numberOfRecords) { -// final ArrayList> returnCollection = new ArrayList<>(); -// for (int i = 0; i < numberOfRecords; i++) { -// JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("testMessage"); -// final EventHandle mockEventHandle = mock(EventHandle.class); -// mockJacksonEvent.setEventHandle(mockEventHandle); -// returnCollection.add(new Record<>(mockJacksonEvent)); -// } -// -// return returnCollection; -// } -// -// Collection> getSampleRecordsLarge(int numberOfRecords, int sizeOfRecordsBytes) { -// final ArrayList> returnCollection = new ArrayList<>(); -// final String testMessage = "a"; -// for (int i = 0; i < numberOfRecords; i++) { -// JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage(testMessage.repeat(sizeOfRecordsBytes)); -// final EventHandle mockEventHandle = mock(EventHandle.class); -// mockJacksonEvent.setEventHandle(mockEventHandle); -// returnCollection.add(new Record<>(mockJacksonEvent)); -// } -// -// return returnCollection; -// } -// -// @Test -// void client_creation_test() { -// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); -// } -// -// @Test -// void retry_count_limit_reached_test() { -// setMockClientThrowCWLException(); -// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); -// -// try { -// cloudWatchLogsService.processLogEvents(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 2)); -// } catch (RetransmissionLimitException e) { //TODO: Create a dedicated RuntimeException for this. -// assertThat(e, notNullValue()); -// } -// } -// -// @Test -// void check_failed_event_transmission_test() { -// setMockClientThrowCWLException(); -// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); -// -// try { -// cloudWatchLogsService.processLogEvents(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE)); -// } catch (RetransmissionLimitException e) { -// verify(failedEventCounter).increment(ThresholdConfig.DEFAULT_BATCH_SIZE); -// } -// } -// -// @Test -// void check_successful_event_transmission_test() { -// setMockClientNoErrors(); -// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); -// -// cloudWatchLogsService.processLogEvents(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 2)); -// -// verify(successEventCounter, atLeast(2)).increment(anyDouble()); -// } -// -// @Test -// void check_failed_event_test() { -// setMockClientThrowCWLException(); -// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); -// -// try { -// cloudWatchLogsService.processLogEvents(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 4)); -// } catch (RetransmissionLimitException e) { -// verify(requestFailCounter, atLeast(ThresholdConfig.DEFAULT_RETRY_COUNT)).increment(); -// } -// } -// -// @Test -// void check_successful_event_test() { -// setMockClientNoErrors(); -// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); -// -// cloudWatchLogsService.processLogEvents(getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 4)); -// -// verify(requestSuccessCounter, atLeast(4)).increment(); -// } -// -// @Test -// void check_event_handles_successfully_released_test() { -// setMockClientNoErrors(); -// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); -// -// final Collection> sampleEvents = getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE * 2); -// final Collection sampleEventHandles = sampleEvents.stream().map(Record::getData).map(Event::getEventHandle).collect(Collectors.toList()); -// -// cloudWatchLogsService.processLogEvents(sampleEvents); -// -// for (EventHandle sampleEventHandle: sampleEventHandles) { -// verify(sampleEventHandle).release(true); -// } -// } -// -// @Test -// void check_event_handles_failed_released_test() { -// setMockClientThrowCWLException(); -// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); -// -// final Collection> sampleEvents = getSampleRecords(ThresholdConfig.DEFAULT_BATCH_SIZE); -// final Collection sampleEventHandles = sampleEvents.stream().map(Record::getData).map(Event::getEventHandle).collect(Collectors.toList()); -// -// try { -// cloudWatchLogsService.processLogEvents(sampleEvents); -// } catch (RetransmissionLimitException e) { -// for (EventHandle sampleEventHandle: sampleEventHandles) { -// verify(sampleEventHandle).release(false); -// } -// } -// } -// -// /** -// * Tests if our json string is equal to the default event size in bytes. -// * 14 accounts for the "message": byte size. -// */ -// @Test -// void check_event_size_correct_test() { -// ArrayList> sampleEvents = (ArrayList>) getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - messageKeyByteSize); //Accounts for the key string value. -// -// assertThat(sampleEvents.get(0).getData().toJsonString().length(), equalTo(ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes)); -// } -// -// @Test -// void check_max_size_threshold_fail_test() { -// setMockClientNoErrors(); -// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); -// -// final Collection> sampleEvents = getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE - messageKeyByteSize + 1); -// -// cloudWatchLogsService.processLogEvents(sampleEvents); -// -// verify(successEventCounter, never()).increment(anyDouble()); -// verify(requestSuccessCounter, never()).increment(); -// } -// -// @Test -// void check_max_size_threshold_success_test() { -// setMockClientNoErrors(); -// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); -// -// final Collection> sampleEvents = getSampleRecordsLarge(ThresholdConfig.DEFAULT_BATCH_SIZE, (ThresholdConfig.DEFAULT_EVENT_SIZE * convertToBytesFromKiloBytes - messageKeyByteSize) - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); -// -// cloudWatchLogsService.processLogEvents(sampleEvents); -// -// verify(successEventCounter, atLeastOnce()).increment(anyDouble()); -// verify(requestSuccessCounter, atLeastOnce()).increment(); -// } -// -// @Test -// void check_max_request_size_threshold_fail_test() { -// setThresholdForTestingRequestSize(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST); -// setMockClientNoErrors(); -// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); -// -// cloudWatchLogsService.processLogEvents(getSampleRecordsLarge(1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - messageKeyByteSize - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE + 1)); -// -// verify(requestSuccessCounter, never()).increment(); -// } -// -// @Test -// void check_max_request_size_threshold_success_test() { -// setThresholdForTestingRequestSize(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST); -// setMockClientNoErrors(); -// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); -// -// cloudWatchLogsService.processLogEvents(getSampleRecordsLarge(1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - messageKeyByteSize - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); -// -// verify(requestSuccessCounter, atLeast(1)).increment(); -// } -// -// @Test -// void check_max_api_request_size_threshold_success_test() { -// setThresholdForTestingMaxRequestRequestSize(); -// setMockClientNoErrors(); -// CloudWatchLogsService cloudWatchLogsService = getCwlClientWithMemoryBuffer(); -// -// cloudWatchLogsService.processLogEvents(getSampleRecordsLarge(1, (ThresholdConfig.DEFAULT_SIZE_OF_REQUEST * 2) - messageKeyByteSize - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); -// -// verify(requestSuccessCounter, atLeast(1)).increment(); -// } -//} \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfigTest.java index cd43b3ee0f..e8324755af 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfigTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfigTest.java @@ -31,7 +31,7 @@ void check_default_values() { assertThat(thresholdConfig.getBackOffTime(), equalTo(ThresholdConfig.DEFAULT_BACKOFF_TIME)); assertThat(thresholdConfig.getRetryCount(), equalTo(ThresholdConfig.DEFAULT_RETRY_COUNT)); assertThat(thresholdConfig.getBatchSize(), equalTo(ThresholdConfig.DEFAULT_BATCH_SIZE)); - assertThat(thresholdConfig.getMaxEventSizeBytes(), equalTo(ThresholdConfig.DEFAULT_EVENT_SIZE)); + assertThat(thresholdConfig.getMaxEventSizeBytes(), equalTo(ThresholdConfig.DEFAULT_EVENT_SIZE * ThresholdConfig.CONVERT_TO_BYTES_FROM_KB)); assertThat(thresholdConfig.getMaxRequestSize(), equalTo(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST)); assertThat(thresholdConfig.getLogSendInterval(), equalTo(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME)); } @@ -49,7 +49,7 @@ void check_valid_batch_size(final int batchSize) { void check_valid_max_event_size(final int max_event_size) { final Map jsonMap = Map.of("max_event_size", max_event_size); final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); - assertThat(thresholdConfigTest.getMaxEventSizeBytes(), equalTo(max_event_size)); + assertThat(thresholdConfigTest.getMaxEventSizeBytes(), equalTo(max_event_size * ThresholdConfig.CONVERT_TO_BYTES_FROM_KB)); } @ParameterizedTest diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/something/CloudWatchLogsLimitsTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimitsTest.java similarity index 78% rename from data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/something/CloudWatchLogsLimitsTest.java rename to data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimitsTest.java index 2e283b1006..18eff78eef 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/something/CloudWatchLogsLimitsTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimitsTest.java @@ -3,14 +3,13 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.something; +package org.opensearch.dataprepper.plugins.sink.push_condition; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; -import org.opensearch.dataprepper.plugins.sink.push_condition.CloudWatchLogsLimits; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -20,21 +19,21 @@ public class CloudWatchLogsLimitsTest { @BeforeEach void setUp() { - cloudWatchLogsLimits = new CloudWatchLogsLimits(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE, + cloudWatchLogsLimits = new CloudWatchLogsLimits(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE * ThresholdConfig.CONVERT_TO_BYTES_FROM_KB, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME); } @ParameterizedTest - @ValueSource(ints = {55, 80, 100}) + @ValueSource(ints = {257, 560, 1000}) void check_max_event_size_invalid(final int event_size) { - boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize(event_size); + boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize(event_size * ThresholdConfig.CONVERT_TO_BYTES_FROM_KB); assertTrue(isEventGreater); } @ParameterizedTest @ValueSource(ints = {10, 30, 50}) void check_max_event_size_valid(final int event_size) { - boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize(event_size); + boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize((event_size * ThresholdConfig.CONVERT_TO_BYTES_FROM_KB) - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); assertFalse(isEventGreater); } @@ -48,12 +47,13 @@ void check_greater_than_threshold_conditions_time_true(final int send_interval) @ParameterizedTest @ValueSource(ints = {1, 3, 59}) void check_greater_than_threshold_conditions_time_false(final int send_interval) { - boolean thresholdMetTime = cloudWatchLogsLimits.isGreaterThanLimitReached(send_interval, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST,ThresholdConfig.DEFAULT_BATCH_SIZE); + int validRequestSize = ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + boolean thresholdMetTime = cloudWatchLogsLimits.isGreaterThanLimitReached(send_interval, validRequestSize ,ThresholdConfig.DEFAULT_BATCH_SIZE); assertFalse(thresholdMetTime); } @ParameterizedTest - @ValueSource(ints = {550000, 750000, 1000000}) + @ValueSource(ints = {1500000, 3000000, 10000000}) void check_greater_than_threshold_conditions_request_size_true(final int request_size) { boolean thresholdMetRequestSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, request_size, ThresholdConfig.DEFAULT_BATCH_SIZE); assertTrue(thresholdMetRequestSize); @@ -76,13 +76,15 @@ void check_greater_than_threshold_conditions_batch_size_true(final int batch_siz @ParameterizedTest @ValueSource(ints = {1, 10, 25}) void check_greater_than_threshold_conditions_batch_size_false(final int batch_size) { - boolean thresholdMetBatchSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME -1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, batch_size); + int validRequestSize = ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + boolean thresholdMetBatchSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME -1, validRequestSize, batch_size); assertFalse(thresholdMetBatchSize); } @Test void check_equal_than_threshold_conditions_request_size_true() { - boolean thresholdMetRequestSize = cloudWatchLogsLimits.isEqualToLimitReached(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); + int validRequestSize = ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + boolean thresholdMetRequestSize = cloudWatchLogsLimits.isEqualToLimitReached(validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); assertTrue(thresholdMetRequestSize); } From 9bbfeddb8a7a1b46a0817fe36cacd6f11cd361d6 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Thu, 20 Jul 2023 12:21:21 -0700 Subject: [PATCH 30/43] Added exponential backofftime Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/client/CloudWatchLogsDispatcher.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java index f661949086..ac02ed0f9d 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java @@ -21,6 +21,8 @@ import java.util.concurrent.BlockingQueue; public class CloudWatchLogsDispatcher implements Runnable { + public static final long UPPER_RETRY_TIME_BOUND = 20000; + public static final float EXP_TIME_SCALER = 1.5F; private static final Logger LOG = LoggerFactory.getLogger(CloudWatchLogsDispatcher.class); private final BlockingQueue taskQueue; private final CloudWatchLogsClient cloudWatchLogsClient; @@ -111,7 +113,13 @@ public boolean dispatchLogs(List inputLogEvents, Collection= UPPER_RETRY_TIME_BOUND) { + return UPPER_RETRY_TIME_BOUND; + } + + return scale * backOffTimeBase; } @Override From 6e28adc6a2a0ce1d58e1797d9c3251ce3cdc7ccc Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Thu, 20 Jul 2023 13:55:25 -0700 Subject: [PATCH 31/43] Fixed unused imports Signed-off-by: Marcos Gonzalez Mayedo --- .../dataprepper/plugins/sink/buffer/Buffer.java | 1 - .../plugins/sink/client/CloudWatchLogsService.java | 1 - .../plugins/sink/client/CloudWatchLogsDispatcherTest.java | 8 +++++++- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java index a0c20c13ab..d0c05cf0a4 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java @@ -5,7 +5,6 @@ package org.opensearch.dataprepper.plugins.sink.buffer; -import java.util.ArrayList; import java.util.List; /** diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index 3cf12f4cbb..7b8bb3d801 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -18,7 +18,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Executor; import java.util.concurrent.locks.ReentrantLock; diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java index a16d21ef18..7186e9ee4c 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java @@ -20,7 +20,13 @@ import java.util.ArrayList; import java.util.concurrent.BlockingQueue; -import static org.mockito.Mockito.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + public class CloudWatchLogsDispatcherTest { private CloudWatchLogsDispatcher cloudWatchLogsDispatcher; From 90418aacd1ef8b099f7b73b14a97488b65e58db7 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Thu, 20 Jul 2023 16:19:35 -0700 Subject: [PATCH 32/43] Fixed up deepcopy of arraylist for service workers in CloudWatchLogsService, and fixed Log calling methods Signed-off-by: Marcos Gonzalez Mayedo --- .../sink/client/CloudWatchLogsClientFactory.java | 3 +++ .../sink/client/CloudWatchLogsDispatcher.java | 7 +++---- .../sink/client/CloudWatchLogsService.java | 15 ++++++++++++--- .../plugins/sink/config/AwsConfig.java | 2 +- .../sink/client/CloudWatchLogsDispatcherTest.java | 3 +-- .../plugins/sink/config/AwsConfigTest.java | 2 +- 6 files changed, 21 insertions(+), 11 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java index c9a5c91d90..8d2672e9ab 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java @@ -20,6 +20,9 @@ * CloudWatchLogs services. */ public final class CloudWatchLogsClientFactory { + private CloudWatchLogsClientFactory() { + throw new IllegalStateException("Static Factory Class!"); + } /** * Generates a CloudWatchLogs Client based on STS role ARN system credentials. diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java index ac02ed0f9d..1c3137f99e 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java @@ -86,26 +86,25 @@ public boolean dispatchLogs(List inputLogEvents, Collection> logs) { int logLength = log.getData().toJsonString().length(); if (cloudWatchLogsLimits.isGreaterThanMaxEventSize(logLength)) { - LOG.warn("Event blocked due to Max Size restriction! {Event Size: " + (logLength + CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE) + " bytes}"); + LOG.warn("Event blocked due to Max Size restriction! {Event Size: {} bytes}", (logLength + CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); continue; } @@ -91,11 +91,14 @@ public void processLogEvents(final Collection> logs) { private void stageLogEvents() { sinkStopWatch.stopAndResetStopWatch(); - ThreadTaskEvents dataToPush = new ThreadTaskEvents(buffer.getBufferedData(), bufferedEventHandles); + ArrayList eventMessageCloneList = new ArrayList<>(); + cloneLists(buffer.getBufferedData(), eventMessageCloneList); + + ThreadTaskEvents dataToPush = new ThreadTaskEvents(eventMessageCloneList, bufferedEventHandles); taskQueue.add(dataToPush); - bufferedEventHandles = new ArrayList<>(); buffer.clearBuffer(); + bufferedEventHandles = new ArrayList<>(); sinkThreadManager.execute(dispatcher); } @@ -106,4 +109,10 @@ private void addToBuffer(final Record log) { } buffer.writeEvent(log.getData().toString().getBytes()); } + + private void cloneLists(List listToCopy, List listToCopyInto) { + for (byte[] holder: listToCopy) { + listToCopyInto.add(holder.clone()); + } + } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java index da5ba6dd1d..cb1cb3ae5f 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java @@ -17,7 +17,7 @@ * and resources. */ public class AwsConfig { - public static int DEFAULT_CONNECTION_ATTEMPTS = 5; + public static final int DEFAULT_CONNECTION_ATTEMPTS = 5; @JsonProperty("region") @Size(min = 1, message = "Region cannot be empty string") diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java index 7186e9ee4c..16bb10dd16 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java @@ -27,8 +27,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; - -public class CloudWatchLogsDispatcherTest { + public class CloudWatchLogsDispatcherTest { private CloudWatchLogsDispatcher cloudWatchLogsDispatcher; private BlockingQueue mockTaskQueue; private CloudWatchLogsClient cloudWatchLogsClient; diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfigTest.java index 43e9c9b285..225a9ed30b 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfigTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfigTest.java @@ -20,7 +20,7 @@ import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; -public class AwsConfigTest { +class AwsConfigTest { private ObjectMapper objectMapper; @BeforeEach From 5971190eb523a50bf5a79d9e019943853990c572 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Mon, 24 Jul 2023 14:34:52 -0700 Subject: [PATCH 33/43] Added CloudWatchLogsDispatcher builder pattern, fixed tests for Service and Dispatcher and modified backOffTimeBase Signed-off-by: Marcos Gonzalez Mayedo --- .../cloudwatch-logs/build.gradle | 3 + .../plugins/sink/buffer/InMemoryBuffer.java | 1 - .../client/CloudWatchLogsClientFactory.java | 4 +- .../sink/client/CloudWatchLogsDispatcher.java | 36 +-- .../sink/client/CloudWatchLogsService.java | 50 +++- .../CloudWatchLogsLimits.java | 12 +- .../sink/{time => utils}/SinkStopWatch.java | 7 +- .../client/CloudWatchLogsServiceTest.java | 230 ++++-------------- .../CloudWatchLogsLimitsTest.java | 1 + 9 files changed, 125 insertions(+), 219 deletions(-) rename data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/{push_condition => utils}/CloudWatchLogsLimits.java (91%) rename data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/{time => utils}/SinkStopWatch.java (86%) diff --git a/data-prepper-plugins/cloudwatch-logs/build.gradle b/data-prepper-plugins/cloudwatch-logs/build.gradle index bd387d69ef..0a3c815c0b 100644 --- a/data-prepper-plugins/cloudwatch-logs/build.gradle +++ b/data-prepper-plugins/cloudwatch-logs/build.gradle @@ -17,9 +17,12 @@ dependencies { implementation 'software.amazon.awssdk:cloudwatch' implementation 'software.amazon.awssdk:cloudwatchlogs' implementation 'org.apache.commons:commons-lang3:3.12.0' + implementation 'org.projectlombok:lombok:1.18.26' testImplementation project(path: ':data-prepper-test-common') testImplementation testLibs.mockito.inline testImplementation 'org.junit.jupiter:junit-jupiter' + compileOnly 'org.projectlombok:lombok:1.18.24' + annotationProcessor 'org.projectlombok:lombok:1.18.24' } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java index a93cb99ad7..572f1ad9c1 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java @@ -41,7 +41,6 @@ public byte[] popEvent() { @Override public List getBufferedData() { - Collections.unmodifiableList(eventsBuffered); return Collections.unmodifiableList(eventsBuffered); } diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java index 8d2672e9ab..00ba993060 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java @@ -41,10 +41,8 @@ public static CloudWatchLogsClient createCwlClient(final AwsConfig awsConfig, fi } private static ClientOverrideConfiguration createOverrideConfiguration() { - final RetryPolicy retryPolicy = RetryPolicy.builder().numRetries(AwsConfig.DEFAULT_CONNECTION_ATTEMPTS).build(); - return ClientOverrideConfiguration.builder() - .retryPolicy(retryPolicy) + .retryPolicy(r -> r.numRetries(AwsConfig.DEFAULT_CONNECTION_ATTEMPTS)) .build(); } diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java index 1c3137f99e..bd49248288 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java @@ -5,7 +5,9 @@ package org.opensearch.dataprepper.plugins.sink.client; +import lombok.Builder; import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.plugins.sink.packaging.ThreadTaskEvents; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -20,22 +22,23 @@ import java.util.List; import java.util.concurrent.BlockingQueue; +@Builder public class CloudWatchLogsDispatcher implements Runnable { - public static final long UPPER_RETRY_TIME_BOUND = 20000; - public static final float EXP_TIME_SCALER = 1.5F; + private static final long UPPER_RETRY_TIME_BOUND_MILLISECONDS = 5000; + private static final float EXP_TIME_SCALE = 1.5F; private static final Logger LOG = LoggerFactory.getLogger(CloudWatchLogsDispatcher.class); - private final BlockingQueue taskQueue; - private final CloudWatchLogsClient cloudWatchLogsClient; - private final CloudWatchLogsMetrics cloudWatchLogsMetrics; - private final String logGroup; - private final String logStream; - final int retryCount; - final long backOffTimeBase; + private BlockingQueue taskQueue; + private CloudWatchLogsClient cloudWatchLogsClient; + private CloudWatchLogsMetrics cloudWatchLogsMetrics; + private String logGroup; + private String logStream; + private int retryCount; + private long backOffTimeBase; public CloudWatchLogsDispatcher(final BlockingQueue taskQueue, final CloudWatchLogsClient cloudWatchLogsClient, final CloudWatchLogsMetrics cloudWatchLogsMetrics, final String logGroup, final String logStream, - final int retryCount, final int backOffTimeBase) { + final int retryCount, final long backOffTimeBase) { this.taskQueue = taskQueue; this.cloudWatchLogsClient = cloudWatchLogsClient; this.cloudWatchLogsMetrics = cloudWatchLogsMetrics; @@ -64,7 +67,7 @@ private List prepareInputLogEvents(final ThreadTaskEvents eventDa * @param inputLogEvents Collection of inputLogEvents to be flushed * @return true if successful, false otherwise */ - public boolean dispatchLogs(List inputLogEvents, Collection eventHandles) { + public void dispatchLogs(List inputLogEvents, Collection eventHandles) { boolean failedPost = true; int failCounter = 0; @@ -74,7 +77,6 @@ public boolean dispatchLogs(List inputLogEvents, Collection inputLogEvents, Collection= UPPER_RETRY_TIME_BOUND) { - return UPPER_RETRY_TIME_BOUND; + if (scale >= UPPER_RETRY_TIME_BOUND_MILLISECONDS) { + return UPPER_RETRY_TIME_BOUND_MILLISECONDS; } return scale * backOffTimeBase; diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index 1de8fc9b72..8480f9b874 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -10,10 +10,11 @@ import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.sink.buffer.Buffer; import org.opensearch.dataprepper.plugins.sink.packaging.ThreadTaskEvents; -import org.opensearch.dataprepper.plugins.sink.push_condition.CloudWatchLogsLimits; -import org.opensearch.dataprepper.plugins.sink.time.SinkStopWatch; +import org.opensearch.dataprepper.plugins.sink.utils.CloudWatchLogsLimits; +import org.opensearch.dataprepper.plugins.sink.utils.SinkStopWatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; import java.util.ArrayList; import java.util.Collection; @@ -26,28 +27,43 @@ public class CloudWatchLogsService { private static final Logger LOG = LoggerFactory.getLogger(CloudWatchLogsService.class); + private final CloudWatchLogsClient cloudWatchLogsClient; + private final CloudWatchLogsMetrics cloudWatchLogsMetrics; private final Buffer buffer; private final CloudWatchLogsLimits cloudWatchLogsLimits; private List bufferedEventHandles; private final BlockingQueue taskQueue; private final SinkStopWatch sinkStopWatch; private final ReentrantLock bufferLock; - private final Executor sinkThreadManager; - private final CloudWatchLogsDispatcher dispatcher; + private final Executor asyncExecutor; + private final String logGroup; + private final String logStream; + private final int retryCount; + private final long backOffTimeBase; public CloudWatchLogsService(final Buffer buffer, + final CloudWatchLogsClient cloudWatchLogsClient, + final CloudWatchLogsMetrics cloudWatchLogsMetrics, final CloudWatchLogsLimits cloudWatchLogsLimits, - final CloudWatchLogsDispatcher dispatcher, BlockingQueue blockingQueue) { + final BlockingQueue blockingQueue, + final String logGroup, final String logStream, + final int retryCount, final long backOffTimeBase) { this.buffer = buffer; - this.dispatcher = dispatcher; - this.bufferedEventHandles = new ArrayList<>(); + this.cloudWatchLogsClient = cloudWatchLogsClient; + this.cloudWatchLogsMetrics = cloudWatchLogsMetrics; this.cloudWatchLogsLimits = cloudWatchLogsLimits; this.taskQueue = blockingQueue; + this.logGroup = logGroup; + this.logStream = logStream; + this.retryCount = retryCount; + this.backOffTimeBase = backOffTimeBase; + + this.bufferedEventHandles = new ArrayList<>(); bufferLock = new ReentrantLock(); sinkStopWatch = new SinkStopWatch(); - sinkThreadManager = newCachedThreadPool(); + asyncExecutor = newCachedThreadPool(); } /** @@ -91,16 +107,26 @@ public void processLogEvents(final Collection> logs) { private void stageLogEvents() { sinkStopWatch.stopAndResetStopWatch(); - ArrayList eventMessageCloneList = new ArrayList<>(); - cloneLists(buffer.getBufferedData(), eventMessageCloneList); + ArrayList eventMessageClone = new ArrayList<>(); + cloneLists(buffer.getBufferedData(), eventMessageClone); - ThreadTaskEvents dataToPush = new ThreadTaskEvents(eventMessageCloneList, bufferedEventHandles); + ThreadTaskEvents dataToPush = new ThreadTaskEvents(eventMessageClone, bufferedEventHandles); taskQueue.add(dataToPush); buffer.clearBuffer(); bufferedEventHandles = new ArrayList<>(); - sinkThreadManager.execute(dispatcher); + CloudWatchLogsDispatcher newTaskDispatcher = CloudWatchLogsDispatcher.builder() + .taskQueue(taskQueue) + .cloudWatchLogsClient(cloudWatchLogsClient) + .cloudWatchLogsMetrics(cloudWatchLogsMetrics) + .logGroup(logGroup) + .logStream(logStream) + .retryCount(retryCount) + .backOffTimeBase(backOffTimeBase) + .build(); + + asyncExecutor.execute(newTaskDispatcher); } private void addToBuffer(final Record log) { diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimits.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/CloudWatchLogsLimits.java similarity index 91% rename from data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimits.java rename to data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/CloudWatchLogsLimits.java index d1fc837da9..64fdb4afcc 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimits.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/CloudWatchLogsLimits.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.push_condition; +package org.opensearch.dataprepper.plugins.sink.utils; /** * ThresholdCheck receives parameters for which to reference the * limits of a buffer and CloudWatchLogsClient before making a @@ -11,13 +11,13 @@ */ public class CloudWatchLogsLimits { public static final int APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE = 26; //Size of overhead for each log event message. - private final int batchSize; + private final int maxBatchSize; private final int maxEventSizeBytes; private final int maxRequestSizeBytes; private final long logSendInterval; - public CloudWatchLogsLimits(final int batchSize, final int maxEventSizeBytes, final int maxRequestSizeBytes, final int logSendInterval) { - this.batchSize = batchSize; + public CloudWatchLogsLimits(final int maxBatchSize, final int maxEventSizeBytes, final int maxRequestSizeBytes, final int logSendInterval) { + this.maxBatchSize = maxBatchSize; this.maxEventSizeBytes = maxEventSizeBytes; this.maxRequestSizeBytes = maxRequestSizeBytes; this.logSendInterval = logSendInterval; @@ -82,7 +82,7 @@ private boolean isGreaterThanMaxRequestSize(final int currentRequestSize) { * @return boolean - true if greater, false otherwise. */ private boolean isGreaterThanBatchSize(final int batchSize) { - return batchSize > this.batchSize; + return batchSize > this.maxBatchSize; } /** @@ -95,6 +95,6 @@ private boolean isEqualMaxRequestSize(final int currentRequestSize) { } private boolean isEqualBatchSize(final int batchSize) { - return batchSize == this.batchSize; + return batchSize == this.maxBatchSize; } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/time/SinkStopWatch.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java similarity index 86% rename from data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/time/SinkStopWatch.java rename to data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java index 52352d6cd1..036fa949f5 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/time/SinkStopWatch.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java @@ -3,13 +3,18 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.time; +package org.opensearch.dataprepper.plugins.sink.utils; import org.apache.commons.lang3.time.StopWatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantLock; +/** + * A synchronized watch for accessing time + * related data. (Wrapper around StopWatch class + * from "commons.apache.lang3") + */ public class SinkStopWatch { private final StopWatch stopWatch; private final ReentrantLock stopWatchLock; diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java index da548a55a1..30980ad10a 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java @@ -7,40 +7,53 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventHandle; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.sink.buffer.Buffer; import org.opensearch.dataprepper.plugins.sink.buffer.InMemoryBufferFactory; +import org.opensearch.dataprepper.plugins.sink.client.CloudWatchLogsDispatcher.CloudWatchLogsDispatcherBuilder; import org.opensearch.dataprepper.plugins.sink.config.CloudWatchLogsSinkConfig; import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; import org.opensearch.dataprepper.plugins.sink.packaging.ThreadTaskEvents; -import org.opensearch.dataprepper.plugins.sink.push_condition.CloudWatchLogsLimits; +import org.opensearch.dataprepper.plugins.sink.utils.CloudWatchLogsLimits; +import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; +import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; import java.util.ArrayList; import java.util.Collection; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.RETURNS_DEEP_STUBS; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.never; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.when; public class CloudWatchLogsServiceTest { - private static final int NUMBER_THREADS_SMALL = 5; - private static final int NUMBER_THREADS_BIG = 10; - private static final int NUMBER_THREADS_LARGE = 20; - private BlockingQueue mockQueue; + private static final int MAX_QUEUE_SIZE = 100; + private CloudWatchLogsClient mockClient; + private CloudWatchLogsMetrics mockMetrics; + private BlockingQueue testQueue; private CloudWatchLogsService cloudWatchLogsService; + private CloudWatchLogsDispatcherBuilder mockDispatchBuilder; private CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig; private ThresholdConfig thresholdConfig; private CloudWatchLogsLimits cloudWatchLogsLimits; private InMemoryBufferFactory inMemoryBufferFactory; private Buffer buffer; - private CloudWatchLogsDispatcher dispatcher; - private volatile int testCounter; + private CloudWatchLogsDispatcher testDispatcher; + private final String logGroup = "testGroup"; + private final String logStream = "testStream"; @BeforeEach void setUp() { @@ -50,14 +63,33 @@ void setUp() { cloudWatchLogsLimits = new CloudWatchLogsLimits(thresholdConfig.getBatchSize(), thresholdConfig.getMaxEventSizeBytes(), thresholdConfig.getMaxRequestSize(), thresholdConfig.getLogSendInterval()); + mockClient = mock(CloudWatchLogsClient.class); + mockMetrics = mock(CloudWatchLogsMetrics.class); inMemoryBufferFactory = new InMemoryBufferFactory(); buffer = inMemoryBufferFactory.getBuffer(); - dispatcher = mock(CloudWatchLogsDispatcher.class); - mockQueue = mock(BlockingQueue.class); - - cloudWatchLogsService = new CloudWatchLogsService(buffer, cloudWatchLogsLimits, dispatcher, mockQueue); - - testCounter = 0; + testDispatcher = mock(CloudWatchLogsDispatcher.class); + testQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE); + + mockDispatchBuilder = mock(CloudWatchLogsDispatcherBuilder.class, RETURNS_DEEP_STUBS); + when(mockDispatchBuilder.taskQueue(any(BlockingQueue.class))).thenReturn(mockDispatchBuilder); + when(mockDispatchBuilder.cloudWatchLogsClient(any(CloudWatchLogsClient.class))).thenReturn(mockDispatchBuilder); + when(mockDispatchBuilder.cloudWatchLogsMetrics(any(CloudWatchLogsMetrics.class))).thenReturn(mockDispatchBuilder); + when(mockDispatchBuilder.logGroup(anyString())).thenReturn(mockDispatchBuilder); + when(mockDispatchBuilder.logStream(anyString())).thenReturn(mockDispatchBuilder); + when(mockDispatchBuilder.retryCount(anyInt())).thenReturn(mockDispatchBuilder); + when(mockDispatchBuilder.backOffTimeBase(anyInt())).thenReturn(mockDispatchBuilder); + when(mockDispatchBuilder.taskQueue(any(BlockingQueue.class)) + .cloudWatchLogsClient(any(CloudWatchLogsClient.class)) + .cloudWatchLogsMetrics(any(CloudWatchLogsMetrics.class)).logGroup(logGroup) + .logStream(anyString()) + .retryCount(anyInt()) + .backOffTimeBase(anyLong()) + .build()).thenReturn(testDispatcher); + + cloudWatchLogsService = new CloudWatchLogsService(buffer, mockClient, mockMetrics, + cloudWatchLogsLimits, testQueue, + logGroup, logStream, + thresholdConfig.getRetryCount(), thresholdConfig.getBackOffTime()); } Collection> getSampleRecordsLess() { @@ -99,178 +131,20 @@ Collection> getSampleRecordsLarge() { @Test void check_dispatcher_run_was_not_called() { cloudWatchLogsService.processLogEvents(getSampleRecordsLess()); - verify(dispatcher, never()).run(); + verify(mockClient, never()).putLogEvents(any(PutLogEventsRequest.class)); } @Test - void check_dispatcher_run_was_called_test() { + void check_dispatcher_run_was_called_test() throws InterruptedException { cloudWatchLogsService.processLogEvents(getSampleRecords()); - verify(dispatcher, atLeastOnce()).run(); + Thread.sleep(100); + verify(mockClient, atLeastOnce()).putLogEvents(any(PutLogEventsRequest.class)); } @Test - void check_dispatcher_run_called_heavy_load() { + void check_dispatcher_run_called_heavy_load() throws InterruptedException { cloudWatchLogsService.processLogEvents(getSampleRecordsLarge()); - verify(dispatcher, atLeast(4)).run(); - } - - //TODO: Add multithreaded testing to ensure that the proper methods (run) gets called. - - @Test - void test_less_threads_normal_load() { - Collection threadsToRun = new ArrayList<>(); - - for (int i = 0; i < NUMBER_THREADS_SMALL; i++) { - Thread testingThread = new Thread(new CloudWatchLogsServiceTester(getSampleRecords(), cloudWatchLogsService)); - threadsToRun.add(testingThread); - } - - for (Thread serviceTester: threadsToRun) { - serviceTester.start(); - } - - for (Thread serviceTester: threadsToRun) { - try { - serviceTester.join(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - - verify(dispatcher, atLeast(NUMBER_THREADS_SMALL)).run(); - } - - @Test - void test_less_threads_heavy_load() { - Collection threadsToRun = new ArrayList<>(); - - for (int i = 0; i < NUMBER_THREADS_SMALL; i++) { - Thread testingThread = new Thread(new CloudWatchLogsServiceTester(getSampleRecordsLarge(), cloudWatchLogsService)); - threadsToRun.add(testingThread); - } - - for (Thread serviceTester: threadsToRun) { - serviceTester.start(); - } - - for (Thread serviceTester: threadsToRun) { - try { - serviceTester.join(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - - verify(dispatcher, atLeast(NUMBER_THREADS_SMALL * 4)).run(); - } - - @Test - void test_more_threads_normal_load() { - Collection threadsToRun = new ArrayList<>(); - - for (int i = 0; i < NUMBER_THREADS_BIG; i++) { - Thread testingThread = new Thread(new CloudWatchLogsServiceTester(getSampleRecords(), cloudWatchLogsService)); - threadsToRun.add(testingThread); - } - - for (Thread serviceTester: threadsToRun) { - serviceTester.start(); - } - - for (Thread serviceTester: threadsToRun) { - try { - serviceTester.join(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - - verify(dispatcher, atLeast(NUMBER_THREADS_BIG)).run(); - } - - @Test - void test_more_threads_heavy_load() { - Collection threadsToRun = new ArrayList<>(); - - for (int i = 0; i < NUMBER_THREADS_BIG; i++) { - Thread testingThread = new Thread(new CloudWatchLogsServiceTester(getSampleRecordsLarge(), cloudWatchLogsService)); - threadsToRun.add(testingThread); - } - - for (Thread serviceTester: threadsToRun) { - serviceTester.start(); - } - - for (Thread serviceTester: threadsToRun) { - try { - serviceTester.join(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - - verify(dispatcher, atLeast(NUMBER_THREADS_BIG * 4)).run(); - } - - @Test - void test_large_threads_normal_load() { - Collection threadsToRun = new ArrayList<>(); - - for (int i = 0; i < NUMBER_THREADS_LARGE; i++) { - Thread testingThread = new Thread(new CloudWatchLogsServiceTester(getSampleRecords(), cloudWatchLogsService)); - threadsToRun.add(testingThread); - } - - for (Thread serviceTester: threadsToRun) { - serviceTester.start(); - } - - for (Thread serviceTester: threadsToRun) { - try { - serviceTester.join(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - - verify(dispatcher, atLeast(NUMBER_THREADS_LARGE)).run(); - } - - @Test - void test_large_threads_heavy_load() { - Collection threadsToRun = new ArrayList<>(); - - for (int i = 0; i < NUMBER_THREADS_LARGE; i++) { - Thread testingThread = new Thread(new CloudWatchLogsServiceTester(getSampleRecordsLarge(), cloudWatchLogsService)); - threadsToRun.add(testingThread); - } - - for (Thread serviceTester: threadsToRun) { - serviceTester.start(); - } - - for (Thread serviceTester: threadsToRun) { - try { - serviceTester.join(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - - verify(dispatcher, atLeast(NUMBER_THREADS_LARGE * 4)).run(); - } - - static class CloudWatchLogsServiceTester implements Runnable { - Collection> testEvents; - CloudWatchLogsService testCloudWatchLogsService; - CloudWatchLogsServiceTester(Collection> events, CloudWatchLogsService cloudWatchLogsService) { - testEvents = events; - testCloudWatchLogsService = cloudWatchLogsService; - } - - @Override - public void run() { - testCloudWatchLogsService.processLogEvents(testEvents); - } + Thread.sleep(100); + verify(mockClient, atLeast(4)).putLogEvents(any(PutLogEventsRequest.class)); } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimitsTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimitsTest.java index 18eff78eef..1ae09ce471 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimitsTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimitsTest.java @@ -10,6 +10,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; +import org.opensearch.dataprepper.plugins.sink.utils.CloudWatchLogsLimits; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; From 53086c4507a91184c760d8c8a46eb4b744b0a1b3 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Mon, 24 Jul 2023 18:45:40 -0700 Subject: [PATCH 34/43] Removed unused imports Signed-off-by:Marcos Gonzalez Mayedo Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/client/CloudWatchLogsClientFactory.java | 1 - .../plugins/sink/client/CloudWatchLogsDispatcher.java | 3 +-- .../plugins/sink/client/CloudWatchLogsService.java | 8 ++++++++ .../plugins/sink/client/CloudWatchLogsServiceTest.java | 1 - 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java index 00ba993060..a1d60aabf8 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java @@ -10,7 +10,6 @@ import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; -import software.amazon.awssdk.core.retry.RetryPolicy; import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; /** diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java index bd49248288..703474d6a3 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java @@ -7,7 +7,6 @@ import lombok.Builder; import org.opensearch.dataprepper.model.event.EventHandle; -import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.plugins.sink.packaging.ThreadTaskEvents; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,7 +64,7 @@ private List prepareInputLogEvents(final ThreadTaskEvents eventDa /** * Flush function to handle the flushing of logs to CloudWatchLogs services; * @param inputLogEvents Collection of inputLogEvents to be flushed - * @return true if successful, false otherwise + * @param eventHandles Collection of EventHandles for events */ public void dispatchLogs(List inputLogEvents, Collection eventHandles) { boolean failedPost = true; diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index 8480f9b874..3404f4b827 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -25,6 +25,14 @@ import static java.util.concurrent.Executors.newCachedThreadPool; //TODO: Can implement a more strict pooling method if needed. +/** + * CloudWatchLogs Service encapsulates the log processing step. + * It accomplishes this by: + * 1. Reading in log events. + * 2. Buffering data. + * 3. Checking for limit conditions. + * 4. Making PLE calls to CloudWatchLogs. + */ public class CloudWatchLogsService { private static final Logger LOG = LoggerFactory.getLogger(CloudWatchLogsService.class); private final CloudWatchLogsClient cloudWatchLogsClient; diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java index 30980ad10a..688d3bbf91 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java @@ -7,7 +7,6 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.mockito.MockedStatic; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventHandle; import org.opensearch.dataprepper.model.event.JacksonEvent; From 26f18a1df2577a37d690cb0a5e700852a4cb9f01 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Tue, 25 Jul 2023 13:11:12 -0700 Subject: [PATCH 35/43] Added resetBuffer method, removed unnecessary RetransmissionException, and added logString pass in parameter for staging log events. Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/buffer/Buffer.java | 2 ++ .../plugins/sink/buffer/InMemoryBuffer.java | 8 +++++++- .../sink/client/CloudWatchLogsService.java | 18 +++++++++--------- .../RetransmissionLimitException.java | 12 ------------ 4 files changed, 18 insertions(+), 22 deletions(-) delete mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/exception/RetransmissionLimitException.java diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java index d0c05cf0a4..6124cb7381 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java @@ -34,4 +34,6 @@ public interface Buffer { List getBufferedData(); void clearBuffer(); + + void resetBuffer(); } \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java index 572f1ad9c1..e3794676eb 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java @@ -10,7 +10,7 @@ import java.util.List; public class InMemoryBuffer implements Buffer { - private final List eventsBuffered; + private List eventsBuffered; private int bufferSize = 0; InMemoryBuffer() { @@ -49,4 +49,10 @@ public void clearBuffer() { bufferSize = 0; eventsBuffered.clear(); } + + @Override + public void resetBuffer() { + bufferSize = 0; + eventsBuffered = new ArrayList<>(); + } } \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index 3404f4b827..21d084b49f 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -82,7 +82,8 @@ public CloudWatchLogsService(final Buffer buffer, public void processLogEvents(final Collection> logs) { sinkStopWatch.startIfNotRunning(); for (Record log: logs) { - int logLength = log.getData().toJsonString().length(); + String logString = log.getData().toJsonString(); + int logLength = logString.length(); if (cloudWatchLogsLimits.isGreaterThanMaxEventSize(logLength)) { LOG.warn("Event blocked due to Max Size restriction! {Event Size: {} bytes}", (logLength + CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); @@ -100,12 +101,12 @@ public void processLogEvents(final Collection> logs) { if ((cloudWatchLogsLimits.isGreaterThanLimitReached(time, bufferSizeWithAddedEvent, bufferEventCountWithEvent) && (bufferEventCount > 0))) { stageLogEvents(); - addToBuffer(log); + addToBuffer(log, logString); } else if (cloudWatchLogsLimits.isEqualToLimitReached(bufferSizeWithAddedEvent, bufferEventCountWithEvent)) { - addToBuffer(log); + addToBuffer(log, logString); stageLogEvents(); } else { - addToBuffer(log); + addToBuffer(log, logString); } bufferLock.unlock(); @@ -115,13 +116,12 @@ public void processLogEvents(final Collection> logs) { private void stageLogEvents() { sinkStopWatch.stopAndResetStopWatch(); - ArrayList eventMessageClone = new ArrayList<>(); - cloneLists(buffer.getBufferedData(), eventMessageClone); + List eventMessageClone = buffer.getBufferedData(); ThreadTaskEvents dataToPush = new ThreadTaskEvents(eventMessageClone, bufferedEventHandles); taskQueue.add(dataToPush); - buffer.clearBuffer(); + buffer.resetBuffer(); bufferedEventHandles = new ArrayList<>(); CloudWatchLogsDispatcher newTaskDispatcher = CloudWatchLogsDispatcher.builder() @@ -137,11 +137,11 @@ private void stageLogEvents() { asyncExecutor.execute(newTaskDispatcher); } - private void addToBuffer(final Record log) { + private void addToBuffer(final Record log, final String logString) { if (log.getData().getEventHandle() != null) { bufferedEventHandles.add(log.getData().getEventHandle()); } - buffer.writeEvent(log.getData().toString().getBytes()); + buffer.writeEvent(logString.getBytes()); } private void cloneLists(List listToCopy, List listToCopyInto) { diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/exception/RetransmissionLimitException.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/exception/RetransmissionLimitException.java deleted file mode 100644 index c476dabc55..0000000000 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/exception/RetransmissionLimitException.java +++ /dev/null @@ -1,12 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.sink.exception; - -public class RetransmissionLimitException extends RuntimeException{ - public RetransmissionLimitException(String message) { - super(message); - } -} \ No newline at end of file From a5b8be7b67f7b5384096319924f870626140d285 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Tue, 25 Jul 2023 21:06:47 -0700 Subject: [PATCH 36/43] Started making changes to the tests to implement the new class structure (performance enhancement) Signed-off-by: Marcos Gonzalez Mayedo --- .../sink/client/CloudWatchLogsDispatcher.java | 134 ++++++++++-------- .../sink/client/CloudWatchLogsService.java | 27 +--- .../client/CloudWatchLogsDispatcherTest.java | 12 +- .../client/CloudWatchLogsServiceTest.java | 20 +-- 4 files changed, 86 insertions(+), 107 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java index 703474d6a3..a7c96053c7 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java @@ -19,35 +19,36 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Executor; -@Builder -public class CloudWatchLogsDispatcher implements Runnable { +import static java.util.concurrent.Executors.newCachedThreadPool; + +public class CloudWatchLogsDispatcher { private static final long UPPER_RETRY_TIME_BOUND_MILLISECONDS = 5000; private static final float EXP_TIME_SCALE = 1.5F; private static final Logger LOG = LoggerFactory.getLogger(CloudWatchLogsDispatcher.class); - private BlockingQueue taskQueue; private CloudWatchLogsClient cloudWatchLogsClient; private CloudWatchLogsMetrics cloudWatchLogsMetrics; + private Executor asyncExecutor; private String logGroup; private String logStream; private int retryCount; private long backOffTimeBase; - public CloudWatchLogsDispatcher(final BlockingQueue taskQueue, - final CloudWatchLogsClient cloudWatchLogsClient, + public CloudWatchLogsDispatcher(final CloudWatchLogsClient cloudWatchLogsClient, final CloudWatchLogsMetrics cloudWatchLogsMetrics, final String logGroup, final String logStream, final int retryCount, final long backOffTimeBase) { - this.taskQueue = taskQueue; this.cloudWatchLogsClient = cloudWatchLogsClient; this.cloudWatchLogsMetrics = cloudWatchLogsMetrics; this.logGroup = logGroup; this.logStream = logStream; this.retryCount = retryCount; this.backOffTimeBase = backOffTimeBase; + + asyncExecutor = newCachedThreadPool(); } - private List prepareInputLogEvents(final ThreadTaskEvents eventData) { + public List prepareInputLogEvents(final ThreadTaskEvents eventData) { List logEventList = new ArrayList<>(); for (byte[] data: eventData.getEventMessages()) { @@ -67,78 +68,91 @@ private List prepareInputLogEvents(final ThreadTaskEvents eventDa * @param eventHandles Collection of EventHandles for events */ public void dispatchLogs(List inputLogEvents, Collection eventHandles) { - boolean failedPost = true; - int failCounter = 0; - PutLogEventsRequest putLogEventsRequest = PutLogEventsRequest.builder() .logEvents(inputLogEvents) .logGroupName(logGroup) .logStreamName(logStream) .build(); - try { - while (failedPost && (failCounter < retryCount)) { - try { - cloudWatchLogsClient.putLogEvents(putLogEventsRequest); + asyncExecutor.execute(Uploader.builder().cloudWatchLogsClient(cloudWatchLogsClient) + .cloudWatchLogsMetrics(cloudWatchLogsMetrics) + .putLogEventsRequest(putLogEventsRequest) + .eventHandles(eventHandles) + .backOffTimeBase(backOffTimeBase) + .retryCount(retryCount) + .build()); + } + + @Builder + private static class Uploader implements Runnable { + private CloudWatchLogsClient cloudWatchLogsClient; + private CloudWatchLogsMetrics cloudWatchLogsMetrics; + private PutLogEventsRequest putLogEventsRequest; + private Collection eventHandles; + private int retryCount; + private long backOffTimeBase; + + @Override + public void run() { + upload(); + } + + public void upload() { + boolean failedPost = true; + int failCounter = 0; + + try { + while (failedPost && (failCounter < retryCount)) { + try { + cloudWatchLogsClient.putLogEvents(putLogEventsRequest); - cloudWatchLogsMetrics.increaseRequestSuccessCounter(1); - failedPost = false; + cloudWatchLogsMetrics.increaseRequestSuccessCounter(1); + failedPost = false; - //TODO: When a log is rejected by the service, we cannot send it, can probably push to a DLQ here. + //TODO: When a log is rejected by the service, we cannot send it, can probably push to a DLQ here. - } catch (CloudWatchLogsException | SdkClientException e) { - LOG.error("Service-Worker {} Failed to push logs with error: {}", Thread.currentThread().getName(), e.getMessage()); - cloudWatchLogsMetrics.increaseRequestFailCounter(1); - Thread.sleep(calculateBackOffTime(backOffTimeBase, failCounter)); - LOG.warn("Service-Worker {} Trying to retransmit request... {Attempt: {} }", Thread.currentThread().getName(), (++failCounter)); + } catch (CloudWatchLogsException | SdkClientException e) { + LOG.error("Service-Worker {} Failed to push logs with error: {}", Thread.currentThread().getName(), e.getMessage()); + cloudWatchLogsMetrics.increaseRequestFailCounter(1); + Thread.sleep(calculateBackOffTime(backOffTimeBase, failCounter)); + LOG.warn("Service-Worker {} Trying to retransmit request... {Attempt: {} }", Thread.currentThread().getName(), (++failCounter)); + } } + } catch (InterruptedException e) { + LOG.warn("Got interrupted while waiting!"); + //TODO: Push to DLQ. + Thread.currentThread().interrupt(); } - } catch (InterruptedException e) { - LOG.warn("Got interrupted while waiting!"); - //TODO: Push to DLQ. - Thread.currentThread().interrupt(); - } - if (failedPost) { - cloudWatchLogsMetrics.increaseLogEventFailCounter(inputLogEvents.size()); - releaseEventHandles(false, eventHandles); - } else { - cloudWatchLogsMetrics.increaseLogEventSuccessCounter(inputLogEvents.size()); - releaseEventHandles(true, eventHandles); + if (failedPost) { + cloudWatchLogsMetrics.increaseLogEventFailCounter(eventHandles.size()); + releaseEventHandles(false, eventHandles); + } else { + cloudWatchLogsMetrics.increaseLogEventSuccessCounter(eventHandles.size()); + releaseEventHandles(true, eventHandles); + } } - } - - //TODO: Can abstract this if clients want more choice. - private long calculateBackOffTime(final long backOffTimeBase, final int failCounter) { - long scale = (long)Math.pow(EXP_TIME_SCALE, failCounter); - if (scale >= UPPER_RETRY_TIME_BOUND_MILLISECONDS) { - return UPPER_RETRY_TIME_BOUND_MILLISECONDS; - } + //TODO: Can abstract this if clients want more choice. + private long calculateBackOffTime(final long backOffTimeBase, final int failCounter) { + long scale = (long)Math.pow(EXP_TIME_SCALE, failCounter); - return scale * backOffTimeBase; - } + if (scale >= UPPER_RETRY_TIME_BOUND_MILLISECONDS) { + return UPPER_RETRY_TIME_BOUND_MILLISECONDS; + } - @Override - public void run() { - try { - ThreadTaskEvents taskData = taskQueue.take(); - List inputLogEvents = prepareInputLogEvents(taskData); - dispatchLogs(inputLogEvents, taskData.getEventHandles()); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - //TODO: Implement back up to taskQueue read failure. + return scale * backOffTimeBase; } - } - private void releaseEventHandles(final boolean result, final Collection eventHandles) { - if (eventHandles.isEmpty()) { - return; - } + private void releaseEventHandles(final boolean result, final Collection eventHandles) { + if (eventHandles.isEmpty()) { + return; + } - for (EventHandle eventHandle : eventHandles) { - eventHandle.release(result); + for (EventHandle eventHandle : eventHandles) { + eventHandle.release(result); + } } } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index 21d084b49f..fc3a074bbd 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -15,12 +15,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; +import software.amazon.awssdk.services.cloudwatchlogs.model.InputLogEvent; import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.Executor; import java.util.concurrent.locks.ReentrantLock; import static java.util.concurrent.Executors.newCachedThreadPool; //TODO: Can implement a more strict pooling method if needed. @@ -35,15 +34,14 @@ */ public class CloudWatchLogsService { private static final Logger LOG = LoggerFactory.getLogger(CloudWatchLogsService.class); + private final CloudWatchLogsDispatcher cloudWatchLogsDispatcher; private final CloudWatchLogsClient cloudWatchLogsClient; private final CloudWatchLogsMetrics cloudWatchLogsMetrics; private final Buffer buffer; private final CloudWatchLogsLimits cloudWatchLogsLimits; private List bufferedEventHandles; - private final BlockingQueue taskQueue; private final SinkStopWatch sinkStopWatch; private final ReentrantLock bufferLock; - private final Executor asyncExecutor; private final String logGroup; private final String logStream; private final int retryCount; @@ -53,7 +51,6 @@ public CloudWatchLogsService(final Buffer buffer, final CloudWatchLogsClient cloudWatchLogsClient, final CloudWatchLogsMetrics cloudWatchLogsMetrics, final CloudWatchLogsLimits cloudWatchLogsLimits, - final BlockingQueue blockingQueue, final String logGroup, final String logStream, final int retryCount, final long backOffTimeBase) { @@ -61,7 +58,6 @@ public CloudWatchLogsService(final Buffer buffer, this.cloudWatchLogsClient = cloudWatchLogsClient; this.cloudWatchLogsMetrics = cloudWatchLogsMetrics; this.cloudWatchLogsLimits = cloudWatchLogsLimits; - this.taskQueue = blockingQueue; this.logGroup = logGroup; this.logStream = logStream; this.retryCount = retryCount; @@ -71,7 +67,9 @@ public CloudWatchLogsService(final Buffer buffer, bufferLock = new ReentrantLock(); sinkStopWatch = new SinkStopWatch(); - asyncExecutor = newCachedThreadPool(); + + cloudWatchLogsDispatcher = new CloudWatchLogsDispatcher(cloudWatchLogsClient, + cloudWatchLogsMetrics, logGroup, logStream, retryCount, backOffTimeBase); } /** @@ -117,24 +115,13 @@ private void stageLogEvents() { sinkStopWatch.stopAndResetStopWatch(); List eventMessageClone = buffer.getBufferedData(); - ThreadTaskEvents dataToPush = new ThreadTaskEvents(eventMessageClone, bufferedEventHandles); - taskQueue.add(dataToPush); buffer.resetBuffer(); bufferedEventHandles = new ArrayList<>(); - CloudWatchLogsDispatcher newTaskDispatcher = CloudWatchLogsDispatcher.builder() - .taskQueue(taskQueue) - .cloudWatchLogsClient(cloudWatchLogsClient) - .cloudWatchLogsMetrics(cloudWatchLogsMetrics) - .logGroup(logGroup) - .logStream(logStream) - .retryCount(retryCount) - .backOffTimeBase(backOffTimeBase) - .build(); - - asyncExecutor.execute(newTaskDispatcher); + List inputLogEvents = cloudWatchLogsDispatcher.prepareInputLogEvents(dataToPush); + cloudWatchLogsDispatcher.dispatchLogs(inputLogEvents, dataToPush.getEventHandles()); } private void addToBuffer(final Record log, final String logString) { diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java index 16bb10dd16..5b3141ebcc 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java @@ -18,7 +18,6 @@ import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; import java.util.ArrayList; -import java.util.concurrent.BlockingQueue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.atLeastOnce; @@ -29,7 +28,6 @@ public class CloudWatchLogsDispatcherTest { private CloudWatchLogsDispatcher cloudWatchLogsDispatcher; - private BlockingQueue mockTaskQueue; private CloudWatchLogsClient cloudWatchLogsClient; private CloudWatchLogsMetrics cloudWatchLogsMetrics; private PluginMetrics pluginMetrics; @@ -43,7 +41,6 @@ public class CloudWatchLogsDispatcherTest { @BeforeEach void setUp() throws InterruptedException { - mockTaskQueue = mock(BlockingQueue.class); cloudWatchLogsClient = mock(CloudWatchLogsClient.class); pluginMetrics = mock(PluginMetrics.class); @@ -52,8 +49,6 @@ void setUp() throws InterruptedException { successEventCounter = mock(Counter.class); failedEventCounter = mock(Counter.class); - when(mockTaskQueue.take()).thenReturn(getSampleBufferedData()); - cloudWatchLogsMetrics = mock(CloudWatchLogsMetrics.class); when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_EVENTS_SUCCEEDED)).thenReturn(successEventCounter); @@ -76,7 +71,7 @@ ThreadTaskEvents getSampleBufferedData() { } CloudWatchLogsDispatcher getCloudWatchLogsDispatcher() { - return new CloudWatchLogsDispatcher(mockTaskQueue, cloudWatchLogsClient, + return new CloudWatchLogsDispatcher(cloudWatchLogsClient, cloudWatchLogsMetrics, LOG_GROUP, LOG_STREAM, ThresholdConfig.DEFAULT_RETRY_COUNT, ThresholdConfig.DEFAULT_BACKOFF_TIME); } @@ -90,13 +85,14 @@ void establishFailingClientWithSdkClientExcept() { } void setUpInterruptedQueueException() throws InterruptedException { - when(mockTaskQueue.take()).thenThrow(InterruptedException.class); +// when(mockTaskQueue.take()).thenThrow(InterruptedException.class); } @Test void check_successful_transmission_test() { cloudWatchLogsDispatcher = getCloudWatchLogsDispatcher(); - cloudWatchLogsDispatcher.run(); + cloudWatchLogsDispatcher.prepareInputLogEvents(getSampleBufferedData()); + cloudWatchLogsDispatcher.dispatchLogs(); verify(cloudWatchLogsMetrics, atLeastOnce()).increaseRequestSuccessCounter(1); verify(cloudWatchLogsMetrics, atLeastOnce()).increaseLogEventSuccessCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java index 688d3bbf91..42283e614d 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java @@ -13,7 +13,6 @@ import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.sink.buffer.Buffer; import org.opensearch.dataprepper.plugins.sink.buffer.InMemoryBufferFactory; -import org.opensearch.dataprepper.plugins.sink.client.CloudWatchLogsDispatcher.CloudWatchLogsDispatcherBuilder; import org.opensearch.dataprepper.plugins.sink.config.CloudWatchLogsSinkConfig; import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; import org.opensearch.dataprepper.plugins.sink.packaging.ThreadTaskEvents; @@ -44,7 +43,6 @@ public class CloudWatchLogsServiceTest { private CloudWatchLogsMetrics mockMetrics; private BlockingQueue testQueue; private CloudWatchLogsService cloudWatchLogsService; - private CloudWatchLogsDispatcherBuilder mockDispatchBuilder; private CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig; private ThresholdConfig thresholdConfig; private CloudWatchLogsLimits cloudWatchLogsLimits; @@ -69,24 +67,8 @@ void setUp() { testDispatcher = mock(CloudWatchLogsDispatcher.class); testQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE); - mockDispatchBuilder = mock(CloudWatchLogsDispatcherBuilder.class, RETURNS_DEEP_STUBS); - when(mockDispatchBuilder.taskQueue(any(BlockingQueue.class))).thenReturn(mockDispatchBuilder); - when(mockDispatchBuilder.cloudWatchLogsClient(any(CloudWatchLogsClient.class))).thenReturn(mockDispatchBuilder); - when(mockDispatchBuilder.cloudWatchLogsMetrics(any(CloudWatchLogsMetrics.class))).thenReturn(mockDispatchBuilder); - when(mockDispatchBuilder.logGroup(anyString())).thenReturn(mockDispatchBuilder); - when(mockDispatchBuilder.logStream(anyString())).thenReturn(mockDispatchBuilder); - when(mockDispatchBuilder.retryCount(anyInt())).thenReturn(mockDispatchBuilder); - when(mockDispatchBuilder.backOffTimeBase(anyInt())).thenReturn(mockDispatchBuilder); - when(mockDispatchBuilder.taskQueue(any(BlockingQueue.class)) - .cloudWatchLogsClient(any(CloudWatchLogsClient.class)) - .cloudWatchLogsMetrics(any(CloudWatchLogsMetrics.class)).logGroup(logGroup) - .logStream(anyString()) - .retryCount(anyInt()) - .backOffTimeBase(anyLong()) - .build()).thenReturn(testDispatcher); - cloudWatchLogsService = new CloudWatchLogsService(buffer, mockClient, mockMetrics, - cloudWatchLogsLimits, testQueue, + cloudWatchLogsLimits, logGroup, logStream, thresholdConfig.getRetryCount(), thresholdConfig.getBackOffTime()); } From 070beefd4c7dcd04f80ff0b07a3dcd66115c5479 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Thu, 27 Jul 2023 12:13:08 -0700 Subject: [PATCH 37/43] Refactored the CloudWatchLogsDispatcher into two classes with the addition of Uploader, introduced simple multithread tests for CloudWatchLogsService Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/buffer/Buffer.java | 7 ++ .../sink/client/CloudWatchLogsDispatcher.java | 16 +-- .../sink/client/CloudWatchLogsService.java | 105 ++++++---------- .../sink/packaging/ThreadTaskEvents.java | 30 ----- .../client/CloudWatchLogsDispatcherTest.java | 99 +++++---------- .../client/CloudWatchLogsServiceTest.java | 113 ++++++++++++------ .../plugins/sink/client/UploaderTest.java | 93 ++++++++++++++ 7 files changed, 251 insertions(+), 212 deletions(-) delete mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/packaging/ThreadTaskEvents.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/UploaderTest.java diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java index 6124cb7381..3c3ccf9f77 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/Buffer.java @@ -14,6 +14,13 @@ * 2. Transforms to Byte type. * 3. Returns a Byte type. */ + +/* + TODO: + Need to add PriorityQueue for extracting timestamp, this will need the timestamp and the actual string message itself. + Can refactor the buffer to contain + */ + public interface Buffer { /** * Size of buffer in events. diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java index a7c96053c7..15e4446e6f 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java @@ -7,7 +7,6 @@ import lombok.Builder; import org.opensearch.dataprepper.model.event.EventHandle; -import org.opensearch.dataprepper.plugins.sink.packaging.ThreadTaskEvents; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.core.exception.SdkClientException; @@ -16,13 +15,13 @@ import software.amazon.awssdk.services.cloudwatchlogs.model.InputLogEvent; import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.concurrent.Executor; -import static java.util.concurrent.Executors.newCachedThreadPool; - +@Builder public class CloudWatchLogsDispatcher { private static final long UPPER_RETRY_TIME_BOUND_MILLISECONDS = 5000; private static final float EXP_TIME_SCALE = 1.5F; @@ -36,6 +35,7 @@ public class CloudWatchLogsDispatcher { private long backOffTimeBase; public CloudWatchLogsDispatcher(final CloudWatchLogsClient cloudWatchLogsClient, final CloudWatchLogsMetrics cloudWatchLogsMetrics, + Executor asyncExecutor, final String logGroup, final String logStream, final int retryCount, final long backOffTimeBase) { this.cloudWatchLogsClient = cloudWatchLogsClient; @@ -45,15 +45,15 @@ public CloudWatchLogsDispatcher(final CloudWatchLogsClient cloudWatchLogsClient, this.retryCount = retryCount; this.backOffTimeBase = backOffTimeBase; - asyncExecutor = newCachedThreadPool(); + this.asyncExecutor = asyncExecutor; } - public List prepareInputLogEvents(final ThreadTaskEvents eventData) { + public List prepareInputLogEvents(final Collection eventMessageBytes) { List logEventList = new ArrayList<>(); - for (byte[] data: eventData.getEventMessages()) { + for (byte[] data : eventMessageBytes) { InputLogEvent tempLogEvent = InputLogEvent.builder() - .message(new String(data)) + .message(new String(data, StandardCharsets.UTF_8)) .timestamp(System.currentTimeMillis()) .build(); logEventList.add(tempLogEvent); @@ -84,7 +84,7 @@ public void dispatchLogs(List inputLogEvents, Collection bufferedEventHandles; private final SinkStopWatch sinkStopWatch; - private final ReentrantLock bufferLock; - private final String logGroup; - private final String logStream; - private final int retryCount; - private final long backOffTimeBase; - + private final ReentrantLock processLock; public CloudWatchLogsService(final Buffer buffer, - final CloudWatchLogsClient cloudWatchLogsClient, - final CloudWatchLogsMetrics cloudWatchLogsMetrics, final CloudWatchLogsLimits cloudWatchLogsLimits, - final String logGroup, final String logStream, - final int retryCount, final long backOffTimeBase) { + final CloudWatchLogsDispatcher cloudWatchLogsDispatcher) { this.buffer = buffer; - this.cloudWatchLogsClient = cloudWatchLogsClient; - this.cloudWatchLogsMetrics = cloudWatchLogsMetrics; this.cloudWatchLogsLimits = cloudWatchLogsLimits; - this.logGroup = logGroup; - this.logStream = logStream; - this.retryCount = retryCount; - this.backOffTimeBase = backOffTimeBase; - this.bufferedEventHandles = new ArrayList<>(); - bufferLock = new ReentrantLock(); + processLock = new ReentrantLock(); sinkStopWatch = new SinkStopWatch(); - cloudWatchLogsDispatcher = new CloudWatchLogsDispatcher(cloudWatchLogsClient, - cloudWatchLogsMetrics, logGroup, logStream, retryCount, backOffTimeBase); + this.cloudWatchLogsDispatcher = cloudWatchLogsDispatcher; } /** @@ -78,62 +57,54 @@ public CloudWatchLogsService(final Buffer buffer, * @param logs - Collection of Record events which hold log data. */ public void processLogEvents(final Collection> logs) { - sinkStopWatch.startIfNotRunning(); - for (Record log: logs) { - String logString = log.getData().toJsonString(); - int logLength = logString.length(); - - if (cloudWatchLogsLimits.isGreaterThanMaxEventSize(logLength)) { - LOG.warn("Event blocked due to Max Size restriction! {Event Size: {} bytes}", (logLength + CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); - continue; + try { + sinkStopWatch.startIfNotRunning(); + for (Record log : logs) { + String logString = log.getData().toJsonString(); + int logLength = logString.length(); + + if (cloudWatchLogsLimits.isGreaterThanMaxEventSize(logLength)) { + LOG.warn("Event blocked due to Max Size restriction! {Event Size: {} bytes}", (logLength + CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + continue; + } + + long time = sinkStopWatch.getStopWatchTimeSeconds(); + + processLock.lock(); + int bufferSize = buffer.getBufferSize(); + int bufferEventCount = buffer.getEventCount(); + int bufferEventCountWithEvent = bufferEventCount + 1; + int bufferSizeWithAddedEvent = bufferSize + logLength; + + if ((cloudWatchLogsLimits.isGreaterThanLimitReached(time, bufferSizeWithAddedEvent, bufferEventCountWithEvent) && (bufferEventCount > 0))) { + stageLogEvents(); + addToBuffer(log, logString); + } else if (cloudWatchLogsLimits.isEqualToLimitReached(bufferSizeWithAddedEvent, bufferEventCountWithEvent)) { + addToBuffer(log, logString); + stageLogEvents(); + } else { + addToBuffer(log, logString); + } } - - long time = sinkStopWatch.getStopWatchTimeSeconds(); - - bufferLock.lock(); - - int bufferSize = buffer.getBufferSize(); - int bufferEventCount = buffer.getEventCount(); - int bufferEventCountWithEvent = bufferEventCount + 1; - int bufferSizeWithAddedEvent = bufferSize + logLength; - - if ((cloudWatchLogsLimits.isGreaterThanLimitReached(time, bufferSizeWithAddedEvent, bufferEventCountWithEvent) && (bufferEventCount > 0))) { - stageLogEvents(); - addToBuffer(log, logString); - } else if (cloudWatchLogsLimits.isEqualToLimitReached(bufferSizeWithAddedEvent, bufferEventCountWithEvent)) { - addToBuffer(log, logString); - stageLogEvents(); - } else { - addToBuffer(log, logString); - } - - bufferLock.unlock(); + } finally { + processLock.unlock(); } } private void stageLogEvents() { sinkStopWatch.stopAndResetStopWatch(); - List eventMessageClone = buffer.getBufferedData(); - ThreadTaskEvents dataToPush = new ThreadTaskEvents(eventMessageClone, bufferedEventHandles); + List inputLogEvents = cloudWatchLogsDispatcher.prepareInputLogEvents(buffer.getBufferedData()); + cloudWatchLogsDispatcher.dispatchLogs(inputLogEvents, bufferedEventHandles); buffer.resetBuffer(); bufferedEventHandles = new ArrayList<>(); - - List inputLogEvents = cloudWatchLogsDispatcher.prepareInputLogEvents(dataToPush); - cloudWatchLogsDispatcher.dispatchLogs(inputLogEvents, dataToPush.getEventHandles()); } private void addToBuffer(final Record log, final String logString) { if (log.getData().getEventHandle() != null) { bufferedEventHandles.add(log.getData().getEventHandle()); } - buffer.writeEvent(logString.getBytes()); - } - - private void cloneLists(List listToCopy, List listToCopyInto) { - for (byte[] holder: listToCopy) { - listToCopyInto.add(holder.clone()); - } + buffer.writeEvent(logString.getBytes(StandardCharsets.UTF_8)); } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/packaging/ThreadTaskEvents.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/packaging/ThreadTaskEvents.java deleted file mode 100644 index fb9653af92..0000000000 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/packaging/ThreadTaskEvents.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.sink.packaging; - -import org.opensearch.dataprepper.model.event.EventHandle; - -import java.util.Collection; - -/** - * Simple data class for packaging event messages and their handles into a queue. - */ -public class ThreadTaskEvents { - Collection eventMessages; - Collection eventHandles; - public ThreadTaskEvents(Collection eventMessages, Collection eventHandles) { - this.eventMessages = eventMessages; - this.eventHandles = eventHandles; - } - - public Collection getEventMessages() { - return eventMessages; - } - - public Collection getEventHandles() { - return eventHandles; - } -} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java index 5b3141ebcc..715aafb24c 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java @@ -5,36 +5,28 @@ package org.opensearch.dataprepper.plugins.sink.client; -import io.micrometer.core.instrument.Counter; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.EventHandle; import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; -import org.opensearch.dataprepper.plugins.sink.packaging.ThreadTaskEvents; -import software.amazon.awssdk.core.exception.SdkClientException; import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; -import software.amazon.awssdk.services.cloudwatchlogs.model.CloudWatchLogsException; -import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; +import software.amazon.awssdk.services.cloudwatchlogs.model.InputLogEvent; import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.Executor; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.atMostOnce; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - public class CloudWatchLogsDispatcherTest { +class CloudWatchLogsDispatcherTest { private CloudWatchLogsDispatcher cloudWatchLogsDispatcher; private CloudWatchLogsClient cloudWatchLogsClient; private CloudWatchLogsMetrics cloudWatchLogsMetrics; - private PluginMetrics pluginMetrics; - private Counter requestSuccessCounter; - private Counter requestFailCounter; - private Counter successEventCounter; - private Counter failedEventCounter; + private Executor asyncExecutor; private static final String LOG_GROUP = "testGroup"; private static final String LOG_STREAM = "testStream"; private static final String TEST_STRING = "testMessage"; @@ -42,79 +34,48 @@ public class CloudWatchLogsDispatcherTest { @BeforeEach void setUp() throws InterruptedException { cloudWatchLogsClient = mock(CloudWatchLogsClient.class); - - pluginMetrics = mock(PluginMetrics.class); - requestSuccessCounter = mock(Counter.class); - requestFailCounter = mock(Counter.class); - successEventCounter = mock(Counter.class); - failedEventCounter = mock(Counter.class); - cloudWatchLogsMetrics = mock(CloudWatchLogsMetrics.class); - - when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_EVENTS_SUCCEEDED)).thenReturn(successEventCounter); - when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED)).thenReturn(requestSuccessCounter); - when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_EVENTS_FAILED)).thenReturn(failedEventCounter); - when(pluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_REQUESTS_FAILED)).thenReturn(requestFailCounter); + asyncExecutor = mock(Executor.class); } - ThreadTaskEvents getSampleBufferedData() { + Collection getSampleBufferedData() { final ArrayList returnCollection = new ArrayList<>(); - final ArrayList eventHandles = new ArrayList<>(); for (int i = 0; i < ThresholdConfig.DEFAULT_BATCH_SIZE; i++) { returnCollection.add(new String(TEST_STRING).getBytes()); - final EventHandle mockEventHandle = mock(EventHandle.class); - eventHandles.add(mockEventHandle); } - return new ThreadTaskEvents(returnCollection, eventHandles); - } - - CloudWatchLogsDispatcher getCloudWatchLogsDispatcher() { - return new CloudWatchLogsDispatcher(cloudWatchLogsClient, - cloudWatchLogsMetrics, LOG_GROUP, LOG_STREAM, ThresholdConfig.DEFAULT_RETRY_COUNT, - ThresholdConfig.DEFAULT_BACKOFF_TIME); - } - - void establishFailingClientWithCloudWatchLogsExcept() { - when(cloudWatchLogsClient.putLogEvents(any(PutLogEventsRequest.class))).thenThrow(CloudWatchLogsException.class); - } - - void establishFailingClientWithSdkClientExcept() { - when(cloudWatchLogsClient.putLogEvents(any(PutLogEventsRequest.class))).thenThrow(SdkClientException.class); + return returnCollection; } - void setUpInterruptedQueueException() throws InterruptedException { -// when(mockTaskQueue.take()).thenThrow(InterruptedException.class); - } + Collection getSampleEventHandles() { + final ArrayList eventHandles = new ArrayList<>(); - @Test - void check_successful_transmission_test() { - cloudWatchLogsDispatcher = getCloudWatchLogsDispatcher(); - cloudWatchLogsDispatcher.prepareInputLogEvents(getSampleBufferedData()); - cloudWatchLogsDispatcher.dispatchLogs(); + for (int i = 0; i < ThresholdConfig.DEFAULT_BATCH_SIZE; i++) { + final EventHandle mockEventHandle = mock(EventHandle.class); + eventHandles.add(mockEventHandle); + } - verify(cloudWatchLogsMetrics, atLeastOnce()).increaseRequestSuccessCounter(1); - verify(cloudWatchLogsMetrics, atLeastOnce()).increaseLogEventSuccessCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); + return eventHandles; } - @Test - void check_unsuccesful_transmission_with_cloudwatchlogsexcept_test() { - establishFailingClientWithCloudWatchLogsExcept(); - cloudWatchLogsDispatcher = getCloudWatchLogsDispatcher(); - cloudWatchLogsDispatcher.run(); - - verify(cloudWatchLogsMetrics, times(ThresholdConfig.DEFAULT_RETRY_COUNT)).increaseRequestFailCounter(1); - verify(cloudWatchLogsMetrics, atLeastOnce()).increaseLogEventFailCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); + CloudWatchLogsDispatcher getCloudWatchLogsDispatcher() { + return CloudWatchLogsDispatcher.builder().cloudWatchLogsClient(cloudWatchLogsClient) + .cloudWatchLogsMetrics(cloudWatchLogsMetrics) + .asyncExecutor(asyncExecutor) + .logGroup(LOG_GROUP) + .logStream(LOG_STREAM) + .retryCount(ThresholdConfig.DEFAULT_RETRY_COUNT) + .backOffTimeBase(ThresholdConfig.DEFAULT_BACKOFF_TIME) + .build(); } @Test - void check_unsuccesful_transmission_with_sdkexcept_test() { - establishFailingClientWithSdkClientExcept(); + void check_execute_called_test() { cloudWatchLogsDispatcher = getCloudWatchLogsDispatcher(); - cloudWatchLogsDispatcher.run(); + List inputLogEventList = cloudWatchLogsDispatcher.prepareInputLogEvents(getSampleBufferedData()); + cloudWatchLogsDispatcher.dispatchLogs(inputLogEventList, getSampleEventHandles()); - verify(cloudWatchLogsMetrics, times(ThresholdConfig.DEFAULT_RETRY_COUNT)).increaseRequestFailCounter(1); - verify(cloudWatchLogsMetrics, atLeastOnce()).increaseLogEventFailCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); + verify(asyncExecutor, atMostOnce()).execute(any(CloudWatchLogsDispatcher.Uploader.class)); } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java index 42283e614d..767d553f9a 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java @@ -12,36 +12,31 @@ import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.sink.buffer.Buffer; +import org.opensearch.dataprepper.plugins.sink.buffer.InMemoryBuffer; import org.opensearch.dataprepper.plugins.sink.buffer.InMemoryBufferFactory; import org.opensearch.dataprepper.plugins.sink.config.CloudWatchLogsSinkConfig; import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; -import org.opensearch.dataprepper.plugins.sink.packaging.ThreadTaskEvents; import org.opensearch.dataprepper.plugins.sink.utils.CloudWatchLogsLimits; import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; -import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; import java.util.ArrayList; import java.util.Collection; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; +import java.util.List; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.never; -import static org.mockito.Mockito.atLeastOnce; -import static org.mockito.Mockito.atLeast; -import static org.mockito.Mockito.when; public class CloudWatchLogsServiceTest { - private static final int MAX_QUEUE_SIZE = 100; + private static int SMALL_THREAD_COUNT = 50; + private static int MEDIUM_THREAD_COUNT = 100; + private static int HIGH_THREAD_COUNT = 500; + private static int LARGE_THREAD_COUNT = 1000; private CloudWatchLogsClient mockClient; private CloudWatchLogsMetrics mockMetrics; - private BlockingQueue testQueue; private CloudWatchLogsService cloudWatchLogsService; private CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig; private ThresholdConfig thresholdConfig; @@ -49,8 +44,6 @@ public class CloudWatchLogsServiceTest { private InMemoryBufferFactory inMemoryBufferFactory; private Buffer buffer; private CloudWatchLogsDispatcher testDispatcher; - private final String logGroup = "testGroup"; - private final String logStream = "testStream"; @BeforeEach void setUp() { @@ -63,14 +56,9 @@ void setUp() { mockClient = mock(CloudWatchLogsClient.class); mockMetrics = mock(CloudWatchLogsMetrics.class); inMemoryBufferFactory = new InMemoryBufferFactory(); - buffer = inMemoryBufferFactory.getBuffer(); testDispatcher = mock(CloudWatchLogsDispatcher.class); - testQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE); - - cloudWatchLogsService = new CloudWatchLogsService(buffer, mockClient, mockMetrics, - cloudWatchLogsLimits, - logGroup, logStream, - thresholdConfig.getRetryCount(), thresholdConfig.getBackOffTime()); + cloudWatchLogsService = new CloudWatchLogsService(buffer, + cloudWatchLogsLimits, testDispatcher); } Collection> getSampleRecordsLess() { @@ -97,35 +85,84 @@ Collection> getSampleRecords() { return returnCollection; } - Collection> getSampleRecordsLarge() { - final ArrayList> returnCollection = new ArrayList<>(); - for (int i = 0; i < (thresholdConfig.getBatchSize() * 4); i++) { - JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("testMessage"); - final EventHandle mockEventHandle = mock(EventHandle.class); - mockJacksonEvent.setEventHandle(mockEventHandle); - returnCollection.add(new Record<>(mockJacksonEvent)); - } + void setUpSpyBuffer() { + buffer = spy(InMemoryBuffer.class); + } - return returnCollection; + void setUpRealBuffer() { + buffer = inMemoryBufferFactory.getBuffer(); + } + + CloudWatchLogsService getSampleService() { + return new CloudWatchLogsService(buffer, cloudWatchLogsLimits, testDispatcher); } @Test void check_dispatcher_run_was_not_called() { + setUpRealBuffer(); + cloudWatchLogsService = getSampleService(); cloudWatchLogsService.processLogEvents(getSampleRecordsLess()); - verify(mockClient, never()).putLogEvents(any(PutLogEventsRequest.class)); + verify(testDispatcher, never()).dispatchLogs(any(List.class), any(Collection.class)); } @Test void check_dispatcher_run_was_called_test() throws InterruptedException { + setUpRealBuffer(); + cloudWatchLogsService = getSampleService(); cloudWatchLogsService.processLogEvents(getSampleRecords()); - Thread.sleep(100); - verify(mockClient, atLeastOnce()).putLogEvents(any(PutLogEventsRequest.class)); + verify(testDispatcher, atLeast(1)).dispatchLogs(any(List.class), any(Collection.class)); + } + + //Multithreaded tests: + void testThreadsProcessingLogsWithNormalSample(final int numberOfThreads) throws InterruptedException { + Thread[] threads = new Thread[numberOfThreads]; + Collection> sampleEvents = getSampleRecords(); + + for (int i = 0; i < numberOfThreads; i++) { + threads[i] = new Thread(() -> { + cloudWatchLogsService.processLogEvents(sampleEvents); + }); + threads[i].start(); + } + + for (int i = 0; i < numberOfThreads; i++) { + threads[i].join(); + } } @Test - void check_dispatcher_run_called_heavy_load() throws InterruptedException { - cloudWatchLogsService.processLogEvents(getSampleRecordsLarge()); - Thread.sleep(100); - verify(mockClient, atLeast(4)).putLogEvents(any(PutLogEventsRequest.class)); + void test_buffer_access_with_small_thread_count_test() throws InterruptedException { + setUpSpyBuffer(); + cloudWatchLogsService = getSampleService(); + testThreadsProcessingLogsWithNormalSample(SMALL_THREAD_COUNT); + + verify(buffer, atLeast(SMALL_THREAD_COUNT)).getBufferedData(); + } + + @Test + void test_buffer_access_with_medium_thread_count_test() throws InterruptedException { + setUpSpyBuffer(); + cloudWatchLogsService = getSampleService(); + testThreadsProcessingLogsWithNormalSample(MEDIUM_THREAD_COUNT); + + verify(buffer, atLeast(MEDIUM_THREAD_COUNT)).getBufferedData(); + } + + @Test + void test_buffer_access_with_high_thread_count_test() throws InterruptedException { + setUpSpyBuffer(); + cloudWatchLogsService = getSampleService(); + testThreadsProcessingLogsWithNormalSample(HIGH_THREAD_COUNT); + + verify(buffer, atLeast(HIGH_THREAD_COUNT)).getBufferedData(); + } + + @Test + void test_buffer_access_with_large_thread_count_test() throws InterruptedException { + setUpSpyBuffer(); + cloudWatchLogsService = getSampleService(); + testThreadsProcessingLogsWithNormalSample(LARGE_THREAD_COUNT); + + verify(buffer, atLeast(LARGE_THREAD_COUNT)).getBufferedData(); } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/UploaderTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/UploaderTest.java new file mode 100644 index 0000000000..04ecc76bc7 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/UploaderTest.java @@ -0,0 +1,93 @@ +package org.opensearch.dataprepper.plugins.sink.client; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; +import software.amazon.awssdk.services.cloudwatchlogs.model.CloudWatchLogsException; +import software.amazon.awssdk.services.cloudwatchlogs.model.PutLogEventsRequest; + +import java.util.ArrayList; +import java.util.Collection; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +class UploaderTest { + private CloudWatchLogsClient cloudWatchLogsClient; + private CloudWatchLogsMetrics cloudWatchLogsMetrics; + + @BeforeEach + void setUp() { + cloudWatchLogsClient = mock(CloudWatchLogsClient.class); + cloudWatchLogsMetrics = mock(CloudWatchLogsMetrics.class); + } + + Collection getTestEventHandles() { + final ArrayList eventHandles = new ArrayList<>(); + for (int i = 0; i < ThresholdConfig.DEFAULT_BATCH_SIZE; i++) { + final EventHandle mockEventHandle = mock(EventHandle.class); + eventHandles.add(mockEventHandle); + } + + return eventHandles; + } + + PutLogEventsRequest getMockPutLogEventsRequest() { + return mock(PutLogEventsRequest.class); + } + + CloudWatchLogsDispatcher.Uploader getUploader() { + return CloudWatchLogsDispatcher.Uploader.builder() + .cloudWatchLogsClient(cloudWatchLogsClient) + .cloudWatchLogsMetrics(cloudWatchLogsMetrics) + .putLogEventsRequest(getMockPutLogEventsRequest()) + .eventHandles(getTestEventHandles()) + .retryCount(ThresholdConfig.DEFAULT_RETRY_COUNT) + .backOffTimeBase(ThresholdConfig.DEFAULT_BACKOFF_TIME) + .build(); + } + + void establishFailingClientWithCloudWatchLogsExcept() { + when(cloudWatchLogsClient.putLogEvents(any(PutLogEventsRequest.class))).thenThrow(CloudWatchLogsException.class); + } + + void establishFailingClientWithSdkClientExcept() { + when(cloudWatchLogsClient.putLogEvents(any(PutLogEventsRequest.class))).thenThrow(SdkClientException.class); + } + + @Test + void check_successful_transmission_test() throws InterruptedException { + CloudWatchLogsDispatcher.Uploader testUploader = getUploader(); + testUploader.run(); + + verify(cloudWatchLogsMetrics, atLeastOnce()).increaseRequestSuccessCounter(1); + verify(cloudWatchLogsMetrics, atLeastOnce()).increaseLogEventSuccessCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); + } + + @Test + void check_unsuccesful_transmission_with_cloudwatchlogsexcept_test() throws InterruptedException { + establishFailingClientWithCloudWatchLogsExcept(); + CloudWatchLogsDispatcher.Uploader testUploader = getUploader(); + testUploader.run(); + + verify(cloudWatchLogsMetrics, times(ThresholdConfig.DEFAULT_RETRY_COUNT)).increaseRequestFailCounter(1); + verify(cloudWatchLogsMetrics, atLeastOnce()).increaseLogEventFailCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); + } + + @Test + void check_unsuccesful_transmission_with_sdkexcept_test() { + establishFailingClientWithSdkClientExcept(); + CloudWatchLogsDispatcher.Uploader testUploader = getUploader(); + testUploader.run(); + + verify(cloudWatchLogsMetrics, times(ThresholdConfig.DEFAULT_RETRY_COUNT)).increaseRequestFailCounter(1); + verify(cloudWatchLogsMetrics, atLeastOnce()).increaseLogEventFailCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); + } +} From 6130b0839aad9aca65e46d8d2cc3dd4ee1f59406 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Thu, 27 Jul 2023 16:17:50 -0700 Subject: [PATCH 38/43] Fixed issues with locking in try block and added final multithreaded tests to the CloudWatchLogsService class Signed-off-by: Marcos Gonzalez Mayedo --- .../sink/client/CloudWatchLogsService.java | 34 +++++++++--------- .../client/CloudWatchLogsServiceTest.java | 36 +++++++++++++++++++ 2 files changed, 53 insertions(+), 17 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index 471d045308..531d592931 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -57,7 +57,6 @@ public CloudWatchLogsService(final Buffer buffer, * @param logs - Collection of Record events which hold log data. */ public void processLogEvents(final Collection> logs) { - try { sinkStopWatch.startIfNotRunning(); for (Record log : logs) { String logString = log.getData().toJsonString(); @@ -71,24 +70,25 @@ public void processLogEvents(final Collection> logs) { long time = sinkStopWatch.getStopWatchTimeSeconds(); processLock.lock(); - int bufferSize = buffer.getBufferSize(); - int bufferEventCount = buffer.getEventCount(); - int bufferEventCountWithEvent = bufferEventCount + 1; - int bufferSizeWithAddedEvent = bufferSize + logLength; - - if ((cloudWatchLogsLimits.isGreaterThanLimitReached(time, bufferSizeWithAddedEvent, bufferEventCountWithEvent) && (bufferEventCount > 0))) { - stageLogEvents(); - addToBuffer(log, logString); - } else if (cloudWatchLogsLimits.isEqualToLimitReached(bufferSizeWithAddedEvent, bufferEventCountWithEvent)) { - addToBuffer(log, logString); - stageLogEvents(); - } else { - addToBuffer(log, logString); + try { + int bufferSize = buffer.getBufferSize(); + int bufferEventCount = buffer.getEventCount(); + int bufferEventCountWithEvent = bufferEventCount + 1; + int bufferSizeWithAddedEvent = bufferSize + logLength; + + if ((cloudWatchLogsLimits.isGreaterThanLimitReached(time, bufferSizeWithAddedEvent, bufferEventCountWithEvent) && (bufferEventCount > 0))) { + stageLogEvents(); + addToBuffer(log, logString); + } else if (cloudWatchLogsLimits.isEqualToLimitReached(bufferSizeWithAddedEvent, bufferEventCountWithEvent)) { + addToBuffer(log, logString); + stageLogEvents(); + } else { + addToBuffer(log, logString); + } + } finally { + processLock.unlock(); } } - } finally { - processLock.unlock(); - } } private void stageLogEvents() { diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java index 767d553f9a..ce9b7949c3 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java @@ -165,4 +165,40 @@ void test_buffer_access_with_large_thread_count_test() throws InterruptedExcepti verify(buffer, atLeast(LARGE_THREAD_COUNT)).getBufferedData(); } + + @Test + void test_dispatcher_access_with_small_thread_count_test() throws InterruptedException { + setUpSpyBuffer(); + cloudWatchLogsService = getSampleService(); + testThreadsProcessingLogsWithNormalSample(SMALL_THREAD_COUNT); + + verify(testDispatcher, atLeast(SMALL_THREAD_COUNT)).dispatchLogs(any(List.class), any(Collection.class)); + } + + @Test + void test_dispatcher_access_with_medium_thread_count_test() throws InterruptedException { + setUpSpyBuffer(); + cloudWatchLogsService = getSampleService(); + testThreadsProcessingLogsWithNormalSample(MEDIUM_THREAD_COUNT); + + verify(testDispatcher, atLeast(MEDIUM_THREAD_COUNT)).dispatchLogs(any(List.class), any(Collection.class)); + } + + @Test + void test_dispatcher_access_with_high_thread_count_test() throws InterruptedException { + setUpSpyBuffer(); + cloudWatchLogsService = getSampleService(); + testThreadsProcessingLogsWithNormalSample(HIGH_THREAD_COUNT); + + verify(testDispatcher, atLeast(HIGH_THREAD_COUNT)).dispatchLogs(any(List.class), any(Collection.class)); + } + + @Test + void test_dispatcher_access_with_large_thread_count_test() throws InterruptedException { + setUpSpyBuffer(); + cloudWatchLogsService = getSampleService(); + testThreadsProcessingLogsWithNormalSample(LARGE_THREAD_COUNT); + + verify(testDispatcher, atLeast(LARGE_THREAD_COUNT)).dispatchLogs(any(List.class), any(Collection.class)); + } } From 69320eca5ef1d8e4f6846d91950c5db46406ffe0 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Fri, 28 Jul 2023 09:42:23 -0700 Subject: [PATCH 39/43] Added CloudWatchLogsMetricsTest, changed upper back off time bound and scale, and refactoring changes for better code syntax (renaming, refactoring methods for conciseness, etc...) Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/buffer/InMemoryBuffer.java | 3 + .../client/CloudWatchLogsClientFactory.java | 13 +- .../sink/client/CloudWatchLogsDispatcher.java | 61 ++++---- .../sink/client/CloudWatchLogsMetrics.java | 8 +- .../sink/client/CloudWatchLogsService.java | 28 ++-- .../plugins/sink/config/ThresholdConfig.java | 4 +- .../sink/utils/CloudWatchLogsLimits.java | 27 ++-- .../plugins/sink/utils/SinkStopWatch.java | 11 +- .../buffer/InMemoryBufferFactoryTest.java | 2 +- .../sink/buffer/InMemoryBufferTest.java | 21 ++- .../CloudWatchLogsClientFactoryTest.java | 51 +++--- .../client/CloudWatchLogsDispatcherTest.java | 25 +-- .../client/CloudWatchLogsMetricsTest.java | 66 ++++++++ .../client/CloudWatchLogsServiceTest.java | 145 ++++++++---------- .../plugins/sink/client/UploaderTest.java | 34 ++-- .../plugins/sink/config/AwsConfigTest.java | 16 +- .../config/CloudWatchLogsSinkConfigTest.java | 16 +- .../sink/config/ThresholdConfigTest.java | 20 +-- .../CloudWatchLogsLimitsTest.java | 109 ------------- .../sink/utils/CloudWatchLogsLimitsTest.java | 144 +++++++++++++++++ 20 files changed, 450 insertions(+), 354 deletions(-) create mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsMetricsTest.java delete mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimitsTest.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/utils/CloudWatchLogsLimitsTest.java diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java index e3794676eb..8716915ee7 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBuffer.java @@ -35,6 +35,9 @@ public void writeEvent(final byte[] event) { @Override public byte[] popEvent() { + if (eventsBuffered.isEmpty()) { + return new byte[0]; + } bufferSize -= eventsBuffered.get(0).length; return eventsBuffered.remove(0); } diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java index a1d60aabf8..fbf3d83ba6 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactory.java @@ -13,21 +13,18 @@ import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; /** - * CwlClientFactory is in charge of reading in - * aws config parameters to return a working - * client for interfacing with - * CloudWatchLogs services. + * CwlClientFactory is in charge of reading in aws config parameters to return a working + * client for interfacing with CloudWatchLogs services. */ public final class CloudWatchLogsClientFactory { private CloudWatchLogsClientFactory() { - throw new IllegalStateException("Static Factory Class!"); } /** * Generates a CloudWatchLogs Client based on STS role ARN system credentials. - * @param awsConfig - AwsConfig specifying region, roles, and header overrides. - * @param awsCredentialsSupplier - AwsCredentialsSupplier Interface for which to create CredentialsProvider for Client config. - * @return CloudWatchLogsClient - used to interact with CloudWatch Logs services. + * @param awsConfig AwsConfig specifying region, roles, and header overrides. + * @param awsCredentialsSupplier AwsCredentialsSupplier Interface for which to create CredentialsProvider for Client config. + * @return CloudWatchLogsClient used to interact with CloudWatch Logs services. */ public static CloudWatchLogsClient createCwlClient(final AwsConfig awsConfig, final AwsCredentialsSupplier awsCredentialsSupplier) { final AwsCredentialsOptions awsCredentialsOptions = convertToCredentialOptions(awsConfig); diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java index 15e4446e6f..412407be06 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java @@ -23,19 +23,19 @@ @Builder public class CloudWatchLogsDispatcher { - private static final long UPPER_RETRY_TIME_BOUND_MILLISECONDS = 5000; - private static final float EXP_TIME_SCALE = 1.5F; + private static final long UPPER_RETRY_TIME_BOUND_MILLISECONDS = 2000; + private static final float EXP_TIME_SCALE = 1.25F; private static final Logger LOG = LoggerFactory.getLogger(CloudWatchLogsDispatcher.class); private CloudWatchLogsClient cloudWatchLogsClient; private CloudWatchLogsMetrics cloudWatchLogsMetrics; - private Executor asyncExecutor; + private Executor executor; private String logGroup; private String logStream; private int retryCount; private long backOffTimeBase; public CloudWatchLogsDispatcher(final CloudWatchLogsClient cloudWatchLogsClient, final CloudWatchLogsMetrics cloudWatchLogsMetrics, - Executor asyncExecutor, + final Executor executor, final String logGroup, final String logStream, final int retryCount, final long backOffTimeBase) { this.cloudWatchLogsClient = cloudWatchLogsClient; @@ -45,12 +45,24 @@ public CloudWatchLogsDispatcher(final CloudWatchLogsClient cloudWatchLogsClient, this.retryCount = retryCount; this.backOffTimeBase = backOffTimeBase; - this.asyncExecutor = asyncExecutor; + this.executor = executor; } + /** + * Will read in a collection of log messages in byte form and transform them into a collection of InputLogEvents. + * @param eventMessageBytes + * @return + */ public List prepareInputLogEvents(final Collection eventMessageBytes) { List logEventList = new ArrayList<>(); + /** + * Current implementation, timestamp is generated by system time during transmission. + * To properly extract timestamp we need to order the InputLogEvents. Can be done by + * refactoring buffer class with timestamp param, or adding a sorting algorithm in between + * making the PLE object (in prepareInputLogEvents). + */ + for (byte[] data : eventMessageBytes) { InputLogEvent tempLogEvent = InputLogEvent.builder() .message(new String(data, StandardCharsets.UTF_8)) @@ -62,11 +74,6 @@ public List prepareInputLogEvents(final Collection eventM return logEventList; } - /** - * Flush function to handle the flushing of logs to CloudWatchLogs services; - * @param inputLogEvents Collection of inputLogEvents to be flushed - * @param eventHandles Collection of EventHandles for events - */ public void dispatchLogs(List inputLogEvents, Collection eventHandles) { PutLogEventsRequest putLogEventsRequest = PutLogEventsRequest.builder() .logEvents(inputLogEvents) @@ -74,7 +81,8 @@ public void dispatchLogs(List inputLogEvents, Collection inputLogEvents, Collection eventHandles; - private int retryCount; - private long backOffTimeBase; + private final CloudWatchLogsClient cloudWatchLogsClient; + private final CloudWatchLogsMetrics cloudWatchLogsMetrics; + private final PutLogEventsRequest putLogEventsRequest; + private final Collection eventHandles; + private final int retryCount; + private final long backOffTimeBase; @Override public void run() { @@ -98,24 +106,22 @@ public void run() { } public void upload() { - boolean failedPost = true; - int failCounter = 0; + boolean failedToTransmit = true; + int failCount = 0; try { - while (failedPost && (failCounter < retryCount)) { + while (failedToTransmit && (failCount < retryCount)) { try { cloudWatchLogsClient.putLogEvents(putLogEventsRequest); cloudWatchLogsMetrics.increaseRequestSuccessCounter(1); - failedPost = false; - - //TODO: When a log is rejected by the service, we cannot send it, can probably push to a DLQ here. + failedToTransmit = false; } catch (CloudWatchLogsException | SdkClientException e) { - LOG.error("Service-Worker {} Failed to push logs with error: {}", Thread.currentThread().getName(), e.getMessage()); + LOG.error("Failed to push logs with error: {}", e.getMessage()); cloudWatchLogsMetrics.increaseRequestFailCounter(1); - Thread.sleep(calculateBackOffTime(backOffTimeBase, failCounter)); - LOG.warn("Service-Worker {} Trying to retransmit request... {Attempt: {} }", Thread.currentThread().getName(), (++failCounter)); + Thread.sleep(calculateBackOffTime(backOffTimeBase, failCount)); + failCount++; } } } catch (InterruptedException e) { @@ -125,7 +131,7 @@ public void upload() { } - if (failedPost) { + if (failedToTransmit) { cloudWatchLogsMetrics.increaseLogEventFailCounter(eventHandles.size()); releaseEventHandles(false, eventHandles); } else { @@ -134,7 +140,6 @@ public void upload() { } } - //TODO: Can abstract this if clients want more choice. private long calculateBackOffTime(final long backOffTimeBase, final int failCounter) { long scale = (long)Math.pow(EXP_TIME_SCALE, failCounter); diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsMetrics.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsMetrics.java index 1c2624f935..bee439e855 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsMetrics.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsMetrics.java @@ -14,10 +14,10 @@ * may refer to it. */ public class CloudWatchLogsMetrics { - public static final String CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED = "cloudWatchLogsRequestsSucceeded"; - public static final String CLOUDWATCH_LOGS_EVENTS_SUCCEEDED = "cloudWatchLogsEventsSucceeded"; - public static final String CLOUDWATCH_LOGS_EVENTS_FAILED = "cloudWatchLogsEventsFailed"; - public static final String CLOUDWATCH_LOGS_REQUESTS_FAILED = "cloudWatchLogsRequestsFailed"; + protected static final String CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED = "cloudWatchLogsRequestsSucceeded"; + protected static final String CLOUDWATCH_LOGS_EVENTS_SUCCEEDED = "cloudWatchLogsEventsSucceeded"; + protected static final String CLOUDWATCH_LOGS_EVENTS_FAILED = "cloudWatchLogsEventsFailed"; + protected static final String CLOUDWATCH_LOGS_REQUESTS_FAILED = "cloudWatchLogsRequestsFailed"; private final Counter logEventSuccessCounter; private final Counter logEventFailCounter; private final Counter requestSuccessCount; diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index 531d592931..92751ee441 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -24,10 +24,12 @@ /** * CloudWatchLogs Service encapsulates the log processing step. * It accomplishes this by: - * 1. Reading in log events. - * 2. Buffering data. - * 3. Checking for limit conditions. - * 4. Making PLE calls to CloudWatchLogs. + *
    + *
  1. Reading in log events.
  2. + *
  3. Buffering data.
  4. + *
  5. Checking for limit conditions.
  6. + *
  7. Making PLE calls to CloudWatchLogs.
  8. + *
*/ public class CloudWatchLogsService { private static final Logger LOG = LoggerFactory.getLogger(CloudWatchLogsService.class); @@ -43,7 +45,8 @@ public CloudWatchLogsService(final Buffer buffer, this.buffer = buffer; this.cloudWatchLogsLimits = cloudWatchLogsLimits; - this.bufferedEventHandles = new ArrayList<>(); + + bufferedEventHandles = new ArrayList<>(); processLock = new ReentrantLock(); sinkStopWatch = new SinkStopWatch(); @@ -53,8 +56,7 @@ public CloudWatchLogsService(final Buffer buffer, /** * Function handles the packaging of events into log events before sending a bulk request to CloudWatchLogs. - * Implements simple conditional buffer. (Sends once batch size, request size in bytes, or time limit is reached) - * @param logs - Collection of Record events which hold log data. + * @param logs Collection of Record events which hold {@link Record} */ public void processLogEvents(final Collection> logs) { sinkStopWatch.startIfNotRunning(); @@ -67,19 +69,19 @@ public void processLogEvents(final Collection> logs) { continue; } - long time = sinkStopWatch.getStopWatchTimeSeconds(); + long time = sinkStopWatch.getElapsedTimeInSeconds(); processLock.lock(); try { int bufferSize = buffer.getBufferSize(); int bufferEventCount = buffer.getEventCount(); - int bufferEventCountWithEvent = bufferEventCount + 1; - int bufferSizeWithAddedEvent = bufferSize + logLength; + int newBufferEventCount = bufferEventCount + 1; + int newBufferSizeCount = bufferSize + logLength; - if ((cloudWatchLogsLimits.isGreaterThanLimitReached(time, bufferSizeWithAddedEvent, bufferEventCountWithEvent) && (bufferEventCount > 0))) { + if ((cloudWatchLogsLimits.isGreaterThanLimitReached(time, newBufferSizeCount, newBufferEventCount) && (bufferEventCount > 0))) { stageLogEvents(); addToBuffer(log, logString); - } else if (cloudWatchLogsLimits.isEqualToLimitReached(bufferSizeWithAddedEvent, bufferEventCountWithEvent)) { + } else if (cloudWatchLogsLimits.isEqualToLimitReached(newBufferSizeCount, newBufferEventCount)) { addToBuffer(log, logString); stageLogEvents(); } else { @@ -92,7 +94,7 @@ public void processLogEvents(final Collection> logs) { } private void stageLogEvents() { - sinkStopWatch.stopAndResetStopWatch(); + sinkStopWatch.stopAndReset(); List inputLogEvents = cloudWatchLogsDispatcher.prepareInputLogEvents(buffer.getBufferedData()); cloudWatchLogsDispatcher.dispatchLogs(inputLogEvents, bufferedEventHandles); diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java index 69839f6937..b244b9a9da 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java @@ -20,7 +20,7 @@ public class ThresholdConfig { public static final int DEFAULT_RETRY_COUNT = 5; public static final int DEFAULT_LOG_SEND_INTERVAL_TIME = 60; public static final int DEFAULT_BACKOFF_TIME = 500; - public static final int CONVERT_TO_BYTES_FROM_KB = 1024; + public static final int BYTE_TO_KB_FACTOR = 1024; @JsonProperty("batch_size") @Size(min = 1, max = 10000, message = "batch_size amount should be between 1 to 10000") @@ -51,7 +51,7 @@ public int getBatchSize() { } public int getMaxEventSizeBytes() { - return maxEventSize * CONVERT_TO_BYTES_FROM_KB; + return maxEventSize * BYTE_TO_KB_FACTOR; } public int getMaxRequestSize() { diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/CloudWatchLogsLimits.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/CloudWatchLogsLimits.java index 64fdb4afcc..f7f382dd8d 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/CloudWatchLogsLimits.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/CloudWatchLogsLimits.java @@ -10,7 +10,8 @@ * PutLogEvent request to AWS. */ public class CloudWatchLogsLimits { - public static final int APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE = 26; //Size of overhead for each log event message. + // Size of overhead for each log event message. See https://docs.aws.amazon.com/AmazonCloudWatchLogs/latest/APIReference/API_PutLogEvents.html + public static final int APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE = 26; private final int maxBatchSize; private final int maxEventSizeBytes; private final int maxRequestSizeBytes; @@ -25,10 +26,10 @@ public CloudWatchLogsLimits(final int maxBatchSize, final int maxEventSizeBytes, /** * Checks to see if we exceed any of the threshold conditions. - * @param currentTime - (long) denoting the time in seconds. - * @param currentRequestSize - size of request in bytes. - * @param batchSize - size of batch in events. - * @return boolean - true if we exceed the threshold events or false otherwise. + * @param currentTime (long) denoting the time in seconds. + * @param currentRequestSize size of request in bytes. + * @param batchSize size of batch in events. + * @return boolean true if we exceed the threshold events or false otherwise. */ public boolean isGreaterThanLimitReached(final long currentTime, final int currentRequestSize, final int batchSize) { int bufferSizeWithOverhead = (currentRequestSize + ((batchSize) * APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); @@ -38,9 +39,9 @@ public boolean isGreaterThanLimitReached(final long currentTime, final int curre /** * Checks to see if we equal any of the threshold conditions. - * @param currentRequestSize - size of request in bytes. - * @param batchSize - size of batch in events. - * @return boolean - true if we equal the threshold events or false otherwise. + * @param currentRequestSize size of request in bytes. + * @param batchSize size of batch in events. + * @return boolean true if we equal the threshold events or false otherwise. */ public boolean isEqualToLimitReached(final int currentRequestSize, final int batchSize) { int bufferSizeWithOverhead = (currentRequestSize + ((batchSize) * APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); @@ -51,7 +52,7 @@ public boolean isEqualToLimitReached(final int currentRequestSize, final int bat * Checks if the interval passed in is equal to or greater * than the threshold interval for sending PutLogEvents. * @param currentTimeSeconds int denoting seconds. - * @return boolean - true if greater than or equal to logInterval, false otherwise. + * @return boolean true if greater than or equal to logInterval, false otherwise. */ private boolean isGreaterEqualToLogSendInterval(final long currentTimeSeconds) { return currentTimeSeconds >= logSendInterval; @@ -60,7 +61,7 @@ private boolean isGreaterEqualToLogSendInterval(final long currentTimeSeconds) { /** * Determines if the event size is greater than the max event size. * @param eventSize int denoting size of event. - * @return boolean - true if greater than MaxEventSize, false otherwise. + * @return boolean true if greater than MaxEventSize, false otherwise. */ public boolean isGreaterThanMaxEventSize(final int eventSize) { return (eventSize + APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE) > maxEventSizeBytes; @@ -69,7 +70,7 @@ public boolean isGreaterThanMaxEventSize(final int eventSize) { /** * Checks if the request size is greater than or equal to the current size passed in. * @param currentRequestSize int denoting size of request(Sum of PutLogEvent messages). - * @return boolean - true if greater than Max request size, smaller otherwise. + * @return boolean true if greater than Max request size, smaller otherwise. */ private boolean isGreaterThanMaxRequestSize(final int currentRequestSize) { return currentRequestSize > maxRequestSizeBytes; @@ -79,7 +80,7 @@ private boolean isGreaterThanMaxRequestSize(final int currentRequestSize) { * Checks if the current batch size is greater to the threshold * batch size. * @param batchSize int denoting the size of the batch of PutLogEvents. - * @return boolean - true if greater, false otherwise. + * @return boolean true if greater, false otherwise. */ private boolean isGreaterThanBatchSize(final int batchSize) { return batchSize > this.maxBatchSize; @@ -88,7 +89,7 @@ private boolean isGreaterThanBatchSize(final int batchSize) { /** * Checks if the request size is greater than or equal to the current size passed in. * @param currentRequestSize int denoting size of request(Sum of PutLogEvent messages). - * @return boolean - true if equal Max request size, smaller otherwise. + * @return boolean true if equal Max request size, smaller otherwise. */ private boolean isEqualMaxRequestSize(final int currentRequestSize) { return currentRequestSize == maxRequestSizeBytes; diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java index 036fa949f5..51b77a1ab6 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/utils/SinkStopWatch.java @@ -24,21 +24,24 @@ public SinkStopWatch() { stopWatchLock = new ReentrantLock(); } + /** + * Starts the stopwatch timer if not already running. + */ public void startIfNotRunning() { stopWatchLock.lock(); if (!stopWatch.isStarted()) { - startStopWatch(); + start(); } stopWatchLock.unlock(); } - public void startStopWatch() { + private void start() { stopWatchLock.lock(); stopWatch.start(); stopWatchLock.unlock(); } - public void stopAndResetStopWatch() { + public void stopAndReset() { stopWatchLock.lock(); stopWatch.stop(); stopWatch.reset(); @@ -46,7 +49,7 @@ public void stopAndResetStopWatch() { stopWatchLock.unlock(); } - public long getStopWatchTimeSeconds() { + public long getElapsedTimeInSeconds() { stopWatchLock.lock(); long time = stopWatch.getTime(TimeUnit.SECONDS); stopWatchLock.unlock(); diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBufferFactoryTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBufferFactoryTest.java index e13e87acf6..3419788aed 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBufferFactoryTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBufferFactoryTest.java @@ -11,7 +11,7 @@ import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.typeCompatibleWith; -public class InMemoryBufferFactoryTest { +class InMemoryBufferFactoryTest { @Test void check_buffer_not_null() { Buffer buffer = new InMemoryBufferFactory().getBuffer(); diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBufferTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBufferTest.java index 1b14aab052..b88b9e7826 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBufferTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/buffer/InMemoryBufferTest.java @@ -15,6 +15,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; public class InMemoryBufferTest { private static InMemoryBuffer inMemoryBuffer; @@ -45,13 +46,13 @@ int getStringJsonMessageSize() { } @Test - void check_empty_buffer() { + void GIVEN_empty_buffer_SHOULD_return_valid_event_count() { assertThat(inMemoryBuffer.getBufferSize(), equalTo(0)); assertThat(inMemoryBuffer.getEventCount(), equalTo(0)); } @Test - void check_buffer_has_right_number_of_events_test() { + void GIVEN_filled_buffer_SHOULD_return_valid_event_count() { for (Record eventToTest: getTestCollection()) { inMemoryBuffer.writeEvent(eventToTest.getData().toJsonString().getBytes()); } @@ -60,7 +61,7 @@ void check_buffer_has_right_number_of_events_test() { } @Test - void check_right_event_count_after_event_fetch_test() { + void GIVEN_filled_buffer_WHEN_pop_event_SHOULD_return_valid_event_count() { for (Record eventToTest: getTestCollection()) { inMemoryBuffer.writeEvent(eventToTest.getData().toJsonString().getBytes()); } @@ -71,7 +72,7 @@ void check_right_event_count_after_event_fetch_test() { } @Test - void check_right_buffer_size_after_event_fetch_test() { + void GIVEN_filled_buffer_WHEN_pop_event_SHOULD_return_valid_buffer_size() { for (Record eventToTest: getTestCollection()) { inMemoryBuffer.writeEvent(eventToTest.getData().toJsonString().getBytes()); } @@ -82,7 +83,7 @@ void check_right_buffer_size_after_event_fetch_test() { } @Test - void check_buffer_has_right_size_test() { + void GIVEN_filled_buffer_WHEN_get_buffer_size_SHOULD_return_valid_buffer_size() { for (Record eventToTest: getTestCollection()) { inMemoryBuffer.writeEvent(eventToTest.getData().toJsonString().getBytes()); } @@ -90,9 +91,8 @@ void check_buffer_has_right_size_test() { assertThat(inMemoryBuffer.getBufferSize(), equalTo(getStringJsonMessageSize() * TEST_COLLECTION_SIZE)); } - //TODO: Add tests for getting events. @Test - void check_if_event_matches_test() { + void GIVEN_filled_buffer_WHEN_pop_event_SHOULD_return_valid_string() { for (Record eventToTest: getTestCollection()) { inMemoryBuffer.writeEvent(eventToTest.getData().toJsonString().getBytes()); } @@ -103,4 +103,11 @@ void check_if_event_matches_test() { assertThat(new String(inMemoryBuffer.popEvent()), equalTo(getStringJsonMessage())); } } + + @Test + void GIVEN_empty_buffer_WHEN_pop_SHOULD_return_empty_byte_array() { + byte[] popResult = inMemoryBuffer.popEvent(); + + assertArrayEquals(new byte[0], popResult); + } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactoryTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactoryTest.java index 6935108a2b..32727254aa 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactoryTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsClientFactoryTest.java @@ -30,53 +30,46 @@ import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.verify; -public class CloudWatchLogsClientFactoryTest { - private AwsConfig awsConfig; - private AwsCredentialsSupplier awsCredentialsSupplier; - private AwsCredentialsOptions awsCredentialsOptions; +class CloudWatchLogsClientFactoryTest { + private AwsConfig mockAwsConfig; + private AwsCredentialsSupplier mockAwsCredentialsSupplier; + private AwsCredentialsOptions mockAwsCredentialsOptions; @BeforeEach void setUp() { - awsConfig = mock(AwsConfig.class); - awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); - awsCredentialsOptions = mock(AwsCredentialsOptions.class); - when(awsConfig.getAwsRegion()).thenReturn(Region.US_EAST_1); + mockAwsConfig = mock(AwsConfig.class); + mockAwsCredentialsSupplier = mock(AwsCredentialsSupplier.class); + mockAwsCredentialsOptions = mock(AwsCredentialsOptions.class); + when(mockAwsConfig.getAwsRegion()).thenReturn(Region.US_EAST_1); } @Test - void check_created_real_default_client_test() { - final CloudWatchLogsClient cloudWatchLogsClientToTest = CloudWatchLogsClientFactory.createCwlClient(awsConfig, awsCredentialsSupplier); + void GIVEN_default_credentials_SHOULD_return_non_null_client() { + final CloudWatchLogsClient cloudWatchLogsClientToTest = CloudWatchLogsClientFactory.createCwlClient(mockAwsConfig, mockAwsCredentialsSupplier); assertNotNull(cloudWatchLogsClientToTest); } @Test - void check_created_client_with_no_params() { - final CloudWatchLogsClient cloudWatchLogsClient = CloudWatchLogsClientFactory.createCwlClient(awsConfig, awsCredentialsSupplier); - - assertNotNull(cloudWatchLogsClient); - } - - @Test - void check_CwlClient_with_correct_inputs() { + void GIVEN_valid_credential_and_aws_parameters_SHOULD_generate_valid_provider_and_options() { final String stsRoleArn = UUID.randomUUID().toString(); final String externalId = UUID.randomUUID().toString(); final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); - when(awsConfig.getAwsStsRoleArn()).thenReturn(stsRoleArn); - when(awsConfig.getAwsStsExternalId()).thenReturn(externalId); - when(awsConfig.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); + when(mockAwsConfig.getAwsStsRoleArn()).thenReturn(stsRoleArn); + when(mockAwsConfig.getAwsStsExternalId()).thenReturn(externalId); + when(mockAwsConfig.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); final AwsCredentialsProvider expectedCredentialsProvider = mock(AwsCredentialsProvider.class); - when(awsCredentialsSupplier.getProvider(ArgumentMatchers.any())).thenReturn(expectedCredentialsProvider); + when(mockAwsCredentialsSupplier.getProvider(ArgumentMatchers.any())).thenReturn(expectedCredentialsProvider); final CloudWatchLogsClientBuilder cloudWatchLogsClientBuilder = mock(CloudWatchLogsClientBuilder.class); - when(cloudWatchLogsClientBuilder.region(awsConfig.getAwsRegion())).thenReturn(cloudWatchLogsClientBuilder); + when(cloudWatchLogsClientBuilder.region(mockAwsConfig.getAwsRegion())).thenReturn(cloudWatchLogsClientBuilder); when(cloudWatchLogsClientBuilder.credentialsProvider(ArgumentMatchers.any())).thenReturn(cloudWatchLogsClientBuilder); when(cloudWatchLogsClientBuilder.overrideConfiguration(ArgumentMatchers.any(ClientOverrideConfiguration.class))).thenReturn(cloudWatchLogsClientBuilder); try(final MockedStatic cloudWatchLogsClientMockedStatic = mockStatic(CloudWatchLogsClient.class)) { cloudWatchLogsClientMockedStatic.when(CloudWatchLogsClient::builder) .thenReturn(cloudWatchLogsClientBuilder); - CloudWatchLogsClientFactory.createCwlClient(awsConfig, awsCredentialsSupplier); + CloudWatchLogsClientFactory.createCwlClient(mockAwsConfig, mockAwsCredentialsSupplier); } final ArgumentCaptor credentialsProviderArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsProvider.class); @@ -87,12 +80,12 @@ void check_CwlClient_with_correct_inputs() { assertThat(actualProvider, equalTo(expectedCredentialsProvider)); final ArgumentCaptor credentialsOptionsArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsOptions.class); - verify(awsCredentialsSupplier).getProvider(credentialsOptionsArgumentCaptor.capture()); + verify(mockAwsCredentialsSupplier).getProvider(credentialsOptionsArgumentCaptor.capture()); final AwsCredentialsOptions actualOptions = credentialsOptionsArgumentCaptor.getValue(); - assertThat(actualOptions.getRegion(), equalTo(awsConfig.getAwsRegion())); - assertThat(actualOptions.getStsRoleArn(), equalTo(awsConfig.getAwsStsRoleArn())); - assertThat(actualOptions.getStsExternalId(), equalTo(awsConfig.getAwsStsExternalId())); - assertThat(actualOptions.getStsHeaderOverrides(), equalTo(awsConfig.getAwsStsHeaderOverrides())); + assertThat(actualOptions.getRegion(), equalTo(mockAwsConfig.getAwsRegion())); + assertThat(actualOptions.getStsRoleArn(), equalTo(mockAwsConfig.getAwsStsRoleArn())); + assertThat(actualOptions.getStsExternalId(), equalTo(mockAwsConfig.getAwsStsExternalId())); + assertThat(actualOptions.getStsHeaderOverrides(), equalTo(mockAwsConfig.getAwsStsHeaderOverrides())); } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java index 715aafb24c..7ca4ae098b 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcherTest.java @@ -24,18 +24,18 @@ class CloudWatchLogsDispatcherTest { private CloudWatchLogsDispatcher cloudWatchLogsDispatcher; - private CloudWatchLogsClient cloudWatchLogsClient; - private CloudWatchLogsMetrics cloudWatchLogsMetrics; - private Executor asyncExecutor; + private CloudWatchLogsClient mockCloudWatchLogsClient; + private CloudWatchLogsMetrics mockCloudWatchLogsMetrics; + private Executor mockExecutor; private static final String LOG_GROUP = "testGroup"; private static final String LOG_STREAM = "testStream"; private static final String TEST_STRING = "testMessage"; @BeforeEach - void setUp() throws InterruptedException { - cloudWatchLogsClient = mock(CloudWatchLogsClient.class); - cloudWatchLogsMetrics = mock(CloudWatchLogsMetrics.class); - asyncExecutor = mock(Executor.class); + void setUp() { + mockCloudWatchLogsClient = mock(CloudWatchLogsClient.class); + mockCloudWatchLogsMetrics = mock(CloudWatchLogsMetrics.class); + mockExecutor = mock(Executor.class); } Collection getSampleBufferedData() { @@ -60,9 +60,10 @@ Collection getSampleEventHandles() { } CloudWatchLogsDispatcher getCloudWatchLogsDispatcher() { - return CloudWatchLogsDispatcher.builder().cloudWatchLogsClient(cloudWatchLogsClient) - .cloudWatchLogsMetrics(cloudWatchLogsMetrics) - .asyncExecutor(asyncExecutor) + return CloudWatchLogsDispatcher.builder() + .cloudWatchLogsClient(mockCloudWatchLogsClient) + .cloudWatchLogsMetrics(mockCloudWatchLogsMetrics) + .executor(mockExecutor) .logGroup(LOG_GROUP) .logStream(LOG_STREAM) .retryCount(ThresholdConfig.DEFAULT_RETRY_COUNT) @@ -71,11 +72,11 @@ CloudWatchLogsDispatcher getCloudWatchLogsDispatcher() { } @Test - void check_execute_called_test() { + void GIVEN_valid_input_log_events_SHOULD_call_executor() { cloudWatchLogsDispatcher = getCloudWatchLogsDispatcher(); List inputLogEventList = cloudWatchLogsDispatcher.prepareInputLogEvents(getSampleBufferedData()); cloudWatchLogsDispatcher.dispatchLogs(inputLogEventList, getSampleEventHandles()); - verify(asyncExecutor, atMostOnce()).execute(any(CloudWatchLogsDispatcher.Uploader.class)); + verify(mockExecutor, atMostOnce()).execute(any(CloudWatchLogsDispatcher.Uploader.class)); } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsMetricsTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsMetricsTest.java new file mode 100644 index 0000000000..2e19e2aa54 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsMetricsTest.java @@ -0,0 +1,66 @@ +package org.opensearch.dataprepper.plugins.sink.client; + +import io.micrometer.core.instrument.Counter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.metrics.PluginMetrics; + +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +class CloudWatchLogsMetricsTest { + private CloudWatchLogsMetrics testCloudWatchLogsMetrics; + private PluginMetrics mockPluginMetrics; + private Counter mockSuccessEventCounter; + private Counter mockSuccessRequestCounter; + private Counter mockFailedEventCounter; + private Counter mockFailedRequestCounter; + + @BeforeEach + void setUp() { + mockPluginMetrics = mock(PluginMetrics.class); + mockSuccessEventCounter = mock(Counter.class); + mockSuccessRequestCounter = mock(Counter.class); + mockFailedEventCounter = mock(Counter.class); + mockFailedRequestCounter = mock(Counter.class); + + when(mockPluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_EVENTS_SUCCEEDED)).thenReturn(mockSuccessEventCounter); + when(mockPluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_REQUESTS_SUCCEEDED)).thenReturn(mockSuccessRequestCounter); + when(mockPluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_EVENTS_FAILED)).thenReturn(mockFailedEventCounter); + when(mockPluginMetrics.counter(CloudWatchLogsMetrics.CLOUDWATCH_LOGS_REQUESTS_FAILED)).thenReturn(mockFailedRequestCounter); + + testCloudWatchLogsMetrics = new CloudWatchLogsMetrics(mockPluginMetrics); + } + + @Test + void GIVEN_valid_plugin_metrics_WHEN_cloud_watch_metrics_initialized_SHOULD_not_be_null() { + assertNotNull(testCloudWatchLogsMetrics); + } + + @Test + void WHEN_increase_event_success_counter_called_THEN_event_success_counter_increase_method_should_be_called() { + testCloudWatchLogsMetrics.increaseLogEventSuccessCounter(1); + verify(mockSuccessEventCounter, times(1)).increment(1); + } + + @Test + void WHEN_increase_request_success_counter_called_THEN_request_success_counter_increase_method_should_be_called() { + testCloudWatchLogsMetrics.increaseRequestSuccessCounter(1); + verify(mockSuccessRequestCounter, times(1)).increment(1); + } + + @Test + void WHEN_increase_event_failed_counter_called_THEN_event_failed_counter_increase_method_should_be_called() { + testCloudWatchLogsMetrics.increaseLogEventFailCounter(1); + verify(mockFailedEventCounter, times(1)).increment(1); + } + + @Test + void WHEN_increase_request_failed_counter_called_THEN_request_failed_counter_increase_method_should_be_called() { + testCloudWatchLogsMetrics.increaseRequestFailCounter(1); + verify(mockFailedRequestCounter, times(1)).increment(1); + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java index ce9b7949c3..a0864d2207 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsServiceTest.java @@ -30,38 +30,35 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.never; -public class CloudWatchLogsServiceTest { - private static int SMALL_THREAD_COUNT = 50; - private static int MEDIUM_THREAD_COUNT = 100; - private static int HIGH_THREAD_COUNT = 500; - private static int LARGE_THREAD_COUNT = 1000; +class CloudWatchLogsServiceTest { + private static final int LARGE_THREAD_COUNT = 1000; private CloudWatchLogsClient mockClient; private CloudWatchLogsMetrics mockMetrics; private CloudWatchLogsService cloudWatchLogsService; - private CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig; + private CloudWatchLogsSinkConfig mockCloudWatchLogsSinkConfig; private ThresholdConfig thresholdConfig; private CloudWatchLogsLimits cloudWatchLogsLimits; private InMemoryBufferFactory inMemoryBufferFactory; private Buffer buffer; - private CloudWatchLogsDispatcher testDispatcher; + private CloudWatchLogsDispatcher mockDispatcher; @BeforeEach void setUp() { - cloudWatchLogsSinkConfig = mock(CloudWatchLogsSinkConfig.class); + mockCloudWatchLogsSinkConfig = mock(CloudWatchLogsSinkConfig.class); - thresholdConfig = new ThresholdConfig(); //Class can stay as is. + thresholdConfig = new ThresholdConfig(); cloudWatchLogsLimits = new CloudWatchLogsLimits(thresholdConfig.getBatchSize(), thresholdConfig.getMaxEventSizeBytes(), thresholdConfig.getMaxRequestSize(), thresholdConfig.getLogSendInterval()); mockClient = mock(CloudWatchLogsClient.class); mockMetrics = mock(CloudWatchLogsMetrics.class); inMemoryBufferFactory = new InMemoryBufferFactory(); - testDispatcher = mock(CloudWatchLogsDispatcher.class); + mockDispatcher = mock(CloudWatchLogsDispatcher.class); cloudWatchLogsService = new CloudWatchLogsService(buffer, - cloudWatchLogsLimits, testDispatcher); + cloudWatchLogsLimits, mockDispatcher); } - Collection> getSampleRecordsLess() { + Collection> getSampleRecordsCollectionSmall() { final ArrayList> returnCollection = new ArrayList<>(); for (int i = 0; i < 5; i++) { JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("testMessage"); @@ -73,7 +70,7 @@ Collection> getSampleRecordsLess() { return returnCollection; } - Collection> getSampleRecords() { + Collection> getSampleRecordsCollection() { final ArrayList> returnCollection = new ArrayList<>(); for (int i = 0; i < thresholdConfig.getBatchSize(); i++) { JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("testMessage"); @@ -85,6 +82,30 @@ Collection> getSampleRecords() { return returnCollection; } + Collection> getSampleRecordsOfLargerSize() { + final ArrayList> returnCollection = new ArrayList<>(); + for (int i = 0; i < thresholdConfig.getBatchSize() * 2; i++) { + JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("a".repeat((ThresholdConfig.DEFAULT_SIZE_OF_REQUEST/24))); + final EventHandle mockEventHandle = mock(EventHandle.class); + mockJacksonEvent.setEventHandle(mockEventHandle); + returnCollection.add(new Record<>(mockJacksonEvent)); + } + + return returnCollection; + } + + Collection> getSampleRecordsOfLimitSize() { + final ArrayList> returnCollection = new ArrayList<>(); + for (int i = 0; i < thresholdConfig.getBatchSize(); i++) { + JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("testMessage".repeat(ThresholdConfig.DEFAULT_EVENT_SIZE * ThresholdConfig.BYTE_TO_KB_FACTOR)); + final EventHandle mockEventHandle = mock(EventHandle.class); + mockJacksonEvent.setEventHandle(mockEventHandle); + returnCollection.add(new Record<>(mockJacksonEvent)); + } + + return returnCollection; + } + void setUpSpyBuffer() { buffer = spy(InMemoryBuffer.class); } @@ -94,29 +115,45 @@ void setUpRealBuffer() { } CloudWatchLogsService getSampleService() { - return new CloudWatchLogsService(buffer, cloudWatchLogsLimits, testDispatcher); + return new CloudWatchLogsService(buffer, cloudWatchLogsLimits, mockDispatcher); } @Test - void check_dispatcher_run_was_not_called() { + void SHOULD_not_call_dispatcher_WHEN_process_log_events_called_with_small_collection() { setUpRealBuffer(); cloudWatchLogsService = getSampleService(); - cloudWatchLogsService.processLogEvents(getSampleRecordsLess()); - verify(testDispatcher, never()).dispatchLogs(any(List.class), any(Collection.class)); + cloudWatchLogsService.processLogEvents(getSampleRecordsCollectionSmall()); + verify(mockDispatcher, never()).dispatchLogs(any(List.class), any(Collection.class)); } @Test - void check_dispatcher_run_was_called_test() throws InterruptedException { + void SHOULD_call_dispatcher_WHEN_process_log_events_called_with_limit_sized_collection() { setUpRealBuffer(); cloudWatchLogsService = getSampleService(); - cloudWatchLogsService.processLogEvents(getSampleRecords()); - verify(testDispatcher, atLeast(1)).dispatchLogs(any(List.class), any(Collection.class)); + cloudWatchLogsService.processLogEvents(getSampleRecordsCollection()); + verify(mockDispatcher, atLeast(1)).dispatchLogs(any(List.class), any(Collection.class)); + } + + @Test + void SHOULD_not_call_buffer_WHEN_process_log_events_called_with_limit_sized_records() { + setUpSpyBuffer(); + cloudWatchLogsService = getSampleService(); + cloudWatchLogsService.processLogEvents(getSampleRecordsOfLimitSize()); + verify(buffer, never()).writeEvent(any(byte[].class)); + } + + @Test + void SHOULD_call_buffer_WHEN_process_log_events_called_with_larger_sized_records() { + setUpSpyBuffer(); + cloudWatchLogsService = getSampleService(); + cloudWatchLogsService.processLogEvents(getSampleRecordsOfLargerSize()); + verify(buffer, atLeast(1)).writeEvent(any(byte[].class)); } //Multithreaded tests: - void testThreadsProcessingLogsWithNormalSample(final int numberOfThreads) throws InterruptedException { + void setUpThreadsProcessingLogsWithNormalSample(final int numberOfThreads) throws InterruptedException { Thread[] threads = new Thread[numberOfThreads]; - Collection> sampleEvents = getSampleRecords(); + Collection> sampleEvents = getSampleRecordsCollection(); for (int i = 0; i < numberOfThreads; i++) { threads[i] = new Thread(() -> { @@ -131,74 +168,20 @@ void testThreadsProcessingLogsWithNormalSample(final int numberOfThreads) throws } @Test - void test_buffer_access_with_small_thread_count_test() throws InterruptedException { - setUpSpyBuffer(); - cloudWatchLogsService = getSampleService(); - testThreadsProcessingLogsWithNormalSample(SMALL_THREAD_COUNT); - - verify(buffer, atLeast(SMALL_THREAD_COUNT)).getBufferedData(); - } - - @Test - void test_buffer_access_with_medium_thread_count_test() throws InterruptedException { - setUpSpyBuffer(); - cloudWatchLogsService = getSampleService(); - testThreadsProcessingLogsWithNormalSample(MEDIUM_THREAD_COUNT); - - verify(buffer, atLeast(MEDIUM_THREAD_COUNT)).getBufferedData(); - } - - @Test - void test_buffer_access_with_high_thread_count_test() throws InterruptedException { + void GIVEN_large_thread_count_WHEN_processing_log_events_THEN_buffer_should_be_called_large_thread_count_times() throws InterruptedException { setUpSpyBuffer(); cloudWatchLogsService = getSampleService(); - testThreadsProcessingLogsWithNormalSample(HIGH_THREAD_COUNT); - - verify(buffer, atLeast(HIGH_THREAD_COUNT)).getBufferedData(); - } - - @Test - void test_buffer_access_with_large_thread_count_test() throws InterruptedException { - setUpSpyBuffer(); - cloudWatchLogsService = getSampleService(); - testThreadsProcessingLogsWithNormalSample(LARGE_THREAD_COUNT); + setUpThreadsProcessingLogsWithNormalSample(LARGE_THREAD_COUNT); verify(buffer, atLeast(LARGE_THREAD_COUNT)).getBufferedData(); } @Test - void test_dispatcher_access_with_small_thread_count_test() throws InterruptedException { - setUpSpyBuffer(); - cloudWatchLogsService = getSampleService(); - testThreadsProcessingLogsWithNormalSample(SMALL_THREAD_COUNT); - - verify(testDispatcher, atLeast(SMALL_THREAD_COUNT)).dispatchLogs(any(List.class), any(Collection.class)); - } - - @Test - void test_dispatcher_access_with_medium_thread_count_test() throws InterruptedException { - setUpSpyBuffer(); - cloudWatchLogsService = getSampleService(); - testThreadsProcessingLogsWithNormalSample(MEDIUM_THREAD_COUNT); - - verify(testDispatcher, atLeast(MEDIUM_THREAD_COUNT)).dispatchLogs(any(List.class), any(Collection.class)); - } - - @Test - void test_dispatcher_access_with_high_thread_count_test() throws InterruptedException { - setUpSpyBuffer(); - cloudWatchLogsService = getSampleService(); - testThreadsProcessingLogsWithNormalSample(HIGH_THREAD_COUNT); - - verify(testDispatcher, atLeast(HIGH_THREAD_COUNT)).dispatchLogs(any(List.class), any(Collection.class)); - } - - @Test - void test_dispatcher_access_with_large_thread_count_test() throws InterruptedException { + void GIVEN_large_thread_count_WHEN_processing_log_events_THEN_dispatcher_should_be_called_large_thread_count_times() throws InterruptedException { setUpSpyBuffer(); cloudWatchLogsService = getSampleService(); - testThreadsProcessingLogsWithNormalSample(LARGE_THREAD_COUNT); + setUpThreadsProcessingLogsWithNormalSample(LARGE_THREAD_COUNT); - verify(testDispatcher, atLeast(LARGE_THREAD_COUNT)).dispatchLogs(any(List.class), any(Collection.class)); + verify(mockDispatcher, atLeast(LARGE_THREAD_COUNT)).dispatchLogs(any(List.class), any(Collection.class)); } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/UploaderTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/UploaderTest.java index 04ecc76bc7..56703ebc51 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/UploaderTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/client/UploaderTest.java @@ -20,13 +20,13 @@ import static org.mockito.Mockito.when; class UploaderTest { - private CloudWatchLogsClient cloudWatchLogsClient; - private CloudWatchLogsMetrics cloudWatchLogsMetrics; + private CloudWatchLogsClient mockCloudWatchLogsClient; + private CloudWatchLogsMetrics mockCloudWatchLogsMetrics; @BeforeEach void setUp() { - cloudWatchLogsClient = mock(CloudWatchLogsClient.class); - cloudWatchLogsMetrics = mock(CloudWatchLogsMetrics.class); + mockCloudWatchLogsClient = mock(CloudWatchLogsClient.class); + mockCloudWatchLogsMetrics = mock(CloudWatchLogsMetrics.class); } Collection getTestEventHandles() { @@ -45,8 +45,8 @@ PutLogEventsRequest getMockPutLogEventsRequest() { CloudWatchLogsDispatcher.Uploader getUploader() { return CloudWatchLogsDispatcher.Uploader.builder() - .cloudWatchLogsClient(cloudWatchLogsClient) - .cloudWatchLogsMetrics(cloudWatchLogsMetrics) + .cloudWatchLogsClient(mockCloudWatchLogsClient) + .cloudWatchLogsMetrics(mockCloudWatchLogsMetrics) .putLogEventsRequest(getMockPutLogEventsRequest()) .eventHandles(getTestEventHandles()) .retryCount(ThresholdConfig.DEFAULT_RETRY_COUNT) @@ -55,39 +55,39 @@ CloudWatchLogsDispatcher.Uploader getUploader() { } void establishFailingClientWithCloudWatchLogsExcept() { - when(cloudWatchLogsClient.putLogEvents(any(PutLogEventsRequest.class))).thenThrow(CloudWatchLogsException.class); + when(mockCloudWatchLogsClient.putLogEvents(any(PutLogEventsRequest.class))).thenThrow(CloudWatchLogsException.class); } void establishFailingClientWithSdkClientExcept() { - when(cloudWatchLogsClient.putLogEvents(any(PutLogEventsRequest.class))).thenThrow(SdkClientException.class); + when(mockCloudWatchLogsClient.putLogEvents(any(PutLogEventsRequest.class))).thenThrow(SdkClientException.class); } @Test - void check_successful_transmission_test() throws InterruptedException { + void GIVEN_valid_uploader_SHOULD_update_cloud_watch_logs_metrics() { CloudWatchLogsDispatcher.Uploader testUploader = getUploader(); testUploader.run(); - verify(cloudWatchLogsMetrics, atLeastOnce()).increaseRequestSuccessCounter(1); - verify(cloudWatchLogsMetrics, atLeastOnce()).increaseLogEventSuccessCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); + verify(mockCloudWatchLogsMetrics, atLeastOnce()).increaseRequestSuccessCounter(1); + verify(mockCloudWatchLogsMetrics, atLeastOnce()).increaseLogEventSuccessCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); } @Test - void check_unsuccesful_transmission_with_cloudwatchlogsexcept_test() throws InterruptedException { + void GIVEN_valid_uploader_WHEN_run_throws_cloud_watch_logs_exception_SHOULD_update_fail_counters() { establishFailingClientWithCloudWatchLogsExcept(); CloudWatchLogsDispatcher.Uploader testUploader = getUploader(); testUploader.run(); - verify(cloudWatchLogsMetrics, times(ThresholdConfig.DEFAULT_RETRY_COUNT)).increaseRequestFailCounter(1); - verify(cloudWatchLogsMetrics, atLeastOnce()).increaseLogEventFailCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); + verify(mockCloudWatchLogsMetrics, times(ThresholdConfig.DEFAULT_RETRY_COUNT)).increaseRequestFailCounter(1); + verify(mockCloudWatchLogsMetrics, atLeastOnce()).increaseLogEventFailCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); } @Test - void check_unsuccesful_transmission_with_sdkexcept_test() { + void GIVEN_valid_uploader_WHEN_run_throws_sdk_client_except_SHOULD_update_fail_counters() { establishFailingClientWithSdkClientExcept(); CloudWatchLogsDispatcher.Uploader testUploader = getUploader(); testUploader.run(); - verify(cloudWatchLogsMetrics, times(ThresholdConfig.DEFAULT_RETRY_COUNT)).increaseRequestFailCounter(1); - verify(cloudWatchLogsMetrics, atLeastOnce()).increaseLogEventFailCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); + verify(mockCloudWatchLogsMetrics, times(ThresholdConfig.DEFAULT_RETRY_COUNT)).increaseRequestFailCounter(1); + verify(mockCloudWatchLogsMetrics, atLeastOnce()).increaseLogEventFailCounter(ThresholdConfig.DEFAULT_BATCH_SIZE); } } diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfigTest.java index 225a9ed30b..edd379641c 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfigTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfigTest.java @@ -30,7 +30,7 @@ void setUp() { @ParameterizedTest @ValueSource(strings = {"us-east-1", "us-west-2", "eu-central-1"}) - void getAwsRegion_returns_Region_of(final String regionString) { + void GIVEN_valid_regions_WHEN_deserialized_SHOULD_return_regions_as_valid_strings(final String regionString) { final Region expectedRegionObject = Region.of(regionString); final Map jsonMap = Map.of("region", regionString); final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); @@ -38,14 +38,14 @@ void getAwsRegion_returns_Region_of(final String regionString) { } @Test - void getAwsRegion_returns_null_when_region_is_null() { + void GIVEN_no_region_WHEN_deserialized_SHOULD_return_region_as_null() { final Map jsonMap = Collections.emptyMap(); final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); assertThat(objectUnderTest.getAwsRegion(), nullValue()); } @Test - void getAwsStsRoleArn_returns_value_from_deserialized_JSON() { + void GIVEN_valid_sts_role_arn_WHEN_deserialized_SHOULD_return_as_string() { final String stsRoleArn = UUID.randomUUID().toString(); final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); @@ -53,14 +53,14 @@ void getAwsStsRoleArn_returns_value_from_deserialized_JSON() { } @Test - void getAwsStsRoleArn_returns_null_if_not_in_JSON() { + void GIVEN_empty_sts_role_arn_WHEN_deserialized_SHOULD_return_as_null() { final Map jsonMap = Collections.emptyMap(); final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); assertThat(objectUnderTest.getAwsStsRoleArn(), nullValue()); } @Test - void getAwsStsExternalId_returns_value_from_deserialized_JSON() { + void GIVEN_valid_aws_sts_external_id_WHEN_deserialized_SHOULD_return_as_string() { final String stsExternalId = UUID.randomUUID().toString(); final Map jsonMap = Map.of("sts_external_id", stsExternalId); final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); @@ -68,14 +68,14 @@ void getAwsStsExternalId_returns_value_from_deserialized_JSON() { } @Test - void getAwsStsExternalId_returns_null_if_not_in_JSON() { + void GIVEN_valid_aws_sts_external_id_WHEN_deserialized_SHOULD_return_as_null() { final Map jsonMap = Collections.emptyMap(); final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); assertThat(objectUnderTest.getAwsStsExternalId(), nullValue()); } @Test - void getAwsStsHeaderOverrides_returns_value_from_deserialized_JSON() { + void GIVEN_valid_aws_sts_header_overrides_WHEN_deserialized_SHOULD_return_as_string_map() { final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); final Map jsonMap = Map.of("sts_header_overrides", stsHeaderOverrides); final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); @@ -83,7 +83,7 @@ void getAwsStsHeaderOverrides_returns_value_from_deserialized_JSON() { } @Test - void getAwsStsHeaderOverrides_returns_null_if_not_in_JSON() { + void GIVEN_valid_aws_sts_header_overrides_WHEN_deserialized_SHOULD_return_as_null() { final Map jsonMap = Collections.emptyMap(); final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); assertThat(objectUnderTest.getAwsStsHeaderOverrides(), nullValue()); diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/CloudWatchLogsSinkConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/CloudWatchLogsSinkConfigTest.java index 94a6d4c81a..77ff49db8a 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/CloudWatchLogsSinkConfigTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/CloudWatchLogsSinkConfigTest.java @@ -13,7 +13,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; -public class CloudWatchLogsSinkConfigTest { +class CloudWatchLogsSinkConfigTest { private CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig; private AwsConfig awsConfig; private ThresholdConfig thresholdConfig; @@ -28,32 +28,32 @@ void setUp() { } @Test - void check_null_auth_config_test() { + void GIVEN_new_sink_config_WHEN_get_aws_config_called_SHOULD_return_null() { assertThat(new CloudWatchLogsSinkConfig().getAwsConfig(), equalTo(null)); } @Test - void check_null_threshold_config_test() { + void GIVEN_new_sink_config_WHEN_get_threshold_config_called_SHOULD_return_null() { assertThat(new CloudWatchLogsSinkConfig().getThresholdConfig(), notNullValue()); } @Test - void check_default_buffer_type_test() { + void GIVEN_new_sink_config_WHEN_get_buffer_type_called_SHOULD_return_default_buffer_type() { assertThat(new CloudWatchLogsSinkConfig().getBufferType(), equalTo(CloudWatchLogsSinkConfig.DEFAULT_BUFFER_TYPE)); } @Test - void check_null_log_group_test() { + void GIVEN_new_sink_config_WHEN_get_log_group_called_SHOULD_return_null() { assertThat(new CloudWatchLogsSinkConfig().getLogGroup(), equalTo(null)); } @Test - void check_null_log_stream_test() { + void GIVEN_new_sink_config_WHEN_get_log_stream_called_SHOULD_return_null() { assertThat(new CloudWatchLogsSinkConfig().getLogStream(), equalTo(null)); } @Test - void check_valid_log_group_and_log_stream_test() throws NoSuchFieldException, IllegalAccessException { + void GIVEN_empty_sink_config_WHEN_deserialized_from_json_SHOULD_return_valid_log_group_and_log_stream() throws NoSuchFieldException, IllegalAccessException { ReflectivelySetField.setField(cloudWatchLogsSinkConfig.getClass(), cloudWatchLogsSinkConfig, "logGroup", LOG_GROUP); ReflectivelySetField.setField(cloudWatchLogsSinkConfig.getClass(), cloudWatchLogsSinkConfig, "logStream", LOG_STREAM); @@ -62,7 +62,7 @@ void check_valid_log_group_and_log_stream_test() throws NoSuchFieldException, Il } @Test - void check_valid_sub_config_test() throws NoSuchFieldException, IllegalAccessException { + void GIVEN_empty_sink_config_WHEN_deserialized_from_json_SHOULD_return_valid_threshold_config_and_aws_config() throws NoSuchFieldException, IllegalAccessException { ReflectivelySetField.setField(cloudWatchLogsSinkConfig.getClass(), cloudWatchLogsSinkConfig, "thresholdConfig", thresholdConfig); ReflectivelySetField.setField(cloudWatchLogsSinkConfig.getClass(), cloudWatchLogsSinkConfig, "awsConfig", awsConfig); diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfigTest.java index e8324755af..59f8721d05 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfigTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfigTest.java @@ -16,7 +16,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; -public class ThresholdConfigTest { +class ThresholdConfigTest { private ObjectMapper objectMapper; @BeforeEach @@ -25,20 +25,20 @@ void setUp() { } @Test - void check_default_values() { + void GIVEN_new_threshold_config_SHOULD_return_valid_default_values() { final ThresholdConfig thresholdConfig = new ThresholdConfig(); assertThat(thresholdConfig.getBackOffTime(), equalTo(ThresholdConfig.DEFAULT_BACKOFF_TIME)); assertThat(thresholdConfig.getRetryCount(), equalTo(ThresholdConfig.DEFAULT_RETRY_COUNT)); assertThat(thresholdConfig.getBatchSize(), equalTo(ThresholdConfig.DEFAULT_BATCH_SIZE)); - assertThat(thresholdConfig.getMaxEventSizeBytes(), equalTo(ThresholdConfig.DEFAULT_EVENT_SIZE * ThresholdConfig.CONVERT_TO_BYTES_FROM_KB)); + assertThat(thresholdConfig.getMaxEventSizeBytes(), equalTo(ThresholdConfig.DEFAULT_EVENT_SIZE * ThresholdConfig.BYTE_TO_KB_FACTOR)); assertThat(thresholdConfig.getMaxRequestSize(), equalTo(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST)); assertThat(thresholdConfig.getLogSendInterval(), equalTo(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME)); } @ParameterizedTest @ValueSource(ints = {1, 10, 10000}) - void check_valid_batch_size(final int batchSize) { + void GIVEN_deserialized_threshold_config_SHOULD_return_valid_batch_size(final int batchSize) { final Map jsonMap = Map.of("batch_size", batchSize); final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); assertThat(thresholdConfigTest.getBatchSize(), equalTo(batchSize)); @@ -46,15 +46,15 @@ void check_valid_batch_size(final int batchSize) { @ParameterizedTest @ValueSource(ints = {1, 10, 256}) - void check_valid_max_event_size(final int max_event_size) { + void GIVEN_deserialized_threshold_config_SHOULD_return_valid_max_event_size(final int max_event_size) { final Map jsonMap = Map.of("max_event_size", max_event_size); final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); - assertThat(thresholdConfigTest.getMaxEventSizeBytes(), equalTo(max_event_size * ThresholdConfig.CONVERT_TO_BYTES_FROM_KB)); + assertThat(thresholdConfigTest.getMaxEventSizeBytes(), equalTo(max_event_size * ThresholdConfig.BYTE_TO_KB_FACTOR)); } @ParameterizedTest @ValueSource(ints = {1, 100, 1048576}) - void check_valid_request_size(final int max_batch_request_size) { + void GIVEN_deserialized_threshold_config_SHOULD_return_valid_max_request_size(final int max_batch_request_size) { final Map jsonMap = Map.of("max_request_size", max_batch_request_size); final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); assertThat(thresholdConfigTest.getMaxRequestSize(), equalTo(max_batch_request_size)); @@ -62,7 +62,7 @@ void check_valid_request_size(final int max_batch_request_size) { @ParameterizedTest @ValueSource(ints = {1, 10, 15}) - void check_valid_retry_count(final int retry_count) { + void GIVEN_deserialized_threshold_config_SHOULD_return_valid_max_retry_count(final int retry_count) { final Map jsonMap = Map.of("retry_count", retry_count); final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); assertThat(thresholdConfigTest.getRetryCount(), equalTo(retry_count)); @@ -70,7 +70,7 @@ void check_valid_retry_count(final int retry_count) { @ParameterizedTest @ValueSource(ints = {5, 10, 300}) - void check_valid_log_send_interval(final int log_send_interval) { + void GIVEN_deserialized_threshold_config_SHOULD_return_valid_max_log_send_interval(final int log_send_interval) { final Map jsonMap = Map.of("log_send_interval", log_send_interval); final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); assertThat(thresholdConfigTest.getLogSendInterval(), equalTo(log_send_interval)); @@ -78,7 +78,7 @@ void check_valid_log_send_interval(final int log_send_interval) { @ParameterizedTest @ValueSource(ints = {0, 100, 5000}) - void check_valid_back_off_time(final int back_off_time) { + void GIVEN_deserialized_threshold_config_SHOULD_return_valid_back_off_time(final int back_off_time) { final Map jsonMap = Map.of("back_off_time", back_off_time); final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); assertThat(thresholdConfigTest.getBackOffTime(), equalTo(back_off_time)); diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimitsTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimitsTest.java deleted file mode 100644 index 1ae09ce471..0000000000 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/push_condition/CloudWatchLogsLimitsTest.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.sink.push_condition; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; -import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; -import org.opensearch.dataprepper.plugins.sink.utils.CloudWatchLogsLimits; - -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; - -public class CloudWatchLogsLimitsTest { - private CloudWatchLogsLimits cloudWatchLogsLimits; - - @BeforeEach - void setUp() { - cloudWatchLogsLimits = new CloudWatchLogsLimits(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE * ThresholdConfig.CONVERT_TO_BYTES_FROM_KB, - ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME); - } - - @ParameterizedTest - @ValueSource(ints = {257, 560, 1000}) - void check_max_event_size_invalid(final int event_size) { - boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize(event_size * ThresholdConfig.CONVERT_TO_BYTES_FROM_KB); - assertTrue(isEventGreater); - } - - @ParameterizedTest - @ValueSource(ints = {10, 30, 50}) - void check_max_event_size_valid(final int event_size) { - boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize((event_size * ThresholdConfig.CONVERT_TO_BYTES_FROM_KB) - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); - assertFalse(isEventGreater); - } - - @ParameterizedTest - @ValueSource(ints = {60, 80, 100}) - void check_greater_than_threshold_conditions_time_true(final int send_interval) { - boolean thresholdMetTime = cloudWatchLogsLimits.isGreaterThanLimitReached(send_interval, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST,ThresholdConfig.DEFAULT_BATCH_SIZE); - assertTrue(thresholdMetTime); - } - - @ParameterizedTest - @ValueSource(ints = {1, 3, 59}) - void check_greater_than_threshold_conditions_time_false(final int send_interval) { - int validRequestSize = ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); - boolean thresholdMetTime = cloudWatchLogsLimits.isGreaterThanLimitReached(send_interval, validRequestSize ,ThresholdConfig.DEFAULT_BATCH_SIZE); - assertFalse(thresholdMetTime); - } - - @ParameterizedTest - @ValueSource(ints = {1500000, 3000000, 10000000}) - void check_greater_than_threshold_conditions_request_size_true(final int request_size) { - boolean thresholdMetRequestSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, request_size, ThresholdConfig.DEFAULT_BATCH_SIZE); - assertTrue(thresholdMetRequestSize); - } - - @ParameterizedTest - @ValueSource(ints = {10000, 250000, 500000}) - void check_greater_than_threshold_conditions_request_size_false(final int request_size) { - boolean thresholdMetRequestSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, request_size, ThresholdConfig.DEFAULT_BATCH_SIZE); - assertFalse(thresholdMetRequestSize); - } - - @ParameterizedTest - @ValueSource(ints = {26, 50, 100}) - void check_greater_than_threshold_conditions_batch_size_true(final int batch_size) { - boolean thresholdMetBatchSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME -1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, batch_size); - assertTrue(thresholdMetBatchSize); - } - - @ParameterizedTest - @ValueSource(ints = {1, 10, 25}) - void check_greater_than_threshold_conditions_batch_size_false(final int batch_size) { - int validRequestSize = ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); - boolean thresholdMetBatchSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME -1, validRequestSize, batch_size); - assertFalse(thresholdMetBatchSize); - } - - @Test - void check_equal_than_threshold_conditions_request_size_true() { - int validRequestSize = ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); - boolean thresholdMetRequestSize = cloudWatchLogsLimits.isEqualToLimitReached(validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); - assertTrue(thresholdMetRequestSize); - } - - @Test - void check_equal_than_threshold_conditions_request_size_false() { - boolean thresholdMetRequestSize = cloudWatchLogsLimits.isEqualToLimitReached(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); - assertFalse(thresholdMetRequestSize); - } - - @Test - void check_equal_than_threshold_conditions_batch_size_true() { - boolean thresholdMetBatchSize = cloudWatchLogsLimits.isEqualToLimitReached(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1, ThresholdConfig.DEFAULT_BATCH_SIZE); - assertTrue(thresholdMetBatchSize); - } - - @Test - void check_equal_than_threshold_conditions_batch_size_false() { - boolean thresholdMetBatchSize = cloudWatchLogsLimits.isEqualToLimitReached(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); - assertFalse(thresholdMetBatchSize); - } -} \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/utils/CloudWatchLogsLimitsTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/utils/CloudWatchLogsLimitsTest.java new file mode 100644 index 0000000000..1d249ccbad --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/utils/CloudWatchLogsLimitsTest.java @@ -0,0 +1,144 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.utils; + +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class CloudWatchLogsLimitsTest { + private static CloudWatchLogsLimits cloudWatchLogsLimits; + + @BeforeAll + static void setUp() { + cloudWatchLogsLimits = new CloudWatchLogsLimits(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE * ThresholdConfig.BYTE_TO_KB_FACTOR, + ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME); + } + + @Test + void GIVEN_greater_than_limit_method_WHEN_event_size_greater_than_max_event_size_THEN_return_true() { + boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize(((ThresholdConfig.DEFAULT_EVENT_SIZE + 1) * ThresholdConfig.BYTE_TO_KB_FACTOR) - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + assertTrue(isEventGreater); + } + + @Test + void GIVEN_greater_than_limit_method_WHEN_event_size_less_than_max_event_size_THEN_return_false() { + boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize(((ThresholdConfig.DEFAULT_EVENT_SIZE - 1) * ThresholdConfig.BYTE_TO_KB_FACTOR) - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + assertFalse(isEventGreater); + } + + @Test + void GIVEN_greater_than_limit_method_WHEN_event_size_equal_to_max_event_size_THEN_return_false() { + boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize((ThresholdConfig.DEFAULT_EVENT_SIZE * ThresholdConfig.BYTE_TO_KB_FACTOR) - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + assertFalse(isEventGreater); + } + + @Test + void GIVEN_greater_than_limit_method_WHEN_log_send_interval_equal_to_max_log_send_interval_THEN_return_true() { + boolean thresholdMetTime = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST,ThresholdConfig.DEFAULT_BATCH_SIZE); + assertTrue(thresholdMetTime); + } + + @Test + void SGIVEN_greater_than_limit_method_WHEN_log_send_interval_greater_than_max_log_send_interval_THEN_return_true() { + boolean thresholdMetTime = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME + 1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST,ThresholdConfig.DEFAULT_BATCH_SIZE); + assertTrue(thresholdMetTime); + } + + @Test + void GIVEN_greater_than_limit_method_WHEN_log_send_interval_less_than_max_log_send_interval_THEN_return_false() { + int validRequestSize = ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + boolean thresholdMetTime = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, validRequestSize ,ThresholdConfig.DEFAULT_BATCH_SIZE); + assertFalse(thresholdMetTime); + } + + @Test + void GIVEN_greater_than_limit_method_WHEN_request_size_greater_than_max_request_size_THEN_return_true() { + int requestSizeWithoutOverhead = (ThresholdConfig.DEFAULT_SIZE_OF_REQUEST + 1) - ThresholdConfig.DEFAULT_BATCH_SIZE * (CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + boolean thresholdMetRequestSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, requestSizeWithoutOverhead, ThresholdConfig.DEFAULT_BATCH_SIZE); + assertTrue(thresholdMetRequestSize); + } + + @Test + void GIVEN_greater_than_limit_method_WHEN_request_size_equal_to_max_request_size_THEN_return_false() { + int requestSizeWithoutOverhead = (ThresholdConfig.DEFAULT_SIZE_OF_REQUEST) - ThresholdConfig.DEFAULT_BATCH_SIZE * (CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + boolean thresholdMetRequestSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, requestSizeWithoutOverhead, ThresholdConfig.DEFAULT_BATCH_SIZE); + assertFalse(thresholdMetRequestSize); + } + + @Test + void GIVEN_greater_than_limit_method_WHEN_request_size_less_than_max_request_size_THEN_return_false() { + int requestSizeWithoutOverhead = (ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1) - ThresholdConfig.DEFAULT_BATCH_SIZE * (CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + boolean thresholdMetRequestSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, requestSizeWithoutOverhead, ThresholdConfig.DEFAULT_BATCH_SIZE); + assertFalse(thresholdMetRequestSize); + } + + @Test + void GIVEN_greater_than_limit_method_WHEN_batch_size_greater_than_max_batch_size_THEN_return_true() { + int requestSizeWithoutOverhead = (ThresholdConfig.DEFAULT_SIZE_OF_REQUEST) - ThresholdConfig.DEFAULT_BATCH_SIZE * (CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + boolean thresholdMetBatchSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME -1, requestSizeWithoutOverhead, ThresholdConfig.DEFAULT_BATCH_SIZE + 1); + assertTrue(thresholdMetBatchSize); + } + + @Test + void GIVEN_greater_than_limit_method_WHEN_batch_size_equal_to_max_batch_size_THEN_return_false() { + int validRequestSize = ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + boolean thresholdMetBatchSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME -1, validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE); + assertFalse(thresholdMetBatchSize); + } + + @Test + void GIVEN_greater_than_limit_method_WHEN_batch_size_less_than_max_batch_size_THEN_return_false() { + int validRequestSize = ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + boolean thresholdMetBatchSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME -1, validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); + assertFalse(thresholdMetBatchSize); + } + + @Test + void GIVEN_equal_to_limit_method_WHEN_request_size_equal_to_max_batch_size_THEN_return_true() { + int validRequestSize = ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + boolean thresholdMetRequestSize = cloudWatchLogsLimits.isEqualToLimitReached(validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE); + assertTrue(thresholdMetRequestSize); + } + + @Test + void GIVEN_equal_to_limit_method_WHEN_request_size_greater_than_max_batch_size_THEN_return_false() { + int validRequestSize = ((ThresholdConfig.DEFAULT_SIZE_OF_REQUEST + 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + boolean thresholdMetRequestSize = cloudWatchLogsLimits.isEqualToLimitReached(validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); + assertFalse(thresholdMetRequestSize); + } + + @Test + void GIVEN_equal_to_limit_method_WHEN_request_size_less_than_max_batch_size_THEN_return_false() { + int validRequestSize = ((ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + boolean thresholdMetRequestSize = cloudWatchLogsLimits.isEqualToLimitReached(validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); + assertFalse(thresholdMetRequestSize); + } + + @Test + void GIVEN_equal_to_limit_method_WHEN_batch_size_equal_to_max_batch_size_THEN_return_true() { + int validRequestSize = ((ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + boolean thresholdMetBatchSize = cloudWatchLogsLimits.isEqualToLimitReached(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, ThresholdConfig.DEFAULT_BATCH_SIZE); + assertTrue(thresholdMetBatchSize); + } + + @Test + void GIVEN_equal_to_limit_method_WHEN_batch_size_greater_than_max_batch_size_THEN_return_false() { + int validRequestSize = ((ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + boolean thresholdMetBatchSize = cloudWatchLogsLimits.isEqualToLimitReached(validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE + 1); + assertFalse(thresholdMetBatchSize); + } + + @Test + void GIVEN_equal_to_limit_method_WHEN_batch_size_less_than_max_batch_size_THEN_return_false() { + int validRequestSize = ((ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + boolean thresholdMetBatchSize = cloudWatchLogsLimits.isEqualToLimitReached(validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); + assertFalse(thresholdMetBatchSize); + } +} \ No newline at end of file From 993cbd083099a7be213053f6539e4ecad9064492 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Fri, 28 Jul 2023 13:15:08 -0700 Subject: [PATCH 40/43] Added changes to javadoc Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/client/CloudWatchLogsDispatcher.java | 4 ++-- .../plugins/sink/client/CloudWatchLogsService.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java index 412407be06..7266c37212 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java @@ -50,8 +50,8 @@ public CloudWatchLogsDispatcher(final CloudWatchLogsClient cloudWatchLogsClient, /** * Will read in a collection of log messages in byte form and transform them into a collection of InputLogEvents. - * @param eventMessageBytes - * @return + * @param eventMessageBytes Collection of byte arrays holding event messages. + * @return List of InputLogEvents holding the wrapped event messages. */ public List prepareInputLogEvents(final Collection eventMessageBytes) { List logEventList = new ArrayList<>(); diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java index 92751ee441..38d1b32f9a 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsService.java @@ -56,7 +56,7 @@ public CloudWatchLogsService(final Buffer buffer, /** * Function handles the packaging of events into log events before sending a bulk request to CloudWatchLogs. - * @param logs Collection of Record events which hold {@link Record} + * @param logs Collection of Record events. */ public void processLogEvents(final Collection> logs) { sinkStopWatch.startIfNotRunning(); From 2b18115609c576f2e7cc754795651be2aa4b6353 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo <95880281+MaGonzalMayedo@users.noreply.github.com> Date: Fri, 28 Jul 2023 14:40:56 -0700 Subject: [PATCH 41/43] Update data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java Co-authored-by: Mark Kuhn Signed-off-by: Marcos Gonzalez Mayedo <95880281+MaGonzalMayedo@users.noreply.github.com> --- .../plugins/sink/client/CloudWatchLogsDispatcher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java index 7266c37212..d4affea31c 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java @@ -57,7 +57,7 @@ public List prepareInputLogEvents(final Collection eventM List logEventList = new ArrayList<>(); /** - * Current implementation, timestamp is generated by system time during transmission. + * In the current implementation, the timestamp is generated during transmission. * To properly extract timestamp we need to order the InputLogEvents. Can be done by * refactoring buffer class with timestamp param, or adding a sorting algorithm in between * making the PLE object (in prepareInputLogEvents). From eee6197d0e34ea8ad6e00a24f9ec643bc0939c38 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Fri, 28 Jul 2023 14:46:52 -0700 Subject: [PATCH 42/43] Fixed comment on CloudWatchLogsDispatcher Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/client/CloudWatchLogsDispatcher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java index d4affea31c..58de7c7068 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/client/CloudWatchLogsDispatcher.java @@ -125,7 +125,7 @@ public void upload() { } } } catch (InterruptedException e) { - LOG.warn("Got interrupted while waiting!"); + LOG.warn("Uploader Thread got interrupted during retransmission with exception: {}", e.getMessage()); //TODO: Push to DLQ. Thread.currentThread().interrupt(); } From de0621fa7c65632f255601f625944e089fb0b142 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo Date: Fri, 28 Jul 2023 16:00:36 -0700 Subject: [PATCH 43/43] Added CloudWatchLogsSink and CloudWatchLogsSinkTest Signed-off-by: Marcos Gonzalez Mayedo --- .../plugins/sink/CloudWatchLogsSink.java | 91 +++++++++++++++++++ .../plugins/sink/CloudWatchLogsSinkTest.java | 57 ++++++++++++ 2 files changed, 148 insertions(+) create mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/CloudWatchLogsSink.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/CloudWatchLogsSinkTest.java diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/CloudWatchLogsSink.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/CloudWatchLogsSink.java new file mode 100644 index 0000000000..14be97d38d --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/CloudWatchLogsSink.java @@ -0,0 +1,91 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +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.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.AbstractSink; +import org.opensearch.dataprepper.model.sink.Sink; +import org.opensearch.dataprepper.plugins.sink.buffer.Buffer; +import org.opensearch.dataprepper.plugins.sink.buffer.BufferFactory; +import org.opensearch.dataprepper.plugins.sink.buffer.InMemoryBufferFactory; +import org.opensearch.dataprepper.plugins.sink.client.CloudWatchLogsDispatcher; +import org.opensearch.dataprepper.plugins.sink.client.CloudWatchLogsMetrics; +import org.opensearch.dataprepper.plugins.sink.client.CloudWatchLogsService; +import org.opensearch.dataprepper.plugins.sink.client.CloudWatchLogsClientFactory; +import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; +import org.opensearch.dataprepper.plugins.sink.config.CloudWatchLogsSinkConfig; +import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; +import org.opensearch.dataprepper.plugins.sink.utils.CloudWatchLogsLimits; +import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; + +import java.util.Collection; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; + +@DataPrepperPlugin(name = "cloudwatchlogs-sink", pluginType = Sink.class, pluginConfigurationType = CloudWatchLogsSinkConfig.class) +public class CloudWatchLogsSink extends AbstractSink> { + private final CloudWatchLogsService cloudWatchLogsService; + private boolean isInitialized; + @DataPrepperPluginConstructor + public CloudWatchLogsSink(final PluginSetting pluginSetting, + final PluginMetrics pluginMetrics, + final CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig, + final AwsCredentialsSupplier awsCredentialsSupplier) { + super(pluginSetting); + + AwsConfig awsConfig = cloudWatchLogsSinkConfig.getAwsConfig(); + ThresholdConfig thresholdConfig = cloudWatchLogsSinkConfig.getThresholdConfig(); + + CloudWatchLogsMetrics cloudWatchLogsMetrics = new CloudWatchLogsMetrics(pluginMetrics); + CloudWatchLogsLimits cloudWatchLogsLimits = new CloudWatchLogsLimits(thresholdConfig.getBatchSize(), + thresholdConfig.getMaxEventSizeBytes(), + thresholdConfig.getMaxRequestSize(),thresholdConfig.getLogSendInterval()); + + CloudWatchLogsClient cloudWatchLogsClient = CloudWatchLogsClientFactory.createCwlClient(awsConfig, awsCredentialsSupplier); + BufferFactory bufferFactory = new InMemoryBufferFactory(); + + Executor executor = Executors.newCachedThreadPool(); + + CloudWatchLogsDispatcher cloudWatchLogsDispatcher = CloudWatchLogsDispatcher.builder() + .cloudWatchLogsClient(cloudWatchLogsClient) + .cloudWatchLogsMetrics(cloudWatchLogsMetrics) + .logGroup(cloudWatchLogsSinkConfig.getLogGroup()) + .logStream(cloudWatchLogsSinkConfig.getLogStream()) + .backOffTimeBase(thresholdConfig.getBackOffTime()) + .retryCount(thresholdConfig.getRetryCount()) + .executor(executor) + .build(); + + Buffer buffer = bufferFactory.getBuffer(); + + cloudWatchLogsService = new CloudWatchLogsService(buffer, cloudWatchLogsLimits, cloudWatchLogsDispatcher); + } + + @Override + public void doInitialize() { + isInitialized = true; + } + + @Override + public void doOutput(Collection> records) { + if (records.isEmpty()) { + return; + } + + cloudWatchLogsService.processLogEvents(records); + } + + @Override + public boolean isReady() { + return isInitialized; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/CloudWatchLogsSinkTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/CloudWatchLogsSinkTest.java new file mode 100644 index 0000000000..1ac889a3c2 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/CloudWatchLogsSinkTest.java @@ -0,0 +1,57 @@ +package org.opensearch.dataprepper.plugins.sink; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.plugins.sink.client.CloudWatchLogsMetrics; +import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; +import org.opensearch.dataprepper.plugins.sink.config.CloudWatchLogsSinkConfig; +import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class CloudWatchLogsSinkTest { + private CloudWatchLogsSink testCloudWatchSink; + private PluginSetting mockPluginSetting; + private PluginMetrics mockPluginMetrics; + private CloudWatchLogsSinkConfig mockCloudWatchLogsSinkConfig; + private AwsCredentialsSupplier mockCredentialSupplier; + private AwsConfig mockAwsConfig; + private ThresholdConfig thresholdConfig; + private CloudWatchLogsMetrics mockCloudWatchLogsMetrics; + private static String TEST_LOG_GROUP = "testLogGroup"; + private static String TEST_LOG_STREAM= "testLogStream"; + private static String TEST_PLUGIN_NAME = "testPluginName"; + private static String TEST_PIPELINE_NAME = "testPipelineName"; + @BeforeEach + void setUp() { + mockPluginSetting = mock(PluginSetting.class); + mockPluginMetrics = mock(PluginMetrics.class); + mockCloudWatchLogsSinkConfig = mock(CloudWatchLogsSinkConfig.class); + mockCredentialSupplier = mock(AwsCredentialsSupplier.class); + mockAwsConfig = mock(AwsConfig.class); + thresholdConfig = new ThresholdConfig(); + mockCloudWatchLogsMetrics = mock(CloudWatchLogsMetrics.class); + + when(mockCloudWatchLogsSinkConfig.getAwsConfig()).thenReturn(mockAwsConfig); + when(mockCloudWatchLogsSinkConfig.getThresholdConfig()).thenReturn(thresholdConfig); + when(mockCloudWatchLogsSinkConfig.getLogGroup()).thenReturn(TEST_LOG_GROUP); + when(mockCloudWatchLogsSinkConfig.getLogStream()).thenReturn(TEST_LOG_STREAM); + + when(mockPluginSetting.getName()).thenReturn(TEST_PLUGIN_NAME); + when(mockPluginSetting.getPipelineName()).thenReturn(TEST_PIPELINE_NAME); + + testCloudWatchSink = new CloudWatchLogsSink(mockPluginSetting, mockPluginMetrics, mockCloudWatchLogsSinkConfig, + mockCredentialSupplier); + } + + @Test + void WHEN_sink_is_initialized_THEN_sink_is_ready_returns_true() { + testCloudWatchSink.doInitialize(); + assertTrue(testCloudWatchSink.isReady()); + } +}