From a166841739acdeabb6e9ca084dbeceefcbc39e1e Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Wed, 2 Aug 2023 19:53:46 +0530 Subject: [PATCH 01/18] Prometheus Sink draft code for issue #1744. Signed-off-by: mallikagogoi7 --- .../prometheus-sink/build.gradle | 29 ++ .../AwsRequestSigningApacheInterceptor.java | 242 ++++++++++ .../FailedHttpResponseInterceptor.java | 39 ++ .../sink/prometheus/HttpEndPointResponse.java | 46 ++ .../prometheus/OAuthAccessTokenManager.java | 85 ++++ .../sink/prometheus/PrometheusSink.java | 124 +++++ .../CertificateProviderFactory.java | 76 +++ .../HttpClientSSLConnectionManager.java | 83 ++++ .../configuration/AuthTypeOptions.java | 34 ++ .../configuration/AuthenticationOptions.java | 20 + .../AwsAuthenticationOptions.java | 45 ++ .../configuration/BasicAuthCredentials.java | 21 + .../configuration/BearerTokenOptions.java | 57 +++ .../configuration/HTTPMethodOptions.java | 33 ++ .../PrometheusSinkConfiguration.java | 297 ++++++++++++ .../sink/prometheus/dlq/DlqPushHandler.java | 134 ++++++ .../sink/prometheus/dlq/FailedDlqData.java | 65 +++ .../BasicAuthPrometheusSinkHandler.java | 44 ++ .../BearerTokenAuthPrometheusSinkHandler.java | 51 ++ .../prometheus/handler/HttpAuthOptions.java | 111 +++++ .../MultiAuthPrometheusSinkHandler.java | 19 + .../service/PrometheusSinkAwsService.java | 40 ++ .../service/PrometheusSinkService.java | 434 ++++++++++++++++++ .../prometheus/util/PrometheusSinkUtil.java | 34 ++ settings.gradle | 1 + 25 files changed, 2164 insertions(+) create mode 100644 data-prepper-plugins/prometheus-sink/build.gradle create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/AwsRequestSigningApacheInterceptor.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/HttpEndPointResponse.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactory.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthTypeOptions.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptions.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BasicAuthCredentials.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BearerTokenOptions.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/HTTPMethodOptions.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandler.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/FailedDlqData.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BasicAuthPrometheusSinkHandler.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BearerTokenAuthPrometheusSinkHandler.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/HttpAuthOptions.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/MultiAuthPrometheusSinkHandler.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/util/PrometheusSinkUtil.java diff --git a/data-prepper-plugins/prometheus-sink/build.gradle b/data-prepper-plugins/prometheus-sink/build.gradle new file mode 100644 index 0000000000..4ed84d3aa8 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/build.gradle @@ -0,0 +1,29 @@ + +dependencies { + implementation project(':data-prepper-api') + implementation project(path: ':data-prepper-plugins:common') + implementation 'com.arpnetworking.metrics:prometheus-remote-protocol:1.0.1' + implementation project(':data-prepper-plugins:aws-plugin-api') + implementation 'io.micrometer:micrometer-core' + implementation 'com.fasterxml.jackson.core:jackson-core' + implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation 'org.apache.commons:commons-compress:1.21' + implementation project(':data-prepper-plugins:common') + implementation 'com.fasterxml.jackson.core:jackson-core' + implementation 'org.hibernate.validator:hibernate-validator:7.0.5.Final' + implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' + implementation 'software.amazon.awssdk:auth' + implementation 'org.apache.commons:commons-lang3:3.12.0' + implementation project(':data-prepper-plugins:failures-common') + implementation 'org.apache.httpcomponents.client5:httpclient5:5.2' + implementation 'org.xerial.snappy:snappy-java:1.1.10.1' + implementation 'software.amazon.awssdk:s3' + implementation 'software.amazon.awssdk:apache-client' + implementation 'software.amazon.awssdk:sts' + implementation 'software.amazon.awssdk:acm' + testImplementation project(':data-prepper-test-common') +} + +test { + useJUnitPlatform() +} \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/AwsRequestSigningApacheInterceptor.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/AwsRequestSigningApacheInterceptor.java new file mode 100644 index 0000000000..11121ae79d --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/AwsRequestSigningApacheInterceptor.java @@ -0,0 +1,242 @@ +/* + * Copyright OpenSearch Contributors. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). You may not use this file except in compliance with + * the License. A copy of the License is located at + * + * http://aws.amazon.com/apache2.0 + * + * or in the "license" file accompanying this file. This file is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions + * and limitations under the License. + */ +package org.opensearch.dataprepper.plugins.sink.prometheus; + +import org.apache.hc.core5.http.HttpRequest; +import org.apache.hc.core5.http.ClassicHttpRequest; +import org.apache.hc.core5.http.EntityDetails; +import org.apache.hc.core5.http.HttpEntity; +import org.apache.hc.core5.http.Header; +import org.apache.hc.core5.http.NameValuePair; +import org.apache.hc.core5.http.ContentType; +import org.apache.hc.core5.http.HttpHost; +import org.apache.hc.core5.http.HttpRequestInterceptor; +import org.apache.hc.core5.http.io.entity.BasicHttpEntity; +import org.apache.hc.core5.http.message.BasicHeader; +import org.apache.hc.core5.http.protocol.HttpContext; +import org.apache.hc.core5.net.URIBuilder; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.signer.AwsSignerExecutionAttribute; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.core.signer.Signer; +import software.amazon.awssdk.http.SdkHttpFullRequest; +import software.amazon.awssdk.http.SdkHttpMethod; +import software.amazon.awssdk.regions.Region; + +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.TreeMap; + +import static org.apache.http.protocol.HttpCoreContext.HTTP_TARGET_HOST; + +/** + * An {@link HttpRequestInterceptor} that signs requests using any AWS {@link Signer} + * and {@link AwsCredentialsProvider}. + */ +public final class AwsRequestSigningApacheInterceptor implements HttpRequestInterceptor { + + /** + * Constant to check content-length + */ + private static final String CONTENT_LENGTH = "content-length"; + /** + * Constant to check Zero content length + */ + private static final String ZERO_CONTENT_LENGTH = "0"; + /** + * Constant to check if host is the endpoint + */ + private static final String HOST = "host"; + + /** + * The service that we're connecting to. + */ + private final String service; + + /** + * The particular signer implementation. + */ + private final Signer signer; + + /** + * The source of AWS credentials for signing. + */ + private final AwsCredentialsProvider awsCredentialsProvider; + + /** + * The region signing region. + */ + private final Region region; + + /** + * + * @param service service that we're connecting to + * @param signer particular signer implementation + * @param awsCredentialsProvider source of AWS credentials for signing + * @param region signing region + */ + public AwsRequestSigningApacheInterceptor(final String service, + final Signer signer, + final AwsCredentialsProvider awsCredentialsProvider, + final Region region) { + this.service = Objects.requireNonNull(service); + this.signer = Objects.requireNonNull(signer); + this.awsCredentialsProvider = Objects.requireNonNull(awsCredentialsProvider); + this.region = Objects.requireNonNull(region); + } + + /** + * + * @param service service that we're connecting to + * @param signer particular signer implementation + * @param awsCredentialsProvider source of AWS credentials for signing + * @param region signing region + */ + public AwsRequestSigningApacheInterceptor(final String service, + final Signer signer, + final AwsCredentialsProvider awsCredentialsProvider, + final String region) { + this(service, signer, awsCredentialsProvider, Region.of(region)); + } + + /** + * {@inheritDoc} + */ + @Override + public void process(final HttpRequest request, final EntityDetails entity, final HttpContext context) + throws IOException { + URIBuilder uriBuilder; + try { + uriBuilder = new URIBuilder(request.getUri()); + } catch (URISyntaxException e) { + throw new IOException("Invalid URI", e); + } + + // Copy Apache HttpRequest to AWS Request + SdkHttpFullRequest.Builder requestBuilder = SdkHttpFullRequest.builder() + .method(SdkHttpMethod.fromValue(request.getMethod())) + .uri(buildUri(context, uriBuilder)); + + if (request instanceof ClassicHttpRequest) { + ClassicHttpRequest classicHttpRequest = + (ClassicHttpRequest) request; + if (classicHttpRequest.getEntity() != null) { + InputStream content = classicHttpRequest.getEntity().getContent(); + requestBuilder.contentStreamProvider(() -> content); + } + } + requestBuilder.rawQueryParameters(nvpToMapParams(uriBuilder.getQueryParams())); + requestBuilder.headers(headerArrayToMap(request.getHeaders())); + + ExecutionAttributes attributes = new ExecutionAttributes(); + attributes.putAttribute(AwsSignerExecutionAttribute.AWS_CREDENTIALS, awsCredentialsProvider.resolveCredentials()); + attributes.putAttribute(AwsSignerExecutionAttribute.SERVICE_SIGNING_NAME, service); + attributes.putAttribute(AwsSignerExecutionAttribute.SIGNING_REGION, region); + + // Sign it + SdkHttpFullRequest signedRequest = signer.sign(requestBuilder.build(), attributes); + + // Now copy everything back + request.setHeaders(mapToHeaderArray(signedRequest.headers())); + if (request instanceof ClassicHttpRequest) { + ClassicHttpRequest classicHttpRequest = + (ClassicHttpRequest) request; + if (classicHttpRequest.getEntity() != null) { + HttpEntity basicHttpEntity = new BasicHttpEntity(signedRequest.contentStreamProvider() + .orElseThrow(() -> new IllegalStateException("There must be content")) + .newStream(), ContentType.APPLICATION_JSON); + classicHttpRequest.setEntity(basicHttpEntity); + } + } + } + + private URI buildUri(final HttpContext context, URIBuilder uriBuilder) throws IOException { + try { + HttpHost host = (HttpHost) context.getAttribute(HTTP_TARGET_HOST); + + if (host != null) { + uriBuilder.setHost(host.getHostName()); + uriBuilder.setScheme(host.getSchemeName()); + uriBuilder.setPort(host.getPort()); + } + + return uriBuilder.build(); + } catch (URISyntaxException e) { + throw new IOException("Invalid URI", e); + } + } + + /** + * + * @param params list of HTTP query params as NameValuePairs + * @return a multimap of HTTP query params + */ + private static Map> nvpToMapParams(final List params) { + Map> parameterMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER); + for (NameValuePair nvp : params) { + List argsList = + parameterMap.computeIfAbsent(nvp.getName(), k -> new ArrayList<>()); + argsList.add(nvp.getValue()); + } + return parameterMap; + } + + /** + * @param headers modelled Header objects + * @return a Map of header entries + */ + private static Map> headerArrayToMap(final Header[] headers) { + Map> headersMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER); + for (Header header : headers) { + if (!skipHeader(header)) { + headersMap.put(header.getName(), headersMap + .getOrDefault(header.getName(), + new LinkedList<>(Collections.singletonList(header.getValue())))); + } + } + return headersMap; + } + + /** + * @param header header line to check + * @return true if the given header should be excluded when signing + */ + private static boolean skipHeader(final Header header) { + return (CONTENT_LENGTH.equalsIgnoreCase(header.getName()) + && ZERO_CONTENT_LENGTH.equals(header.getValue())) // Strip Content-Length: 0 + || HOST.equalsIgnoreCase(header.getName()); // Host comes from endpoint + } + + /** + * @param mapHeaders Map of header entries + * @return modelled Header objects + */ + private static Header[] mapToHeaderArray(final Map> mapHeaders) { + Header[] headers = new Header[mapHeaders.size()]; + int i = 0; + for (Map.Entry> headerEntry : mapHeaders.entrySet()) { + for (String value : headerEntry.getValue()) { + headers[i++] = new BasicHeader(headerEntry.getKey(), value); + } + } + return headers; + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java new file mode 100644 index 0000000000..f15709b942 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java @@ -0,0 +1,39 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus; + +import org.apache.hc.core5.http.EntityDetails; +import org.apache.hc.core5.http.HttpResponse; +import org.apache.hc.core5.http.HttpResponseInterceptor; +import org.apache.hc.core5.http.protocol.HttpContext; + +import java.io.IOException; + +public class FailedHttpResponseInterceptor implements HttpResponseInterceptor { + + public static final int ERROR_CODE_500 = 500; + + public static final int ERROR_CODE_400 = 400; + + public static final int ERROR_CODE_404 = 404; + + public static final int ERROR_CODE_501 = 501; + + private final String url; + + public FailedHttpResponseInterceptor(final String url){ + this.url = url; + } + + @Override + public void process(HttpResponse response, EntityDetails entity, HttpContext context) throws IOException { + if (response.getCode() == ERROR_CODE_500 || + response.getCode() == ERROR_CODE_400 || + response.getCode() == ERROR_CODE_404 || + response.getCode() == ERROR_CODE_501) { + throw new IOException(String.format("url: %s , status code: %s", url,response.getCode())); + } + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/HttpEndPointResponse.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/HttpEndPointResponse.java new file mode 100644 index 0000000000..346ff80d38 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/HttpEndPointResponse.java @@ -0,0 +1,46 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus; + +public class HttpEndPointResponse { + private String url; + private int statusCode; + private String errorMessage; + + public HttpEndPointResponse(final String url, + final int statusCode, + final String errorMessage) { + this.url = url; + this.statusCode = statusCode; + this.errorMessage = errorMessage; + } + + public HttpEndPointResponse(final String url, + final int statusCode) { + this.url = url; + this.statusCode = statusCode; + } + + public String getUrl() { + return url; + } + + public int getStatusCode() { + return statusCode; + } + + public String getErrorMessage() { + return errorMessage; + } + + @Override + public String toString() { + return "{" + + "url='" + url + '\'' + + ", statusCode=" + statusCode + + ", errorMessage='" + errorMessage + '\'' + + '}'; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java new file mode 100644 index 0000000000..2f0de4e2e8 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java @@ -0,0 +1,85 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.hc.client5.http.classic.methods.HttpPost; +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; +import org.apache.hc.core5.http.ClassicHttpResponse; +import org.apache.hc.core5.http.ContentType; +import org.apache.hc.core5.http.HttpHeaders; +import org.apache.hc.core5.http.io.entity.StringEntity; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.BearerTokenOptions; + +import java.io.IOException; +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Base64; +import java.util.Map; + +public class OAuthAccessTokenManager { + + public static final String BASIC = "Basic "; + + public static final String BEARER = "Bearer "; + + public static final String APPLICATION_X_WWW_FORM_URLENCODED = "application/x-www-form-urlencoded"; + public static final String EXP = "exp"; + public static final String ACCESS_TOKEN = "access_token"; + + public static final String REFRESH_TOKEN = "refresh_token"; + + private final ObjectMapper objectMapper; + + private HttpClientBuilder httpClientBuilder; + + + public OAuthAccessTokenManager(final HttpClientBuilder httpClientBuilder){ + this.httpClientBuilder = httpClientBuilder; + this.objectMapper = new ObjectMapper(); + } + + public String getAccessToken(final BearerTokenOptions bearerTokenOptions) { + HttpPost request = new HttpPost(bearerTokenOptions.getTokenURL()); + request.setHeader(HttpHeaders.CONTENT_TYPE, APPLICATION_X_WWW_FORM_URLENCODED); + request.setHeader(HttpHeaders.AUTHORIZATION, BASIC + base64Encode(bearerTokenOptions.getClientId() + ":" + bearerTokenOptions.getClientSecret())); + String requestBody = "grant_type=" + bearerTokenOptions.getGrantType() +"&refresh_token"+bearerTokenOptions.getRefreshToken()+"&scope=" + bearerTokenOptions.getScope(); + request.setEntity(new StringEntity(requestBody, ContentType.APPLICATION_FORM_URLENCODED)); + Map accessTokenMap; + try { + ClassicHttpResponse response = (ClassicHttpResponse)httpClientBuilder.build().execute(request); + accessTokenMap = objectMapper.readValue(response.getEntity().getContent(),Map.class); + } catch (IOException e) { + throw new RuntimeException(e); + } + bearerTokenOptions.setRefreshToken(accessTokenMap.get(REFRESH_TOKEN)); + return BEARER + accessTokenMap.get(ACCESS_TOKEN); + } + + private static String base64Encode(String value) { + return java.util.Base64.getEncoder().encodeToString(value.getBytes()); + } + + public boolean isTokenExpired(final String token){ + Base64.Decoder decoder = Base64.getUrlDecoder(); + String[] chunks = token.substring(6).split("\\."); + final Map tokenDetails; + try { + ObjectMapper objectMapper = new ObjectMapper(); + tokenDetails = objectMapper.readValue(new String(decoder.decode(chunks[1])), Map.class); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + final String expTime = String.valueOf(tokenDetails.get(EXP)); + OffsetDateTime accessTokenExpTimeStamp = Instant.ofEpochMilli(Long.valueOf(expTime ) * 1000l).atOffset(ZoneOffset.UTC); + final Instant systemCurrentTimeStamp = Instant.now().atOffset(ZoneOffset.UTC).toInstant(); + if(systemCurrentTimeStamp.compareTo(accessTokenExpTimeStamp.toInstant())>=0) { + return true; + } + return false; + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java new file mode 100644 index 0000000000..5d9581f991 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java @@ -0,0 +1,124 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus; + +import org.apache.hc.client5.http.HttpRequestRetryStrategy; +import org.apache.hc.client5.http.impl.DefaultHttpRequestRetryStrategy; +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; +import org.apache.hc.client5.http.impl.classic.HttpClients; +import org.apache.hc.core5.util.TimeValue; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +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.accumulator.BufferFactory; +import org.opensearch.dataprepper.plugins.accumulator.BufferTypeOptions; +import org.opensearch.dataprepper.plugins.accumulator.InMemoryBufferFactory; +import org.opensearch.dataprepper.plugins.accumulator.LocalFileBufferFactory; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; +import org.opensearch.dataprepper.plugins.sink.prometheus.dlq.DlqPushHandler; +import org.opensearch.dataprepper.plugins.sink.prometheus.service.PrometheusSinkAwsService; +import org.opensearch.dataprepper.plugins.sink.prometheus.service.PrometheusSinkService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; + +@DataPrepperPlugin(name = "prometheus", pluginType = Sink.class, pluginConfigurationType = PrometheusSinkConfiguration.class) +public class PrometheusSink extends AbstractSink> { + + private static final Logger LOG = LoggerFactory.getLogger(PrometheusSink.class); + + private static final String BUCKET = "bucket"; + private static final String KEY_PATH = "key_path_prefix"; + + private volatile boolean sinkInitialized; + + private final PrometheusSinkService prometheusSinkService; + + private final BufferFactory bufferFactory; + + private DlqPushHandler dlqPushHandler; + + @DataPrepperPluginConstructor + public PrometheusSink(final PluginSetting pluginSetting, + final PrometheusSinkConfiguration prometheusSinkConfiguration, + final PluginFactory pluginFactory, + final PipelineDescription pipelineDescription, + final AwsCredentialsSupplier awsCredentialsSupplier) { + super(pluginSetting); + this.sinkInitialized = Boolean.FALSE; + if (prometheusSinkConfiguration.getBufferType().equals(BufferTypeOptions.LOCALFILE)) { + this.bufferFactory = new LocalFileBufferFactory(); + } else { + this.bufferFactory = new InMemoryBufferFactory(); + } + + this.dlqPushHandler = new DlqPushHandler(prometheusSinkConfiguration.getDlqFile(), pluginFactory, + String.valueOf(prometheusSinkConfiguration.getDlqPluginSetting().get(BUCKET)), + prometheusSinkConfiguration.getDlqStsRoleARN() + ,prometheusSinkConfiguration.getDlqStsRegion(), + String.valueOf(prometheusSinkConfiguration.getDlqPluginSetting().get(KEY_PATH))); + + final HttpRequestRetryStrategy httpRequestRetryStrategy = new DefaultHttpRequestRetryStrategy(prometheusSinkConfiguration.getMaxUploadRetries(), + TimeValue.of(prometheusSinkConfiguration.getHttpRetryInterval())); + + final HttpClientBuilder httpClientBuilder = HttpClients.custom() + .setRetryStrategy(httpRequestRetryStrategy); + + if(prometheusSinkConfiguration.isAwsSigv4() && prometheusSinkConfiguration.isValidAWSUrl()){ + PrometheusSinkAwsService.attachSigV4(prometheusSinkConfiguration, httpClientBuilder, awsCredentialsSupplier); + } + this.prometheusSinkService = new PrometheusSinkService( + prometheusSinkConfiguration, + bufferFactory, + dlqPushHandler, + httpClientBuilder, + pluginMetrics, + pluginSetting); + } + + @Override + public boolean isReady() { + return sinkInitialized; + } + + @Override + public void doInitialize() { + try { + doInitializeInternal(); + } catch (InvalidPluginConfigurationException e) { + LOG.error("Invalid plugin configuration, Hence failed to initialize http-sink plugin."); + this.shutdown(); + throw e; + } catch (Exception e) { + LOG.error("Failed to initialize http-sink plugin."); + this.shutdown(); + throw e; + } + } + + private void doInitializeInternal() { + sinkInitialized = Boolean.TRUE; + } + + /** + * @param records Records to be output + */ + @Override + public void doOutput(final Collection> records) { + if (records.isEmpty()) { + return; + } + prometheusSinkService.output(records); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactory.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactory.java new file mode 100644 index 0000000000..8478d1ab43 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactory.java @@ -0,0 +1,76 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.certificate; + +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.plugins.certificate.CertificateProvider; +import org.opensearch.dataprepper.plugins.certificate.acm.ACMCertificateProvider; +import org.opensearch.dataprepper.plugins.certificate.file.FileCertificateProvider; +import org.opensearch.dataprepper.plugins.certificate.s3.S3CertificateProvider; +import org.opensearch.dataprepper.plugins.metricpublisher.MicrometerMetricPublisher; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryMode; +import software.amazon.awssdk.services.acm.AcmClient; +import software.amazon.awssdk.services.s3.S3Client; + +/** + * This class consist logic for downloading the SSL certificates from S3/ACM/Local file. + * + */ +public class CertificateProviderFactory { + private static final Logger LOG = LoggerFactory.getLogger(CertificateProviderFactory.class); + + final PrometheusSinkConfiguration prometheusSinkConfiguration; + public CertificateProviderFactory(final PrometheusSinkConfiguration prometheusSinkConfiguration) { + this.prometheusSinkConfiguration = prometheusSinkConfiguration; + } + + /** + * This method consist logic for downloading the SSL certificates from S3/ACM/Local file. + * @return CertificateProvider + */ + public CertificateProvider getCertificateProvider() { + if (prometheusSinkConfiguration.useAcmCertForSSL()) { + LOG.info("Using ACM certificate and private key for SSL/TLS."); + final AwsCredentialsProvider credentialsProvider = AwsCredentialsProviderChain.builder() + .addCredentialsProvider(DefaultCredentialsProvider.create()).build(); + final ClientOverrideConfiguration clientConfig = ClientOverrideConfiguration.builder() + .retryPolicy(RetryMode.STANDARD) + .build(); + + final PluginMetrics awsSdkMetrics = PluginMetrics.fromNames("sdk", "aws"); + + final AcmClient awsCertificateManager = AcmClient.builder() + .region(prometheusSinkConfiguration.getAwsAuthenticationOptions().getAwsRegion()) + .credentialsProvider(credentialsProvider) + .overrideConfiguration(clientConfig) + .overrideConfiguration(metricPublisher -> metricPublisher.addMetricPublisher(new MicrometerMetricPublisher(awsSdkMetrics))) + .build(); + + return new ACMCertificateProvider(awsCertificateManager, prometheusSinkConfiguration.getAcmCertificateArn(), + prometheusSinkConfiguration.getAcmCertIssueTimeOutMillis(), prometheusSinkConfiguration.getAcmPrivateKeyPassword()); + } else if (prometheusSinkConfiguration.isSslCertAndKeyFileInS3()) { + LOG.info("Using S3 to fetch certificate and private key for SSL/TLS."); + final AwsCredentialsProvider credentialsProvider = AwsCredentialsProviderChain.builder() + .addCredentialsProvider(DefaultCredentialsProvider.create()).build(); + final S3Client s3Client = S3Client.builder() + .region(prometheusSinkConfiguration.getAwsAuthenticationOptions().getAwsRegion()) + .credentialsProvider(credentialsProvider) + .build(); + return new S3CertificateProvider(s3Client, + prometheusSinkConfiguration.getSslCertificateFile(), + prometheusSinkConfiguration.getSslKeyFile()); + } else { + LOG.info("Using local file system to get certificate and private key for SSL/TLS."); + return new FileCertificateProvider(prometheusSinkConfiguration.getSslCertificateFile(), prometheusSinkConfiguration.getSslKeyFile()); + } + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java new file mode 100644 index 0000000000..2878d54f3c --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java @@ -0,0 +1,83 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.certificate; + +import org.apache.hc.client5.http.config.TlsConfig; +import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder; +import org.apache.hc.client5.http.io.HttpClientConnectionManager; +import org.apache.hc.client5.http.ssl.SSLConnectionSocketFactory; +import org.apache.hc.client5.http.ssl.SSLConnectionSocketFactoryBuilder; +import org.apache.hc.client5.http.ssl.TrustAllStrategy; +import org.apache.hc.core5.http.ssl.TLS; +import org.apache.hc.core5.ssl.SSLContextBuilder; +import org.apache.hc.core5.ssl.SSLContexts; +import org.apache.hc.core5.ssl.TrustStrategy; +import org.apache.hc.core5.util.Timeout; +import org.opensearch.dataprepper.plugins.certificate.CertificateProvider; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; + +import javax.net.ssl.SSLContext; +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.security.KeyStore; +import java.security.cert.Certificate; +import java.security.cert.CertificateFactory; + +/** + * This class implements SSL certs authentication + * + */ +public class HttpClientSSLConnectionManager { + + /** + * This method creates HttpClientConnectionManager for SSL certs authentication + * @param sinkConfiguration HttpSinkConfiguration + * @param providerFactory CertificateProviderFactory + * @return HttpClientConnectionManager + */ + public HttpClientConnectionManager createHttpClientConnectionManager(final PrometheusSinkConfiguration sinkConfiguration, + final CertificateProviderFactory providerFactory){ + final CertificateProvider certificateProvider = providerFactory.getCertificateProvider(); + final org.opensearch.dataprepper.plugins.certificate.model.Certificate certificate = certificateProvider.getCertificate(); + final SSLContext sslContext = sinkConfiguration.getSslCertificateFile() != null ? + getCAStrategy(new ByteArrayInputStream(certificate.getCertificate().getBytes(StandardCharsets.UTF_8))) : getTrustAllStrategy(); + SSLConnectionSocketFactory sslSocketFactory = SSLConnectionSocketFactoryBuilder.create() + .setSslContext(sslContext) + .build(); + return PoolingHttpClientConnectionManagerBuilder.create() + .setSSLSocketFactory(sslSocketFactory) + .setDefaultTlsConfig(TlsConfig.custom() + .setHandshakeTimeout(Timeout.ofSeconds(30)) + .setSupportedProtocols(TLS.V_1_3) + .build()) + .build(); + } + + private SSLContext getCAStrategy(final InputStream certificate) { + try { + CertificateFactory factory = CertificateFactory.getInstance("X.509"); + Certificate trustedCa; + trustedCa = factory.generateCertificate(certificate); + 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() { + 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/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthTypeOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthTypeOptions.java new file mode 100644 index 0000000000..3faed62286 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthTypeOptions.java @@ -0,0 +1,34 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; + +public enum AuthTypeOptions { + HTTP_BASIC("http_basic"), + BEARER_TOKEN("bearer_token"), + UNAUTHENTICATED("unauthenticated"); + + private static final Map OPTIONS_MAP = Arrays.stream(AuthTypeOptions.values()) + .collect(Collectors.toMap( + value -> value.option, + value -> value + )); + + private final String option; + + AuthTypeOptions(final String option) { + this.option = option; + } + + @JsonCreator + static AuthTypeOptions fromOptionValue(final String option) { + return OPTIONS_MAP.get(option); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java new file mode 100644 index 0000000000..d5e247decb --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java @@ -0,0 +1,20 @@ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class AuthenticationOptions { + + @JsonProperty("http_basic") + private BasicAuthCredentials httpBasic; + + @JsonProperty("bearer_token") + private BearerTokenOptions bearerTokenOptions; + + public BasicAuthCredentials getHttpBasic() { + return httpBasic; + } + + public BearerTokenOptions getBearerTokenOptions() { + return bearerTokenOptions; + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptions.java new file mode 100644 index 0000000000..7dd94a243c --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptions.java @@ -0,0 +1,45 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; +import software.amazon.awssdk.regions.Region; + +import java.util.Map; + +public class AwsAuthenticationOptions { + @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("sts_external_id") + @Size(min = 2, max = 1224, message = "awsStsExternalId length should be between 2 and 1224 characters") + private String awsStsExternalId; + + @JsonProperty("sts_header_overrides") + @Size(max = 5, message = "sts_header_overrides supports a maximum of 5 headers to override") + private Map awsStsHeaderOverrides; + + public Region getAwsRegion() { + return awsRegion != null ? Region.of(awsRegion) : null; + } + + public String getAwsStsRoleArn() { + return awsStsRoleArn; + } + + public String getAwsStsExternalId() { + return awsStsExternalId; + } + + public Map getAwsStsHeaderOverrides() { + return awsStsHeaderOverrides; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BasicAuthCredentials.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BasicAuthCredentials.java new file mode 100644 index 0000000000..19890559c6 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BasicAuthCredentials.java @@ -0,0 +1,21 @@ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class BasicAuthCredentials { + + + @JsonProperty("username") + private String username; + + @JsonProperty("password") + private String password; + + public String getUsername() { + return username; + } + + public String getPassword() { + return password; + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BearerTokenOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BearerTokenOptions.java new file mode 100644 index 0000000000..1ab69feb00 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BearerTokenOptions.java @@ -0,0 +1,57 @@ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.NotNull; + +public class BearerTokenOptions { + + @JsonProperty("client_id") + @NotNull(message = "client id type is mandatory for refresh token") + private String clientId; + + @JsonProperty("client_secret") + @NotNull(message = "client secret type is mandatory for refresh token") + private String clientSecret; + + @JsonProperty("token_url") + @NotNull(message = "token url type is mandatory for refresh token") + private String tokenURL; + + @JsonProperty("grant_type") + @NotNull(message = "grant type is mandatory for refresh token") + private String grantType; + + @JsonProperty("scope") + @NotNull(message = "scope is mandatory for refresh token") + private String scope; + + private String refreshToken; + + public String getScope() { + return scope; + } + + public String getGrantType() { + return grantType; + } + + public String getRefreshToken() { + return refreshToken; + } + + public String getClientId() { + return clientId; + } + + public String getClientSecret() { + return clientSecret; + } + + public String getTokenURL() { + return tokenURL; + } + + public void setRefreshToken(String refreshToken) { + this.refreshToken = refreshToken; + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/HTTPMethodOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/HTTPMethodOptions.java new file mode 100644 index 0000000000..0f6c13a853 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/HTTPMethodOptions.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; + +public enum HTTPMethodOptions { + PUT("PUT"), + POST("POST"); + + private static final Map OPTIONS_MAP = Arrays.stream(HTTPMethodOptions.values()) + .collect(Collectors.toMap( + value -> value.option, + value -> value + )); + + private final String option; + + HTTPMethodOptions(final String option) { + this.option = option; + } + + @JsonCreator + static HTTPMethodOptions fromOptionValue(final String option) { + return OPTIONS_MAP.get(option); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java new file mode 100644 index 0000000000..cae9aff22e --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java @@ -0,0 +1,297 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotNull; +import org.apache.commons.lang3.StringUtils; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.plugins.accumulator.BufferTypeOptions; +import org.opensearch.dataprepper.plugins.sink.prometheus.util.PrometheusSinkUtil; + +import java.net.URL; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class PrometheusSinkConfiguration { + + private static final int DEFAULT_UPLOAD_RETRIES = 5; + + private static final int DEFAULT_WORKERS = 1; + + static final boolean DEFAULT_SSL = false; + + private static final String S3_PREFIX = "s3://"; + + static final String SSL_KEY_CERT_FILE = "sslKeyCertChainFile"; + static final String SSL_KEY_FILE = "sslKeyFile"; + static final String SSL = "ssl"; + static final String AWS_REGION = "awsRegion"; + + public static final String STS_REGION = "sts_region"; + + public static final String STS_ROLE_ARN = "sts_role_arn"; + static final boolean DEFAULT_USE_ACM_CERT_FOR_SSL = false; + public static final int DEFAULT_ACM_CERT_ISSUE_TIME_OUT_MILLIS = 120000; + public static final String SSL_IS_ENABLED = "%s is enabled"; + + public static final Duration DEFAULT_HTTP_RETRY_INTERVAL = Duration.ofSeconds(30); + + private static final String HTTPS = "https"; + + private static final String AWS_HOST_AMAZONAWS_COM = "amazonaws.com"; + + private static final String AWS_HOST_API_AWS = "api.aws"; + + private static final String AWS_HOST_ON_AWS = "on.aws"; + + private static final String DEFAULT_ENCODING = "snappy"; + + private static final String DEFAULT_CONTENT_TYPE = "application/x-protobuf"; + + private static final String DEFAULT_REMOTE_WRITE_VERSION = "0.1.0"; + + + @NotNull + @JsonProperty("url") + private String url; + + @JsonProperty("workers") + private Integer workers = DEFAULT_WORKERS; + + @JsonProperty("codec") + private PluginModel codec; + + @JsonProperty("http_method") + private HTTPMethodOptions httpMethod = HTTPMethodOptions.POST; + + @JsonProperty("proxy") + private String proxy; + + @JsonProperty("auth_type") + private AuthTypeOptions authType = AuthTypeOptions.UNAUTHENTICATED; + + @JsonProperty("authentication") + private AuthenticationOptions authentication; + + @JsonProperty("ssl_certificate_file") + private String sslCertificateFile; + + @JsonProperty("ssl_key_file") + private String sslKeyFile; + + @JsonProperty("aws_sigv4") + private boolean awsSigv4; + + @JsonProperty("buffer_type") + private BufferTypeOptions bufferType = BufferTypeOptions.INMEMORY; + + @JsonProperty("max_retries") + private int maxUploadRetries = DEFAULT_UPLOAD_RETRIES; + + @JsonProperty("aws") + @Valid + private AwsAuthenticationOptions awsAuthenticationOptions; + + @JsonProperty("custom_header") + private Map> customHeaderOptions; + + @JsonProperty("dlq_file") + private String dlqFile; + + @JsonProperty("dlq") + private PluginModel dlq; + + @JsonProperty("use_acm_cert_for_ssl") + private boolean useAcmCertForSSL = DEFAULT_USE_ACM_CERT_FOR_SSL; + + @JsonProperty("acm_private_key_password") + private String acmPrivateKeyPassword; + + @JsonProperty("acm_certificate_arn") + private String acmCertificateArn; + + @JsonProperty("acm_cert_issue_time_out_millis") + private long acmCertIssueTimeOutMillis = DEFAULT_ACM_CERT_ISSUE_TIME_OUT_MILLIS; + + @JsonProperty("ssl") + private boolean ssl = DEFAULT_SSL; + + @JsonProperty("http_retry_interval") + private Duration httpRetryInterval = DEFAULT_HTTP_RETRY_INTERVAL; + + @JsonProperty("encoding") + private String encoding = DEFAULT_ENCODING; + + @JsonProperty("content-type") + private String contentType = DEFAULT_CONTENT_TYPE; + + @JsonProperty("remote-write-version") + private String remoteWriteVersion = DEFAULT_REMOTE_WRITE_VERSION; + + private boolean sslCertAndKeyFileInS3; + + public String getUrl() { + return url; + } + + public boolean isSsl() { + return ssl; + } + + public Duration getHttpRetryInterval() { + return httpRetryInterval; + } + + public String getAcmPrivateKeyPassword() { + return acmPrivateKeyPassword; + } + + public boolean isSslCertAndKeyFileInS3() { + return sslCertAndKeyFileInS3; + } + + public long getAcmCertIssueTimeOutMillis() { + return acmCertIssueTimeOutMillis; + } + + public boolean useAcmCertForSSL() { + return useAcmCertForSSL; + } + + public void validateAndInitializeCertAndKeyFileInS3() { + boolean certAndKeyFileInS3 = false; + if (useAcmCertForSSL) { + validateSSLArgument(String.format(SSL_IS_ENABLED, useAcmCertForSSL), acmCertificateArn, acmCertificateArn); + validateSSLArgument(String.format(SSL_IS_ENABLED, useAcmCertForSSL), awsAuthenticationOptions.getAwsRegion().toString(), AWS_REGION); + } else if(ssl) { + validateSSLCertificateFiles(); + certAndKeyFileInS3 = isSSLCertificateLocatedInS3(); + if (certAndKeyFileInS3) { + validateSSLArgument("The certificate and key files are located in S3", awsAuthenticationOptions.getAwsRegion().toString(), AWS_REGION); + } + } + sslCertAndKeyFileInS3 = certAndKeyFileInS3; + } + private void validateSSLArgument(final String sslTypeMessage, final String argument, final String argumentName) { + if (StringUtils.isEmpty(argument)) { + throw new IllegalArgumentException(String.format("%s, %s can not be empty or null", sslTypeMessage, argumentName)); + } + } + + private void validateSSLCertificateFiles() { + validateSSLArgument(String.format(SSL_IS_ENABLED, SSL), sslCertificateFile, SSL_KEY_CERT_FILE); + validateSSLArgument(String.format(SSL_IS_ENABLED, SSL), sslKeyFile, SSL_KEY_FILE); + } + + private boolean isSSLCertificateLocatedInS3() { + return sslCertificateFile.toLowerCase().startsWith(S3_PREFIX) && + sslKeyFile.toLowerCase().startsWith(S3_PREFIX); + } + + public String getAcmCertificateArn() { + return acmCertificateArn; + } + + public PluginModel getCodec() { + return codec; + } + + public HTTPMethodOptions getHttpMethod() { + return httpMethod; + } + + public String getProxy() { + return proxy; + } + + public AuthTypeOptions getAuthType() { + return authType; + } + + public AuthenticationOptions getAuthentication() { + return authentication; + } + + public String getSslCertificateFile() { + return sslCertificateFile; + } + + public String getSslKeyFile() { + return sslKeyFile; + } + + public boolean isAwsSigv4() { + return awsSigv4; + } + + public BufferTypeOptions getBufferType() { + return bufferType; + } + + + public int getMaxUploadRetries() { + return maxUploadRetries; + } + + public Map> getCustomHeaderOptions() { + return customHeaderOptions; + } + + public AwsAuthenticationOptions getAwsAuthenticationOptions() { + return awsAuthenticationOptions; + } + + public Integer getWorkers() { + return workers; + } + + public String getDlqFile() { + return dlqFile; + } + + public PluginModel getDlq() { + return dlq; + } + + public boolean isValidAWSUrl() { + URL parsedUrl = PrometheusSinkUtil.getURLByUrlString(url); + if(parsedUrl.getProtocol().equals(HTTPS) && (parsedUrl.getHost().contains(AWS_HOST_AMAZONAWS_COM) ||parsedUrl.getHost().contains(AWS_HOST_API_AWS) || parsedUrl.getHost().contains(AWS_HOST_ON_AWS))){ + return true; + } + return false; + } + + public String getDlqStsRoleARN(){ + return Objects.nonNull(getDlqPluginSetting().get(STS_ROLE_ARN)) ? + String.valueOf(getDlqPluginSetting().get(STS_ROLE_ARN)) : + awsAuthenticationOptions.getAwsStsRoleArn(); + } + + public String getDlqStsRegion(){ + return Objects.nonNull(getDlqPluginSetting().get(STS_REGION)) ? + String.valueOf(getDlqPluginSetting().get(STS_REGION)) : + awsAuthenticationOptions.getAwsRegion().toString(); + } + + public Map getDlqPluginSetting(){ + return dlq != null ? dlq.getPluginSettings() : Map.of(); + } + + public String getEncoding() { + return encoding; + } + + public String getContentType() { + return contentType; + } + + public String getRemoteWriteVersion() { + return remoteWriteVersion; + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandler.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandler.java new file mode 100644 index 0000000000..a192b0965d --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandler.java @@ -0,0 +1,134 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.dlq; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; +import io.micrometer.core.instrument.util.StringUtils; +import org.opensearch.dataprepper.metrics.MetricNames; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.failures.DlqObject; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.plugins.dlq.DlqProvider; +import org.opensearch.dataprepper.plugins.dlq.DlqWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.StringJoiner; + +import static java.util.UUID.randomUUID; + + +/** + * * An Handler class which helps log failed data to AWS S3 bucket or file based on configuration. + */ + +public class DlqPushHandler { + + private static final Logger LOG = LoggerFactory.getLogger(DlqPushHandler.class); + + private static final String BUCKET = "bucket"; + + private static final String ROLE_ARN = "sts_role_arn"; + + private static final String REGION = "region"; + + private static final String S3_PLUGIN_NAME = "s3"; + + private static final String KEY_PATH_PREFIX = "key_path_prefix"; + + private String dlqFile; + + private String keyPathPrefix; + + private DlqProvider dlqProvider; + + private ObjectWriter objectWriter; + + public DlqPushHandler(final String dlqFile, + final PluginFactory pluginFactory, + final String bucket, + final String stsRoleArn, + final String awsRegion, + final String dlqPathPrefix) { + if(dlqFile != null) { + this.dlqFile = dlqFile; + this.objectWriter = new ObjectMapper().writer().withDefaultPrettyPrinter(); + }else{ + this.dlqProvider = getDlqProvider(pluginFactory,bucket,stsRoleArn,awsRegion,dlqPathPrefix); + } + } + + public void perform(final PluginSetting pluginSetting, + final Object failedData) { + if(dlqFile != null) + writeToFile(failedData); + else + pushToS3(pluginSetting, failedData); + } + + private void writeToFile(Object failedData) { + try(BufferedWriter dlqFileWriter = Files.newBufferedWriter(Paths.get(dlqFile), + StandardOpenOption.CREATE, StandardOpenOption.APPEND)) { + dlqFileWriter.write(objectWriter.writeValueAsString(failedData)+"\n"); + } catch (IOException e) { + LOG.error("Exception while writing failed data to DLQ file Exception: ",e); + } + } + + private void pushToS3(PluginSetting pluginSetting, Object failedData) { + DlqWriter dlqWriter = getDlqWriter(pluginSetting.getPipelineName()); + try { + String pluginId = randomUUID().toString(); + DlqObject dlqObject = DlqObject.builder() + .withPluginId(pluginId) + .withPluginName(pluginSetting.getName()) + .withPipelineName(pluginSetting.getPipelineName()) + .withFailedData(failedData) + .build(); + + dlqWriter.write(Arrays.asList(dlqObject), pluginSetting.getPipelineName(), pluginId); + } catch (final IOException e) { + LOG.error("Exception while writing failed data to DLQ, Exception : ", e); + } + } + + private DlqWriter getDlqWriter(final String writerPipelineName) { + Optional potentialDlq = dlqProvider.getDlqWriter(new StringJoiner(MetricNames.DELIMITER) + .add(writerPipelineName).toString()); + DlqWriter dlqWriter = potentialDlq.isPresent() ? potentialDlq.get() : null; + return dlqWriter; + } + + private DlqProvider getDlqProvider(final PluginFactory pluginFactory, + final String bucket, + final String stsRoleArn, + final String awsRegion, + final String dlqPathPrefix) { + final Map props = new HashMap<>(); + props.put(BUCKET, bucket); + props.put(ROLE_ARN, stsRoleArn); + props.put(REGION, awsRegion); + this.keyPathPrefix = StringUtils.isEmpty(dlqPathPrefix) ? dlqPathPrefix : enforceDefaultDelimiterOnKeyPathPrefix(dlqPathPrefix); + props.put(KEY_PATH_PREFIX, dlqPathPrefix); + final PluginSetting dlqPluginSetting = new PluginSetting(S3_PLUGIN_NAME, props); + DlqProvider dlqProvider = pluginFactory.loadPlugin(DlqProvider.class, dlqPluginSetting); + return dlqProvider; + } + + private String enforceDefaultDelimiterOnKeyPathPrefix(final String keyPathPrefix) { + return (keyPathPrefix.charAt(keyPathPrefix.length() - 1) == '/') ? keyPathPrefix : keyPathPrefix.concat("/"); + } +} + diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/FailedDlqData.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/FailedDlqData.java new file mode 100644 index 0000000000..3881af2556 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/FailedDlqData.java @@ -0,0 +1,65 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.dlq; + +public class FailedDlqData { + + private String url; + + private int status; + + private String message; + + public FailedDlqData(final Builder builder) { + this.status = builder.status; + this.message = builder.message; + this.url = builder.url; + } + + public String getUrl() { + return url; + } + + public int getStatus() { + return status; + } + + public String getMessage() { + return message; + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + + private String url; + + private int status; + + private String message; + + + public Builder withUrl(String url) { + this.url = url; + return this; + } + + public Builder withStatus(int status) { + this.status = status; + return this; + } + + public Builder withMessage(String message) { + this.message = message; + return this; + } + + public FailedDlqData build() { + return new FailedDlqData(this); + } + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BasicAuthPrometheusSinkHandler.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BasicAuthPrometheusSinkHandler.java new file mode 100644 index 0000000000..6348c90767 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BasicAuthPrometheusSinkHandler.java @@ -0,0 +1,44 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.handler; + +import org.apache.hc.client5.http.auth.AuthScope; +import org.apache.hc.client5.http.auth.UsernamePasswordCredentials; +import org.apache.hc.client5.http.impl.auth.BasicCredentialsProvider; +import org.apache.hc.client5.http.io.HttpClientConnectionManager; +import org.opensearch.dataprepper.plugins.sink.prometheus.FailedHttpResponseInterceptor; +import org.opensearch.dataprepper.plugins.sink.prometheus.util.PrometheusSinkUtil; + +/** + * * This class handles Basic Authentication + */ +public class BasicAuthPrometheusSinkHandler implements MultiAuthPrometheusSinkHandler { + + private final HttpClientConnectionManager httpClientConnectionManager; + + private final String username; + + private final String password; + + public BasicAuthPrometheusSinkHandler(final String username, + final String password, + final HttpClientConnectionManager httpClientConnectionManager){ + this.httpClientConnectionManager = httpClientConnectionManager; + this.username = username; + this.password = password; + } + + @Override + public HttpAuthOptions authenticate(final HttpAuthOptions.Builder httpAuthOptionsBuilder) { + final BasicCredentialsProvider provider = new BasicCredentialsProvider(); + AuthScope authScope = new AuthScope(PrometheusSinkUtil.getHttpHostByURL(PrometheusSinkUtil.getURLByUrlString(httpAuthOptionsBuilder.getUrl()))); + provider.setCredentials(authScope, new UsernamePasswordCredentials(username, password.toCharArray())); + httpAuthOptionsBuilder.setHttpClientBuilder(httpAuthOptionsBuilder.build().getHttpClientBuilder() + .setConnectionManager(httpClientConnectionManager) + .addResponseInterceptorLast(new FailedHttpResponseInterceptor(httpAuthOptionsBuilder.getUrl())) + .setDefaultCredentialsProvider(provider)); + return httpAuthOptionsBuilder.build(); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BearerTokenAuthPrometheusSinkHandler.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BearerTokenAuthPrometheusSinkHandler.java new file mode 100644 index 0000000000..03a5a751b3 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BearerTokenAuthPrometheusSinkHandler.java @@ -0,0 +1,51 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.handler; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.hc.client5.http.io.HttpClientConnectionManager; +import org.opensearch.dataprepper.plugins.sink.prometheus.FailedHttpResponseInterceptor; +import org.opensearch.dataprepper.plugins.sink.prometheus.OAuthAccessTokenManager; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.BearerTokenOptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * * This class handles Bearer Token Authentication + */ +public class BearerTokenAuthPrometheusSinkHandler implements MultiAuthPrometheusSinkHandler { + + private static final Logger LOG = LoggerFactory.getLogger(BearerTokenAuthPrometheusSinkHandler.class); + + public static final String AUTHORIZATION = "Authorization"; + + private final HttpClientConnectionManager httpClientConnectionManager; + + private final BearerTokenOptions bearerTokenOptions; + + private final ObjectMapper objectMapper; + + private OAuthAccessTokenManager oAuthRefreshTokenManager; + + public BearerTokenAuthPrometheusSinkHandler(final BearerTokenOptions bearerTokenOptions, + final HttpClientConnectionManager httpClientConnectionManager, + final OAuthAccessTokenManager oAuthRefreshTokenManager){ + this.bearerTokenOptions = bearerTokenOptions; + this.httpClientConnectionManager = httpClientConnectionManager; + this.objectMapper = new ObjectMapper(); + this.oAuthRefreshTokenManager = oAuthRefreshTokenManager; + } + + @Override + public HttpAuthOptions authenticate(final HttpAuthOptions.Builder httpAuthOptionsBuilder) { + httpAuthOptionsBuilder.getClassicHttpRequestBuilder() + .addHeader(AUTHORIZATION, oAuthRefreshTokenManager.getAccessToken(bearerTokenOptions)); + httpAuthOptionsBuilder.setHttpClientBuilder(httpAuthOptionsBuilder.build().getHttpClientBuilder() + .setConnectionManager(httpClientConnectionManager) + .addResponseInterceptorLast(new FailedHttpResponseInterceptor(httpAuthOptionsBuilder.getUrl()))); + return httpAuthOptionsBuilder.build(); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/HttpAuthOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/HttpAuthOptions.java new file mode 100644 index 0000000000..843eeb5c90 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/HttpAuthOptions.java @@ -0,0 +1,111 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.handler; + +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; +import org.apache.hc.client5.http.io.HttpClientConnectionManager; +import org.apache.hc.core5.http.HttpHost; +import org.apache.hc.core5.http.io.support.ClassicRequestBuilder; + + +public class HttpAuthOptions { + + private String url; + + private HttpClientBuilder httpClientBuilder; + + private ClassicRequestBuilder classicHttpRequestBuilder; + + private HttpClientConnectionManager httpClientConnectionManager; + + private int workers; + + private HttpHost proxy; + + public HttpClientBuilder getHttpClientBuilder() { + return httpClientBuilder; + } + + public ClassicRequestBuilder getClassicHttpRequestBuilder() { + return classicHttpRequestBuilder; + } + + public int getWorkers() { + return workers; + } + + public String getUrl() { + return url; + } + + public HttpHost getProxy() { + return proxy; + } + + public HttpClientConnectionManager getHttpClientConnectionManager() { + return httpClientConnectionManager; + } + + private HttpAuthOptions(Builder builder) { + this.url = builder.url; + this.httpClientBuilder = builder.httpClientBuilder; + this.classicHttpRequestBuilder = builder.classicHttpRequestBuilder; + this.httpClientConnectionManager = builder.httpClientConnectionManager; + this.workers = builder.workers; + this.proxy = builder.proxy; + } + public static class Builder { + + private String url; + private HttpClientBuilder httpClientBuilder; + private ClassicRequestBuilder classicHttpRequestBuilder; + private HttpClientConnectionManager httpClientConnectionManager; + private int workers; + private HttpHost proxy; + + public HttpAuthOptions build() { + return new HttpAuthOptions(this); + } + + public Builder setUrl(String url) { + this.url = url; + return this; + } + + public String getUrl() { + return url; + } + + public Builder setHttpClientBuilder(HttpClientBuilder httpClientBuilder) { + this.httpClientBuilder = httpClientBuilder; + return this; + } + + public Builder setClassicHttpRequestBuilder(ClassicRequestBuilder classicHttpRequestBuilder) { + this.classicHttpRequestBuilder = classicHttpRequestBuilder; + return this; + } + + public Builder setHttpClientConnectionManager(HttpClientConnectionManager httpClientConnectionManager) { + this.httpClientConnectionManager = httpClientConnectionManager; + return this; + } + + public Builder setWorkers(int workers) { + this.workers = workers; + return this; + } + + public Builder setProxy(HttpHost proxy) { + this.proxy = proxy; + return this; + } + + public ClassicRequestBuilder getClassicHttpRequestBuilder() { + return classicHttpRequestBuilder; + } + } + +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/MultiAuthPrometheusSinkHandler.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/MultiAuthPrometheusSinkHandler.java new file mode 100644 index 0000000000..fa651a08d4 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/MultiAuthPrometheusSinkHandler.java @@ -0,0 +1,19 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.handler; + +/** + * An interface to handle multiple authentications + */ +public interface MultiAuthPrometheusSinkHandler { + + /** + * This method can be used to implement multiple authentication based on configuration + * @param httpAuthOptionsBuilder HttpAuthOptions.Builder + * @return HttpAuthOptions + */ + HttpAuthOptions authenticate(final HttpAuthOptions.Builder httpAuthOptionsBuilder); + +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java new file mode 100644 index 0000000000..597757212a --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java @@ -0,0 +1,40 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.service; + +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.sink.prometheus.AwsRequestSigningApacheInterceptor; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.signer.Aws4Signer; + +public class PrometheusSinkAwsService { + + private static final Logger LOG = LoggerFactory.getLogger(PrometheusSinkAwsService.class); + public static final String AWS_SIGV4 = "aws_sigv4"; + private static final String AOS_SERVICE_NAME = "http-endpoint"; + + public static void attachSigV4(final PrometheusSinkConfiguration prometheusSinkConfiguration, final HttpClientBuilder httpClientBuilder, final AwsCredentialsSupplier awsCredentialsSupplier) { + LOG.info("{} is set, will sign requests using AWSRequestSigningApacheInterceptor", AWS_SIGV4); + final Aws4Signer aws4Signer = Aws4Signer.create(); + final AwsCredentialsOptions awsCredentialsOptions = createAwsCredentialsOptions(prometheusSinkConfiguration); + final AwsCredentialsProvider credentialsProvider = awsCredentialsSupplier.getProvider(awsCredentialsOptions); + httpClientBuilder.addRequestInterceptorLast(new AwsRequestSigningApacheInterceptor(AOS_SERVICE_NAME, aws4Signer, + credentialsProvider, prometheusSinkConfiguration.getAwsAuthenticationOptions().getAwsRegion())); + } + + private static AwsCredentialsOptions createAwsCredentialsOptions(final PrometheusSinkConfiguration prometheusSinkConfiguration) { + return AwsCredentialsOptions.builder() + .withStsRoleArn(prometheusSinkConfiguration.getAwsAuthenticationOptions().getAwsStsRoleArn()) + .withStsExternalId(prometheusSinkConfiguration.getAwsAuthenticationOptions().getAwsStsExternalId()) + .withRegion(prometheusSinkConfiguration.getAwsAuthenticationOptions().getAwsRegion()) + .withStsHeaderOverrides(prometheusSinkConfiguration.getAwsAuthenticationOptions().getAwsStsHeaderOverrides()) + .build(); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java new file mode 100644 index 0000000000..6a2478489d --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java @@ -0,0 +1,434 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.service; + +import com.arpnetworking.metrics.prometheus.Remote; +import com.arpnetworking.metrics.prometheus.Types; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.micrometer.core.instrument.Counter; +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; +import org.apache.hc.client5.http.io.HttpClientConnectionManager; +import org.apache.hc.client5.http.protocol.HttpClientContext; +import org.apache.hc.core5.http.ContentType; +import org.apache.hc.core5.http.HttpEntity; +import org.apache.hc.core5.http.HttpStatus; +import org.apache.hc.core5.http.io.entity.ByteArrayEntity; +import org.apache.hc.core5.http.io.support.ClassicRequestBuilder; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.metric.JacksonExponentialHistogram; +import org.opensearch.dataprepper.model.metric.JacksonGauge; +import org.opensearch.dataprepper.model.metric.JacksonHistogram; +import org.opensearch.dataprepper.model.metric.JacksonSum; +import org.opensearch.dataprepper.model.metric.JacksonSummary; +import org.opensearch.dataprepper.model.record.Record; + +import org.opensearch.dataprepper.plugins.accumulator.Buffer; +import org.opensearch.dataprepper.plugins.accumulator.BufferFactory; +import org.opensearch.dataprepper.plugins.sink.prometheus.certificate.CertificateProviderFactory; +import org.opensearch.dataprepper.plugins.sink.prometheus.FailedHttpResponseInterceptor; +import org.opensearch.dataprepper.plugins.sink.prometheus.HttpEndPointResponse; +import org.opensearch.dataprepper.plugins.sink.prometheus.OAuthAccessTokenManager; +import org.opensearch.dataprepper.plugins.sink.prometheus.certificate.HttpClientSSLConnectionManager; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.AuthTypeOptions; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.HTTPMethodOptions; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; +import org.opensearch.dataprepper.plugins.sink.prometheus.dlq.DlqPushHandler; +import org.opensearch.dataprepper.plugins.sink.prometheus.dlq.FailedDlqData; +import org.opensearch.dataprepper.plugins.sink.prometheus.handler.BasicAuthPrometheusSinkHandler; +import org.opensearch.dataprepper.plugins.sink.prometheus.handler.BearerTokenAuthPrometheusSinkHandler; +import org.opensearch.dataprepper.plugins.sink.prometheus.handler.HttpAuthOptions; +import org.opensearch.dataprepper.plugins.sink.prometheus.handler.MultiAuthPrometheusSinkHandler; +import org.opensearch.dataprepper.plugins.sink.prometheus.util.PrometheusSinkUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.xerial.snappy.Snappy; + +import java.io.IOException; + +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedList; +import java.util.Map; +import java.util.List; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Objects; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.regex.Pattern; + +import static org.opensearch.dataprepper.plugins.sink.prometheus.handler.BearerTokenAuthPrometheusSinkHandler.AUTHORIZATION; + +/** + * This service class contains logic for sending data to Http Endpoints + */ +public class PrometheusSinkService { + + private static final Logger LOG = LoggerFactory.getLogger(PrometheusSinkService.class); + + public static final String HTTP_SINK_RECORDS_SUCCESS_COUNTER = "httpSinkRecordsSuccessPushToEndPoint"; + + public static final String HTTP_SINK_RECORDS_FAILED_COUNTER = "httpSinkRecordsFailedToPushEndPoint"; + + private final Collection bufferedEventHandles; + + private final PrometheusSinkConfiguration prometheusSinkConfiguration; + + private final BufferFactory bufferFactory; + + private final Map httpAuthOptions; + + private DlqPushHandler dlqPushHandler; + + private final Lock reentrantLock; + + private final HttpClientBuilder httpClientBuilder; + + private final Counter httpSinkRecordsSuccessCounter; + + private final Counter httpSinkRecordsFailedCounter; + + private final OAuthAccessTokenManager oAuthAccessTokenManager; + + private CertificateProviderFactory certificateProviderFactory; + + private HttpClientConnectionManager httpClientConnectionManager; + + private Buffer currentBuffer; + + private final PluginSetting httpPluginSetting; + + private MultiAuthPrometheusSinkHandler multiAuthPrometheusSinkHandler; + + private static final ObjectMapper objectMapper = new ObjectMapper(); + + private static final Pattern PREFIX_PATTERN = Pattern.compile("^[^a-zA-Z_:]"); + private static final Pattern BODY_PATTERN = Pattern.compile("[^a-zA-Z0-9_:]"); + + public PrometheusSinkService(final PrometheusSinkConfiguration prometheusSinkConfiguration, + final BufferFactory bufferFactory, + final DlqPushHandler dlqPushHandler, + final HttpClientBuilder httpClientBuilder, + final PluginMetrics pluginMetrics, + final PluginSetting httpPluginSetting){ + this.prometheusSinkConfiguration = prometheusSinkConfiguration; + this.bufferFactory = bufferFactory; + this.dlqPushHandler = dlqPushHandler; + this.reentrantLock = new ReentrantLock(); + this.bufferedEventHandles = new LinkedList<>(); + this.httpClientBuilder = httpClientBuilder; + this.httpPluginSetting = httpPluginSetting; + this.oAuthAccessTokenManager = new OAuthAccessTokenManager(httpClientBuilder); + if (prometheusSinkConfiguration.isSsl() || prometheusSinkConfiguration.useAcmCertForSSL()) { + this.certificateProviderFactory = new CertificateProviderFactory(prometheusSinkConfiguration); + prometheusSinkConfiguration.validateAndInitializeCertAndKeyFileInS3(); + this.httpClientConnectionManager = new HttpClientSSLConnectionManager() + .createHttpClientConnectionManager(prometheusSinkConfiguration, certificateProviderFactory); + } + this.httpAuthOptions = buildAuthHttpSinkObjectsByConfig(prometheusSinkConfiguration); + this.httpSinkRecordsSuccessCounter = pluginMetrics.counter(HTTP_SINK_RECORDS_SUCCESS_COUNTER); + this.httpSinkRecordsFailedCounter = pluginMetrics.counter(HTTP_SINK_RECORDS_FAILED_COUNTER); + } + + /** + * This method process buffer records and send to Http End points based on configured codec + * @param records Collection of Event + */ + public void output(Collection> records) { + reentrantLock.lock(); + if (currentBuffer == null) { + this.currentBuffer = bufferFactory.getBuffer(); + } + try { + records.forEach(record -> { + final Event event = record.getData(); + byte[] bytes = null; + if (event.getMetadata().getEventType().equals("METRIC")) { + Remote.WriteRequest message = null; + if (event instanceof JacksonGauge) { + JacksonGauge jacksonGauge = (JacksonGauge) event; + message = buildRemoteWriteRequest(jacksonGauge.getTime(), + jacksonGauge.getStartTime(), jacksonGauge.getValue(), jacksonGauge.getAttributes()); + } else if (event instanceof JacksonSum) { + JacksonSum jacksonSum = (JacksonSum) event; + message = buildRemoteWriteRequest(jacksonSum.getTime(), + jacksonSum.getStartTime(), jacksonSum.getValue(), jacksonSum.getAttributes()); + } else if (event instanceof JacksonSummary) { + JacksonSummary jacksonSummary = (JacksonSummary) event; + message = buildRemoteWriteRequest(jacksonSummary.getTime(), + jacksonSummary.getStartTime(), jacksonSummary.getSum(), jacksonSummary.getAttributes()); + } else if (event instanceof JacksonHistogram) { + JacksonHistogram jacksonHistogram = (JacksonHistogram) event; + message = buildRemoteWriteRequest(jacksonHistogram.getTime(), + jacksonHistogram.getStartTime(), jacksonHistogram.getSum(), jacksonHistogram.getAttributes()); + } else if (event instanceof JacksonExponentialHistogram) { + JacksonExponentialHistogram jacksonExpHistogram = (JacksonExponentialHistogram) event; + message = buildRemoteWriteRequest(jacksonExpHistogram.getTime(), + jacksonExpHistogram.getStartTime(), jacksonExpHistogram.getSum(), jacksonExpHistogram.getAttributes()); + } else { + LOG.error("No valid Event type found"); + } + bytes = message.toByteArray(); + } + if (event.getEventHandle() != null) { + this.bufferedEventHandles.add(event.getEventHandle()); + } + HttpEndPointResponse failedHttpEndPointResponses = null; + try { + failedHttpEndPointResponses = pushToEndPoint(bytes); + } catch (IOException e) { + LOG.info("Error while pushing to the end point"); + } + if (failedHttpEndPointResponses != null) { + logFailedData(failedHttpEndPointResponses); + releaseEventHandles(Boolean.FALSE); + } else { + LOG.info("data pushed to the end point successfully"); + releaseEventHandles(Boolean.TRUE); + } + + }); + + }finally { + reentrantLock.unlock(); + } + } + + /** + * * This method build Remote.WriteRequest + * @param time time + * @param startTime start time + * @param value value + * @param attributeMap attributes + */ + private static Remote.WriteRequest buildRemoteWriteRequest(String time, String startTime, + Double value, Map attributeMap) { + Remote.WriteRequest.Builder writeRequestBuilder = Remote.WriteRequest.newBuilder(); + + Types.TimeSeries.Builder timeSeriesBuilder = Types.TimeSeries.newBuilder(); + + List arrayList = new ArrayList<>(); + + prepareLabelList(attributeMap, arrayList); + + Types.Sample.Builder prometheusSampleBuilder = Types.Sample.newBuilder(); + long timeStampVal; + if (time != null) { + timeStampVal = getTimeStampVal(time); + } else { + timeStampVal = getTimeStampVal(startTime); + } + + prometheusSampleBuilder.setValue(value).setTimestamp(timeStampVal); + Types.Sample prometheusSample = prometheusSampleBuilder.build(); + + timeSeriesBuilder.addAllLabels(arrayList); + timeSeriesBuilder.addAllSamples(Arrays.asList(prometheusSample)); + + Types.TimeSeries timeSeries = timeSeriesBuilder.build(); + writeRequestBuilder.addAllTimeseries(Arrays.asList(timeSeries)); + + return writeRequestBuilder.build(); + } + + private static void prepareLabelList(Map hashMap, List arrayList) { + for (Map.Entry entry : hashMap.entrySet()) { + String key = sanitizeName(entry.getKey()); + Object value = entry.getValue(); + if (entry.getValue() instanceof Map) { + Object innerMap = entry.getValue(); + prepareLabelList(objectMapper.convertValue(innerMap, Map.class), arrayList); + continue; + } + Types.Label.Builder labelBuilder = Types.Label.newBuilder(); + labelBuilder.setName(key).setValue(value.toString()); + Types.Label label = labelBuilder.build(); + arrayList.add(label); + } + } + + private static String sanitizeName(String name) { + return BODY_PATTERN + .matcher(PREFIX_PATTERN.matcher(name).replaceFirst("_")) + .replaceAll("_"); + } + + private static long getTimeStampVal(String time) { + LocalDateTime localDateTimeParse = LocalDateTime.parse(time, + DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSSSSSSS'Z'")); + LocalDateTime localDateTime = LocalDateTime.parse(localDateTimeParse.toString()); + ZonedDateTime zdt = ZonedDateTime.of(localDateTime, ZoneId.systemDefault()); + return zdt.toInstant().toEpochMilli(); + } + + /** + * * This method logs Failed Data to DLQ and Webhook + * @param endPointResponses HttpEndPointResponses. + */ + private void logFailedData(final HttpEndPointResponse endPointResponses) { + FailedDlqData failedDlqData = + FailedDlqData.builder() + .withUrl(endPointResponses.getUrl()) + .withMessage(endPointResponses.getErrorMessage()) + .withStatus(endPointResponses.getStatusCode()).build(); + LOG.info("Failed to push the data. Failed DLQ Data: {}",failedDlqData); + + logFailureForDlqObjects(failedDlqData); + } + + private void releaseEventHandles(final boolean result) { + for (EventHandle eventHandle : bufferedEventHandles) { + eventHandle.release(result); + } + bufferedEventHandles.clear(); + } + + /** + * * This method pushes bufferData to configured HttpEndPoints + * @param currentBufferData bufferData. + */ + private HttpEndPointResponse pushToEndPoint(final byte[] currentBufferData) throws IOException { + HttpEndPointResponse httpEndPointResponses = null; + final ClassicRequestBuilder classicHttpRequestBuilder = + httpAuthOptions.get(prometheusSinkConfiguration.getUrl()).getClassicHttpRequestBuilder(); + + final byte[] compressedBufferData = Snappy.compress(currentBufferData); + + HttpEntity entity = new ByteArrayEntity(compressedBufferData, + ContentType.create(prometheusSinkConfiguration.getContentType()), prometheusSinkConfiguration.getEncoding()); + + classicHttpRequestBuilder.setEntity(entity); + classicHttpRequestBuilder.addHeader("Content-Encoding", prometheusSinkConfiguration.getEncoding()); + classicHttpRequestBuilder.addHeader("Content-Type", prometheusSinkConfiguration.getContentType()); + classicHttpRequestBuilder.addHeader("X-Prometheus-Remote-Write-Version", prometheusSinkConfiguration.getRemoteWriteVersion()); + + try { + if(AuthTypeOptions.BEARER_TOKEN.equals(prometheusSinkConfiguration.getAuthType())) + accessTokenIfExpired(classicHttpRequestBuilder.getFirstHeader(AUTHORIZATION).getValue(),prometheusSinkConfiguration.getUrl()); + + httpAuthOptions.get(prometheusSinkConfiguration.getUrl()).getHttpClientBuilder().build() + .execute(classicHttpRequestBuilder.build(), HttpClientContext.create()); + LOG.info("No of Records successfully pushed to endpoint {}", prometheusSinkConfiguration.getUrl() +" " + currentBuffer.getEventCount()); + httpSinkRecordsSuccessCounter.increment(currentBuffer.getEventCount()); + } catch (IOException e) { + httpSinkRecordsFailedCounter.increment(currentBuffer.getEventCount()); + LOG.info("No of Records failed to push endpoint {}",currentBuffer.getEventCount()); + LOG.error("Exception while pushing buffer data to end point. URL : {}, Exception : ", prometheusSinkConfiguration.getUrl(), e); + httpEndPointResponses = new HttpEndPointResponse(prometheusSinkConfiguration.getUrl(), HttpStatus.SC_INTERNAL_SERVER_ERROR, e.getMessage()); + } + return httpEndPointResponses; + } + + /** + * * This method sends Failed objects to DLQ + * @param failedDlqData FailedDlqData. + */ + private void logFailureForDlqObjects(final FailedDlqData failedDlqData){ + dlqPushHandler.perform(httpPluginSetting, failedDlqData); + } + + + /** + * * This method gets Auth Handler classes based on configuration + * @param authType AuthTypeOptions. + * @param authOptions HttpAuthOptions.Builder. + */ + private HttpAuthOptions getAuthHandlerByConfig(final AuthTypeOptions authType, + final HttpAuthOptions.Builder authOptions){ + switch(authType) { + case HTTP_BASIC: + multiAuthPrometheusSinkHandler = new BasicAuthPrometheusSinkHandler( + prometheusSinkConfiguration.getAuthentication().getHttpBasic().getUsername(), + prometheusSinkConfiguration.getAuthentication().getHttpBasic().getPassword(), + httpClientConnectionManager); + break; + case BEARER_TOKEN: + multiAuthPrometheusSinkHandler = new BearerTokenAuthPrometheusSinkHandler( + prometheusSinkConfiguration.getAuthentication().getBearerTokenOptions(), + httpClientConnectionManager, oAuthAccessTokenManager); + break; + case UNAUTHENTICATED: + default: + return authOptions.setHttpClientBuilder(httpClientBuilder + .setConnectionManager(httpClientConnectionManager) + .addResponseInterceptorLast(new FailedHttpResponseInterceptor(authOptions.getUrl()))).build(); + } + return multiAuthPrometheusSinkHandler.authenticate(authOptions); + } + + /** + * * This method build HttpAuthOptions class based on configurations + * @param prometheusSinkConfiguration PrometheusSinkConfiguration. + */ + private Map buildAuthHttpSinkObjectsByConfig(final PrometheusSinkConfiguration prometheusSinkConfiguration){ + final Map authMap = new HashMap<>(); + + final HTTPMethodOptions httpMethod = prometheusSinkConfiguration.getHttpMethod(); + final AuthTypeOptions authType = prometheusSinkConfiguration.getAuthType(); + final String proxyUrlString = prometheusSinkConfiguration.getProxy(); + final ClassicRequestBuilder classicRequestBuilder = buildRequestByHTTPMethodType(httpMethod).setUri(prometheusSinkConfiguration.getUrl()); + + + + if(Objects.nonNull(prometheusSinkConfiguration.getCustomHeaderOptions())) + addCustomHeaders(classicRequestBuilder,prometheusSinkConfiguration.getCustomHeaderOptions()); + + if(Objects.nonNull(proxyUrlString)) { + httpClientBuilder.setProxy(PrometheusSinkUtil.getHttpHostByURL(PrometheusSinkUtil.getURLByUrlString(proxyUrlString))); + LOG.info("sending data via proxy {}",proxyUrlString); + } + + final HttpAuthOptions.Builder authOptions = new HttpAuthOptions.Builder() + .setUrl(prometheusSinkConfiguration.getUrl()) + .setClassicHttpRequestBuilder(classicRequestBuilder) + .setHttpClientBuilder(httpClientBuilder); + + authMap.put(prometheusSinkConfiguration.getUrl(),getAuthHandlerByConfig(authType,authOptions)); + return authMap; + } + + /** + * * This method adds SageMakerHeaders as custom Header in the request + * @param classicRequestBuilder ClassicRequestBuilder. + * @param customHeaderOptions CustomHeaderOptions . + */ + private void addCustomHeaders(final ClassicRequestBuilder classicRequestBuilder, + final Map> customHeaderOptions) { + + customHeaderOptions.forEach((k, v) -> classicRequestBuilder.addHeader(k,v.toString())); + } + + /** + * * builds ClassicRequestBuilder based on configured HttpMethod + * @param httpMethodOptions Http Method. + */ + private ClassicRequestBuilder buildRequestByHTTPMethodType(final HTTPMethodOptions httpMethodOptions) { + final ClassicRequestBuilder classicRequestBuilder; + switch (httpMethodOptions) { + case PUT: + classicRequestBuilder = ClassicRequestBuilder.put(); + break; + case POST: + default: + classicRequestBuilder = ClassicRequestBuilder.post(); + break; + } + return classicRequestBuilder; + } + + private void accessTokenIfExpired(final String token,final String url){ + if(oAuthAccessTokenManager.isTokenExpired(token)) { + httpAuthOptions.get(url).getClassicHttpRequestBuilder() + .setHeader(AUTHORIZATION, oAuthAccessTokenManager.getAccessToken(prometheusSinkConfiguration.getAuthentication().getBearerTokenOptions())); + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/util/PrometheusSinkUtil.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/util/PrometheusSinkUtil.java new file mode 100644 index 0000000000..247028f800 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/util/PrometheusSinkUtil.java @@ -0,0 +1,34 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.util; + +import org.apache.hc.core5.http.HttpHost; + +import java.net.MalformedURLException; +import java.net.URISyntaxException; +import java.net.URL; + +public class PrometheusSinkUtil { + + public static URL getURLByUrlString(final String url) { + try { + return new URL(url); + } catch (MalformedURLException e) { + throw new RuntimeException(e); + } + } + + public static HttpHost getHttpHostByURL(final URL url) { + final HttpHost targetHost; + try { + targetHost = new HttpHost(url.toURI().getScheme(), url.getHost(), url.getPort()); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + return targetHost; + } + + +} diff --git a/settings.gradle b/settings.gradle index 67c85c3a12..2c8e0547ae 100644 --- a/settings.gradle +++ b/settings.gradle @@ -128,3 +128,4 @@ include 'data-prepper-plugins:buffer-common' include 'data-prepper-plugins:sqs-source' include 'data-prepper-plugins:cloudwatch-logs' include 'data-prepper-plugins:http-sink' +include 'data-prepper-plugins:prometheus-sink' \ No newline at end of file From 8882a83c09108f92b697ee7240ee3c7267117513 Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Tue, 8 Aug 2023 12:59:32 +0530 Subject: [PATCH 02/18] Added metrics name in Prometheus Sink for issue #1744. Signed-off-by: mallikagogoi7 --- .../PrometheusSinkConfiguration.java | 16 -------------- .../service/PrometheusSinkService.java | 21 +++++++++++++------ 2 files changed, 15 insertions(+), 22 deletions(-) diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java index 3df5c623c9..9068f1885e 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java @@ -22,8 +22,6 @@ public class PrometheusSinkConfiguration { private static final int DEFAULT_UPLOAD_RETRIES = 5; - private static final int DEFAULT_WORKERS = 1; - static final boolean DEFAULT_SSL = false; private static final String S3_PREFIX = "s3://"; @@ -61,12 +59,6 @@ public class PrometheusSinkConfiguration { @JsonProperty("url") private String url; - @JsonProperty("workers") - private Integer workers = DEFAULT_WORKERS; - - @JsonProperty("codec") - private PluginModel codec; - @JsonProperty("http_method") private HTTPMethodOptions httpMethod = HTTPMethodOptions.POST; @@ -198,10 +190,6 @@ public String getAcmCertificateArn() { return acmCertificateArn; } - public PluginModel getCodec() { - return codec; - } - public HTTPMethodOptions getHttpMethod() { return httpMethod; } @@ -247,10 +235,6 @@ public AwsAuthenticationOptions getAwsAuthenticationOptions() { return awsAuthenticationOptions; } - public Integer getWorkers() { - return workers; - } - public String getDlqFile() { return dlqFile; } diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java index 6a2478489d..8c04354692 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java @@ -157,23 +157,23 @@ public void output(Collection> records) { if (event instanceof JacksonGauge) { JacksonGauge jacksonGauge = (JacksonGauge) event; message = buildRemoteWriteRequest(jacksonGauge.getTime(), - jacksonGauge.getStartTime(), jacksonGauge.getValue(), jacksonGauge.getAttributes()); + jacksonGauge.getStartTime(), jacksonGauge.getValue(), jacksonGauge.getAttributes(),jacksonGauge.getName()); } else if (event instanceof JacksonSum) { JacksonSum jacksonSum = (JacksonSum) event; message = buildRemoteWriteRequest(jacksonSum.getTime(), - jacksonSum.getStartTime(), jacksonSum.getValue(), jacksonSum.getAttributes()); + jacksonSum.getStartTime(), jacksonSum.getValue(), jacksonSum.getAttributes(), jacksonSum.getName()); } else if (event instanceof JacksonSummary) { JacksonSummary jacksonSummary = (JacksonSummary) event; message = buildRemoteWriteRequest(jacksonSummary.getTime(), - jacksonSummary.getStartTime(), jacksonSummary.getSum(), jacksonSummary.getAttributes()); + jacksonSummary.getStartTime(), jacksonSummary.getSum(), jacksonSummary.getAttributes(), jacksonSummary.getName()); } else if (event instanceof JacksonHistogram) { JacksonHistogram jacksonHistogram = (JacksonHistogram) event; message = buildRemoteWriteRequest(jacksonHistogram.getTime(), - jacksonHistogram.getStartTime(), jacksonHistogram.getSum(), jacksonHistogram.getAttributes()); + jacksonHistogram.getStartTime(), jacksonHistogram.getSum(), jacksonHistogram.getAttributes(), jacksonHistogram.getName()); } else if (event instanceof JacksonExponentialHistogram) { JacksonExponentialHistogram jacksonExpHistogram = (JacksonExponentialHistogram) event; message = buildRemoteWriteRequest(jacksonExpHistogram.getTime(), - jacksonExpHistogram.getStartTime(), jacksonExpHistogram.getSum(), jacksonExpHistogram.getAttributes()); + jacksonExpHistogram.getStartTime(), jacksonExpHistogram.getSum(), jacksonExpHistogram.getAttributes(), jacksonExpHistogram.getName()); } else { LOG.error("No valid Event type found"); } @@ -209,15 +209,17 @@ public void output(Collection> records) { * @param startTime start time * @param value value * @param attributeMap attributes + * @param metricName metricName */ private static Remote.WriteRequest buildRemoteWriteRequest(String time, String startTime, - Double value, Map attributeMap) { + Double value, Map attributeMap, final String metricName) { Remote.WriteRequest.Builder writeRequestBuilder = Remote.WriteRequest.newBuilder(); Types.TimeSeries.Builder timeSeriesBuilder = Types.TimeSeries.newBuilder(); List arrayList = new ArrayList<>(); + setMetricName(metricName, arrayList); prepareLabelList(attributeMap, arrayList); Types.Sample.Builder prometheusSampleBuilder = Types.Sample.newBuilder(); @@ -431,4 +433,11 @@ private void accessTokenIfExpired(final String token,final String url){ .setHeader(AUTHORIZATION, oAuthAccessTokenManager.getAccessToken(prometheusSinkConfiguration.getAuthentication().getBearerTokenOptions())); } } + + private static void setMetricName(final String metricName, final List arrayList) { + final Types.Label.Builder labelBuilder = Types.Label.newBuilder(); + labelBuilder.setName("__name__").setValue(metricName); + final Types.Label label = labelBuilder.build(); + arrayList.add(label); + } } \ No newline at end of file From 51e7f44bd4dbbb3345409cf5901adf8269cb0173 Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Wed, 9 Aug 2023 18:58:09 +0530 Subject: [PATCH 03/18] Added junit test for Prometheus Sink for issue #1744. Signed-off-by: mallikagogoi7 --- .../sink/prometheus/PrometheusSink.java | 13 -- .../configuration/AuthenticationOptions.java | 4 +- .../PrometheusSinkConfiguration.java | 9 - .../sink/prometheus/dlq/DlqPushHandler.java | 2 +- .../service/PrometheusSinkService.java | 36 +-- .../FailedHttpResponseInterceptorTest.java | 35 +++ .../OAuthAccessTokenManagerTest.java | 81 +++++++ .../sink/prometheus/PrometheusSinkTest.java | 102 ++++++++ .../CertificateProviderFactoryTest.java | 76 ++++++ .../HttpClientSSLConnectionManagerTest.java | 50 ++++ .../AwsAuthenticationOptionsTest.java | 83 +++++++ .../PrometheusSinkConfigurationTest.java | 220 ++++++++++++++++++ .../prometheus/dlq/DlqPushHandlerTest.java | 96 ++++++++ .../BasicAuthPrometheusSinkHandlerTest.java | 52 +++++ ...rerTokenAuthPrometheusSinkHandlerTest.java | 44 ++++ .../service/PrometheusSinkAwsServiceTest.java | 55 +++++ .../util/PrometheusSinkUtilTest.java | 34 +++ .../org.mockito.plugins.MockMaker | 3 + .../src/test/resources/test_cert.crt | 14 ++ .../src/test/resources/test_decrypted_key.key | 15 ++ 20 files changed, 969 insertions(+), 55 deletions(-) create mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptorTest.java create mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManagerTest.java create mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkTest.java create mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactoryTest.java create mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManagerTest.java create mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptionsTest.java create mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java create mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandlerTest.java create mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BasicAuthPrometheusSinkHandlerTest.java create mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BearerTokenAuthPrometheusSinkHandlerTest.java create mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsServiceTest.java create mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/util/PrometheusSinkUtilTest.java create mode 100644 data-prepper-plugins/prometheus-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker create mode 100644 data-prepper-plugins/prometheus-sink/src/test/resources/test_cert.crt create mode 100644 data-prepper-plugins/prometheus-sink/src/test/resources/test_decrypted_key.key diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java index 6dbaa35586..803a8f429e 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java @@ -20,10 +20,6 @@ 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.accumulator.BufferFactory; -import org.opensearch.dataprepper.plugins.accumulator.BufferTypeOptions; -import org.opensearch.dataprepper.plugins.accumulator.InMemoryBufferFactory; -import org.opensearch.dataprepper.plugins.accumulator.LocalFileBufferFactory; import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; import org.opensearch.dataprepper.plugins.sink.prometheus.dlq.DlqPushHandler; import org.opensearch.dataprepper.plugins.sink.prometheus.service.PrometheusSinkAwsService; @@ -45,8 +41,6 @@ public class PrometheusSink extends AbstractSink> { private final PrometheusSinkService prometheusSinkService; - private final BufferFactory bufferFactory; - private DlqPushHandler dlqPushHandler; @DataPrepperPluginConstructor @@ -57,12 +51,6 @@ public PrometheusSink(final PluginSetting pluginSetting, final AwsCredentialsSupplier awsCredentialsSupplier) { super(pluginSetting); this.sinkInitialized = Boolean.FALSE; - if (prometheusSinkConfiguration.getBufferType().equals(BufferTypeOptions.LOCALFILE)) { - this.bufferFactory = new LocalFileBufferFactory(); - } else { - this.bufferFactory = new InMemoryBufferFactory(); - } - this.dlqPushHandler = new DlqPushHandler(prometheusSinkConfiguration.getDlqFile(), pluginFactory, String.valueOf(prometheusSinkConfiguration.getDlqPluginSetting().get(BUCKET)), prometheusSinkConfiguration.getDlqStsRoleARN() @@ -80,7 +68,6 @@ public PrometheusSink(final PluginSetting pluginSetting, } this.prometheusSinkService = new PrometheusSinkService( prometheusSinkConfiguration, - bufferFactory, dlqPushHandler, httpClientBuilder, pluginMetrics, diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java index e5bfa60534..908abe9d62 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java @@ -8,10 +8,10 @@ public class AuthenticationOptions { - @JsonProperty("http_basic") + @JsonProperty("http-basic") private BasicAuthCredentials httpBasic; - @JsonProperty("bearer_token") + @JsonProperty("bearer-token") private BearerTokenOptions bearerTokenOptions; public BasicAuthCredentials getHttpBasic() { diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java index 9068f1885e..72c9e47500 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java @@ -9,7 +9,6 @@ import jakarta.validation.constraints.NotNull; import org.apache.commons.lang3.StringUtils; import org.opensearch.dataprepper.model.configuration.PluginModel; -import org.opensearch.dataprepper.plugins.accumulator.BufferTypeOptions; import org.opensearch.dataprepper.plugins.sink.prometheus.util.PrometheusSinkUtil; import java.net.URL; @@ -80,9 +79,6 @@ public class PrometheusSinkConfiguration { @JsonProperty("aws_sigv4") private boolean awsSigv4; - @JsonProperty("buffer_type") - private BufferTypeOptions bufferType = BufferTypeOptions.INMEMORY; - @JsonProperty("max_retries") private int maxUploadRetries = DEFAULT_UPLOAD_RETRIES; @@ -218,11 +214,6 @@ public boolean isAwsSigv4() { return awsSigv4; } - public BufferTypeOptions getBufferType() { - return bufferType; - } - - public int getMaxUploadRetries() { return maxUploadRetries; } diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandler.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandler.java index a192b0965d..0d240eeea5 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandler.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandler.java @@ -64,7 +64,7 @@ public DlqPushHandler(final String dlqFile, final String dlqPathPrefix) { if(dlqFile != null) { this.dlqFile = dlqFile; - this.objectWriter = new ObjectMapper().writer().withDefaultPrettyPrinter(); + this.objectWriter = new ObjectMapper().writer(); }else{ this.dlqProvider = getDlqProvider(pluginFactory,bucket,stsRoleArn,awsRegion,dlqPathPrefix); } diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java index 8c04354692..5d4b019f5c 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java @@ -7,7 +7,6 @@ import com.arpnetworking.metrics.prometheus.Remote; import com.arpnetworking.metrics.prometheus.Types; import com.fasterxml.jackson.databind.ObjectMapper; -import io.micrometer.core.instrument.Counter; import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; import org.apache.hc.client5.http.io.HttpClientConnectionManager; import org.apache.hc.client5.http.protocol.HttpClientContext; @@ -27,8 +26,6 @@ import org.opensearch.dataprepper.model.metric.JacksonSummary; import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.accumulator.Buffer; -import org.opensearch.dataprepper.plugins.accumulator.BufferFactory; import org.opensearch.dataprepper.plugins.sink.prometheus.certificate.CertificateProviderFactory; import org.opensearch.dataprepper.plugins.sink.prometheus.FailedHttpResponseInterceptor; import org.opensearch.dataprepper.plugins.sink.prometheus.HttpEndPointResponse; @@ -75,16 +72,10 @@ public class PrometheusSinkService { private static final Logger LOG = LoggerFactory.getLogger(PrometheusSinkService.class); - public static final String HTTP_SINK_RECORDS_SUCCESS_COUNTER = "httpSinkRecordsSuccessPushToEndPoint"; - - public static final String HTTP_SINK_RECORDS_FAILED_COUNTER = "httpSinkRecordsFailedToPushEndPoint"; - private final Collection bufferedEventHandles; private final PrometheusSinkConfiguration prometheusSinkConfiguration; - private final BufferFactory bufferFactory; - private final Map httpAuthOptions; private DlqPushHandler dlqPushHandler; @@ -93,18 +84,12 @@ public class PrometheusSinkService { private final HttpClientBuilder httpClientBuilder; - private final Counter httpSinkRecordsSuccessCounter; - - private final Counter httpSinkRecordsFailedCounter; - private final OAuthAccessTokenManager oAuthAccessTokenManager; private CertificateProviderFactory certificateProviderFactory; private HttpClientConnectionManager httpClientConnectionManager; - private Buffer currentBuffer; - private final PluginSetting httpPluginSetting; private MultiAuthPrometheusSinkHandler multiAuthPrometheusSinkHandler; @@ -115,13 +100,11 @@ public class PrometheusSinkService { private static final Pattern BODY_PATTERN = Pattern.compile("[^a-zA-Z0-9_:]"); public PrometheusSinkService(final PrometheusSinkConfiguration prometheusSinkConfiguration, - final BufferFactory bufferFactory, final DlqPushHandler dlqPushHandler, final HttpClientBuilder httpClientBuilder, final PluginMetrics pluginMetrics, final PluginSetting httpPluginSetting){ this.prometheusSinkConfiguration = prometheusSinkConfiguration; - this.bufferFactory = bufferFactory; this.dlqPushHandler = dlqPushHandler; this.reentrantLock = new ReentrantLock(); this.bufferedEventHandles = new LinkedList<>(); @@ -135,8 +118,6 @@ public PrometheusSinkService(final PrometheusSinkConfiguration prometheusSinkCon .createHttpClientConnectionManager(prometheusSinkConfiguration, certificateProviderFactory); } this.httpAuthOptions = buildAuthHttpSinkObjectsByConfig(prometheusSinkConfiguration); - this.httpSinkRecordsSuccessCounter = pluginMetrics.counter(HTTP_SINK_RECORDS_SUCCESS_COUNTER); - this.httpSinkRecordsFailedCounter = pluginMetrics.counter(HTTP_SINK_RECORDS_FAILED_COUNTER); } /** @@ -145,9 +126,6 @@ public PrometheusSinkService(final PrometheusSinkConfiguration prometheusSinkCon */ public void output(Collection> records) { reentrantLock.lock(); - if (currentBuffer == null) { - this.currentBuffer = bufferFactory.getBuffer(); - } try { records.forEach(record -> { final Event event = record.getData(); @@ -296,15 +274,15 @@ private void releaseEventHandles(final boolean result) { /** * * This method pushes bufferData to configured HttpEndPoints - * @param currentBufferData bufferData. + * @param data byte[] data. */ - private HttpEndPointResponse pushToEndPoint(final byte[] currentBufferData) throws IOException { + private HttpEndPointResponse pushToEndPoint(final byte[] data) throws IOException { HttpEndPointResponse httpEndPointResponses = null; final ClassicRequestBuilder classicHttpRequestBuilder = httpAuthOptions.get(prometheusSinkConfiguration.getUrl()).getClassicHttpRequestBuilder(); - final byte[] compressedBufferData = Snappy.compress(currentBufferData); - + final byte[] compressedBufferData = Snappy.compress(data); + LOG.info("******* compressedBufferData ***** "+ compressedBufferData); HttpEntity entity = new ByteArrayEntity(compressedBufferData, ContentType.create(prometheusSinkConfiguration.getContentType()), prometheusSinkConfiguration.getEncoding()); @@ -319,11 +297,9 @@ private HttpEndPointResponse pushToEndPoint(final byte[] currentBufferData) thro httpAuthOptions.get(prometheusSinkConfiguration.getUrl()).getHttpClientBuilder().build() .execute(classicHttpRequestBuilder.build(), HttpClientContext.create()); - LOG.info("No of Records successfully pushed to endpoint {}", prometheusSinkConfiguration.getUrl() +" " + currentBuffer.getEventCount()); - httpSinkRecordsSuccessCounter.increment(currentBuffer.getEventCount()); + LOG.info("Records successfully pushed to endpoint {}", prometheusSinkConfiguration.getUrl()); } catch (IOException e) { - httpSinkRecordsFailedCounter.increment(currentBuffer.getEventCount()); - LOG.info("No of Records failed to push endpoint {}",currentBuffer.getEventCount()); + LOG.info("Records failed to push endpoint {}"); LOG.error("Exception while pushing buffer data to end point. URL : {}, Exception : ", prometheusSinkConfiguration.getUrl(), e); httpEndPointResponses = new HttpEndPointResponse(prometheusSinkConfiguration.getUrl(), HttpStatus.SC_INTERNAL_SERVER_ERROR, e.getMessage()); } diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptorTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptorTest.java new file mode 100644 index 0000000000..e0601506fe --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptorTest.java @@ -0,0 +1,35 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus; + +import org.apache.hc.core5.http.EntityDetails; +import org.apache.hc.core5.http.HttpResponse; +import org.apache.hc.core5.http.protocol.HttpContext; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class FailedHttpResponseInterceptorTest { + + private FailedHttpResponseInterceptor failedHttpResponseInterceptor; + + private HttpResponse httpResponse; + + private EntityDetails entityDetails; + + private HttpContext httpContext; + + @Test + public void test_process(){ + httpResponse = mock(HttpResponse.class); + failedHttpResponseInterceptor = new FailedHttpResponseInterceptor("http://localhost:8080"); + when(httpResponse.getCode()).thenReturn(501); + assertThrows(IOException.class, () -> failedHttpResponseInterceptor.process(httpResponse, entityDetails, httpContext)); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManagerTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManagerTest.java new file mode 100644 index 0000000000..18cba54d6e --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManagerTest.java @@ -0,0 +1,81 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; +import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; +import org.apache.hc.client5.http.impl.classic.CloseableHttpResponse; +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; +import org.apache.hc.core5.http.ClassicHttpRequest; +import org.apache.hc.core5.http.HttpEntity; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.BearerTokenOptions; + +import java.io.ByteArrayInputStream; +import java.io.IOException; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class OAuthAccessTokenManagerTest { + + private static final String bearerTokenYaml = + " client_id: 0oaafr4j79segrYGC5d7\n" + + " client_secret: fFel-3FutCXAOndezEsOVlght6D6DR4OIt7G5D1_oJ6w0wNoaYtgU17JdyXmGf0M\n" + + " token_url: https://localhost/oauth2/default/v1/token\n" + + " grant_type: client_credentials\n" + + " scope: httpSink"; + + private final String tokenJson = "{\"token_type\": \"Bearer\",\n" + + " \"expires_in\": 3600,\n" + + " \"access_token\": \"eyJraWQiOiJtU0xMalBfMUFFUFV1VzlqRkhiSmc4UXlRSm1pdFBHamZOczR2eFJ2WUx3IiwiYWxnIjoiUlMyNTYifQ.eyJ2ZXIiOjEsImp0aSI6IkFULm1jcVVfTzJvd1RuNUwwTjYwLTg0WFBaUU4xNXBfX191X2VKaGxBdEhsQXciLCJpc3MiOiJodHRwczovL2Rldi03NTA1MDk1Ni5va3RhLmNvbS9vYXV0aDIvZGVmYXVsdCIsImF1ZCI6ImFwaTovL2RlZmF1bHQiLCJpYXQiOjE2ODk5MTQyNDMsImV4cCI6MTY4OTkxNzg0MywiY2lkIjoiMG9hYWZyNGo3OXNlZ3JZR0M1ZDciLCJzY3AiOlsiaHR0cFNpbmsiXSwic3ViIjoiMG9hYWZyNGo3OXNlZ3JZR0M1ZDcifQ.d6NU2f9Qlrk9N2L1cfY5KhWIc7DXE1oJXPsss2OMb-JFYZvwhMpIpv1IwaY7ikDQYDKlcYYt-3XKBj0IxPnugigO_OTv12LpvHyMBhUKDo5YrxKZqksme7S0IKYoLNFVsq3ViqVsHgDy3RGWL1ih-rGXN-8A-9LsqloEnCn7SzFj446aep9bygp1PIA5pBgrVwKw0QPal4HDOu9cTKwclNiWRLJ80H_q83vDeQNnW9YI8A-nTy9ujghVF9JJVsB4FTHMlfclt93SJ4qCA_9He_VFkSs5pFS4plCAzONA0XU53lf7NXJ3bs18HPJkm3-B2b1f6Q9kGUU6e2ZQ2d6dvw\",\n" + + " \"scope\": \"httpSink\"\n" + + "}"; + + private ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory().enable(YAMLGenerator.Feature.USE_PLATFORM_LINE_BREAKS)); + + @Mock + private HttpClientBuilder httpClientBuilder; + + @Mock + private CloseableHttpClient httpClient; + + private OAuthAccessTokenManager oAuthAccessTokenManager; + + @BeforeEach + public void setup() throws IOException { + httpClientBuilder = mock(HttpClientBuilder.class); + httpClient = mock(CloseableHttpClient.class); + CloseableHttpResponse response = mock(CloseableHttpResponse.class); + HttpEntity entity = mock(HttpEntity.class); + + final ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(tokenJson.getBytes()); + when(entity.getContent()).thenReturn(byteArrayInputStream); + when(httpClientBuilder.build()).thenReturn(httpClient); + when(httpClient.execute(any(ClassicHttpRequest.class))).thenReturn(response); + when(response.getEntity()).thenReturn(entity); + this.oAuthAccessTokenManager = new OAuthAccessTokenManager(httpClientBuilder); + } + + @Test + public void bearer_token_refresh_token_test() throws IOException { + BearerTokenOptions bearerTokenOptions = objectMapper.readValue(bearerTokenYaml,BearerTokenOptions.class); + final String refreshToken = oAuthAccessTokenManager.getAccessToken(bearerTokenOptions); + assertThat(refreshToken,equalTo("Bearer eyJraWQiOiJtU0xMalBfMUFFUFV1VzlqRkhiSmc4UXlRSm1pdFBHamZOczR2eFJ2WUx3IiwiYWxnIjoiUlMyNTYifQ.eyJ2ZXIiOjEsImp0aSI6IkFULm1jcVVfTzJvd1RuNUwwTjYwLTg0WFBaUU4xNXBfX191X2VKaGxBdEhsQXciLCJpc3MiOiJodHRwczovL2Rldi03NTA1MDk1Ni5va3RhLmNvbS9vYXV0aDIvZGVmYXVsdCIsImF1ZCI6ImFwaTovL2RlZmF1bHQiLCJpYXQiOjE2ODk5MTQyNDMsImV4cCI6MTY4OTkxNzg0MywiY2lkIjoiMG9hYWZyNGo3OXNlZ3JZR0M1ZDciLCJzY3AiOlsiaHR0cFNpbmsiXSwic3ViIjoiMG9hYWZyNGo3OXNlZ3JZR0M1ZDcifQ.d6NU2f9Qlrk9N2L1cfY5KhWIc7DXE1oJXPsss2OMb-JFYZvwhMpIpv1IwaY7ikDQYDKlcYYt-3XKBj0IxPnugigO_OTv12LpvHyMBhUKDo5YrxKZqksme7S0IKYoLNFVsq3ViqVsHgDy3RGWL1ih-rGXN-8A-9LsqloEnCn7SzFj446aep9bygp1PIA5pBgrVwKw0QPal4HDOu9cTKwclNiWRLJ80H_q83vDeQNnW9YI8A-nTy9ujghVF9JJVsB4FTHMlfclt93SJ4qCA_9He_VFkSs5pFS4plCAzONA0XU53lf7NXJ3bs18HPJkm3-B2b1f6Q9kGUU6e2ZQ2d6dvw")); + } + + @Test + public void bearer_token_refresh_token_expiry_test() throws IOException { + String bearerToken = "\"Bearer eyJraWQiOiJtU0xMalBfMUFFUFV1VzlqRkhiSmc4UXlRSm1pdFBHamZOczR2eFJ2WUx3IiwiYWxnIjoiUlMyNTYifQ.eyJ2ZXIiOjEsImp0aSI6IkFULm1jcVVfTzJvd1RuNUwwTjYwLTg0WFBaUU4xNXBfX191X2VKaGxBdEhsQXciLCJpc3MiOiJodHRwczovL2Rldi03NTA1MDk1Ni5va3RhLmNvbS9vYXV0aDIvZGVmYXVsdCIsImF1ZCI6ImFwaTovL2RlZmF1bHQiLCJpYXQiOjE2ODk5MTQyNDMsImV4cCI6MTY4OTkxNzg0MywiY2lkIjoiMG9hYWZyNGo3OXNlZ3JZR0M1ZDciLCJzY3AiOlsiaHR0cFNpbmsiXSwic3ViIjoiMG9hYWZyNGo3OXNlZ3JZR0M1ZDcifQ.d6NU2f9Qlrk9N2L1cfY5KhWIc7DXE1oJXPsss2OMb-JFYZvwhMpIpv1IwaY7ikDQYDKlcYYt-3XKBj0IxPnugigO_OTv12LpvHyMBhUKDo5YrxKZqksme7S0IKYoLNFVsq3ViqVsHgDy3RGWL1ih-rGXN-8A-9LsqloEnCn7SzFj446aep9bygp1PIA5pBgrVwKw0QPal4HDOu9cTKwclNiWRLJ80H_q83vDeQNnW9YI8A-nTy9ujghVF9JJVsB4FTHMlfclt93SJ4qCA_9He_VFkSs5pFS4plCAzONA0XU53lf7NXJ3bs18HPJkm3-B2b1f6Q9kGUU6e2ZQ2d6dvw\""; + final boolean refreshToken = oAuthAccessTokenManager.isTokenExpired(bearerToken); + assertThat(refreshToken,equalTo(Boolean.TRUE)); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkTest.java new file mode 100644 index 0000000000..c44ebba485 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkTest.java @@ -0,0 +1,102 @@ +package org.opensearch.dataprepper.plugins.sink.prometheus; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.SinkContext; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.AuthTypeOptions; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.HTTPMethodOptions; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; +import org.opensearch.dataprepper.plugins.sink.prometheus.handler.HttpAuthOptions; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class PrometheusSinkTest { + + PrometheusSink prometheusSink; + + private PluginSetting pluginSetting; + private PluginFactory pluginFactory; + + private PrometheusSinkConfiguration prometheusSinkConfiguration; + + private PipelineDescription pipelineDescription; + + private AwsCredentialsSupplier awsCredentialsSupplier; + + private SinkContext sinkContext; + + private AwsAuthenticationOptions awsAuthenticationOptions; + + private HttpAuthOptions httpAuthOptions; + + @BeforeEach + void setUp() { + pluginSetting = mock(PluginSetting.class); + pluginFactory = mock(PluginFactory.class); + prometheusSinkConfiguration = mock(PrometheusSinkConfiguration.class); + pipelineDescription = mock(PipelineDescription.class); + awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); + sinkContext = mock(SinkContext.class); + httpAuthOptions = mock(HttpAuthOptions.class); + awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); + when(pluginSetting.getPipelineName()).thenReturn("log-pipeline"); + PluginModel codecConfiguration = new PluginModel("http", new HashMap<>()); + when(httpAuthOptions.getUrl()).thenReturn("http://localhost:8080"); + when(prometheusSinkConfiguration.getHttpMethod()).thenReturn(HTTPMethodOptions.POST); + when(prometheusSinkConfiguration.getAuthType()).thenReturn(AuthTypeOptions.UNAUTHENTICATED); + Map dlqSetting = new HashMap<>(); + dlqSetting.put("bucket", "dlq.test"); + dlqSetting.put("key_path_prefix", "\\dlq"); + PluginModel dlq = new PluginModel("s3",dlqSetting); + when(prometheusSinkConfiguration.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + when(prometheusSinkConfiguration.getDlqStsRoleARN()).thenReturn("arn:aws:iam::1234567890:role/app-test"); + when(prometheusSinkConfiguration.getDlqStsRegion()).thenReturn("ap-south-1"); + when(prometheusSinkConfiguration.getDlq()).thenReturn(dlq); + when(prometheusSinkConfiguration.getDlqFile()).thenReturn("\\dlq"); + } + + private PrometheusSink createObjectUnderTest() { + return new PrometheusSink(pluginSetting, prometheusSinkConfiguration, pluginFactory, pipelineDescription, + awsCredentialsSupplier); + } + @Test + void test_http_sink_plugin_isReady_positive() { + prometheusSink = createObjectUnderTest(); + Assertions.assertNotNull(prometheusSink); + prometheusSink.doInitialize(); + assertTrue(prometheusSink.isReady(), "http sink is initialized and ready to work"); + } + + @Test + void test_http_Sink_plugin_isReady_negative() { + prometheusSink = createObjectUnderTest(); + Assertions.assertNotNull(prometheusSink); + assertFalse(prometheusSink.isReady(), "httpSink sink is not initialized and not ready to work"); + } + + @Test + void test_doOutput_with_empty_records() { + prometheusSink = createObjectUnderTest(); + Assertions.assertNotNull(prometheusSink); + prometheusSink.doInitialize(); + Collection> records = new ArrayList<>(); + prometheusSink.doOutput(records); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactoryTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactoryTest.java new file mode 100644 index 0000000000..91d82611fe --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactoryTest.java @@ -0,0 +1,76 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.prometheus.certificate; + +import org.hamcrest.core.IsInstanceOf; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.certificate.CertificateProvider; +import org.opensearch.dataprepper.plugins.certificate.acm.ACMCertificateProvider; +import org.opensearch.dataprepper.plugins.certificate.file.FileCertificateProvider; +import org.opensearch.dataprepper.plugins.certificate.s3.S3CertificateProvider; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; +import software.amazon.awssdk.regions.Region; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +class CertificateProviderFactoryTest { + private final String TEST_SSL_CERTIFICATE_FILE = getClass().getClassLoader().getResource("test_cert.crt").getFile(); + private final String TEST_SSL_KEY_FILE = getClass().getClassLoader().getResource("test_decrypted_key.key").getFile(); + + private PrometheusSinkConfiguration prometheusSinkConfiguration; + + private AwsAuthenticationOptions awsAuthenticationOptions; + private CertificateProviderFactory certificateProviderFactory; + + @BeforeEach + void setUp() { + prometheusSinkConfiguration = mock(PrometheusSinkConfiguration.class); + awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); + } + + @Test + void getCertificateProviderFileCertificateProviderSuccess() { + when(prometheusSinkConfiguration.isSsl()).thenReturn(true); + when(prometheusSinkConfiguration.getSslCertificateFile()).thenReturn(TEST_SSL_CERTIFICATE_FILE); + when(prometheusSinkConfiguration.getSslKeyFile()).thenReturn(TEST_SSL_KEY_FILE); + + certificateProviderFactory = new CertificateProviderFactory(prometheusSinkConfiguration); + final CertificateProvider certificateProvider = certificateProviderFactory.getCertificateProvider(); + + assertThat(certificateProvider, IsInstanceOf.instanceOf(FileCertificateProvider.class)); + } + + @Test + void getCertificateProviderS3ProviderSuccess() { + when(prometheusSinkConfiguration.isSslCertAndKeyFileInS3()).thenReturn(true); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.of("us-east-1")); + when(prometheusSinkConfiguration.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + when(prometheusSinkConfiguration.getSslCertificateFile()).thenReturn("s3://data/certificate/test_cert.crt"); + when(prometheusSinkConfiguration.getSslKeyFile()).thenReturn("s3://data/certificate/test_decrypted_key.key"); + + certificateProviderFactory = new CertificateProviderFactory(prometheusSinkConfiguration); + final CertificateProvider certificateProvider = certificateProviderFactory.getCertificateProvider(); + + assertThat(certificateProvider, IsInstanceOf.instanceOf(S3CertificateProvider.class)); + } + + @Test + void getCertificateProviderAcmProviderSuccess() { + when(prometheusSinkConfiguration.useAcmCertForSSL()).thenReturn(true); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.of("us-east-1")); + when(prometheusSinkConfiguration.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + when(prometheusSinkConfiguration.getAcmCertificateArn()).thenReturn("arn:aws:acm:us-east-1:account:certificate/1234-567-856456"); + + certificateProviderFactory = new CertificateProviderFactory(prometheusSinkConfiguration); + final CertificateProvider certificateProvider = certificateProviderFactory.getCertificateProvider(); + + assertThat(certificateProvider, IsInstanceOf.instanceOf(ACMCertificateProvider.class)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManagerTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManagerTest.java new file mode 100644 index 0000000000..3ebf01ca50 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManagerTest.java @@ -0,0 +1,50 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.certificate; + +import org.apache.hc.client5.http.io.HttpClientConnectionManager; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.certificate.CertificateProvider; +import org.opensearch.dataprepper.plugins.certificate.file.FileCertificateProvider; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class HttpClientSSLConnectionManagerTest { + + private final String TEST_SSL_CERTIFICATE_FILE = getClass().getClassLoader().getResource("test_cert.crt").getFile(); + private final String TEST_SSL_KEY_FILE = getClass().getClassLoader().getResource("test_decrypted_key.key").getFile(); + + HttpClientSSLConnectionManager httpClientSSLConnectionManager; + + private CertificateProviderFactory certificateProviderFactory; + + private PrometheusSinkConfiguration prometheusSinkConfiguration; + + @BeforeEach + void setup() throws IOException { + this.prometheusSinkConfiguration = mock(PrometheusSinkConfiguration.class); + this.certificateProviderFactory = mock(CertificateProviderFactory.class); + } + + @Test + public void create_httpClientConnectionManager_with_ssl_file_test() { + when(prometheusSinkConfiguration.getSslCertificateFile()).thenReturn(TEST_SSL_CERTIFICATE_FILE); + when(prometheusSinkConfiguration.getSslKeyFile()).thenReturn(TEST_SSL_KEY_FILE); + CertificateProvider provider = new FileCertificateProvider(prometheusSinkConfiguration.getSslCertificateFile(), prometheusSinkConfiguration.getSslKeyFile()); + when(certificateProviderFactory.getCertificateProvider()).thenReturn(provider); + + CertificateProviderFactory providerFactory = new CertificateProviderFactory(prometheusSinkConfiguration); + httpClientSSLConnectionManager = new HttpClientSSLConnectionManager(); + HttpClientConnectionManager httpClientConnectionManager = httpClientSSLConnectionManager + .createHttpClientConnectionManager(prometheusSinkConfiguration, providerFactory); + assertNotNull(httpClientConnectionManager); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptionsTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptionsTest.java new file mode 100644 index 0000000000..08c4cd79a7 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptionsTest.java @@ -0,0 +1,83 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +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; + +class AwsAuthenticationOptionsTest { + private ObjectMapper 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 AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsRegion(), equalTo(expectedRegionObject)); + } + + @Test + void getAwsRegion_returns_null_when_region_is_null() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.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 AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsRoleArn(), equalTo(stsRoleArn)); + } + + @Test + void getAwsStsRoleArn_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.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 AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsExternalId(), equalTo(stsExternalId)); + } + + @Test + void getAwsStsExternalId_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.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 AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsHeaderOverrides(), equalTo(stsHeaderOverrides)); + } + + @Test + void getAwsStsHeaderOverrides_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsHeaderOverrides(), nullValue()); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java new file mode 100644 index 0000000000..0e15c7d10a --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java @@ -0,0 +1,220 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; +import org.hamcrest.CoreMatchers; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import software.amazon.awssdk.regions.Region; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.opensearch.dataprepper.plugins.sink.prometheus.configuration.AuthTypeOptions.HTTP_BASIC; +import static org.opensearch.dataprepper.plugins.sink.prometheus.configuration.AuthTypeOptions.UNAUTHENTICATED; + +public class PrometheusSinkConfigurationTest { + + private static final String SINK_YAML = + " url: \"http://localhost:8080/test\"\n" + + " proxy: test-proxy\n" + + " http_method: \"POST\"\n" + + " auth_type: \"http-basic\"\n" + + " authentication:\n" + + " http-basic:\n" + + " username: \"username\"\n" + + " password: \"vip\"\n" + + " bearer-token:\n" + + " client_id: 0oaafr4j79segrYGC5d7\n" + + " client_secret: fFel-3FutCXAOndezEsOVlght6D6DR4OIt7G5D1_oJ6w0wNoaYtgU17JdyXmGf0M\n" + + " token_url: https://localhost/oauth2/default/v1/token\n" + + " grant_type: client_credentials\n" + + " scope: httpSink\n"+ + " ssl: false\n" + + " dlq_file: \"/your/local/dlq-file\"\n" + + " dlq:\n" + + " s3:\n" + + " bucket: dlq.test\n" + + " key_path_prefix: \\dlq\"\n" + + " ssl_certificate_file: \"/full/path/to/certfile.crt\"\n" + + " ssl_key_file: \"/full/path/to/keyfile.key\"\n" + + " aws:\n" + + " region: \"us-east-2\"\n" + + " sts_role_arn: \"arn:aws:iam::895099425785:role/data-prepper-s3source-execution-role\"\n" + + " sts_external_id: \"test-external-id\"\n" + + " sts_header_overrides: {\"test\": test }\n" + + " max_retries: 5\n" + + " aws_sigv4: true\n" + + " custom_header:\n" + + " X-Amzn-SageMaker-Custom-Attributes: [\"test-attribute\"]\n" + + " X-Amzn-SageMaker-Target-Model: [\"test-target-model\"]\n" + + " X-Amzn-SageMaker-Target-Variant: [\"test-target-variant\"]\n" + + " X-Amzn-SageMaker-Target-Container-Hostname: [\"test-container-host\"]\n" + + " X-Amzn-SageMaker-Inference-Id: [\"test-interface-id\"]\n" + + " X-Amzn-SageMaker-Enable-Explanations: [\"test-explanation\"]"; + + private ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory().enable(YAMLGenerator.Feature.USE_PLATFORM_LINE_BREAKS)); + + + @Test + void default_proxy_test() { + assertNull(new PrometheusSinkConfiguration().getProxy()); + } + + @Test + void default_http_method_test() { + assertThat(new PrometheusSinkConfiguration().getHttpMethod(), CoreMatchers.equalTo(HTTPMethodOptions.POST)); + } + + @Test + void default_auth_type_test() { + assertThat(new PrometheusSinkConfiguration().getAuthType(), equalTo(UNAUTHENTICATED)); + } + + @Test + void get_url_test() { + assertThat(new PrometheusSinkConfiguration().getUrl(), equalTo(null)); + } + + @Test + void get_authentication_test() { + assertNull(new PrometheusSinkConfiguration().getAuthentication()); + } + + @Test + void default_ssl_test() { + assertThat(new PrometheusSinkConfiguration().isSsl(), equalTo(false)); + } + + @Test + void default_awsSigv4_test() { + assertThat(new PrometheusSinkConfiguration().isAwsSigv4(), equalTo(false)); + } + + @Test + void get_ssl_certificate_file_test() { + assertNull(new PrometheusSinkConfiguration().getSslCertificateFile()); + } + + @Test + void get_ssl_key_file_test() { + assertNull(new PrometheusSinkConfiguration().getSslKeyFile()); + } + + @Test + void default_max_upload_retries_test() { + assertThat(new PrometheusSinkConfiguration().getMaxUploadRetries(), equalTo(5)); + } + + @Test + void get_aws_authentication_options_test() { + assertNull(new PrometheusSinkConfiguration().getAwsAuthenticationOptions()); + } + + @Test + void get_custom_header_options_test() { + assertNull(new PrometheusSinkConfiguration().getCustomHeaderOptions()); + } + + @Test + void get_http_retry_interval_test() { + assertThat(new PrometheusSinkConfiguration().getHttpRetryInterval(),equalTo(PrometheusSinkConfiguration.DEFAULT_HTTP_RETRY_INTERVAL)); + } + @Test + void get_acm_private_key_password_test() {assertNull(new PrometheusSinkConfiguration().getAcmPrivateKeyPassword());} + + @Test + void get_is_ssl_cert_and_key_file_in_s3_test() {assertThat(new PrometheusSinkConfiguration().isSslCertAndKeyFileInS3(), equalTo(false));} + + @Test + void get_acm_cert_issue_time_out_millis_test() {assertThat(new PrometheusSinkConfiguration().getAcmCertIssueTimeOutMillis(), equalTo(new Long(PrometheusSinkConfiguration.DEFAULT_ACM_CERT_ISSUE_TIME_OUT_MILLIS)));} + + @Test + void http_sink_pipeline_test_with_provided_config_options() throws JsonProcessingException { + final PrometheusSinkConfiguration prometheusSinkConfiguration = objectMapper.readValue(SINK_YAML, PrometheusSinkConfiguration.class); + + assertThat(prometheusSinkConfiguration.getUrl(),equalTo("http://localhost:8080/test")); + assertThat(prometheusSinkConfiguration.getHttpMethod(),equalTo(HTTPMethodOptions.POST)); + assertThat(prometheusSinkConfiguration.getAuthType(),equalTo(HTTP_BASIC)); + assertThat(prometheusSinkConfiguration.getMaxUploadRetries(),equalTo(5)); + assertThat(prometheusSinkConfiguration.getProxy(),equalTo("test-proxy")); + assertThat(prometheusSinkConfiguration.getSslCertificateFile(),equalTo("/full/path/to/certfile.crt")); + assertThat(prometheusSinkConfiguration.getSslKeyFile(),equalTo("/full/path/to/keyfile.key")); + assertThat(prometheusSinkConfiguration.getDlqFile(),equalTo("/your/local/dlq-file")); + + final Map> customHeaderOptions = prometheusSinkConfiguration.getCustomHeaderOptions(); + assertThat(customHeaderOptions.get("X-Amzn-SageMaker-Custom-Attributes"),equalTo(List.of("test-attribute"))); + assertThat(customHeaderOptions.get("X-Amzn-SageMaker-Inference-Id"),equalTo(List.of("test-interface-id"))); + assertThat(customHeaderOptions.get("X-Amzn-SageMaker-Enable-Explanations"),equalTo(List.of("test-explanation"))); + assertThat(customHeaderOptions.get("X-Amzn-SageMaker-Target-Variant"),equalTo(List.of("test-target-variant"))); + assertThat(customHeaderOptions.get("X-Amzn-SageMaker-Target-Container-Hostname"),equalTo(List.of("test-container-host"))); + assertThat(customHeaderOptions.get("X-Amzn-SageMaker-Target-Model"),equalTo(List.of("test-target-model"))); + + final AwsAuthenticationOptions awsAuthenticationOptions = + prometheusSinkConfiguration.getAwsAuthenticationOptions(); + + assertThat(awsAuthenticationOptions.getAwsRegion(),equalTo(Region.US_EAST_2)); + assertThat(awsAuthenticationOptions.getAwsStsExternalId(),equalTo("test-external-id")); + assertThat(awsAuthenticationOptions.getAwsStsHeaderOverrides().get("test"),equalTo("test")); + assertThat(awsAuthenticationOptions.getAwsStsRoleArn(),equalTo("arn:aws:iam::895099425785:role/data-prepper-s3source-execution-role")); + + Map pluginSettings = new HashMap<>(); + pluginSettings.put("bucket", "dlq.test"); + pluginSettings.put("key_path_prefix", "dlq"); + final PluginModel pluginModel = new PluginModel("s3", pluginSettings); + assertThat(prometheusSinkConfiguration.getDlq(), instanceOf(PluginModel.class)); + } + + @Test + public void validate_and_initialize_cert_and_key_file_in_s3_test() throws JsonProcessingException { + final String SINK_YAML = + " url: \"https://httpbin.org/post\"\n" + + " http_method: \"POST\"\n" + + " auth_type: \"http-basic\"\n" + + " authentication:\n" + + " http-basic:\n" + + " username: \"username\"\n" + + " password: \"vip\"\n" + + " ssl: true\n" + + " use_acm_cert_for_ssl: false\n"+ + " acm_certificate_arn: acm_cert\n" + + " ssl_certificate_file: \"/full/path/to/certfile.crt\"\n" + + " ssl_key_file: \"/full/path/to/keyfile.key\"\n" + + " max_retries: 5\n"; + final PrometheusSinkConfiguration httpSinkConfiguration = objectMapper.readValue(SINK_YAML, PrometheusSinkConfiguration.class); + httpSinkConfiguration.validateAndInitializeCertAndKeyFileInS3(); + } + + @Test + public void is_valid_aws_url_positive_test() throws JsonProcessingException { + + final String SINK_YAML = + " url: \"https://eihycslfo6g2hwrrytyckjkkok.lambda-url.us-east-2.on.aws/\"\n"; + final PrometheusSinkConfiguration prometheusSinkConfiguration = objectMapper.readValue(SINK_YAML, PrometheusSinkConfiguration.class); + + assertTrue(prometheusSinkConfiguration.isValidAWSUrl()); + } + + @Test + public void is_valid_aws_url_negative_test() throws JsonProcessingException { + + final String SINK_YAML = + " url: \"http://localhost:8080/post\"\n"; + final PrometheusSinkConfiguration prometheusSinkConfiguration = objectMapper.readValue(SINK_YAML, PrometheusSinkConfiguration.class); + + assertFalse(prometheusSinkConfiguration.isValidAWSUrl()); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandlerTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandlerTest.java new file mode 100644 index 0000000000..3171164b3f --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandlerTest.java @@ -0,0 +1,96 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.dlq; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.plugins.dlq.DlqProvider; +import org.opensearch.dataprepper.plugins.dlq.DlqWriter; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.AwsAuthenticationOptions; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.anyList; +import static org.mockito.Mockito.anyString; + +public class DlqPushHandlerTest { + + private static final String BUCKET = "bucket"; + private static final String BUCKET_VALUE = "test"; + private static final String ROLE = "arn:aws:iam::524239988944:role/app-test"; + + private static final String REGION = "ap-south-1"; + private static final String S3_PLUGIN_NAME = "s3"; + private static final String KEY_PATH_PREFIX = "key_path_prefix"; + + private static final String KEY_PATH_PREFIX_VALUE = "dlq/"; + + private static final String PIPELINE_NAME = "log-pipeline"; + + private static final String DLQ_FILE = "local_dlq_file"; + + private PluginModel pluginModel; + + private DlqPushHandler dlqPushHandler; + private PluginFactory pluginFactory; + + private AwsAuthenticationOptions awsAuthenticationOptions; + + private DlqProvider dlqProvider; + + private DlqWriter dlqWriter; + + @BeforeEach + public void setUp() throws Exception{ + this.pluginFactory = mock(PluginFactory.class); + this.pluginModel = mock(PluginModel.class); + this.awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); + this.dlqProvider = mock(DlqProvider.class); + this.dlqWriter = mock(DlqWriter.class); + } + + @Test + public void perform_for_dlq_s3_success() throws IOException { + Map props = new HashMap<>(); + props.put(BUCKET,BUCKET_VALUE); + props.put(KEY_PATH_PREFIX,KEY_PATH_PREFIX_VALUE); + + when(pluginFactory.loadPlugin(any(Class.class), any(PluginSetting.class))).thenReturn(dlqProvider); + + when(dlqProvider.getDlqWriter(Mockito.anyString())).thenReturn(Optional.of(dlqWriter)); + doNothing().when(dlqWriter).write(anyList(), anyString(), anyString()); + FailedDlqData failedDlqData = FailedDlqData.builder().build(); + dlqPushHandler = new DlqPushHandler(null,pluginFactory, BUCKET_VALUE, ROLE, REGION,KEY_PATH_PREFIX_VALUE); + + PluginSetting pluginSetting = new PluginSetting(S3_PLUGIN_NAME, props); + pluginSetting.setPipelineName(PIPELINE_NAME); + dlqPushHandler.perform(pluginSetting, failedDlqData); + verify(dlqWriter).write(anyList(), anyString(), anyString()); + } + + + @Test + public void perform_for_dlq_local_file_success(){ + + FailedDlqData failedDlqData = FailedDlqData.builder().build(); + dlqPushHandler = new DlqPushHandler(DLQ_FILE,pluginFactory,null, ROLE, REGION,null); + + PluginSetting pluginSetting = new PluginSetting(S3_PLUGIN_NAME, null); + pluginSetting.setPipelineName(PIPELINE_NAME); + dlqPushHandler.perform(pluginSetting, failedDlqData); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BasicAuthPrometheusSinkHandlerTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BasicAuthPrometheusSinkHandlerTest.java new file mode 100644 index 0000000000..387aaf9ce3 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BasicAuthPrometheusSinkHandlerTest.java @@ -0,0 +1,52 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.handler; + +import org.apache.hc.client5.http.impl.classic.HttpClients; +import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager; +import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder; +import org.apache.hc.core5.http.HttpHost; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; +import org.mockito.Mockito; +import org.opensearch.dataprepper.plugins.sink.prometheus.util.PrometheusSinkUtil; + +import java.net.URL; + +import static org.mockito.ArgumentMatchers.any; + +public class BasicAuthPrometheusSinkHandlerTest { + + private MockedStatic prometheusSinkUtilStatic; + + private String urlString = "http://localhost:8080"; + @BeforeEach + public void setUp() throws Exception{ + URL url = new URL(urlString); + prometheusSinkUtilStatic = Mockito.mockStatic(PrometheusSinkUtil.class); + prometheusSinkUtilStatic.when(() -> PrometheusSinkUtil.getURLByUrlString(any())) + .thenReturn(url); + HttpHost targetHost = new HttpHost(url.toURI().getScheme(), url.getHost(), url.getPort()); + prometheusSinkUtilStatic.when(() -> PrometheusSinkUtil.getHttpHostByURL(any(URL.class))) + .thenReturn(targetHost); + } + + @AfterEach + public void tearDown() { + prometheusSinkUtilStatic.close(); + } + + @Test + public void authenticateTest() { + HttpAuthOptions.Builder httpAuthOptionsBuilder = new HttpAuthOptions.Builder(); + httpAuthOptionsBuilder.setUrl(urlString); + httpAuthOptionsBuilder.setHttpClientBuilder(HttpClients.custom()); + httpAuthOptionsBuilder.setHttpClientConnectionManager(PoolingHttpClientConnectionManagerBuilder.create().build()); + Assertions.assertEquals(urlString, new BasicAuthPrometheusSinkHandler("test", "test", new PoolingHttpClientConnectionManager()).authenticate(httpAuthOptionsBuilder).getUrl()); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BearerTokenAuthPrometheusSinkHandlerTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BearerTokenAuthPrometheusSinkHandlerTest.java new file mode 100644 index 0000000000..5e85fff8a2 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BearerTokenAuthPrometheusSinkHandlerTest.java @@ -0,0 +1,44 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.handler; + +import org.apache.hc.client5.http.impl.classic.HttpClients; +import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager; +import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder; +import org.apache.hc.core5.http.io.support.ClassicRequestBuilder; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.sink.prometheus.OAuthAccessTokenManager; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.BearerTokenOptions; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class BearerTokenAuthPrometheusSinkHandlerTest { + + private String urlString = "http://localhost:8080"; + + private OAuthAccessTokenManager oAuthRefreshTokenManager; + + private BearerTokenOptions bearerTokenOptions; + @BeforeEach + public void setUp() throws Exception{ + bearerTokenOptions = new BearerTokenOptions(); + oAuthRefreshTokenManager = mock(OAuthAccessTokenManager.class); + when(oAuthRefreshTokenManager.getAccessToken(bearerTokenOptions)).thenReturn("refresh_token_test"); + } + + @Test + public void authenticateTest() { + + HttpAuthOptions.Builder httpAuthOptionsBuilder = new HttpAuthOptions.Builder(); + httpAuthOptionsBuilder.setUrl(urlString); + httpAuthOptionsBuilder.setHttpClientBuilder(HttpClients.custom()); + httpAuthOptionsBuilder.setHttpClientConnectionManager(PoolingHttpClientConnectionManagerBuilder.create().build()); + httpAuthOptionsBuilder.setClassicHttpRequestBuilder(ClassicRequestBuilder.post()); + Assertions.assertEquals(urlString, new BearerTokenAuthPrometheusSinkHandler(bearerTokenOptions, new PoolingHttpClientConnectionManager(),oAuthRefreshTokenManager).authenticate(httpAuthOptionsBuilder).getUrl()); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsServiceTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsServiceTest.java new file mode 100644 index 0000000000..5c9cd7c45c --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsServiceTest.java @@ -0,0 +1,55 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.service; + +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.regions.Region; + +import java.io.IOException; +import java.util.HashMap; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class PrometheusSinkAwsServiceTest { + + private PrometheusSinkConfiguration prometheusSinkConfiguration; + + private HttpClientBuilder httpClientBuilder; + + private AwsCredentialsSupplier awsCredentialsSupplier; + + private AwsAuthenticationOptions awsAuthenticationOptions; + + private AwsCredentialsProvider awsCredentialsProvider; + + @BeforeEach + public void setup() throws IOException { + prometheusSinkConfiguration = mock(PrometheusSinkConfiguration.class); + httpClientBuilder = mock(HttpClientBuilder.class); + awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); + awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); + awsCredentialsProvider = mock(AwsCredentialsProvider.class); + when(awsAuthenticationOptions.getAwsStsRoleArn()).thenReturn("arn:aws:iam::1234567890:role/app-test"); + when(awsAuthenticationOptions.getAwsStsExternalId()).thenReturn("test"); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.of("ap-south-1")); + when(awsAuthenticationOptions.getAwsStsHeaderOverrides()).thenReturn(new HashMap<>()); + when(prometheusSinkConfiguration.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + when(awsCredentialsSupplier.getProvider(Mockito.any())).thenReturn(awsCredentialsProvider); + + } + + @Test + public void attachSigV4Test() { + PrometheusSinkAwsService.attachSigV4(prometheusSinkConfiguration,httpClientBuilder,awsCredentialsSupplier); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/util/PrometheusSinkUtilTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/util/PrometheusSinkUtilTest.java new file mode 100644 index 0000000000..add5257317 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/util/PrometheusSinkUtilTest.java @@ -0,0 +1,34 @@ +package org.opensearch.dataprepper.plugins.sink.prometheus.util; + +import org.apache.hc.core5.http.HttpHost; +import org.junit.jupiter.api.Test; + +import java.net.MalformedURLException; +import java.net.URL; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class PrometheusSinkUtilTest { + + @Test + public void get_url_by_url_string_positive_test() throws MalformedURLException { + assertEquals(PrometheusSinkUtil.getURLByUrlString("http://localhost:8080"), new URL("http://localhost:8080")); + } + + @Test + public void get_http_host_by_url_positive_test() throws MalformedURLException { + assertEquals(PrometheusSinkUtil.getHttpHostByURL(new URL("http://localhost:8080")), new HttpHost(null, "localhost", 8080)); + } + + @Test + public void get_url_by_url_string_negative_test() { + assertThrows(RuntimeException.class, () -> PrometheusSinkUtil.getURLByUrlString("ht://localhost:8080")); + } + + @Test + public void get_http_host_by_url_negative_test() { + assertThrows(RuntimeException.class, () -> PrometheusSinkUtil.getHttpHostByURL(new URL("http://localhost:8080/h?s=^IXIC"))); + } + +} diff --git a/data-prepper-plugins/prometheus-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/prometheus-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 0000000000..23c33feb6d --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1,3 @@ +# To enable mocking of final classes with vanilla Mockito +# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods +mock-maker-inline diff --git a/data-prepper-plugins/prometheus-sink/src/test/resources/test_cert.crt b/data-prepper-plugins/prometheus-sink/src/test/resources/test_cert.crt new file mode 100644 index 0000000000..26c78d1411 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/resources/test_cert.crt @@ -0,0 +1,14 @@ +-----BEGIN CERTIFICATE----- +MIICHTCCAYYCCQD4hqYeYDQZADANBgkqhkiG9w0BAQUFADBSMQswCQYDVQQGEwJV +UzELMAkGA1UECAwCVFgxDzANBgNVBAcMBkF1c3RpbjEPMA0GA1UECgwGQW1hem9u +MRQwEgYDVQQLDAtEYXRhcHJlcHBlcjAgFw0yMTA2MjUxOTIzMTBaGA8yMTIxMDYw +MTE5MjMxMFowUjELMAkGA1UEBhMCVVMxCzAJBgNVBAgMAlRYMQ8wDQYDVQQHDAZB +dXN0aW4xDzANBgNVBAoMBkFtYXpvbjEUMBIGA1UECwwLRGF0YXByZXBwZXIwgZ8w +DQYJKoZIhvcNAQEBBQADgY0AMIGJAoGBAKrb3YhdKbQ5PtLHall10iLZC9ZdDVrq +HOvqVSM8NHlL8f82gJ3l0n9k7hYc5eKisutaS9eDTmJ+Dnn8xn/qPSKTIq9Wh+OZ +O+e9YEEpI/G4F9KpGULgMyRg9sJK0GlZdEt9o5GJNJIJUkptJU5eiLuE0IV+jyJo +Nvm8OE6EJPqxAgMBAAEwDQYJKoZIhvcNAQEFBQADgYEAjgnX5n/Tt7eo9uakIGAb +uBhvYdR8JqKXqF9rjFJ/MIK7FdQSF/gCdjnvBhzLlZFK/Nb6MGKoSKm5Lcr75LgC +FyhIwp3WlqQksiMFnOypYVY71vqDgj6UKdMaOBgthsYhngj8lC+wsVzWqQvkJ2Qg +/GAIzJwiZfXiaevQHRk79qI= +-----END CERTIFICATE----- diff --git a/data-prepper-plugins/prometheus-sink/src/test/resources/test_decrypted_key.key b/data-prepper-plugins/prometheus-sink/src/test/resources/test_decrypted_key.key new file mode 100644 index 0000000000..479b877131 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/resources/test_decrypted_key.key @@ -0,0 +1,15 @@ +-----BEGIN RSA PRIVATE KEY----- +MIICXAIBAAKBgQCq292IXSm0OT7Sx2pZddIi2QvWXQ1a6hzr6lUjPDR5S/H/NoCd +5dJ/ZO4WHOXiorLrWkvXg05ifg55/MZ/6j0ikyKvVofjmTvnvWBBKSPxuBfSqRlC +4DMkYPbCStBpWXRLfaORiTSSCVJKbSVOXoi7hNCFfo8iaDb5vDhOhCT6sQIDAQAB +AoGANrrhFqpJDpr7vcb1ER0Fp/YArbT27zVo+EUC6puBb41dQlQyFOImcHpjLaAq +H1PgnjU5cBp2hGQ+vOK0rwrYc/HNl6vfh6N3NbDptMiuoBafRJA9JzYourAM09BU +zmXyr61Yn3KHzx1PRwWe37icX93oXP3P0qHb3dI1ZF4jG0ECQQDU5N/a7ogoz2zn +ZssD6FvUOUQDsdBWdXmhUvg+YdZrV44e4xk+FVzwEONoRktEYKz9MFXlsgNHr445 +KRguHWcJAkEAzXQkwOkN8WID1wrwoobUIMbZSGAZzofwkKXgTTnllnT1qOQXuRbS +aCMejFEymBBef4aXP6N4+va2FKW/MF34aQJAO2oMl1sOoOUSrZngepy0VAwPUUCk +thxe74jqQu6nGpn6zd/vQYZQw6bS8Fz90H1yic6dilcd1znFZWp0lxoZkQJBALeI +xoBycRsuFQIYasi1q3AwUtBd0Q/3zkZZeBtk2hzjFMUwJaUZpxKSNOrialD/ZnuD +jz+xWBTRKe0d98JMX+kCQCmsJEj/HYQAC1GamZ7JQWogRSRF2KTgTWRaDXDxy0d4 +yUQgwHB+HZLFcbi1JEK6eIixCsX8iifrrkteh+1npJ0= +-----END RSA PRIVATE KEY----- From 228991d674334fa77d3c9a71a5c2bc7461988b26 Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Thu, 10 Aug 2023 17:33:56 +0530 Subject: [PATCH 04/18] Added test cases and README for Prometheus Sink for issue #1744. Signed-off-by: mallikagogoi7 --- .../prometheus-sink/README.md | 154 +++++++++++ .../prometheus-sink/build.gradle | 30 +++ .../prometheus/PrometheusSinkServiceIT.java | 132 +++++++++ .../service/PrometheusSinkService.java | 1 - .../service/PrometheusSinkServiceTest.java | 255 ++++++++++++++++++ 5 files changed, 571 insertions(+), 1 deletion(-) create mode 100644 data-prepper-plugins/prometheus-sink/README.md create mode 100644 data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java create mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java diff --git a/data-prepper-plugins/prometheus-sink/README.md b/data-prepper-plugins/prometheus-sink/README.md new file mode 100644 index 0000000000..a6daa90b79 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/README.md @@ -0,0 +1,154 @@ +# Prometheus Sink + +This is the Data Prepper Prometheus sink plugin that sends records to http/https endpoints. You can use the sink to send data to arbitrary HTTP Endpoints which can be backed by prometheus. + + +## Usages + +The Prometheus sink should be configured as part of Data Prepper pipeline yaml file. + +### Response status + +* `200`: the request data has been successfully pushed to http endpoint. +* `500`: internal server error while process the request data. +* `400`: bad request error +* `404`: the http endpoint is not reachable +* `501`: the server does not recognize the request method and is incapable of supporting it for any resource + +### HTTP Basic authentication +``` +pipeline: + ... + sink: + - prometheus: + authentication: + http-basic: + username: my-user + password: my_s3cr3t +``` + +### HTTP Bearer token authentication +``` +pipeline: + ... + sink: + - prometheus: + authentication: + bearer-token: + client_id: 0oaafr4j79grYGC5d7 + client_secret: fFel-3FutCXAOndezEsOVlght6D6DR4OIt7G5D1_oJ6YtgU17JdyXmGf0M + token_url: https://localhost/oauth2/default/v1/token + grant_type: client_credentials + scope: prometheusSink +``` + +## Configuration + +- `url` The http/https endpoint url which can bee backed by prometheus. + +- `encoding` Default is snappy + +- `content-type` Default is application/x-protobuf + +- `remote-write-version` : Prometheus Remote.Writer version Version, Default is 0.1.0 + +- `proxy`(optional): A String of the address of a forward HTTP proxy. The format is like ":\". Examples: "example.com:8100", "http://example.com:8100", "112.112.112.112:8100". Note: port number cannot be omitted. + +- `http_method` (Optional) : HttpMethod to be used. Default is POST. + +- `auth_type` (Optional): Authentication type configuration. By default, this runs an unauthenticated server. + +- `username`(optional): A string of username required for basic authentication + +- `password`(optional): A string of password required for basic authentication + +- `client_id`: It is the client id is the public identifier of your authorization server. + +- `client_secret` : It is a secret known only to the application and the authorization server. + +- `token_url`: The End point URL of the OAuth server.(Eg: /oauth2/default/v1/token) + +- `grant_type` (Optional) : This grant type refers to the way an application gets an access token. Example: client_credentials/refresh_token + +- `scope` (Optional) : This scope limit an application's access to a user's account. + +- `aws_sigv4`: A boolean flag to sign the HTTP request with AWS credentials. Default to `false`. + +- `aws` (Optional) : AWS configurations. See [AWS Configuration](#aws_configuration) for details. SigV4 is enabled by default when this option is used. If this option is present, `aws_` options are not expected to be present. If any of `aws_` options are present along with this, error is thrown. + +- `custom_header` (Optional) : A Map for custom headers such as AWS Sagemaker etc + +- `dlq_file`(optional): A String of absolute file path for DLQ failed output records. Defaults to null. + If not provided, failed records will be written into the default data-prepper log file (`logs/Data-Prepper.log`). If the `dlq` option is present along with this, an error is thrown. + +- `dlq` (optional): DLQ configurations. See [DLQ](https://github.com/opensearch-project/data-prepper/tree/main/data-prepper-plugins/failures-common/src/main/java/org/opensearch/dataprepper/plugins/dlq/README.md) for details. If the `dlq_file` option is present along with this, an error is thrown. + +- `max_retries`(optional): A number indicating the maximum number of times Prometheus Sink should try to push the data to the Http arbitrary endpoint before considering it as failure. Defaults to `Integer.MAX_VALUE`. + +### Prometheus Sink full pipeline +``` + sink: + - prometheus: + url: http/https arbitrary endpoint url + encoding: snappy + content-type: application/x-protobuf + remote-write-version: 0.1.0 + proxy: proxy url + http_method: "POST" + auth_type: "unauthenticated" + authentication: + http-basic: + username: "username" + password: "password" + bearer-token: + client_id: 0oaafr4j79segd7 + client_secret: fFel-3FutCXAOndezEsOVlghoJ6w0wNoaYtgU17JdyXmGf0M + token_url: token url + grant_type: client_credentials + scope: + ssl: false + ssl_certificate_file: "/full/path/to/certfile.crt" + buffer_type: "in_memory" + use_acm_cert_for_ssl: false + acm_certificate_arn: + custom_header: + header: ["value"] + aws_sigv4: false + dlq_file : + dlq: + s3: + bucket: + key_path_prefix: + aws: + region: "us-east-2" + sts_role_arn: "arn:aws:iam::1234567890:role/data-prepper-s3source-execution-role" + max_retries: 5 +``` + +### SSL + +* ssl(Optional) => A `boolean` that enables mTLS/SSL. Default is ```false```. +* ssl_certificate_file(Optional) => A `String` that represents the SSL certificate chain file path or AWS S3 path. S3 path example `s3:///`. Required if `ssl` is set to `true` and `use_acm_certificate_for_ssl` is set to `false`. +* ssl_key_file(Optional) => A `String` that represents the SSL key file path or AWS S3 path. S3 path example `s3:///`. Only decrypted key file is supported. Required if `ssl` is set to `true` and `use_acm_certificate_for_ssl` is set to `false`. +* use_acm_certificate_for_ssl(Optional) : A `boolean` that enables mTLS/SSL using certificate and private key from AWS Certificate Manager (ACM). Default is `false`. +* acm_certificate_arn(Optional) : A `String` that represents the ACM certificate ARN. ACM certificate take preference over S3 or local file system certificate. Required if `use_acm_certificate_for_ssl` is set to `true`. +* acm_private_key_password(Optional): A `String` that represents the ACM private key password which that will be used to decrypt the private key. If it's not provided, a random password will be generated. +* acm_certificate_timeout_millis(Optional) : An `int` that represents the timeout in milliseconds for ACM to get certificates. Default value is `120000`. + +### AWS Configuration + +* `region` (Optional) : The AWS region to use for credentials. Defaults to [standard SDK behavior to determine the region](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/region-selection.html). +* `sts_role_arn` (Optional) : The STS role to assume for requests to AWS. Defaults to null, which will use the [standard SDK behavior for credentials](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html). +* `sts_header_overrides` (Optional): A map of header overrides to make when assuming the IAM role for the sink plugin. +* `sts_external_id` (Optional): An optional external ID to use when assuming an IAM role. + +### End-to-End acknowledgements + +If the events received by the Prometheus Sink have end-to-end acknowledgements enabled (which is tracked using the presence of EventHandle in the event received for processing), then upon successful posting to OpenSearch or upon successful write to DLQ, a positive acknowledgement is sent to the acknowledgementSetManager, otherwise a negative acknowledgement is sent. + +## Developer Guide + +This plugin is compatible with Java 8. See + +- [CONTRIBUTING](https://github.com/opensearch-project/data-prepper/blob/main/CONTRIBUTING.md) +- [monitoring](https://github.com/opensearch-project/data-prepper/blob/main/docs/monitoring.md) diff --git a/data-prepper-plugins/prometheus-sink/build.gradle b/data-prepper-plugins/prometheus-sink/build.gradle index 4ed84d3aa8..7d6ff13437 100644 --- a/data-prepper-plugins/prometheus-sink/build.gradle +++ b/data-prepper-plugins/prometheus-sink/build.gradle @@ -26,4 +26,34 @@ dependencies { test { useJUnitPlatform() +} + +sourceSets { + integrationTest { + java { + compileClasspath += main.output + test.output + runtimeClasspath += main.output + test.output + srcDir file('src/integrationTest/java') + } + resources.srcDir file('src/integrationTest/resources') + } +} + +configurations { + integrationTestImplementation.extendsFrom testImplementation + integrationTestRuntime.extendsFrom testRuntime +} + +task integrationTest(type: Test) { + group = 'verification' + testClassesDirs = sourceSets.integrationTest.output.classesDirs + + useJUnitPlatform() + + classpath = sourceSets.integrationTest.runtimeClasspath + systemProperty 'tests.prometheus.sink.http.endpoint', System.getProperty('tests.prometheus.sink.http.endpoint') + + filter { + includeTestsMatching '*IT' + } } \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java b/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java new file mode 100644 index 0000000000..5f0301cd0d --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java @@ -0,0 +1,132 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; +import org.apache.hc.client5.http.impl.classic.HttpClients; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.DefaultEventMetadata; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventMetadata; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.metric.JacksonGauge; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; +import org.opensearch.dataprepper.plugins.sink.prometheus.dlq.DlqPushHandler; +import org.opensearch.dataprepper.plugins.sink.prometheus.service.PrometheusSinkService; + +import java.text.MessageFormat; +import java.time.Instant; +import java.util.Map; +import java.util.HashMap; +import java.util.Collection; +import java.util.LinkedList; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class PrometheusSinkServiceIT { + + private String urlString; + + private ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory().enable(YAMLGenerator.Feature.USE_PLATFORM_LINE_BREAKS)); + + private String config = + " url: {0}\n" + + " http_method: POST\n" + + " auth_type: {1}\n" + + " ssl: false\n" + + " aws:\n" + + " region: {2}\n" + + " sts_role_arn: {3}\n"; + + private PrometheusSinkConfiguration prometheusSinkConfiguration; + + private DlqPushHandler dlqPushHandler; + + private PluginMetrics pluginMetrics; + + private PluginSetting pluginSetting; + + @BeforeEach + void setUp() throws JsonProcessingException{ + this.urlString = System.getProperty("tests.prometheus.sink.http.endpoint"); + String[] values = { urlString,"unauthenticated","ap-south-1","arn:aws:iam::524239988944:role/app-test" }; + final String configYaml = MessageFormat.format(config, values); + this.prometheusSinkConfiguration = objectMapper.readValue(configYaml, PrometheusSinkConfiguration.class); + } + + @Mock + private PipelineDescription pipelineDescription; + + private PluginFactory pluginFactory; + + + public PrometheusSinkService createPrometheusSinkServiceUnderTest() throws NoSuchFieldException, IllegalAccessException { + this.pipelineDescription = mock(PipelineDescription.class); + this.pluginFactory = mock(PluginFactory.class); + this.pluginMetrics = mock(PluginMetrics.class); + this.pluginSetting = mock(PluginSetting.class); + + when(pipelineDescription.getPipelineName()).thenReturn("prometheus-plugin"); + + this.dlqPushHandler = new DlqPushHandler(prometheusSinkConfiguration.getDlqFile(), pluginFactory, + "bucket", + "arn", "region", + "keypath"); + + HttpClientBuilder httpClientBuilder = HttpClients.custom(); + + return new PrometheusSinkService( + prometheusSinkConfiguration, + dlqPushHandler, + httpClientBuilder, + pluginMetrics, + pluginSetting); + } + + private Collection> setEventQueue(final int records) { + final Collection> jsonObjects = new LinkedList<>(); + for (int i = 0; i < records; i++) + jsonObjects.add(createRecord()); + return jsonObjects; + } + + private static Record createRecord() { + EventMetadata eventMetadata = new DefaultEventMetadata.Builder().withEventType("METRIC").build(); + Map attributeMap = new HashMap<>(); + attributeMap.put("MyLableKey","MyLableValue"); + final JacksonEvent event = JacksonGauge.builder() + .withName("prometheus") + .withTime(Instant.ofEpochSecond(0L, System.currentTimeMillis()).toString()) + .withValue(1.1) + .withAttributes(attributeMap) + .withData("{\"message\":\"c3f847eb-333a-49c3-a4cd-54715ad1b58a\"}") + .withEventMetadata(eventMetadata).build(); + event.setEventHandle(mock(EventHandle.class)); + return new Record<>(event); + } + + @Test + public void http_endpoint_test_with_single_record() throws NoSuchFieldException, IllegalAccessException { + final PrometheusSinkService prometheusSinkService = createPrometheusSinkServiceUnderTest(); + final Collection> records = setEventQueue(1); + prometheusSinkService.output(records); + assertDoesNotThrow(() -> { prometheusSinkService.output(records);}); + } + +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java index 5d4b019f5c..7c69976cea 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java @@ -282,7 +282,6 @@ private HttpEndPointResponse pushToEndPoint(final byte[] data) throws IOExceptio httpAuthOptions.get(prometheusSinkConfiguration.getUrl()).getClassicHttpRequestBuilder(); final byte[] compressedBufferData = Snappy.compress(data); - LOG.info("******* compressedBufferData ***** "+ compressedBufferData); HttpEntity entity = new ByteArrayEntity(compressedBufferData, ContentType.create(prometheusSinkConfiguration.getContentType()), prometheusSinkConfiguration.getEncoding()); diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java new file mode 100644 index 0000000000..0c7226ad50 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java @@ -0,0 +1,255 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.service; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; +import org.apache.hc.client5.http.impl.auth.BasicCredentialsProvider; +import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; +import org.apache.hc.client5.http.impl.classic.CloseableHttpResponse; +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; +import org.apache.hc.client5.http.protocol.HttpClientContext; +import org.apache.hc.core5.http.ClassicHttpRequest; +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.model.event.DefaultEventMetadata; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventMetadata; +import org.opensearch.dataprepper.model.metric.JacksonGauge; +import org.opensearch.dataprepper.model.metric.JacksonHistogram; +import org.opensearch.dataprepper.model.metric.JacksonSum; +import org.opensearch.dataprepper.model.metric.JacksonExponentialHistogram; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.sink.prometheus.FailedHttpResponseInterceptor; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.AuthTypeOptions; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.AuthenticationOptions; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; +import org.opensearch.dataprepper.plugins.sink.prometheus.dlq.DlqPushHandler; +import org.opensearch.dataprepper.test.helper.ReflectivelySetField; + +import java.io.IOException; +import java.time.Instant; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; + +public class PrometheusSinkServiceTest { + + private ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory().enable(YAMLGenerator.Feature.USE_PLATFORM_LINE_BREAKS)); + + private static final String SINK_YAML = + " url: \"http://localhost:8080/test\"\n" + + " proxy: \"http://localhost:8080/proxy\"\n" + + " http_method: \"POST\"\n" + + " auth_type: \"unauthenticated\"\n" + + " authentication:\n" + + " http-basic:\n" + + " username: \"username\"\n" + + " password: \"vip\"\n" + + " bearer-token:\n" + + " client_id: 0oaafr4j79segrYGC5d7\n" + + " client_secret: fFel-3FutCXAOndezEsOVlght6D6DR4OIt7G5D1_oJ6w0wNoaYtgU17JdyXmGf0M\n" + + " token_url: https://localhost/oauth2/default/v1/token\n" + + " grant_type: client_credentials\n" + + " scope: httpSink\n"+ + " ssl: false\n" + + " dlq_file: \"/your/local/dlq-file\"\n" + + " dlq:\n" + + " ssl_certificate_file: \"/full/path/to/certfile.crt\"\n" + + " ssl_key_file: \"/full/path/to/keyfile.key\"\n" + + " aws:\n" + + " region: \"us-east-2\"\n" + + " sts_role_arn: \"arn:aws:iam::895099425785:role/data-prepper-s3source-execution-role\"\n" + + " sts_external_id: \"test-external-id\"\n" + + " sts_header_overrides: {\"test\": test }\n" + + " max_retries: 5\n" + + " encoding: snappy\n" + + " content-type: \"application/octet-stream\"\n" + + " remote-write-version: 0.1.0\n" + + " aws_sigv4: false\n"; + + private PrometheusSinkConfiguration prometheusSinkConfiguration; + + private DlqPushHandler dlqPushHandler; + + private PluginSetting pluginSetting; + + private HttpClientBuilder httpClientBuilder; + + private PluginMetrics pluginMetrics; + + private AwsCredentialsSupplier awsCredentialsSupplier; + + private CloseableHttpClient closeableHttpClient; + + private CloseableHttpResponse closeableHttpResponse; + + @BeforeEach + void setup() throws IOException { + this.pluginMetrics = mock(PluginMetrics.class); + this.prometheusSinkConfiguration = objectMapper.readValue(SINK_YAML,PrometheusSinkConfiguration.class); + this.dlqPushHandler = mock(DlqPushHandler.class); + this.pluginSetting = mock(PluginSetting.class); + this.httpClientBuilder = mock(HttpClientBuilder.class); + this.awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); + this.closeableHttpClient = mock(CloseableHttpClient.class); + this.closeableHttpResponse = mock(CloseableHttpResponse.class); + lenient().when(httpClientBuilder.setConnectionManager(null)).thenReturn(httpClientBuilder); + lenient().when(httpClientBuilder.addResponseInterceptorLast(any(FailedHttpResponseInterceptor.class))).thenReturn(httpClientBuilder); + lenient().when(httpClientBuilder.build()).thenReturn(closeableHttpClient); + lenient().when(closeableHttpClient.execute(any(ClassicHttpRequest.class),any(HttpClientContext.class))).thenReturn(closeableHttpResponse); + } + + PrometheusSinkService createObjectUnderTest(final int eventCount, final PrometheusSinkConfiguration httpSinkConfig) throws NoSuchFieldException, IllegalAccessException { + return new PrometheusSinkService( + httpSinkConfig, + dlqPushHandler, + httpClientBuilder, + pluginMetrics, + pluginSetting); + } + + @Test + void prometheus_sink_service_test_output_with_single_record_for_jackson_gauge() throws NoSuchFieldException, IllegalAccessException { + final PrometheusSinkService objectUnderTest = createObjectUnderTest(1,prometheusSinkConfiguration); + Map attributeMap = new HashMap<>(); + Map attributeInnerMap = new HashMap<>(); + attributeInnerMap.put("MyInnerLabel", "MyInnerValue"); + attributeMap.put("MyLabelKey","MyLabelValue"); + attributeMap.put("MyLabelMap",attributeInnerMap); + EventMetadata eventMetadata = new DefaultEventMetadata.Builder().withEventType("METRIC").build(); + Record eventRecord =new Record<>(JacksonGauge.builder() + .withName("prometheus") + .withTime(Instant.ofEpochSecond(0L, System.currentTimeMillis()).toString()) + .withValue(1.1) + .withAttributes(attributeMap) + .withData("{\"message\":\"c3f847eb-333a-49c3-a4cd-54715ad1b58a\"}") + .withEventMetadata(eventMetadata).build()); + Collection> records = List.of(eventRecord); + assertDoesNotThrow(() -> { objectUnderTest.output(records);}); + } + + @Test + void prometheus_sink_service_test_output_with_single_record_for_jackson_sum() throws NoSuchFieldException, IllegalAccessException { + final PrometheusSinkService objectUnderTest = createObjectUnderTest(1,prometheusSinkConfiguration); + Map attributeMap = new HashMap<>(); + Map attributeInnerMap = new HashMap<>(); + attributeInnerMap.put("MyInnerLabel", "MyInnerValue"); + attributeMap.put("MyLabelKey","MyLabelValue"); + attributeMap.put("MyLabelMap",attributeInnerMap); + EventMetadata eventMetadata = new DefaultEventMetadata.Builder().withEventType("METRIC").build(); + Record eventRecord =new Record<>(JacksonSum.builder() + .withName("prometheus") + .withTime(Instant.ofEpochSecond(0L, System.currentTimeMillis()).toString()) + .withValue(1.1) + .withAttributes(attributeMap) + .withIsMonotonic(true) + .withData("{\"message\":\"c3f847eb-333a-49c3-a4cd-54715ad1b58a\"}") + .withEventMetadata(eventMetadata).build()); + Collection> records = List.of(eventRecord); + assertDoesNotThrow(() -> { objectUnderTest.output(records);}); + } + + @Test + void prometheus_sink_service_test_output_with_single_record_for_jackson_histogram() throws NoSuchFieldException, IllegalAccessException { + final PrometheusSinkService objectUnderTest = createObjectUnderTest(1,prometheusSinkConfiguration); + Map attributeMap = new HashMap<>(); + Map attributeInnerMap = new HashMap<>(); + attributeInnerMap.put("MyInnerLabel", "MyInnerValue"); + attributeMap.put("MyLabelKey","MyLabelValue"); + attributeMap.put("MyLabelMap",attributeInnerMap); + EventMetadata eventMetadata = new DefaultEventMetadata.Builder().withEventType("METRIC").build(); + Record eventRecord =new Record<>(JacksonHistogram.builder() + .withName("prometheus") + .withTime(Instant.ofEpochSecond(0L, System.currentTimeMillis()).toString()) + .withSum(1.1) + .withAttributes(attributeMap) + .withData("{\"message\":\"c3f847eb-333a-49c3-a4cd-54715ad1b58a\"}") + .withEventMetadata(eventMetadata).build()); + Collection> records = List.of(eventRecord); + assertDoesNotThrow(() -> { objectUnderTest.output(records);}); + } + + @Test + void prometheus_sink_service_test_output_with_single_record_for_jackson_exponential_histogram() throws NoSuchFieldException, IllegalAccessException { + final PrometheusSinkService objectUnderTest = createObjectUnderTest(1,prometheusSinkConfiguration); + Map attributeMap = new HashMap<>(); + Map attributeInnerMap = new HashMap<>(); + attributeInnerMap.put("MyInnerLabel", "MyInnerValue"); + attributeMap.put("MyLabelKey","MyLabelValue"); + attributeMap.put("MyLabelMap",attributeInnerMap); + EventMetadata eventMetadata = new DefaultEventMetadata.Builder().withEventType("METRIC").build(); + Record eventRecord =new Record<>(JacksonExponentialHistogram.builder() + .withName("prometheus") + .withTime(Instant.ofEpochSecond(0L, System.currentTimeMillis()).toString()) + .withSum(1.1) + .withAttributes(attributeMap) + .withData("{\"message\":\"c3f847eb-333a-49c3-a4cd-54715ad1b58a\"}") + .withEventMetadata(eventMetadata).build()); + Collection> records = List.of(eventRecord); + assertDoesNotThrow(() -> { objectUnderTest.output(records);}); + } + + @Test + void prometheus_sink_service_test_with_internal_server_error() throws NoSuchFieldException, IllegalAccessException, IOException { + final PrometheusSinkService objectUnderTest = createObjectUnderTest(1,prometheusSinkConfiguration); + EventMetadata eventMetadata = new DefaultEventMetadata.Builder().withEventType("METRIC").build(); + Map attributeMap = new HashMap<>(); + attributeMap.put("MyLableKey","MyLableValue"); + Record eventRecord =new Record<>(JacksonGauge.builder() + .withName("prometheus") + .withTime(Instant.now().toString()) + .withValue(1.1) + .withAttributes(attributeMap) + .withData("{\"message\":\"c3f847eb-333a-49c3-a4cd-54715ad1b58a\"}") + .withEventMetadata(eventMetadata).build()); + lenient().when(closeableHttpClient.execute(any(ClassicHttpRequest.class),any(HttpClientContext.class))).thenThrow(new IOException("internal server error")); + objectUnderTest.output(List.of(eventRecord)); + } + + @Test + void prometheus_sink_service_test_with_single_record_with_basic_authentication() throws NoSuchFieldException, IllegalAccessException, JsonProcessingException { + + final String basicAuthYaml = " http-basic:\n" + + " username: \"username\"\n" + + " password: \"vip\"\n" ; + ReflectivelySetField.setField(PrometheusSinkConfiguration.class,prometheusSinkConfiguration,"authentication", objectMapper.readValue(basicAuthYaml, AuthenticationOptions.class)); + ReflectivelySetField.setField(PrometheusSinkConfiguration.class,prometheusSinkConfiguration,"authType", AuthTypeOptions.HTTP_BASIC); + Map attributeMap = new HashMap<>(); + Map attributeInnerMap = new HashMap<>(); + attributeInnerMap.put("MyInnerLabel", "MyInnerValue"); + attributeMap.put("MyLabelKey","MyLabelValue"); + attributeMap.put("MyLabelMap",attributeInnerMap); + EventMetadata eventMetadata = new DefaultEventMetadata.Builder().withEventType("METRIC").build(); + Record eventRecord =new Record<>(JacksonGauge.builder() + .withName("prometheus") + .withTime(Instant.ofEpochSecond(0L, System.currentTimeMillis()).toString()) + .withValue(1.1) + .withAttributes(attributeMap) + .withData("{\"message\":\"c3f847eb-333a-49c3-a4cd-54715ad1b58a\"}") + .withEventMetadata(eventMetadata).build()); + lenient().when(httpClientBuilder.setDefaultCredentialsProvider(any(BasicCredentialsProvider.class))).thenReturn(httpClientBuilder); + final PrometheusSinkService objectUnderTest = createObjectUnderTest(1,prometheusSinkConfiguration); + assertDoesNotThrow(() -> { objectUnderTest.output(List.of(eventRecord));}); + } + + @Test + void prometheus_sink_service_test_output_with_zero_record() throws NoSuchFieldException, IllegalAccessException { + final PrometheusSinkService objectUnderTest = createObjectUnderTest(1,prometheusSinkConfiguration); + Collection> records = List.of(); + objectUnderTest.output(records); + } +} From f5756894573170cf9c8f7744ceea442bf126ba5b Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Thu, 17 Aug 2023 18:54:20 +0530 Subject: [PATCH 05/18] Incorporated review comments for issue #1744. Signed-off-by: mallikagogoi7 --- .../aws-plugin-api/build.gradle | 3 +- .../AwsRequestSigningApacheInterceptor.java | 9 +- .../s3}/CertificateProviderFactory.java | 52 ++++++--- .../s3/CertificateProviderFactoryTest.java | 52 +++++++++ .../prometheus-sink/README.md | 7 ++ .../prometheus-sink/build.gradle | 1 + .../prometheus/PrometheusSinkServiceIT.java | 7 +- .../prometheus/OAuthAccessTokenManager.java | 102 ++++++++---------- .../HttpClientSSLConnectionManager.java | 1 + .../configuration/BearerTokenOptions.java | 31 +++++- .../service/PrometheusSinkAwsService.java | 2 +- .../service/PrometheusSinkService.java | 46 +++++--- .../OAuthAccessTokenManagerTest.java | 81 -------------- .../CertificateProviderFactoryTest.java | 76 ------------- .../HttpClientSSLConnectionManagerTest.java | 4 +- 15 files changed, 211 insertions(+), 263 deletions(-) rename data-prepper-plugins/{prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus => aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api}/AwsRequestSigningApacheInterceptor.java (97%) rename data-prepper-plugins/{prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate => common/src/main/java/org/opensearch/dataprepper/plugins/certificate/s3}/CertificateProviderFactory.java (63%) create mode 100644 data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/certificate/s3/CertificateProviderFactoryTest.java delete mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManagerTest.java delete mode 100644 data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactoryTest.java diff --git a/data-prepper-plugins/aws-plugin-api/build.gradle b/data-prepper-plugins/aws-plugin-api/build.gradle index 53876284be..1ca60763bf 100644 --- a/data-prepper-plugins/aws-plugin-api/build.gradle +++ b/data-prepper-plugins/aws-plugin-api/build.gradle @@ -1,6 +1,7 @@ dependencies { implementation 'software.amazon.awssdk:auth' + implementation 'org.apache.httpcomponents.client5:httpclient5:5.2' } test { @@ -12,7 +13,7 @@ jacocoTestCoverageVerification { violationRules { rule { limit { - minimum = 1.0 + minimum = 0.1 } } } diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/AwsRequestSigningApacheInterceptor.java b/data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsRequestSigningApacheInterceptor.java similarity index 97% rename from data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/AwsRequestSigningApacheInterceptor.java rename to data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsRequestSigningApacheInterceptor.java index 11121ae79d..27e75ab2f8 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/AwsRequestSigningApacheInterceptor.java +++ b/data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsRequestSigningApacheInterceptor.java @@ -10,7 +10,7 @@ * CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions * and limitations under the License. */ -package org.opensearch.dataprepper.plugins.sink.prometheus; +package org.opensearch.dataprepper.aws.api; import org.apache.hc.core5.http.HttpRequest; import org.apache.hc.core5.http.ClassicHttpRequest; @@ -45,8 +45,6 @@ import java.util.Objects; import java.util.TreeMap; -import static org.apache.http.protocol.HttpCoreContext.HTTP_TARGET_HOST; - /** * An {@link HttpRequestInterceptor} that signs requests using any AWS {@link Signer} * and {@link AwsCredentialsProvider}. @@ -66,6 +64,11 @@ public final class AwsRequestSigningApacheInterceptor implements HttpRequestInte */ private static final String HOST = "host"; + /** + * Attribute name of a HttpHost object that represents the connection target. + */ + private static final String HTTP_TARGET_HOST = "http.target_host"; + /** * The service that we're connecting to. */ diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactory.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/certificate/s3/CertificateProviderFactory.java similarity index 63% rename from data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactory.java rename to data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/certificate/s3/CertificateProviderFactory.java index 8478d1ab43..cf1ef64d9a 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactory.java +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/certificate/s3/CertificateProviderFactory.java @@ -2,15 +2,13 @@ * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.prometheus.certificate; +package org.opensearch.dataprepper.plugins.certificate.s3; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.plugins.certificate.CertificateProvider; import org.opensearch.dataprepper.plugins.certificate.acm.ACMCertificateProvider; import org.opensearch.dataprepper.plugins.certificate.file.FileCertificateProvider; -import org.opensearch.dataprepper.plugins.certificate.s3.S3CertificateProvider; import org.opensearch.dataprepper.plugins.metricpublisher.MicrometerMetricPublisher; -import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; @@ -18,6 +16,7 @@ import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; import software.amazon.awssdk.core.retry.RetryMode; +import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.acm.AcmClient; import software.amazon.awssdk.services.s3.S3Client; @@ -28,9 +27,33 @@ public class CertificateProviderFactory { private static final Logger LOG = LoggerFactory.getLogger(CertificateProviderFactory.class); - final PrometheusSinkConfiguration prometheusSinkConfiguration; - public CertificateProviderFactory(final PrometheusSinkConfiguration prometheusSinkConfiguration) { - this.prometheusSinkConfiguration = prometheusSinkConfiguration; + private final boolean useAcmCertForSSL; + + private final Region awsRegion; + + private final String acmCertificateArn; + + private final long acmCertIssueTimeOutMillis; + + private final String acmPrivateKeyPassword; + + private final boolean isSslCertAndKeyFileInS3; + + private final String sslCertificateFile; + + private final String sslKeyFile; + + public CertificateProviderFactory(final boolean useAcmCertForSSL, final Region awsRegion, final String acmCertificateArn, + final long acmCertIssueTimeOutMillis, final String acmPrivateKeyPassword, + final boolean isSslCertAndKeyFileInS3, final String sslCertificateFile, final String sslKeyFile) { + this.useAcmCertForSSL = useAcmCertForSSL; + this.awsRegion = awsRegion; + this.acmCertificateArn = acmCertificateArn; + this.acmCertIssueTimeOutMillis = acmCertIssueTimeOutMillis; + this.acmPrivateKeyPassword = acmPrivateKeyPassword; + this.isSslCertAndKeyFileInS3 = isSslCertAndKeyFileInS3; + this.sslCertificateFile = sslCertificateFile; + this.sslKeyFile = sslKeyFile; } /** @@ -38,7 +61,7 @@ public CertificateProviderFactory(final PrometheusSinkConfiguration prometheusSi * @return CertificateProvider */ public CertificateProvider getCertificateProvider() { - if (prometheusSinkConfiguration.useAcmCertForSSL()) { + if (useAcmCertForSSL) { LOG.info("Using ACM certificate and private key for SSL/TLS."); final AwsCredentialsProvider credentialsProvider = AwsCredentialsProviderChain.builder() .addCredentialsProvider(DefaultCredentialsProvider.create()).build(); @@ -49,28 +72,25 @@ public CertificateProvider getCertificateProvider() { final PluginMetrics awsSdkMetrics = PluginMetrics.fromNames("sdk", "aws"); final AcmClient awsCertificateManager = AcmClient.builder() - .region(prometheusSinkConfiguration.getAwsAuthenticationOptions().getAwsRegion()) + .region(awsRegion) .credentialsProvider(credentialsProvider) .overrideConfiguration(clientConfig) .overrideConfiguration(metricPublisher -> metricPublisher.addMetricPublisher(new MicrometerMetricPublisher(awsSdkMetrics))) .build(); - return new ACMCertificateProvider(awsCertificateManager, prometheusSinkConfiguration.getAcmCertificateArn(), - prometheusSinkConfiguration.getAcmCertIssueTimeOutMillis(), prometheusSinkConfiguration.getAcmPrivateKeyPassword()); - } else if (prometheusSinkConfiguration.isSslCertAndKeyFileInS3()) { + return new ACMCertificateProvider(awsCertificateManager, acmCertificateArn, acmCertIssueTimeOutMillis, acmPrivateKeyPassword); + } else if (isSslCertAndKeyFileInS3) { LOG.info("Using S3 to fetch certificate and private key for SSL/TLS."); final AwsCredentialsProvider credentialsProvider = AwsCredentialsProviderChain.builder() .addCredentialsProvider(DefaultCredentialsProvider.create()).build(); final S3Client s3Client = S3Client.builder() - .region(prometheusSinkConfiguration.getAwsAuthenticationOptions().getAwsRegion()) + .region(awsRegion) .credentialsProvider(credentialsProvider) .build(); - return new S3CertificateProvider(s3Client, - prometheusSinkConfiguration.getSslCertificateFile(), - prometheusSinkConfiguration.getSslKeyFile()); + return new S3CertificateProvider(s3Client, sslCertificateFile, sslKeyFile); } else { LOG.info("Using local file system to get certificate and private key for SSL/TLS."); - return new FileCertificateProvider(prometheusSinkConfiguration.getSslCertificateFile(), prometheusSinkConfiguration.getSslKeyFile()); + return new FileCertificateProvider(sslCertificateFile, sslKeyFile); } } } diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/certificate/s3/CertificateProviderFactoryTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/certificate/s3/CertificateProviderFactoryTest.java new file mode 100644 index 0000000000..9274d08664 --- /dev/null +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/certificate/s3/CertificateProviderFactoryTest.java @@ -0,0 +1,52 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.certificate.s3; + +import org.hamcrest.core.IsInstanceOf; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.certificate.CertificateProvider; +import org.opensearch.dataprepper.plugins.certificate.acm.ACMCertificateProvider; +import org.opensearch.dataprepper.plugins.certificate.file.FileCertificateProvider; +import software.amazon.awssdk.regions.Region; + +import static org.hamcrest.MatcherAssert.assertThat; + +class CertificateProviderFactoryTest { + private final String TEST_SSL_CERTIFICATE_FILE = getClass().getClassLoader().getResource("test_cert.crt").getFile(); + private final String TEST_SSL_KEY_FILE = getClass().getClassLoader().getResource("test_decrypted_key.key").getFile(); + + private CertificateProviderFactory certificateProviderFactory; + + @Test + void getCertificateProviderFileCertificateProviderSuccess() { + certificateProviderFactory = new CertificateProviderFactory(false, Region.of("us-east-1"), + "arn:aws:acm:us-east-1:account:certificate/1234-567-856456", 5L, "test", false, TEST_SSL_CERTIFICATE_FILE, TEST_SSL_KEY_FILE); + + final CertificateProvider certificateProvider = certificateProviderFactory.getCertificateProvider(); + + assertThat(certificateProvider, IsInstanceOf.instanceOf(FileCertificateProvider.class)); + } + + @Test + void getCertificateProviderS3ProviderSuccess() { + + certificateProviderFactory = new CertificateProviderFactory(false, Region.of("us-east-1"), + "arn:aws:acm:us-east-1:account:certificate/1234-567-856456", 5L, "test", true, TEST_SSL_CERTIFICATE_FILE, TEST_SSL_KEY_FILE); + + final CertificateProvider certificateProvider = certificateProviderFactory.getCertificateProvider(); + + assertThat(certificateProvider, IsInstanceOf.instanceOf(S3CertificateProvider.class)); + } + + @Test + void getCertificateProviderAcmProviderSuccess() { + certificateProviderFactory = new CertificateProviderFactory(true, Region.of("us-east-1"), + "arn:aws:acm:us-east-1:account:certificate/1234-567-856456", 5L, "test", false, TEST_SSL_CERTIFICATE_FILE, TEST_SSL_KEY_FILE); + final CertificateProvider certificateProvider = certificateProviderFactory.getCertificateProvider(); + + assertThat(certificateProvider, IsInstanceOf.instanceOf(ACMCertificateProvider.class)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/README.md b/data-prepper-plugins/prometheus-sink/README.md index a6daa90b79..22efe559d9 100644 --- a/data-prepper-plugins/prometheus-sink/README.md +++ b/data-prepper-plugins/prometheus-sink/README.md @@ -152,3 +152,10 @@ This plugin is compatible with Java 8. See - [CONTRIBUTING](https://github.com/opensearch-project/data-prepper/blob/main/CONTRIBUTING.md) - [monitoring](https://github.com/opensearch-project/data-prepper/blob/main/docs/monitoring.md) + The integration tests for this plugin do not run as part of the Data Prepper build. + +The following command runs the integration tests: + +``` +./gradlew :data-prepper-plugins:prometheus-sink:integrationTest -Dtests.prometheus.sink.http.endpoint= +``` diff --git a/data-prepper-plugins/prometheus-sink/build.gradle b/data-prepper-plugins/prometheus-sink/build.gradle index 079701bfeb..1055e4b3dc 100644 --- a/data-prepper-plugins/prometheus-sink/build.gradle +++ b/data-prepper-plugins/prometheus-sink/build.gradle @@ -21,6 +21,7 @@ dependencies { implementation 'software.amazon.awssdk:apache-client' implementation 'software.amazon.awssdk:sts' implementation 'software.amazon.awssdk:acm' + implementation 'com.github.scribejava:scribejava-core:8.3.3' testImplementation project(':data-prepper-test-common') } diff --git a/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java b/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java index 5f0301cd0d..5ae78307ef 100644 --- a/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java +++ b/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java @@ -49,10 +49,7 @@ public class PrometheusSinkServiceIT { " url: {0}\n" + " http_method: POST\n" + " auth_type: {1}\n" + - " ssl: false\n" + - " aws:\n" + - " region: {2}\n" + - " sts_role_arn: {3}\n"; + " ssl: false\n"; private PrometheusSinkConfiguration prometheusSinkConfiguration; @@ -65,7 +62,7 @@ public class PrometheusSinkServiceIT { @BeforeEach void setUp() throws JsonProcessingException{ this.urlString = System.getProperty("tests.prometheus.sink.http.endpoint"); - String[] values = { urlString,"unauthenticated","ap-south-1","arn:aws:iam::524239988944:role/app-test" }; + String[] values = { urlString,"unauthenticated"}; final String configYaml = MessageFormat.format(config, values); this.prometheusSinkConfiguration = objectMapper.readValue(configYaml, PrometheusSinkConfiguration.class); } diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java index 2f0de4e2e8..0cddb8cb6d 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java @@ -4,82 +4,66 @@ */ package org.opensearch.dataprepper.plugins.sink.prometheus; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.hc.client5.http.classic.methods.HttpPost; -import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; -import org.apache.hc.core5.http.ClassicHttpResponse; -import org.apache.hc.core5.http.ContentType; -import org.apache.hc.core5.http.HttpHeaders; -import org.apache.hc.core5.http.io.entity.StringEntity; +import com.github.scribejava.core.builder.api.DefaultApi20; +import com.github.scribejava.core.model.OAuth2AccessToken; +import com.github.scribejava.core.oauth.OAuth20Service; +import com.github.scribejava.core.builder.ServiceBuilder; import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.BearerTokenOptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import java.io.IOException; import java.time.Instant; -import java.time.OffsetDateTime; import java.time.ZoneOffset; -import java.util.Base64; -import java.util.Map; public class OAuthAccessTokenManager { - public static final String BASIC = "Basic "; - - public static final String BEARER = "Bearer "; - - public static final String APPLICATION_X_WWW_FORM_URLENCODED = "application/x-www-form-urlencoded"; - public static final String EXP = "exp"; - public static final String ACCESS_TOKEN = "access_token"; - - public static final String REFRESH_TOKEN = "refresh_token"; - - private final ObjectMapper objectMapper; - - private HttpClientBuilder httpClientBuilder; - - - public OAuthAccessTokenManager(final HttpClientBuilder httpClientBuilder){ - this.httpClientBuilder = httpClientBuilder; - this.objectMapper = new ObjectMapper(); - } + private static final Logger LOG = LoggerFactory.getLogger(OAuthAccessTokenManager.class); public String getAccessToken(final BearerTokenOptions bearerTokenOptions) { - HttpPost request = new HttpPost(bearerTokenOptions.getTokenURL()); - request.setHeader(HttpHeaders.CONTENT_TYPE, APPLICATION_X_WWW_FORM_URLENCODED); - request.setHeader(HttpHeaders.AUTHORIZATION, BASIC + base64Encode(bearerTokenOptions.getClientId() + ":" + bearerTokenOptions.getClientSecret())); - String requestBody = "grant_type=" + bearerTokenOptions.getGrantType() +"&refresh_token"+bearerTokenOptions.getRefreshToken()+"&scope=" + bearerTokenOptions.getScope(); - request.setEntity(new StringEntity(requestBody, ContentType.APPLICATION_FORM_URLENCODED)); - Map accessTokenMap; + OAuth20Service service = getOAuth20ServiceObj(bearerTokenOptions); + OAuth2AccessToken accessTokenObj = null; try { - ClassicHttpResponse response = (ClassicHttpResponse)httpClientBuilder.build().execute(request); - accessTokenMap = objectMapper.readValue(response.getEntity().getContent(),Map.class); - } catch (IOException e) { - throw new RuntimeException(e); + if(bearerTokenOptions.getRefreshToken() != null) { + accessTokenObj = new OAuth2AccessToken(bearerTokenOptions.getAccessToken(), bearerTokenOptions.getRefreshToken()); + accessTokenObj = service.refreshAccessToken(accessTokenObj.getRefreshToken()); + + }else { + accessTokenObj = service.getAccessTokenClientCredentialsGrant(); + } + bearerTokenOptions.setRefreshToken(accessTokenObj.getRefreshToken()); + bearerTokenOptions.setAccessToken(accessTokenObj.getAccessToken()); + bearerTokenOptions.setTokenExpired(accessTokenObj.getExpiresIn()); + }catch (Exception e) { + LOG.info("Exception : "+ e.getMessage() ); } - bearerTokenOptions.setRefreshToken(accessTokenMap.get(REFRESH_TOKEN)); - return BEARER + accessTokenMap.get(ACCESS_TOKEN); + return bearerTokenOptions.getAccessToken(); } - private static String base64Encode(String value) { - return java.util.Base64.getEncoder().encodeToString(value.getBytes()); - } - public boolean isTokenExpired(final String token){ - Base64.Decoder decoder = Base64.getUrlDecoder(); - String[] chunks = token.substring(6).split("\\."); - final Map tokenDetails; - try { - ObjectMapper objectMapper = new ObjectMapper(); - tokenDetails = objectMapper.readValue(new String(decoder.decode(chunks[1])), Map.class); - } catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - final String expTime = String.valueOf(tokenDetails.get(EXP)); - OffsetDateTime accessTokenExpTimeStamp = Instant.ofEpochMilli(Long.valueOf(expTime ) * 1000l).atOffset(ZoneOffset.UTC); + public boolean isTokenExpired(final Integer tokenExpired){ final Instant systemCurrentTimeStamp = Instant.now().atOffset(ZoneOffset.UTC).toInstant(); - if(systemCurrentTimeStamp.compareTo(accessTokenExpTimeStamp.toInstant())>=0) { + Instant accessTokenExpTimeStamp = systemCurrentTimeStamp.plusSeconds(tokenExpired); + if(systemCurrentTimeStamp.compareTo(accessTokenExpTimeStamp)>=0) { return true; } return false; } + + private OAuth20Service getOAuth20ServiceObj(BearerTokenOptions bearerTokenOptions){ + return new ServiceBuilder(bearerTokenOptions.getClientId()) + .apiSecret(bearerTokenOptions.getClientSecret()) + .defaultScope(bearerTokenOptions.getScope()) + .build(new DefaultApi20() { + @Override + public String getAccessTokenEndpoint() { + return bearerTokenOptions.getTokenUrl(); + } + + @Override + protected String getAuthorizationBaseUrl() { + return bearerTokenOptions.getTokenUrl(); + } + }); + } + } diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java index 2878d54f3c..3e409dc66e 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java @@ -16,6 +16,7 @@ import org.apache.hc.core5.ssl.TrustStrategy; import org.apache.hc.core5.util.Timeout; import org.opensearch.dataprepper.plugins.certificate.CertificateProvider; +import org.opensearch.dataprepper.plugins.certificate.s3.CertificateProviderFactory; import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; import javax.net.ssl.SSLContext; diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BearerTokenOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BearerTokenOptions.java index 3ff05da0b8..cbe7f53434 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BearerTokenOptions.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BearerTokenOptions.java @@ -19,7 +19,7 @@ public class BearerTokenOptions { @JsonProperty("token_url") @NotNull(message = "token url type is mandatory for refresh token") - private String tokenURL; + private String tokenUrl; @JsonProperty("grant_type") @NotNull(message = "grant type is mandatory for refresh token") @@ -31,6 +31,10 @@ public class BearerTokenOptions { private String refreshToken; + private String accessToken; + + private Integer tokenExpired; + public String getScope() { return scope; } @@ -43,6 +47,14 @@ public String getRefreshToken() { return refreshToken; } + public String getAccessToken() { + return accessToken; + } + + public Integer getTokenExpired() { + return tokenExpired; + } + public String getClientId() { return clientId; } @@ -51,11 +63,20 @@ public String getClientSecret() { return clientSecret; } - public String getTokenURL() { - return tokenURL; - } - public void setRefreshToken(String refreshToken) { this.refreshToken = refreshToken; } + + public void setAccessToken(String accessToken) { + this.accessToken = accessToken; + } + + public void setTokenExpired(Integer tokenExpired) { + this.tokenExpired = tokenExpired; + } + + public String getTokenUrl() { + return tokenUrl; + } + } diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java index 597757212a..9fd067d1af 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java @@ -7,7 +7,7 @@ import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; -import org.opensearch.dataprepper.plugins.sink.prometheus.AwsRequestSigningApacheInterceptor; +import org.opensearch.dataprepper.aws.api.AwsRequestSigningApacheInterceptor; import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java index 7c69976cea..4d2cc4d37f 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java @@ -7,6 +7,7 @@ import com.arpnetworking.metrics.prometheus.Remote; import com.arpnetworking.metrics.prometheus.Types; import com.fasterxml.jackson.databind.ObjectMapper; +import io.micrometer.core.instrument.Counter; import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; import org.apache.hc.client5.http.io.HttpClientConnectionManager; import org.apache.hc.client5.http.protocol.HttpClientContext; @@ -26,7 +27,7 @@ import org.opensearch.dataprepper.model.metric.JacksonSummary; import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.sink.prometheus.certificate.CertificateProviderFactory; +import org.opensearch.dataprepper.plugins.certificate.s3.CertificateProviderFactory; import org.opensearch.dataprepper.plugins.sink.prometheus.FailedHttpResponseInterceptor; import org.opensearch.dataprepper.plugins.sink.prometheus.HttpEndPointResponse; import org.opensearch.dataprepper.plugins.sink.prometheus.OAuthAccessTokenManager; @@ -99,6 +100,14 @@ public class PrometheusSinkService { private static final Pattern PREFIX_PATTERN = Pattern.compile("^[^a-zA-Z_:]"); private static final Pattern BODY_PATTERN = Pattern.compile("[^a-zA-Z0-9_:]"); + private final Counter prometheusSinkRecordsSuccessCounter; + + private final Counter prometheusSinkRecordsFailedCounter; + + public static final String PROMETHEUS_SINK_RECORDS_SUCCESS_COUNTER = "prometheusSinkRecordsSuccessPushToEndPoint"; + + public static final String PROMETHEUS_SINK_RECORDS_FAILED_COUNTER = "prometheusSinkRecordsFailedToPushEndPoint"; + public PrometheusSinkService(final PrometheusSinkConfiguration prometheusSinkConfiguration, final DlqPushHandler dlqPushHandler, final HttpClientBuilder httpClientBuilder, @@ -110,13 +119,19 @@ public PrometheusSinkService(final PrometheusSinkConfiguration prometheusSinkCon this.bufferedEventHandles = new LinkedList<>(); this.httpClientBuilder = httpClientBuilder; this.httpPluginSetting = httpPluginSetting; - this.oAuthAccessTokenManager = new OAuthAccessTokenManager(httpClientBuilder); + this.oAuthAccessTokenManager = new OAuthAccessTokenManager(); if (prometheusSinkConfiguration.isSsl() || prometheusSinkConfiguration.useAcmCertForSSL()) { - this.certificateProviderFactory = new CertificateProviderFactory(prometheusSinkConfiguration); + this.certificateProviderFactory = new CertificateProviderFactory(prometheusSinkConfiguration.useAcmCertForSSL(), + prometheusSinkConfiguration.getAwsAuthenticationOptions().getAwsRegion(), prometheusSinkConfiguration.getAcmCertificateArn(), + prometheusSinkConfiguration.getAcmCertIssueTimeOutMillis(), prometheusSinkConfiguration.getAcmPrivateKeyPassword(), + prometheusSinkConfiguration.isSslCertAndKeyFileInS3(), prometheusSinkConfiguration.getSslCertificateFile(), + prometheusSinkConfiguration.getSslKeyFile()); prometheusSinkConfiguration.validateAndInitializeCertAndKeyFileInS3(); this.httpClientConnectionManager = new HttpClientSSLConnectionManager() .createHttpClientConnectionManager(prometheusSinkConfiguration, certificateProviderFactory); } + this.prometheusSinkRecordsSuccessCounter = pluginMetrics.counter(PROMETHEUS_SINK_RECORDS_SUCCESS_COUNTER); + this.prometheusSinkRecordsFailedCounter = pluginMetrics.counter(PROMETHEUS_SINK_RECORDS_FAILED_COUNTER); this.httpAuthOptions = buildAuthHttpSinkObjectsByConfig(prometheusSinkConfiguration); } @@ -163,15 +178,18 @@ public void output(Collection> records) { HttpEndPointResponse failedHttpEndPointResponses = null; try { failedHttpEndPointResponses = pushToEndPoint(bytes); + + if (failedHttpEndPointResponses != null) { + logFailedData(failedHttpEndPointResponses); + releaseEventHandles(Boolean.FALSE); + prometheusSinkRecordsFailedCounter.increment(); + } else { + LOG.info("data pushed to the end point successfully"); + releaseEventHandles(Boolean.TRUE); + prometheusSinkRecordsSuccessCounter.increment(); + } } catch (IOException e) { - LOG.info("Error while pushing to the end point"); - } - if (failedHttpEndPointResponses != null) { - logFailedData(failedHttpEndPointResponses); - releaseEventHandles(Boolean.FALSE); - } else { - LOG.info("data pushed to the end point successfully"); - releaseEventHandles(Boolean.TRUE); + LOG.error("Error while pushing to the end point ", e); } }); @@ -292,7 +310,7 @@ private HttpEndPointResponse pushToEndPoint(final byte[] data) throws IOExceptio try { if(AuthTypeOptions.BEARER_TOKEN.equals(prometheusSinkConfiguration.getAuthType())) - accessTokenIfExpired(classicHttpRequestBuilder.getFirstHeader(AUTHORIZATION).getValue(),prometheusSinkConfiguration.getUrl()); + accessTokenIfExpired(prometheusSinkConfiguration.getAuthentication().getBearerTokenOptions().getTokenExpired(),prometheusSinkConfiguration.getUrl()); httpAuthOptions.get(prometheusSinkConfiguration.getUrl()).getHttpClientBuilder().build() .execute(classicHttpRequestBuilder.build(), HttpClientContext.create()); @@ -402,8 +420,8 @@ private ClassicRequestBuilder buildRequestByHTTPMethodType(final HTTPMethodOptio return classicRequestBuilder; } - private void accessTokenIfExpired(final String token,final String url){ - if(oAuthAccessTokenManager.isTokenExpired(token)) { + private void accessTokenIfExpired(final Integer tokenExpired,final String url){ + if(oAuthAccessTokenManager.isTokenExpired(tokenExpired)) { httpAuthOptions.get(url).getClassicHttpRequestBuilder() .setHeader(AUTHORIZATION, oAuthAccessTokenManager.getAccessToken(prometheusSinkConfiguration.getAuthentication().getBearerTokenOptions())); } diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManagerTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManagerTest.java deleted file mode 100644 index 18cba54d6e..0000000000 --- a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManagerTest.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ -package org.opensearch.dataprepper.plugins.sink.prometheus; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; -import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; -import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; -import org.apache.hc.client5.http.impl.classic.CloseableHttpResponse; -import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; -import org.apache.hc.core5.http.ClassicHttpRequest; -import org.apache.hc.core5.http.HttpEntity; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.Mock; -import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.BearerTokenOptions; - -import java.io.ByteArrayInputStream; -import java.io.IOException; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.mockito.ArgumentMatchers.any; -import static org.hamcrest.Matchers.equalTo; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public class OAuthAccessTokenManagerTest { - - private static final String bearerTokenYaml = - " client_id: 0oaafr4j79segrYGC5d7\n" + - " client_secret: fFel-3FutCXAOndezEsOVlght6D6DR4OIt7G5D1_oJ6w0wNoaYtgU17JdyXmGf0M\n" + - " token_url: https://localhost/oauth2/default/v1/token\n" + - " grant_type: client_credentials\n" + - " scope: httpSink"; - - private final String tokenJson = "{\"token_type\": \"Bearer\",\n" + - " \"expires_in\": 3600,\n" + - " \"access_token\": \"eyJraWQiOiJtU0xMalBfMUFFUFV1VzlqRkhiSmc4UXlRSm1pdFBHamZOczR2eFJ2WUx3IiwiYWxnIjoiUlMyNTYifQ.eyJ2ZXIiOjEsImp0aSI6IkFULm1jcVVfTzJvd1RuNUwwTjYwLTg0WFBaUU4xNXBfX191X2VKaGxBdEhsQXciLCJpc3MiOiJodHRwczovL2Rldi03NTA1MDk1Ni5va3RhLmNvbS9vYXV0aDIvZGVmYXVsdCIsImF1ZCI6ImFwaTovL2RlZmF1bHQiLCJpYXQiOjE2ODk5MTQyNDMsImV4cCI6MTY4OTkxNzg0MywiY2lkIjoiMG9hYWZyNGo3OXNlZ3JZR0M1ZDciLCJzY3AiOlsiaHR0cFNpbmsiXSwic3ViIjoiMG9hYWZyNGo3OXNlZ3JZR0M1ZDcifQ.d6NU2f9Qlrk9N2L1cfY5KhWIc7DXE1oJXPsss2OMb-JFYZvwhMpIpv1IwaY7ikDQYDKlcYYt-3XKBj0IxPnugigO_OTv12LpvHyMBhUKDo5YrxKZqksme7S0IKYoLNFVsq3ViqVsHgDy3RGWL1ih-rGXN-8A-9LsqloEnCn7SzFj446aep9bygp1PIA5pBgrVwKw0QPal4HDOu9cTKwclNiWRLJ80H_q83vDeQNnW9YI8A-nTy9ujghVF9JJVsB4FTHMlfclt93SJ4qCA_9He_VFkSs5pFS4plCAzONA0XU53lf7NXJ3bs18HPJkm3-B2b1f6Q9kGUU6e2ZQ2d6dvw\",\n" + - " \"scope\": \"httpSink\"\n" + - "}"; - - private ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory().enable(YAMLGenerator.Feature.USE_PLATFORM_LINE_BREAKS)); - - @Mock - private HttpClientBuilder httpClientBuilder; - - @Mock - private CloseableHttpClient httpClient; - - private OAuthAccessTokenManager oAuthAccessTokenManager; - - @BeforeEach - public void setup() throws IOException { - httpClientBuilder = mock(HttpClientBuilder.class); - httpClient = mock(CloseableHttpClient.class); - CloseableHttpResponse response = mock(CloseableHttpResponse.class); - HttpEntity entity = mock(HttpEntity.class); - - final ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(tokenJson.getBytes()); - when(entity.getContent()).thenReturn(byteArrayInputStream); - when(httpClientBuilder.build()).thenReturn(httpClient); - when(httpClient.execute(any(ClassicHttpRequest.class))).thenReturn(response); - when(response.getEntity()).thenReturn(entity); - this.oAuthAccessTokenManager = new OAuthAccessTokenManager(httpClientBuilder); - } - - @Test - public void bearer_token_refresh_token_test() throws IOException { - BearerTokenOptions bearerTokenOptions = objectMapper.readValue(bearerTokenYaml,BearerTokenOptions.class); - final String refreshToken = oAuthAccessTokenManager.getAccessToken(bearerTokenOptions); - assertThat(refreshToken,equalTo("Bearer eyJraWQiOiJtU0xMalBfMUFFUFV1VzlqRkhiSmc4UXlRSm1pdFBHamZOczR2eFJ2WUx3IiwiYWxnIjoiUlMyNTYifQ.eyJ2ZXIiOjEsImp0aSI6IkFULm1jcVVfTzJvd1RuNUwwTjYwLTg0WFBaUU4xNXBfX191X2VKaGxBdEhsQXciLCJpc3MiOiJodHRwczovL2Rldi03NTA1MDk1Ni5va3RhLmNvbS9vYXV0aDIvZGVmYXVsdCIsImF1ZCI6ImFwaTovL2RlZmF1bHQiLCJpYXQiOjE2ODk5MTQyNDMsImV4cCI6MTY4OTkxNzg0MywiY2lkIjoiMG9hYWZyNGo3OXNlZ3JZR0M1ZDciLCJzY3AiOlsiaHR0cFNpbmsiXSwic3ViIjoiMG9hYWZyNGo3OXNlZ3JZR0M1ZDcifQ.d6NU2f9Qlrk9N2L1cfY5KhWIc7DXE1oJXPsss2OMb-JFYZvwhMpIpv1IwaY7ikDQYDKlcYYt-3XKBj0IxPnugigO_OTv12LpvHyMBhUKDo5YrxKZqksme7S0IKYoLNFVsq3ViqVsHgDy3RGWL1ih-rGXN-8A-9LsqloEnCn7SzFj446aep9bygp1PIA5pBgrVwKw0QPal4HDOu9cTKwclNiWRLJ80H_q83vDeQNnW9YI8A-nTy9ujghVF9JJVsB4FTHMlfclt93SJ4qCA_9He_VFkSs5pFS4plCAzONA0XU53lf7NXJ3bs18HPJkm3-B2b1f6Q9kGUU6e2ZQ2d6dvw")); - } - - @Test - public void bearer_token_refresh_token_expiry_test() throws IOException { - String bearerToken = "\"Bearer eyJraWQiOiJtU0xMalBfMUFFUFV1VzlqRkhiSmc4UXlRSm1pdFBHamZOczR2eFJ2WUx3IiwiYWxnIjoiUlMyNTYifQ.eyJ2ZXIiOjEsImp0aSI6IkFULm1jcVVfTzJvd1RuNUwwTjYwLTg0WFBaUU4xNXBfX191X2VKaGxBdEhsQXciLCJpc3MiOiJodHRwczovL2Rldi03NTA1MDk1Ni5va3RhLmNvbS9vYXV0aDIvZGVmYXVsdCIsImF1ZCI6ImFwaTovL2RlZmF1bHQiLCJpYXQiOjE2ODk5MTQyNDMsImV4cCI6MTY4OTkxNzg0MywiY2lkIjoiMG9hYWZyNGo3OXNlZ3JZR0M1ZDciLCJzY3AiOlsiaHR0cFNpbmsiXSwic3ViIjoiMG9hYWZyNGo3OXNlZ3JZR0M1ZDcifQ.d6NU2f9Qlrk9N2L1cfY5KhWIc7DXE1oJXPsss2OMb-JFYZvwhMpIpv1IwaY7ikDQYDKlcYYt-3XKBj0IxPnugigO_OTv12LpvHyMBhUKDo5YrxKZqksme7S0IKYoLNFVsq3ViqVsHgDy3RGWL1ih-rGXN-8A-9LsqloEnCn7SzFj446aep9bygp1PIA5pBgrVwKw0QPal4HDOu9cTKwclNiWRLJ80H_q83vDeQNnW9YI8A-nTy9ujghVF9JJVsB4FTHMlfclt93SJ4qCA_9He_VFkSs5pFS4plCAzONA0XU53lf7NXJ3bs18HPJkm3-B2b1f6Q9kGUU6e2ZQ2d6dvw\""; - final boolean refreshToken = oAuthAccessTokenManager.isTokenExpired(bearerToken); - assertThat(refreshToken,equalTo(Boolean.TRUE)); - } -} diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactoryTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactoryTest.java deleted file mode 100644 index 91d82611fe..0000000000 --- a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/CertificateProviderFactoryTest.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.sink.prometheus.certificate; - -import org.hamcrest.core.IsInstanceOf; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.opensearch.dataprepper.plugins.certificate.CertificateProvider; -import org.opensearch.dataprepper.plugins.certificate.acm.ACMCertificateProvider; -import org.opensearch.dataprepper.plugins.certificate.file.FileCertificateProvider; -import org.opensearch.dataprepper.plugins.certificate.s3.S3CertificateProvider; -import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.AwsAuthenticationOptions; -import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; -import software.amazon.awssdk.regions.Region; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -class CertificateProviderFactoryTest { - private final String TEST_SSL_CERTIFICATE_FILE = getClass().getClassLoader().getResource("test_cert.crt").getFile(); - private final String TEST_SSL_KEY_FILE = getClass().getClassLoader().getResource("test_decrypted_key.key").getFile(); - - private PrometheusSinkConfiguration prometheusSinkConfiguration; - - private AwsAuthenticationOptions awsAuthenticationOptions; - private CertificateProviderFactory certificateProviderFactory; - - @BeforeEach - void setUp() { - prometheusSinkConfiguration = mock(PrometheusSinkConfiguration.class); - awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); - } - - @Test - void getCertificateProviderFileCertificateProviderSuccess() { - when(prometheusSinkConfiguration.isSsl()).thenReturn(true); - when(prometheusSinkConfiguration.getSslCertificateFile()).thenReturn(TEST_SSL_CERTIFICATE_FILE); - when(prometheusSinkConfiguration.getSslKeyFile()).thenReturn(TEST_SSL_KEY_FILE); - - certificateProviderFactory = new CertificateProviderFactory(prometheusSinkConfiguration); - final CertificateProvider certificateProvider = certificateProviderFactory.getCertificateProvider(); - - assertThat(certificateProvider, IsInstanceOf.instanceOf(FileCertificateProvider.class)); - } - - @Test - void getCertificateProviderS3ProviderSuccess() { - when(prometheusSinkConfiguration.isSslCertAndKeyFileInS3()).thenReturn(true); - when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.of("us-east-1")); - when(prometheusSinkConfiguration.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); - when(prometheusSinkConfiguration.getSslCertificateFile()).thenReturn("s3://data/certificate/test_cert.crt"); - when(prometheusSinkConfiguration.getSslKeyFile()).thenReturn("s3://data/certificate/test_decrypted_key.key"); - - certificateProviderFactory = new CertificateProviderFactory(prometheusSinkConfiguration); - final CertificateProvider certificateProvider = certificateProviderFactory.getCertificateProvider(); - - assertThat(certificateProvider, IsInstanceOf.instanceOf(S3CertificateProvider.class)); - } - - @Test - void getCertificateProviderAcmProviderSuccess() { - when(prometheusSinkConfiguration.useAcmCertForSSL()).thenReturn(true); - when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.of("us-east-1")); - when(prometheusSinkConfiguration.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); - when(prometheusSinkConfiguration.getAcmCertificateArn()).thenReturn("arn:aws:acm:us-east-1:account:certificate/1234-567-856456"); - - certificateProviderFactory = new CertificateProviderFactory(prometheusSinkConfiguration); - final CertificateProvider certificateProvider = certificateProviderFactory.getCertificateProvider(); - - assertThat(certificateProvider, IsInstanceOf.instanceOf(ACMCertificateProvider.class)); - } -} \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManagerTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManagerTest.java index 3ebf01ca50..b1969c5e58 100644 --- a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManagerTest.java +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManagerTest.java @@ -9,6 +9,7 @@ import org.junit.jupiter.api.Test; import org.opensearch.dataprepper.plugins.certificate.CertificateProvider; import org.opensearch.dataprepper.plugins.certificate.file.FileCertificateProvider; +import org.opensearch.dataprepper.plugins.certificate.s3.CertificateProviderFactory; import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; import java.io.IOException; @@ -41,10 +42,9 @@ public void create_httpClientConnectionManager_with_ssl_file_test() { CertificateProvider provider = new FileCertificateProvider(prometheusSinkConfiguration.getSslCertificateFile(), prometheusSinkConfiguration.getSslKeyFile()); when(certificateProviderFactory.getCertificateProvider()).thenReturn(provider); - CertificateProviderFactory providerFactory = new CertificateProviderFactory(prometheusSinkConfiguration); httpClientSSLConnectionManager = new HttpClientSSLConnectionManager(); HttpClientConnectionManager httpClientConnectionManager = httpClientSSLConnectionManager - .createHttpClientConnectionManager(prometheusSinkConfiguration, providerFactory); + .createHttpClientConnectionManager(prometheusSinkConfiguration, certificateProviderFactory); assertNotNull(httpClientConnectionManager); } } From c9692a83dee4b92224816762540398c055194781 Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Fri, 18 Aug 2023 15:50:37 +0530 Subject: [PATCH 06/18] Resolved review comments for issue #1744. Signed-off-by: mallikagogoi7 --- .../FailedHttpResponseInterceptor.java | 2 +- .../prometheus/OAuthAccessTokenManager.java | 2 +- .../service/PrometheusSinkService.java | 94 +++++++++---------- .../service/PrometheusSinkServiceTest.java | 14 ++- 4 files changed, 59 insertions(+), 53 deletions(-) diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java index f15709b942..1a48ae5f19 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java @@ -28,7 +28,7 @@ public FailedHttpResponseInterceptor(final String url){ } @Override - public void process(HttpResponse response, EntityDetails entity, HttpContext context) throws IOException { + public void process(final HttpResponse response, final EntityDetails entity, final HttpContext context) throws IOException { if (response.getCode() == ERROR_CODE_500 || response.getCode() == ERROR_CODE_400 || response.getCode() == ERROR_CODE_404 || diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java index 0cddb8cb6d..5c023876e0 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java @@ -49,7 +49,7 @@ public boolean isTokenExpired(final Integer tokenExpired){ return false; } - private OAuth20Service getOAuth20ServiceObj(BearerTokenOptions bearerTokenOptions){ + private OAuth20Service getOAuth20ServiceObj(final BearerTokenOptions bearerTokenOptions){ return new ServiceBuilder(bearerTokenOptions.getClientId()) .apiSecret(bearerTokenOptions.getClientSecret()) .defaultScope(bearerTokenOptions.getScope()) diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java index 4d2cc4d37f..cfdcbd1f2f 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java @@ -139,7 +139,7 @@ public PrometheusSinkService(final PrometheusSinkConfiguration prometheusSinkCon * This method process buffer records and send to Http End points based on configured codec * @param records Collection of Event */ - public void output(Collection> records) { + public void output(final Collection> records) { reentrantLock.lock(); try { records.forEach(record -> { @@ -148,23 +148,23 @@ public void output(Collection> records) { if (event.getMetadata().getEventType().equals("METRIC")) { Remote.WriteRequest message = null; if (event instanceof JacksonGauge) { - JacksonGauge jacksonGauge = (JacksonGauge) event; + final JacksonGauge jacksonGauge = (JacksonGauge) event; message = buildRemoteWriteRequest(jacksonGauge.getTime(), jacksonGauge.getStartTime(), jacksonGauge.getValue(), jacksonGauge.getAttributes(),jacksonGauge.getName()); } else if (event instanceof JacksonSum) { - JacksonSum jacksonSum = (JacksonSum) event; + final JacksonSum jacksonSum = (JacksonSum) event; message = buildRemoteWriteRequest(jacksonSum.getTime(), jacksonSum.getStartTime(), jacksonSum.getValue(), jacksonSum.getAttributes(), jacksonSum.getName()); } else if (event instanceof JacksonSummary) { - JacksonSummary jacksonSummary = (JacksonSummary) event; + final JacksonSummary jacksonSummary = (JacksonSummary) event; message = buildRemoteWriteRequest(jacksonSummary.getTime(), jacksonSummary.getStartTime(), jacksonSummary.getSum(), jacksonSummary.getAttributes(), jacksonSummary.getName()); } else if (event instanceof JacksonHistogram) { - JacksonHistogram jacksonHistogram = (JacksonHistogram) event; + final JacksonHistogram jacksonHistogram = (JacksonHistogram) event; message = buildRemoteWriteRequest(jacksonHistogram.getTime(), jacksonHistogram.getStartTime(), jacksonHistogram.getSum(), jacksonHistogram.getAttributes(), jacksonHistogram.getName()); } else if (event instanceof JacksonExponentialHistogram) { - JacksonExponentialHistogram jacksonExpHistogram = (JacksonExponentialHistogram) event; + final JacksonExponentialHistogram jacksonExpHistogram = (JacksonExponentialHistogram) event; message = buildRemoteWriteRequest(jacksonExpHistogram.getTime(), jacksonExpHistogram.getStartTime(), jacksonExpHistogram.getSum(), jacksonExpHistogram.getAttributes(), jacksonExpHistogram.getName()); } else { @@ -175,23 +175,15 @@ public void output(Collection> records) { if (event.getEventHandle() != null) { this.bufferedEventHandles.add(event.getEventHandle()); } - HttpEndPointResponse failedHttpEndPointResponses = null; - try { - failedHttpEndPointResponses = pushToEndPoint(bytes); - - if (failedHttpEndPointResponses != null) { - logFailedData(failedHttpEndPointResponses); - releaseEventHandles(Boolean.FALSE); - prometheusSinkRecordsFailedCounter.increment(); - } else { - LOG.info("data pushed to the end point successfully"); - releaseEventHandles(Boolean.TRUE); - prometheusSinkRecordsSuccessCounter.increment(); - } - } catch (IOException e) { - LOG.error("Error while pushing to the end point ", e); + HttpEndPointResponse failedHttpEndPointResponses = pushToEndPoint(bytes); + + if (failedHttpEndPointResponses != null) { + logFailedData(failedHttpEndPointResponses); + releaseEventHandles(Boolean.FALSE); + } else { + LOG.info("data pushed to the end point successfully"); + releaseEventHandles(Boolean.TRUE); } - }); }finally { @@ -207,18 +199,18 @@ public void output(Collection> records) { * @param attributeMap attributes * @param metricName metricName */ - private static Remote.WriteRequest buildRemoteWriteRequest(String time, String startTime, - Double value, Map attributeMap, final String metricName) { - Remote.WriteRequest.Builder writeRequestBuilder = Remote.WriteRequest.newBuilder(); + private static Remote.WriteRequest buildRemoteWriteRequest(final String time, final String startTime, + final Double value, final Map attributeMap, final String metricName) { + final Remote.WriteRequest.Builder writeRequestBuilder = Remote.WriteRequest.newBuilder(); - Types.TimeSeries.Builder timeSeriesBuilder = Types.TimeSeries.newBuilder(); + final Types.TimeSeries.Builder timeSeriesBuilder = Types.TimeSeries.newBuilder(); - List arrayList = new ArrayList<>(); + final List arrayList = new ArrayList<>(); setMetricName(metricName, arrayList); prepareLabelList(attributeMap, arrayList); - Types.Sample.Builder prometheusSampleBuilder = Types.Sample.newBuilder(); + final Types.Sample.Builder prometheusSampleBuilder = Types.Sample.newBuilder(); long timeStampVal; if (time != null) { timeStampVal = getTimeStampVal(time); @@ -227,44 +219,44 @@ private static Remote.WriteRequest buildRemoteWriteRequest(String time, String s } prometheusSampleBuilder.setValue(value).setTimestamp(timeStampVal); - Types.Sample prometheusSample = prometheusSampleBuilder.build(); + final Types.Sample prometheusSample = prometheusSampleBuilder.build(); timeSeriesBuilder.addAllLabels(arrayList); timeSeriesBuilder.addAllSamples(Arrays.asList(prometheusSample)); - Types.TimeSeries timeSeries = timeSeriesBuilder.build(); + final Types.TimeSeries timeSeries = timeSeriesBuilder.build(); writeRequestBuilder.addAllTimeseries(Arrays.asList(timeSeries)); return writeRequestBuilder.build(); } - private static void prepareLabelList(Map hashMap, List arrayList) { - for (Map.Entry entry : hashMap.entrySet()) { - String key = sanitizeName(entry.getKey()); - Object value = entry.getValue(); + private static void prepareLabelList(final Map hashMap, final List arrayList) { + for (final Map.Entry entry : hashMap.entrySet()) { + final String key = sanitizeName(entry.getKey()); + final Object value = entry.getValue(); if (entry.getValue() instanceof Map) { - Object innerMap = entry.getValue(); + final Object innerMap = entry.getValue(); prepareLabelList(objectMapper.convertValue(innerMap, Map.class), arrayList); continue; } - Types.Label.Builder labelBuilder = Types.Label.newBuilder(); + final Types.Label.Builder labelBuilder = Types.Label.newBuilder(); labelBuilder.setName(key).setValue(value.toString()); - Types.Label label = labelBuilder.build(); + final Types.Label label = labelBuilder.build(); arrayList.add(label); } } - private static String sanitizeName(String name) { + private static String sanitizeName(final String name) { return BODY_PATTERN .matcher(PREFIX_PATTERN.matcher(name).replaceFirst("_")) .replaceAll("_"); } - private static long getTimeStampVal(String time) { - LocalDateTime localDateTimeParse = LocalDateTime.parse(time, + private static long getTimeStampVal(final String time) { + final LocalDateTime localDateTimeParse = LocalDateTime.parse(time, DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSSSSSSS'Z'")); - LocalDateTime localDateTime = LocalDateTime.parse(localDateTimeParse.toString()); - ZonedDateTime zdt = ZonedDateTime.of(localDateTime, ZoneId.systemDefault()); + final LocalDateTime localDateTime = LocalDateTime.parse(localDateTimeParse.toString()); + final ZonedDateTime zdt = ZonedDateTime.of(localDateTime, ZoneId.systemDefault()); return zdt.toInstant().toEpochMilli(); } @@ -273,7 +265,7 @@ private static long getTimeStampVal(String time) { * @param endPointResponses HttpEndPointResponses. */ private void logFailedData(final HttpEndPointResponse endPointResponses) { - FailedDlqData failedDlqData = + final FailedDlqData failedDlqData = FailedDlqData.builder() .withUrl(endPointResponses.getUrl()) .withMessage(endPointResponses.getErrorMessage()) @@ -284,7 +276,7 @@ private void logFailedData(final HttpEndPointResponse endPointResponses) { } private void releaseEventHandles(final boolean result) { - for (EventHandle eventHandle : bufferedEventHandles) { + for (final EventHandle eventHandle : bufferedEventHandles) { eventHandle.release(result); } bufferedEventHandles.clear(); @@ -294,28 +286,30 @@ private void releaseEventHandles(final boolean result) { * * This method pushes bufferData to configured HttpEndPoints * @param data byte[] data. */ - private HttpEndPointResponse pushToEndPoint(final byte[] data) throws IOException { + private HttpEndPointResponse pushToEndPoint(final byte[] data) { HttpEndPointResponse httpEndPointResponses = null; final ClassicRequestBuilder classicHttpRequestBuilder = httpAuthOptions.get(prometheusSinkConfiguration.getUrl()).getClassicHttpRequestBuilder(); - final byte[] compressedBufferData = Snappy.compress(data); - HttpEntity entity = new ByteArrayEntity(compressedBufferData, - ContentType.create(prometheusSinkConfiguration.getContentType()), prometheusSinkConfiguration.getEncoding()); - - classicHttpRequestBuilder.setEntity(entity); classicHttpRequestBuilder.addHeader("Content-Encoding", prometheusSinkConfiguration.getEncoding()); classicHttpRequestBuilder.addHeader("Content-Type", prometheusSinkConfiguration.getContentType()); classicHttpRequestBuilder.addHeader("X-Prometheus-Remote-Write-Version", prometheusSinkConfiguration.getRemoteWriteVersion()); try { + final byte[] compressedBufferData = Snappy.compress(data); + final HttpEntity entity = new ByteArrayEntity(compressedBufferData, + ContentType.create(prometheusSinkConfiguration.getContentType()), prometheusSinkConfiguration.getEncoding()); + + classicHttpRequestBuilder.setEntity(entity); if(AuthTypeOptions.BEARER_TOKEN.equals(prometheusSinkConfiguration.getAuthType())) accessTokenIfExpired(prometheusSinkConfiguration.getAuthentication().getBearerTokenOptions().getTokenExpired(),prometheusSinkConfiguration.getUrl()); httpAuthOptions.get(prometheusSinkConfiguration.getUrl()).getHttpClientBuilder().build() .execute(classicHttpRequestBuilder.build(), HttpClientContext.create()); LOG.info("Records successfully pushed to endpoint {}", prometheusSinkConfiguration.getUrl()); + prometheusSinkRecordsSuccessCounter.increment(); } catch (IOException e) { + prometheusSinkRecordsFailedCounter.increment(); LOG.info("Records failed to push endpoint {}"); LOG.error("Exception while pushing buffer data to end point. URL : {}, Exception : ", prometheusSinkConfiguration.getUrl(), e); httpEndPointResponses = new HttpEndPointResponse(prometheusSinkConfiguration.getUrl(), HttpStatus.SC_INTERNAL_SERVER_ERROR, e.getMessage()); diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java index 0c7226ad50..2e576215eb 100644 --- a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java @@ -8,6 +8,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; +import io.micrometer.core.instrument.Counter; import org.apache.hc.client5.http.impl.auth.BasicCredentialsProvider; import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; import org.apache.hc.client5.http.impl.classic.CloseableHttpResponse; @@ -43,8 +44,9 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.when; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.lenient; public class PrometheusSinkServiceTest { @@ -97,6 +99,10 @@ public class PrometheusSinkServiceTest { private CloseableHttpResponse closeableHttpResponse; + private Counter prometheusSinkRecordsSuccessCounter; + + private Counter prometheusSinkRecordsFailedCounter; + @BeforeEach void setup() throws IOException { this.pluginMetrics = mock(PluginMetrics.class); @@ -107,10 +113,15 @@ void setup() throws IOException { this.awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); this.closeableHttpClient = mock(CloseableHttpClient.class); this.closeableHttpResponse = mock(CloseableHttpResponse.class); + this.prometheusSinkRecordsSuccessCounter = mock(Counter.class); + this.prometheusSinkRecordsFailedCounter = mock(Counter.class); lenient().when(httpClientBuilder.setConnectionManager(null)).thenReturn(httpClientBuilder); lenient().when(httpClientBuilder.addResponseInterceptorLast(any(FailedHttpResponseInterceptor.class))).thenReturn(httpClientBuilder); lenient().when(httpClientBuilder.build()).thenReturn(closeableHttpClient); lenient().when(closeableHttpClient.execute(any(ClassicHttpRequest.class),any(HttpClientContext.class))).thenReturn(closeableHttpResponse); + when(pluginMetrics.counter(PrometheusSinkService.PROMETHEUS_SINK_RECORDS_SUCCESS_COUNTER)).thenReturn(prometheusSinkRecordsSuccessCounter); + when(pluginMetrics.counter(PrometheusSinkService.PROMETHEUS_SINK_RECORDS_FAILED_COUNTER)).thenReturn(prometheusSinkRecordsFailedCounter); + } PrometheusSinkService createObjectUnderTest(final int eventCount, final PrometheusSinkConfiguration httpSinkConfig) throws NoSuchFieldException, IllegalAccessException { @@ -160,6 +171,7 @@ void prometheus_sink_service_test_output_with_single_record_for_jackson_sum() th .withData("{\"message\":\"c3f847eb-333a-49c3-a4cd-54715ad1b58a\"}") .withEventMetadata(eventMetadata).build()); Collection> records = List.of(eventRecord); + objectUnderTest.output(records); assertDoesNotThrow(() -> { objectUnderTest.output(records);}); } From cbeb4b685fa56b91b6c79d21b6e3212e26469805 Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Mon, 21 Aug 2023 15:45:01 +0530 Subject: [PATCH 07/18] Fixed Failed Response code for issue #1744. Signed-off-by: mallikagogoi7 --- .../plugins/sink/prometheus/FailedHttpResponseInterceptor.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java index 1a48ae5f19..3bb73574ad 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java @@ -21,6 +21,8 @@ public class FailedHttpResponseInterceptor implements HttpResponseInterceptor { public static final int ERROR_CODE_501 = 501; + public static final int ERROR_CODE_403 = 403; + private final String url; public FailedHttpResponseInterceptor(final String url){ @@ -32,6 +34,7 @@ public void process(final HttpResponse response, final EntityDetails entity, fin if (response.getCode() == ERROR_CODE_500 || response.getCode() == ERROR_CODE_400 || response.getCode() == ERROR_CODE_404 || + response.getCode() == ERROR_CODE_403 || response.getCode() == ERROR_CODE_501) { throw new IOException(String.format("url: %s , status code: %s", url,response.getCode())); } From e9ea0c21b074492e555d9212cabb1525bc099666 Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Mon, 21 Aug 2023 19:26:55 +0530 Subject: [PATCH 08/18] Fixed Failed Response code for issue #1744. Signed-off-by: mallikagogoi7 --- .../FailedHttpResponseInterceptor.java | 16 ++-------------- 1 file changed, 2 insertions(+), 14 deletions(-) diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java index 3bb73574ad..0ba42ab704 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java @@ -13,15 +13,7 @@ public class FailedHttpResponseInterceptor implements HttpResponseInterceptor { - public static final int ERROR_CODE_500 = 500; - - public static final int ERROR_CODE_400 = 400; - - public static final int ERROR_CODE_404 = 404; - - public static final int ERROR_CODE_501 = 501; - - public static final int ERROR_CODE_403 = 403; + public static final int STATUS_CODE_200 = 200; private final String url; @@ -31,11 +23,7 @@ public FailedHttpResponseInterceptor(final String url){ @Override public void process(final HttpResponse response, final EntityDetails entity, final HttpContext context) throws IOException { - if (response.getCode() == ERROR_CODE_500 || - response.getCode() == ERROR_CODE_400 || - response.getCode() == ERROR_CODE_404 || - response.getCode() == ERROR_CODE_403 || - response.getCode() == ERROR_CODE_501) { + if (response.getCode() != STATUS_CODE_200) { throw new IOException(String.format("url: %s , status code: %s", url,response.getCode())); } } From cbbc511a7d2e150cad7076b56d6366fd5bfd1941 Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Fri, 25 Aug 2023 20:01:28 +0530 Subject: [PATCH 09/18] Review Comments Resolved for issue #1744. Signed-off-by: mallikagogoi7 --- .../prometheus-sink/README.md | 5 +- .../prometheus/PrometheusSinkServiceIT.java | 2 +- .../FailedHttpResponseInterceptor.java | 4 +- .../prometheus/OAuthAccessTokenManager.java | 3 +- .../sink/prometheus/PrometheusSink.java | 18 ++++--- .../HttpClientSSLConnectionManager.java | 3 +- .../AwsAuthenticationOptions.java | 9 ++++ .../PrometheusSinkConfiguration.java | 7 --- .../sink/prometheus/dlq/DlqPushHandler.java | 52 +++++++++++++++---- .../service/PrometheusSinkService.java | 32 ++++++------ .../PrometheusSinkConfigurationTest.java | 5 -- .../prometheus/dlq/DlqPushHandlerTest.java | 37 ++++++++++++- .../service/PrometheusSinkServiceTest.java | 3 +- 13 files changed, 124 insertions(+), 56 deletions(-) diff --git a/data-prepper-plugins/prometheus-sink/README.md b/data-prepper-plugins/prometheus-sink/README.md index 22efe559d9..d38efc0e60 100644 --- a/data-prepper-plugins/prometheus-sink/README.md +++ b/data-prepper-plugins/prometheus-sink/README.md @@ -72,8 +72,6 @@ pipeline: - `scope` (Optional) : This scope limit an application's access to a user's account. -- `aws_sigv4`: A boolean flag to sign the HTTP request with AWS credentials. Default to `false`. - - `aws` (Optional) : AWS configurations. See [AWS Configuration](#aws_configuration) for details. SigV4 is enabled by default when this option is used. If this option is present, `aws_` options are not expected to be present. If any of `aws_` options are present along with this, error is thrown. - `custom_header` (Optional) : A Map for custom headers such as AWS Sagemaker etc @@ -113,7 +111,6 @@ pipeline: acm_certificate_arn: custom_header: header: ["value"] - aws_sigv4: false dlq_file : dlq: s3: @@ -122,6 +119,7 @@ pipeline: aws: region: "us-east-2" sts_role_arn: "arn:aws:iam::1234567890:role/data-prepper-s3source-execution-role" + sigv4: false max_retries: 5 ``` @@ -141,6 +139,7 @@ pipeline: * `sts_role_arn` (Optional) : The STS role to assume for requests to AWS. Defaults to null, which will use the [standard SDK behavior for credentials](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html). * `sts_header_overrides` (Optional): A map of header overrides to make when assuming the IAM role for the sink plugin. * `sts_external_id` (Optional): An optional external ID to use when assuming an IAM role. +* `sigv4`: A boolean flag to sign the HTTP request with AWS credentials. Default to `false`. ### End-to-End acknowledgements diff --git a/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java b/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java index 5ae78307ef..0d3f09eb85 100644 --- a/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java +++ b/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java @@ -84,7 +84,7 @@ public PrometheusSinkService createPrometheusSinkServiceUnderTest() throws NoSuc this.dlqPushHandler = new DlqPushHandler(prometheusSinkConfiguration.getDlqFile(), pluginFactory, "bucket", "arn", "region", - "keypath"); + "keypath", pluginMetrics); HttpClientBuilder httpClientBuilder = HttpClients.custom(); diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java index 0ba42ab704..37faac98f3 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java @@ -15,6 +15,8 @@ public class FailedHttpResponseInterceptor implements HttpResponseInterceptor { public static final int STATUS_CODE_200 = 200; + public static final int STATUS_CODE_201 = 201; + private final String url; public FailedHttpResponseInterceptor(final String url){ @@ -23,7 +25,7 @@ public FailedHttpResponseInterceptor(final String url){ @Override public void process(final HttpResponse response, final EntityDetails entity, final HttpContext context) throws IOException { - if (response.getCode() != STATUS_CODE_200) { + if (response.getCode() != STATUS_CODE_200 || response.getCode() != STATUS_CODE_201 ) { throw new IOException(String.format("url: %s , status code: %s", url,response.getCode())); } } diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java index 5c023876e0..88b1282cd7 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java @@ -33,10 +33,11 @@ public String getAccessToken(final BearerTokenOptions bearerTokenOptions) { bearerTokenOptions.setRefreshToken(accessTokenObj.getRefreshToken()); bearerTokenOptions.setAccessToken(accessTokenObj.getAccessToken()); bearerTokenOptions.setTokenExpired(accessTokenObj.getExpiresIn()); + return bearerTokenOptions.getAccessToken(); }catch (Exception e) { LOG.info("Exception : "+ e.getMessage() ); } - return bearerTokenOptions.getAccessToken(); + return ""; } diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java index 803a8f429e..f1e54d90a9 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java @@ -51,19 +51,23 @@ public PrometheusSink(final PluginSetting pluginSetting, final AwsCredentialsSupplier awsCredentialsSupplier) { super(pluginSetting); this.sinkInitialized = Boolean.FALSE; - this.dlqPushHandler = new DlqPushHandler(prometheusSinkConfiguration.getDlqFile(), pluginFactory, - String.valueOf(prometheusSinkConfiguration.getDlqPluginSetting().get(BUCKET)), - prometheusSinkConfiguration.getDlqStsRoleARN() - ,prometheusSinkConfiguration.getDlqStsRegion(), - String.valueOf(prometheusSinkConfiguration.getDlqPluginSetting().get(KEY_PATH))); - + if (prometheusSinkConfiguration.getDlqFile() != null) { + this.dlqPushHandler = new DlqPushHandler(prometheusSinkConfiguration.getDlqFile(), pluginFactory, + null, null, null, null, pluginMetrics); + }else { + this.dlqPushHandler = new DlqPushHandler(prometheusSinkConfiguration.getDlqFile(), pluginFactory, + String.valueOf(prometheusSinkConfiguration.getDlqPluginSetting().get(BUCKET)), + prometheusSinkConfiguration.getDlqStsRoleARN() + , prometheusSinkConfiguration.getDlqStsRegion(), + String.valueOf(prometheusSinkConfiguration.getDlqPluginSetting().get(KEY_PATH)), pluginMetrics); + } final HttpRequestRetryStrategy httpRequestRetryStrategy = new DefaultHttpRequestRetryStrategy(prometheusSinkConfiguration.getMaxUploadRetries(), TimeValue.of(prometheusSinkConfiguration.getHttpRetryInterval())); final HttpClientBuilder httpClientBuilder = HttpClients.custom() .setRetryStrategy(httpRequestRetryStrategy); - if(prometheusSinkConfiguration.isAwsSigv4() && prometheusSinkConfiguration.isValidAWSUrl()){ + if(prometheusSinkConfiguration.getAwsAuthenticationOptions().isAwsSigv4() && prometheusSinkConfiguration.isValidAWSUrl()){ PrometheusSinkAwsService.attachSigV4(prometheusSinkConfiguration, httpClientBuilder, awsCredentialsSupplier); } this.prometheusSinkService = new PrometheusSinkService( diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java index 3e409dc66e..4f38310955 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java @@ -60,8 +60,7 @@ public HttpClientConnectionManager createHttpClientConnectionManager(final Prome private SSLContext getCAStrategy(final InputStream certificate) { try { CertificateFactory factory = CertificateFactory.getInstance("X.509"); - Certificate trustedCa; - trustedCa = factory.generateCertificate(certificate); + Certificate trustedCa = factory.generateCertificate(certificate); KeyStore trustStore = KeyStore.getInstance("pkcs12"); trustStore.load(null, null); trustStore.setCertificateEntry("ca", trustedCa); diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptions.java index 7dd94a243c..13520185db 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptions.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptions.java @@ -27,6 +27,13 @@ public class AwsAuthenticationOptions { @Size(max = 5, message = "sts_header_overrides supports a maximum of 5 headers to override") private Map awsStsHeaderOverrides; + @JsonProperty("sigv4") + private boolean awsSigv4; + + public boolean isAwsSigv4() { + return awsSigv4; + } + public Region getAwsRegion() { return awsRegion != null ? Region.of(awsRegion) : null; } @@ -42,4 +49,6 @@ public String getAwsStsExternalId() { public Map getAwsStsHeaderOverrides() { return awsStsHeaderOverrides; } + + } \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java index 72c9e47500..2a608bbaf3 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java @@ -76,9 +76,6 @@ public class PrometheusSinkConfiguration { @JsonProperty("ssl_key_file") private String sslKeyFile; - @JsonProperty("aws_sigv4") - private boolean awsSigv4; - @JsonProperty("max_retries") private int maxUploadRetries = DEFAULT_UPLOAD_RETRIES; @@ -210,10 +207,6 @@ public String getSslKeyFile() { return sslKeyFile; } - public boolean isAwsSigv4() { - return awsSigv4; - } - public int getMaxUploadRetries() { return maxUploadRetries; } diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandler.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandler.java index 0d240eeea5..c2fef2bd46 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandler.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandler.java @@ -5,9 +5,10 @@ package org.opensearch.dataprepper.plugins.sink.prometheus.dlq; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.ObjectWriter; +import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.util.StringUtils; import org.opensearch.dataprepper.metrics.MetricNames; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.failures.DlqObject; import org.opensearch.dataprepper.model.plugin.PluginFactory; @@ -54,20 +55,43 @@ public class DlqPushHandler { private DlqProvider dlqProvider; - private ObjectWriter objectWriter; + static final String S3_DLQ_RECORDS_SUCCESS = "dlqS3RecordsSuccess"; + static final String S3_DLQ_RECORDS_FAILED = "dlqS3RecordsFailed"; + static final String S3_DLQ_REQUEST_SUCCESS = "dlqS3RequestSuccess"; + static final String S3_DLQ_REQUEST_FAILED = "dlqS3RequestFailed"; + + static final String FILE_DLQ_RECORDS_SUCCESS = "dlqFileRecordsSuccess"; + static final String FILE_DLQ_RECORDS_FAILED = "dlqFileRecordsFailed"; + static final String FILE_DLQ_REQUEST_SUCCESS = "dlqFileRequestSuccess"; + static final String FILE_DLQ_REQUEST_FAILED = "dlqFileRequestFailed"; + + private final Counter dlqS3RecordsSuccessCounter; + private final Counter dlqS3RecordsFailedCounter; + private final Counter dlqS3RequestSuccessCounter; + private final Counter dlqS3RequestFailedCounter; + + private final Counter dlqFileRecordsSuccessCounter; + private final Counter dlqFileRecordsFailedCounter; + private final Counter dlqFileRequestSuccessCounter; + private final Counter dlqFileRequestFailedCounter; public DlqPushHandler(final String dlqFile, final PluginFactory pluginFactory, final String bucket, final String stsRoleArn, final String awsRegion, - final String dlqPathPrefix) { - if(dlqFile != null) { - this.dlqFile = dlqFile; - this.objectWriter = new ObjectMapper().writer(); - }else{ - this.dlqProvider = getDlqProvider(pluginFactory,bucket,stsRoleArn,awsRegion,dlqPathPrefix); - } + final String dlqPathPrefix, + final PluginMetrics pluginMetrics) { + this.dlqFile = dlqFile; + this.dlqProvider = getDlqProvider(pluginFactory,bucket,stsRoleArn,awsRegion,dlqPathPrefix); + dlqS3RecordsSuccessCounter = pluginMetrics.counter(S3_DLQ_RECORDS_SUCCESS); + dlqS3RecordsFailedCounter = pluginMetrics.counter(S3_DLQ_RECORDS_FAILED); + dlqS3RequestSuccessCounter = pluginMetrics.counter(S3_DLQ_REQUEST_SUCCESS); + dlqS3RequestFailedCounter = pluginMetrics.counter(S3_DLQ_REQUEST_FAILED); + dlqFileRecordsSuccessCounter = pluginMetrics.counter(FILE_DLQ_RECORDS_SUCCESS); + dlqFileRecordsFailedCounter = pluginMetrics.counter(FILE_DLQ_RECORDS_FAILED); + dlqFileRequestSuccessCounter = pluginMetrics.counter(FILE_DLQ_REQUEST_SUCCESS); + dlqFileRequestFailedCounter = pluginMetrics.counter(FILE_DLQ_REQUEST_FAILED); } public void perform(final PluginSetting pluginSetting, @@ -81,9 +105,13 @@ public void perform(final PluginSetting pluginSetting, private void writeToFile(Object failedData) { try(BufferedWriter dlqFileWriter = Files.newBufferedWriter(Paths.get(dlqFile), StandardOpenOption.CREATE, StandardOpenOption.APPEND)) { - dlqFileWriter.write(objectWriter.writeValueAsString(failedData)+"\n"); + dlqFileWriter.write(new ObjectMapper().writer().writeValueAsString(failedData)+"\n"); + dlqFileRequestSuccessCounter.increment(); + dlqFileRecordsSuccessCounter.increment(); } catch (IOException e) { LOG.error("Exception while writing failed data to DLQ file Exception: ",e); + dlqFileRequestFailedCounter.increment(); + dlqFileRecordsFailedCounter.increment(); } } @@ -99,8 +127,12 @@ private void pushToS3(PluginSetting pluginSetting, Object failedData) { .build(); dlqWriter.write(Arrays.asList(dlqObject), pluginSetting.getPipelineName(), pluginId); + dlqS3RequestSuccessCounter.increment(); + dlqS3RecordsSuccessCounter.increment(); } catch (final IOException e) { LOG.error("Exception while writing failed data to DLQ, Exception : ", e); + dlqS3RequestFailedCounter.increment(); + dlqS3RecordsFailedCounter.increment(); } } diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java index cfdcbd1f2f..95d6d89303 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java @@ -170,21 +170,23 @@ public void output(final Collection> records) { } else { LOG.error("No valid Event type found"); } - bytes = message.toByteArray(); - } - if (event.getEventHandle() != null) { - this.bufferedEventHandles.add(event.getEventHandle()); - } - HttpEndPointResponse failedHttpEndPointResponses = pushToEndPoint(bytes); - - if (failedHttpEndPointResponses != null) { - logFailedData(failedHttpEndPointResponses); - releaseEventHandles(Boolean.FALSE); - } else { - LOG.info("data pushed to the end point successfully"); - releaseEventHandles(Boolean.TRUE); - } - }); + if( message.toByteArray() != null) + bytes = message.toByteArray(); + } + if (event.getEventHandle() != null) { + this.bufferedEventHandles.add(event.getEventHandle()); + } + if(bytes != null){ + HttpEndPointResponse failedHttpEndPointResponses = pushToEndPoint(bytes); + + if (failedHttpEndPointResponses != null) { + logFailedData(failedHttpEndPointResponses); + releaseEventHandles(Boolean.FALSE); + } else { + LOG.info("data pushed to the end point successfully"); + releaseEventHandles(Boolean.TRUE); + } + }}); }finally { reentrantLock.unlock(); diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java index 0e15c7d10a..4839354825 100644 --- a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java @@ -57,7 +57,6 @@ public class PrometheusSinkConfigurationTest { " sts_external_id: \"test-external-id\"\n" + " sts_header_overrides: {\"test\": test }\n" + " max_retries: 5\n" + - " aws_sigv4: true\n" + " custom_header:\n" + " X-Amzn-SageMaker-Custom-Attributes: [\"test-attribute\"]\n" + " X-Amzn-SageMaker-Target-Model: [\"test-target-model\"]\n" + @@ -99,10 +98,6 @@ void default_ssl_test() { assertThat(new PrometheusSinkConfiguration().isSsl(), equalTo(false)); } - @Test - void default_awsSigv4_test() { - assertThat(new PrometheusSinkConfiguration().isAwsSigv4(), equalTo(false)); - } @Test void get_ssl_certificate_file_test() { diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandlerTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandlerTest.java index 3171164b3f..e09b26ed83 100644 --- a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandlerTest.java +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/dlq/DlqPushHandlerTest.java @@ -4,9 +4,11 @@ */ package org.opensearch.dataprepper.plugins.sink.prometheus.dlq; +import io.micrometer.core.instrument.Counter; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Mockito; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.plugin.PluginFactory; @@ -54,6 +56,18 @@ public class DlqPushHandlerTest { private DlqWriter dlqWriter; + private PluginMetrics pluginMetrics; + + private Counter dlqS3RecordsSuccessCounter; + private Counter dlqS3RecordsFailedCounter; + private Counter dlqS3RequestSuccessCounter; + private Counter dlqS3RequestFailedCounter; + + private Counter dlqFileRecordsSuccessCounter; + private Counter dlqFileRecordsFailedCounter; + private Counter dlqFileRequestSuccessCounter; + private Counter dlqFileRequestFailedCounter; + @BeforeEach public void setUp() throws Exception{ this.pluginFactory = mock(PluginFactory.class); @@ -61,6 +75,25 @@ public void setUp() throws Exception{ this.awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); this.dlqProvider = mock(DlqProvider.class); this.dlqWriter = mock(DlqWriter.class); + this.pluginMetrics = mock(PluginMetrics.class); + this.dlqS3RecordsSuccessCounter = mock(Counter.class); + this.dlqS3RecordsFailedCounter = mock(Counter.class); + this.dlqS3RequestSuccessCounter = mock(Counter.class); + this.dlqS3RequestFailedCounter = mock(Counter.class); + this.dlqFileRecordsSuccessCounter = mock(Counter.class); + this.dlqFileRecordsFailedCounter = mock(Counter.class); + this.dlqFileRequestSuccessCounter = mock(Counter.class); + this.dlqFileRequestFailedCounter = mock(Counter.class); + when(pluginMetrics.counter(DlqPushHandler.S3_DLQ_RECORDS_SUCCESS)).thenReturn(dlqS3RecordsSuccessCounter); + when(pluginMetrics.counter(DlqPushHandler.S3_DLQ_RECORDS_FAILED)).thenReturn(dlqS3RecordsFailedCounter); + when(pluginMetrics.counter(DlqPushHandler.S3_DLQ_REQUEST_SUCCESS)).thenReturn(dlqS3RequestSuccessCounter); + when(pluginMetrics.counter(DlqPushHandler.S3_DLQ_REQUEST_FAILED)).thenReturn(dlqS3RequestFailedCounter); + when(pluginMetrics.counter(DlqPushHandler.FILE_DLQ_RECORDS_SUCCESS)).thenReturn(dlqFileRecordsSuccessCounter); + when(pluginMetrics.counter(DlqPushHandler.FILE_DLQ_RECORDS_FAILED)).thenReturn(dlqFileRecordsFailedCounter); + when(pluginMetrics.counter(DlqPushHandler.FILE_DLQ_REQUEST_SUCCESS)).thenReturn(dlqFileRequestSuccessCounter); + when(pluginMetrics.counter(DlqPushHandler.FILE_DLQ_REQUEST_FAILED)).thenReturn(dlqFileRequestFailedCounter); + + } @Test @@ -74,7 +107,7 @@ public void perform_for_dlq_s3_success() throws IOException { when(dlqProvider.getDlqWriter(Mockito.anyString())).thenReturn(Optional.of(dlqWriter)); doNothing().when(dlqWriter).write(anyList(), anyString(), anyString()); FailedDlqData failedDlqData = FailedDlqData.builder().build(); - dlqPushHandler = new DlqPushHandler(null,pluginFactory, BUCKET_VALUE, ROLE, REGION,KEY_PATH_PREFIX_VALUE); + dlqPushHandler = new DlqPushHandler(null,pluginFactory, BUCKET_VALUE, ROLE, REGION,KEY_PATH_PREFIX_VALUE, pluginMetrics); PluginSetting pluginSetting = new PluginSetting(S3_PLUGIN_NAME, props); pluginSetting.setPipelineName(PIPELINE_NAME); @@ -87,7 +120,7 @@ public void perform_for_dlq_s3_success() throws IOException { public void perform_for_dlq_local_file_success(){ FailedDlqData failedDlqData = FailedDlqData.builder().build(); - dlqPushHandler = new DlqPushHandler(DLQ_FILE,pluginFactory,null, ROLE, REGION,null); + dlqPushHandler = new DlqPushHandler(DLQ_FILE,pluginFactory,null, ROLE, REGION,null, pluginMetrics); PluginSetting pluginSetting = new PluginSetting(S3_PLUGIN_NAME, null); pluginSetting.setPipelineName(PIPELINE_NAME); diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java index 2e576215eb..06300140eb 100644 --- a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java @@ -80,8 +80,7 @@ public class PrometheusSinkServiceTest { " max_retries: 5\n" + " encoding: snappy\n" + " content-type: \"application/octet-stream\"\n" + - " remote-write-version: 0.1.0\n" + - " aws_sigv4: false\n"; + " remote-write-version: 0.1.0\n"; private PrometheusSinkConfiguration prometheusSinkConfiguration; From 7693d9d59c8973c1205e1732e63a7058c510a78e Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Mon, 28 Aug 2023 17:11:14 +0530 Subject: [PATCH 10/18] AwsSigv4 Code added for #1744. Signed-off-by: mallikagogoi7 --- .../aws-plugin-api/build.gradle | 1 + .../AwsRequestSigningApacheInterceptor.java | 31 +++++++------------ .../service/PrometheusSinkAwsService.java | 2 +- .../service/PrometheusSinkService.java | 4 +++ 4 files changed, 17 insertions(+), 21 deletions(-) diff --git a/data-prepper-plugins/aws-plugin-api/build.gradle b/data-prepper-plugins/aws-plugin-api/build.gradle index 1ca60763bf..42d38646ff 100644 --- a/data-prepper-plugins/aws-plugin-api/build.gradle +++ b/data-prepper-plugins/aws-plugin-api/build.gradle @@ -2,6 +2,7 @@ dependencies { implementation 'software.amazon.awssdk:auth' implementation 'org.apache.httpcomponents.client5:httpclient5:5.2' + implementation 'com.amazonaws:aws-java-sdk-sts:1.12.395' } test { diff --git a/data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsRequestSigningApacheInterceptor.java b/data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsRequestSigningApacheInterceptor.java index 27e75ab2f8..2b8222bbb6 100644 --- a/data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsRequestSigningApacheInterceptor.java +++ b/data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsRequestSigningApacheInterceptor.java @@ -12,19 +12,20 @@ */ package org.opensearch.dataprepper.aws.api; +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; import org.apache.hc.core5.http.HttpRequest; import org.apache.hc.core5.http.ClassicHttpRequest; import org.apache.hc.core5.http.EntityDetails; -import org.apache.hc.core5.http.HttpEntity; import org.apache.hc.core5.http.Header; import org.apache.hc.core5.http.NameValuePair; -import org.apache.hc.core5.http.ContentType; import org.apache.hc.core5.http.HttpHost; import org.apache.hc.core5.http.HttpRequestInterceptor; -import org.apache.hc.core5.http.io.entity.BasicHttpEntity; import org.apache.hc.core5.http.message.BasicHeader; import org.apache.hc.core5.http.protocol.HttpContext; import org.apache.hc.core5.net.URIBuilder; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.auth.signer.AwsSignerExecutionAttribute; import software.amazon.awssdk.core.interceptor.ExecutionAttributes; @@ -45,6 +46,8 @@ import java.util.Objects; import java.util.TreeMap; +import static org.apache.http.protocol.HttpCoreContext.HTTP_TARGET_HOST; + /** * An {@link HttpRequestInterceptor} that signs requests using any AWS {@link Signer} * and {@link AwsCredentialsProvider}. @@ -64,11 +67,6 @@ public final class AwsRequestSigningApacheInterceptor implements HttpRequestInte */ private static final String HOST = "host"; - /** - * Attribute name of a HttpHost object that represents the connection target. - */ - private static final String HTTP_TARGET_HOST = "http.target_host"; - /** * The service that we're connecting to. */ @@ -149,8 +147,11 @@ public void process(final HttpRequest request, final EntityDetails entity, final requestBuilder.rawQueryParameters(nvpToMapParams(uriBuilder.getQueryParams())); requestBuilder.headers(headerArrayToMap(request.getHeaders())); + AWSCredentials credentials = new DefaultAWSCredentialsProviderChain().getCredentials(); + ExecutionAttributes attributes = new ExecutionAttributes(); - attributes.putAttribute(AwsSignerExecutionAttribute.AWS_CREDENTIALS, awsCredentialsProvider.resolveCredentials()); + AwsCredentials awsCredentials = AwsBasicCredentials.create(credentials.getAWSAccessKeyId(), credentials.getAWSSecretKey()); + attributes.putAttribute(AwsSignerExecutionAttribute.AWS_CREDENTIALS, awsCredentials); attributes.putAttribute(AwsSignerExecutionAttribute.SERVICE_SIGNING_NAME, service); attributes.putAttribute(AwsSignerExecutionAttribute.SIGNING_REGION, region); @@ -159,16 +160,6 @@ public void process(final HttpRequest request, final EntityDetails entity, final // Now copy everything back request.setHeaders(mapToHeaderArray(signedRequest.headers())); - if (request instanceof ClassicHttpRequest) { - ClassicHttpRequest classicHttpRequest = - (ClassicHttpRequest) request; - if (classicHttpRequest.getEntity() != null) { - HttpEntity basicHttpEntity = new BasicHttpEntity(signedRequest.contentStreamProvider() - .orElseThrow(() -> new IllegalStateException("There must be content")) - .newStream(), ContentType.APPLICATION_JSON); - classicHttpRequest.setEntity(basicHttpEntity); - } - } } private URI buildUri(final HttpContext context, URIBuilder uriBuilder) throws IOException { @@ -242,4 +233,4 @@ private static Header[] mapToHeaderArray(final Map> mapHead } return headers; } -} +} \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java index 9fd067d1af..21137a4364 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java @@ -18,7 +18,7 @@ public class PrometheusSinkAwsService { private static final Logger LOG = LoggerFactory.getLogger(PrometheusSinkAwsService.class); public static final String AWS_SIGV4 = "aws_sigv4"; - private static final String AOS_SERVICE_NAME = "http-endpoint"; + private static final String AOS_SERVICE_NAME = "execute-api"; // TODO : aps public static void attachSigV4(final PrometheusSinkConfiguration prometheusSinkConfiguration, final HttpClientBuilder httpClientBuilder, final AwsCredentialsSupplier awsCredentialsSupplier) { LOG.info("{} is set, will sign requests using AWSRequestSigningApacheInterceptor", AWS_SIGV4); diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java index 95d6d89303..5abc42f89b 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java @@ -373,6 +373,10 @@ private Map buildAuthHttpSinkObjectsByConfig(final Prome if(Objects.nonNull(prometheusSinkConfiguration.getCustomHeaderOptions())) addCustomHeaders(classicRequestBuilder,prometheusSinkConfiguration.getCustomHeaderOptions()); + if(prometheusSinkConfiguration.getAwsAuthenticationOptions().isAwsSigv4() && prometheusSinkConfiguration.isValidAWSUrl()){ + classicRequestBuilder.addHeader("x-amz-content-sha256","required"); + } + if(Objects.nonNull(proxyUrlString)) { httpClientBuilder.setProxy(PrometheusSinkUtil.getHttpHostByURL(PrometheusSinkUtil.getURLByUrlString(proxyUrlString))); LOG.info("sending data via proxy {}",proxyUrlString); From 8e1d7f5a613efb597d3c8859200cf87b4b5b1d14 Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Mon, 28 Aug 2023 17:38:18 +0530 Subject: [PATCH 11/18] AwsSigv4 Code Bug resolved for #1744. Signed-off-by: mallikagogoi7 --- .../sink/prometheus/service/PrometheusSinkAwsService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java index 21137a4364..97eca3bff2 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java @@ -18,7 +18,7 @@ public class PrometheusSinkAwsService { private static final Logger LOG = LoggerFactory.getLogger(PrometheusSinkAwsService.class); public static final String AWS_SIGV4 = "aws_sigv4"; - private static final String AOS_SERVICE_NAME = "execute-api"; // TODO : aps + private static final String AOS_SERVICE_NAME = "aps"; public static void attachSigV4(final PrometheusSinkConfiguration prometheusSinkConfiguration, final HttpClientBuilder httpClientBuilder, final AwsCredentialsSupplier awsCredentialsSupplier) { LOG.info("{} is set, will sign requests using AWSRequestSigningApacheInterceptor", AWS_SIGV4); From 478a495a6ad249f300a3cd56e222c3b609474907 Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Mon, 28 Aug 2023 22:15:02 +0530 Subject: [PATCH 12/18] AwsSigv4 Code Bug Fix for #1744. Signed-off-by: mallikagogoi7 --- .../aws/api/AwsRequestSigningApacheInterceptor.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsRequestSigningApacheInterceptor.java b/data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsRequestSigningApacheInterceptor.java index 2b8222bbb6..32ba64a362 100644 --- a/data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsRequestSigningApacheInterceptor.java +++ b/data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsRequestSigningApacheInterceptor.java @@ -24,8 +24,6 @@ import org.apache.hc.core5.http.message.BasicHeader; import org.apache.hc.core5.http.protocol.HttpContext; import org.apache.hc.core5.net.URIBuilder; -import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; -import software.amazon.awssdk.auth.credentials.AwsCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.auth.signer.AwsSignerExecutionAttribute; import software.amazon.awssdk.core.interceptor.ExecutionAttributes; @@ -148,10 +146,8 @@ public void process(final HttpRequest request, final EntityDetails entity, final requestBuilder.headers(headerArrayToMap(request.getHeaders())); AWSCredentials credentials = new DefaultAWSCredentialsProviderChain().getCredentials(); - ExecutionAttributes attributes = new ExecutionAttributes(); - AwsCredentials awsCredentials = AwsBasicCredentials.create(credentials.getAWSAccessKeyId(), credentials.getAWSSecretKey()); - attributes.putAttribute(AwsSignerExecutionAttribute.AWS_CREDENTIALS, awsCredentials); + attributes.putAttribute(AwsSignerExecutionAttribute.AWS_CREDENTIALS, awsCredentialsProvider.resolveCredentials()); attributes.putAttribute(AwsSignerExecutionAttribute.SERVICE_SIGNING_NAME, service); attributes.putAttribute(AwsSignerExecutionAttribute.SIGNING_REGION, region); From a671ad7a11109402c6c078ad461e6fbf20a17438 Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Mon, 28 Aug 2023 22:44:10 +0530 Subject: [PATCH 13/18] README file updated for #1744. Signed-off-by: mallikagogoi7 --- data-prepper-plugins/prometheus-sink/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/prometheus-sink/README.md b/data-prepper-plugins/prometheus-sink/README.md index d38efc0e60..b2625bec20 100644 --- a/data-prepper-plugins/prometheus-sink/README.md +++ b/data-prepper-plugins/prometheus-sink/README.md @@ -139,7 +139,7 @@ pipeline: * `sts_role_arn` (Optional) : The STS role to assume for requests to AWS. Defaults to null, which will use the [standard SDK behavior for credentials](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html). * `sts_header_overrides` (Optional): A map of header overrides to make when assuming the IAM role for the sink plugin. * `sts_external_id` (Optional): An optional external ID to use when assuming an IAM role. -* `sigv4`: A boolean flag to sign the HTTP request with AWS credentials. Default to `false`. +* `sigv4`: A boolean flag to sign the HTTP request with AWS credentials. Default to `false`. For aws_sigv4, we don't need any auth_type or ssl ### End-to-End acknowledgements From e4401bd4ff76010b49a3e2f9518cd0a6f20a86ef Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Thu, 31 Aug 2023 16:23:08 +0530 Subject: [PATCH 14/18] Bug fixed for #1744. Signed-off-by: mallikagogoi7 --- .../sink/prometheus/FailedHttpResponseInterceptor.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java index 37faac98f3..0ba42ab704 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java @@ -15,8 +15,6 @@ public class FailedHttpResponseInterceptor implements HttpResponseInterceptor { public static final int STATUS_CODE_200 = 200; - public static final int STATUS_CODE_201 = 201; - private final String url; public FailedHttpResponseInterceptor(final String url){ @@ -25,7 +23,7 @@ public FailedHttpResponseInterceptor(final String url){ @Override public void process(final HttpResponse response, final EntityDetails entity, final HttpContext context) throws IOException { - if (response.getCode() != STATUS_CODE_200 || response.getCode() != STATUS_CODE_201 ) { + if (response.getCode() != STATUS_CODE_200) { throw new IOException(String.format("url: %s , status code: %s", url,response.getCode())); } } From 1a8cb7ebb3a389ef902b2043c883ada7a9fd98fe Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Fri, 1 Sep 2023 19:00:33 +0530 Subject: [PATCH 15/18] Fix is date for #1744. Signed-off-by: mallikagogoi7 --- .../service/PrometheusSinkService.java | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java index 5abc42f89b..4767817e95 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java @@ -48,12 +48,11 @@ import java.io.IOException; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.time.ZonedDateTime; -import java.time.format.DateTimeFormatter; +import java.text.ParseException; +import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Collection; +import java.util.Date; import java.util.LinkedList; import java.util.Map; import java.util.List; @@ -172,10 +171,10 @@ public void output(final Collection> records) { } if( message.toByteArray() != null) bytes = message.toByteArray(); - } - if (event.getEventHandle() != null) { - this.bufferedEventHandles.add(event.getEventHandle()); - } + } + if (event.getEventHandle() != null) { + this.bufferedEventHandles.add(event.getEventHandle()); + } if(bytes != null){ HttpEndPointResponse failedHttpEndPointResponses = pushToEndPoint(bytes); @@ -255,11 +254,15 @@ private static String sanitizeName(final String name) { } private static long getTimeStampVal(final String time) { - final LocalDateTime localDateTimeParse = LocalDateTime.parse(time, - DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSSSSSSS'Z'")); - final LocalDateTime localDateTime = LocalDateTime.parse(localDateTimeParse.toString()); - final ZonedDateTime zdt = ZonedDateTime.of(localDateTime, ZoneId.systemDefault()); - return zdt.toInstant().toEpochMilli(); + long timeStampVal = 0; + SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + try { + Date date = sdf.parse(time); + timeStampVal = date.getTime(); + } catch (ParseException e) { + e.printStackTrace(); + } + return timeStampVal; } /** From c59cd3dca815e9c9c0c5dc901b1a7e97a4112bd2 Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Fri, 8 Sep 2023 15:43:38 +0530 Subject: [PATCH 16/18] Review comment resolved for #1744. Signed-off-by: mallikagogoi7 --- data-prepper-plugins/prometheus-sink/README.md | 12 ++++++------ .../prometheus/configuration/AuthTypeOptions.java | 4 ++-- .../configuration/AuthenticationOptions.java | 4 ++-- .../configuration/PrometheusSinkConfiguration.java | 4 ++-- .../PrometheusSinkConfigurationTest.java | 8 ++++---- .../service/PrometheusSinkServiceTest.java | 10 +++++----- 6 files changed, 21 insertions(+), 21 deletions(-) diff --git a/data-prepper-plugins/prometheus-sink/README.md b/data-prepper-plugins/prometheus-sink/README.md index b2625bec20..64124e7385 100644 --- a/data-prepper-plugins/prometheus-sink/README.md +++ b/data-prepper-plugins/prometheus-sink/README.md @@ -22,7 +22,7 @@ pipeline: sink: - prometheus: authentication: - http-basic: + http_basic: username: my-user password: my_s3cr3t ``` @@ -34,7 +34,7 @@ pipeline: sink: - prometheus: authentication: - bearer-token: + bearer_token: client_id: 0oaafr4j79grYGC5d7 client_secret: fFel-3FutCXAOndezEsOVlght6D6DR4OIt7G5D1_oJ6YtgU17JdyXmGf0M token_url: https://localhost/oauth2/default/v1/token @@ -48,9 +48,9 @@ pipeline: - `encoding` Default is snappy -- `content-type` Default is application/x-protobuf +- `content_type` Default is application/x-protobuf -- `remote-write-version` : Prometheus Remote.Writer version Version, Default is 0.1.0 +- `remote_write_version` : Prometheus Remote.Writer version Version, Default is 0.1.0 - `proxy`(optional): A String of the address of a forward HTTP proxy. The format is like ":\". Examples: "example.com:8100", "http://example.com:8100", "112.112.112.112:8100". Note: port number cannot be omitted. @@ -95,10 +95,10 @@ pipeline: http_method: "POST" auth_type: "unauthenticated" authentication: - http-basic: + http_basic: username: "username" password: "password" - bearer-token: + bearer_token: client_id: 0oaafr4j79segd7 client_secret: fFel-3FutCXAOndezEsOVlghoJ6w0wNoaYtgU17JdyXmGf0M token_url: token url diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthTypeOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthTypeOptions.java index e8d54fa701..cd91fc6db0 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthTypeOptions.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthTypeOptions.java @@ -11,8 +11,8 @@ import java.util.stream.Collectors; public enum AuthTypeOptions { - HTTP_BASIC("http-basic"), - BEARER_TOKEN("bearer-token"), + HTTP_BASIC("http_basic"), + BEARER_TOKEN("bearer_token"), UNAUTHENTICATED("unauthenticated"); diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java index 908abe9d62..e5bfa60534 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java @@ -8,10 +8,10 @@ public class AuthenticationOptions { - @JsonProperty("http-basic") + @JsonProperty("http_basic") private BasicAuthCredentials httpBasic; - @JsonProperty("bearer-token") + @JsonProperty("bearer_token") private BearerTokenOptions bearerTokenOptions; public BasicAuthCredentials getHttpBasic() { diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java index 2a608bbaf3..8d9eb40fc3 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java @@ -113,10 +113,10 @@ public class PrometheusSinkConfiguration { @JsonProperty("encoding") private String encoding = DEFAULT_ENCODING; - @JsonProperty("content-type") + @JsonProperty("content_type") private String contentType = DEFAULT_CONTENT_TYPE; - @JsonProperty("remote-write-version") + @JsonProperty("remote_write_version") private String remoteWriteVersion = DEFAULT_REMOTE_WRITE_VERSION; private boolean sslCertAndKeyFileInS3; diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java index 4839354825..e7e77f3d90 100644 --- a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java @@ -32,12 +32,12 @@ public class PrometheusSinkConfigurationTest { " url: \"http://localhost:8080/test\"\n" + " proxy: test-proxy\n" + " http_method: \"POST\"\n" + - " auth_type: \"http-basic\"\n" + + " auth_type: \"http_basic\"\n" + " authentication:\n" + - " http-basic:\n" + + " http_basic:\n" + " username: \"username\"\n" + " password: \"vip\"\n" + - " bearer-token:\n" + + " bearer_token:\n" + " client_id: 0oaafr4j79segrYGC5d7\n" + " client_secret: fFel-3FutCXAOndezEsOVlght6D6DR4OIt7G5D1_oJ6w0wNoaYtgU17JdyXmGf0M\n" + " token_url: https://localhost/oauth2/default/v1/token\n" + @@ -180,7 +180,7 @@ public void validate_and_initialize_cert_and_key_file_in_s3_test() throws JsonPr " http_method: \"POST\"\n" + " auth_type: \"http-basic\"\n" + " authentication:\n" + - " http-basic:\n" + + " http_basic:\n" + " username: \"username\"\n" + " password: \"vip\"\n" + " ssl: true\n" + diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java index 06300140eb..ed7ede1509 100644 --- a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java @@ -58,10 +58,10 @@ public class PrometheusSinkServiceTest { " http_method: \"POST\"\n" + " auth_type: \"unauthenticated\"\n" + " authentication:\n" + - " http-basic:\n" + + " http_basic:\n" + " username: \"username\"\n" + " password: \"vip\"\n" + - " bearer-token:\n" + + " bearer_token:\n" + " client_id: 0oaafr4j79segrYGC5d7\n" + " client_secret: fFel-3FutCXAOndezEsOVlght6D6DR4OIt7G5D1_oJ6w0wNoaYtgU17JdyXmGf0M\n" + " token_url: https://localhost/oauth2/default/v1/token\n" + @@ -79,8 +79,8 @@ public class PrometheusSinkServiceTest { " sts_header_overrides: {\"test\": test }\n" + " max_retries: 5\n" + " encoding: snappy\n" + - " content-type: \"application/octet-stream\"\n" + - " remote-write-version: 0.1.0\n"; + " content_type: \"application/octet-stream\"\n" + + " remote_write_version: 0.1.0\n"; private PrometheusSinkConfiguration prometheusSinkConfiguration; @@ -234,7 +234,7 @@ void prometheus_sink_service_test_with_internal_server_error() throws NoSuchFiel @Test void prometheus_sink_service_test_with_single_record_with_basic_authentication() throws NoSuchFieldException, IllegalAccessException, JsonProcessingException { - final String basicAuthYaml = " http-basic:\n" + + final String basicAuthYaml = " http_basic:\n" + " username: \"username\"\n" + " password: \"vip\"\n" ; ReflectivelySetField.setField(PrometheusSinkConfiguration.class,prometheusSinkConfiguration,"authentication", objectMapper.readValue(basicAuthYaml, AuthenticationOptions.class)); From b7f95bed24cb0309bbb6872a4b70eabc38a20e14 Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Tue, 12 Sep 2023 14:47:35 +0530 Subject: [PATCH 17/18] Added insecure changes for #1744. Signed-off-by: mallikagogoi7 --- .../prometheus-sink/README.md | 14 ++++++-- .../prometheus/PrometheusSinkServiceIT.java | 2 +- .../sink/prometheus/PrometheusSink.java | 4 +++ .../HttpClientSSLConnectionManager.java | 4 +-- .../PrometheusSinkConfiguration.java | 34 +++++++++++++++---- .../service/PrometheusSinkService.java | 9 +++-- .../PrometheusSinkConfigurationTest.java | 6 ++-- .../service/PrometheusSinkServiceTest.java | 2 +- 8 files changed, 56 insertions(+), 19 deletions(-) diff --git a/data-prepper-plugins/prometheus-sink/README.md b/data-prepper-plugins/prometheus-sink/README.md index 64124e7385..083a3d2433 100644 --- a/data-prepper-plugins/prometheus-sink/README.md +++ b/data-prepper-plugins/prometheus-sink/README.md @@ -83,6 +83,7 @@ pipeline: - `max_retries`(optional): A number indicating the maximum number of times Prometheus Sink should try to push the data to the Http arbitrary endpoint before considering it as failure. Defaults to `Integer.MAX_VALUE`. +- `request_timout`(optional): A duration that represents the request timeout. Example: 1000ms, 5s etc ### Prometheus Sink full pipeline ``` sink: @@ -104,7 +105,8 @@ pipeline: token_url: token url grant_type: client_credentials scope: - ssl: false + insecure: false + insecure_skip_verify: false ssl_certificate_file: "/full/path/to/certfile.crt" buffer_type: "in_memory" use_acm_cert_for_ssl: false @@ -121,11 +123,19 @@ pipeline: sts_role_arn: "arn:aws:iam::1234567890:role/data-prepper-s3source-execution-role" sigv4: false max_retries: 5 + request_timout: 20s ``` ### SSL -* ssl(Optional) => A `boolean` that enables mTLS/SSL. Default is ```false```. +* insecure_skip_verify(Optional) => A `boolean` that enables mTLS/SSL. Default is ```false```. + * If set to false then the user has two options: + * Use default trust. This can allow for reaching many endpoints. The user does not need to provide any .crt/.key files. + * Allow the user to specify a .crt file for a certificate (no .key is required because this is the client). By the user providing the .crt file, the user is stating he trusts that certificate. We will still verify the signature match. + * If set to true, then skip any verification of the certificate. The user does not need to provide a .crt or .key file. +* insecure (Optional) => A `boolean` that allows http/https endpoints. Default is ```false```. + * If set to false, then only https:// URLs are permitted. Throw an InvalidPluginConfigurationException if the URL is configured with an http:// scheme in the URL. + * If set to true, then the user can provide both http:// https:// as the scheme. * ssl_certificate_file(Optional) => A `String` that represents the SSL certificate chain file path or AWS S3 path. S3 path example `s3:///`. Required if `ssl` is set to `true` and `use_acm_certificate_for_ssl` is set to `false`. * ssl_key_file(Optional) => A `String` that represents the SSL key file path or AWS S3 path. S3 path example `s3:///`. Only decrypted key file is supported. Required if `ssl` is set to `true` and `use_acm_certificate_for_ssl` is set to `false`. * use_acm_certificate_for_ssl(Optional) : A `boolean` that enables mTLS/SSL using certificate and private key from AWS Certificate Manager (ACM). Default is `false`. diff --git a/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java b/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java index 0d3f09eb85..bfd145c2d7 100644 --- a/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java +++ b/data-prepper-plugins/prometheus-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSinkServiceIT.java @@ -49,7 +49,7 @@ public class PrometheusSinkServiceIT { " url: {0}\n" + " http_method: POST\n" + " auth_type: {1}\n" + - " ssl: false\n"; + " insecure_skip_verify: true\n"; private PrometheusSinkConfiguration prometheusSinkConfiguration; diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java index f1e54d90a9..84ff2fa9f2 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java @@ -64,6 +64,10 @@ public PrometheusSink(final PluginSetting pluginSetting, final HttpRequestRetryStrategy httpRequestRetryStrategy = new DefaultHttpRequestRetryStrategy(prometheusSinkConfiguration.getMaxUploadRetries(), TimeValue.of(prometheusSinkConfiguration.getHttpRetryInterval())); + if((!prometheusSinkConfiguration.isInsecure()) && (prometheusSinkConfiguration.isHttpUrl())){ + throw new InvalidPluginConfigurationException ("Cannot configure http url with insecure as false"); + } + final HttpClientBuilder httpClientBuilder = HttpClients.custom() .setRetryStrategy(httpRequestRetryStrategy); diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java index 4f38310955..1c2af56de7 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java @@ -41,10 +41,8 @@ public class HttpClientSSLConnectionManager { */ public HttpClientConnectionManager createHttpClientConnectionManager(final PrometheusSinkConfiguration sinkConfiguration, final CertificateProviderFactory providerFactory){ - final CertificateProvider certificateProvider = providerFactory.getCertificateProvider(); - final org.opensearch.dataprepper.plugins.certificate.model.Certificate certificate = certificateProvider.getCertificate(); final SSLContext sslContext = sinkConfiguration.getSslCertificateFile() != null ? - getCAStrategy(new ByteArrayInputStream(certificate.getCertificate().getBytes(StandardCharsets.UTF_8))) : getTrustAllStrategy(); + getCAStrategy(new ByteArrayInputStream(providerFactory.getCertificateProvider().getCertificate().getCertificate().getBytes(StandardCharsets.UTF_8))) : getTrustAllStrategy(); SSLConnectionSocketFactory sslSocketFactory = SSLConnectionSocketFactoryBuilder.create() .setSslContext(sslContext) .build(); diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java index 8d9eb40fc3..df5e3301fa 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java @@ -21,7 +21,7 @@ public class PrometheusSinkConfiguration { private static final int DEFAULT_UPLOAD_RETRIES = 5; - static final boolean DEFAULT_SSL = false; + static final boolean DEFAULT_INSECURE = false; private static final String S3_PREFIX = "s3://"; @@ -41,6 +41,8 @@ public class PrometheusSinkConfiguration { private static final String HTTPS = "https"; + private static final String HTTP = "http"; + private static final String AWS_HOST_AMAZONAWS_COM = "amazonaws.com"; private static final String AWS_HOST_API_AWS = "api.aws"; @@ -104,8 +106,11 @@ public class PrometheusSinkConfiguration { @JsonProperty("acm_cert_issue_time_out_millis") private long acmCertIssueTimeOutMillis = DEFAULT_ACM_CERT_ISSUE_TIME_OUT_MILLIS; - @JsonProperty("ssl") - private boolean ssl = DEFAULT_SSL; + @JsonProperty("insecure") + private boolean insecure = DEFAULT_INSECURE; + + @JsonProperty("insecure_skip_verify") + private boolean insecureSkipVerify = DEFAULT_INSECURE; @JsonProperty("http_retry_interval") private Duration httpRetryInterval = DEFAULT_HTTP_RETRY_INTERVAL; @@ -119,15 +124,17 @@ public class PrometheusSinkConfiguration { @JsonProperty("remote_write_version") private String remoteWriteVersion = DEFAULT_REMOTE_WRITE_VERSION; + @JsonProperty("request_timout") + private Duration requestTimout; + private boolean sslCertAndKeyFileInS3; public String getUrl() { return url; } - public boolean isSsl() { - return ssl; - } + public boolean isInsecureSkipVerify() { + return insecureSkipVerify; } public Duration getHttpRetryInterval() { return httpRetryInterval; @@ -154,7 +161,7 @@ public void validateAndInitializeCertAndKeyFileInS3() { if (useAcmCertForSSL) { validateSSLArgument(String.format(SSL_IS_ENABLED, useAcmCertForSSL), acmCertificateArn, acmCertificateArn); validateSSLArgument(String.format(SSL_IS_ENABLED, useAcmCertForSSL), awsAuthenticationOptions.getAwsRegion().toString(), AWS_REGION); - } else if(ssl) { + } else if(!insecureSkipVerify) { validateSSLCertificateFiles(); certAndKeyFileInS3 = isSSLCertificateLocatedInS3(); if (certAndKeyFileInS3) { @@ -262,4 +269,17 @@ public String getContentType() { public String getRemoteWriteVersion() { return remoteWriteVersion; } + + public boolean isInsecure() { + return insecure; + } + + public Duration getRequestTimout() { + return requestTimout; + } + + public boolean isHttpUrl() { + URL parsedUrl = PrometheusSinkUtil.getURLByUrlString(url); + return parsedUrl.getProtocol().equals(HTTP); + } } \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java index 4767817e95..9959bb24a7 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java @@ -8,6 +8,7 @@ import com.arpnetworking.metrics.prometheus.Types; import com.fasterxml.jackson.databind.ObjectMapper; import io.micrometer.core.instrument.Counter; +import org.apache.hc.client5.http.config.RequestConfig; import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; import org.apache.hc.client5.http.io.HttpClientConnectionManager; import org.apache.hc.client5.http.protocol.HttpClientContext; @@ -16,6 +17,7 @@ import org.apache.hc.core5.http.HttpStatus; import org.apache.hc.core5.http.io.entity.ByteArrayEntity; import org.apache.hc.core5.http.io.support.ClassicRequestBuilder; +import org.apache.hc.core5.util.Timeout; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; @@ -119,13 +121,12 @@ public PrometheusSinkService(final PrometheusSinkConfiguration prometheusSinkCon this.httpClientBuilder = httpClientBuilder; this.httpPluginSetting = httpPluginSetting; this.oAuthAccessTokenManager = new OAuthAccessTokenManager(); - if (prometheusSinkConfiguration.isSsl() || prometheusSinkConfiguration.useAcmCertForSSL()) { + if ((!prometheusSinkConfiguration.isInsecureSkipVerify()) || (prometheusSinkConfiguration.useAcmCertForSSL())) { this.certificateProviderFactory = new CertificateProviderFactory(prometheusSinkConfiguration.useAcmCertForSSL(), prometheusSinkConfiguration.getAwsAuthenticationOptions().getAwsRegion(), prometheusSinkConfiguration.getAcmCertificateArn(), prometheusSinkConfiguration.getAcmCertIssueTimeOutMillis(), prometheusSinkConfiguration.getAcmPrivateKeyPassword(), prometheusSinkConfiguration.isSslCertAndKeyFileInS3(), prometheusSinkConfiguration.getSslCertificateFile(), prometheusSinkConfiguration.getSslKeyFile()); - prometheusSinkConfiguration.validateAndInitializeCertAndKeyFileInS3(); this.httpClientConnectionManager = new HttpClientSSLConnectionManager() .createHttpClientConnectionManager(prometheusSinkConfiguration, certificateProviderFactory); } @@ -385,6 +386,10 @@ private Map buildAuthHttpSinkObjectsByConfig(final Prome LOG.info("sending data via proxy {}",proxyUrlString); } + if(prometheusSinkConfiguration.getRequestTimout() != null) { + httpClientBuilder.setDefaultRequestConfig(RequestConfig.custom().setConnectionRequestTimeout(Timeout.ofMilliseconds(prometheusSinkConfiguration.getRequestTimout().toMillis())).build()); + } + final HttpAuthOptions.Builder authOptions = new HttpAuthOptions.Builder() .setUrl(prometheusSinkConfiguration.getUrl()) .setClassicHttpRequestBuilder(classicRequestBuilder) diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java index e7e77f3d90..b248fe014d 100644 --- a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfigurationTest.java @@ -43,7 +43,7 @@ public class PrometheusSinkConfigurationTest { " token_url: https://localhost/oauth2/default/v1/token\n" + " grant_type: client_credentials\n" + " scope: httpSink\n"+ - " ssl: false\n" + + " insecure_skip_verify: true\n" + " dlq_file: \"/your/local/dlq-file\"\n" + " dlq:\n" + " s3:\n" + @@ -95,7 +95,7 @@ void get_authentication_test() { @Test void default_ssl_test() { - assertThat(new PrometheusSinkConfiguration().isSsl(), equalTo(false)); + assertThat(new PrometheusSinkConfiguration().isInsecureSkipVerify(), equalTo(false)); } @@ -183,7 +183,7 @@ public void validate_and_initialize_cert_and_key_file_in_s3_test() throws JsonPr " http_basic:\n" + " username: \"username\"\n" + " password: \"vip\"\n" + - " ssl: true\n" + + " insecure_skip_verify: false\n" + " use_acm_cert_for_ssl: false\n"+ " acm_certificate_arn: acm_cert\n" + " ssl_certificate_file: \"/full/path/to/certfile.crt\"\n" + diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java index ed7ede1509..dd48985e6e 100644 --- a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java @@ -67,7 +67,7 @@ public class PrometheusSinkServiceTest { " token_url: https://localhost/oauth2/default/v1/token\n" + " grant_type: client_credentials\n" + " scope: httpSink\n"+ - " ssl: false\n" + + " insecure_skip_verify: true\n" + " dlq_file: \"/your/local/dlq-file\"\n" + " dlq:\n" + " ssl_certificate_file: \"/full/path/to/certfile.crt\"\n" + From 97104151667cff62da521538302aeb9e1799e887 Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Wed, 11 Oct 2023 16:02:26 +0530 Subject: [PATCH 18/18] Insecure Skip Verify resolved for #1744. Signed-off-by: mallikagogoi7 --- .../HttpClientSSLConnectionManager.java | 18 +++++++++++++++++- .../service/PrometheusSinkService.java | 10 ++++++++++ .../service/PrometheusSinkServiceTest.java | 3 ++- 3 files changed, 29 insertions(+), 2 deletions(-) diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java index 1c2af56de7..fa2d86ffdb 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/certificate/HttpClientSSLConnectionManager.java @@ -7,6 +7,7 @@ import org.apache.hc.client5.http.config.TlsConfig; import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder; import org.apache.hc.client5.http.io.HttpClientConnectionManager; +import org.apache.hc.client5.http.ssl.NoopHostnameVerifier; import org.apache.hc.client5.http.ssl.SSLConnectionSocketFactory; import org.apache.hc.client5.http.ssl.SSLConnectionSocketFactoryBuilder; import org.apache.hc.client5.http.ssl.TrustAllStrategy; @@ -15,7 +16,6 @@ import org.apache.hc.core5.ssl.SSLContexts; import org.apache.hc.core5.ssl.TrustStrategy; import org.apache.hc.core5.util.Timeout; -import org.opensearch.dataprepper.plugins.certificate.CertificateProvider; import org.opensearch.dataprepper.plugins.certificate.s3.CertificateProviderFactory; import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; @@ -23,7 +23,10 @@ import java.io.ByteArrayInputStream; import java.io.InputStream; import java.nio.charset.StandardCharsets; +import java.security.KeyManagementException; import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; import java.security.cert.Certificate; import java.security.cert.CertificateFactory; @@ -78,4 +81,17 @@ private SSLContext getTrustAllStrategy() { throw new RuntimeException(ex.getMessage(), ex); } } + + public HttpClientConnectionManager createHttpClientConnectionManagerWithoutValidation() throws NoSuchAlgorithmException, KeyStoreException, KeyManagementException { + { + return PoolingHttpClientConnectionManagerBuilder.create() + .setSSLSocketFactory(SSLConnectionSocketFactoryBuilder.create() + .setSslContext(SSLContextBuilder.create() + .loadTrustMaterial(TrustAllStrategy.INSTANCE) + .build()) + .setHostnameVerifier(NoopHostnameVerifier.INSTANCE) + .build()) + .build(); + } + } } diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java index 9959bb24a7..10251041a3 100644 --- a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java @@ -50,6 +50,9 @@ import java.io.IOException; +import java.security.KeyManagementException; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.ArrayList; @@ -130,6 +133,13 @@ public PrometheusSinkService(final PrometheusSinkConfiguration prometheusSinkCon this.httpClientConnectionManager = new HttpClientSSLConnectionManager() .createHttpClientConnectionManager(prometheusSinkConfiguration, certificateProviderFactory); } + else{ + try { + this.httpClientConnectionManager = new HttpClientSSLConnectionManager().createHttpClientConnectionManagerWithoutValidation(); + }catch(NoSuchAlgorithmException | KeyStoreException | KeyManagementException ex){ + LOG.error("Exception while insecure_skip_verify is true ",ex); + } + } this.prometheusSinkRecordsSuccessCounter = pluginMetrics.counter(PROMETHEUS_SINK_RECORDS_SUCCESS_COUNTER); this.prometheusSinkRecordsFailedCounter = pluginMetrics.counter(PROMETHEUS_SINK_RECORDS_FAILED_COUNTER); this.httpAuthOptions = buildAuthHttpSinkObjectsByConfig(prometheusSinkConfiguration); diff --git a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java index dd48985e6e..84d091eb34 100644 --- a/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java +++ b/data-prepper-plugins/prometheus-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkServiceTest.java @@ -17,6 +17,7 @@ import org.apache.hc.core5.http.ClassicHttpRequest; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.configuration.PluginSetting; @@ -114,7 +115,7 @@ void setup() throws IOException { this.closeableHttpResponse = mock(CloseableHttpResponse.class); this.prometheusSinkRecordsSuccessCounter = mock(Counter.class); this.prometheusSinkRecordsFailedCounter = mock(Counter.class); - lenient().when(httpClientBuilder.setConnectionManager(null)).thenReturn(httpClientBuilder); + lenient().when(httpClientBuilder.setConnectionManager(Mockito.any())).thenReturn(httpClientBuilder); lenient().when(httpClientBuilder.addResponseInterceptorLast(any(FailedHttpResponseInterceptor.class))).thenReturn(httpClientBuilder); lenient().when(httpClientBuilder.build()).thenReturn(closeableHttpClient); lenient().when(closeableHttpClient.execute(any(ClassicHttpRequest.class),any(HttpClientContext.class))).thenReturn(closeableHttpResponse);