amogh-jahagirdar commented on code in PR #7444: URL: https://github.com/apache/iceberg/pull/7444#discussion_r1178657313
########## aws/src/main/java/org/apache/iceberg/aws/metrics/SqsMetricsReporter.java: ########## @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.aws.metrics; + +import java.util.Map; +import org.apache.iceberg.aws.AwsProperties; +import org.apache.iceberg.metrics.CommitReport; +import org.apache.iceberg.metrics.CommitReportParser; +import org.apache.iceberg.metrics.MetricsReport; +import org.apache.iceberg.metrics.MetricsReporter; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.metrics.ScanReportParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; +import software.amazon.awssdk.services.sqs.SqsClient; +import software.amazon.awssdk.services.sqs.model.SendMessageRequest; + +/** An implementation of {@link MetricsReporter} which reports {@link MetricsReport} to SQS */ +public class SqsMetricsReporter implements MetricsReporter { + + private static final Logger LOG = LoggerFactory.getLogger(SqsMetricsReporter.class); + + private SqsClient sqsClient; + + private String sqsQueueUrl; + + @Override + public void initialize(Map<String, String> properties) { + AwsProperties awsProperties = new AwsProperties(properties); + sqsClient = SqsClient.builder().httpClient(UrlConnectionHttpClient.builder().build()).build(); + sqsQueueUrl = awsProperties.sqsQueueUrl(); + Preconditions.checkArgument( + null != sqsQueueUrl, "%s should be be set", AwsProperties.SQS_QUEUE_URL); + } + + @Override + public void report(MetricsReport report) { + String message = null; + if (report instanceof CommitReport) { + message = CommitReportParser.toJson((CommitReport) report); + } else if (report instanceof ScanReport) { + message = ScanReportParser.toJson((ScanReport) report); + } + + LOG.info("Received metrics report: {}", message); + sqsClient.sendMessage( + SendMessageRequest.builder().messageBody(message).queueUrl(sqsQueueUrl).build()); + sqsClient.close(); Review Comment: Also just wanted to confirm with @nastra what is the expectation of implementors of MetricsReporter, are all MetricsReporter#report calls expected to be best effort (so wherever metricsReporter is plugged in, it catches all exceptions generically), or do implementors need to handle that themselves? I imagine the first case since generally metrics reporting shouldn't block the critical path but just wanted to make sure -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
