Skip to content

Conversation

@rajarshisarkar
Copy link
Contributor

@rajarshisarkar rajarshisarkar commented Apr 27, 2023

This PR adds support for AWS SQS MetricsReporter.

Spark SQL launch command:

sh spark-sql --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \
    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
    --conf spark.sql.catalog.my_catalog.warehouse=s3://<warehouse-path> \
    --conf spark.sql.catalog.my_catalog.metrics-reporter-impl=org.apache.iceberg.aws.metrics.SqsMetricsReporter \
    --conf spark.sql.catalog.my_catalog.metrics-reporter.sqs.queue-url=<sqs-queue-url>

cc: @jackye1995 @amogh-jahagirdar @singhpk234

Copy link
Contributor

@amogh-jahagirdar amogh-jahagirdar left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for publishing @rajarshisarkar had some comments, also can we add unit tests?

@jackye1995
Copy link
Contributor

Feels like we should use the async client for this? We should not be blocked by metrics reporting calls

@rajarshisarkar
Copy link
Contributor Author

Feels like we should use the async client for this? We should not be blocked by metrics reporting calls

This would mean we introduce the AsyncHttp clients too: https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/http-configuration.html#http-clients-available. Thoughts?

@rajarshisarkar rajarshisarkar force-pushed the aws-metrics-reporter branch 3 times, most recently from 172fca2 to f18775e Compare May 30, 2023 08:34
@rajarshisarkar
Copy link
Contributor Author

Feels like we should use the async client for this? We should not be blocked by metrics reporting calls

Added NettyNioAsyncHttpClient.

@rajarshisarkar rajarshisarkar force-pushed the aws-metrics-reporter branch 2 times, most recently from bf2cabc to 5812918 Compare June 7, 2023 12:16
@rajarshisarkar
Copy link
Contributor Author

Introduced SqsMetricsReporterAwsClientFactory.

@rajarshisarkar rajarshisarkar force-pushed the aws-metrics-reporter branch 2 times, most recently from 73d443e to 9d4fd29 Compare June 7, 2023 13:11
@rajarshisarkar rajarshisarkar force-pushed the aws-metrics-reporter branch from 9d4fd29 to 9c919ce Compare June 8, 2023 08:36
* <p>For more details, see
* https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClient.Builder.html
*/
public static final String NETTYNIO_WRITE_TIMEOUT = "http-client.nettynio.write-timeout";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: can you add time units to this, like -ms?

* <p>For more details, see
* https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClient.Builder.html
*/
public static final String NETTYNIO_READ_TIMEOUT = "http-client.nettynio.read-timeout";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: can you add time units to this, like -ms?

PropertyUtil.propertyAsString(
properties,
SqsMetricsReporterProperties.METRICS_REPORTER_SQS_CLIENT_FACTORY_IMPL,
DefaultSqsMetricsReporterAwsClientFactory.class.getName());
Copy link
Contributor

@jackye1995 jackye1995 Jun 8, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

similar to the old client factory, we should be able to make DefaultSqsMetricsReporterAwsClientFactory a singleton, and directly return that if METRICS_REPORTER_SQS_CLIENT_FACTORY_IMPL is not set.

import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import software.amazon.awssdk.services.sqs.SqsAsyncClient;

public class DefaultSqsMetricsReporterAwsClientFactory
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this can be an inner package protected class in the factories class.

message = ScanReportParser.toJson((ScanReport) report);
}

if (null == message) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this can be just else clause of the last clause? The parsers cannot return null.

}
}
});
} catch (Exception e) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we at least use RuntimeException? Exception is too generic

}
});
} catch (Exception e) {
LOG.warn("Failed to report metrics to SQS queue {}", sqsQueueUrl, e);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this deserves a LOG.error?

future.whenComplete(
(response, error) -> {
if (response != null) {
LOG.info("Metrics {} reported to: {}", response, sqsQueueUrl);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think we need to log the response? Just logging error when it exists seems sufficient to me, what do you think?

}

@Test
public void report() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: use testXXX for testing methods


@Test
public void report() {
sqsMetricsReporter.report(commitReport);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we test both scan and commit reports, and more test cases for success and failure cases that you are catching in the implementation?

@github-actions
Copy link

This pull request has been marked as stale due to 30 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the dev@iceberg.apache.org list. Thank you for your contributions.

@github-actions github-actions bot added the stale label Aug 29, 2024
@github-actions
Copy link

github-actions bot commented Sep 5, 2024

This pull request has been closed due to lack of activity. This is not a judgement on the merit of the PR in any way. It is just a way of keeping the PR queue manageable. If you think that is incorrect, or the pull request requires review, you can revive the PR at any time.

@github-actions github-actions bot closed this Sep 5, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants