From c61b26bedea9dc374ccf36737609ed4e960f942d Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 8 Feb 2021 18:06:31 +0000 Subject: [PATCH 1/3] HADOOP-17511. Add audit/telemetry logging to S3A connector Notion of AuditSpan which is created for a given operation; goal is to pass it along everywhere. It's thread local per FS-instance; store operations pick this up in their constructor from the StoreContext. The entryPoint() method in S3A FS has been enhanced to initiate the spans. For this to work, internal code SHALL NOT call those entry points (Done) and all public API points MUST be declared as entry points. This is done, with a marker attribute @AuditEntryPoint to indicate this. The audit span create/deactivate sequence is ~the same as the duration tracking so the operation is generally merged: most of the metrics S3AFS collects are now durations Part of the isolation into spans means that there's explicit operations for mkdirs() and getContentSummary() The auditing is intended to be a plugin point; currently there is the LoggingAuditor which logs at debug - adds an HTTP "referer" header with audit tracing - can be set to raise an exception if the SDK is handed an AWS Request and there's no active span (skipped for the multipart upload part and complete calls as TransferManager in the SDK does that out of span). NoopAuditor which: - does nothing Change-Id: If11a2c48b00db530fb6bc1ad363e24b202acb827 HADOOP-17511 Auditing: getContentSummary and dynamic evaluation (wip) * added getContentSummary as a single span, with some minor speedups * still trying to do best design to wire up dynamically evaluated attributes Currently logs should include the current thread ID; we don't yet pick up and include the thread where the span was created, which is equally important Change-Id: Ieea88e4228da0ac4761d8c006051cd1095c5fce8 HADOOP-17511. Audit Spans to have unique IDs. + an interface ActiveThreadSpanSource to give current thread span. This is to allow integrations with the AWS SDK &C to query the active span for an FS and immediately be able to identify span by ID, for logging etc. Adding a unique ID to all audit spans and supporting active thread span (with activate/deactivate) causes major changes in the no-op code, as suddenly there's a lot more state there across manager, auditor and span. will be testable though. Change-Id: Id4dddcab7b735bd01f3d3b8a8236ff6da8f97671 HADOOP-17511. Audit review * SpanIDs must now be unique over time (helps in log analysis) * All AWS SDK events go to AuditSpan * FS.access() check also goes to Auditor. This is used by Hive Change-Id: Id1ffffd928f2e274f1bac73109d16e6624ba0e9d HADOOP-17511. Audit -timestamp, headers and handlers - timestamp of span creation picked up (epoch millis in UTC) and passed in to referrer - docs on referrer fields - section on privacy implications - referrer header can be disabled (privacy...) - custom request handlers will be created (TODO: tests) Change-Id: I6e94b43a209eee53748ac14270f318352d512fb8 HADOOP-17511: Unit test work There's lots of implicit tests in the functional suites, but this adds tests for * binding of both bundled auditors * adding extra request handlers * wiring up of context to span * and to TransferManager lifecycle events * common context static and dynamic eval * WiP: parsing back of the http referrer header. This gives reasonable coverage of what's going on, though another day's work would round it out. Change-Id: I6b2d0f1dff223875268c18ded481d9e9fea2f250 HADOOP-17511. Unit and integration tests * Tests of audit execution and stats update. * Integration test suite for S3A.access() * Tuning of implementation classes for ease of testing. * Exporting auditor from S3AFS. * More stats collected * Move AuditCheckingException under CredentialInitializationException so that s3a translateException doesn't need changing. * audit p1 & p2 paths moving to be key only * auditing docs includes real s3 logs and breakdown of referrer (TODO update) The main remaining bits of testing would be to take existing code and verify that the headers got through, especially some of the commit logic and job ID. Distcp might if the jobID is in the config. Change-Id: I5723db55ba189f6c400cf29a90aa5605b0d98ad0 HADOOP-17511. Improving auditor binding inside AWS SDK Audit opname in Span callbacks; always set active span from request. This allows the active operation to always be determined, including from any other plugins (Cred provider, signer...) used in same execution. This is also why Auditor is now in StoreContext. Tests for RequestFactory. Change-Id: I9528253cf21253e14714b838d3a8ae85d52ba8b7 HADOOP-17511. checkstyle and more testing Change-Id: If12f8204237eb0d79f2edcff03fc45f31b7d196a HADOOP-17511. Auditing: move AuditSpan and common context to hadoop-common Small move of code, changes in imports more traumatic Change-Id: Ide158d884bd7a873e07f0ddaff8334882eb28595 HADOOP-17511. Auditing * avoiding accidentally deactivating spans * caching of and switching to active span in rename/delete callbacks * consistent access/use of AuditorId for FS ID * which is printed in S3AFileSystem.toString(). * S3Guard Tools doing more for audit; also printing IOStats on -verbose. * Store Context taking/returning an AuditSpanSource, not the full OperationAuditor interface. Change-Id: Ifc5f807a2d3a329b8a1184dd1fcba63205c1f174 HADOOP-17511. Auditing - marker tool. Marker tool always uses a span; this is created in the FS, rather than the marker Change-Id: I03e31dd58c76a41e8a1b73e958b130ed405a29fe HADOOP-17511. Auditing -explicit integration tests. Tests to deliberately create problems and so verify that things fail as expected. Change-Id: If2e863cee54aa303c24a3d02174e466c272f24b2 HADOOP-17511. Auditing code/docs cleanup * Move audit classes in hadoop-common into their own package * move ActiveThreadSpanSource interface there and implement in S3AFS. that's not for public consumption, but it may be good to have there so that abfs can implement the same API. Change-Id: I1b7d924555a1294f7acb3f47dc613adc32ffb003 HADOOP-17511. Auditing: S3 Log parser Pattern with real tests Show everything works with a full parse of the output captured from a log entry of a real test run. This is the most complicated Regexp I have ever written. Change-Id: I090b2dcefad9938bea3b95ef717a7cb2e9eea10c HADOOP-17511 add filtering of header fields; with docs Change-Id: I0da9487a708b5a8fd700ffddd0290d6c0621f3e2 HADOOP-17511. Audit: add o.a.h.fs.audit package with the public classes. fix up yetus complaints, where valid. Change-Id: I98e4f7a9c277c993555db6d62a20f2a00515c5e8 HADOOP-17511 review * Moved HttpReferrerAuditHeader class * Added version string to URL * Explained some design decisions in the architecture doc * Added mukund's comments Change-Id: I356e5428c51f74b25584bfb1674296ac193c81d5 --- .../hadoop/fs/audit/AuditConstants.java | 108 ++ .../hadoop/fs/audit/AuditStatisticNames.java | 50 + .../hadoop/fs/audit/CommonAuditContext.java | 288 ++++ .../apache/hadoop/fs/audit/package-info.java | 29 + .../org/apache/hadoop/fs/shell/Command.java | 3 + .../org/apache/hadoop/fs/shell/PathData.java | 18 +- .../fs/statistics/StoreStatisticNames.java | 10 + .../impl/EmptyIOStatisticsStore.java | 182 +++ .../statistics/impl/IOStatisticsBinding.java | 42 + .../hadoop/fs/store}/LogExactlyOnce.java | 12 +- .../store/audit/ActiveThreadSpanSource.java | 38 + .../fs/store/audit/AuditEntryPoint.java | 55 + .../hadoop/fs/store/audit/AuditSpan.java | 109 ++ .../fs/store/audit/AuditSpanSource.java | 50 + .../fs/store/audit/AuditingFunctions.java | 121 ++ .../store/audit/HttpReferrerAuditHeader.java | 501 ++++++ .../hadoop/fs/store/audit/package-info.java | 29 + .../service/launcher/ServiceLauncher.java | 2 + .../org/apache/hadoop/util/ToolRunner.java | 5 + .../util/functional/RemoteIterators.java | 34 +- .../fs/audit/TestCommonAuditContext.java | 161 ++ .../hadoop/fs/contract/ContractTestUtils.java | 10 +- .../CredentialInitializationException.java | 9 + .../org/apache/hadoop/fs/s3a/Invoker.java | 38 +- .../org/apache/hadoop/fs/s3a/Listing.java | 131 +- .../apache/hadoop/fs/s3a/MultipartUtils.java | 110 +- .../hadoop/fs/s3a/S3ABlockOutputStream.java | 12 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 1382 +++++++++++------ .../apache/hadoop/fs/s3a/S3AInputStream.java | 45 +- .../hadoop/fs/s3a/S3AInstrumentation.java | 72 +- .../hadoop/fs/s3a/S3ALocatedFileStatus.java | 1 + .../hadoop/fs/s3a/S3AReadOpContext.java | 18 +- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 26 +- .../apache/hadoop/fs/s3a/S3ClientFactory.java | 5 + .../org/apache/hadoop/fs/s3a/Statistic.java | 72 +- .../hadoop/fs/s3a/WriteOperationHelper.java | 290 ++-- .../apache/hadoop/fs/s3a/WriteOperations.java | 18 +- .../hadoop/fs/s3a/api/RequestFactory.java | 296 ++++ .../hadoop/fs/s3a/api/package-info.java | 34 + .../fs/s3a/audit/AWSAuditEventCallbacks.java | 160 ++ .../fs/s3a/audit/AWSRequestAnalyzer.java | 296 ++++ .../fs/s3a/audit/AuditFailureException.java | 41 + .../hadoop/fs/s3a/audit/AuditIntegration.java | 137 ++ .../hadoop/fs/s3a/audit/AuditManagerS3A.java | 92 ++ .../hadoop/fs/s3a/audit/AuditSpanS3A.java | 29 + .../hadoop/fs/s3a/audit/OperationAuditor.java | 73 + .../fs/s3a/audit/OperationAuditorOptions.java | 74 + .../fs/s3a/audit/S3AAuditConstants.java | 104 ++ .../hadoop/fs/s3a/audit/S3LogParser.java | 309 ++++ .../hadoop/fs/s3a/audit/S3LogVerbs.java | 51 + .../s3a/audit/impl/AbstractAuditSpanImpl.java | 99 ++ .../audit/impl/AbstractOperationAuditor.java | 121 ++ .../s3a/audit/impl/ActiveAuditManagerS3A.java | 773 +++++++++ .../fs/s3a/audit/impl/LoggingAuditor.java | 447 ++++++ .../s3a/audit/impl/NoopAuditManagerS3A.java | 185 +++ .../hadoop/fs/s3a/audit/impl/NoopAuditor.java | 99 ++ .../hadoop/fs/s3a/audit/impl/NoopSpan.java | 109 ++ .../audit/impl/S3AInternalAuditConstants.java | 44 + .../fs/s3a/audit/impl/package-info.java | 28 + .../hadoop/fs/s3a/audit/package-info.java | 41 + .../fs/s3a/commit/AbstractS3ACommitter.java | 66 +- .../fs/s3a/commit/CommitOperations.java | 31 +- .../hadoop/fs/s3a/commit/CommitUtils.java | 24 + .../commit/InternalCommitterConstants.java | 6 + .../fs/s3a/commit/MagicCommitIntegration.java | 15 +- .../s3a/commit/magic/MagicCommitTracker.java | 2 +- .../s3a/commit/staging/StagingCommitter.java | 2 + .../fs/s3a/impl/AbstractStoreOperation.java | 40 +- .../hadoop/fs/s3a/impl/CallableSupplier.java | 52 +- .../fs/s3a/impl/ChangeDetectionPolicy.java | 1 + .../hadoop/fs/s3a/impl/ContextAccessors.java | 21 +- .../hadoop/fs/s3a/impl/DeleteOperation.java | 17 +- .../fs/s3a/impl/ExecutingStoreOperation.java | 40 +- .../s3a/impl/GetContentSummaryOperation.java | 208 +++ .../hadoop/fs/s3a/impl/HeaderProcessing.java | 44 +- .../hadoop/fs/s3a/impl/InternalConstants.java | 10 + .../s3a/impl/ListingOperationCallbacks.java | 26 +- .../hadoop/fs/s3a/impl/MkdirOperation.java | 184 +++ .../hadoop/fs/s3a/impl/RenameOperation.java | 22 +- .../fs/s3a/impl/RequestFactoryImpl.java | 695 +++++++++ .../hadoop/fs/s3a/impl/StoreContext.java | 60 +- .../fs/s3a/impl/StoreContextBuilder.java | 18 +- .../s3a/s3guard/DumpS3GuardDynamoTable.java | 32 +- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 4 +- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 57 +- .../hadoop/fs/s3a/select/SelectBinding.java | 29 +- .../impl/ForwardingIOStatisticsStore.java | 186 +++ .../hadoop/fs/s3a/tools/MarkerTool.java | 9 +- .../markdown/tools/hadoop-aws/auditing.md | 330 ++++ .../tools/hadoop-aws/auditing_architecture.md | 323 ++++ .../site/markdown/tools/hadoop-aws/index.md | 2 + .../tools/hadoop-aws/audit-architecture.png | Bin 0 -> 55701 bytes .../fs/contract/s3a/ITestS3AContractSeek.java | 2 +- .../hadoop/fs/s3a/AbstractS3ATestBase.java | 46 + .../hadoop/fs/s3a/ITestS3AEmptyDirectory.java | 7 +- .../fs/s3a/ITestS3AFailureHandling.java | 13 +- .../hadoop/fs/s3a/ITestS3AMiscOperations.java | 23 +- .../hadoop/fs/s3a/ITestS3AMultipartUtils.java | 4 +- .../fs/s3a/ITestS3ARemoteFileChanged.java | 2 +- .../hadoop/fs/s3a/ITestS3GuardEmptyDirs.java | 45 +- .../hadoop/fs/s3a/MockS3AFileSystem.java | 44 +- .../hadoop/fs/s3a/MultipartTestUtils.java | 57 +- .../apache/hadoop/fs/s3a/S3ATestUtils.java | 46 - .../fs/s3a/TestS3ABlockOutputStream.java | 11 +- .../fs/s3a/audit/AbstractAuditingTest.java | 213 +++ .../fs/s3a/audit/AccessCheckingAuditor.java | 53 + .../hadoop/fs/s3a/audit/AuditTestSupport.java | 123 ++ .../fs/s3a/audit/ITestAuditAccessChecks.java | 201 +++ .../fs/s3a/audit/ITestAuditManager.java | 135 ++ .../fs/s3a/audit/SimpleAWSRequestHandler.java | 54 + .../fs/s3a/audit/TestAuditIntegration.java | 216 +++ .../fs/s3a/audit/TestAuditSpanLifecycle.java | 133 ++ .../audit/TestHttpReferrerAuditHeader.java | 323 ++++ .../fs/s3a/audit/TestLoggingAuditor.java | 203 +++ .../hadoop/fs/s3a/auth/ITestAssumeRole.java | 3 +- .../fs/s3a/commit/AbstractCommitITest.java | 12 +- .../s3a/commit/AbstractITCommitProtocol.java | 3 +- .../fs/s3a/commit/ITestCommitOperations.java | 3 +- .../hadoop/fs/s3a/impl/ITestXAttrCost.java | 3 +- .../fs/s3a/impl/TestHeaderProcessing.java | 27 +- .../s3a/impl/TestPartialDeleteFailures.java | 16 +- .../fs/s3a/impl/TestRequestFactory.java | 185 +++ .../s3a/performance/AbstractS3ACostTest.java | 33 +- .../ITestDirectoryMarkerListing.java | 13 +- .../ITestS3AMiscOperationCost.java | 143 ++ .../fs/s3a/performance/ITestS3AMkdirCost.java | 185 +++ .../performance/OperationCostValidator.java | 3 +- .../fs/s3a/s3guard/ITestS3GuardFsck.java | 2 - .../fs/s3a/s3guard/ITestS3GuardToolLocal.java | 47 +- .../s3a/scale/AbstractSTestS3AHugeFiles.java | 3 +- .../scale/ITestS3ADirectoryPerformance.java | 137 +- .../MinimalListingOperationCallbacks.java | 10 +- .../src/test/resources/core-site.xml | 13 + .../src/test/resources/log4j.properties | 6 + 134 files changed, 11933 insertions(+), 1152 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditStatisticNames.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/package-info.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatisticsStore.java rename {hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl => hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store}/LogExactlyOnce.java (81%) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/ActiveThreadSpanSource.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditEntryPoint.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditSpan.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditSpanSource.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditingFunctions.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/HttpReferrerAuditHeader.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/package-info.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/audit/TestCommonAuditContext.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/package-info.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSAuditEventCallbacks.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditFailureException.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanS3A.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditorOptions.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3AAuditConstants.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogParser.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogVerbs.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractAuditSpanImpl.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManagerS3A.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditor.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopSpan.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/S3AInternalAuditConstants.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/package-info.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/package-info.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/ForwardingIOStatisticsStore.java create mode 100644 hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md create mode 100644 hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing_architecture.md create mode 100644 hadoop-tools/hadoop-aws/src/site/resources/tools/hadoop-aws/audit-architecture.png create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AccessCheckingAuditor.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditAccessChecks.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/SimpleAWSRequestHandler.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditIntegration.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditSpanLifecycle.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestLoggingAuditor.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMiscOperationCost.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMkdirCost.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java new file mode 100644 index 0000000000000..d9629e388b384 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java @@ -0,0 +1,108 @@ +/* + * 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.hadoop.fs.audit; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Constants related to auditing. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class AuditConstants { + + private AuditConstants() { + } + + /** + * The host from where requests originate: {@value}. + * example.org is used as the IETF require that it never resolves. + * This isn't always met by some mobile/consumer DNS services, but + * we don't worry about that. What is important is that + * a scan for "example.org" in the logs will exclusively find + * entries from this referrer. + */ + public static final String REFERRER_ORIGIN_HOST = "audit.example.org"; + + /** + * Header: Command: {@value}. + * Set by tool runner. + */ + public static final String PARAM_COMMAND = "cm"; + + /** + * Header: FileSystem ID: {@value}. + */ + public static final String PARAM_FILESYSTEM_ID = "fs"; + + /** + * Header: operation ID: {@value}. + */ + public static final String PARAM_ID = "id"; + + /** + * JobID query header: {@value}. + */ + public static final String PARAM_JOB_ID = "ji"; + + /** + * Header: operation: {@value}. + * These should be from StoreStatisticNames or similar, + * and are expected to be at the granularity of FS + * API operations. + */ + public static final String PARAM_OP = "op"; + + /** + * Header: first path of operation: {@value}. + */ + public static final String PARAM_PATH = "p1"; + + /** + * Header: second path of operation: {@value}. + */ + public static final String PARAM_PATH2 = "p2"; + + /** + * Header: Principal: {@value}. + */ + public static final String PARAM_PRINCIPAL = "pr"; + + /** + * Header: Process ID: {@value}. + */ + public static final String PARAM_PROCESS = "ps"; + + /** + * Thread 0: the thread which created a span {@value}. + */ + public static final String PARAM_THREAD0 = "t0"; + + /** + * Thread 1: the thread making the S3 request: {@value}. + */ + public static final String PARAM_THREAD1 = "t1"; + + /** + * Timestamp of span creation: {@value}. + */ + public static final String PARAM_TIMESTAMP = "ts"; + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditStatisticNames.java new file mode 100644 index 0000000000000..0ee9d626bd9c7 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditStatisticNames.java @@ -0,0 +1,50 @@ +/* + * 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.hadoop.fs.audit; + +/** + * Statistic Names for Auditing. + */ +public final class AuditStatisticNames { + + private AuditStatisticNames() { + } + + /** + * Audit failure: {@value}. + */ + public static final String AUDIT_FAILURE = "audit_failure"; + + /** + * A request was executed and the auditor invoked: {@value}. + */ + public static final String AUDIT_REQUEST_EXECUTION + = "audit_request_execution"; + + /** + * Audit span created: {@value}. + */ + public static final String AUDIT_SPAN_CREATION = "audit_span_creation"; + + /** + * Access check during audit rejected: {@value}. + */ + public static final String AUDIT_ACCESS_CHECK_FAILURE + = "audit_access_check_failure"; +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java new file mode 100644 index 0000000000000..11681546e3d0a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java @@ -0,0 +1,288 @@ +/* + * 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.hadoop.fs.audit; + +import java.util.Iterator; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Supplier; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_COMMAND; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PROCESS; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD1; + +/** + * The common audit context is a map of common context information + * which can be used with any audit span. + * This context is shared across all Filesystems within the + * thread. + * Audit spans will be created with a reference to the current + * context of their thread; + * That reference is retained even as they are moved across threads, so + * context information (including thread ID Java runtime). + * + * The Global context entries are a set of key-value pairs which span + * all threads; the {@code HttpReferrerAuditHeader} picks these + * up automatically. It is intended for minimal use of + * shared constant values (process ID, entry point). + * + * An attribute set in {@link #setGlobalContextEntry(String, String)} + * will be set across all audit spans in all threads. + * + * The {@link #noteEntryPoint(Object)} method should be + * used in entry points (ToolRunner.run, etc). It extracts + * the final element of the classname and attaches that + * to the global context with the attribute key + * {@link AuditConstants#PARAM_COMMAND}, if not already + * set. + * This helps identify the application being executued. + * + * All other values set are specific to this context, which + * is thread local. + * The attributes which can be added to ths common context include + * evaluator methods which will be evaluated in whichever thread + * invokes {@link #getEvaluatedEntries()} and then evaluates them. + * That map of evaluated options may evaluated later, in a different + * thread. + * + * For setting and clearing thread-level options, use + * {@link #currentAuditContext()} to get the thread-local + * context for the caller, which can then be manipulated. + * + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class CommonAuditContext { + + /** + * Process ID; currently built from UUID and timestamp. + */ + public static final String PROCESS_ID = UUID.randomUUID().toString(); + + /** + * Context values which are global. + * To be used very sparingly. + */ + private static final Map GLOBAL_CONTEXT_MAP = + new ConcurrentHashMap<>(); + + /** + * Map of data. Concurrent so when shared across threads + * there are no problems. + * Supplier operations must themselves be thread safe. + */ + private final Map> evaluatedEntries = + new ConcurrentHashMap<>(); + + static { + // process ID is fixed. + setGlobalContextEntry(PARAM_PROCESS, PROCESS_ID); + } + + /** + * Thread local context. + * Use a weak reference just to keep memory costs down. + * The S3A committers all have a strong reference, so if they are + * retained, context is retained. + * If a span retains the context, then it will also stay valid until + * the span is finalized. + */ + private static final ThreadLocal ACTIVE_CONTEXT = + ThreadLocal.withInitial(() -> createInstance()); + + private CommonAuditContext() { + } + + /** + * Put a context entry. + * @param key key + * @param value new value + * @return old value or null + */ + public Supplier put(String key, String value) { + return evaluatedEntries.put(key, () -> value); + } + + /** + * Put a context entry dynamically evaluated on demand. + * @param key key + * @param value new value + * @return old value or null + */ + public Supplier put(String key, Supplier value) { + return evaluatedEntries.put(key, value); + } + + /** + * Remove a context entry. + * @param key key + */ + public void remove(String key) { + evaluatedEntries.remove(key); + } + + /** + * Get a context entry. + * @param key key + * @return value or null + */ + public String get(String key) { + Supplier supplier = evaluatedEntries.get(key); + return supplier != null + ? supplier.get() + : null; + } + + /** + * Rest the context; will set the standard options again. + * Primarily for testing. + */ + public void reset() { + evaluatedEntries.clear(); + init(); + } + + /** + * Initialize. + */ + private void init() { + + // thread 1 is dynamic + put(PARAM_THREAD1, () -> currentThreadID()); + } + + /** + * Does the context contain a specific key? + * @param key key + * @return true if it is in the context. + */ + public boolean containsKey(String key) { + return evaluatedEntries.containsKey(key); + } + + /** + * Demand invoked to create the instance for this thread. + * @return an instance. + */ + private static CommonAuditContext createInstance() { + CommonAuditContext context = new CommonAuditContext(); + context.init(); + return context; + } + + /** + * Get the current common audit context. Thread local. + * @return the audit context of this thread. + */ + public static CommonAuditContext currentAuditContext() { + return ACTIVE_CONTEXT.get(); + } + + /** + * A thread ID which is unique for this process and shared across all + * S3A clients on the same thread, even those using different FS instances. + * @return a thread ID for reporting. + */ + public static String currentThreadID() { + return Long.toString(Thread.currentThread().getId()); + } + + /** + * Get the evaluated operations. + * This is the map unique to this context. + * @return the operations map. + */ + public Map> getEvaluatedEntries() { + return evaluatedEntries; + } + + /** + * Set a global entry. + * @param key key + * @param value value + */ + public static void setGlobalContextEntry(String key, String value) { + GLOBAL_CONTEXT_MAP.put(key, value); + } + + /** + * Get a global entry. + * @param key key + * @return value or null + */ + public static String getGlobalContextEntry(String key) { + return GLOBAL_CONTEXT_MAP.get(key); + } + + /** + * Remove a global entry. + * @param key key to clear. + */ + public static void removeGlobalContextEntry(String key) { + GLOBAL_CONTEXT_MAP.remove(key); + } + + /** + * Add the entry point as a context entry with the key + * {@link AuditConstants#PARAM_COMMAND} + * if it has not already been recorded. + * This is called via ToolRunner but may be used at any + * other entry point. + * @param tool object loaded/being launched. + */ + public static void noteEntryPoint(Object tool) { + if (tool != null && !GLOBAL_CONTEXT_MAP.containsKey(PARAM_COMMAND)) { + String classname = tool.getClass().toString(); + int lastDot = classname.lastIndexOf('.'); + int l = classname.length(); + if (lastDot > 0 && lastDot < (l - 1)) { + String name = classname.substring(lastDot + 1, l); + setGlobalContextEntry(PARAM_COMMAND, name); + } + } + } + + /** + * Get an iterator over the global entries. + * Thread safe. + * @return an iterable to enumerate the values. + */ + public static Iterable> + getGlobalContextEntries() { + return new GlobalIterable(); + } + + /** + * Iterable to the global iterator. Avoids serving + * up full access to the map. + */ + private static final class GlobalIterable + implements Iterable> { + + @Override + public Iterator> iterator() { + return GLOBAL_CONTEXT_MAP.entrySet().iterator(); + } + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/package-info.java new file mode 100644 index 0000000000000..16c224940dd03 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/package-info.java @@ -0,0 +1,29 @@ +/* + * 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. + */ + +/** + * Public classes for adding information to any auditing information + * picked up by filesystem clients. + * + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +package org.apache.hadoop.fs.audit; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java index c81825776a613..0bdb47730a929 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java @@ -38,6 +38,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator; + /** * An abstract class for the execution of a file system command */ @@ -361,6 +363,7 @@ protected void processPaths(PathData parent, } } } + cleanupRemoteIterator(itemsIterator); } private void processPathInternal(PathData item) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java index dad54ea07bdf1..1ff8d8f0494a1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java @@ -39,6 +39,8 @@ import org.apache.hadoop.fs.PathNotFoundException; import org.apache.hadoop.fs.RemoteIterator; +import static org.apache.hadoop.util.functional.RemoteIterators.mappingRemoteIterator; + /** * Encapsulates a Path (path), its FileStatus (stat), and its FileSystem (fs). * PathData ensures that the returned path string will be the same as the @@ -287,20 +289,8 @@ public RemoteIterator getDirectoryContentsIterator() throws IOException { checkIfExists(FileTypeRequirement.SHOULD_BE_DIRECTORY); final RemoteIterator stats = this.fs.listStatusIterator(path); - return new RemoteIterator() { - - @Override - public boolean hasNext() throws IOException { - return stats.hasNext(); - } - - @Override - public PathData next() throws IOException { - FileStatus file = stats.next(); - String child = getStringForChildPath(file.getPath()); - return new PathData(fs, child, file); - } - }; + return mappingRemoteIterator(stats, + file -> new PathData(fs, getStringForChildPath(file.getPath()), file)); } /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java index 95144393585a1..9ec8dcdb3dc9b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java @@ -40,6 +40,9 @@ public final class StoreStatisticNames { /** {@value}. */ public static final String OP_ABORT = "op_abort"; + /** access() API call {@value}. */ + public static final String OP_ACCESS = "op_access"; + /** {@value}. */ public static final String OP_APPEND = "op_append"; @@ -161,6 +164,10 @@ public final class StoreStatisticNames { public static final String DELEGATION_TOKENS_ISSUED = "delegation_tokens_issued"; + /** Probe for store existing: {@value}. */ + public static final String STORE_EXISTS_PROBE + = "store_exists_probe"; + /** Requests throttled and retried: {@value}. */ public static final String STORE_IO_THROTTLED = "store_io_throttled"; @@ -379,6 +386,9 @@ public final class StoreStatisticNames { public static final String MULTIPART_UPLOAD_STARTED = "multipart_upload_started"; + public static final String MULTIPART_UPLOAD_LIST + = "multipart_upload_list"; + private StoreStatisticNames() { } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatisticsStore.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatisticsStore.java new file mode 100644 index 0000000000000..c970546e6dcb8 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatisticsStore.java @@ -0,0 +1,182 @@ +/* + * 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.hadoop.fs.statistics.impl; + +import javax.annotation.Nullable; +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.MeanStatistic; + +import static java.util.Collections.emptyMap; + +/** + * An Empty IOStatisticsStore implementation. + */ +final class EmptyIOStatisticsStore implements IOStatisticsStore { + + /** + * The sole instance of this class. + */ + private static final EmptyIOStatisticsStore INSTANCE = + new EmptyIOStatisticsStore(); + /** + * Get the single instance of this class. + * @return a shared, empty instance. + */ + static IOStatisticsStore getInstance() { + return INSTANCE; + } + + private EmptyIOStatisticsStore() { + } + + @Override + public Map counters() { + return emptyMap(); + } + + @Override + public Map gauges() { + return emptyMap(); + } + + @Override + public Map minimums() { + return emptyMap(); + } + + @Override + public Map maximums() { + return emptyMap(); + } + + @Override + public Map meanStatistics() { + return emptyMap(); + } + + @Override + public boolean aggregate(@Nullable final IOStatistics statistics) { + return false; + } + + @Override + public long incrementCounter(final String key, final long value) { + return 0; + } + + @Override + public void setCounter(final String key, final long value) { + + } + + @Override + public void setGauge(final String key, final long value) { + + } + + @Override + public long incrementGauge(final String key, final long value) { + return 0; + } + + @Override + public void setMaximum(final String key, final long value) { + + } + + @Override + public long incrementMaximum(final String key, final long value) { + return 0; + } + + @Override + public void setMinimum(final String key, final long value) { + + } + + @Override + public long incrementMinimum(final String key, final long value) { + return 0; + } + + @Override + public void addMinimumSample(final String key, final long value) { + + } + + @Override + public void addMaximumSample(final String key, final long value) { + + } + + @Override + public void setMeanStatistic(final String key, final MeanStatistic value) { + + } + + @Override + public void addMeanStatisticSample(final String key, final long value) { + + } + + @Override + public void reset() { + + } + + @Override + public AtomicLong getCounterReference(final String key) { + return null; + } + + @Override + public AtomicLong getMaximumReference(final String key) { + return null; + } + + @Override + public AtomicLong getMinimumReference(final String key) { + return null; + } + + @Override + public AtomicLong getGaugeReference(final String key) { + return null; + } + + @Override + public MeanStatistic getMeanStatistic(final String key) { + return null; + } + + @Override + public void addTimedOperation(final String prefix, + final long durationMillis) { + + } + + @Override + public void addTimedOperation(final String prefix, final Duration duration) { + + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java index c3507dbc73ef4..1aece01e2fe31 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java @@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; import java.util.function.Function; +import java.util.function.Supplier; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; @@ -97,6 +98,15 @@ public static IOStatistics emptyStatistics() { return EmptyIOStatistics.getInstance(); } + /** + * Get the shared instance of the immutable empty statistics + * store. + * @return an empty statistics object. + */ + public static IOStatisticsStore emptyStatisticsStore() { + return EmptyIOStatisticsStore.getInstance(); + } + /** * Take an IOStatistics instance and wrap it in a source. * @param statistics statistics. @@ -573,6 +583,38 @@ public static Callable trackDurationOfCallable( }; } + /** + * Given a Java supplier, evaluate it while + * tracking the duration of the operation and success/failure. + * @param factory factory of duration trackers + * @param statistic statistic key + * @param input input callable. + * @param return type. + * @return the output of the supplier. + */ + public static B trackDurationOfSupplier( + @Nullable DurationTrackerFactory factory, + String statistic, + Supplier input) { + // create the tracker outside try-with-resources so + // that failures can be set in the catcher. + DurationTracker tracker = createTracker(factory, statistic); + try { + // exec the input function and return its value + return input.get(); + } catch (RuntimeException e) { + // input function failed: note it + tracker.failed(); + // and rethrow + throw e; + } finally { + // update the tracker. + // this is called after any catch() call will have + // set the failed flag. + tracker.close(); + } + } + /** * Create the tracker. If the factory is null, a stub * tracker is returned. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/LogExactlyOnce.java similarity index 81% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/LogExactlyOnce.java index 54a8836d02ba4..04cd5111e90a2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/LogExactlyOnce.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.impl; +package org.apache.hadoop.fs.store; import java.util.concurrent.atomic.AtomicBoolean; @@ -39,4 +39,14 @@ public void warn(String format, Object...args) { log.warn(format, args); } } + public void info(String format, Object...args) { + if (!logged.getAndSet(true)) { + log.info(format, args); + } + } + public void error(String format, Object...args) { + if (!logged.getAndSet(true)) { + log.error(format, args); + } + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/ActiveThreadSpanSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/ActiveThreadSpanSource.java new file mode 100644 index 0000000000000..4ddb8e1f29072 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/ActiveThreadSpanSource.java @@ -0,0 +1,38 @@ +/* + * 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.hadoop.fs.store.audit; + +/** + * Interface to get the active thread span. + * This can be used to collect the active span to + * propagate it into other threads. + * + * FileSystems which track their active span may implement + * this and offer their active span. + */ +public interface ActiveThreadSpanSource { + + /** + * The active span. This may not be a valid span, i.e. there is no guarantee + * that {@code getActiveAuditSpan().isValidSpan()} is true, but + * implementations MUST always return a non-null span. + * @return the currently active span. + */ + T getActiveAuditSpan(); +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditEntryPoint.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditEntryPoint.java new file mode 100644 index 0000000000000..6210dd0c8987c --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditEntryPoint.java @@ -0,0 +1,55 @@ +/* + * 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.hadoop.fs.store.audit; + +import java.lang.annotation.Documented; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * A marker attribute simply to highlight which of the methods + * in a FileSystem why are audit entry points. + *
    + *
  1. + * A FS method is an AuditEntryPoint if, on invocation it + * creates and activates an Audit Span for that FS. + *
  2. + *
  3. + * The audit span SHOULD be deactivated before returning, + *
  4. + *
  5. + * Objects returned by the API call which go on + * to make calls of the filesystem MUST perform + * all IO within the same audit span. + *
  6. + *
  7. + * Audit Entry points SHOULD NOT invoke other Audit Entry Points. + * This is to ensure the original audit span information + * is not replaced. + *
  8. + *
+ * FileSystem methods the entry point then invokes + * SHOULD NOT invoke audit entry points internally. + * + * All external methods MUST be audit entry points. + */ +@Documented +@Retention(RetentionPolicy.SOURCE) +public @interface AuditEntryPoint { +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditSpan.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditSpan.java new file mode 100644 index 0000000000000..ecdaf71c11132 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditSpan.java @@ -0,0 +1,109 @@ +/* + * 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.hadoop.fs.store.audit; + +import java.io.Closeable; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * This is a span created by an {@link AuditSpanSource}. + * An implementation of a span may carry context which can be picked + * up by the filesystem when activated. + * Each FS can have one active span per thread. + * Different filesystem instances SHALL have different active + * spans (if they support them) + * A span is activated in a thread when {@link #activate()} + * is called. + * The span stays active in that thread until {@link #deactivate()} + * is called. + * When deactivated in one thread, it MAY still be active in others. + * There's no explicit "end of span"; this is too hard to manage in + * terms of API lifecycle. + * Similarly, there's no stack of spans. Once a span is activated, + * the previous span is forgotten about. + * Therefore each FS will need a fallback "inactive span" which + * will be reverted to on deactivation of any other span. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public interface AuditSpan extends Closeable { + + /** + * Return a span ID which must be unique for all spans within + * everywhere. That effectively means part of the + * span SHOULD be derived from a UUID. + * Callers MUST NOT make any assumptions about the actual + * contents or structure of this string other than the + * uniqueness. + * @return a non-empty string + */ + String getSpanId(); + + /** + * Get the name of the operation. + * @return the operation name. + */ + String getOperationName(); + + /** + * Timestamp in UTC of span creation. + * @return timestamp. + */ + long getTimestamp(); + + /** + * Make this span active in the current thread. + * @return the activated span. + * This is makes it easy to use in try with resources + */ + AuditSpan activate(); + + /** + * Deactivate the span in the current thread. + */ + void deactivate(); + + /** + * Close calls {@link #deactivate()}; subclasses may override + * but the audit manager's wrapping span will always relay to + * {@link #deactivate()} rather + * than call this method on the wrapped span. + */ + default void close() { + deactivate(); + } + + /** + * Is the span valid? False == this is a span to indicate unbonded. + * @return true if this span represents a real operation. + */ + default boolean isValidSpan() { + return true; + } + + /** + * Set an attribute. + * This may or may not be propagated to audit logs. + * @param key attribute name + * @param value value + */ + default void set(String key, String value) { } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditSpanSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditSpanSource.java new file mode 100644 index 0000000000000..4f9f5a64564c0 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditSpanSource.java @@ -0,0 +1,50 @@ +/* + * 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.hadoop.fs.store.audit; + +import javax.annotation.Nullable; +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * A source of audit spans. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public interface AuditSpanSource { + + /** + * Create a span for an operation. + * + * All operation names SHOULD come from + * {@code StoreStatisticNames} or + * {@code StreamStatisticNames}. + * @param operation operation name. + * @param path1 first path of operation + * @param path2 second path of operation + * @return a span for the audit + * @throws IOException failure + */ + T createSpan(String operation, + @Nullable String path1, + @Nullable String path2) + throws IOException; +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditingFunctions.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditingFunctions.java new file mode 100644 index 0000000000000..acc82766be190 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditingFunctions.java @@ -0,0 +1,121 @@ +/* + * 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.hadoop.fs.store.audit; + +import javax.annotation.Nullable; +import java.util.concurrent.Callable; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.util.functional.CallableRaisingIOE; +import org.apache.hadoop.util.functional.FunctionRaisingIOE; +import org.apache.hadoop.util.functional.InvocationRaisingIOE; + +/** + * Static methods to assist in working with Audit Spans. + * the {@code withinX} calls take a span and a closure/function etc. + * and return a new function of the same types but which will + * activate and the span. + * They do not deactivate it afterwards to avoid accidentally deactivating + * the already-active span during a chain of operations in the same thread. + * All they do is ensure that the given span is guaranteed to be + * active when the passed in callable/function/invokable is evaluated. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class AuditingFunctions { + + private AuditingFunctions() { + } + + /** + * Given a callable, return a new callable which + * activates and deactivates the span around the inner invocation. + * @param auditSpan audit span + * @param operation operation + * @param type of result + * @return a new invocation. + */ + public static CallableRaisingIOE withinAuditSpan( + @Nullable AuditSpan auditSpan, + CallableRaisingIOE operation) { + return auditSpan == null + ? operation + : () -> { + auditSpan.activate(); + return operation.apply(); + }; + } + + /** + * Given an invocation, return a new invocation which + * activates and deactivates the span around the inner invocation. + * @param auditSpan audit span + * @param operation operation + * @return a new invocation. + */ + public static InvocationRaisingIOE withinAuditSpan( + @Nullable AuditSpan auditSpan, + InvocationRaisingIOE operation) { + return auditSpan == null + ? operation + : () -> { + auditSpan.activate(); + operation.apply(); + }; + } + + /** + * Given a function, return a new function which + * activates and deactivates the span around the inner one. + * @param auditSpan audit span + * @param operation operation + * @return a new invocation. + */ + public static FunctionRaisingIOE withinAuditSpan( + @Nullable AuditSpan auditSpan, + FunctionRaisingIOE operation) { + return auditSpan == null + ? operation + : (x) -> { + auditSpan.activate(); + return operation.apply(x); + }; + } + + /** + * Given a callable, return a new callable which + * activates and deactivates the span around the inner invocation. + * @param auditSpan audit span + * @param operation operation + * @param type of result + * @return a new invocation. + */ + public static Callable callableWithinAuditSpan( + @Nullable AuditSpan auditSpan, + Callable operation) { + return auditSpan == null + ? operation + : () -> { + auditSpan.activate(); + return operation.call(); + }; + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/HttpReferrerAuditHeader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/HttpReferrerAuditHeader.java new file mode 100644 index 0000000000000..3e841efa86435 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/HttpReferrerAuditHeader.java @@ -0,0 +1,501 @@ +/* + * 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.hadoop.fs.store.audit; + +import java.net.URI; +import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.StringJoiner; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.audit.CommonAuditContext; +import org.apache.hadoop.fs.store.LogExactlyOnce; +import org.apache.http.NameValuePair; +import org.apache.http.client.utils.URLEncodedUtils; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_ID; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_OP; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PATH; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PATH2; +import static org.apache.hadoop.fs.audit.AuditConstants.REFERRER_ORIGIN_HOST; + +/** + * Contains all the logic for generating an HTTP "Referer" + * entry; includes escaping query params. + * Tests for this are in + * {@code org.apache.hadoop.fs.s3a.audit.TestHttpReferrerAuditHeader} + * so as to verify that header generation in the S3A auditors, and + * S3 log parsing, all work. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class HttpReferrerAuditHeader { + + /** + * Format of path to build: {@value}. + * the params passed in are (context ID, span ID, op) + */ + public static final String REFERRER_PATH_FORMAT = "hadoop/1/%3$s/%2$s/"; + + private static final Logger LOG = + LoggerFactory.getLogger(HttpReferrerAuditHeader.class); + + /** + * Log for warning of problems creating headers will only log of + * a problem once per process instance. + * This is to avoid logs being flooded with errors. + */ + private static final LogExactlyOnce WARN_OF_URL_CREATION = + new LogExactlyOnce(LOG); + + /** Context ID. */ + private final String contextId; + + /** operation name. */ + private final String operationName; + + /** Span ID. */ + private final String spanId; + + /** optional first path. */ + private final String path1; + + /** optional second path. */ + private final String path2; + + /** + * The header as created in the constructor; used in toString(). + * A new header is built on demand in {@link #buildHttpReferrer()} + * so that evaluated attributes are dynamically evaluated + * in the correct thread/place. + */ + private final String initialHeader; + + /** + * Map of simple attributes. + */ + private final Map attributes; + + /** + * Parameters dynamically evaluated on the thread just before + * the request is made. + */ + private final Map> evaluated; + + /** + * Elements to filter from the final header. + */ + private final Set filter; + + /** + * Instantiate. + * + * Context and operationId are expected to be well formed + * numeric/hex strings, at least adequate to be + * used as individual path elements in a URL. + */ + private HttpReferrerAuditHeader( + final Builder builder) { + this.contextId = requireNonNull(builder.contextId); + this.evaluated = builder.evaluated; + this.filter = builder.filter; + this.operationName = requireNonNull(builder.operationName); + this.path1 = builder.path1; + this.path2 = builder.path2; + this.spanId = requireNonNull(builder.spanId); + + // copy the parameters from the builder and extend + attributes = builder.attributes; + + addAttribute(PARAM_OP, operationName); + addAttribute(PARAM_PATH, path1); + addAttribute(PARAM_PATH2, path2); + addAttribute(PARAM_ID, spanId); + + // patch in global context values where not set + Iterable> globalContextValues + = builder.globalContextValues; + if (globalContextValues != null) { + for (Map.Entry entry : globalContextValues) { + attributes.putIfAbsent(entry.getKey(), entry.getValue()); + } + } + + // build the referrer up. so as to find/report problems early + initialHeader = buildHttpReferrer(); + } + + /** + * Build the referrer string. + * This includes dynamically evaluating all of the evaluated + * attributes. + * If there is an error creating the string it will be logged once + * per entry, and "" returned. + * @return a referrer string or "" + */ + public String buildHttpReferrer() { + + String header; + try { + String queries; + // Update any params which are dynamically evaluated + evaluated.forEach((key, eval) -> + addAttribute(key, eval.get())); + // now build the query parameters from all attributes, static and + // evaluated, stripping out any from the filter + queries = attributes.entrySet().stream() + .filter(e -> !filter.contains(e.getKey())) + .map(e -> e.getKey() + "=" + e.getValue()) + .collect(Collectors.joining("&")); + final URI uri = new URI("https", REFERRER_ORIGIN_HOST, + String.format(Locale.ENGLISH, REFERRER_PATH_FORMAT, + contextId, spanId, operationName), + queries, + null); + header = uri.toASCIIString(); + } catch (URISyntaxException e) { + WARN_OF_URL_CREATION.warn("Failed to build URI for {}/{}", e); + header = ""; + } + return header; + } + + /** + * Add a query parameter if not null/empty + * There's no need to escape here as it is done in the URI + * constructor. + * @param key query key + * @param value query value + */ + private void addAttribute(String key, + String value) { + if (StringUtils.isNotEmpty(value)) { + attributes.put(key, value); + } + } + + /** + * Set an attribute. If the value is non-null/empty, + * it will be used as a query parameter. + * + * @param key key to set + * @param value value. + */ + public void set(final String key, final String value) { + addAttribute(requireNonNull(key), value); + } + + public String getContextId() { + return contextId; + } + + public String getOperationName() { + return operationName; + } + + public String getSpanId() { + return spanId; + } + + public String getPath1() { + return path1; + } + + public String getPath2() { + return path2; + } + + @Override + public String toString() { + return new StringJoiner(", ", + HttpReferrerAuditHeader.class.getSimpleName() + "[", "]") + .add(initialHeader) + .toString(); + } + + /** + * Perform any escaping to valid path elements in advance of + * new URI() doing this itself. Only path separators need to + * be escaped/converted at this point. + * @param source source string + * @return an escaped path element. + */ + public static String escapeToPathElement(CharSequence source) { + int len = source.length(); + StringBuilder r = new StringBuilder(len); + for (int i = 0; i < len; i++) { + char c = source.charAt(i); + String s = Character.toString(c); + switch (c) { + case '/': + case '@': + s = "+"; + break; + default: + break; + } + r.append(s); + } + return r.toString(); + + } + + /** + * Strip any quotes from around a header. + * This is needed when processing log entries. + * @param header field. + * @return field without quotes. + */ + public static String maybeStripWrappedQuotes(String header) { + String h = header; + // remove quotes if needed. + while (h.startsWith("\"")) { + h = h.substring(1); + } + while (h.endsWith("\"")) { + h = h.substring(0, h.length() - 1); + } + return h; + } + + /** + * Split up the string. Uses httpClient: make sure it is on the classpath. + * Any query param with a name but no value, e.g ?something is + * returned in the map with an empty string as the value. + * @param header URI to parse + * @return a map of parameters. + * @throws URISyntaxException failure to build URI from header. + */ + public static Map extractQueryParameters(String header) + throws URISyntaxException { + URI uri = new URI(maybeStripWrappedQuotes(header)); + // get the decoded query + List params = URLEncodedUtils.parse(uri, + StandardCharsets.UTF_8); + Map result = new HashMap<>(params.size()); + for (NameValuePair param : params) { + String name = param.getName(); + String value = param.getValue(); + if (value == null) { + value = ""; + } + result.put(name, value); + } + return result; + } + + /** + * Get a builder. + * @return a new builder. + */ + public static Builder builder() { + return new Builder(); + } + + /** + * Builder. + * + * Context and operationId are expected to be well formed + * numeric/hex strings, at least adequate to be + * used as individual path elements in a URL. + */ + public static final class Builder { + + /** Context ID. */ + private String contextId; + + /** operation name. */ + private String operationName; + + /** operation ID. */ + private String spanId; + + /** optional first path. */ + private String path1; + + /** optional second path. */ + private String path2; + + /** Map of attributes to add as query parameters. */ + private final Map attributes = new HashMap<>(); + + /** + * Parameters dynamically evaluated on the thread just before + * the request is made. + */ + private final Map> evaluated = + new HashMap<>(); + + /** + * Global context values; defaults to that of + * {@link CommonAuditContext#getGlobalContextEntries()} and + * should not need to be changed. + */ + private Iterable> globalContextValues = + CommonAuditContext.getGlobalContextEntries(); + + /** + * Elements to filter from the final header. + */ + private Set filter = new HashSet<>(); + + private Builder() { + } + + /** + * Build. + * @return an HttpReferrerAuditHeader + */ + public HttpReferrerAuditHeader build() { + return new HttpReferrerAuditHeader(this); + } + + /** + * Set context ID. + * @param value context + * @return the builder + */ + public Builder withContextId(final String value) { + contextId = value; + return this; + } + + /** + * Set Operation name. + * @param value new value + * @return the builder + */ + public Builder withOperationName(final String value) { + operationName = value; + return this; + } + + /** + * Set ID. + * @param value new value + * @return the builder + */ + public Builder withSpanId(final String value) { + spanId = value; + return this; + } + + /** + * Set Path1 of operation. + * @param value new value + * @return the builder + */ + public Builder withPath1(final String value) { + path1 = value; + return this; + } + + /** + * Set Path2 of operation. + * @param value new value + * @return the builder + */ + public Builder withPath2(final String value) { + path2 = value; + return this; + } + + /** + * Add all attributes to the current map. + * @param value new value + * @return the builder + */ + public Builder withAttributes(final Map value) { + attributes.putAll(value); + return this; + } + + /** + * Add an attribute to the current map. + * Replaces any with the existing key. + * @param key key to set/update + * @param value new value + * @return the builder + */ + public Builder withAttribute(String key, String value) { + attributes.put(key, value); + return this; + } + + /** + * Add all evaluated attributes to the current map. + * @param value new value + * @return the builder + */ + public Builder withEvaluated(final Map> value) { + evaluated.putAll(value); + return this; + } + + /** + * Add an evaluated attribute to the current map. + * Replaces any with the existing key. + * Set evaluated methods. + * @param key key + * @param value new value + * @return the builder + */ + public Builder withEvaluated(String key, Supplier value) { + evaluated.put(key, value); + return this; + } + + /** + * Set the global context values (replaces the default binding + * to {@link CommonAuditContext#getGlobalContextEntries()}). + * @param value new value + * @return the builder + */ + public Builder withGlobalContextValues( + final Iterable> value) { + globalContextValues = value; + return this; + } + + /** + * Declare the fields to filter. + * @param fields iterable of field names. + * @return the builder + */ + public Builder withFilter(final Collection fields) { + this.filter = new HashSet<>(fields); + return this; + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/package-info.java new file mode 100644 index 0000000000000..98fb5b59c3ac0 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/package-info.java @@ -0,0 +1,29 @@ +/* + * 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. + */ + +/** + * Auditing classes for internal + * use within the hadoop-* modules only. No stability guarantees. + * The public/evolving API is in {@code org.apache.hadoop.fs.audit}. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.store.audit; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java index 47aabed89cdf0..70e093e1e4870 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java @@ -36,6 +36,7 @@ import org.apache.commons.cli.OptionBuilder; import org.apache.commons.cli.Options; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.audit.CommonAuditContext; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.service.Service; import org.apache.hadoop.util.ExitCodeProvider; @@ -591,6 +592,7 @@ protected int coreServiceLaunch(Configuration conf, } String name = getServiceName(); LOG.debug("Launched service {}", name); + CommonAuditContext.noteEntryPoint(service); LaunchableService launchableService = null; if (service instanceof LaunchableService) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ToolRunner.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ToolRunner.java index 8740be49d97bc..336700a6e276a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ToolRunner.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ToolRunner.java @@ -23,6 +23,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.audit.CommonAuditContext; import org.apache.hadoop.ipc.CallerContext; /** @@ -63,6 +64,10 @@ public static int run(Configuration conf, Tool tool, String[] args) CallerContext ctx = new CallerContext.Builder("CLI").build(); CallerContext.setCurrent(ctx); } + // Note the entry point in the audit context; this + // may be used in audit events set to cloud store logs + // or elsewhere. + CommonAuditContext.noteEntryPoint(tool); if(conf == null) { conf = new Configuration(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java index 3ac0fced1493d..5fdea4f5b747a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java @@ -189,6 +189,7 @@ public static RemoteIterator closingRemoteIterator( /** * Build a list from a RemoteIterator. + * @param source source iterator * @param type * @return a list of the values. * @throws IOException if the source RemoteIterator raises it. @@ -202,12 +203,17 @@ public static List toList(RemoteIterator source) /** * Build an array from a RemoteIterator. + * @param source source iterator + * @param a destination array; if too small a new array + * of the same type is created * @param type * @return an array of the values. * @throws IOException if the source RemoteIterator raises it. */ - public static T[] toArray(RemoteIterator source) throws IOException { - return (T[]) toList(source).toArray(); + public static T[] toArray(RemoteIterator source, + T[] a) throws IOException { + List list = toList(source); + return list.toArray(a); } /** @@ -240,18 +246,28 @@ public static long foreach( consumer.accept(source.next()); } - // maybe log the results - logIOStatisticsAtDebug(LOG, "RemoteIterator Statistics: {}", source); } finally { - if (source instanceof Closeable) { - // source is closeable, so close. - IOUtils.cleanupWithLogger(LOG, (Closeable) source); - } + cleanupRemoteIterator(source); } - return count; } + /** + * Clean up after an iteration. + * If the log is at debug, calculate and log the IOStatistics. + * If the iterator is closeable, cast and then cleanup the iterator + * @param source iterator source + * @param type of source + */ + public static void cleanupRemoteIterator(RemoteIterator source) { + // maybe log the results + logIOStatisticsAtDebug(LOG, "RemoteIterator Statistics: {}", source); + if (source instanceof Closeable) { + /* source is closeable, so close.*/ + IOUtils.cleanupWithLogger(LOG, (Closeable) source); + } + } + /** * A remote iterator from a singleton. It has a single next() * value, after which hasNext() returns false and next() fails. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/audit/TestCommonAuditContext.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/audit/TestCommonAuditContext.java new file mode 100644 index 0000000000000..798841a2d6905 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/audit/TestCommonAuditContext.java @@ -0,0 +1,161 @@ +/* + * 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.hadoop.fs.audit; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import org.assertj.core.api.AbstractStringAssert; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_COMMAND; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PROCESS; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD1; +import static org.apache.hadoop.fs.audit.CommonAuditContext.PROCESS_ID; +import static org.apache.hadoop.fs.audit.CommonAuditContext.removeGlobalContextEntry; +import static org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext; +import static org.apache.hadoop.fs.audit.CommonAuditContext.getGlobalContextEntry; +import static org.apache.hadoop.fs.audit.CommonAuditContext.getGlobalContextEntries; +import static org.apache.hadoop.fs.audit.CommonAuditContext.noteEntryPoint; +import static org.apache.hadoop.fs.audit.CommonAuditContext.setGlobalContextEntry; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Tests of the common audit context. + */ +public class TestCommonAuditContext extends AbstractHadoopTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TestCommonAuditContext.class); + + private final CommonAuditContext context = currentAuditContext(); + /** + * We can set, get and enumerate global context values. + */ + @Test + public void testGlobalSetGetEnum() throws Throwable { + + String s = "command"; + setGlobalContextEntry(PARAM_COMMAND, s); + assertGlobalEntry(PARAM_COMMAND) + .isEqualTo(s); + // and the iterators. + List> list = StreamSupport + .stream(getGlobalContextEntries().spliterator(), + false) + .filter(e -> e.getKey().equals(PARAM_COMMAND)) + .collect(Collectors.toList()); + assertThat(list) + .hasSize(1) + .allMatch(e -> e.getValue().equals(s)); + } + + @Test + public void testVerifyProcessID() throws Throwable { + assertThat( + getGlobalContextEntry(PARAM_PROCESS)) + .describedAs("global context value of %s", PARAM_PROCESS) + .isEqualTo(PROCESS_ID); + } + + + @Test + public void testNullValue() throws Throwable { + assertThat(context.get(PARAM_PROCESS)) + .describedAs("Value of context element %s", PARAM_PROCESS) + .isNull(); + } + + @Test + public void testThreadId() throws Throwable { + String t1 = getContextValue(PARAM_THREAD1); + Long tid = Long.valueOf(t1); + assertThat(tid).describedAs("thread ID") + .isEqualTo(Thread.currentThread().getId()); + } + + /** + * Verify functions are dynamically evaluated. + */ + @Test + public void testDynamicEval() throws Throwable { + context.reset(); + final AtomicBoolean ab = new AtomicBoolean(false); + context.put("key", () -> + Boolean.toString(ab.get())); + assertContextValue("key") + .isEqualTo("false"); + // update the reference and the next get call will + // pick up the new value. + ab.set(true); + assertContextValue("key") + .isEqualTo("true"); + } + + private String getContextValue(final String key) { + String val = context.get(key); + assertThat(val).isNotBlank(); + return val; + } + + /** + * Start an assertion on a context value. + * @param key key to look up + * @return an assert which can be extended call + */ + private AbstractStringAssert assertContextValue(final String key) { + String val = context.get(key); + return assertThat(val) + .describedAs("Value of context element %s", key) + .isNotBlank(); + } + + @Test + public void testNoteEntryPoint() throws Throwable { + setAndAssertEntryPoint(this).isEqualTo("TestCommonAuditContext"); + + } + + @Test + public void testNoteNullEntryPoint() throws Throwable { + setAndAssertEntryPoint(null).isNull(); + } + + private AbstractStringAssert setAndAssertEntryPoint(final Object tool) { + removeGlobalContextEntry(PARAM_COMMAND); + noteEntryPoint(tool); + AbstractStringAssert anAssert = assertGlobalEntry( + PARAM_COMMAND); + return anAssert; + } + + private AbstractStringAssert assertGlobalEntry(final String key) { + AbstractStringAssert anAssert = assertThat(getGlobalContextEntry(key)) + .describedAs("Global context value %s", key); + return anAssert; + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java index 35193fa2dc712..e13a49ca10e70 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java @@ -399,9 +399,7 @@ public static boolean rm(FileSystem fileSystem, IOException { if (fileSystem != null) { rejectRootOperation(path, allowRootDelete); - if (fileSystem.exists(path)) { - return fileSystem.delete(path, recursive); - } + return fileSystem.delete(path, recursive); } return false; @@ -728,8 +726,10 @@ public static void assertDeleted(FileSystem fs, assertPathExists(fs, "about to be deleted file", file); } boolean deleted = fs.delete(file, recursive); - String dir = ls(fs, file.getParent()); - assertTrue("Delete failed on " + file + ": " + dir, deleted); + if (!deleted) { + String dir = ls(fs, file.getParent()); + assertTrue("Delete failed on " + file + ": " + dir, deleted); + } assertPathDoesNotExist(fs, "Deleted file", file); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java index 46655bc99e6fe..2f0cfd37ad37c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java @@ -20,12 +20,21 @@ import com.amazonaws.AmazonClientException; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + /** * Exception which Hadoop's AWSCredentialsProvider implementations should * throw when there is a problem with the credential setup. This * is a subclass of {@link AmazonClientException} which sets * {@link #isRetryable()} to false, so as to fail fast. + * This is used in credential providers and elsewhere. + * When passed through {@code S3AUtils.translateException()} it + * is mapped to an AccessDeniedException. As a result, the Invoker + * code will automatically translate */ +@InterfaceAudience.Public +@InterfaceStability.Stable public class CredentialInitializationException extends AmazonClientException { public CredentialInitializationException(String message, Throwable t) { super(message, t); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java index 19cd6c985b531..a851f0fbac687 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java @@ -34,6 +34,7 @@ import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.functional.CallableRaisingIOE; +import org.apache.hadoop.util.functional.InvocationRaisingIOE; /** * Class to provide lambda expression invocation of AWS operations. @@ -43,7 +44,8 @@ * the other {@code retry() and retryUntranslated()} calls are wrappers. * * The static {@link #once(String, String, CallableRaisingIOE)} and - * {@link #once(String, String, VoidOperation)} calls take an operation and + * {@link #once(String, String, InvocationRaisingIOE)} calls take an + * operation and * return it with AWS exceptions translated to IOEs of some form. * * The retry logic on a failure is defined by the retry policy passed in @@ -57,7 +59,7 @@ * but before the sleep. * These callbacks can be used for reporting and incrementing statistics. * - * The static {@link #quietly(String, String, VoidOperation)} and + * The static {@link #quietly(String, String, InvocationRaisingIOE)} and * {@link #quietlyEval(String, String, CallableRaisingIOE)} calls exist to * take any operation and quietly catch and log at debug. * The return value of {@link #quietlyEval(String, String, CallableRaisingIOE)} @@ -126,11 +128,11 @@ public static T once(String action, String path, * @throws IOException any IOE raised, or translated exception */ @Retries.OnceTranslated - public static void once(String action, String path, VoidOperation operation) - throws IOException { + public static void once(String action, String path, + InvocationRaisingIOE operation) throws IOException { once(action, path, () -> { - operation.execute(); + operation.apply(); return null; }); } @@ -171,10 +173,10 @@ public static void ignoreIOExceptions( Logger log, String action, String path, - VoidOperation operation) { + InvocationRaisingIOE operation) { ignoreIOExceptions(log, action, path, () -> { - operation.execute(); + operation.apply(); return null; }); } @@ -194,11 +196,11 @@ public void retry(String action, String path, boolean idempotent, Retried retrying, - VoidOperation operation) + InvocationRaisingIOE operation) throws IOException { retry(action, path, idempotent, retrying, () -> { - operation.execute(); + operation.apply(); return null; }); } @@ -221,11 +223,11 @@ public void maybeRetry(boolean doRetry, String path, boolean idempotent, Retried retrying, - VoidOperation operation) + InvocationRaisingIOE operation) throws IOException { maybeRetry(doRetry, action, path, idempotent, retrying, () -> { - operation.execute(); + operation.apply(); return null; }); } @@ -243,7 +245,7 @@ public void maybeRetry(boolean doRetry, public void retry(String action, String path, boolean idempotent, - VoidOperation operation) + InvocationRaisingIOE operation) throws IOException { retry(action, path, idempotent, retryCallback, operation); } @@ -265,7 +267,7 @@ public void maybeRetry( String action, String path, boolean idempotent, - VoidOperation operation) + InvocationRaisingIOE operation) throws IOException { maybeRetry(doRetry, action, path, idempotent, retryCallback, operation); } @@ -475,7 +477,7 @@ public T retryUntranslated( */ public static void quietly(String action, String path, - VoidOperation operation) { + InvocationRaisingIOE operation) { try { once(action, path, operation); } catch (Exception e) { @@ -515,14 +517,6 @@ private static String toDescription(String action, @Nullable String path) { (StringUtils.isNotEmpty(path) ? (" on " + path) : ""); } - /** - * Void operation which may raise an IOException. - */ - @FunctionalInterface - public interface VoidOperation { - void execute() throws IOException; - } - /** * Callback for retry and notification operations. * Even if the interface is throwing up "raw" exceptions, this handler diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java index 3cb3d5d832df3..113e6f4de2abd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java @@ -22,6 +22,7 @@ import com.amazonaws.AmazonClientException; import com.amazonaws.services.s3.model.S3ObjectSummary; + import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.commons.lang3.tuple.Triple; @@ -41,10 +42,12 @@ import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.util.functional.RemoteIterators; import org.slf4j.Logger; +import java.io.Closeable; import java.io.FileNotFoundException; import java.io.IOException; import java.time.Instant; @@ -79,6 +82,9 @@ /** * Place for the S3A listing classes; keeps all the small classes under control. + * + * Spans passed in are attached to the listing iterators returned, but are not + * closed at the end of the iteration. This is because the same span */ @InterfaceAudience.Private public class Listing extends AbstractStoreOperation { @@ -137,16 +143,19 @@ public static RemoteIterator toProvidedFileStatusIterator( * @param filter the filter on which paths to accept * @param acceptor the class/predicate to decide which entries to accept * in the listing based on the full file status. + * @param span audit span for this iterator * @return the iterator * @throws IOException IO Problems */ + @Retries.RetryRaw public FileStatusListingIterator createFileStatusListingIterator( Path listPath, S3ListRequest request, PathFilter filter, - Listing.FileStatusAcceptor acceptor) throws IOException { + Listing.FileStatusAcceptor acceptor, + AuditSpan span) throws IOException { return createFileStatusListingIterator(listPath, request, filter, acceptor, - null); + null, span); } /** @@ -159,6 +168,7 @@ public FileStatusListingIterator createFileStatusListingIterator( * in the listing based on the full file status. * @param providedStatus the provided list of file status, which may contain * items that are not listed from source. + * @param span audit span for this iterator * @return the iterator * @throws IOException IO Problems */ @@ -168,9 +178,10 @@ public FileStatusListingIterator createFileStatusListingIterator( S3ListRequest request, PathFilter filter, Listing.FileStatusAcceptor acceptor, - RemoteIterator providedStatus) throws IOException { + RemoteIterator providedStatus, + AuditSpan span) throws IOException { return new FileStatusListingIterator( - createObjectListingIterator(listPath, request), + createObjectListingIterator(listPath, request, span), filter, acceptor, providedStatus); @@ -181,14 +192,16 @@ public FileStatusListingIterator createFileStatusListingIterator( * list object request. * @param listPath path of the listing * @param request initial request to make + * @param span audit span for this iterator * @return the iterator * @throws IOException IO Problems */ @Retries.RetryRaw - public ObjectListingIterator createObjectListingIterator( + private ObjectListingIterator createObjectListingIterator( final Path listPath, - final S3ListRequest request) throws IOException { - return new ObjectListingIterator(listPath, request); + final S3ListRequest request, + final AuditSpan span) throws IOException { + return new ObjectListingIterator(listPath, request, span); } /** @@ -245,6 +258,7 @@ public RemoteIterator createSingleStatusIterator( * @param forceNonAuthoritativeMS forces metadata store to act like non * authoritative. This is useful when * listFiles output is used by import tool. + * @param span audit span for this iterator * @return an iterator over listing. * @throws IOException any exception. */ @@ -252,7 +266,8 @@ public RemoteIterator getListFilesAssumingDir( Path path, boolean recursive, Listing.FileStatusAcceptor acceptor, boolean collectTombstones, - boolean forceNonAuthoritativeMS) throws IOException { + boolean forceNonAuthoritativeMS, + AuditSpan span) throws IOException { String key = maybeAddTrailingSlash(pathToKey(path)); String delimiter = recursive ? null : "/"; @@ -325,10 +340,13 @@ public RemoteIterator getListFilesAssumingDir( createLocatedFileStatusIterator( createFileStatusListingIterator(path, listingOperationCallbacks - .createListObjectsRequest(key, delimiter), + .createListObjectsRequest(key, + delimiter, + span), ACCEPT_ALL, acceptor, - cachedFilesIterator)), + cachedFilesIterator, + span)), collectTombstones ? tombstones : null); } @@ -337,11 +355,13 @@ public RemoteIterator getListFilesAssumingDir( * Also performing tombstone reconciliation for guarded directories. * @param dir directory to check. * @param filter a path filter. + * @param span audit span for this iterator * @return an iterator that traverses statuses of the given dir. * @throws IOException in case of failure. */ public RemoteIterator getLocatedFileStatusIteratorForDir( - Path dir, PathFilter filter) throws IOException { + Path dir, PathFilter filter, AuditSpan span) throws IOException { + span.activate(); final String key = maybeAddTrailingSlash(pathToKey(dir)); final Listing.FileStatusAcceptor acceptor = new Listing.AcceptAllButSelfAndS3nDirs(dir); @@ -353,39 +373,55 @@ public RemoteIterator getLocatedFileStatusIteratorForDir( listingOperationCallbacks .getUpdatedTtlTimeProvider(), allowAuthoritative); - Set tombstones = meta != null - ? meta.listTombstones() - : null; - final RemoteIterator cachedFileStatusIterator = - createProvidedFileStatusIterator( - S3Guard.dirMetaToStatuses(meta), filter, acceptor); - return (allowAuthoritative && meta != null - && meta.isAuthoritative()) - ? createLocatedFileStatusIterator( - cachedFileStatusIterator) - : createTombstoneReconcilingIterator( + if (meta != null) { + // there's metadata + // convert to an iterator + final RemoteIterator cachedFileStatusIterator = + createProvidedFileStatusIterator( + S3Guard.dirMetaToStatuses(meta), filter, acceptor); + + // if the dir is authoritative and the data considers itself + // to be authorititative. + if (allowAuthoritative && meta.isAuthoritative()) { + // return the list + return createLocatedFileStatusIterator(cachedFileStatusIterator); + } else { + // merge the datasets + return createTombstoneReconcilingIterator( createLocatedFileStatusIterator( - createFileStatusListingIterator(dir, - listingOperationCallbacks - .createListObjectsRequest(key, "/"), - filter, - acceptor, - cachedFileStatusIterator)), - tombstones); + createFileStatusListingIterator(dir, + listingOperationCallbacks + .createListObjectsRequest(key, "/", span), + filter, + acceptor, + cachedFileStatusIterator, + span)), + meta.listTombstones()); + } + } else { + // Unguarded + return createLocatedFileStatusIterator( + createFileStatusListingIterator(dir, + listingOperationCallbacks + .createListObjectsRequest(key, "/", span), + filter, + acceptor, + span)); + } } /** * Calculate list of file statuses assuming path * to be a non-empty directory. * @param path input path. + * @param span audit span for this iterator * @return Triple of file statuses, metaData, auth flag. * @throws IOException Any IO problems. */ public Triple, DirListingMetadata, Boolean> - getFileStatusesAssumingNonEmptyDir(Path path) + getFileStatusesAssumingNonEmptyDir(Path path, final AuditSpan span) throws IOException { String key = pathToKey(path); - List result; if (!key.isEmpty()) { key = key + '/'; } @@ -408,14 +444,15 @@ public RemoteIterator getLocatedFileStatusIteratorForDir( dirMeta, Boolean.TRUE); } - S3ListRequest request = createListObjectsRequest(key, "/"); + S3ListRequest request = createListObjectsRequest(key, "/", span); LOG.debug("listStatus: doing listObjects for directory {}", key); FileStatusListingIterator filesItr = createFileStatusListingIterator( path, request, ACCEPT_ALL, - new Listing.AcceptAllButSelfAndS3nDirs(path)); + new Listing.AcceptAllButSelfAndS3nDirs(path), + span); // return the results obtained from s3. return Triple.of( @@ -424,8 +461,11 @@ public RemoteIterator getLocatedFileStatusIteratorForDir( Boolean.FALSE); } - public S3ListRequest createListObjectsRequest(String key, String delimiter) { - return listingOperationCallbacks.createListObjectsRequest(key, delimiter); + public S3ListRequest createListObjectsRequest(String key, + String delimiter, + final AuditSpan span) { + return listingOperationCallbacks.createListObjectsRequest(key, delimiter, + span); } /** @@ -730,11 +770,13 @@ public String toString() { * Thread safety: none. */ class ObjectListingIterator implements RemoteIterator, - IOStatisticsSource { + IOStatisticsSource, Closeable { /** The path listed. */ private final Path listPath; + private final AuditSpan span; + /** The most recent listing results. */ private S3ListResult objects; @@ -772,12 +814,14 @@ class ObjectListingIterator implements RemoteIterator, * initial set of results/fail if there was a problem talking to the bucket. * @param listPath path of the listing * @param request initial request to make + * @param span audit span for this iterator. * @throws IOException if listObjects raises one. */ @Retries.RetryRaw ObjectListingIterator( Path listPath, - S3ListRequest request) throws IOException { + S3ListRequest request, + AuditSpan span) throws IOException { this.listPath = listPath; this.maxKeys = listingOperationCallbacks.getMaxKeys(); this.request = request; @@ -786,8 +830,9 @@ class ObjectListingIterator implements RemoteIterator, .withDurationTracking(OBJECT_LIST_REQUEST) .withDurationTracking(OBJECT_CONTINUE_LIST_REQUEST) .build(); + this.span = span; this.s3ListResultFuture = listingOperationCallbacks - .listObjectsAsync(request, iostats); + .listObjectsAsync(request, iostats, span); } /** @@ -851,7 +896,7 @@ private void fetchNextBatchAsyncIfPresent() throws IOException { LOG.debug("[{}], Requesting next {} objects under {}", listingCount, maxKeys, listPath); s3ListResultFuture = listingOperationCallbacks - .continueListObjectsAsync(request, objects, iostats); + .continueListObjectsAsync(request, objects, iostats, span); } } @@ -883,6 +928,14 @@ public Path getListPath() { public int getListingCount() { return listingCount; } + + /** + * Close, if actually called, will close the span + * this listing was created with. + */ + @Override + public void close() { + } } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java index 6eb490f2df482..d8c820cd8a121 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java @@ -31,11 +31,20 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.store.audit.AuditSpan; + +import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_LIST; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation; /** * MultipartUtils upload-specific functions for use by S3AFileSystem and Hadoop * CLI. + * The Audit span active when + * {@link #listMultipartUploads(StoreContext, AmazonS3, String, int)} + * was invoked is retained for all subsequent operations. */ public final class MultipartUtils { @@ -48,33 +57,47 @@ private MultipartUtils() { } /** * List outstanding multipart uploads. * Package private: S3AFileSystem and tests are the users of this. + * + * @param storeContext store context * @param s3 AmazonS3 client to use. - * @param bucketName name of S3 bucket to use. - * @param maxKeys maximum batch size to request at a time from S3. * @param prefix optional key prefix to narrow search. If null then whole * bucket will be searched. + * @param maxKeys maximum batch size to request at a time from S3. * @return an iterator of matching uploads */ - static MultipartUtils.UploadIterator listMultipartUploads(AmazonS3 s3, - Invoker invoker, String bucketName, int maxKeys, @Nullable String prefix) + static MultipartUtils.UploadIterator listMultipartUploads( + final StoreContext storeContext, + AmazonS3 s3, + @Nullable String prefix, + int maxKeys) throws IOException { - return new MultipartUtils.UploadIterator(s3, invoker, bucketName, maxKeys, + return new MultipartUtils.UploadIterator(storeContext, + s3, + maxKeys, prefix); } /** * Simple RemoteIterator wrapper for AWS `listMultipartUpload` API. * Iterates over batches of multipart upload metadata listings. + * All requests are in the StoreContext's active span + * at the time the iterator was constructed. */ static class ListingIterator implements RemoteIterator { - private final String bucketName; private final String prefix; + + private final RequestFactory requestFactory; + private final int maxKeys; private final AmazonS3 s3; private final Invoker invoker; + private final AuditSpan auditSpan; + + private final StoreContext storeContext; + /** * Most recent listing results. */ @@ -85,16 +108,24 @@ static class ListingIterator implements */ private boolean firstListing = true; - private int listCount = 1; + /** + * Count of list calls made. + */ + private int listCount = 0; - ListingIterator(AmazonS3 s3, Invoker invoker, String bucketName, - int maxKeys, @Nullable String prefix) throws IOException { + ListingIterator(final StoreContext storeContext, + AmazonS3 s3, + @Nullable String prefix, + int maxKeys) throws IOException { + this.storeContext = storeContext; this.s3 = s3; - this.bucketName = bucketName; + this.requestFactory = storeContext.getRequestFactory(); this.maxKeys = maxKeys; this.prefix = prefix; - this.invoker = invoker; + this.invoker = storeContext.getInvoker(); + this.auditSpan = storeContext.getActiveAuditSpan(); + // request the first listing. requestNextBatch(); } @@ -138,31 +169,36 @@ public MultipartUploadListing next() throws IOException { @Override public String toString() { - return "Upload iterator: prefix " + prefix + "; list count " + - listCount + "; isTruncated=" + listing.isTruncated(); + return "Upload iterator: prefix " + prefix + + "; list count " + listCount + + "; upload count " + listing.getMultipartUploads().size() + + "; isTruncated=" + listing.isTruncated(); } @Retries.RetryTranslated private void requestNextBatch() throws IOException { - ListMultipartUploadsRequest req = - new ListMultipartUploadsRequest(bucketName); - if (prefix != null) { - req.setPrefix(prefix); - } - if (!firstListing) { - req.setKeyMarker(listing.getNextKeyMarker()); - req.setUploadIdMarker(listing.getNextUploadIdMarker()); - } - req.setMaxUploads(listCount); + try (AuditSpan span = auditSpan.activate()) { + ListMultipartUploadsRequest req = requestFactory + .newListMultipartUploadsRequest(prefix); + if (!firstListing) { + req.setKeyMarker(listing.getNextKeyMarker()); + req.setUploadIdMarker(listing.getNextUploadIdMarker()); + } + req.setMaxUploads(maxKeys); - LOG.debug("[{}], Requesting next {} uploads prefix {}, " + - "next key {}, next upload id {}", listCount, maxKeys, prefix, - req.getKeyMarker(), req.getUploadIdMarker()); - listCount++; + LOG.debug("[{}], Requesting next {} uploads prefix {}, " + + "next key {}, next upload id {}", listCount, maxKeys, prefix, + req.getKeyMarker(), req.getUploadIdMarker()); + listCount++; - listing = invoker.retry("listMultipartUploads", prefix, true, - () -> s3.listMultipartUploads(req)); - LOG.debug("New listing state: {}", this); + listing = invoker.retry("listMultipartUploads", prefix, true, + trackDurationOfOperation(storeContext.getInstrumentation(), + MULTIPART_UPLOAD_LIST.getSymbol(), + () -> s3.listMultipartUploads(req))); + LOG.debug("Listing found {} upload(s)", + listing.getMultipartUploads().size()); + LOG.debug("New listing state: {}", this); + } } } @@ -174,6 +210,10 @@ private void requestNextBatch() throws IOException { public static class UploadIterator implements RemoteIterator { + /** + * Iterator for issuing new upload list requests from + * where the previous one ended. + */ private ListingIterator lister; /** Current listing: the last upload listing we fetched. */ private MultipartUploadListing listing; @@ -181,11 +221,15 @@ public static class UploadIterator private ListIterator batchIterator; @Retries.RetryTranslated - public UploadIterator(AmazonS3 s3, Invoker invoker, String bucketName, - int maxKeys, @Nullable String prefix) + public UploadIterator( + final StoreContext storeContext, + AmazonS3 s3, + int maxKeys, + @Nullable String prefix) throws IOException { - lister = new ListingIterator(s3, invoker, bucketName, maxKeys, prefix); + lister = new ListingIterator(storeContext, s3, prefix, + maxKeys); requestNextBatch(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index 65b9535ba65ea..5ba39aae2e29a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import com.amazonaws.SdkBaseException; import com.amazonaws.event.ProgressEvent; import com.amazonaws.event.ProgressEventType; import com.amazonaws.event.ProgressListener; @@ -55,12 +56,12 @@ import org.apache.hadoop.fs.Syncable; import org.apache.hadoop.fs.s3a.commit.CommitConstants; import org.apache.hadoop.fs.s3a.commit.PutTracker; -import org.apache.hadoop.fs.s3a.impl.LogExactlyOnce; import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.fs.statistics.DurationTracker; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsLogging; import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.util.Progressable; import static java.util.Objects.requireNonNull; @@ -134,6 +135,8 @@ class S3ABlockOutputStream extends OutputStream implements /** * Write operation helper; encapsulation of the filesystem operations. + * This contains the audit span for the operation, and activates/deactivates + * it within calls. */ private final WriteOperations writeOperationHelper; @@ -393,6 +396,7 @@ public void close() throws IOException { final List partETags = multiPartUpload.waitForAllPartUploads(); bytes = bytesSubmitted; + // then complete the operation if (putTracker.aboutToComplete(multiPartUpload.getUploadId(), partETags, @@ -777,6 +781,12 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block) uploadData.getUploadStream(), uploadData.getFile(), 0L); + } catch (SdkBaseException aws) { + // catch and translate + IOException e = translateException("upload", key, aws); + // failure to start the upload. + noteUploadFailure(e); + throw e; } catch (IOException e) { // failure to start the upload. noteUploadFailure(e); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 751038bb6d3c3..439d52edc14f5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -21,7 +21,6 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; -import java.io.InputStream; import java.io.InterruptedIOException; import java.net.URI; import java.nio.file.AccessDeniedException; @@ -53,12 +52,12 @@ import com.amazonaws.AmazonServiceException; import com.amazonaws.SdkBaseException; import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; import com.amazonaws.services.s3.model.CannedAccessControlList; import com.amazonaws.services.s3.model.CopyObjectRequest; import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.DeleteObjectsResult; import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadResult; import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; @@ -69,8 +68,7 @@ import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.PutObjectResult; -import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; -import com.amazonaws.services.s3.model.SSECustomerKey; +import com.amazonaws.services.s3.model.S3Object; import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; import com.amazonaws.services.s3.transfer.Copy; @@ -80,6 +78,9 @@ import com.amazonaws.services.s3.transfer.model.CopyResult; import com.amazonaws.services.s3.transfer.model.UploadResult; import com.amazonaws.event.ProgressListener; + +import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; +import org.apache.hadoop.fs.store.audit.ActiveThreadSpanSource; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; @@ -91,11 +92,13 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonPathCapabilities; +import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Globber; import org.apache.hadoop.fs.impl.OpenFileParameters; +import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.s3a.auth.SignerManager; import org.apache.hadoop.fs.s3a.auth.delegation.DelegationOperations; import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider; @@ -106,12 +109,15 @@ import org.apache.hadoop.fs.s3a.impl.DeleteOperation; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicyImpl; +import org.apache.hadoop.fs.s3a.impl.GetContentSummaryOperation; import org.apache.hadoop.fs.s3a.impl.HeaderProcessing; import org.apache.hadoop.fs.s3a.impl.InternalConstants; import org.apache.hadoop.fs.s3a.impl.ListingOperationCallbacks; +import org.apache.hadoop.fs.s3a.impl.MkdirOperation; import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; import org.apache.hadoop.fs.s3a.impl.OperationCallbacks; import org.apache.hadoop.fs.s3a.impl.RenameOperation; +import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl; import org.apache.hadoop.fs.s3a.impl.S3AMultipartUploaderBuilder; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.impl.StoreContext; @@ -123,9 +129,14 @@ import org.apache.hadoop.fs.statistics.DurationTracker; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.store.audit.AuditEntryPoint; +import org.apache.hadoop.fs.store.audit.AuditSpan; +import org.apache.hadoop.fs.store.audit.AuditSpanSource; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.token.DelegationTokenIssuer; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.util.DurationInfo; @@ -144,9 +155,12 @@ import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditManagerS3A; +import org.apache.hadoop.fs.s3a.audit.AuditIntegration; +import org.apache.hadoop.fs.s3a.audit.OperationAuditor; import org.apache.hadoop.fs.s3a.auth.RoleModel; import org.apache.hadoop.fs.s3a.auth.delegation.AWSPolicyProvider; -import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecretOperations; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens; import org.apache.hadoop.fs.s3a.auth.delegation.AbstractS3ATokenIdentifier; @@ -175,6 +189,7 @@ import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.SemaphoredDelegatingExecutor; import org.apache.hadoop.util.concurrent.HadoopExecutors; +import org.apache.hadoop.util.functional.CallableRaisingIOE; import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL; @@ -186,7 +201,7 @@ import static org.apache.hadoop.fs.s3a.Listing.toLocatedFileStatusIterator; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; -import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.INITIALIZE_SPAN; import static org.apache.hadoop.fs.s3a.auth.RolePolicies.STATEMENT_ALLOW_SSE_KMS_RW; import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3Operations; import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.TokenIssuingPolicy.NoTokensAvailable; @@ -197,7 +212,10 @@ import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.UPLOAD_PART_COUNT_LIMIT; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.logDnsLookup; import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.dirMetaToStatuses; @@ -205,8 +223,10 @@ import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.pairedTrackerFactory; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; import static org.apache.hadoop.util.functional.RemoteIterators.typeCastingRemoteIterator; @@ -226,22 +246,14 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class S3AFileSystem extends FileSystem implements StreamCapabilities, - AWSPolicyProvider, DelegationTokenProvider, IOStatisticsSource { + AWSPolicyProvider, DelegationTokenProvider, IOStatisticsSource, + AuditSpanSource, ActiveThreadSpanSource { + /** * Default blocksize as used in blocksize and FS status queries. */ public static final int DEFAULT_BLOCKSIZE = 32 * 1024 * 1024; - /** - * This declared delete as idempotent. - * This is an "interesting" topic in past Hadoop FS work. - * Essentially: with a single caller, DELETE is idempotent - * but in a shared filesystem, it is is very much not so. - * Here, on the basis that isn't a filesystem with consistency guarantees, - * retryable results in files being deleted. - */ - public static final boolean DELETE_CONSIDERED_IDEMPOTENT = true; - private URI uri; private Path workingDir; private String username; @@ -304,7 +316,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private S3ADataBlocks.BlockFactory blockFactory; private int blockOutputActiveBlocks; private WriteOperationHelper writeHelper; - private SelectBinding selectBinding; private boolean useListV1; private MagicCommitIntegration committerIntegration; @@ -318,12 +329,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, */ private int pageSize; - /** - * Specific operations used by rename and delete operations. - */ - private final S3AFileSystem.OperationCallbacksImpl - operationCallbacks = new OperationCallbacksImpl(); - private final ListingOperationCallbacks listingOperationCallbacks = new ListingOperationCallbacksImpl(); /** @@ -332,14 +337,24 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private DirectoryPolicy directoryPolicy; /** - * Header processing for XAttr. + * Context accessors for re-use. */ - private HeaderProcessing headerProcessing; + private final ContextAccessors contextAccessors = new ContextAccessorsImpl(); /** - * Context accessors for re-use. + * Factory for AWS requests. */ - private final ContextAccessors contextAccessors = new ContextAccessorsImpl(); + private RequestFactory requestFactory; + + /** + * Audit manager (service lifecycle). + * Creates the audit service and manages the binding of different audit spans + * to different threads. + * Initially this is a no-op manager; once the service is initialized it will + * be replaced with a configured one. + */ + private AuditManagerS3A auditManager = + AuditIntegration.stubAuditManager(); /** Add any deprecated keys. */ @SuppressWarnings("deprecation") @@ -371,6 +386,7 @@ public void initialize(URI name, Configuration originalConf) throws IOException { // get the host; this is guaranteed to be non-null, non-empty bucket = name.getHost(); + AuditSpan span = null; try { LOG.debug("Initializing S3AFileSystem for {}", bucket); // clone the configuration into one with propagated bucket options @@ -411,8 +427,6 @@ public void initialize(URI name, Configuration originalConf) s3guardInvoker = new Invoker(new S3GuardExistsRetryPolicy(getConf()), onRetry); - writeHelper = new WriteOperationHelper(this, getConf(), - statisticsContext); failOnMetadataWriteError = conf.getBoolean(FAIL_ON_METADATA_WRITE_ERROR, FAIL_ON_METADATA_WRITE_ERROR_DEFAULT); @@ -442,6 +456,21 @@ public void initialize(URI name, Configuration originalConf) signerManager = new SignerManager(bucket, this, conf, owner); signerManager.initCustomSigners(); + // start auditing + initializeAuditService(); + + // create the requestFactory. + // requires the audit manager to be initialized. + requestFactory = createRequestFactory(); + + // create the static write operation helper. + // this doesn't have a short-lived span; auditors which + // require one may reject usages. + writeHelper = createWriteOperationHelper(getActiveAuditSpan()); + + // create an initial span for all other operations. + span = createSpan(INITIALIZE_SPAN, bucket, null); + // creates the AWS client, including overriding auth chain if // the FS came with a DT // this may do some patching of the configuration (e.g. setting @@ -467,11 +496,6 @@ public void initialize(URI name, Configuration originalConf) magicCommitterEnabled ? "is" : "is not"); committerIntegration = new MagicCommitIntegration( this, magicCommitterEnabled); - // header processing for rename and magic committer - headerProcessing = new HeaderProcessing(createStoreContext()); - - // instantiate S3 Select support - selectBinding = new SelectBinding(writeHelper); boolean blockUploadEnabled = conf.getBoolean(FAST_UPLOAD, true); @@ -519,10 +543,12 @@ public void initialize(URI name, Configuration originalConf) listing = new Listing(listingOperationCallbacks, createStoreContext()); } catch (AmazonClientException e) { // amazon client exception: stop all services then throw the translation + cleanupWithLogger(LOG, span); stopAllServices(); throw translateException("initializing ", new Path(name), e); } catch (IOException | RuntimeException e) { // other exceptions: stop the services. + cleanupWithLogger(LOG, span); stopAllServices(); throw e; } @@ -602,6 +628,7 @@ public Statistics getInstanceStatistics() { * @param conf configuration. */ private void initThreadPools(Configuration conf) { + final String name = "s3a-transfer-" + getBucket(); int maxThreads = conf.getInt(MAX_THREADS, DEFAULT_MAX_THREADS); if (maxThreads < 2) { LOG.warn(MAX_THREADS + " must be at least 2: forcing to 2."); @@ -615,13 +642,13 @@ private void initThreadPools(Configuration conf) { maxThreads, maxThreads + totalTasks, keepAliveTime, TimeUnit.SECONDS, - "s3a-transfer-shared"); + name + "-bounded"); unboundedThreadPool = new ThreadPoolExecutor( maxThreads, Integer.MAX_VALUE, keepAliveTime, TimeUnit.SECONDS, new LinkedBlockingQueue<>(), BlockingThreadPoolExecutorService.newDaemonThreadFactory( - "s3a-transfer-unbounded")); + name + "-unbounded")); unboundedThreadPool.allowCoreThreadTimeOut(true); executorCapacity = intOption(conf, EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1); @@ -651,7 +678,9 @@ protected static S3AStorageStatistics createStorageStatistics( protected void verifyBucketExists() throws UnknownStoreException, IOException { if (!invoker.retry("doesBucketExist", bucket, true, - () -> s3.doesBucketExist(bucket))) { + trackDurationOfOperation(getDurationTrackerFactory(), + STORE_EXISTS_PROBE.getSymbol(), + () -> s3.doesBucketExist(bucket)))) { throw new UnknownStoreException("Bucket " + bucket + " does not exist"); } } @@ -667,7 +696,9 @@ protected void verifyBucketExists() protected void verifyBucketExistsV2() throws UnknownStoreException, IOException { if (!invoker.retry("doesBucketExistV2", bucket, true, - () -> s3.doesBucketExistV2(bucket))) { + trackDurationOfOperation(getDurationTrackerFactory(), + STORE_EXISTS_PROBE.getSymbol(), + () -> s3.doesBucketExistV2(bucket)))) { throw new UnknownStoreException("Bucket " + bucket + " does not exist"); } } @@ -750,13 +781,117 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException { .withEndpoint(conf.getTrimmed(ENDPOINT, DEFAULT_ENDPOINT)) .withMetrics(statisticsContext.newStatisticsFromAwsSdk()) .withPathStyleAccess(conf.getBoolean(PATH_STYLE_ACCESS, false)) - .withUserAgentSuffix(uaSuffix); + .withUserAgentSuffix(uaSuffix) + .withRequestHandlers(auditManager.createRequestHandlers()); s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf) .createS3Client(getUri(), parameters); } + /** + * Initialize and launch the audit manager and service. + * As this takes the FS IOStatistics store, it must be invoked + * after instrumentation is initialized. + * @throws IOException failure to instantiate/initialize. + */ + protected void initializeAuditService() throws IOException { + auditManager = AuditIntegration.createAndStartAuditManager( + getConf(), + instrumentation.createMetricsUpdatingStore()); + } + + /** + * The audit manager. + * @return the audit manager + */ + @InterfaceAudience.Private + public AuditManagerS3A getAuditManager() { + return auditManager; + } + + /** + * Get the auditor; valid once initialized. + * @return the auditor. + */ + @InterfaceAudience.Private + public OperationAuditor getAuditor() { + return getAuditManager().getAuditor(); + } + + /** + * Get the active audit span. + * @return the span. + */ + @InterfaceAudience.Private + @Override + public AuditSpanS3A getActiveAuditSpan() { + return getAuditManager().getActiveAuditSpan(); + } + + /** + * Get the audit span source; allows for components like the committers + * to have a source of spans without being hard coded to the FS only. + * @return the source of spans -base implementation is this instance. + */ + @InterfaceAudience.Private + public AuditSpanSource getAuditSpanSource() { + return this; + } + + /** + * Start an operation; this informs the audit service of the event + * and then sets it as the active span. + * @param operation operation name. + * @param path1 first path of operation + * @param path2 second path of operation + * @return a span for the audit + * @throws IOException failure + */ + public AuditSpanS3A createSpan(String operation, + @Nullable String path1, + @Nullable String path2) + throws IOException { + + return getAuditManager().createSpan(operation, path1, path2); + } + + /** + * Build the request factory. + * MUST be called after reading encryption secrets from settings/ + * delegation token. + * Protected, in case test/mock classes want to implement their + * own variants. + * @return request factory. + */ + protected RequestFactory createRequestFactory() { + long partCountLimit = longOption(getConf(), + UPLOAD_PART_COUNT_LIMIT, + DEFAULT_UPLOAD_PART_COUNT_LIMIT, + 1); + if (partCountLimit != DEFAULT_UPLOAD_PART_COUNT_LIMIT) { + LOG.warn("Configuration property {} shouldn't be overridden by client", + UPLOAD_PART_COUNT_LIMIT); + } + + return RequestFactoryImpl.builder() + .withBucket(requireNonNull(bucket)) + .withCannedACL(getCannedACL()) + .withEncryptionSecrets(requireNonNull(encryptionSecrets)) + .withMultipartPartCountLimit(partCountLimit) + .withRequestPreparer(getAuditManager()::requestCreated) + .build(); + } + + /** + * Get the request factory which uses this store's audit span. + * @return the request factory. + */ + @VisibleForTesting + public RequestFactory getRequestFactory() { + return requestFactory; + } + /** * Implementation of all operations used by delegation tokens. */ @@ -783,6 +918,9 @@ public DelegationOperations createDelegationOperations() { */ protected void setEncryptionSecrets(final EncryptionSecrets secrets) { this.encryptionSecrets = secrets; + if (requestFactory != null) { + requestFactory.setEncryptionSecrets(secrets); + } } /** @@ -936,11 +1074,13 @@ protected void setAmazonS3Client(AmazonS3 client) { /** * Get the region of a bucket. + * Invoked from StoreContext; consider an entry point. * @return the region in which a bucket is located * @throws AccessDeniedException if the caller lacks permission. * @throws IOException on any failure. */ @Retries.RetryTranslated + @InterfaceAudience.LimitedPrivate("diagnostics") public String getBucketLocation() throws IOException { return getBucketLocation(bucket); } @@ -957,10 +1097,13 @@ public String getBucketLocation() throws IOException { * @throws IOException on any failure. */ @VisibleForTesting + @AuditEntryPoint @Retries.RetryTranslated public String getBucketLocation(String bucketName) throws IOException { - final String region = invoker.retry("getBucketLocation()", bucketName, true, - () -> s3.getBucketLocation(bucketName)); + final String region = trackDurationAndSpan( + STORE_EXISTS_PROBE, bucketName, null, () -> + invoker.retry("getBucketLocation()", bucketName, true, () -> + s3.getBucketLocation(bucketName))); return fixBucketRegion(region); } @@ -1192,14 +1335,16 @@ public FSDataInputStream open(Path f, int bufferSize) * @throws IOException IO failure. */ @Retries.RetryTranslated + @AuditEntryPoint private FSDataInputStream open( final Path file, final Optional options, final Optional providedStatus) throws IOException { - entryPoint(INVOCATION_OPEN); final Path path = qualify(file); + // this span is passed into the stream. + final AuditSpan auditSpan = entryPoint(INVOCATION_OPEN, path); S3AFileStatus fileStatus = extractOrFetchSimpleFileStatus(path, providedStatus); @@ -1217,13 +1362,15 @@ private FSDataInputStream open( fileStatus, policy, changeDetectionPolicy, - readAheadRange2); + readAheadRange2, + auditSpan); } else { readContext = createReadContext( fileStatus, inputPolicy, changeDetectionPolicy, - readAhead); + readAhead, + auditSpan); } LOG.debug("Opening '{}'", readContext); @@ -1231,7 +1378,60 @@ private FSDataInputStream open( new S3AInputStream( readContext, createObjectAttributes(fileStatus), - s3)); + createInputStreamCallbacks(auditSpan))); + } + + /** + * Override point: create the callbacks for S3AInputStream. + * @return an implementation of the InputStreamCallbacks, + */ + private S3AInputStream.InputStreamCallbacks createInputStreamCallbacks( + final AuditSpan auditSpan) { + return new InputStreamCallbacksImpl(auditSpan); + } + + /** + * Operations needed by S3AInputStream to read data. + */ + private final class InputStreamCallbacksImpl implements + S3AInputStream.InputStreamCallbacks { + + /** + * Audit span to activate before each call. + */ + private final AuditSpan auditSpan; + + /** + * Create. + * @param auditSpan Audit span to activate before each call. + */ + private InputStreamCallbacksImpl(final AuditSpan auditSpan) { + this.auditSpan = requireNonNull(auditSpan); + } + + /** + * Closes the audit span. + */ + @Override + public void close() { + auditSpan.close(); + } + + @Override + public GetObjectRequest newGetRequest(final String key) { + // active the audit span used for the operation + try (AuditSpan span = auditSpan.activate()) { + return getRequestFactory().newGetObjectRequest(key); + } + } + + @Override + public S3Object getObject(GetObjectRequest request) { + // active the audit span used for the operation + try (AuditSpan span = auditSpan.activate()) { + return s3.getObject(request); + } + } } /** @@ -1246,7 +1446,8 @@ private S3AReadOpContext createReadContext( final FileStatus fileStatus, final S3AInputPolicy seekPolicy, final ChangeDetectionPolicy changePolicy, - final long readAheadRange) { + final long readAheadRange, + final AuditSpan auditSpan) { return new S3AReadOpContext(fileStatus.getPath(), hasMetadataStore(), invoker, @@ -1256,7 +1457,8 @@ private S3AReadOpContext createReadContext( fileStatus, seekPolicy, changePolicy, - readAheadRange); + readAheadRange, + auditSpan); } /** @@ -1313,12 +1515,43 @@ private S3ObjectAttributes createObjectAttributes( * @see #setPermission(Path, FsPermission) */ @Override + @AuditEntryPoint @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { - entryPoint(INVOCATION_CREATE); final Path path = qualify(f); + // the span will be picked up inside the output stream + return trackDurationAndSpan(INVOCATION_CREATE, path, () -> + innerCreateFile(path, permission, overwrite, bufferSize, replication, + blockSize, progress)); + } + + /** + * Create an FSDataOutputStream at the indicated Path with write-progress + * reporting; in the active span. + * Retry policy: retrying, translated on the getFileStatus() probe. + * No data is uploaded to S3 in this call, so no retry issues related to that. + * @param path the file name to open + * @param permission the permission to set. + * @param overwrite if a file with this name already exists, then if true, + * the file will be overwritten, and if false an error will be thrown. + * @param bufferSize the size of the buffer to be used. + * @param replication required block replication for the file. + * @param blockSize the requested block size. + * @param progress the progress reporter. + * @throws IOException in the event of IO related errors. + * @see #setPermission(Path, FsPermission) + */ + @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") + @Retries.RetryTranslated + private FSDataOutputStream innerCreateFile(Path path, + FsPermission permission, + boolean overwrite, + int bufferSize, + short replication, + long blockSize, + Progressable progress) throws IOException { String key = pathToKey(path); FileStatus status = null; try { @@ -1359,7 +1592,8 @@ public FSDataOutputStream create(Path f, FsPermission permission, .withStatistics(outputStreamStatistics) .withProgress(progress) .withPutTracker(putTracker) - .withWriteOperations(getWriteOperationHelper()) + .withWriteOperations( + createWriteOperationHelper(getActiveAuditSpan())) .withExecutorService( new SemaphoredDelegatingExecutor( boundedThreadPool, @@ -1376,7 +1610,9 @@ public FSDataOutputStream create(Path f, FsPermission permission, } /** - * Get a {@code WriteOperationHelper} instance. + * Create a Write Operation Helper with the current active span. + * All operations made through this helper will activate the + * span before execution. * * This class permits other low-level operations against the store. * It is unstable and @@ -1386,7 +1622,23 @@ public FSDataOutputStream create(Path f, FsPermission permission, */ @InterfaceAudience.Private public WriteOperationHelper getWriteOperationHelper() { - return writeHelper; + return createWriteOperationHelper(getActiveAuditSpan()); + } + + /** + * Create a Write Operation Helper with the given span. + * All operations made through this helper will activate the + * span before execution. + * @param auditSpan audit span + * @return a new helper. + */ + @InterfaceAudience.Private + public WriteOperationHelper createWriteOperationHelper(AuditSpan auditSpan) { + return new WriteOperationHelper(this, + getConf(), + statisticsContext, + getAuditSpanSource(), + auditSpan); } /** @@ -1395,6 +1647,7 @@ public WriteOperationHelper getWriteOperationHelper() { * is not a directory. */ @Override + @AuditEntryPoint public FSDataOutputStream createNonRecursive(Path p, FsPermission permission, EnumSet flags, @@ -1402,29 +1655,31 @@ public FSDataOutputStream createNonRecursive(Path p, short replication, long blockSize, Progressable progress) throws IOException { - entryPoint(INVOCATION_CREATE_NON_RECURSIVE); final Path path = makeQualified(p); - Path parent = path.getParent(); - // expect this to raise an exception if there is no parent dir - if (parent != null && !parent.isRoot()) { - S3AFileStatus status; - try { - // optimize for the directory existing: Call list first - status = innerGetFileStatus(parent, false, - StatusProbeEnum.DIRECTORIES); - } catch (FileNotFoundException e) { - // no dir, fall back to looking for a file - // (failure condition if true) - status = innerGetFileStatus(parent, false, - StatusProbeEnum.HEAD_ONLY); - } - if (!status.isDirectory()) { - throw new FileAlreadyExistsException("Not a directory: " + parent); + // this span is passed into the stream. + try (AuditSpan span = entryPoint(INVOCATION_CREATE_NON_RECURSIVE, path)) { + Path parent = path.getParent(); + // expect this to raise an exception if there is no parent dir + if (parent != null && !parent.isRoot()) { + S3AFileStatus status; + try { + // optimize for the directory existing: Call list first + status = innerGetFileStatus(parent, false, + StatusProbeEnum.DIRECTORIES); + } catch (FileNotFoundException e) { + // no dir, fall back to looking for a file + // (failure condition if true) + status = innerGetFileStatus(parent, false, + StatusProbeEnum.HEAD_ONLY); + } + if (!status.isDirectory()) { + throw new FileAlreadyExistsException("Not a directory: " + parent); + } } + return innerCreateFile(path, permission, + flags.contains(CreateFlag.OVERWRITE), bufferSize, + replication, blockSize, progress); } - return create(path, permission, - flags.contains(CreateFlag.OVERWRITE), bufferSize, - replication, blockSize, progress); } /** @@ -1462,11 +1717,13 @@ public FSDataOutputStream append(Path f, int bufferSize, * @throws IOException on IO failure * @return true if rename is successful */ + @AuditEntryPoint @Retries.RetryTranslated public boolean rename(Path src, Path dst) throws IOException { - try (DurationInfo ignored = new DurationInfo(LOG, false, - "rename(%s, %s", src, dst)) { - long bytesCopied = innerRename(src, dst); + try { + long bytesCopied = trackDurationAndSpan( + INVOCATION_RENAME, src.toString(), dst.toString(), () -> + innerRename(src, dst)); LOG.debug("Copied {} bytes", bytesCopied); return true; } catch (AmazonClientException e) { @@ -1604,7 +1861,6 @@ private long innerRename(Path source, Path dest) Path dst = qualify(dest); LOG.debug("Rename path {} to {}", src, dst); - entryPoint(INVOCATION_RENAME); String srcKey = pathToKey(src); String dstKey = pathToKey(dst); @@ -1618,7 +1874,7 @@ private long innerRename(Path source, Path dest) createStoreContext(), src, srcKey, p.getLeft(), dst, dstKey, p.getRight(), - operationCallbacks, + new OperationCallbacksImpl(), pageSize); return renameOperation.execute(); } @@ -1632,8 +1888,17 @@ private long innerRename(Path source, Path dest) * The callbacks made by the rename and delete operations. * This separation allows the operation to be factored out and * still avoid knowledge of the S3AFilesystem implementation. + * The Audit span active at the time of creation is cached and activated + * before every call. */ - private class OperationCallbacksImpl implements OperationCallbacks { + private final class OperationCallbacksImpl implements OperationCallbacks { + + /** Audit Span at time of creation. */ + private final AuditSpan auditSpan; + + private OperationCallbacksImpl() { + auditSpan = getActiveAuditSpan(); + } @Override public S3ObjectAttributes createObjectAttributes(final Path path, @@ -1654,7 +1919,8 @@ public S3ObjectAttributes createObjectAttributes( public S3AReadOpContext createReadContext(final FileStatus fileStatus) { return S3AFileSystem.this.createReadContext(fileStatus, inputPolicy, - changeDetectionPolicy, readAhead); + changeDetectionPolicy, readAhead, + auditSpan); } @Override @@ -1664,6 +1930,7 @@ public void deleteObjectAtPath(final Path path, final boolean isFile, final BulkOperationState operationState) throws IOException { + auditSpan.activate(); once("delete", path.toString(), () -> S3AFileSystem.this.deleteObjectAtPath(path, key, isFile, operationState)); @@ -1676,6 +1943,7 @@ public RemoteIterator listFilesAndDirectoryMarkers( final S3AFileStatus status, final boolean collectTombstones, final boolean includeSelf) throws IOException { + auditSpan.activate(); return innerListFiles( path, true, @@ -1692,6 +1960,7 @@ public CopyResult copyFile(final String srcKey, final String destKey, final S3ObjectAttributes srcAttributes, final S3AReadOpContext readContext) throws IOException { + auditSpan.activate(); return S3AFileSystem.this.copyFile(srcKey, destKey, srcAttributes.getLen(), srcAttributes, readContext); } @@ -1704,6 +1973,7 @@ public DeleteObjectsResult removeKeys( final BulkOperationState operationState, final boolean quiet) throws MultiObjectDeleteException, AmazonClientException, IOException { + auditSpan.activate(); return S3AFileSystem.this.removeKeys(keysToDelete, deleteFakeDir, undeletedObjectsOnFailure, operationState, quiet); } @@ -1711,6 +1981,7 @@ public DeleteObjectsResult removeKeys( @Override public void finishRename(final Path sourceRenamed, final Path destCreated) throws IOException { + auditSpan.activate(); Path destParent = destCreated.getParent(); if (!sourceRenamed.getParent().equals(destParent)) { LOG.debug("source & dest parents are different; fix up dir markers"); @@ -1737,10 +2008,17 @@ public RemoteIterator listObjects( createListObjectsRequest(key, null), ACCEPT_ALL, Listing.ACCEPT_ALL_BUT_S3N, - null)); + null, + auditSpan)); } } + /** + * Callbacks from {@link Listing}. + * Auditing: the listing object is long-lived; the audit span + * for a single listing is passed in from the listing + * method calls and then down to the callbacks. + */ protected class ListingOperationCallbacksImpl implements ListingOperationCallbacks { @@ -1748,9 +2026,10 @@ protected class ListingOperationCallbacksImpl implements @Retries.RetryRaw public CompletableFuture listObjectsAsync( S3ListRequest request, - DurationTrackerFactory trackerFactory) + DurationTrackerFactory trackerFactory, + AuditSpan span) throws IOException { - return submit(unboundedThreadPool, () -> + return submit(unboundedThreadPool, span, () -> listObjects(request, pairedTrackerFactory(trackerFactory, getDurationTrackerFactory()))); @@ -1761,9 +2040,10 @@ public CompletableFuture listObjectsAsync( public CompletableFuture continueListObjectsAsync( S3ListRequest request, S3ListResult prevResult, - DurationTrackerFactory trackerFactory) + DurationTrackerFactory trackerFactory, + AuditSpan span) throws IOException { - return submit(unboundedThreadPool, + return submit(unboundedThreadPool, span, () -> continueListObjects(request, prevResult, pairedTrackerFactory(trackerFactory, getDurationTrackerFactory()))); @@ -1778,8 +2058,10 @@ public S3ALocatedFileStatus toLocatedFileStatus( @Override public S3ListRequest createListObjectsRequest( - String key, - String delimiter) { + String key, + String delimiter, + AuditSpan span) { + span.activate(); return S3AFileSystem.this.createListObjectsRequest(key, delimiter); } @@ -1806,15 +2088,22 @@ public boolean allowAuthoritative(final Path p) { /** * Low-level call to get at the object metadata. + * This method is used in some external applications and so + * must be viewed as a public entry point. + * Auditing: An audit entry point. * @param path path to the object. This will be qualified. * @return metadata * @throws IOException IO and object access problems. */ @VisibleForTesting + @AuditEntryPoint + @InterfaceAudience.LimitedPrivate("utilities") @Retries.RetryTranslated + @InterfaceStability.Evolving public ObjectMetadata getObjectMetadata(Path path) throws IOException { - return getObjectMetadata(makeQualified(path), null, invoker, - "getObjectMetadata"); + return trackDurationAndSpan(INVOCATION_GET_FILE_STATUS, path, () -> + getObjectMetadata(makeQualified(path), null, invoker, + "getObjectMetadata")); } /** @@ -1830,7 +2119,6 @@ public ObjectMetadata getObjectMetadata(Path path) throws IOException { private ObjectMetadata getObjectMetadata(Path path, ChangeTracker changeTracker, Invoker changeInvoker, String operation) throws IOException { - checkNotClosed(); String key = pathToKey(path); return once(operation, path.toString(), () -> @@ -1878,12 +2166,76 @@ void setMetadataStore(MetadataStore ms) { /** * Entry point to an operation. * Increments the statistic; verifies the FS is active. - * @param operation The operation to increment - * @throws IOException if the - */ - protected void entryPoint(Statistic operation) throws IOException { + * @param operation The operation being invoked + * @param path first path of operation + * @return a span for the audit + * @throws IOException failure of audit service + */ + protected AuditSpan entryPoint(Statistic operation, + Path path) throws IOException { + return entryPoint(operation, + (path != null ? pathToKey(path): null), + null); + } + + /** + * Entry point to an operation. + * Increments the statistic; verifies the FS is active. + * @param operation The operation being invoked + * @param path1 first path of operation + * @param path2 second path of operation + * @return a span for the audit + * @throws IOException failure of audit service + */ + protected AuditSpan entryPoint(Statistic operation, + @Nullable String path1, + @Nullable String path2) throws IOException { checkNotClosed(); incrementStatistic(operation); + return createSpan(operation.getSymbol(), + path1, path2); + } + + /** + * Given an IOException raising callable/lambda expression, + * execute it and update the relevant statistic within a span + * of the same statistic. + * @param statistic statistic key + * @param path first path for span (nullable) + * @param path2 second path for span + * @param input input callable. + * @param return type. + * @return the result of the operation. + * @throws IOException if raised in the callable + */ + private B trackDurationAndSpan( + Statistic statistic, String path, String path2, + CallableRaisingIOE input) throws IOException { + checkNotClosed(); + try (AuditSpan span = createSpan(statistic.getSymbol(), + path, path2)) { + return trackDuration(getDurationTrackerFactory(), + statistic.getSymbol(), input); + } + } + + /** + * Overloaded version of {@code trackDurationAndSpan()}. + * Takes a single nullable path as the path param, + * @param statistic statistic key + * @param path path for span (nullable) + * @param input input callable. + * @param return type. + * @return the result of the operation. + * @throws IOException if raised in the callable + */ + private B trackDurationAndSpan( + Statistic statistic, + @Nullable Path path, + CallableRaisingIOE input) throws IOException { + return trackDurationAndSpan(statistic, + path != null ? pathToKey(path): null, + null, input); } /** @@ -2026,12 +2378,17 @@ protected DurationTrackerFactory getDurationTrackerFactory() { /** * Request object metadata; increments counters in the process. * Retry policy: retry untranslated. + * This method is used in some external applications and so + * must be viewed as a public entry point. + * Auditing: this call does NOT initiate a new AuditSpan; the expectation + * is that there is already an active span. * @param key key * @return the metadata * @throws IOException if the retry invocation raises one (it shouldn't). */ @Retries.RetryRaw @VisibleForTesting + @InterfaceAudience.LimitedPrivate("external utilities") ObjectMetadata getObjectMetadata(String key) throws IOException { return getObjectMetadata(key, null, invoker, "getObjectMetadata"); } @@ -2053,12 +2410,10 @@ protected ObjectMetadata getObjectMetadata(String key, ChangeTracker changeTracker, Invoker changeInvoker, String operation) throws IOException { - GetObjectMetadataRequest request = - new GetObjectMetadataRequest(bucket, key); - //SSE-C requires to be filled in if enabled for object metadata - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); ObjectMetadata meta = changeInvoker.retryUntranslated("GET " + key, true, () -> { + GetObjectMetadataRequest request + = getRequestFactory().newGetObjectMetadataRequest(key); incrementStatistic(OBJECT_METADATA_REQUESTS); DurationTracker duration = getDurationTrackerFactory() .trackDuration(ACTION_HTTP_HEAD_REQUEST.getSymbol()); @@ -2161,7 +2516,10 @@ protected S3ListResult continueListObjects(S3ListRequest request, () -> { if (useListV1) { return S3ListResult.v1( - s3.listNextBatchOfObjects(prevResult.getV1())); + s3.listNextBatchOfObjects( + getRequestFactory() + .newListNextBatchOfObjectsRequest( + prevResult.getV1()))); } else { request.getV2().setContinuationToken(prevResult.getV2() .getNextContinuationToken()); @@ -2215,7 +2573,8 @@ protected void deleteObject(String key) incrementStatistic(OBJECT_DELETE_OBJECTS); trackDurationOfInvocation(getDurationTrackerFactory(), OBJECT_DELETE_REQUEST.getSymbol(), - () -> s3.deleteObject(bucket, key)); + () -> s3.deleteObject(getRequestFactory() + .newDeleteObjectRequest(key))); return null; }); } @@ -2328,46 +2687,7 @@ private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest) */ public PutObjectRequest newPutObjectRequest(String key, ObjectMetadata metadata, File srcfile) { - Preconditions.checkNotNull(srcfile); - PutObjectRequest putObjectRequest = new PutObjectRequest(bucket, key, - srcfile); - setOptionalPutRequestParameters(putObjectRequest); - putObjectRequest.setCannedAcl(cannedACL); - putObjectRequest.setMetadata(metadata); - return putObjectRequest; - } - - /** - * Create a {@link PutObjectRequest} request. - * The metadata is assumed to have been configured with the size of the - * operation. - * @param key key of object - * @param metadata metadata header - * @param inputStream source data. - * @return the request - */ - PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, - InputStream inputStream) { - Preconditions.checkNotNull(inputStream); - Preconditions.checkArgument(isNotEmpty(key), "Null/empty key"); - PutObjectRequest putObjectRequest = new PutObjectRequest(bucket, key, - inputStream, metadata); - setOptionalPutRequestParameters(putObjectRequest); - putObjectRequest.setCannedAcl(cannedACL); - return putObjectRequest; - } - - /** - * Create a new object metadata instance. - * Any standard metadata headers are added here, for example: - * encryption. - * @return a new metadata instance - */ - public ObjectMetadata newObjectMetadata() { - final ObjectMetadata om = new ObjectMetadata(); - setOptionalObjectMetadata(om); - return om; + return requestFactory.newPutObjectRequest(key, metadata, srcfile); } /** @@ -2379,11 +2699,7 @@ public ObjectMetadata newObjectMetadata() { * @return a new metadata instance */ public ObjectMetadata newObjectMetadata(long length) { - final ObjectMetadata om = newObjectMetadata(); - if (length >= 0) { - om.setContentLength(length); - } - return om; + return requestFactory.newObjectMetadata(length); } /** @@ -2398,6 +2714,7 @@ public ObjectMetadata newObjectMetadata(long length) { * must reference data (files, buffers) which stay valid until the upload * completes. * Retry policy: N/A: the transfer manager is performing the upload. + * Auditing: must be inside an audit span. * @param putObjectRequest the request * @return the upload initiated */ @@ -2417,6 +2734,7 @@ public UploadInfo putObject(PutObjectRequest putObjectRequest) { * file, from the content length of the header. * * Retry Policy: none. + * Auditing: must be inside an audit span. * Important: this call will close any input stream in the request. * @param putObjectRequest the request * @return the upload initiated @@ -2433,13 +2751,16 @@ PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest) LOG.debug("PUT {} bytes to {}", len, putObjectRequest.getKey()); incrementPutStartStatistics(len); try { - PutObjectResult result = s3.putObject(putObjectRequest); + PutObjectResult result = trackDurationOfSupplier( + getDurationTrackerFactory(), + OBJECT_PUT_REQUESTS.getSymbol(), () -> + s3.putObject(putObjectRequest)); incrementPutCompletedStatistics(true, len); // update metadata finishedWrite(putObjectRequest.getKey(), len, result.getETag(), result.getVersionId(), null); return result; - } catch (AmazonClientException e) { + } catch (SdkBaseException e) { incrementPutCompletedStatistics(false, len); throw e; } @@ -2478,7 +2799,6 @@ UploadPartResult uploadPart(UploadPartRequest request) long len = request.getPartSize(); incrementPutStartStatistics(len); try { - setOptionalUploadPartRequestParameters(request); UploadPartResult uploadPartResult = s3.uploadPart(request); incrementPutCompletedStatistics(true, len); return uploadPartResult; @@ -2497,7 +2817,6 @@ UploadPartResult uploadPart(UploadPartRequest request) public void incrementPutStartStatistics(long bytes) { LOG.debug("PUT start {} bytes", bytes); incrementWriteOperations(); - incrementStatistic(OBJECT_PUT_REQUESTS); incrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1); if (bytes > 0) { incrementGauge(OBJECT_PUT_BYTES_PENDING, bytes); @@ -2513,7 +2832,6 @@ public void incrementPutStartStatistics(long bytes) { */ public void incrementPutCompletedStatistics(boolean success, long bytes) { LOG.debug("PUT completed success={}; {} bytes", success, bytes); - incrementWriteOperations(); if (bytes > 0) { incrementStatistic(OBJECT_PUT_BYTES, bytes); decrementGauge(OBJECT_PUT_BYTES_PENDING, bytes); @@ -2582,9 +2900,7 @@ private DeleteObjectsResult removeKeysS3( try { if (enableMultiObjectsDelete) { result = deleteObjects( - new DeleteObjectsRequest(bucket) - .withKeys(keysToDelete) - .withQuiet(quiet)); + getRequestFactory().newBulkDeleteRequest(keysToDelete, quiet)); } else { for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) { deleteObject(keyVersion.getKey()); @@ -2672,7 +2988,7 @@ public void removeKeys( * @throws IOException other IO Exception. */ @Retries.RetryMixed - DeleteObjectsResult removeKeys( + private DeleteObjectsResult removeKeys( final List keysToDelete, final boolean deleteFakeDir, final List undeletedObjectsOnFailure, @@ -2720,33 +3036,39 @@ DeleteObjectsResult removeKeys( * have surfaced. * @throws IOException due to inability to delete a directory or file. */ + @Override @Retries.RetryTranslated + @AuditEntryPoint public boolean delete(Path f, boolean recursive) throws IOException { - try { - entryPoint(INVOCATION_DELETE); - DeleteOperation deleteOperation = new DeleteOperation( - createStoreContext(), - innerGetFileStatus(f, true, StatusProbeEnum.ALL), - recursive, - operationCallbacks, - pageSize); - boolean outcome = deleteOperation.execute(); + checkNotClosed(); + final Path path = qualify(f); + // span covers delete, getFileStatus, fake directory operations. + try (AuditSpan span = createSpan(INVOCATION_DELETE.getSymbol(), + path.toString(), null)) { + boolean outcome = trackDuration(getDurationTrackerFactory(), + INVOCATION_DELETE.getSymbol(), + new DeleteOperation( + createStoreContext(), + innerGetFileStatus(path, true, StatusProbeEnum.ALL), + recursive, + new OperationCallbacksImpl(), + pageSize)); if (outcome) { try { - maybeCreateFakeParentDirectory(f); + maybeCreateFakeParentDirectory(path); } catch (AccessDeniedException e) { LOG.warn("Cannot create directory marker at {}: {}", f.getParent(), e.toString()); - LOG.debug("Failed to create fake dir above {}", f, e); + LOG.debug("Failed to create fake dir above {}", path, e); } } return outcome; } catch (FileNotFoundException e) { - LOG.debug("Couldn't delete {} - does not exist: {}", f, e.toString()); + LOG.debug("Couldn't delete {} - does not exist: {}", path, e.toString()); instrumentation.errorIgnored(); return false; } catch (AmazonClientException e) { - throw translateException("delete", f, e); + throw translateException("delete", path, e); } } @@ -2793,11 +3115,14 @@ protected void maybeCreateFakeParentDirectory(Path path) * */ @Override + @AuditEntryPoint public RemoteIterator listStatusIterator(Path p) throws FileNotFoundException, IOException { - RemoteIterator listStatusItr = once("listStatus", - p.toString(), () -> innerListStatus(p)); - return typeCastingRemoteIterator(listStatusItr); + Path path = qualify(p); + return typeCastingRemoteIterator(trackDurationAndSpan( + INVOCATION_LIST_STATUS, path, () -> + once("listStatus", path.toString(), () -> + innerListStatus(p)))); } /** @@ -2809,18 +3134,25 @@ public RemoteIterator listStatusIterator(Path p) * @throws FileNotFoundException when the path does not exist; * IOException see specific implementation */ + @Override + @AuditEntryPoint public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException { - return once("listStatus", - f.toString(), - () -> iteratorToStatuses(innerListStatus(f), new HashSet<>())); + Path path = qualify(f); + return trackDurationAndSpan(INVOCATION_LIST_STATUS, path, () -> + once("listStatus", path.toString(), + () -> iteratorToStatuses(innerListStatus(path), + new HashSet<>()))); } /** * List the statuses of the files/directories in the given path if the path is - * a directory. + * a directory. The returned iterator is within the current active span. * - * @param f given path + * Auditing: This method MUST be called within a span. + * The span is attached to the iterator. All further S3 calls + * made by the iterator will be within the span. + * @param f qualified path * @return the statuses of the files/directories in the given patch * @throws FileNotFoundException when the path does not exist; * @throws IOException due to an IO problem. @@ -2831,11 +3163,10 @@ private RemoteIterator innerListStatus(Path f) IOException, AmazonClientException { Path path = qualify(f); LOG.debug("List status for path: {}", path); - entryPoint(INVOCATION_LIST_STATUS); Triple, DirListingMetadata, Boolean> statusesAssumingNonEmptyDir = listing - .getFileStatusesAssumingNonEmptyDir(path); + .getFileStatusesAssumingNonEmptyDir(path, getActiveAuditSpan()); if (!statusesAssumingNonEmptyDir.getLeft().hasNext() && statusesAssumingNonEmptyDir.getRight()) { @@ -2897,34 +3228,28 @@ public boolean allowAuthoritative(final Path path) { @VisibleForTesting public S3ListRequest createListObjectsRequest(String key, String delimiter) { - return createListObjectsRequest(key, delimiter, null); + return createListObjectsRequest(key, delimiter, maxKeys); } + /** + * Create the List objects request appropriate for the + * active list request option. + * @param key key for request + * @param delimiter any delimiter + * @param limit limit of keys + * @return the request + */ private S3ListRequest createListObjectsRequest(String key, - String delimiter, Integer overrideMaxKeys) { + String delimiter, int limit) { if (!useListV1) { ListObjectsV2Request request = - new ListObjectsV2Request().withBucketName(bucket) - .withMaxKeys(maxKeys) - .withPrefix(key); - if (delimiter != null) { - request.setDelimiter(delimiter); - } - if (overrideMaxKeys != null) { - request.setMaxKeys(overrideMaxKeys); - } + getRequestFactory().newListObjectsV2Request( + key, delimiter, limit); return S3ListRequest.v2(request); } else { - ListObjectsRequest request = new ListObjectsRequest(); - request.setBucketName(bucket); - request.setMaxKeys(maxKeys); - request.setPrefix(key); - if (delimiter != null) { - request.setDelimiter(delimiter); - } - if (overrideMaxKeys != null) { - request.setMaxKeys(overrideMaxKeys); - } + ListObjectsRequest request = + getRequestFactory().newListObjectsV1Request( + key, delimiter, limit); return S3ListRequest.v1(request); } } @@ -2968,80 +3293,130 @@ public UserGroupInformation getOwner() { * Make the given path and all non-existent parents into * directories. Has the semantics of Unix {@code 'mkdir -p'}. * Existence of the directory hierarchy is not an error. - * @param path path to create - * @param permission to apply to f + * @param p path to create + * @param permission to apply to path * @return true if a directory was created or already existed * @throws FileAlreadyExistsException there is a file at the path specified + * or is discovered on one of its ancestors. * @throws IOException other IO problems */ - // TODO: If we have created an empty file at /foo/bar and we then call - // mkdirs for /foo/bar/baz/roo what happens to the empty file /foo/bar/? - public boolean mkdirs(Path path, FsPermission permission) throws IOException, + @Override + @AuditEntryPoint + public boolean mkdirs(Path p, FsPermission permission) throws IOException, FileAlreadyExistsException { - try { - entryPoint(INVOCATION_MKDIRS); - return innerMkdirs(path, permission); - } catch (AmazonClientException e) { - throw translateException("mkdirs", path, e); + Path path = qualify(p); + return trackDurationAndSpan( + INVOCATION_MKDIRS, path, + new MkdirOperation( + createStoreContext(), + path, + createMkdirOperationCallbacks())); + } + + /** + * Override point: create the callbacks for Mkdir. + * This does not create a new span; caller must be in one. + * @return an implementation of the MkdirCallbacks, + */ + @VisibleForTesting + public MkdirOperation.MkdirCallbacks createMkdirOperationCallbacks() { + return new MkdirOperationCallbacksImpl(); + } + + /** + * Callbacks from the {@link MkdirOperation}. + */ + protected class MkdirOperationCallbacksImpl implements + MkdirOperation.MkdirCallbacks { + + @Override + public S3AFileStatus probePathStatus(final Path path, + final Set probes) throws IOException { + return S3AFileSystem.this.innerGetFileStatus(path, false, probes); + } + + @Override + public void createFakeDirectory(final String key) + throws IOException { + S3AFileSystem.this.createEmptyObject(key); } } /** - * - * Make the given path and all non-existent parents into - * directories. - * See {@link #mkdirs(Path, FsPermission)} - * @param p path to create - * @param permission to apply to f - * @return true if a directory was created or already existed - * @throws FileAlreadyExistsException there is a file at the path specified - * @throws IOException other IO problems - * @throws AmazonClientException on failures inside the AWS SDK + * This is a very slow operation against object storage. + * Execute it as a single span with whatever optimizations + * have been implemented. + * {@inheritDoc} */ - private boolean innerMkdirs(Path p, FsPermission permission) - throws IOException, FileAlreadyExistsException, AmazonClientException { - Path f = qualify(p); - LOG.debug("Making directory: {}", f); - if (p.isRoot()) { - // fast exit for root. - return true; + @Override + @Retries.RetryTranslated + @AuditEntryPoint + public ContentSummary getContentSummary(final Path f) throws IOException { + final Path path = qualify(f); + return trackDurationAndSpan( + INVOCATION_GET_CONTENT_SUMMARY, path, + new GetContentSummaryOperation( + createStoreContext(), + path, + createGetContentSummaryCallbacks())); + } + + /** + * Override point: create the callbacks for getContentSummary. + * This does not create a new span; caller must be in one. + * @return an implementation of the GetContentSummaryCallbacksImpl + */ + protected GetContentSummaryOperation.GetContentSummaryCallbacks + createGetContentSummaryCallbacks() { + return new GetContentSummaryCallbacksImpl(); + } + + /** + * Callbacks from the {@link GetContentSummaryOperation}. + */ + protected class GetContentSummaryCallbacksImpl implements + GetContentSummaryOperation.GetContentSummaryCallbacks { + + @Override + public S3AFileStatus probePathStatus(final Path path, + final Set probes) throws IOException { + return S3AFileSystem.this.innerGetFileStatus(path, false, probes); } - FileStatus fileStatus; - try { - fileStatus = innerGetFileStatus(f, false, - StatusProbeEnum.ALL); + @Override + public RemoteIterator listStatusIterator(final Path path) + throws IOException { + return S3AFileSystem.this.innerListStatus(path); + } + } - if (fileStatus.isDirectory()) { - return true; - } else { - throw new FileAlreadyExistsException("Path is a file: " + f); - } - } catch (FileNotFoundException e) { - // Walk path to root, ensuring closest ancestor is a directory, not file - Path fPart = f.getParent(); - while (fPart != null && !fPart.isRoot()) { - try { - fileStatus = getFileStatus(fPart); - if (fileStatus.isDirectory()) { - break; - } - if (fileStatus.isFile()) { - throw new FileAlreadyExistsException(String.format( - "Can't make directory for path '%s' since it is a file.", - fPart)); + /** + * Soft check of access by forwarding to the audit manager + * and so on to the auditor. + * {@inheritDoc} + */ + @Override + @AuditEntryPoint + public void access(final Path f, final FsAction mode) + throws AccessControlException, FileNotFoundException, IOException { + Path path = qualify(f); + LOG.debug("check access mode {} for {}", path, mode); + trackDurationAndSpan( + INVOCATION_ACCESS, path, () -> { + final S3AFileStatus stat = innerGetFileStatus(path, false, + StatusProbeEnum.ALL); + if (!getAuditManager().checkAccess(path, stat, mode)) { + incrementStatistic(AUDIT_ACCESS_CHECK_FAILURE); + throw new AccessControlException(String.format( + "Permission denied: user=%s, path=\"%s\":%s:%s:%s%s", + getOwner().getUserName(), + stat.getPath(), + stat.getOwner(), stat.getGroup(), + stat.isDirectory() ? "d" : "-", mode)); } - } catch (FileNotFoundException fnfe) { - instrumentation.errorIgnored(); - } - fPart = fPart.getParent(); - } - String key = pathToKey(f); - // this will create the marker file, delete the parent entries - // and update S3Guard - createFakeDirectory(key); - return true; - } + // simply for the API binding. + return true; + }); } /** @@ -3051,10 +3426,14 @@ private boolean innerMkdirs(Path p, FsPermission permission) * @throws FileNotFoundException when the path does not exist * @throws IOException on other problems. */ + @Override + @AuditEntryPoint @Retries.RetryTranslated public FileStatus getFileStatus(final Path f) throws IOException { - entryPoint(INVOCATION_GET_FILE_STATUS); - return innerGetFileStatus(f, false, StatusProbeEnum.ALL); + Path path = qualify(f); + return trackDurationAndSpan( + INVOCATION_GET_FILE_STATUS, path, () -> + innerGetFileStatus(path, false, StatusProbeEnum.ALL)); } /** @@ -3399,12 +3778,16 @@ private boolean s3Exists(final Path path, final Set probes) * @throws AmazonClientException failure in the AWS SDK */ @Override + @AuditEntryPoint public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst) throws IOException { - entryPoint(INVOCATION_COPY_FROM_LOCAL_FILE); + checkNotClosed(); LOG.debug("Copying local file from {} to {}", src, dst); -// innerCopyFromLocalFile(delSrc, overwrite, src, dst); - super.copyFromLocalFile(delSrc, overwrite, src, dst); + trackDurationAndSpan(INVOCATION_COPY_FROM_LOCAL_FILE, dst, () -> { + // innerCopyFromLocalFile(delSrc, overwrite, src, dst); + super.copyFromLocalFile(delSrc, overwrite, src, dst); + return null; + }); } /** @@ -3433,7 +3816,6 @@ public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, private void innerCopyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst) throws IOException, FileAlreadyExistsException, AmazonClientException { - entryPoint(INVOCATION_COPY_FROM_LOCAL_FILE); LOG.debug("Copying local file from {} to {}", src, dst); // Since we have a local file, we don't need to stream into a temporary file @@ -3447,7 +3829,7 @@ private void innerCopyFromLocalFile(boolean delSrc, boolean overwrite, } try { - FileStatus status = getFileStatus(dst); + FileStatus status = innerGetFileStatus(dst, false, StatusProbeEnum.ALL); if (!status.isFile()) { throw new FileAlreadyExistsException(dst + " exists and is not a file"); } @@ -3552,6 +3934,12 @@ public void close() throws IOException { } finally { stopAllServices(); } + // Log IOStatistics at debug. + if (LOG.isDebugEnabled()) { + // robust extract and convert to string + LOG.debug("Statistics for {}: {}", uri, + IOStatisticsLogging.ioStatisticsToPrettyString(getIOStatistics())); + } } /** @@ -3561,6 +3949,8 @@ public void close() throws IOException { * both the expected state of this FS and of failures while being stopped. */ protected synchronized void stopAllServices() { + // shutting down the transfer manager also shuts + // down the S3 client it is bonded to. if (transfers != null) { try { transfers.shutdownNow(true); @@ -3570,17 +3960,21 @@ protected synchronized void stopAllServices() { } transfers = null; } + // At this point the S3A client is shut down, + // now the executor pools are closed HadoopExecutors.shutdown(boundedThreadPool, LOG, THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); boundedThreadPool = null; HadoopExecutors.shutdown(unboundedThreadPool, LOG, THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); unboundedThreadPool = null; + // other services are shutdown. cleanupWithLogger(LOG, metadataStore, instrumentation, delegationTokens.orElse(null), - signerManager); + signerManager, + auditManager); closeAutocloseables(LOG, credentials); delegationTokens = Optional.empty(); signerManager = null; @@ -3638,13 +4032,17 @@ public String getCanonicalServiceName() { * @throws IOException IO failure */ @Override + @AuditEntryPoint public Token getDelegationToken(String renewer) throws IOException { - entryPoint(Statistic.INVOCATION_GET_DELEGATION_TOKEN); + checkNotClosed(); LOG.debug("Delegation token requested"); if (delegationTokens.isPresent()) { - return delegationTokens.get().getBoundOrNewDT(encryptionSecrets, - (renewer != null ? new Text(renewer) : new Text())); + return trackDurationAndSpan( + INVOCATION_GET_DELEGATION_TOKEN, null, () -> + delegationTokens.get().getBoundOrNewDT( + encryptionSecrets, + (renewer != null ? new Text(renewer) : new Text()))); } else { // Delegation token support is not set up LOG.debug("Token support is not enabled"); @@ -3662,6 +4060,7 @@ public Token getDelegationToken(String renewer) @Override public DelegationTokenIssuer[] getAdditionalTokenIssuers() throws IOException { + checkNotClosed(); if (delegationTokens.isPresent()) { return delegationTokens.get().getAdditionalTokenIssuers(); } else { @@ -3680,6 +4079,7 @@ public DelegationTokenIssuer[] getAdditionalTokenIssuers() * @return a policy for use in roles */ @Override + @InterfaceAudience.Private public List listAWSPolicyRules( final Set access) { if (access.isEmpty()) { @@ -3768,24 +4168,17 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, message, e); } - ObjectMetadata dstom = cloneObjectMetadata(srcom); - setOptionalObjectMetadata(dstom); return readInvoker.retry( action, srcKey, true, () -> { CopyObjectRequest copyObjectRequest = - new CopyObjectRequest(bucket, srcKey, bucket, dstKey); + getRequestFactory().newCopyObjectRequest(srcKey, dstKey, srcom); changeTracker.maybeApplyConstraint(copyObjectRequest); - - setOptionalCopyObjectRequestParameters(srcom, copyObjectRequest); - copyObjectRequest.setCannedAccessControlList(cannedACL); - copyObjectRequest.setNewObjectMetadata(dstom); - Optional.ofNullable(srcom.getStorageClass()) - .ifPresent(copyObjectRequest::setStorageClass); incrementStatistic(OBJECT_COPY_REQUESTS); - Copy copy = transfers.copy(copyObjectRequest); + Copy copy = transfers.copy(copyObjectRequest, + getAuditManager().createStateChangeListener()); copy.addProgressListener(progressListener); CopyOutcome copyOutcome = CopyOutcome.waitForCopy(copy); InterruptedException interruptedException = @@ -3810,62 +4203,6 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, }); } - /** - * Propagate encryption parameters from source file if set else use the - * current filesystem encryption settings. - * @param srcom source object meta. - * @param copyObjectRequest copy object request body. - */ - private void setOptionalCopyObjectRequestParameters( - ObjectMetadata srcom, - CopyObjectRequest copyObjectRequest) { - String sourceKMSId = srcom.getSSEAwsKmsKeyId(); - if (isNotEmpty(sourceKMSId)) { - // source KMS ID is propagated - LOG.debug("Propagating SSE-KMS settings from source {}", - sourceKMSId); - copyObjectRequest.setSSEAwsKeyManagementParams( - new SSEAwsKeyManagementParams(sourceKMSId)); - } - switch(getServerSideEncryptionAlgorithm()) { - /** - * Overriding with client encryption settings. - */ - case SSE_C: - generateSSECustomerKey().ifPresent(customerKey -> { - copyObjectRequest.setSourceSSECustomerKey(customerKey); - copyObjectRequest.setDestinationSSECustomerKey(customerKey); - }); - break; - case SSE_KMS: - generateSSEAwsKeyParams().ifPresent( - copyObjectRequest::setSSEAwsKeyManagementParams); - break; - default: - } - } - - /** - * Set the optional parameters when initiating the request (encryption, - * headers, storage, etc). - * @param request request to patch. - */ - protected void setOptionalMultipartUploadRequestParameters( - InitiateMultipartUploadRequest request) { - generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams); - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); - } - - /** - * Sets server side encryption parameters to the part upload - * request when encryption is enabled. - * @param request upload part request - */ - protected void setOptionalUploadPartRequestParameters( - UploadPartRequest request) { - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); - } - /** * Initiate a multipart upload from the preconfigured request. * Retry policy: none + untranslated. @@ -3878,42 +4215,9 @@ protected void setOptionalUploadPartRequestParameters( InitiateMultipartUploadResult initiateMultipartUpload( InitiateMultipartUploadRequest request) throws IOException { LOG.debug("Initiate multipart upload to {}", request.getKey()); - incrementStatistic(OBJECT_MULTIPART_UPLOAD_INITIATED); - return getAmazonS3Client().initiateMultipartUpload(request); - } - - private void setOptionalPutRequestParameters(PutObjectRequest request) { - generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams); - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); - } - - private void setOptionalObjectMetadata(ObjectMetadata metadata) { - final S3AEncryptionMethods algorithm - = getServerSideEncryptionAlgorithm(); - if (S3AEncryptionMethods.SSE_S3.equals(algorithm)) { - metadata.setSSEAlgorithm(algorithm.getMethod()); - } - } - - /** - * Create the AWS SDK structure used to configure SSE, - * if the encryption secrets contain the information/settings for this. - * @return an optional set of KMS Key settings - */ - private Optional generateSSEAwsKeyParams() { - return EncryptionSecretOperations.createSSEAwsKeyManagementParams( - encryptionSecrets); - } - - /** - * Create the SSE-C structure for the AWS SDK, if the encryption secrets - * contain the information/settings for this. - * This will contain a secret extracted from the bucket/configuration. - * @return an optional customer key. - */ - private Optional generateSSECustomerKey() { - return EncryptionSecretOperations.createSSECustomerKey( - encryptionSecrets); + return trackDurationOfSupplier(getDurationTrackerFactory(), + OBJECT_MULTIPART_UPLOAD_INITIATED.getSymbol(), + () -> getAmazonS3Client().initiateMultipartUpload(request)); } /** @@ -3958,7 +4262,7 @@ void finishedWrite(String key, long length, String eTag, String versionId, CompletableFuture deletion; if (!keepDirectoryMarkers(p)) { deletion = submit( - unboundedThreadPool, + unboundedThreadPool, getActiveAuditSpan(), () -> { deleteUnnecessaryFakeDirectories( p.getParent(), @@ -4077,11 +4381,7 @@ private void deleteUnnecessaryFakeDirectories(Path path, @Retries.RetryTranslated private void createFakeDirectory(final String objectName) throws IOException { - if (!objectName.endsWith("/")) { - createEmptyObject(objectName + "/"); - } else { - createEmptyObject(objectName); - } + createEmptyObject(objectName); } /** @@ -4093,36 +4393,14 @@ private void createFakeDirectory(final String objectName) @Retries.RetryTranslated private void createEmptyObject(final String objectName) throws IOException { - final InputStream im = new InputStream() { - @Override - public int read() throws IOException { - return -1; - } - }; - - PutObjectRequest putObjectRequest = newPutObjectRequest(objectName, - newObjectMetadata(0L), - im); invoker.retry("PUT 0-byte object ", objectName, - true, - () -> putObjectDirect(putObjectRequest)); + true, () -> + putObjectDirect(getRequestFactory() + .newDirectoryMarkerRequest(objectName))); incrementPutProgressStatistics(objectName, 0); instrumentation.directoryCreated(); } - /** - * Creates a copy of the passed {@link ObjectMetadata}. - * Does so without using the {@link ObjectMetadata#clone()} method, - * to avoid copying unnecessary headers. - * @param source the {@link ObjectMetadata} to copy - * @return a copy of {@link ObjectMetadata} with only relevant attributes - */ - private ObjectMetadata cloneObjectMetadata(ObjectMetadata source) { - ObjectMetadata ret = newObjectMetadata(source.getContentLength()); - getHeaderProcessing().cloneObjectMetadata(source, ret); - return ret; - } - /** * Return the number of bytes that large input files should be optimally * be split into to minimize I/O time. @@ -4167,6 +4445,7 @@ public String toString() { if (blockFactory != null) { sb.append(", blockFactory=").append(blockFactory); } + sb.append(", auditManager=").append(auditManager); sb.append(", metastore=").append(metadataStore); sb.append(", authoritativeStore=").append(allowAuthoritativeMetadataStore); sb.append(", authoritativePath=").append(allowAuthoritativePaths); @@ -4247,20 +4526,26 @@ public FileStatus[] globStatus(Path pathPattern) throws IOException { * Increments the statistic {@link Statistic#INVOCATION_GLOB_STATUS}. * Override superclass so as to disable symlink resolution as symlinks * are not supported by S3A. + * + * Although an AuditEntryPoint, the globber itself will talk do + * the filesystem through the filesystem API, so its operations will + * all appear part of separate operations. * {@inheritDoc} */ @Override + @AuditEntryPoint public FileStatus[] globStatus( final Path pathPattern, final PathFilter filter) throws IOException { - entryPoint(INVOCATION_GLOB_STATUS); - return Globber.createGlobber(this) - .withPathPattern(pathPattern) - .withPathFiltern(filter) - .withResolveSymlinks(false) - .build() - .glob(); + return trackDurationAndSpan( + INVOCATION_GLOB_STATUS, pathPattern, () -> + Globber.createGlobber(this) + .withPathPattern(pathPattern) + .withPathFiltern(filter) + .withResolveSymlinks(false) + .build() + .glob()); } /** @@ -4268,9 +4553,17 @@ public FileStatus[] globStatus( * {@inheritDoc} */ @Override + @AuditEntryPoint public boolean exists(Path f) throws IOException { - entryPoint(INVOCATION_EXISTS); - return super.exists(f); + final Path path = qualify(f); + try { + trackDurationAndSpan( + INVOCATION_EXISTS, path, () -> + innerGetFileStatus(path, false, StatusProbeEnum.ALL)); + return true; + } catch (FileNotFoundException e) { + return false; + } } /** @@ -4280,12 +4573,15 @@ public boolean exists(Path f) throws IOException { * {@inheritDoc} */ @Override + @AuditEntryPoint @SuppressWarnings("deprecation") public boolean isDirectory(Path f) throws IOException { - entryPoint(INVOCATION_IS_DIRECTORY); + final Path path = qualify(f); try { - return innerGetFileStatus(f, false, StatusProbeEnum.DIRECTORIES) - .isDirectory(); + return trackDurationAndSpan( + INVOCATION_IS_DIRECTORY, path, () -> + innerGetFileStatus(path, false, StatusProbeEnum.DIRECTORIES) + .isDirectory()); } catch (FileNotFoundException e) { // not found or it is a file. return false; @@ -4299,12 +4595,14 @@ public boolean isDirectory(Path f) throws IOException { * {@inheritDoc} */ @Override + @AuditEntryPoint @SuppressWarnings("deprecation") public boolean isFile(Path f) throws IOException { - entryPoint(INVOCATION_IS_FILE); + final Path path = qualify(f); try { - return innerGetFileStatus(f, false, StatusProbeEnum.HEAD_ONLY) - .isFile(); + return trackDurationAndSpan(INVOCATION_IS_FILE, path, () -> + innerGetFileStatus(path, false, StatusProbeEnum.HEAD_ONLY) + .isFile()); } catch (FileNotFoundException e) { // not found or it is a dir. return false; @@ -4332,18 +4630,21 @@ public boolean isFile(Path f) throws IOException { */ @Override @Retries.RetryTranslated + @AuditEntryPoint public EtagChecksum getFileChecksum(Path f, final long length) throws IOException { Preconditions.checkArgument(length >= 0); - entryPoint(INVOCATION_GET_FILE_CHECKSUM); - + final Path path = qualify(f); if (getConf().getBoolean(ETAG_CHECKSUM_ENABLED, ETAG_CHECKSUM_ENABLED_DEFAULT)) { - Path path = qualify(f); - LOG.debug("getFileChecksum({})", path); - ObjectMetadata headers = getObjectMetadata(path); - String eTag = headers.getETag(); - return eTag != null ? new EtagChecksum(eTag) : null; + return trackDurationAndSpan(INVOCATION_GET_FILE_CHECKSUM, path, () -> { + LOG.debug("getFileChecksum({})", path); + ObjectMetadata headers = getObjectMetadata(path, null, + invoker, + "getFileChecksum are"); + String eTag = headers.getETag(); + return eTag != null ? new EtagChecksum(eTag) : null; + }); } else { // disabled return null; @@ -4352,35 +4653,83 @@ public EtagChecksum getFileChecksum(Path f, final long length) /** * Get header processing support. - * @return the header processing of this instance. + * @return a new header processing instance. */ private HeaderProcessing getHeaderProcessing() { - return headerProcessing; + return new HeaderProcessing(createStoreContext(), + createHeaderProcessingCallbacks()); } @Override + @AuditEntryPoint public byte[] getXAttr(final Path path, final String name) throws IOException { - return getHeaderProcessing().getXAttr(path, name); + checkNotClosed(); + try (AuditSpan span = createSpan( + INVOCATION_XATTR_GET_NAMED.getSymbol(), + path.toString(), null)) { + return getHeaderProcessing().getXAttr(path, name); + } } @Override + @AuditEntryPoint public Map getXAttrs(final Path path) throws IOException { - return getHeaderProcessing().getXAttrs(path); + checkNotClosed(); + try (AuditSpan span = createSpan( + INVOCATION_XATTR_GET_MAP.getSymbol(), + path.toString(), null)) { + return getHeaderProcessing().getXAttrs(path); + } } @Override + @AuditEntryPoint public Map getXAttrs(final Path path, final List names) throws IOException { - return getHeaderProcessing().getXAttrs(path, names); + checkNotClosed(); + try (AuditSpan span = createSpan( + INVOCATION_XATTR_GET_NAMED_MAP.getSymbol(), + path.toString(), null)) { + return getHeaderProcessing().getXAttrs(path, names); + } } @Override + @AuditEntryPoint public List listXAttrs(final Path path) throws IOException { - return getHeaderProcessing().listXAttrs(path); + checkNotClosed(); + try (AuditSpan span = createSpan( + INVOCATION_OP_XATTR_LIST.getSymbol(), + path.toString(), null)) { + return getHeaderProcessing().listXAttrs(path); + } + } + + /** + * Create the callbacks. + * @return An implementation of the header processing + * callbacks. + */ + protected HeaderProcessing.HeaderProcessingCallbacks + createHeaderProcessingCallbacks() { + return new HeaderProcessingCallbacksImpl(); } + /** + * Operations needed for Header Processing. + */ + protected final class HeaderProcessingCallbacksImpl implements + HeaderProcessing.HeaderProcessingCallbacks { + + @Override + public ObjectMetadata getObjectMetadata(final String key) + throws IOException { + return once("getObjectMetadata", key, () -> + S3AFileSystem.this.getObjectMetadata(key)); + } + } /** * {@inheritDoc}. * @@ -4408,38 +4757,53 @@ public List listXAttrs(final Path path) throws IOException { */ @Override @Retries.RetryTranslated + @AuditEntryPoint public RemoteIterator listFiles(Path f, boolean recursive) throws FileNotFoundException, IOException { - return toLocatedFileStatusIterator(innerListFiles(f, recursive, - new Listing.AcceptFilesOnly(qualify(f)), null, true, false)); + final Path path = qualify(f); + return toLocatedFileStatusIterator( + trackDurationAndSpan(INVOCATION_LIST_FILES, path, () -> + innerListFiles(path, recursive, + new Listing.AcceptFilesOnly(path), null, true, false))); } /** * Recursive List of files and empty directories. * @param f path to list from + * @param recursive recursive? * @return an iterator. * @throws IOException failure */ + @InterfaceAudience.Private @Retries.RetryTranslated + @AuditEntryPoint public RemoteIterator listFilesAndEmptyDirectories( Path f, boolean recursive) throws IOException { - return innerListFiles(f, recursive, Listing.ACCEPT_ALL_BUT_S3N, - null, true, false); + final Path path = qualify(f); + return trackDurationAndSpan(INVOCATION_LIST_FILES, path, () -> + innerListFiles(path, recursive, + Listing.ACCEPT_ALL_BUT_S3N, + null, true, false)); } /** * Recursive List of files and empty directories, force metadatastore * to act like it is non-authoritative. * @param f path to list from - * @param recursive + * @param recursive recursive listing? * @return an iterator. * @throws IOException failure */ + @InterfaceAudience.Private @Retries.RetryTranslated + @AuditEntryPoint public RemoteIterator listFilesAndEmptyDirectoriesForceNonAuth( Path f, boolean recursive) throws IOException { - return innerListFiles(f, recursive, Listing.ACCEPT_ALL_BUT_S3N, - null, true, true); + final Path path = qualify(f); + return trackDurationAndSpan(INVOCATION_LIST_FILES, path, () -> + innerListFiles(path, recursive, + Listing.ACCEPT_ALL_BUT_S3N, + null, true, true)); } /** @@ -4488,7 +4852,6 @@ private RemoteIterator innerListFiles( final S3AFileStatus status, final boolean collectTombstones, final boolean forceNonAuthoritativeMS) throws IOException { - entryPoint(INVOCATION_LIST_FILES); Path path = qualify(f); LOG.debug("listFiles({}, {})", path, recursive); try { @@ -4506,7 +4869,8 @@ private RemoteIterator innerListFiles( recursive, acceptor, collectTombstones, - forceNonAuthoritativeMS); + forceNonAuthoritativeMS, + getActiveAuditSpan()); // If there are no list entries present, we // fallback to file existence check as the path // can be a file or empty directory. @@ -4514,7 +4878,7 @@ private RemoteIterator innerListFiles( // If file status was already passed, reuse it. final S3AFileStatus fileStatus = status != null ? status - : (S3AFileStatus) getFileStatus(path); + : innerGetFileStatus(path, false, StatusProbeEnum.ALL); if (fileStatus.isFile()) { return listing.createSingleStatusIterator( toLocatedFileStatus(fileStatus)); @@ -4553,11 +4917,12 @@ public RemoteIterator listLocatedStatus(Path f) */ @Override @Retries.OnceTranslated("s3guard not retrying") + @AuditEntryPoint public RemoteIterator listLocatedStatus(final Path f, final PathFilter filter) throws FileNotFoundException, IOException { - entryPoint(INVOCATION_LIST_LOCATED_STATUS); Path path = qualify(f); + AuditSpan span = entryPoint(INVOCATION_LIST_LOCATED_STATUS, path); LOG.debug("listLocatedStatus({}, {}", path, filter); RemoteIterator iterator = once("listLocatedStatus", path.toString(), @@ -4566,12 +4931,13 @@ public RemoteIterator listLocatedStatus(final Path f, // trigger a list call directly. final RemoteIterator locatedFileStatusIteratorForDir = - listing.getLocatedFileStatusIteratorForDir(path, filter); + listing.getLocatedFileStatusIteratorForDir(path, filter, + span); // If no listing is present then path might be a file. if (!locatedFileStatusIteratorForDir.hasNext()) { final S3AFileStatus fileStatus = - (S3AFileStatus) getFileStatus(path); + innerGetFileStatus(path, false, StatusProbeEnum.ALL); if (fileStatus.isFile()) { // simple case: File LOG.debug("Path is a file"); @@ -4587,43 +4953,6 @@ public RemoteIterator listLocatedStatus(final Path f, return toLocatedFileStatusIterator(iterator); } - /** - * Generate list located status for a directory. - * Also performing tombstone reconciliation for guarded directories. - * @param dir directory to check. - * @param filter a path filter. - * @return an iterator that traverses statuses of the given dir. - * @throws IOException in case of failure. - */ - private RemoteIterator getLocatedFileStatusIteratorForDir( - Path dir, PathFilter filter) throws IOException { - final String key = maybeAddTrailingSlash(pathToKey(dir)); - final Listing.FileStatusAcceptor acceptor = - new Listing.AcceptAllButSelfAndS3nDirs(dir); - boolean allowAuthoritative = allowAuthoritative(dir); - DirListingMetadata meta = - S3Guard.listChildrenWithTtl(metadataStore, dir, - ttlTimeProvider, allowAuthoritative); - Set tombstones = meta != null - ? meta.listTombstones() - : null; - final RemoteIterator cachedFileStatusIterator = - listing.createProvidedFileStatusIterator( - dirMetaToStatuses(meta), filter, acceptor); - return (allowAuthoritative && meta != null - && meta.isAuthoritative()) - ? listing.createLocatedFileStatusIterator( - cachedFileStatusIterator) - : listing.createTombstoneReconcilingIterator( - listing.createLocatedFileStatusIterator( - listing.createFileStatusListingIterator(dir, - createListObjectsRequest(key, "/"), - filter, - acceptor, - cachedFileStatusIterator)), - tombstones); - } - /** * Build a {@link S3ALocatedFileStatus} from a {@link FileStatus} instance. * @param status file status @@ -4648,10 +4977,17 @@ S3ALocatedFileStatus toLocatedFileStatus(S3AFileStatus status) * @return Iterator over multipart uploads. * @throws IOException on failure */ + @InterfaceAudience.Private + @Retries.RetryTranslated + @AuditEntryPoint public MultipartUtils.UploadIterator listUploads(@Nullable String prefix) throws IOException { - return MultipartUtils.listMultipartUploads(s3, invoker, bucket, maxKeys, - prefix); + // span is picked up retained in the listing. + return trackDurationAndSpan(MULTIPART_UPLOAD_LIST, prefix, null, () -> + MultipartUtils.listMultipartUploads( + createStoreContext(), + s3, prefix, maxKeys + )); } /** @@ -4667,17 +5003,16 @@ public MultipartUtils.UploadIterator listUploads(@Nullable String prefix) @Retries.RetryTranslated public List listMultipartUploads(String prefix) throws IOException { - ListMultipartUploadsRequest request = new ListMultipartUploadsRequest( - bucket); - if (!prefix.isEmpty()) { - if (!prefix.endsWith("/")) { - prefix = prefix + "/"; - } - request.setPrefix(prefix); + // add a trailing / if needed. + if (prefix != null && !prefix.isEmpty() && !prefix.endsWith("/")) { + prefix = prefix + "/"; } - - return invoker.retry("listMultipartUploads", prefix, true, - () -> s3.listMultipartUploads(request).getMultipartUploads()); + String p = prefix; + return invoker.retry("listMultipartUploads", p, true, () -> { + ListMultipartUploadsRequest request = getRequestFactory() + .newListMultipartUploadsRequest(p); + return s3.listMultipartUploads(request).getMultipartUploads(); + }); } /** @@ -4690,7 +5025,7 @@ public List listMultipartUploads(String prefix) void abortMultipartUpload(String destKey, String uploadId) { LOG.info("Aborting multipart upload {} to {}", uploadId, destKey); getAmazonS3Client().abortMultipartUpload( - new AbortMultipartUploadRequest(getBucket(), + getRequestFactory().newAbortMultipartUploadRequest( destKey, uploadId)); } @@ -4713,7 +5048,7 @@ void abortMultipartUpload(MultipartUpload upload) { df.format(upload.getInitiated())); } getAmazonS3Client().abortMultipartUpload( - new AbortMultipartUploadRequest(getBucket(), + getRequestFactory().newAbortMultipartUploadRequest( destKey, uploadId)); } @@ -4741,7 +5076,7 @@ public boolean hasPathCapability(final Path path, final String capability) case SelectConstants.S3_SELECT_CAPABILITY: // select is only supported if enabled - return selectBinding.isEnabled(); + return SelectBinding.isSelectEnabled(getConf()); case CommonPathCapabilities.FS_CHECKSUMS: // capability depends on FS configuration @@ -4818,8 +5153,6 @@ protected void setTtlTimeProvider(ITtlTimeProvider ttlTimeProvider) { /** * This is a proof of concept of a select API. - * Once a proper factory mechanism for opening files is added to the - * FileSystem APIs, this will be deleted without any warning. * @param source path to source data * @param expression select expression * @param options request configuration from the builder. @@ -4828,12 +5161,13 @@ protected void setTtlTimeProvider(ITtlTimeProvider ttlTimeProvider) { * @throws IOException IO failure */ @Retries.RetryTranslated + @AuditEntryPoint private FSDataInputStream select(final Path source, final String expression, final Configuration options, final Optional providedStatus) throws IOException { - entryPoint(OBJECT_SELECT_REQUESTS); + final AuditSpan auditSpan = entryPoint(OBJECT_SELECT_REQUESTS, source); requireSelectSupport(source); final Path path = makeQualified(source); final S3AFileStatus fileStatus = extractOrFetchSimpleFileStatus(path, @@ -4843,7 +5177,7 @@ private FSDataInputStream select(final Path source, long ra = options.getLong(READAHEAD_RANGE, readAhead); S3ObjectAttributes objectAttributes = createObjectAttributes(fileStatus); S3AReadOpContext readContext = createReadContext(fileStatus, inputPolicy, - changeDetectionPolicy, ra); + changeDetectionPolicy, ra, auditSpan); if (changeDetectionPolicy.getSource() != ChangeDetectionPolicy.Source.None && fileStatus.getETag() != null) { @@ -4867,13 +5201,16 @@ private FSDataInputStream select(final Path source, Invoker readInvoker = readContext.getReadInvoker(); getObjectMetadata(path, changeTracker, readInvoker, "select"); } + // instantiate S3 Select support using the current span + // as the active span for operations. + SelectBinding selectBinding = new SelectBinding( + createWriteOperationHelper(auditSpan)); // build and execute the request return selectBinding.select( readContext, expression, options, - generateSSECustomerKey(), objectAttributes); } @@ -4884,7 +5221,8 @@ private FSDataInputStream select(final Path source, */ private void requireSelectSupport(final Path source) throws UnsupportedOperationException { - if (!selectBinding.isEnabled()) { + if (!SelectBinding.isSelectEnabled(getConf())) { + throw new UnsupportedOperationException( SelectConstants.SELECT_UNSUPPORTED); } @@ -4923,7 +5261,9 @@ private S3AFileStatus extractOrFetchSimpleFileStatus( /** * Initiate the open or select operation. - * This is invoked from both the FileSystem and FileContext APIs + * This is invoked from both the FileSystem and FileContext APIs. + * It's declared as an audit entry point but the span creation is pushed + * down into the open/select methods it ultimately calls. * @param rawPath path to the file * @param parameters open file parameters from the builder. * @return a future which will evaluate to the opened/selected file. @@ -4934,6 +5274,7 @@ private S3AFileStatus extractOrFetchSimpleFileStatus( */ @Override @Retries.RetryTranslated + @AuditEntryPoint public CompletableFuture openFileWithOptions( final Path rawPath, final OpenFileParameters parameters) throws IOException { @@ -4999,15 +5340,19 @@ public CompletableFuture openFileWithOptions( } @Override + @AuditEntryPoint public S3AMultipartUploaderBuilder createMultipartUploader( final Path basePath) throws IOException { - StoreContext ctx = createStoreContext(); - return new S3AMultipartUploaderBuilder(this, - getWriteOperationHelper(), - ctx, - basePath, - statisticsContext.createMultipartUploaderStatistics()); + final Path path = makeQualified(basePath); + try (AuditSpan span = entryPoint(MULTIPART_UPLOAD_INSTANTIATED, path)) { + StoreContext ctx = createStoreContext(); + return new S3AMultipartUploaderBuilder(this, + createWriteOperationHelper(span), + ctx, + path, + statisticsContext.createMultipartUploaderStatistics()); + } } /** @@ -5036,16 +5381,24 @@ public StoreContext createStoreContext() { .setUseListV1(useListV1) .setContextAccessors(new ContextAccessorsImpl()) .setTimeProvider(getTtlTimeProvider()) + .setAuditor(getAuditor()) .build(); } /** * Create a marker tools operations binding for this store. + * Auditing: + * @param target target path * @return callbacks for operations. + * @throws IOException if raised during span creation */ + @AuditEntryPoint @InterfaceAudience.Private - public MarkerToolOperations createMarkerToolOperations() { - return new MarkerToolOperationsImpl(operationCallbacks); + public MarkerToolOperations createMarkerToolOperations(final String target) + throws IOException { + createSpan("marker-tool-scan", target, + null); + return new MarkerToolOperationsImpl(new OperationCallbacksImpl()); } /** @@ -5089,10 +5442,13 @@ public Path makeQualified(final Path path) { } @Override - public ObjectMetadata getObjectMetadata(final String key) - throws IOException { - return once("getObjectMetadata", key, () -> - S3AFileSystem.this.getObjectMetadata(key)); + public AuditSpan getActiveAuditSpan() { + return S3AFileSystem.this.getActiveAuditSpan(); + } + + @Override + public RequestFactory getRequestFactory() { + return S3AFileSystem.this.getRequestFactory(); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java index c725fdf37ff8a..d56d4ac433eb7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java @@ -20,14 +20,11 @@ import javax.annotation.Nullable; -import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.S3Object; import com.amazonaws.services.s3.model.S3ObjectInputStream; -import com.amazonaws.services.s3.model.SSECustomerKey; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.CanSetReadahead; @@ -45,6 +42,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Closeable; import java.io.EOFException; import java.io.IOException; import java.net.SocketTimeoutException; @@ -101,7 +99,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, private S3Object object; private S3ObjectInputStream wrappedStream; private final S3AReadOpContext context; - private final AmazonS3 client; + private final InputStreamCallbacks client; private final String bucket; private final String key; private final String pathStr; @@ -110,8 +108,6 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, private static final Logger LOG = LoggerFactory.getLogger(S3AInputStream.class); private final S3AInputStreamStatistics streamStatistics; - private S3AEncryptionMethods serverSideEncryptionAlgorithm; - private String serverSideEncryptionKey; private S3AInputPolicy inputPolicy; private long readahead = Constants.DEFAULT_READAHEAD_RANGE; @@ -150,7 +146,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, */ public S3AInputStream(S3AReadOpContext ctx, S3ObjectAttributes s3Attributes, - AmazonS3 client) { + InputStreamCallbacks client) { Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()), "No Bucket"); Preconditions.checkArgument(isNotEmpty(s3Attributes.getKey()), "No Key"); @@ -166,9 +162,6 @@ public S3AInputStream(S3AReadOpContext ctx, this.streamStatistics = ctx.getS3AStatisticsContext() .newInputStreamStatistics(); this.ioStatistics = streamStatistics.getIOStatistics(); - this.serverSideEncryptionAlgorithm = - s3Attributes.getServerSideEncryptionAlgorithm(); - this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey(); this.changeTracker = new ChangeTracker(uri, ctx.getChangeDetectionPolicy(), streamStatistics.getChangeTrackerStatistics(), @@ -211,16 +204,13 @@ private synchronized void reopen(String reason, long targetPos, long length, inputPolicy); long opencount = streamStatistics.streamOpened(); - GetObjectRequest request = new GetObjectRequest(bucket, key) + GetObjectRequest request = client.newGetRequest(key) .withRange(targetPos, contentRangeFinish - 1); - if (S3AEncryptionMethods.SSE_C.equals(serverSideEncryptionAlgorithm) && - StringUtils.isNotBlank(serverSideEncryptionKey)){ - request.setSSECustomerKey(new SSECustomerKey(serverSideEncryptionKey)); - } String operation = opencount == 0 ? OPERATION_OPEN : OPERATION_REOPEN; String text = String.format("%s %s at %d", operation, uri, targetPos); changeTracker.maybeApplyConstraint(request); + DurationTracker tracker = streamStatistics.initiateGetRequest(); try { object = Invoker.once(text, uri, @@ -567,6 +557,8 @@ public synchronized void close() throws IOException { // close or abort the stream closeStream("close() operation", this.contentRangeFinish, false); LOG.debug("Statistics of stream {}\n{}", key, streamStatistics); + // end the client+audit span. + client.close(); // this is actually a no-op super.close(); } finally { @@ -908,4 +900,27 @@ boolean isObjectStreamOpen() { public IOStatistics getIOStatistics() { return ioStatistics; } + + /** + * Callbacks for input stream IO. + */ + public interface InputStreamCallbacks extends Closeable { + + /** + * Create a GET request. + * @param key object key + * @return the request + */ + GetObjectRequest newGetRequest(String key); + + /** + * Execute the request. + * @param request the request + * @return the response + */ + @Retries.OnceRaw + S3Object getObject(GetObjectRequest request); + + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index 169a74abef2d7..a185bac2cb4b0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -28,15 +28,16 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation; -import org.apache.hadoop.fs.s3a.statistics.impl.AbstractS3AStatisticsSource; +import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; import org.apache.hadoop.fs.s3a.statistics.CountersAndGauges; -import org.apache.hadoop.fs.s3a.statistics.impl.CountingChangeTracker; import org.apache.hadoop.fs.s3a.statistics.DelegationTokenStatistics; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; -import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum; +import org.apache.hadoop.fs.s3a.statistics.impl.AbstractS3AStatisticsSource; +import org.apache.hadoop.fs.s3a.statistics.impl.CountingChangeTracker; +import org.apache.hadoop.fs.s3a.statistics.impl.ForwardingIOStatisticsStore; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.fs.statistics.IOStatisticsLogging; import org.apache.hadoop.fs.statistics.IOStatisticsSource; @@ -468,6 +469,14 @@ public DurationTracker trackDuration(final String key, final long count) { return durationTrackerFactory.trackDuration(key, count); } + /** + * Create an IOStatistics store which updates FS metrics + * as well as IOStatistics. + */ + public IOStatisticsStore createMetricsUpdatingStore() { + return new MetricsUpdatingIOStatisticsStore(); + } + /** * String representation. Includes the IOStatistics * when logging is at DEBUG. @@ -547,10 +556,24 @@ public void errorIgnored() { * @param count increment value */ public void incrementCounter(Statistic op, long count) { - String name = op.getSymbol(); + incrementNamedCounter(op.getSymbol(), count); + } + + /** + * Increments a mutable counter and the matching + * instance IOStatistics counter. + * No-op if the counter is not defined, or the count == 0. + * @param name counter name + * @param count increment value + * @return the updated value or, if the counter is unknown: 0 + */ + private long incrementNamedCounter(final String name, + final long count) { if (count != 0) { incrementMutableCounter(name, count); - instanceIOStatistics.incrementCounter(name, count); + return instanceIOStatistics.incrementCounter(name, count); + } else { + return 0; } } @@ -1868,4 +1891,43 @@ public Map getMap() { return map; } } + + /** + * An IOStatisticsStore which updates metrics on calls to + * {@link #incrementCounter(String, long)}. + * This helps keeps FS metrics and IOStats in sync. + * Duration tracking methods are forwarded to + * the S3A Instrumentation duration tracker, which will + * update the instance IOStatistics. + */ + private final class MetricsUpdatingIOStatisticsStore + extends ForwardingIOStatisticsStore { + + private MetricsUpdatingIOStatisticsStore() { + super(S3AInstrumentation.this.getIOStatistics()); + } + + /** + * Incrementing the counter also implements the metric alongside + * the IOStatistics value. + * @param key counter key + * @param value increment value. + * @return the value in the wrapped IOStatistics. + */ + @Override + public long incrementCounter(final String key, final long value) { + incrementMutableCounter(key, value); + return super.incrementCounter(key, value); + } + + @Override + public DurationTracker trackDuration(final String key, final long count) { + return S3AInstrumentation.this.trackDuration(key, count); + } + + @Override + public DurationTracker trackDuration(final String key) { + return S3AInstrumentation.this.trackDuration(key); + } + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ALocatedFileStatus.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ALocatedFileStatus.java index 10295484fe815..725a7fe0971b8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ALocatedFileStatus.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ALocatedFileStatus.java @@ -66,6 +66,7 @@ public int hashCode() { /** * Generate an S3AFileStatus instance, including etag and * version ID, if present. + * @return the S3A status. */ public S3AFileStatus toS3AFileStatus() { return new S3AFileStatus( diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java index 3729341dbfe27..43398fc4a167b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; +import org.apache.hadoop.fs.store.audit.AuditSpan; import javax.annotation.Nullable; @@ -55,6 +56,8 @@ public class S3AReadOpContext extends S3AOpContext { */ private final long readahead; + private final AuditSpan auditSpan; + /** * Instantiate. * @param path path of read @@ -65,8 +68,9 @@ public class S3AReadOpContext extends S3AOpContext { * @param instrumentation statistics context * @param dstFileStatus target file status * @param inputPolicy the input policy - * @param readahead readahead for GET operations/skip, etc. * @param changeDetectionPolicy change detection policy. + * @param readahead readahead for GET operations/skip, etc. + * @param auditSpan active audit */ public S3AReadOpContext( final Path path, @@ -78,11 +82,13 @@ public S3AReadOpContext( FileStatus dstFileStatus, S3AInputPolicy inputPolicy, ChangeDetectionPolicy changeDetectionPolicy, - final long readahead) { + final long readahead, + final AuditSpan auditSpan) { super(isS3GuardEnabled, invoker, s3guardInvoker, stats, instrumentation, dstFileStatus); this.path = checkNotNull(path); + this.auditSpan = auditSpan; Preconditions.checkArgument(readahead >= 0, "invalid readahead %d", readahead); this.inputPolicy = checkNotNull(inputPolicy); @@ -133,6 +139,14 @@ public long getReadahead() { return readahead; } + /** + * Get the audit which was active when the file was opened. + * @return active span + */ + public AuditSpan getAuditSpan() { + return auditSpan; + } + @Override public String toString() { final StringBuilder sb = new StringBuilder( diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java index 314f13f0352e9..b6af8e7f27c20 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java @@ -34,7 +34,6 @@ import com.amazonaws.services.s3.model.AmazonS3Exception; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import com.amazonaws.services.s3.model.S3ObjectSummary; - import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -42,7 +41,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; @@ -91,6 +89,7 @@ import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.translateDeleteException; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; +import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator; /** * Utility methods for S3A code. @@ -201,7 +200,8 @@ public static IOException translateException(@Nullable String operation, } if (exception instanceof CredentialInitializationException) { // the exception raised by AWSCredentialProvider list if the - // credentials were not accepted. + // credentials were not accepted, + // or auditing blocked the operation. return (AccessDeniedException)new AccessDeniedException(path, null, exception.toString()).initCause(exception); } @@ -1423,23 +1423,19 @@ private static void initUserAgent(Configuration conf, * an array. Given tombstones are filtered out. If the iterator * does return any item, an empty array is returned. * @param iterator a non-null iterator - * @param tombstones + * @param tombstones possibly empty set of tombstones * @return a possibly-empty array of file status entries - * @throws IOException + * @throws IOException failure */ public static S3AFileStatus[] iteratorToStatuses( RemoteIterator iterator, Set tombstones) throws IOException { - List statuses = new ArrayList<>(); - - while (iterator.hasNext()) { - S3AFileStatus status = iterator.next(); - if (!tombstones.contains(status.getPath())) { - statuses.add(status); - } - } - - return statuses.toArray(new S3AFileStatus[0]); + // this will close the span afterwards + RemoteIterator source = filteringRemoteIterator(iterator, + st -> !tombstones.contains(st.getPath())); + S3AFileStatus[] statuses = RemoteIterators + .toArray(source, new S3AFileStatus[0]); + return statuses; } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java index dbb39fb662408..5ef99ed6f5c3c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java @@ -115,6 +115,11 @@ final class S3ClientCreationParameters { */ private String userAgentSuffix = ""; + /** + * List of request handlers to include in the chain + * of request execution in the SDK. + * @return the handler list + */ public List getRequestHandlers() { return requestHandlers; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index c613c06c9bb4e..7890e2d946b76 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -22,6 +22,7 @@ import java.util.Map; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.audit.AuditStatisticNames; import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum; import org.apache.hadoop.fs.statistics.StoreStatisticNames; import org.apache.hadoop.fs.statistics.StreamStatisticNames; @@ -30,6 +31,7 @@ import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_DURATION; import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_GAUGE; import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_QUANTILE; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_FAILURES; /** * Statistic which are collected in S3A. @@ -93,14 +95,18 @@ public enum Statistic { StoreStatisticNames.OP_ABORT, "Calls of abort()", TYPE_DURATION), + INVOCATION_ACCESS( + StoreStatisticNames.OP_ACCESS, + "Calls of access()", + TYPE_DURATION), INVOCATION_COPY_FROM_LOCAL_FILE( StoreStatisticNames.OP_COPY_FROM_LOCAL_FILE, "Calls of copyFromLocalFile()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_CREATE( StoreStatisticNames.OP_CREATE, "Calls of create()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_CREATE_NON_RECURSIVE( StoreStatisticNames.OP_CREATE_NON_RECURSIVE, "Calls of createNonRecursive()", @@ -108,35 +114,39 @@ public enum Statistic { INVOCATION_DELETE( StoreStatisticNames.OP_DELETE, "Calls of delete()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_EXISTS( StoreStatisticNames.OP_EXISTS, "Calls of exists()", - TYPE_COUNTER), + TYPE_DURATION), + INVOCATION_GET_CONTENT_SUMMARY( + StoreStatisticNames.OP_GET_CONTENT_SUMMARY, + "Calls of getContentSummary()", + TYPE_DURATION), INVOCATION_GET_DELEGATION_TOKEN( StoreStatisticNames.OP_GET_DELEGATION_TOKEN, "Calls of getDelegationToken()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_GET_FILE_CHECKSUM( StoreStatisticNames.OP_GET_FILE_CHECKSUM, "Calls of getFileChecksum()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_GET_FILE_STATUS( StoreStatisticNames.OP_GET_FILE_STATUS, "Calls of getFileStatus()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_GLOB_STATUS( StoreStatisticNames.OP_GLOB_STATUS, "Calls of globStatus()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_IS_DIRECTORY( StoreStatisticNames.OP_IS_DIRECTORY, "Calls of isDirectory()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_IS_FILE( StoreStatisticNames.OP_IS_FILE, "Calls of isFile()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_HFLUSH( StoreStatisticNames.OP_HFLUSH, "Calls of hflush()", @@ -148,7 +158,7 @@ public enum Statistic { INVOCATION_LIST_FILES( StoreStatisticNames.OP_LIST_FILES, "Calls of listFiles()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_LIST_LOCATED_STATUS( StoreStatisticNames.OP_LIST_LOCATED_STATUS, "Calls of listLocatedStatus()", @@ -156,11 +166,11 @@ public enum Statistic { INVOCATION_LIST_STATUS( StoreStatisticNames.OP_LIST_STATUS, "Calls of listStatus()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_MKDIRS( StoreStatisticNames.OP_MKDIRS, "Calls of mkdirs()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_OPEN( StoreStatisticNames.OP_OPEN, "Calls of open()", @@ -168,7 +178,7 @@ public enum Statistic { INVOCATION_RENAME( StoreStatisticNames.OP_RENAME, "Calls of rename()", - TYPE_COUNTER), + TYPE_DURATION), /* The XAttr API metrics are all durations */ INVOCATION_XATTR_GET_MAP( @@ -215,15 +225,15 @@ public enum Statistic { OBJECT_MULTIPART_UPLOAD_INITIATED( StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_INITIATED, "Object multipart upload initiated", - TYPE_COUNTER), + TYPE_DURATION), OBJECT_MULTIPART_UPLOAD_ABORTED( StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_ABORTED, "Object multipart upload aborted", - TYPE_COUNTER), + TYPE_DURATION), OBJECT_PUT_REQUESTS( StoreStatisticNames.OBJECT_PUT_REQUEST, "Object put/multipart upload count", - TYPE_COUNTER), + TYPE_DURATION), OBJECT_PUT_REQUESTS_COMPLETED( StoreStatisticNames.OBJECT_PUT_REQUEST_COMPLETED, "Object put/multipart upload completed count", @@ -421,7 +431,7 @@ public enum Statistic { "Count of bytes uploaded duing commit operations", TYPE_COUNTER), COMMITTER_COMMITS_FAILED( - "committer_commits"+ StoreStatisticNames.SUFFIX_FAILURES, + "committer_commits"+ SUFFIX_FAILURES, "Count of commits failed", TYPE_COUNTER), COMMITTER_COMMITS_ABORTED( @@ -487,6 +497,9 @@ public enum Statistic { /* General Store operations */ + STORE_EXISTS_PROBE(StoreStatisticNames.STORE_EXISTS_PROBE, + "Store Existence Probe", + TYPE_DURATION), STORE_IO_REQUEST(StoreStatisticNames.STORE_IO_REQUEST, "requests made of the remote store", TYPE_COUNTER), @@ -538,9 +551,32 @@ public enum Statistic { StoreStatisticNames.MULTIPART_UPLOAD_COMPLETED, "Multipart Upload Completed", TYPE_COUNTER), + MULTIPART_UPLOAD_LIST( + StoreStatisticNames.MULTIPART_UPLOAD_LIST, + "Multipart Upload List", + TYPE_DURATION), MULTIPART_UPLOAD_STARTED( StoreStatisticNames.MULTIPART_UPLOAD_STARTED, "Multipart Upload Started", + TYPE_COUNTER), + + /* Audit statistics. */ + /* If more are added: update AuditTestSupport to include the new values. */ + AUDIT_ACCESS_CHECK_FAILURE( + AuditStatisticNames.AUDIT_ACCESS_CHECK_FAILURE, + "Audit access check was rejected", + TYPE_COUNTER), + AUDIT_SPAN_CREATION( + AuditStatisticNames.AUDIT_SPAN_CREATION, + "Audit Span Created", + TYPE_COUNTER), + AUDIT_FAILURE( + AuditStatisticNames.AUDIT_FAILURE, + "Audit failure/rejection", + TYPE_COUNTER), + AUDIT_REQUEST_EXECUTION( + AuditStatisticNames.AUDIT_REQUEST_EXECUTION, + "AWS request made", TYPE_COUNTER); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java index 8b71fc3277142..2eb4841d9bd06 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java @@ -23,7 +23,6 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; @@ -51,19 +50,20 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.fs.s3a.s3guard.S3Guard; import org.apache.hadoop.fs.s3a.select.SelectBinding; +import org.apache.hadoop.fs.store.audit.AuditSpan; +import org.apache.hadoop.fs.store.audit.AuditSpanSource; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.functional.CallableRaisingIOE; -import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.Invoker.*; -import static org.apache.hadoop.fs.s3a.S3AUtils.longOption; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.UPLOAD_PART_COUNT_LIMIT; +import static org.apache.hadoop.fs.store.audit.AuditingFunctions.withinAuditSpan; /** * Helper for low-level operations against an S3 Bucket for writing data, @@ -87,6 +87,17 @@ * * * This API is for internal use only. + * Span scoping: This helper is instantiated with span; it will be used + * before operations which query/update S3 + * + * History + *
+ * - A nested class in S3AFileSystem
+ * - Single shared instance created and reused.
+ * - [HADOOP-13786] A separate class, single instance in S3AFS
+ * - [HDFS-13934] Split into interface and implementation
+ * - [HADOOP-15711] Adds audit tracking; one instance per use.
+ * 
*/ @InterfaceAudience.Private @InterfaceStability.Unstable @@ -116,22 +127,50 @@ public class WriteOperationHelper implements WriteOperations { */ private final S3AStatisticsContext statisticsContext; + /** + * Store Context; extracted from owner. + */ + private final StoreContext storeContext; + + /** + * Source of Audit spans. + */ + private final AuditSpanSource auditSpanSource; + + /** + * Audit Span. + */ + private AuditSpan auditSpan; + + /** + * Factory for AWS requests. + */ + private final RequestFactory requestFactory; + /** * Constructor. * @param owner owner FS creating the helper * @param conf Configuration object * @param statisticsContext statistics context + * @param auditSpanSource source of spans + * @param auditSpan span to activate * */ protected WriteOperationHelper(S3AFileSystem owner, Configuration conf, - S3AStatisticsContext statisticsContext) { + S3AStatisticsContext statisticsContext, + final AuditSpanSource auditSpanSource, + final AuditSpan auditSpan) { this.owner = owner; this.invoker = new Invoker(new S3ARetryPolicy(conf), this::operationRetried); this.conf = conf; this.statisticsContext = statisticsContext; - bucket = owner.getBucket(); + this.storeContext = owner.createStoreContext(); + this.bucket = owner.getBucket(); + this.auditSpanSource = auditSpanSource; + this.auditSpan = checkNotNull(auditSpan); + this.requestFactory = owner.getRequestFactory(); } /** @@ -150,6 +189,7 @@ void operationRetried(String text, Exception ex, int retries, /** * Execute a function with retry processing. + * Also activates the current span. * @param type of return value * @param action action to execute (used in error messages) * @param path path of work (used in error messages) @@ -164,10 +204,33 @@ public T retry(String action, boolean idempotent, CallableRaisingIOE operation) throws IOException { - + activateAuditSpan(); return invoker.retry(action, path, idempotent, operation); } + /** + * Get the audit span this object was created with. + * @return the audit span + */ + public AuditSpan getAuditSpan() { + return auditSpan; + } + + /** + * Activate the audit span. + * @return the span + */ + private AuditSpan activateAuditSpan() { + return auditSpan.activate(); + } + + /** + * Deactivate the audit span. + */ + private void deactivateAuditSpan() { + auditSpan.deactivate(); + } + /** * Create a {@link PutObjectRequest} request against the specific key. * @param destKey destination key @@ -176,15 +239,18 @@ public T retry(String action, * @param headers optional map of custom headers. * @return the request */ + @Retries.OnceRaw public PutObjectRequest createPutObjectRequest(String destKey, InputStream inputStream, long length, final Map headers) { + activateAuditSpan(); ObjectMetadata objectMetadata = newObjectMetadata(length); if (headers != null) { objectMetadata.setUserMetadata(headers); } - return owner.newPutObjectRequest(destKey, + return getRequestFactory().newPutObjectRequest( + destKey, objectMetadata, inputStream); } @@ -195,13 +261,16 @@ public PutObjectRequest createPutObjectRequest(String destKey, * @param sourceFile source file * @return the request */ + @Retries.OnceRaw public PutObjectRequest createPutObjectRequest(String dest, File sourceFile) { Preconditions.checkState(sourceFile.length() < Integer.MAX_VALUE, "File length is too big for a single PUT upload"); - return owner.newPutObjectRequest(dest, - newObjectMetadata((int) sourceFile.length()), - sourceFile); + activateAuditSpan(); + return getRequestFactory(). + newPutObjectRequest(dest, + newObjectMetadata((int) sourceFile.length()), + sourceFile); } /** @@ -227,7 +296,7 @@ public void writeFailed(Exception ex) { * @return a new metadata instance */ public ObjectMetadata newObjectMetadata(long length) { - return owner.newObjectMetadata(length); + return getRequestFactory().newObjectMetadata(length); } /** @@ -240,15 +309,16 @@ public ObjectMetadata newObjectMetadata(long length) { @Retries.RetryTranslated public String initiateMultiPartUpload(String destKey) throws IOException { LOG.debug("Initiating Multipart upload to {}", destKey); - final InitiateMultipartUploadRequest initiateMPURequest = - new InitiateMultipartUploadRequest(bucket, - destKey, - newObjectMetadata(-1)); - initiateMPURequest.setCannedACL(owner.getCannedACL()); - owner.setOptionalMultipartUploadRequestParameters(initiateMPURequest); - - return retry("initiate MultiPartUpload", destKey, true, - () -> owner.initiateMultipartUpload(initiateMPURequest).getUploadId()); + try (AuditSpan span = activateAuditSpan()) { + return retry("initiate MultiPartUpload", destKey, true, + () -> { + final InitiateMultipartUploadRequest initiateMPURequest = + getRequestFactory().newMultipartUploadRequest( + destKey); + return owner.initiateMultipartUpload(initiateMPURequest) + .getUploadId(); + }); + } } /** @@ -278,23 +348,22 @@ private CompleteMultipartUploadResult finalizeMultipartUpload( throw new PathIOException(destKey, "No upload parts in multipart upload"); } - CompleteMultipartUploadResult uploadResult = - invoker.retry("Completing multipart upload", destKey, - true, - retrying, - () -> { - // a copy of the list is required, so that the AWS SDK doesn't - // attempt to sort an unmodifiable list. - return owner.getAmazonS3Client().completeMultipartUpload( - new CompleteMultipartUploadRequest(bucket, - destKey, - uploadId, - new ArrayList<>(partETags))); - } - ); - owner.finishedWrite(destKey, length, uploadResult.getETag(), - uploadResult.getVersionId(), operationState); - return uploadResult; + try (AuditSpan span = activateAuditSpan()) { + CompleteMultipartUploadResult uploadResult; + uploadResult = invoker.retry("Completing multipart upload", destKey, + true, + retrying, + () -> { + final CompleteMultipartUploadRequest request = + getRequestFactory().newCompleteMultipartUploadRequest( + destKey, uploadId, partETags); + return owner.getAmazonS3Client().completeMultipartUpload( + request); + }); + owner.finishedWrite(destKey, length, uploadResult.getETag(), + uploadResult.getVersionId(), operationState); + return uploadResult; + } } /** @@ -351,16 +420,17 @@ public void abortMultipartUpload(String destKey, String uploadId, destKey, true, retrying, - () -> owner.abortMultipartUpload( - destKey, - uploadId)); + withinAuditSpan(getAuditSpan(), () -> + owner.abortMultipartUpload( + destKey, uploadId))); } else { // single pass attempt. once("Aborting multipart upload ID " + uploadId, destKey, - () -> owner.abortMultipartUpload( - destKey, - uploadId)); + withinAuditSpan(getAuditSpan(), () -> + owner.abortMultipartUpload( + destKey, + uploadId))); } } @@ -373,7 +443,8 @@ public void abortMultipartUpload(String destKey, String uploadId, public void abortMultipartUpload(MultipartUpload upload) throws IOException { invoker.retry("Aborting multipart commit", upload.getKey(), true, - () -> owner.abortMultipartUpload(upload)); + withinAuditSpan(getAuditSpan(), + () -> owner.abortMultipartUpload(upload))); } @@ -389,7 +460,7 @@ public int abortMultipartUploadsUnderPath(String prefix) throws IOException { LOG.debug("Aborting multipart uploads under {}", prefix); int count = 0; - List multipartUploads = owner.listMultipartUploads(prefix); + List multipartUploads = listMultipartUploads(prefix); LOG.debug("Number of outstanding uploads: {}", multipartUploads.size()); for (MultipartUpload upload: multipartUploads) { try { @@ -402,6 +473,14 @@ public int abortMultipartUploadsUnderPath(String prefix) return count; } + @Override + @Retries.RetryTranslated + public List listMultipartUploads(final String prefix) + throws IOException { + activateAuditSpan(); + return owner.listMultipartUploads(prefix); + } + /** * Abort a multipart commit operation. * @param destKey destination key of ongoing operation @@ -409,6 +488,7 @@ public int abortMultipartUploadsUnderPath(String prefix) * @throws IOException on problems. * @throws FileNotFoundException if the abort ID is unknown */ + @Override @Retries.RetryTranslated public void abortMultipartCommit(String destKey, String uploadId) throws IOException { @@ -423,6 +503,7 @@ public void abortMultipartCommit(String destKey, String uploadId) * in {@code offset} and a length of block in {@code size} equal to * or less than the remaining bytes. * The part number must be less than 10000. + * Retry policy is once-translated; to much effort * @param destKey destination key of ongoing operation * @param uploadId ID of ongoing upload * @param partNumber current part number of the upload @@ -431,9 +512,11 @@ public void abortMultipartCommit(String destKey, String uploadId) * @param sourceFile optional source file. * @param offset offset in file to start reading. * @return the request. - * @throws IllegalArgumentException if the parameters are invalid -including + * @throws IllegalArgumentException if the parameters are invalid. * @throws PathIOException if the part number is out of range. */ + @Override + @Retries.OnceTranslated public UploadPartRequest newUploadPartRequest( String destKey, String uploadId, @@ -441,52 +524,17 @@ public UploadPartRequest newUploadPartRequest( int size, InputStream uploadStream, File sourceFile, - Long offset) throws PathIOException { - checkNotNull(uploadId); - // exactly one source must be set; xor verifies this - checkArgument((uploadStream != null) ^ (sourceFile != null), - "Data source"); - checkArgument(size >= 0, "Invalid partition size %s", size); - checkArgument(partNumber > 0, - "partNumber must be between 1 and %s inclusive, but is %s", - DEFAULT_UPLOAD_PART_COUNT_LIMIT, partNumber); - - LOG.debug("Creating part upload request for {} #{} size {}", - uploadId, partNumber, size); - long partCountLimit = longOption(conf, - UPLOAD_PART_COUNT_LIMIT, - DEFAULT_UPLOAD_PART_COUNT_LIMIT, - 1); - if (partCountLimit != DEFAULT_UPLOAD_PART_COUNT_LIMIT) { - LOG.warn("Configuration property {} shouldn't be overridden by client", - UPLOAD_PART_COUNT_LIMIT); - } - final String pathErrorMsg = "Number of parts in multipart upload exceeded." - + " Current part count = %s, Part count limit = %s "; - if (partNumber > partCountLimit) { - throw new PathIOException(destKey, - String.format(pathErrorMsg, partNumber, partCountLimit)); - } - UploadPartRequest request = new UploadPartRequest() - .withBucketName(bucket) - .withKey(destKey) - .withUploadId(uploadId) - .withPartNumber(partNumber) - .withPartSize(size); - if (uploadStream != null) { - // there's an upload stream. Bind to it. - request.setInputStream(uploadStream); - } else { - checkArgument(sourceFile.exists(), - "Source file does not exist: %s", sourceFile); - checkArgument(offset >= 0, "Invalid offset %s", offset); - long length = sourceFile.length(); - checkArgument(offset == 0 || offset < length, - "Offset %s beyond length of file %s", offset, length); - request.setFile(sourceFile); - request.setFileOffset(offset); - } - return request; + Long offset) throws IOException { + return once("upload part request", destKey, + withinAuditSpan(getAuditSpan(), () -> + getRequestFactory().newUploadPartRequest( + destKey, + uploadId, + partNumber, + size, + uploadStream, + sourceFile, + offset))); } /** @@ -514,7 +562,8 @@ public PutObjectResult putObject(PutObjectRequest putObjectRequest) throws IOException { return retry("Writing Object", putObjectRequest.getKey(), true, - () -> owner.putObjectDirect(putObjectRequest)); + withinAuditSpan(getAuditSpan(), () -> + owner.putObjectDirect(putObjectRequest))); } /** @@ -529,7 +578,8 @@ public UploadResult uploadObject(PutObjectRequest putObjectRequest) // no retry; rely on xfer manager logic return retry("Writing Object", putObjectRequest.getKey(), true, - () -> owner.executePut(putObjectRequest, null)); + withinAuditSpan(getAuditSpan(), () -> + owner.executePut(putObjectRequest, null))); } /** @@ -543,13 +593,12 @@ public UploadResult uploadObject(PutObjectRequest putObjectRequest) public void revertCommit(String destKey, @Nullable BulkOperationState operationState) throws IOException { once("revert commit", destKey, - () -> { + withinAuditSpan(getAuditSpan(), () -> { Path destPath = owner.keyToQualifiedPath(destKey); owner.deleteObjectAtPath(destPath, destKey, true, operationState); owner.maybeCreateFakeParentDirectory(destPath); - } - ); + })); } /** @@ -620,10 +669,11 @@ public BulkOperationState initiateOperation(final Path path, public UploadPartResult uploadPart(UploadPartRequest request) throws IOException { return retry("upload part #" + request.getPartNumber() - + " upload ID "+ request.getUploadId(), + + " upload ID " + request.getUploadId(), request.getKey(), true, - () -> owner.uploadPart(request)); + withinAuditSpan(getAuditSpan(), + () -> owner.uploadPart(request))); } /** @@ -642,10 +692,10 @@ public Configuration getConf() { * @return the request */ public SelectObjectContentRequest newSelectRequest(Path path) { - SelectObjectContentRequest request = new SelectObjectContentRequest(); - request.setBucketName(bucket); - request.setKey(owner.pathToKey(path)); - return request; + try (AuditSpan span = getAuditSpan()) { + return getRequestFactory().newSelectRequest( + storeContext.pathToKey(path)); + } } /** @@ -664,6 +714,8 @@ public SelectObjectContentResult select( final SelectObjectContentRequest request, final String action) throws IOException { + // no setting of span here as the select binding is (statically) created + // without any span. String bucketName = request.getBucketName(); Preconditions.checkArgument(bucket.equals(bucketName), "wrong bucket: %s", bucketName); @@ -676,7 +728,7 @@ public SelectObjectContentResult select( action, source.toString(), true, - () -> { + withinAuditSpan(getAuditSpan(), () -> { try (DurationInfo ignored = new DurationInfo(LOG, "S3 Select operation")) { try { @@ -691,11 +743,35 @@ public SelectObjectContentResult select( throw e; } } - }); + })); + } + + @Override + public AuditSpan createSpan(final String operation, + @Nullable final String path1, + @Nullable final String path2) throws IOException { + return auditSpanSource.createSpan(operation, path1, path2); } @Override public void incrementWriteOperations() { owner.incrementWriteOperations(); } + + /** + * Deactivate the audit span. + */ + @Override + public void close() throws IOException { + deactivateAuditSpan(); + } + + /** + * Get the request factory which uses this store's audit span. + * @return the request factory. + */ + public RequestFactory getRequestFactory() { + return requestFactory; + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java index 0a8150ce34dbb..9400ef2c490fe 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a; import javax.annotation.Nullable; +import java.io.Closeable; import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -43,6 +44,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; +import org.apache.hadoop.fs.store.audit.AuditSpanSource; import org.apache.hadoop.util.functional.CallableRaisingIOE; /** @@ -54,7 +56,7 @@ * use `WriteOperationHelper` directly. * @since Hadoop 3.3.0 */ -public interface WriteOperations { +public interface WriteOperations extends AuditSpanSource, Closeable { /** * Execute a function with retry processing. @@ -184,6 +186,16 @@ void abortMultipartUpload(MultipartUpload upload) int abortMultipartUploadsUnderPath(String prefix) throws IOException; + /** + * Abort multipart uploads under a path: limited to the first + * few hundred. + * @param prefix prefix for uploads to abort + * @return a count of aborts + * @throws IOException trouble; FileNotFoundExceptions are swallowed. + */ + List listMultipartUploads(String prefix) + throws IOException; + /** * Abort a multipart commit operation. * @param destKey destination key of ongoing operation @@ -210,7 +222,7 @@ void abortMultipartCommit(String destKey, String uploadId) * @param sourceFile optional source file. * @param offset offset in file to start reading. * @return the request. - * @throws IllegalArgumentException if the parameters are invalid -including + * @throws IllegalArgumentException if the parameters are invalid * @throws PathIOException if the part number is out of range. */ UploadPartRequest newUploadPartRequest( @@ -220,7 +232,7 @@ UploadPartRequest newUploadPartRequest( int size, InputStream uploadStream, File sourceFile, - Long offset) throws PathIOException; + Long offset) throws IOException; /** * PUT an object directly (i.e. not via the transfer manager). diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java new file mode 100644 index 0000000000000..9bffcc90d0bd4 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java @@ -0,0 +1,296 @@ +/* + * 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.hadoop.fs.s3a.api; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.InputStream; +import java.util.List; +import java.util.Optional; + +import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; +import com.amazonaws.services.s3.model.CannedAccessControlList; +import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; +import com.amazonaws.services.s3.model.CopyObjectRequest; +import com.amazonaws.services.s3.model.DeleteObjectRequest; +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; +import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; +import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest; +import com.amazonaws.services.s3.model.ListObjectsRequest; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ObjectListing; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.PartETag; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; +import com.amazonaws.services.s3.model.SSECustomerKey; +import com.amazonaws.services.s3.model.SelectObjectContentRequest; +import com.amazonaws.services.s3.model.UploadPartRequest; + +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; +import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; + +/** + * Factory for S3 objects. + * + * This is where the owner FS's {@code prepareRequest()} + * callback is invoked to mark up a request for this span. + * + * All AWS request objects MUST BE created through this, in + * the active audit span. + * The standard implementation provides a callback for the S3AFS or + * tests to be invoked to prepare each request. + * Such callbacks SHOULD NOT raise exceptions other + * than argument validation exceptions. + * as there are no guarantees how they are processed. + * That is: no guarantees of retry or translation. + */ +public interface RequestFactory { + + /** + * Set the encryption secrets for all subsequent requests. + * @param secrets encryption secrets. + */ + void setEncryptionSecrets(EncryptionSecrets secrets); + + /** + * Get the canned ACL of this FS. + * @return an ACL, if any + */ + CannedAccessControlList getCannedACL(); + + /** + * Create the AWS SDK structure used to configure SSE, + * if the encryption secrets contain the information/settings for this. + * @return an optional set of KMS Key settings + */ + Optional generateSSEAwsKeyParams(); + + /** + * Create the SSE-C structure for the AWS SDK, if the encryption secrets + * contain the information/settings for this. + * This will contain a secret extracted from the bucket/configuration. + * @return an optional customer key. + */ + Optional generateSSECustomerKey(); + + /** + * Get the encryption algorithm of this endpoint. + * @return the encryption algorithm. + */ + S3AEncryptionMethods getServerSideEncryptionAlgorithm(); + + /** + * Create a new object metadata instance. + * Any standard metadata headers are added here, for example: + * encryption. + * + * @param length length of data to set in header; Ignored if negative + * @return a new metadata instance + */ + ObjectMetadata newObjectMetadata(long length); + + /** + * Create a copy request. + * This includes the work of copying the relevant parts + * of the metadata from the source + * @param srcKey source + * @param dstKey destination + * @param srcom source object metadata. + * @return the request + */ + CopyObjectRequest newCopyObjectRequest(String srcKey, + String dstKey, + ObjectMetadata srcom); + + /** + * Create a putObject request. + * Adds the ACL and metadata + * @param key key of object + * @param metadata metadata header + * @param srcfile source file + * @return the request + */ + PutObjectRequest newPutObjectRequest(String key, + ObjectMetadata metadata, File srcfile); + + /** + * Create a {@link PutObjectRequest} request. + * The metadata is assumed to have been configured with the size of the + * operation. + * @param key key of object + * @param metadata metadata header + * @param inputStream source data. + * @return the request + */ + PutObjectRequest newPutObjectRequest(String key, + ObjectMetadata metadata, + InputStream inputStream); + + /** + * Create a {@link PutObjectRequest} request for creating + * an empty directory. + * + * @param directory destination directory. + * @return request for a zero byte upload. + */ + PutObjectRequest newDirectoryMarkerRequest(String directory); + + /** + * List all multipart uploads under a prefix. + * @param prefix prefix to list under + * @return the request. + */ + ListMultipartUploadsRequest newListMultipartUploadsRequest( + @Nullable String prefix); + + /** + * Abort a multipart upload. + * @param destKey destination object key + * @param uploadId ID of initiated upload + * @return the request. + */ + AbortMultipartUploadRequest newAbortMultipartUploadRequest( + String destKey, + String uploadId); + + /** + * Start a multipart upload. + * @param destKey destination object key + * @return the request. + */ + InitiateMultipartUploadRequest newMultipartUploadRequest( + String destKey); + + /** + * Complete a multipart upload. + * @param destKey destination object key + * @param uploadId ID of initiated upload + * @param partETags ordered list of etags + * @return the request. + */ + CompleteMultipartUploadRequest newCompleteMultipartUploadRequest( + String destKey, + String uploadId, + List partETags); + + /** + * Create a HEAD request. + * @param key key, may have trailing / + * @return the request. + */ + GetObjectMetadataRequest newGetObjectMetadataRequest(String key); + + /** + * Create a GET request. + * @param key object key + * @return the request. + */ + GetObjectRequest newGetObjectRequest(String key); + + /** + * Create and initialize a part request of a multipart upload. + * Exactly one of: {@code uploadStream} or {@code sourceFile} + * must be specified. + * A subset of the file may be posted, by providing the starting point + * in {@code offset} and a length of block in {@code size} equal to + * or less than the remaining bytes. + * @param destKey destination key of ongoing operation + * @param uploadId ID of ongoing upload + * @param partNumber current part number of the upload + * @param size amount of data + * @param uploadStream source of data to upload + * @param sourceFile optional source file. + * @param offset offset in file to start reading. + * @return the request. + * @throws PathIOException if the part number is out of range. + */ + UploadPartRequest newUploadPartRequest( + String destKey, + String uploadId, + int partNumber, + int size, + InputStream uploadStream, + File sourceFile, + long offset) throws PathIOException; + + /** + * Create a S3 Select request for the destination object. + * This does not build the query. + * @param key object key + * @return the request + */ + SelectObjectContentRequest newSelectRequest(String key); + + /** + * Create the (legacy) V1 list request. + * @param key key to list under + * @param delimiter delimiter for keys + * @param maxKeys maximum number in a list page. + * @return the request + */ + ListObjectsRequest newListObjectsV1Request(String key, + String delimiter, + int maxKeys); + + /** + * Create the next V1 page list request, following + * on from the previous response. + * @param prev previous response + * @return the request + */ + + ListNextBatchOfObjectsRequest newListNextBatchOfObjectsRequest( + ObjectListing prev); + + /** + * Create a V2 list request. + * This will be recycled for any subsequent requests. + * @param key key to list under + * @param delimiter delimiter for keys + * @param maxKeys maximum number in a list page. + * @return the request + */ + ListObjectsV2Request newListObjectsV2Request(String key, + String delimiter, + int maxKeys); + + /** + * Create a request to delete a single object. + * @param key object to delete + * @return the request + */ + DeleteObjectRequest newDeleteObjectRequest(String key); + + /** + * Bulk delete request. + * @param keysToDelete list of keys to delete. + * @param quiet should a bulk query be quiet, or should its result list + * all deleted keys? + * @return the request + */ + DeleteObjectsRequest newBulkDeleteRequest( + List keysToDelete, + boolean quiet); + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/package-info.java new file mode 100644 index 0000000000000..625c8a3fca2cc --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/package-info.java @@ -0,0 +1,34 @@ +/* + * 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. + */ + +/** + * Where the interfaces for classes implemented in {@code o.a.h.fs.s3a.impl} + * should go. This is to allow for extension points to use these interfaces + * without having any java module access to the .impl package. + * + * This is public for S3A extension points, however there are no + * guarantees of stability -changes may break things, possibly + * unintentionally. + */ + +@InterfaceAudience.LimitedPrivate("extensions") +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.api; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSAuditEventCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSAuditEventCallbacks.java new file mode 100644 index 0000000000000..8134d5cea942e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSAuditEventCallbacks.java @@ -0,0 +1,160 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.Request; +import com.amazonaws.Response; +import com.amazonaws.SdkBaseException; +import com.amazonaws.handlers.HandlerAfterAttemptContext; +import com.amazonaws.handlers.HandlerBeforeAttemptContext; +import com.amazonaws.http.HttpResponse; + +import org.apache.hadoop.fs.s3a.Retries; + +/** + * Callbacks for audit spans. This is implemented + * in the span manager as well as individual audit spans. + * If any of the code in a callback raises an InterruptedException, + * it must be caught and {@code Thread.interrupt()} called to + * redeclare the thread as interrupted. The AWS SDK will + * detect this and raise an exception. + * + * Look at the documentation for + * {@code com.amazonaws.handlers.IRequestHandler2} for details + * on the callbacks. + */ +public interface AWSAuditEventCallbacks { + + /** + * Return a span ID which must be unique for all spans within + * everywhere. That effectively means part of the + * span SHOULD be derived from a UUID. + * Callers MUST NOT make any assumptions about the actual + * contents or structure of this string other than the + * uniqueness. + * @return a non-empty string + */ + String getSpanId(); + + /** + * Get the name of the operation. + * @return the operation name. + */ + String getOperationName(); + + /** + * Callback when a request is created in the S3A code. + * This is called in {@code RequestFactoryImpl} after + * each request is created. + * It is not invoked on any AWS requests created in the SDK. + * Avoid raising exceptions or talking to any remote service; + * this callback is for annotation rather than validation. + * @param request request request. + * @param type of request + * @return the request, possibly modified. + */ + default T requestCreated(T request) { + return request; + } + + /** + * Preflight preparation of AWS request. + * @param request request + * @param type of request + * @return an updated request. + * @throws AuditFailureException for generic audit failures + * @throws SdkBaseException for other reasons. + */ + @Retries.OnceRaw + default T beforeExecution(T request) + throws AuditFailureException, SdkBaseException { + return request; + } + + /** + * Callback after S3 responded to a request. + * @param request request + * @param response response. + * @throws AuditFailureException for generic audit failures + * @throws SdkBaseException for other reasons. + */ + default void afterResponse(Request request, + Response response) + throws AuditFailureException, SdkBaseException { + } + + /** + * Callback after a request resulted in an error. + * @param request request + * @param response response. + * @param exception exception raised. + * @throws AuditFailureException for generic audit failures + * @throws SdkBaseException for other reasons. + */ + default void afterError(Request request, + Response response, + Exception exception) + throws AuditFailureException, SdkBaseException { + } + + /** + * Request before marshalling. + * @param request request + * @return possibly modified request. + */ + default AmazonWebServiceRequest beforeMarshalling( + AmazonWebServiceRequest request) { + return request; + } + + /** + * Request before marshalling. + * @param request request + */ + default void beforeRequest(Request request) { + } + + /** + * Before any attempt is made. + * @param context full context, including the request. + */ + default void beforeAttempt(HandlerBeforeAttemptContext context) { + } + + /** + * After any attempt is made. + * @param context full context, including the request. + */ + default void afterAttempt( + HandlerAfterAttemptContext context) { + } + + /** + * Before unmarshalling the response. + * @param request request made. + * @param httpResponse response received + * @return updated response. + */ + default HttpResponse beforeUnmarshalling( + final Request request, + final HttpResponse httpResponse) { + return httpResponse; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java new file mode 100644 index 0000000000000..a5a6dbc84797f --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java @@ -0,0 +1,296 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import java.util.List; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; +import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; +import com.amazonaws.services.s3.model.CopyPartRequest; +import com.amazonaws.services.s3.model.DeleteObjectRequest; +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.GetBucketLocationRequest; +import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; +import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; +import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest; +import com.amazonaws.services.s3.model.ListObjectsRequest; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ObjectListing; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.SelectObjectContentRequest; +import com.amazonaws.services.s3.model.UploadPartRequest; + +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_HEAD_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.MULTIPART_UPLOAD_ABORTED; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.MULTIPART_UPLOAD_COMPLETED; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.MULTIPART_UPLOAD_LIST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.MULTIPART_UPLOAD_PART_PUT; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.MULTIPART_UPLOAD_STARTED; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_BULK_DELETE_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_DELETE_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_PUT_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_SELECT_REQUESTS; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.STORE_EXISTS_PROBE; + +/** + * Extract information from a request. + * Intended for reporting and error logs. + */ +public class AWSRequestAnalyzer { + + /** + * Given an AWS request, try to analyze it to operation, + * read/write and path. + * @param request request. + * @return information about the request. + * @param type of request. + */ + public RequestInfo analyze(T request) { + + // this is where Scala's case statement would massively + // simplify life. + // Please Keep in Alphabetical Order. + if (request instanceof AbortMultipartUploadRequest) { + return writing(MULTIPART_UPLOAD_ABORTED, + ((AbortMultipartUploadRequest) request).getKey(), + 0); + } else if (request instanceof CompleteMultipartUploadRequest) { + CompleteMultipartUploadRequest r + = (CompleteMultipartUploadRequest) request; + return writing(MULTIPART_UPLOAD_COMPLETED, + r.getKey(), + r.getPartETags().size()); + } else if (request instanceof DeleteObjectRequest) { + // DeleteObject: single object + return writing(OBJECT_DELETE_REQUEST, + ((DeleteObjectRequest) request).getKey(), + 1); + } else if (request instanceof DeleteObjectsRequest) { + // DeleteObjects: bulk delete + // use first key as the path + DeleteObjectsRequest r = (DeleteObjectsRequest) request; + List keys + = r.getKeys(); + return writing(OBJECT_BULK_DELETE_REQUEST, + keys.isEmpty() ? null : keys.get(0).getKey(), + keys.size()); + } else if (request instanceof GetBucketLocationRequest) { + GetBucketLocationRequest r = (GetBucketLocationRequest) request; + return reading(STORE_EXISTS_PROBE, + r.getBucketName(), + 0); + } else if (request instanceof GetObjectMetadataRequest) { + return reading(ACTION_HTTP_HEAD_REQUEST, + ((GetObjectMetadataRequest) request).getKey(), 0); + } else if (request instanceof GetObjectRequest) { + GetObjectRequest r = (GetObjectRequest) request; + long[] range = r.getRange(); + long size = range == null + ? -1 + : range[1] - range[0]; + return reading(ACTION_HTTP_GET_REQUEST, + r.getKey(), + size); + } else if (request instanceof InitiateMultipartUploadRequest) { + return writing(MULTIPART_UPLOAD_STARTED, + ((InitiateMultipartUploadRequest) request).getKey(), + 0); + } else if (request instanceof ListMultipartUploadsRequest) { + ListMultipartUploadsRequest r + = (ListMultipartUploadsRequest) request; + return reading(MULTIPART_UPLOAD_LIST, + r.getPrefix(), + r.getMaxUploads()); + } else if (request instanceof ListObjectsRequest) { + ListObjectsRequest r = (ListObjectsRequest) request; + return reading(OBJECT_LIST_REQUEST, + r.getPrefix(), + r.getMaxKeys()); + } else if (request instanceof ListNextBatchOfObjectsRequest) { + ListNextBatchOfObjectsRequest r = (ListNextBatchOfObjectsRequest) request; + ObjectListing l = r.getPreviousObjectListing(); + String prefix = ""; + int size = 0; + if (l != null) { + prefix = l.getPrefix(); + size = l.getMaxKeys(); + } + return reading(OBJECT_LIST_REQUEST, + prefix, + size); + } else if (request instanceof ListObjectsV2Request) { + ListObjectsV2Request r = (ListObjectsV2Request) request; + return reading(OBJECT_LIST_REQUEST, + r.getPrefix(), + r.getMaxKeys()); + } else if (request instanceof PutObjectRequest) { + PutObjectRequest r = (PutObjectRequest) request; + return writing(OBJECT_PUT_REQUEST, + r.getKey(), + 0); + } else if (request instanceof SelectObjectContentRequest) { + SelectObjectContentRequest r = + (SelectObjectContentRequest) request; + return reading(OBJECT_SELECT_REQUESTS, + r.getKey(), + 1); + } else if (request instanceof UploadPartRequest) { + UploadPartRequest r = (UploadPartRequest) request; + return writing(MULTIPART_UPLOAD_PART_PUT, + r.getKey(), + r.getPartSize()); + } + // no explicit support, return classname + return writing(request.getClass().getName(), null, 0); + } + + /** + * A request. + * @param verb verb + * @param mutating does this update the store + * @param key object/prefix, etc. + * @param size nullable size + * @return request info + */ + private RequestInfo request(final String verb, + final boolean mutating, + final String key, + final Number size) { + return new RequestInfo(verb, mutating, key, size); + } + + /** + * A read request. + * @param verb verb + * @param key object/prefix, etc. + * @param size nullable size + * @return request info + */ + private RequestInfo reading(final String verb, + final String key, final Number size) { + return request(verb, false, key, size); + } + + /** + * A write request of some form. + * @param verb verb + * @param key object/prefix, etc. + * @param size nullable size + * @return request info + */ + private RequestInfo writing(final String verb, + final String key, final Number size) { + return request(verb, true, key, size); + } + + /** + * Predicate which returns true if the request is of a kind which + * could be outside a span because of how the AWS SDK generates them. + * @param request request + * @return true if the transfer manager creates them. + */ + public static final boolean + isRequestNotAlwaysInSpan(final Object request) { + return request instanceof CopyPartRequest + || request instanceof CompleteMultipartUploadRequest + || request instanceof GetBucketLocationRequest; + } + + /** + * Info about a request. + */ + public static final class RequestInfo { + + /** + * Verb. + */ + private String verb; + + /** + * Is this a mutating call? + */ + private boolean mutating; + + /** + * Key if there is one; maybe first key in a list. + */ + private String key; + + /** + * Size, where the meaning of size depends on the request. + */ + private long size; + + /** + * Construct. + * @param verb operation/classname, etc. + * @param mutating does this update S3 State. + * @param key key/path/bucket operated on. + * @param size size of request (bytes, elements, limit...). Nullable. + */ + private RequestInfo(final String verb, + final boolean mutating, + final String key, + final Number size) { + this.verb = verb; + this.mutating = mutating; + this.key = key; + this.size = toSafeLong(size); + } + + public String getVerb() { + return verb; + } + + public boolean isMutating() { + return mutating; + } + + public String getKey() { + return key; + } + + public long getSize() { + return size; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "{"); + sb.append(verb); + if (key != null) { + sb.append(" '").append(key).append('\''); + } + sb.append(" size=").append(size); + sb.append(", mutating=").append(mutating); + sb.append('}'); + return sb.toString(); + } + } + + private static long toSafeLong(final Number size) { + return size != null ? size.longValue() : 0; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditFailureException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditFailureException.java new file mode 100644 index 0000000000000..5cd2b9647d834 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditFailureException.java @@ -0,0 +1,41 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import org.apache.hadoop.fs.s3a.CredentialInitializationException; + +/** + * This is in the AWS exception tree so that exceptions raised in the + * AWS SDK are correctly reported up. + * It is a subclass of {@link CredentialInitializationException} + * so that + * {@code S3AUtils.translateException()} recognizes these exceptions + * and converts them to AccessDeniedException. + */ +public class AuditFailureException extends CredentialInitializationException { + + public AuditFailureException(final String message, final Throwable t) { + super(message, t); + } + + public AuditFailureException(final String message) { + super(message); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java new file mode 100644 index 0000000000000..7b9e66901ec4c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java @@ -0,0 +1,137 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; + +import com.amazonaws.HandlerContextAware; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.audit.impl.ActiveAuditManagerS3A; +import org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor; +import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditManagerS3A; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.s3a.audit.impl.S3AInternalAuditConstants.AUDIT_SPAN_HANDLER_CONTEXT; + +/** + * Support for integrating auditing within the S3A code. + */ +public final class AuditIntegration { + + /** + * Logging. + */ + private static final Logger LOG = + LoggerFactory.getLogger(AuditIntegration.class); + + private AuditIntegration() { + } + + /** + * Create and start an audit manager. + * @param conf configuration + * @param iostatistics IOStatistics source. + * @return audit manager. + */ + public static AuditManagerS3A createAndStartAuditManager( + Configuration conf, + IOStatisticsStore iostatistics) { + ActiveAuditManagerS3A auditManager = new ActiveAuditManagerS3A( + requireNonNull(iostatistics)); + auditManager.init(conf); + auditManager.start(); + LOG.debug("Started Audit Manager {}", auditManager); + return auditManager; + } + + /** + * Return a stub audit manager. + * @return an audit manager. + */ + public static AuditManagerS3A stubAuditManager() { + return new NoopAuditManagerS3A(); + } + + /** + * Create and initialize an audit service. + * The service start operation is not called: that is left to + * the caller. + * @param conf configuration to read the key from and to use to init + * the service. + * @param key key containing the classname + * @param options options to initialize with. + * @return instantiated class. + * @throws IOException failure to initialise. + */ + public static OperationAuditor createAndInitAuditor( + Configuration conf, + String key, + OperationAuditorOptions options) throws IOException { + final Class auditClassname + = conf.getClass( + key, + LoggingAuditor.class, + OperationAuditor.class); + try { + LOG.debug("Auditor class is {}", auditClassname); + final Constructor constructor + = auditClassname.getConstructor(); + final OperationAuditor instance = constructor.newInstance(); + instance.init(options); + return instance; + } catch (NoSuchMethodException | InstantiationException + | RuntimeException + | IllegalAccessException | InvocationTargetException e) { + throw new IOException("Failed to instantiate class " + + auditClassname + + " defined in " + key + + ": " + e, + e); + } + } + + /** + * Get the span from a handler context. + * @param request request + * @param type of request. + * @return the span callbacks or null + */ + public static AWSAuditEventCallbacks + retrieveAttachedSpan(final T request) { + return request.getHandlerContext(AUDIT_SPAN_HANDLER_CONTEXT); + } + + /** + * Attach a span to a handler context. + * @param request request + * @param span span to attach + * @param type of request. + */ + public static void attachSpanToRequest( + final T request, final AWSAuditEventCallbacks span) { + request.addHandlerContext(AUDIT_SPAN_HANDLER_CONTEXT, span); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java new file mode 100644 index 0000000000000..c1302d57454b1 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java @@ -0,0 +1,92 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import java.io.IOException; +import java.util.List; + +import com.amazonaws.handlers.RequestHandler2; +import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.store.audit.ActiveThreadSpanSource; +import org.apache.hadoop.fs.store.audit.AuditSpanSource; +import org.apache.hadoop.service.Service; + +/** + * Interface for Audit Managers auditing operations through the + * AWS libraries. + * The Audit Manager is the binding between S3AFS and the instantiated + * plugin point -it adds: + *
    + *
  1. per-thread tracking of audit spans
  2. + *
  3. The wiring up to the AWS SDK
  4. + *
  5. State change tracking for copy operations (does not address issue)
  6. + *
+ */ +@InterfaceAudience.Private +public interface AuditManagerS3A extends Service, + AuditSpanSource, + AWSAuditEventCallbacks, + ActiveThreadSpanSource { + + /** + * Get the auditor; valid once initialized. + * @return the auditor. + */ + OperationAuditor getAuditor(); + + /** + * Create the request handler(s) for this audit service. + * The list returned is mutable; new handlers may be added. + * @return list of handlers for the SDK. + * @throws IOException failure. + */ + List createRequestHandlers() throws IOException; + + /** + * Return a transfer state change callback which + * fixes the active span context to be that in which + * the state change listener was created. + * This can be used to audit the creation of the multipart + * upload initiation request which the transfer manager + * makes when a file to be copied is split up. + * This must be invoked/used within the active span. + * @return a state change listener. + */ + TransferStateChangeListener createStateChangeListener(); + + /** + * Check for permission to access a path. + * The path is fully qualified and the status is the + * status of the path. + * This is called from the {@code FileSystem.access()} command + * and is a soft permission check used by Hive. + * @param path path to check + * @param status status of the path. + * @param mode access mode. + * @return true if access is allowed. + * @throws IOException failure + */ + boolean checkAccess(Path path, S3AFileStatus status, FsAction mode) + throws IOException; +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanS3A.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanS3A.java new file mode 100644 index 0000000000000..3e4e70918ac36 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanS3A.java @@ -0,0 +1,29 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import org.apache.hadoop.fs.store.audit.AuditSpan; + +/** + * An Audit Span with S3A integration, specifically + * callbacks from the AWS client SDK. + */ +public interface AuditSpanS3A extends AuditSpan, + AWSAuditEventCallbacks { +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java new file mode 100644 index 0000000000000..672bcdf7f9c77 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java @@ -0,0 +1,73 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.store.audit.AuditSpanSource; +import org.apache.hadoop.service.Service; + +/** + * Interfaces for audit services to implement. + */ +public interface OperationAuditor extends Service, + IOStatisticsSource, AuditSpanSource { + + /** + * Initialize. + * The base class will call {@link Service#init(Configuration)}. + * @param options options to initialize with. + */ + void init(OperationAuditorOptions options); + + /** + * Get the unbonded span to use after deactivating an active + * span. + * @return a span. + */ + AuditSpanS3A getUnbondedSpan(); + + /** + * Check for permission to access a path. + * The path is fully qualified and the status is the + * status of the path. + * This is called from the {@code FileSystem.access()} command + * and is a soft permission check used by Hive. + * @param path path to check + * @param status status of the path. + * @param mode access mode. + * @return true if access is allowed. + * @throws IOException failure + */ + default boolean checkAccess(Path path, S3AFileStatus status, FsAction mode) + throws IOException { + return true; + } + + /** + * Get the Auditor ID. + * @return ID + */ + String getAuditorId(); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditorOptions.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditorOptions.java new file mode 100644 index 0000000000000..40d1b23651b88 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditorOptions.java @@ -0,0 +1,74 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; + +/** + * Options for the {@link OperationAuditor}. + * Done as a builder and passed in so + * that if it is extended, external auditors will still link. + */ +public final class OperationAuditorOptions { + + private Configuration configuration; + private IOStatisticsStore ioStatisticsStore; + + + private OperationAuditorOptions() { + } + + public Configuration getConfiguration() { + return configuration; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public OperationAuditorOptions withConfiguration(final Configuration value) { + configuration = value; + return this; + } + + public IOStatisticsStore getIoStatisticsStore() { + return ioStatisticsStore; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public OperationAuditorOptions withIoStatisticsStore( + final IOStatisticsStore value) { + ioStatisticsStore = value; + return this; + } + + /** + * Create one. + * @return a new option instance + */ + public static OperationAuditorOptions builder() { + return new OperationAuditorOptions(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3AAuditConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3AAuditConstants.java new file mode 100644 index 0000000000000..2b08893df6e82 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3AAuditConstants.java @@ -0,0 +1,104 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +@InterfaceAudience.LimitedPrivate("S3A auditing extensions") +@InterfaceStability.Unstable +public final class S3AAuditConstants { + + private S3AAuditConstants() { + } + + /** + * What to look for in logs for ops outside any audit. + * {@value}. + */ + public static final String UNAUDITED_OPERATION = "unaudited operation"; + + /** + * Name of class used for audit logs: {@value}. + */ + public static final String AUDIT_SERVICE_CLASSNAME = + "fs.s3a.audit.service.classname"; + + /** + * Classname of the logging auditor: {@value}. + */ + public static final String LOGGING_AUDIT_SERVICE = + "org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor"; + + /** + * Classname of the No-op auditor: {@value}. + */ + public static final String NOOP_AUDIT_SERVICE = + "org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor"; + + /** + * List of extra AWS SDK request handlers: {@value}. + * These are added to the SDK request chain after + * any audit service. + */ + public static final String AUDIT_REQUEST_HANDLERS = + "fs.s3a.audit.request.handlers"; + + /** + * Should operations outside spans be rejected? + * This is for testing coverage of the span code; if used + * in production there's a risk of unexpected failures. + * {@value}. + */ + public static final String REJECT_OUT_OF_SPAN_OPERATIONS + = "fs.s3a.audit.reject.out.of.span.operations"; + + /** + * Should the logging auditor add the HTTP Referrer header? + * {@value}. + */ + public static final String REFERRER_HEADER_ENABLED + = "fs.s3a.audit.referrer.enabled"; + + /** + * Should the logging auditor add the HTTP Referrer header? + * Default value: {@value}. + */ + public static final boolean REFERRER_HEADER_ENABLED_DEFAULT + = true; + + /** + * List of audit fields to strip from referrer headers. + * {@value}. + */ + public static final String REFERRER_HEADER_FILTER + = "fs.s3a.audit.referrer.filter"; + + /** + * Span name used during initialization. + */ + public static final String INITIALIZE_SPAN = "initialize"; + + /** + * Operation name for any operation outside of an explicit + * span. + */ + public static final String OUTSIDE_SPAN = + "outside-span"; +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogParser.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogParser.java new file mode 100644 index 0000000000000..ac29234a771e9 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogParser.java @@ -0,0 +1,309 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.regex.Pattern; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Class to help parse AWS S3 Logs. + * see https://docs.aws.amazon.com/AmazonS3/latest/userguide/LogFormat.html + * + * Getting the regexp right is surprisingly hard; this class does it + * explicitly and names each group in the process. + * All group names are included in {@link #AWS_LOG_REGEXP_GROUPS} in the order + * within the log entries. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class S3LogParser { + + private S3LogParser() { + } + + /** + * Simple entry: anything up to a space. + * {@value}. + */ + private static final String SIMPLE = "[^ ]*"; + + /** + * Date/Time. Everything within square braces. + * {@value}. + */ + private static final String DATETIME = "\\[(.*?)\\]"; + + /** + * A natural number or "-". + * {@value}. + */ + private static final String NUMBER = "(-|[0-9]*)"; + + /** + * A Quoted field or "-". + * {@value}. + */ + private static final String QUOTED = "(-|\"[^\"]*\")"; + + + /** + * An entry in the regexp. + * @param name name of the group + * @param pattern pattern to use in the regexp + * @return the pattern for the regexp + */ + private static String e(String name, String pattern) { + return String.format("(?<%s>%s) ", name, pattern); + } + + /** + * An entry in the regexp. + * @param name name of the group + * @param pattern pattern to use in the regexp + * @return the pattern for the regexp + */ + private static String eNoTrailing(String name, String pattern) { + return String.format("(?<%s>%s)", name, pattern); + } + + /** + * Simple entry using the {@link #SIMPLE} pattern. + * @param name name of the element (for code clarity only) + * @return the pattern for the regexp + */ + private static String e(String name) { + return e(name, SIMPLE); + } + + /** + * Quoted entry using the {@link #QUOTED} pattern. + * @param name name of the element (for code clarity only) + * @return the pattern for the regexp + */ + private static String q(String name) { + return e(name, QUOTED); + } + + /** + * Log group {@value}. + */ + public static final String OWNER_GROUP = "owner"; + + /** + * Log group {@value}. + */ + public static final String BUCKET_GROUP = "bucket"; + + /** + * Log group {@value}. + */ + public static final String TIMESTAMP_GROUP = "timestamp"; + + /** + * Log group {@value}. + */ + public static final String REMOTEIP_GROUP = "remoteip"; + + /** + * Log group {@value}. + */ + public static final String REQUESTER_GROUP = "requester"; + + /** + * Log group {@value}. + */ + public static final String REQUESTID_GROUP = "requestid"; + + /** + * Log group {@value}. + */ + public static final String VERB_GROUP = "verb"; + + /** + * Log group {@value}. + */ + public static final String KEY_GROUP = "key"; + + /** + * Log group {@value}. + */ + public static final String REQUESTURI_GROUP = "requesturi"; + + /** + * Log group {@value}. + */ + public static final String HTTP_GROUP = "http"; + + /** + * Log group {@value}. + */ + public static final String AWSERRORCODE_GROUP = "awserrorcode"; + + /** + * Log group {@value}. + */ + public static final String BYTESSENT_GROUP = "bytessent"; + + /** + * Log group {@value}. + */ + public static final String OBJECTSIZE_GROUP = "objectsize"; + + /** + * Log group {@value}. + */ + public static final String TOTALTIME_GROUP = "totaltime"; + + /** + * Log group {@value}. + */ + public static final String TURNAROUNDTIME_GROUP = "turnaroundtime"; + + /** + * Log group {@value}. + */ + public static final String REFERRER_GROUP = "referrer"; + + /** + * Log group {@value}. + */ + public static final String USERAGENT_GROUP = "useragent"; + + /** + * Log group {@value}. + */ + public static final String VERSION_GROUP = "version"; + + /** + * Log group {@value}. + */ + public static final String HOSTID_GROUP = "hostid"; + + /** + * Log group {@value}. + */ + public static final String SIGV_GROUP = "sigv"; + + /** + * Log group {@value}. + */ + public static final String CYPHER_GROUP = "cypher"; + + /** + * Log group {@value}. + */ + public static final String AUTH_GROUP = "auth"; + + /** + * Log group {@value}. + */ + public static final String ENDPOINT_GROUP = "endpoint"; + + /** + * Log group {@value}. + */ + public static final String TLS_GROUP = "tls"; + + /** + * This is where anything at the tail of a log + * entry ends up; it is null unless/until the AWS + * logs are enhanced in future. + * Value {@value}. + */ + public static final String TAIL_GROUP = "tail"; + + /** + * Construct the log entry pattern. + */ + public static final String LOG_ENTRY_REGEXP = "" + + e(OWNER_GROUP) + + e(BUCKET_GROUP) + + e(TIMESTAMP_GROUP, DATETIME) + + e(REMOTEIP_GROUP) + + e(REQUESTER_GROUP) + + e(REQUESTID_GROUP) + + e(VERB_GROUP) + + e(KEY_GROUP) + + q(REQUESTURI_GROUP) + + e(HTTP_GROUP, NUMBER) + + e(AWSERRORCODE_GROUP) + + e(BYTESSENT_GROUP) + + e(OBJECTSIZE_GROUP) + + e(TOTALTIME_GROUP) + + e(TURNAROUNDTIME_GROUP) + + q(REFERRER_GROUP) + + q(USERAGENT_GROUP) + + e(VERSION_GROUP) + + e(HOSTID_GROUP) + + e(SIGV_GROUP) + + e(CYPHER_GROUP) + + e(AUTH_GROUP) + + e(ENDPOINT_GROUP) + + eNoTrailing(TLS_GROUP, SIMPLE) + + eNoTrailing(TAIL_GROUP, ".*") // anything which follows + + "$"; // end of line + + /** + * Groups in order. + */ + private static final String[] GROUPS = { + OWNER_GROUP, + BUCKET_GROUP, + TIMESTAMP_GROUP, + REMOTEIP_GROUP, + REQUESTER_GROUP, + REQUESTID_GROUP, + VERB_GROUP, + KEY_GROUP, + REQUESTURI_GROUP, + HTTP_GROUP, + AWSERRORCODE_GROUP, + BYTESSENT_GROUP, + OBJECTSIZE_GROUP, + TOTALTIME_GROUP, + TURNAROUNDTIME_GROUP, + REFERRER_GROUP, + USERAGENT_GROUP, + VERSION_GROUP, + HOSTID_GROUP, + SIGV_GROUP, + CYPHER_GROUP, + AUTH_GROUP, + ENDPOINT_GROUP, + TLS_GROUP, + TAIL_GROUP + }; + + /** + * Ordered list of regular expression group names. + */ + public static final List AWS_LOG_REGEXP_GROUPS = + Collections.unmodifiableList(Arrays.asList(GROUPS)); + + /** + * And the actual compiled pattern. + */ + public static final Pattern LOG_ENTRY_PATTERN = Pattern.compile( + LOG_ENTRY_REGEXP); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogVerbs.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogVerbs.java new file mode 100644 index 0000000000000..0bafdf2f66766 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogVerbs.java @@ -0,0 +1,51 @@ +/* + * 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.hadoop.fs.s3a.audit; + +/** + * Various verbs in the S3 logs. + * https://stackoverflow.com/questions/42707878/amazon-s3-logs-operation-definition + */ +public final class S3LogVerbs { + + private S3LogVerbs() { + } + + public static final String DELETE = "REST.DELETE.OBJECT"; + public static final String COPY = "REST.COPY.OBJECT"; + public static final String DELETE_BULK = "REST.POST.MULTI_OBJECT_DELETE"; + public static final String DELETE_BULK_ENTRY = "BATCH.DELETE.OBJECT"; + public static final String GET = "REST.GET.OBJECT"; + public static final String HEAD = "REST.HEAD.OBJECT"; + public static final String GET_ACL = "REST.GET.ACL"; + public static final String GET_LOGGING_STATUS = "REST.GET.LOGGING_STATUS"; + public static final String LIST = "REST.GET.BUCKET"; + public static final String MULTIPART_UPLOAD_START = "REST.POST.UPLOADS"; + public static final String MULTIPART_UPLOAD_PART = "REST.PUT.PART"; + public static final String MULTIPART_UPLOAD_COMPLETE = "REST.POST.UPLOAD"; + public static final String MULTIPART_UPLOADS_LIST = "REST.GET.UPLOADS"; + public static final String MULTIPART_UPLOAD_ABORT = "REST.DELETE.UPLOAD"; + public static final String PUT = "REST.PUT.OBJECT"; + public static final String REST_GET_POLICY_STATUS = "REST.GET.POLICY_STATUS"; + public static final String REST_GET_PUBLIC_ACCESS_BLOCK = + "REST.GET.PUBLIC_ACCESS_BLOCK"; + public static final String REST_GET_TAGGING = "REST.GET.TAGGING"; + public static final String S3_EXPIRE_OBJECT = "S3.EXPIRE.OBJECT"; + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractAuditSpanImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractAuditSpanImpl.java new file mode 100644 index 0000000000000..510fbab14f958 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractAuditSpanImpl.java @@ -0,0 +1,99 @@ +/* + * 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.hadoop.fs.s3a.audit.impl; + +import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; +import org.apache.hadoop.fs.store.audit.AuditSpan; +import org.apache.hadoop.util.Time; + +import static java.util.Objects.requireNonNull; + +/** + * Base class for the audit spans implementations.. + */ +public abstract class AbstractAuditSpanImpl implements AuditSpanS3A { + + /** + * Span ID. + */ + private final String spanId; + + /** + * Timestamp in UTC of span creation. + */ + private final long timestamp; + + private final String operationName; + + /** + * Constructor. + * @param spanId span ID. + * @param operationName operation name + */ + protected AbstractAuditSpanImpl( + final String spanId, + final String operationName) { + this(spanId, Time.now(), operationName); + } + + /** + * Constructor. + * @param spanId span ID. + * @param timestamp timestamp in millis + * @param operationName operation name + */ + protected AbstractAuditSpanImpl( + final String spanId, + final long timestamp, + final String operationName) { + this.spanId = requireNonNull(spanId); + this.timestamp = timestamp; + this.operationName = operationName; + } + + @Override + public final String getSpanId() { + return spanId; + } + + @Override + public String getOperationName() { + return operationName; + } + + @Override + public final long getTimestamp() { + return timestamp; + } + + @Override + public AuditSpanS3A activate() { + return this; + } + + /** + * Invoke {@link AuditSpan#deactivate()}. + * This is final: subclasses MUST override the + * {@code deactivate()} method. + */ + @Override + public final void close() { + deactivate(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java new file mode 100644 index 0000000000000..97ee92a20b1e3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java @@ -0,0 +1,121 @@ +/* + * 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.hadoop.fs.s3a.audit.impl; + +import java.util.UUID; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.fs.s3a.audit.OperationAuditor; +import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.service.AbstractService; + +/** + * This is a long-lived service which is created in S3A FS initialize + * (make it fast!) which provides context for tracking operations made to S3. + * An IOStatisticsStore is passed in -in production this is expected to + * be the S3AFileSystem instrumentation, which will have the + * {@code AUDIT_SPAN_START} statistic configured for counting durations. + */ +public abstract class AbstractOperationAuditor extends AbstractService + implements OperationAuditor { + + /** + * Base of IDs is a UUID. + */ + public static final String BASE = UUID.randomUUID().toString(); + + /** + * Counter to create unique auditor IDs. + */ + private static final AtomicLong SPAN_ID_COUNTER = new AtomicLong(1); + + /** + * Destination for recording statistics, especially duration/count of + * operations. + * Set in {@link #init(OperationAuditorOptions)}. + */ + private IOStatisticsStore iostatistics; + + /** + * Options: set in {@link #init(OperationAuditorOptions)}. + */ + private OperationAuditorOptions options; + + /** + * Auditor ID as a UUID. + */ + private final UUID auditorUUID = UUID.randomUUID(); + + /** + * ID of the auditor, which becomes that of the filesystem + * in request contexts. + */ + private final String auditorID = auditorUUID.toString(); + + /** + * Construct. + * @param name name + * + */ + protected AbstractOperationAuditor(final String name) { + super(name); + } + + /** + * Sets the IOStats and then calls init(). + * @param opts options to initialize with. + */ + @Override + public void init(final OperationAuditorOptions opts) { + this.options = opts; + this.iostatistics = opts.getIoStatisticsStore(); + init(opts.getConfiguration()); + } + + @Override + public String getAuditorId() { + return auditorID; + } + + /** + * Get the IOStatistics Store. + * @return the IOStatistics store updated with statistics. + */ + public IOStatisticsStore getIOStatistics() { + return iostatistics; + } + + /** + * Get the options this auditor was initialized with. + * @return options. + */ + protected OperationAuditorOptions getOptions() { + return options; + } + + /** + * Create a span ID. + * @return a unique span ID. + */ + protected final String createSpanID() { + return String.format("%s-%08d", + auditorID, SPAN_ID_COUNTER.incrementAndGet()); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java new file mode 100644 index 0000000000000..e7e775dafc0d0 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java @@ -0,0 +1,773 @@ +/* + * 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.hadoop.fs.s3a.audit.impl; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.util.ArrayList; +import java.util.List; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.HandlerContextAware; +import com.amazonaws.Request; +import com.amazonaws.Response; +import com.amazonaws.SdkBaseException; +import com.amazonaws.handlers.HandlerAfterAttemptContext; +import com.amazonaws.handlers.HandlerBeforeAttemptContext; +import com.amazonaws.handlers.RequestHandler2; +import com.amazonaws.http.HttpResponse; +import com.amazonaws.services.s3.transfer.Transfer; +import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.audit.AWSAuditEventCallbacks; +import org.apache.hadoop.fs.s3a.audit.AWSRequestAnalyzer; +import org.apache.hadoop.fs.s3a.audit.AuditFailureException; +import org.apache.hadoop.fs.s3a.audit.AuditIntegration; +import org.apache.hadoop.fs.s3a.audit.AuditManagerS3A; +import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; +import org.apache.hadoop.fs.s3a.audit.OperationAuditor; +import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions; +import org.apache.hadoop.fs.s3a.audit.S3AAuditConstants; +import org.apache.hadoop.fs.store.LogExactlyOnce; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.util.functional.FutureIO; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_FAILURE; +import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_REQUEST_EXECUTION; +import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.attachSpanToRequest; +import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.retrieveAttachedSpan; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS; + +/** + * Thread management for the active audit. + * This should be created by whatever wants to have active + * audit span tracking. + * + * It creates and starts the actual + * {@link OperationAuditor} for auditing. + * It then stores the thread-local span and returns a wrapping + * span. + * + * When the wrapper is closed/deactivated it + * will deactivate the wrapped span and then + * switch the active span to the unbounded span. + * + * The inner class {@link AWSAuditEventCallbacks} is returned + * as a request handler in {@link #createRequestHandlers()}; + * this forwards all requests to the outer {@code ActiveAuditManagerS3A}, + * which then locates the active span and forwards the request. + * If any such invocation raises an {@link AuditFailureException} + * then the IOStatistics counter for {@code AUDIT_FAILURE} + * is incremented. + * + */ +@InterfaceAudience.Private +public final class ActiveAuditManagerS3A + extends CompositeService + implements AuditManagerS3A { + + /** + * Logging. + */ + private static final Logger LOG = + LoggerFactory.getLogger(ActiveAuditManagerS3A.class); + + /** + * One of logger for warnings about span retrieval. + */ + public static final LogExactlyOnce WARN_OF_SPAN_TYPE = + new LogExactlyOnce(LOG); + + public static final String AUDIT_MANAGER_OPERATION = "AuditManagerS3A"; + + public static final String NOT_A_WRAPPED_SPAN + = "Span attached to request is not a wrapped span"; + + /** + * Audit service. + */ + private OperationAuditor auditor; + + /** + * Some basic analysis for the logs. + */ + private final AWSRequestAnalyzer analyzer = new AWSRequestAnalyzer(); + + /** + * This is the span returned to after a wrapper is closed or + * the span is reset to the unbonded span.. + */ + private WrappingAuditSpan unbondedSpan; + + /** + * Thread local span. This defaults to being + * the unbonded span. + */ + private final ThreadLocal activeSpan = + ThreadLocal.withInitial(() -> getUnbondedSpan()); + + /** + * Destination for recording statistics, especially duration/count of + * operations. + */ + private final IOStatisticsStore ioStatisticsStore; + + /** + * Instantiate. + * @param iostatistics statistics target + */ + public ActiveAuditManagerS3A(final IOStatisticsStore iostatistics) { + super("ActiveAuditManagerS3A"); + this.ioStatisticsStore = iostatistics; + } + + @Override + protected void serviceInit(final Configuration conf) throws Exception { + super.serviceInit(conf); + // create and register the service so it follows the same lifecycle + OperationAuditorOptions options = + OperationAuditorOptions.builder() + .withConfiguration(conf) + .withIoStatisticsStore(ioStatisticsStore); + auditor = AuditIntegration.createAndInitAuditor( + getConfig(), + S3AAuditConstants.AUDIT_SERVICE_CLASSNAME, + options); + addService(auditor); + LOG.debug("Audit manager initialized with audit service {}", auditor); + } + + /** + * After starting the auditor, it is queried for its + * unbonded span, which is then wrapped and stored for + * use. + */ + @Override + protected void serviceStart() throws Exception { + super.serviceStart(); + setUnbondedSpan(new WrappingAuditSpan( + auditor.getUnbondedSpan(), false)); + LOG.debug("Started audit service {}", auditor); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder(super.toString()); + sb.append(", auditor=").append(auditor); + sb.append('}'); + return sb.toString(); + } + + @Override + public OperationAuditor getAuditor() { + return auditor; + } + + /** + * Get the unbounded span. Until this manager + * is fully initialized it will return the no-op + * span. + * @return the unbounded span. + */ + private WrappingAuditSpan getUnbondedSpan() { + return unbondedSpan; + } + + /** + * Set the unbonded span. + * @param unbondedSpan the new unbonded span + */ + private void setUnbondedSpan(final WrappingAuditSpan unbondedSpan) { + this.unbondedSpan = unbondedSpan; + } + + /** + * Return the active wrapped span. + * @return a span. + */ + @Override + public AuditSpanS3A getActiveAuditSpan() { + return activeSpan(); + } + + /** + * Get the active span. + * This is the wrapped span, not the inner one, and it is + * of that type. + * @return the active WrappingAuditSpan + */ + private WrappingAuditSpan activeSpan() { + return activeSpan.get(); + } + + /** + * Set a specific span as the active span. + * This will wrap it. + * @param span span to use. + * @return the wrapped span. + */ + private AuditSpanS3A setActiveThreadSpan(AuditSpanS3A span) { + return switchToActiveSpan( + new WrappingAuditSpan(span, span.isValidSpan())); + } + + /** + * Switch to a given span. If it is null, use the + * unbounded span. + * @param span to switch to; may be null + * @return the span switched to + */ + private WrappingAuditSpan switchToActiveSpan(WrappingAuditSpan span) { + if (span != null && span.isValidSpan()) { + activeSpan.set(span); + } else { + activeSpan.set(unbondedSpan); + } + return activeSpan(); + } + + /** + * The Span ID in the audit manager is the ID of the auditor, + * which can be used in the filesystem toString() method + * to assist in correlating client logs with S3 logs. + * It is returned here as part of the implementation of + * {@link AWSAuditEventCallbacks}. + * @return the unique ID of the FS. + */ + @Override + public String getSpanId() { + return auditor != null + ? auditor.getAuditorId() + : "(auditor not yet created)"; + } + + @Override + public String getOperationName() { + return AUDIT_MANAGER_OPERATION; + } + + /** + * Start an operation; as well as invoking the audit + * service to do this, sets the operation as the + * active operation for this thread. + * @param operation operation name. + * @param path1 first path of operation + * @param path2 second path of operation + * @return a wrapped audit span + * @throws IOException failure + */ + @Override + public AuditSpanS3A createSpan(final String operation, + @Nullable final String path1, + @Nullable final String path2) throws IOException { + // must be started + Preconditions.checkState(isInState(STATE.STARTED), + "Audit Manager %s is in wrong state: %s", + this, getServiceState()); + ioStatisticsStore.incrementCounter( + Statistic.AUDIT_SPAN_CREATION.getSymbol()); + return setActiveThreadSpan(auditor.createSpan( + operation, path1, path2)); + } + + /** + * Return a request handler for the AWS SDK which + * relays to this class. + * @return a request handler. + */ + @Override + public List createRequestHandlers() + throws IOException { + + // wire up the AWS SDK To call back into this class when + // preparing to make S3 calls. + List requestHandlers = new ArrayList<>(); + requestHandlers.add(new SdkRequestHandler()); + // now look for any more handlers + final Class[] handlers = getConfig().getClasses(AUDIT_REQUEST_HANDLERS); + if (handlers != null) { + for (Class handler : handlers) { + try { + Constructor ctor = handler.getConstructor(); + requestHandlers.add((RequestHandler2)ctor.newInstance()); + } catch (ExceptionInInitializerError e) { + throw FutureIO.unwrapInnerException(e); + } catch (Exception e) { + throw new IOException(e); + } + } + } + return requestHandlers; + } + + @Override + public TransferStateChangeListener createStateChangeListener() { + final WrappingAuditSpan span = activeSpan(); + return new TransferStateChangeListener() { + @Override + public void transferStateChanged(final Transfer transfer, + final Transfer.TransferState state) { + switchToActiveSpan(span); + } + }; + } + + @Override + public boolean checkAccess(final Path path, + final S3AFileStatus status, + final FsAction mode) + throws IOException { + return auditor.checkAccess(path, status, mode); + } + + /** + * Attach a reference to the active thread span, then + * invoke the same callback on that active thread. + */ + @Override + public T requestCreated( + final T request) { + AuditSpanS3A span = getActiveAuditSpan(); + if (LOG.isTraceEnabled()) { + LOG.trace("Created Request {} in span {}", + analyzer.analyze(request), span); + } + attachSpanToRequest(request, span); + try { + return span.requestCreated(request); + } catch (AuditFailureException e) { + ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); + throw e; + } + } + + /** + * Forward to the active span. + * All invocations increment the statistics counter for + * {@link Statistic#AUDIT_REQUEST_EXECUTION}; + * failures will also increment + * {@link Statistic#AUDIT_FAILURE}; + * {@inheritDoc} + */ + @Override + public T beforeExecution( + final T request) { + ioStatisticsStore.incrementCounter(AUDIT_REQUEST_EXECUTION.getSymbol()); + + // identify the span and invoke the callback + try { + return extractAndActivateSpanFromRequest(request) + .beforeExecution(request); + } catch (AuditFailureException e) { + ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); + throw e; + } + } + + /** + * Forward to active span. + * @param request request + * @param response response. + */ + @Override + public void afterResponse(final Request request, + final Response response) + throws AuditFailureException, SdkBaseException { + try { + extractAndActivateSpanFromRequest(request) + .afterResponse(request, response); + } catch (AuditFailureException e) { + ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); + throw e; + } + } + + /** + * Get the active span from the handler context, + * falling back to the active thread span if there + * is nothing in the context. + * Provided the span is a wrapped span, the + * @param request request + * @param type of request. + * @return the callbacks + */ + private AWSAuditEventCallbacks + extractAndActivateSpanFromRequest(final T request) { + AWSAuditEventCallbacks span; + span = retrieveAttachedSpan(request); + if (span == null) { + // no span is attached. Not unusual for the copy operations, + // or for calls to GetBucketLocation made by the AWS client + LOG.debug("No audit span attached to request {}", + request); + // fall back to the active thread span. + // this will be the unbonded span if the thread is unbonded. + span = getActiveAuditSpan(); + } else { + if (span instanceof WrappingAuditSpan) { + switchToActiveSpan((WrappingAuditSpan) span); + } else { + // warn/log and continue without switching. + WARN_OF_SPAN_TYPE.warn(NOT_A_WRAPPED_SPAN + ": {}", span); + LOG.debug(NOT_A_WRAPPED_SPAN + ": {}", span); + } + } + return span; + } + + /** + * Forward to active span. + * @param request request + * @param response response. + * @param exception exception raised. + */ + @Override + public void afterError(final Request request, + final Response response, + final Exception exception) + throws AuditFailureException, SdkBaseException { + try { + extractAndActivateSpanFromRequest(request) + .afterError(request, response, exception); + } catch (AuditFailureException e) { + ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); + throw e; + } + } + + @Override + public AmazonWebServiceRequest beforeMarshalling( + final AmazonWebServiceRequest request) { + try { + return extractAndActivateSpanFromRequest(request) + .beforeMarshalling(request); + } catch (AuditFailureException e) { + ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); + throw e; + } + } + + @Override + public void beforeRequest(final Request request) { + try { + extractAndActivateSpanFromRequest(request) + .beforeRequest(request); + } catch (AuditFailureException e) { + ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); + throw e; + } + } + + @Override + public void beforeAttempt(final HandlerBeforeAttemptContext context) { + try { + extractAndActivateSpanFromRequest(context.getRequest()) + .beforeAttempt(context); + } catch (AuditFailureException e) { + ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); + throw e; + } + } + + @Override + public void afterAttempt(final HandlerAfterAttemptContext context) { + try { + extractAndActivateSpanFromRequest(context.getRequest()) + .afterAttempt(context); + } catch (AuditFailureException e) { + ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); + throw e; + } + } + + @Override + public HttpResponse beforeUnmarshalling(final Request request, + final HttpResponse httpResponse) { + try { + extractAndActivateSpanFromRequest(request.getOriginalRequest()) + .beforeUnmarshalling(request, httpResponse); + } catch (AuditFailureException e) { + ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); + throw e; + } + return httpResponse; + } + + /** + * Callbacks from the AWS SDK; all forward to the ActiveAuditManagerS3A. + * We need a separate class because the SDK requires the handler list + * to be list of {@code RequestHandler2} instances. + */ + private class SdkRequestHandler extends RequestHandler2 { + + @Override + public AmazonWebServiceRequest beforeExecution( + final AmazonWebServiceRequest request) { + return ActiveAuditManagerS3A.this.beforeExecution(request); + } + + @Override + public void afterResponse(final Request request, + final Response response) { + ActiveAuditManagerS3A.this.afterResponse(request, response); + } + + @Override + public void afterError(final Request request, + final Response response, + final Exception e) { + ActiveAuditManagerS3A.this.afterError(request, response, e); + } + + @Override + public AmazonWebServiceRequest beforeMarshalling( + final AmazonWebServiceRequest request) { + return ActiveAuditManagerS3A.this.beforeMarshalling(request); + } + + @Override + public void beforeRequest(final Request request) { + ActiveAuditManagerS3A.this.beforeRequest(request); + } + + @Override + public void beforeAttempt( + final HandlerBeforeAttemptContext context) { + ActiveAuditManagerS3A.this.beforeAttempt(context); + } + + @Override + public HttpResponse beforeUnmarshalling( + final Request request, + final HttpResponse httpResponse) { + return ActiveAuditManagerS3A.this.beforeUnmarshalling(request, + httpResponse); + } + + @Override + public void afterAttempt( + final HandlerAfterAttemptContext context) { + ActiveAuditManagerS3A.this.afterAttempt(context); + } + } + + /** + * Wraps the plugged in spans with management of the active thread + * span, including switching to the unbounded span when a valid + * span is deactivated. + * Package-private for testing. + */ + private final class WrappingAuditSpan extends AbstractAuditSpanImpl { + + /** + * Inner span. + */ + private final AuditSpanS3A span; + + /** + * Is this span considered valid? + */ + private final boolean isValid; + + /** + * Create, wrapped. + * The spanID, name, timestamp etc copied from the span being wrapped. + * Why not the isValid state? We want to set our unbonded span without + * relying on the auditor doing the right thing. + * @param span inner span. + * @param isValid is the span valid + */ + private WrappingAuditSpan( + final AuditSpanS3A span, final boolean isValid) { + super(span.getSpanId(), span.getTimestamp(), span.getOperationName()); + this.span = requireNonNull(span); + this.isValid = isValid; + } + + /** + * Is the span active? + * @return true if this span is the active one for the current thread. + */ + private boolean isActive() { + return this == getActiveAuditSpan(); + } + + /** + * Makes this the thread's active span and activate. + * If the span was already active: no-op. + */ + @Override + public AuditSpanS3A activate() { + if (!isActive()) { + switchToActiveSpan(this); + span.activate(); + } + return this; + } + + /** + * Switch to the unbounded span and then deactivate this span. + * No-op for invalid spans, + * so as to prevent the unbounded span from being closed + * and everything getting very confused. + */ + @Override + public void deactivate() { + // no-op for invalid spans, + // so as to prevent the unbounded span from being closed + // and everything getting very confused. + if (!isValid || !isActive()) { + return; + } + // deactivate the span + span.deactivate(); + // and go to the unbounded one. + switchToActiveSpan(getUnbondedSpan()); + } + + /** + * Forward to the wrapped span. + * {@inheritDoc} + */ + @Override + public T requestCreated( + final T request) { + return span.requestCreated(request); + } + + /** + * This span is valid if the span isn't closed and the inner + * span is valid. + * @return true if the span is considered valid. + */ + @Override + public boolean isValidSpan() { + return isValid && span.isValidSpan(); + } + + /** + * Forward to the inner span. + * {@inheritDoc} + */ + @Override + public void set(final String key, final String value) { + span.set(key, value); + } + + /** + * Forward to the inner span. + * @param request request + * @param type of request + * @return an updated request. + */ + @Override + public T beforeExecution( + final T request) { + return span.beforeExecution(request); + } + + /** + * Forward to the inner span. + * @param request request + * @param response response. + */ + @Override + public void afterResponse(final Request request, + final Response response) { + span.afterResponse(request, response); + } + + /** + * Forward to the inner span. + * @param request request + * @param response response. + * @param exception exception raised. + */ + @Override + public void afterError(final Request request, + final Response response, + final Exception exception) { + span.afterError(request, response, exception); + } + + /** + * Forward to the inner span. + * @param request request + * @return request to marshall + */ + @Override + public AmazonWebServiceRequest beforeMarshalling( + final AmazonWebServiceRequest request) { + return span.beforeMarshalling(request); + } + + /** + * Forward to the inner span. + * @param request request + */ + @Override + public void beforeRequest(final Request request) { + span.beforeRequest(request); + } + + /** + * Forward to the inner span. + * @param context full context, including the request. + */ + @Override + public void beforeAttempt( + final HandlerBeforeAttemptContext context) { + span.beforeAttempt(context); + } + + /** + * Forward to the inner span. + * + * @param context full context, including the request. + */ + @Override + public void afterAttempt( + final HandlerAfterAttemptContext context) { + span.afterAttempt(context); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "WrappingAuditSpan{"); + sb.append("span=").append(span); + sb.append(", valid=").append(isValidSpan()); + sb.append('}'); + return sb.toString(); + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java new file mode 100644 index 0000000000000..a130a2a9c9f75 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java @@ -0,0 +1,447 @@ +/* + * 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.hadoop.fs.s3a.audit.impl; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import com.amazonaws.AmazonWebServiceRequest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.audit.AuditConstants; +import org.apache.hadoop.fs.audit.CommonAuditContext; +import org.apache.hadoop.fs.s3a.audit.AWSRequestAnalyzer; +import org.apache.hadoop.fs.s3a.audit.AuditFailureException; +import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; +import org.apache.hadoop.fs.store.audit.HttpReferrerAuditHeader; +import org.apache.hadoop.security.UserGroupInformation; + +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_FILESYSTEM_ID; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PRINCIPAL; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD0; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_TIMESTAMP; +import static org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext; +import static org.apache.hadoop.fs.audit.CommonAuditContext.currentThreadID; +import static org.apache.hadoop.fs.s3a.audit.AWSRequestAnalyzer.isRequestNotAlwaysInSpan; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.OUTSIDE_SPAN; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_ENABLED; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_ENABLED_DEFAULT; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_FILTER; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REJECT_OUT_OF_SPAN_OPERATIONS; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.UNAUDITED_OPERATION; +import static org.apache.hadoop.fs.s3a.commit.CommitUtils.extractJobID; +import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.HEADER_REFERRER; + +/** + * The LoggingAuditor logs operations at DEBUG (in SDK Request) and + * in span lifecycle and S3 request class construction at TRACE. + * The context information is added as the HTTP referrer. + */ +@InterfaceAudience.Private +public class LoggingAuditor + extends AbstractOperationAuditor { + + /** + * This is where the context gets logged to. + */ + private static final Logger LOG = + LoggerFactory.getLogger(LoggingAuditor.class); + + + /** + * Some basic analysis for the logs. + */ + private final AWSRequestAnalyzer analyzer = new AWSRequestAnalyzer(); + + /** + * Default span to use when there is no other. + */ + private AuditSpanS3A warningSpan; + + /** + * Should out of scope ops be rejected? + */ + private boolean rejectOutOfSpan; + + /** + * Map of attributes which will be added to all operations. + */ + private final Map attributes = new HashMap<>(); + + /** + * UGI principal at time of creation. + * This is mapped into the common context if it is not already set there + * when a span is created. + */ + private final String principal; + + /** + * Should the referrer header be added? + */ + private boolean headerEnabled; + + /** + * This is the header sent by the last S3 operation through + * this auditor. + *

+ * It is for testing -allows for Integration tests to + * verify that a header was sent and query what was in it. + * Initially an empty string. + */ + private volatile String lastHeader = ""; + + /** + * Attributes to filter. + */ + private Collection filters; + + /** + * Create the auditor. + * The UGI current user is used to provide the principal; + * this will be cached and provided in the referrer header. + */ + public LoggingAuditor() { + + super("LoggingAuditor "); + attributes.put(PARAM_FILESYSTEM_ID, getAuditorId()); + + + // add the principal + String p; + try { + UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); + p = ugi.getUserName(); + addAttribute(PARAM_PRINCIPAL, p); + } catch (IOException ignored) { + p = ""; + } + principal = p; + } + + /** + * Service init, look for jobID and attach as an attribute in log entries. + * This is where the warning span is created, so the relevant attributes + * (and filtering options) are applied. + * @param conf configuration + * @throws Exception failure + */ + @Override + protected void serviceInit(final Configuration conf) throws Exception { + super.serviceInit(conf); + rejectOutOfSpan = conf.getBoolean( + REJECT_OUT_OF_SPAN_OPERATIONS, false); + // attach the job ID if there is one in the configuration used + // to create this file. + String jobID = extractJobID(conf); + if (jobID != null) { + addAttribute(AuditConstants.PARAM_JOB_ID, jobID); + } + headerEnabled = getConfig().getBoolean(REFERRER_HEADER_ENABLED, + REFERRER_HEADER_ENABLED_DEFAULT); + filters = conf.getTrimmedStringCollection(REFERRER_HEADER_FILTER); + final CommonAuditContext currentContext = currentAuditContext(); + warningSpan = new WarningSpan(OUTSIDE_SPAN, + currentContext, createSpanID(), null, null); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "LoggingAuditor{"); + sb.append("ID='").append(getAuditorId()).append('\''); + sb.append(", headerEnabled=").append(headerEnabled); + sb.append(", rejectOutOfSpan=").append(rejectOutOfSpan); + sb.append('}'); + return sb.toString(); + } + + @Override + public AuditSpanS3A createSpan(final String operation, + @Nullable final String path1, + @Nullable final String path2) { + LoggingAuditSpan span = new LoggingAuditSpan( + createSpanID(), + operation, + prepareActiveContext(), + path1, + path2); + span.start(); + return span; + } + + /** + * Get/Prepare the active context for a span. + * @return the common audit context. + */ + private CommonAuditContext prepareActiveContext() { + return currentAuditContext(); + } + + /** + * Add an attribute. + * @param key key + * @param value value + */ + public final void addAttribute(String key, String value) { + attributes.put(key, value); + } + + @Override + public AuditSpanS3A getUnbondedSpan() { + return warningSpan; + } + + /** + * Get the last header used. + * @return the last referrer header generated. + */ + public String getLastHeader() { + return lastHeader; + } + + /** + * Set that last header. + * @param lastHeader the value for the lastHeader field. + */ + private void setLastHeader(final String lastHeader) { + this.lastHeader = lastHeader; + } + + /** + * Span which logs at debug and sets the HTTP referrer on + * invocations. + * Note: checkstyle complains that this should be final because + * it is private. This is not true, as it is subclassed in + * the same file. + */ + private class LoggingAuditSpan extends AbstractAuditSpanImpl { + + private final HttpReferrerAuditHeader referrer; + + private final String description; + + private LoggingAuditSpan( + final String spanId, + final String operationName, + final CommonAuditContext context, + final String path1, + final String path2) { + super(spanId, operationName); + + this.referrer = HttpReferrerAuditHeader.builder() + .withContextId(getAuditorId()) + .withSpanId(spanId) + .withOperationName(operationName) + .withPath1(path1) + .withPath2(path2) + .withAttributes(attributes) + // thread at the time of creation. + .withAttribute(PARAM_THREAD0, + currentThreadID()) + // principal when the auditor was created. + .withAttribute(PARAM_PRINCIPAL, principal) + .withAttribute(PARAM_TIMESTAMP, Long.toString(getTimestamp())) + .withEvaluated(context.getEvaluatedEntries()) + .withFilter(filters) + .build(); + + this.description = referrer.buildHttpReferrer(); + } + + public void start() { + LOG.trace("{} Start {}", getSpanId(), getDescription()); + } + + /** + * Get the span description built in the constructor. + * @return description text. + */ + protected String getDescription() { + return description; + } + + /** + * Activate: log at TRACE. + * @return this span. + */ + @Override + public AuditSpanS3A activate() { + LOG.trace("[{}] {} Activate {}", + currentThreadID(), getSpanId(), getDescription()); + return this; + } + + /** + * Log at TRACE. + */ + @Override + public void deactivate() { + LOG.trace("[{}] {} Deactivate {}", + currentThreadID(), getSpanId(), getDescription()); + } + + + /** + * Pass to the HTTP referrer. + * {@inheritDoc} + */ + @Override + public void set(final String key, final String value) { + referrer.set(key, value); + } + + /** + * Before execution, the logging auditor always builds + * the referrer header, saves to the outer class + * (where {@link #getLastHeader()} can retrieve it, + * and logs at debug. + * If configured to add the header to the S3 logs, it will + * be set as the HTTP referrer. + * @param request request + * @param type of request. + * @return the request with any extra headers. + */ + @Override + public T beforeExecution( + final T request) { + // build the referrer header + final String header = referrer.buildHttpReferrer(); + // update the outer class's field. + setLastHeader(header); + if (headerEnabled) { + // add the referrer header + request.putCustomRequestHeader(HEADER_REFERRER, + header); + } + if (LOG.isDebugEnabled()) { + LOG.debug("[{}] {} Executing {} with {}; {}", + currentThreadID(), + getSpanId(), + getOperationName(), + analyzer.analyze(request), + header); + } + return request; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "LoggingAuditSpan{"); + sb.append(", id='").append(getSpanId()).append('\''); + sb.append("description='").append(description).append('\''); + sb.append('}'); + return sb.toString(); + } + + /** + * Get the referrer; visible for tests. + * @return the referrer. + */ + HttpReferrerAuditHeader getReferrer() { + return referrer; + } + } + + /** + * Span which logs at WARN; used to highlight spans + * without a containing span. + */ + private final class WarningSpan extends LoggingAuditSpan { + + private WarningSpan( + final String name, + final CommonAuditContext context, + final String spanId, + final String path1, final String path2) { + super(spanId, name, context, path1, path2); + } + + @Override + public void start() { + LOG.warn("[{}] {} Start {}", + currentThreadID(), getSpanId(), getDescription()); + } + + @Override + public AuditSpanS3A activate() { + LOG.warn("[{}] {} Activate {}", + currentThreadID(), getSpanId(), getDescription()); + return this; + } + + @Override + public boolean isValidSpan() { + return false; + } + + @Override + public T requestCreated( + final T request) { + String error = "Creating a request outside an audit span " + + analyzer.analyze(request); + LOG.info(error); + if (LOG.isDebugEnabled()) { + LOG.debug(error, new AuditFailureException("unaudited")); + } + return request; + } + + /** + * Handle requests made without a real context by logging and + * increment the failure count. + * Some requests (e.g. copy part) are not expected in spans due + * to how they are executed; these do not trigger failures. + * @param request request + * @param type of request + * @return an updated request. + * @throws AuditFailureException if failure is enabled. + */ + @Override + public T beforeExecution( + final T request) { + + + String error = "executing a request outside an audit span " + + analyzer.analyze(request); + LOG.warn("{} {}", + getSpanId(), error); + final String unaudited = getSpanId() + " " + + UNAUDITED_OPERATION + " " + error; + if (isRequestNotAlwaysInSpan(request)) { + // can get by auditing during a copy, so don't overreact + LOG.debug(unaudited); + } else { + final RuntimeException ex = new AuditFailureException(unaudited); + LOG.debug(unaudited, ex); + if (rejectOutOfSpan) { + throw ex; + } + } + // now hand off to the superclass for its normal preparation + return super.beforeExecution(request); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManagerS3A.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManagerS3A.java new file mode 100644 index 0000000000000..2e51699da6f44 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManagerS3A.java @@ -0,0 +1,185 @@ +/* + * 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.hadoop.fs.s3a.audit.impl; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; + +import com.amazonaws.handlers.RequestHandler2; +import com.amazonaws.services.s3.transfer.Transfer; +import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.audit.AuditManagerS3A; +import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; +import org.apache.hadoop.fs.s3a.audit.OperationAuditor; +import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions; +import org.apache.hadoop.service.CompositeService; + +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; + +/** + * Simple No-op audit manager for use before a real + * audit chain is set up, and for testing. + * Audit spans always have a unique ID and the activation/deactivation + * operations on them will update this audit manager's active span. + * It does have the service lifecycle, so do + * create a unique instance whenever used. + */ +@InterfaceAudience.Private +public class NoopAuditManagerS3A extends CompositeService + implements AuditManagerS3A, NoopSpan.SpanActivationCallbacks { + + private static final NoopAuditor NOOP_AUDITOR = + NoopAuditor.createAndStartNoopAuditor(new Configuration(), null); + + /** + * The inner auditor. + */ + private NoopAuditor auditor = NOOP_AUDITOR; + + /** + * Thread local span. This defaults to being + * the unbonded span. + */ + private final ThreadLocal activeSpan = + ThreadLocal.withInitial(this::getUnbondedSpan); + + /** + * ID which is returned as a span ID in the audit event + * callbacks. + */ + private final String id; + + /** + * Constructor. + * Will create and start a new instance of the auditor. + */ + public NoopAuditManagerS3A() { + super("NoopAuditManagerS3A"); + id = UUID.randomUUID().toString(); + } + + @Override + protected void serviceInit(final Configuration conf) throws Exception { + super.serviceInit(conf); + NoopAuditor audit = new NoopAuditor(this); + final OperationAuditorOptions options = + OperationAuditorOptions.builder() + .withConfiguration(conf) + .withIoStatisticsStore(iostatisticsStore().build()); + addService(audit); + audit.init(options); + } + + @Override + public String getSpanId() { + return id; + } + + @Override + public String getOperationName() { + return getName(); + } + + @Override + public OperationAuditor getAuditor() { + return auditor; + } + + /** + * Unbonded span to use after deactivation. + */ + private AuditSpanS3A getUnbondedSpan() { + return auditor.getUnbondedSpan(); + } + + @Override + public AuditSpanS3A getActiveAuditSpan() { + return NoopSpan.INSTANCE; + } + + @Override + public AuditSpanS3A createSpan(final String operation, + @Nullable final String path1, + @Nullable final String path2) throws IOException { + return createNewSpan(operation, path1, path2); + } + + @Override + public List createRequestHandlers() throws IOException { + return new ArrayList<>(); + } + + @Override + public TransferStateChangeListener createStateChangeListener() { + return new TransferStateChangeListener() { + public void transferStateChanged(final Transfer transfer, + final Transfer.TransferState state) { + } + }; + } + + /** + * Forward to the auditor. + * @param path path to check + * @param status status of the path. + * @param mode access mode. + * @throws IOException failure + */ + @Override + public boolean checkAccess(final Path path, + final S3AFileStatus status, + final FsAction mode) + throws IOException { + return auditor.checkAccess(path, status, mode); + } + + @Override + public void activate(final AuditSpanS3A span) { + activeSpan.set(span); + } + + @Override + public void deactivate(final AuditSpanS3A span) { + activate(getUnbondedSpan()); + } + + /** + * A static source of no-op spans, using the same span ID + * source as managed spans. + * @param name operation name. + * @param path1 first path of operation + * @param path2 second path of operation + * @return a span for the audit + */ + public static AuditSpanS3A createNewSpan( + final String name, + final String path1, + final String path2) { + return NOOP_AUDITOR.createSpan(name, path1, path2); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditor.java new file mode 100644 index 0000000000000..ca6a7dd840fa8 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditor.java @@ -0,0 +1,99 @@ +/* + * 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.hadoop.fs.s3a.audit.impl; + +import javax.annotation.Nullable; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; +import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions; + +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; + +/** + * An audit service which returns the {@link NoopSpan}. + * Even though the spans are no-ops, each span is still + * created with a unique span ID. + */ +public class NoopAuditor extends AbstractOperationAuditor { + + /** + * unbonded span created in constructor. + */ + private final AuditSpanS3A unbondedSpan; + + /** + * Activation callbacks. + */ + private final NoopSpan.SpanActivationCallbacks activationCallbacks; + + /** + * Constructor. + * This will be used when the auditor is created through + * configuration and classloading. + */ + public NoopAuditor() { + this(null); + } + + /** + * Constructor when explicitly created within + * the {@link NoopAuditManagerS3A}. + * @param activationCallbacks Activation callbacks. + */ + public NoopAuditor( + NoopSpan.SpanActivationCallbacks activationCallbacks) { + super("NoopAuditor"); + this.unbondedSpan = createSpan("unbonded", null, null); + this.activationCallbacks = activationCallbacks; + } + + @Override + public AuditSpanS3A createSpan( + final String operation, + @Nullable final String path1, + @Nullable final String path2) { + return new NoopSpan(createSpanID(), operation, path1, path2, + activationCallbacks); + } + + @Override + public AuditSpanS3A getUnbondedSpan() { + return unbondedSpan; + } + + /** + * Create, init and start an instance. + * @param conf configuration. + * @param activationCallbacks Activation callbacks. + * @return a started instance. + */ + public static NoopAuditor createAndStartNoopAuditor(Configuration conf, + NoopSpan.SpanActivationCallbacks activationCallbacks) { + NoopAuditor noop = new NoopAuditor(activationCallbacks); + final OperationAuditorOptions options = + OperationAuditorOptions.builder() + .withConfiguration(conf) + .withIoStatisticsStore(iostatisticsStore().build()); + noop.init(options); + noop.start(); + return noop; + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopSpan.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopSpan.java new file mode 100644 index 0000000000000..0b311dcdf61ab --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopSpan.java @@ -0,0 +1,109 @@ +/* + * 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.hadoop.fs.s3a.audit.impl; + +import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; + +/** + * A minimal span with no direct side effects. + * It does have an ID and, if given callbacks, + * will notify the callback implementation + * of activation and deactivation. + * Subclassable for tests. + */ +public class NoopSpan extends AbstractAuditSpanImpl { + + private final String path1; + + private final String path2; + + /** Activation callbacks. */ + private final SpanActivationCallbacks activationCallbacks; + + /** + * Static public instance. + */ + public static final NoopSpan INSTANCE = new NoopSpan(); + + /** + * Create a no-op span. + * @param spanId span ID + * @param operationName operation name + * @param path1 path + * @param path2 path 2 + * @param activationCallbacks Activation callbacks. + */ + protected NoopSpan(String spanId, + final String operationName, + final String path1, + final String path2, + final SpanActivationCallbacks activationCallbacks) { + super(spanId, operationName); + this.path1 = path1; + this.path2 = path2; + this.activationCallbacks = activationCallbacks; + } + + protected NoopSpan() { + this("", "no-op", null, null, null); + } + + + @Override + public AuditSpanS3A activate() { + if (activationCallbacks != null) { + activationCallbacks.activate(this); + } + return this; + } + + @Override + public void deactivate() { + if (activationCallbacks != null) { + activationCallbacks.deactivate(this); + } + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("NoopSpan{"); + sb.append("id='").append(getSpanId()).append('\''); + sb.append("name='").append(getOperationName()).append('\''); + sb.append(", path1='").append(path1).append('\''); + sb.append(", path2='").append(path2).append('\''); + sb.append('}'); + return sb.toString(); + } + + /** Activation callbacks. */ + public interface SpanActivationCallbacks { + + /** + * Span was activated. + * @param span span reference. + */ + void activate(AuditSpanS3A span); + + /** + * Span was deactivated. + * @param span span reference. + */ + void deactivate(AuditSpanS3A span); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/S3AInternalAuditConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/S3AInternalAuditConstants.java new file mode 100644 index 0000000000000..f82e3d7f1e5e8 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/S3AInternalAuditConstants.java @@ -0,0 +1,44 @@ +/* + * 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.hadoop.fs.s3a.audit.impl; + +import com.amazonaws.handlers.HandlerContextKey; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.s3a.audit.AWSAuditEventCallbacks; + +/** + * Internal constants; not intended for public use, or + * for use by any external implementations. + */ +@InterfaceAudience.Private +public final class S3AInternalAuditConstants { + + private S3AInternalAuditConstants() { + } + + /** + * Handler key for audit span callbacks. + * This is used to bind the handler in the AWS code. + */ + public static final HandlerContextKey + AUDIT_SPAN_HANDLER_CONTEXT = + new HandlerContextKey<>( + "org.apache.hadoop.fs.s3a.audit.AWSAuditEventCallbacks"); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/package-info.java new file mode 100644 index 0000000000000..d4553cc159fb2 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/package-info.java @@ -0,0 +1,28 @@ +/* + * 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. + */ + +/** + * The internal implementations of auditing support. + * This package is not for use by extensions. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.audit.impl; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/package-info.java new file mode 100644 index 0000000000000..5e4f8580e98c3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/package-info.java @@ -0,0 +1,41 @@ +/* + * 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. + */ + +/** + * Support for auditing and ultimately tracing operations. + * This is a public API for extension points, e.g. opentracing. + * However, it is very unstable as we evolve how best to audit/trace + * operation. + * + * An audit service is instantiated when an S3A Filesystem is initialized + * during creation. + * The choice of service is determined in the configuration option + * {@link org.apache.hadoop.fs.s3a.audit.S3AAuditConstants#AUDIT_SERVICE_CLASSNAME}. + * The service MUST implement the interface + * {@link org.apache.hadoop.fs.s3a.audit.OperationAuditor} + * to provide an {@link org.apache.hadoop.fs.store.audit.AuditSpan} whenever + * an operation is started through a public FileSystem API call + * (+some other operations). + */ + +@InterfaceAudience.LimitedPrivate("S3A auditing extensions") +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.audit; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java index f73689979ec25..1fd24969d723d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.commit; +import javax.annotation.Nullable; import java.io.FileNotFoundException; import java.io.IOException; import java.text.DateFormat; @@ -30,6 +31,7 @@ import java.util.concurrent.TimeUnit; import com.amazonaws.services.s3.model.MultipartUpload; + import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -42,8 +44,11 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.audit.AuditConstants; +import org.apache.hadoop.fs.audit.CommonAuditContext; +import org.apache.hadoop.fs.store.audit.AuditSpan; +import org.apache.hadoop.fs.store.audit.AuditSpanSource; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; @@ -60,11 +65,13 @@ import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.concurrent.HadoopExecutors; +import org.apache.hadoop.util.functional.InvocationRaisingIOE; import static org.apache.hadoop.fs.s3a.Constants.THREAD_POOL_SHUTDOWN_DELAY_SECONDS; import static org.apache.hadoop.fs.s3a.Invoker.ignoreIOExceptions; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_COMMIT_JOB; +import static org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*; @@ -129,6 +136,8 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter */ private final JobUUIDSource uuidSource; + private final CommonAuditContext commonAuditContext; + /** * Has this instance been used for job setup? * If so then it is safe for a locally generated @@ -175,6 +184,11 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter private final CommitterStatistics committerStatistics; + /** + * Source of Audit spans. + */ + private final AuditSpanSource auditSpanSource; + /** * Create a committer. * This constructor binds the destination directory and configuration, but @@ -203,6 +217,13 @@ protected AbstractS3ACommitter( LOG.debug("{} instantiated for job \"{}\" ID {} with destination {}", role, jobName(context), jobIdString(context), outputPath); S3AFileSystem fs = getDestS3AFS(); + // set this thread's context with the job ID. + // audit spans created in this thread will pick + // up this value. + this.commonAuditContext = currentAuditContext(); + updateCommonContext(); + // the filesystem is the span source, always. + auditSpanSource = fs.getAuditSpanSource(); this.createJobMarker = context.getConfiguration().getBoolean( CREATE_SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER); @@ -535,6 +556,8 @@ public void setupJob(JobContext context) throws IOException { @Override public void setupTask(TaskAttemptContext context) throws IOException { TaskAttemptID attemptID = context.getTaskAttemptID(); + updateCommonContext(); + try (DurationInfo d = new DurationInfo(LOG, "Setup Task %s", attemptID)) { // reject attempts to set up the task where the output won't be @@ -947,11 +970,11 @@ public void cleanupJob(JobContext context) throws IOException { protected void maybeIgnore( boolean suppress, String action, - Invoker.VoidOperation operation) throws IOException { + InvocationRaisingIOE operation) throws IOException { if (suppress) { ignoreIOExceptions(LOG, action, "", operation); } else { - operation.execute(); + operation.apply(); } } @@ -1361,6 +1384,43 @@ public String toString() { } } + /** + * Add jobID to current context. + */ + protected final void updateCommonContext() { + currentAuditContext().put(AuditConstants.PARAM_JOB_ID, uuid); + } + + /** + * Remove JobID from the current thread's context. + */ + protected final void resetCommonContext() { + currentAuditContext().remove(AuditConstants.PARAM_JOB_ID); + } + + protected AuditSpanSource getAuditSpanSource() { + return auditSpanSource; + } + + /** + * Start an operation; retrieve an audit span. + * + * All operation names SHOULD come from + * {@code StoreStatisticNames} or + * {@code StreamStatisticNames}. + * @param name operation name. + * @param path1 first path of operation + * @param path2 second path of operation + * @return a span for the audit + * @throws IOException failure + */ + protected AuditSpan startOperation(String name, + @Nullable String path1, + @Nullable String path2) + throws IOException { + return getAuditSpanSource().createSpan(name, path1, path2); + } + /** * State of the active commit operation. * diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java index cd8257f97fb91..a4f13b78a3f0c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java @@ -48,10 +48,11 @@ import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AUtils; -import org.apache.hadoop.fs.s3a.WriteOperationHelper; +import org.apache.hadoop.fs.s3a.WriteOperations; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; +import org.apache.hadoop.fs.s3a.impl.AbstractStoreOperation; import org.apache.hadoop.fs.s3a.impl.HeaderProcessing; import org.apache.hadoop.fs.s3a.impl.InternalConstants; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; @@ -65,11 +66,13 @@ import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_COMMIT_JOB; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MATERIALIZE_FILE; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_STAGE_FILE_UPLOAD; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration; +import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator; /** * The implementation of the various actions a committer needs. @@ -81,7 +84,8 @@ * duplicate that work. * */ -public class CommitOperations implements IOStatisticsSource { +public class CommitOperations extends AbstractStoreOperation + implements IOStatisticsSource { private static final Logger LOG = LoggerFactory.getLogger( CommitOperations.class); @@ -96,7 +100,7 @@ public class CommitOperations implements IOStatisticsSource { /** * Write operations for the destination fs. */ - private final WriteOperationHelper writeOperations; + private final WriteOperations writeOperations; /** * Filter to find all {code .pendingset} files. @@ -113,21 +117,29 @@ public class CommitOperations implements IOStatisticsSource { /** * Instantiate. * @param fs FS to bind to + * @throws IOException failure to bind. */ - public CommitOperations(S3AFileSystem fs) { + public CommitOperations(S3AFileSystem fs) throws IOException { this(requireNonNull(fs), fs.newCommitterStatistics()); } /** - * Instantiate. + * Instantiate. This creates a new audit span for + * the commit operations. * @param fs FS to bind to * @param committerStatistics committer statistics + * @throws IOException failure to bind. */ public CommitOperations(S3AFileSystem fs, - CommitterStatistics committerStatistics) { - this.fs = requireNonNull(fs); + CommitterStatistics committerStatistics) throws IOException { + super(requireNonNull(fs).createStoreContext()); + this.fs = fs; statistics = requireNonNull(committerStatistics); - writeOperations = fs.getWriteOperationHelper(); + // create a span + writeOperations = fs.createWriteOperationHelper( + fs.getAuditSpanSource().createSpan( + COMMITTER_COMMIT_JOB.getSymbol(), + "/", null)); } /** @@ -365,6 +377,7 @@ public MaybeIOE abortAllSinglePendingCommits(Path pendingDir, } } } + cleanupRemoteIterator(pendingFiles); return outcome; } @@ -388,7 +401,7 @@ protected RemoteIterator ls(Path path, boolean recursive) */ public List listPendingUploadsUnderPath(Path dest) throws IOException { - return fs.listMultipartUploads(fs.pathToKey(dest)); + return writeOperations.listMultipartUploads(fs.pathToKey(dest)); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtils.java index 9c684c77e1b25..6ea65b9132f8f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtils.java @@ -125,5 +125,29 @@ public static void validateCollectionClass(Iterable it, Class classname) } } + /** + * Extract the job ID from a configuration. + * @param conf configuration + * @return a job ID or null. + */ + public static String extractJobID(Configuration conf) { + + String jobUUID = conf.getTrimmed(FS_S3A_COMMITTER_UUID, ""); + + if (!jobUUID.isEmpty()) { + return jobUUID; + } + // there is no job UUID. + // look for one from spark + jobUUID = conf.getTrimmed(SPARK_WRITE_UUID, ""); + if (!jobUUID.isEmpty()) { + return jobUUID; + } + jobUUID = conf.getTrimmed(MR_JOB_ID, ""); + if (!jobUUID.isEmpty()) { + return jobUUID; + } + return null; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java index 461c9a5e646c3..fcafdd1ed1280 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java @@ -124,4 +124,10 @@ private InternalCommitterConstants() { "Job/task context does not contain a unique ID in " + SPARK_WRITE_UUID; + /** + * The MR job ID; copies from MRJobConfig so that it can be + * referred to without needing hadoop-mapreduce on the classpath. + */ + public static final String MR_JOB_ID = "mapreduce.job.id"; + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java index 41f95c8620692..41d36b2a8d7a0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker; -import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.impl.AbstractStoreOperation; import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*; @@ -45,14 +45,12 @@ *

Important

: must not directly or indirectly import a class which * uses any datatype in hadoop-mapreduce. */ -public class MagicCommitIntegration { +public class MagicCommitIntegration extends AbstractStoreOperation { private static final Logger LOG = LoggerFactory.getLogger(MagicCommitIntegration.class); private final S3AFileSystem owner; private final boolean magicCommitEnabled; - private final StoreContext storeContext; - /** * Instantiate. * @param owner owner class @@ -60,9 +58,9 @@ public class MagicCommitIntegration { */ public MagicCommitIntegration(S3AFileSystem owner, boolean magicCommitEnabled) { + super(owner.createStoreContext()); this.owner = owner; this.magicCommitEnabled = magicCommitEnabled; - this.storeContext = owner.createStoreContext(); } /** @@ -85,6 +83,9 @@ public String keyOfFinalDestination(List elements, String key) { * Given a path and a key to that same path, create a tracker for it. * This specific tracker will be chosen based on whether or not * the path is a magic one. + * Auditing: the span used to invoke + * this method will be the one used to create the write operation helper + * for the commit tracker. * @param path path of nominal write * @param key key of path of nominal write * @return the tracker for this operation. @@ -98,10 +99,10 @@ public PutTracker createTracker(Path path, String key) { if (isMagicCommitPath(elements)) { final String destKey = keyOfFinalDestination(elements, key); String pendingsetPath = key + CommitConstants.PENDING_SUFFIX; - storeContext.incrementStatistic( + getStoreContext().incrementStatistic( Statistic.COMMITTER_MAGIC_FILES_CREATED); tracker = new MagicCommitTracker(path, - storeContext.getBucket(), + getStoreContext().getBucket(), key, destKey, pendingsetPath, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java index c3a70bffac057..71af2bef24bd8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java @@ -65,7 +65,7 @@ public class MagicCommitTracker extends PutTracker { * @param originalDestKey the original key, in the magic directory. * @param destKey key for the destination * @param pendingsetKey key of the pendingset file - * @param writer writer instance to use for operations + * @param writer writer instance to use for operations; includes audit span */ public MagicCommitTracker(Path path, String bucket, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java index 59114f7ab7340..ea49988c101e8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java @@ -56,6 +56,7 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*; +import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator; /** * Committer based on the contributed work of the @@ -828,6 +829,7 @@ protected PathExistsException failDestinationExists(final Path path, ? " dir" : ("file size " + status.getLen() + " bytes")); } + cleanupRemoteIterator(lf); } catch (IOException e) { LOG.info("Discarding exception raised when listing {}: " + e, path); LOG.debug("stack trace ", e); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java index 98b76b15da48f..cbca14660d35f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java @@ -18,24 +18,46 @@ package org.apache.hadoop.fs.s3a.impl; +import org.apache.hadoop.fs.store.audit.AuditSpan; + import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; /** * Base class of operations in the store. * An operation is something which executes against the context to * perform a single function. - * It is expected to have a limited lifespan. */ public abstract class AbstractStoreOperation { + /** + * Store context. + */ private final StoreContext storeContext; /** - * constructor. + * Audit Span. + */ + private AuditSpan auditSpan; + + /** + * Constructor. + * Picks up the active audit span from the store context and + * stores it for later. * @param storeContext store context. */ protected AbstractStoreOperation(final StoreContext storeContext) { + this(storeContext, storeContext.getActiveAuditSpan()); + } + + /** + * Constructor. + * @param storeContext store context. + * @param auditSpan active span + */ + protected AbstractStoreOperation(final StoreContext storeContext, + final AuditSpan auditSpan) { this.storeContext = checkNotNull(storeContext); + this.auditSpan = checkNotNull(auditSpan); } /** @@ -46,4 +68,18 @@ public final StoreContext getStoreContext() { return storeContext; } + /** + * Get the audit span this object was created with. + * @return the current span + */ + public AuditSpan getAuditSpan() { + return auditSpan; + } + + /** + * Activate the audit span. + */ + public void activateAuditSpan() { + auditSpan.activate(); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java index b788f507735a0..259738f99892a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java @@ -32,6 +32,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.util.DurationInfo; import static org.apache.hadoop.fs.impl.FutureIOSupport.raiseInnerCause; @@ -41,24 +42,48 @@ * raised by the callable and wrapping them as appropriate. * @param return type. */ -public final class CallableSupplier implements Supplier { +public final class CallableSupplier implements Supplier { private static final Logger LOG = LoggerFactory.getLogger(CallableSupplier.class); private final Callable call; + /** + * Audit Span; may be null. + */ + private final AuditSpan auditSpan; + /** * Create. * @param call call to invoke. */ public CallableSupplier(final Callable call) { + this(null, call); + } + + /** + * Create. + * @param auditSpan audit span (or null) + * @param call call to invoke. + */ + public CallableSupplier( + final AuditSpan auditSpan, + final Callable call) { this.call = call; + this.auditSpan = auditSpan; } + /** + * Active any span and then call the supplied callable. + * @return the result. + */ @Override - public Object get() { + public T get() { try { + if (auditSpan != null) { + auditSpan.activate(); + } return call.call(); } catch (RuntimeException e) { throw e; @@ -86,11 +111,32 @@ public static CompletableFuture submit( return CompletableFuture.supplyAsync( new CallableSupplier(call), executor); } + /** + * Submit a callable into a completable future. + * RTEs are rethrown. + * Non RTEs are caught and wrapped; IOExceptions to + * {@code RuntimeIOException} instances. + * @param executor executor. + * @param auditSpan audit span (or null) + * @param call call to invoke + * @param type + * @return the future to wait for + */ + @SuppressWarnings("unchecked") + public static CompletableFuture submit( + final Executor executor, + final AuditSpan auditSpan, + final Callable call) { + return CompletableFuture.supplyAsync( + new CallableSupplier(auditSpan, call), + executor); + } /** * Wait for a list of futures to complete. If the list is empty, * return immediately. * @param futures list of futures. + * @param type * @throws IOException if one of the called futures raised an IOE. * @throws RuntimeException if one of the futures raised one. */ @@ -108,6 +154,7 @@ public static void waitForCompletion( /** * Wait for a single of future to complete, extracting IOEs afterwards. * @param future future to wait for. + * @param type * @throws IOException if one of the called futures raised an IOE. * @throws RuntimeException if one of the futures raised one. */ @@ -127,6 +174,7 @@ public static void waitForCompletion( /** * Wait for a single of future to complete, ignoring exceptions raised. * @param future future to wait for. + * @param type */ public static void waitForCompletionIgnoringExceptions( @Nullable final CompletableFuture future) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java index 2d49cc939a41a..205826dadae6f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java @@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; import org.apache.hadoop.fs.s3a.RemoteFileChangedException; +import org.apache.hadoop.fs.store.LogExactlyOnce; import static org.apache.hadoop.fs.s3a.Constants.*; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ContextAccessors.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ContextAccessors.java index 27ac7dec1dd19..42bee714da52f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ContextAccessors.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ContextAccessors.java @@ -22,10 +22,10 @@ import java.io.IOException; import java.nio.file.AccessDeniedException; -import com.amazonaws.services.s3.model.ObjectMetadata; - import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.store.audit.AuditSpan; /** * An interface to implement for providing accessors to @@ -85,13 +85,16 @@ public interface ContextAccessors { Path makeQualified(Path path); /** - * Retrieve the object metadata. - * - * @param key key to retrieve. - * @return metadata - * @throws IOException IO and object access problems. + * Return the active audit span. + * This is thread local -it MUST be picked up and passed into workers. + * Collect and cache the value during construction. + * @return active audit span. */ - @Retries.RetryTranslated - ObjectMetadata getObjectMetadata(String key) throws IOException; + AuditSpan getActiveAuditSpan(); + /** + * Get the request factory. + * @return the factory for requests. + */ + RequestFactory getRequestFactory(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java index 2292179b3fd66..5db996763885b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java @@ -46,6 +46,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.DurationInfo; +import static org.apache.hadoop.fs.store.audit.AuditingFunctions.callableWithinAuditSpan; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.maybeAwaitCompletion; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; @@ -504,13 +505,15 @@ private CompletableFuture submitDelete( return null; } filesDeleted += keyList.size(); - return submit(executor, () -> { - asyncDeleteAction(operationState, - keyList, - pathList, - LOG.isDebugEnabled()); - return null; - }); + return submit(executor, + callableWithinAuditSpan( + getAuditSpan(), () -> { + asyncDeleteAction(operationState, + keyList, + pathList, + LOG.isDebugEnabled()); + return null; + })); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ExecutingStoreOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ExecutingStoreOperation.java index ba8e743f293cb..fdaeb8ebba457 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ExecutingStoreOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ExecutingStoreOperation.java @@ -23,14 +23,21 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.apache.hadoop.fs.store.audit.AuditSpan; +import org.apache.hadoop.util.functional.CallableRaisingIOE; + /** * A subclass of {@link AbstractStoreOperation} which * provides a method {@link #execute()} that may be invoked * exactly once. + * It declares itself a {@code CallableRaisingIOE} and + * can be handed straight to methods which take those + * as parameters. * @param return type of executed operation. */ public abstract class ExecutingStoreOperation - extends AbstractStoreOperation { + extends AbstractStoreOperation + implements CallableRaisingIOE { /** * Used to stop any re-entrancy of the rename. @@ -39,11 +46,34 @@ public abstract class ExecutingStoreOperation private final AtomicBoolean executed = new AtomicBoolean(false); /** - * constructor. + * Constructor. + * Picks up the active audit span from the store context and + * stores it for later. * @param storeContext store context. */ protected ExecutingStoreOperation(final StoreContext storeContext) { - super(storeContext); + this(storeContext, storeContext.getActiveAuditSpan()); + } + + /** + * Constructor. + * @param storeContext store context. + * @param auditSpan active span + */ + protected ExecutingStoreOperation( + final StoreContext storeContext, + final AuditSpan auditSpan) { + super(storeContext, auditSpan); + } + + /** + * Apply calls {@link #execute()}. + * @return the result. + * @throws IOException IO problem + */ + @Override + public final T apply() throws IOException { + return execute(); } /** @@ -53,17 +83,19 @@ protected ExecutingStoreOperation(final StoreContext storeContext) { * @return the result. * @throws IOException IO problem */ - public abstract T execute() throws IOException ; + public abstract T execute() throws IOException; /** * Check that the operation has not been invoked twice. * This is an atomic check. + * After the check: activates the span. * @throws IllegalStateException on a second invocation. */ protected void executeOnlyOnce() { Preconditions.checkState( !executed.getAndSet(true), "Operation attempted twice"); + activateAuditSpan(); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java new file mode 100644 index 0000000000000..23631c67e30a1 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java @@ -0,0 +1,208 @@ +/* + * 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.hadoop.fs.s3a.impl; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; + +/** + * GetContentSummary operation. + * This is based on {@code FileSystem.get#getContentSummary}; + * its still doing sequential treewalk with the efficiency + * issues. + * + * Changes: + * 1. On the recursive calls there + * is no probe to see if the path is a file: we know the + * recursion only happens with a dir. + * 2. If a subdirectory is not found during the walk, that + * does not trigger an error. The directory is clearly + * not part of the content any more. + * + * The Operation serves up IOStatistics; this counts + * the cost of all the list operations, but not the + * initial HEAD probe to see if the path is a file. + */ +public class GetContentSummaryOperation extends + ExecutingStoreOperation implements IOStatisticsSource { + + private static final Logger LOG = LoggerFactory.getLogger( + GetContentSummaryOperation.class); + + /** + * Directory to scan. + */ + private final Path path; + + /** + * Callbacks to the store. + */ + private final GetContentSummaryCallbacks callbacks; + + /** + * IOStatistics to serve up. + */ + private final IOStatisticsSnapshot iostatistics = + new IOStatisticsSnapshot(); + + /** + * Constructor. + * @param storeContext context. + * @param path path to summarize + * @param callbacks callbacks for S3 access. + */ + public GetContentSummaryOperation( + final StoreContext storeContext, + final Path path, + final GetContentSummaryCallbacks callbacks) { + super(storeContext); + this.path = path; + this.callbacks = callbacks; + } + + @Override + public IOStatistics getIOStatistics() { + return iostatistics; + } + + /** + * Return the {@link ContentSummary} of a given path. + * @return the summary. + * @throws FileNotFoundException if the path does not resolve + * @throws IOException failure + */ + @Override + @Retries.RetryTranslated + public ContentSummary execute() throws IOException { + FileStatus status = probePathStatusOrNull(path, StatusProbeEnum.FILE); + if (status != null && status.isFile()) { + // f is a file + long length = status.getLen(); + return new ContentSummary.Builder().length(length). + fileCount(1).directoryCount(0).spaceConsumed(length).build(); + } + final ContentSummary summary = getDirSummary(path); + // Log the IOStatistics at debug so the cost of the operation + // can be made visible. + LOG.debug("IOStatistics of getContentSummary({}):\n{}", path, iostatistics); + return summary; + } + + /** + * Return the {@link ContentSummary} of a given directory. + * This is a recursive operation (as the original is); + * it'd be more efficient of stack and heap if it managed its + * own stack. + * @param dir dir to scan + * @throws FileNotFoundException if the path does not resolve + * @throws IOException IO failure + * @return the content summary + * @throws FileNotFoundException the path does not exist + * @throws IOException failure + */ + public ContentSummary getDirSummary(Path dir) throws IOException { + long totalLength = 0; + long fileCount = 0; + long dirCount = 1; + final RemoteIterator it + = callbacks.listStatusIterator(dir); + + while (it.hasNext()) { + final S3AFileStatus s = it.next(); + if (s.isDirectory()) { + try { + ContentSummary c = getDirSummary(s.getPath()); + totalLength += c.getLength(); + fileCount += c.getFileCount(); + dirCount += c.getDirectoryCount(); + } catch (FileNotFoundException ignored) { + // path was deleted during the scan; exclude from + // summary. + } + } else { + totalLength += s.getLen(); + fileCount += 1; + } + } + // Add the list's IOStatistics + iostatistics.aggregate(retrieveIOStatistics(it)); + return new ContentSummary.Builder().length(totalLength). + fileCount(fileCount).directoryCount(dirCount). + spaceConsumed(totalLength).build(); + } + + /** + * Get the status of a path, downgrading FNFE to null result. + * @param p path to probe. + * @param probes probes to exec + * @return the status or null + * @throws IOException failure other than FileNotFound + */ + private S3AFileStatus probePathStatusOrNull(final Path p, + final Set probes) throws IOException { + try { + return callbacks.probePathStatus(p, probes); + } catch (FileNotFoundException fnfe) { + return null; + } + } + + /** + * Callbacks used by the operation. + */ + public interface GetContentSummaryCallbacks { + + /** + * Get the status of a path. + * @param path path to probe. + * @param probes probes to exec + * @return the status + * @throws IOException failure + */ + @Retries.RetryTranslated + S3AFileStatus probePathStatus(Path path, + Set probes) throws IOException; + + /** + * Incremental list of all entries in a directory. + * @param path path of dir + * @return an iterator + * @throws IOException failure + */ + RemoteIterator listStatusIterator(Path path) + throws IOException; + + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java index 5efec2b36dafe..8c39aa4632d37 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java @@ -35,6 +35,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.Retries; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; @@ -188,6 +189,15 @@ public class HeaderProcessing extends AbstractStoreOperation { public static final String XA_STORAGE_CLASS = XA_HEADER_PREFIX + Headers.STORAGE_CLASS; + /** + * HTTP Referrer for logs: {@value}. + * This can be found in S3 logs, but is not set as + * an attribute in objects. + * important: the header value is deliberately + * a mis-spelling, as that is defined in RFC-1945. + */ + public static final String HEADER_REFERRER = "Referer"; + /** * Standard headers which are retrieved from HEAD Requests * and set as XAttrs if the response included the relevant header. @@ -230,12 +240,23 @@ public class HeaderProcessing extends AbstractStoreOperation { public static final String CONTENT_TYPE_APPLICATION_XML = "application/xml"; + /** + * Directory content type : {@value}. + * Matches use/expectations of AWS S3 console. + */ + public static final String CONTENT_TYPE_X_DIRECTORY = + "application/x-directory"; + + private final HeaderProcessingCallbacks callbacks; /** * Construct. * @param storeContext store context. + * @param callbacks callbacks to the store */ - public HeaderProcessing(final StoreContext storeContext) { + public HeaderProcessing(final StoreContext storeContext, + final HeaderProcessingCallbacks callbacks) { super(storeContext); + this.callbacks = callbacks; } /** @@ -253,18 +274,17 @@ private Map retrieveHeaders( final Path path, final Statistic statistic) throws IOException { StoreContext context = getStoreContext(); - ContextAccessors accessors = context.getContextAccessors(); - String objectKey = accessors.pathToKey(path); + String objectKey = context.pathToKey(path); ObjectMetadata md; String symbol = statistic.getSymbol(); S3AStatisticsContext instrumentation = context.getInstrumentation(); try { md = trackDuration(instrumentation, symbol, () -> - accessors.getObjectMetadata(objectKey)); + callbacks.getObjectMetadata(objectKey)); } catch (FileNotFoundException e) { // no entry. It could be a directory, so try again. md = trackDuration(instrumentation, symbol, () -> - accessors.getObjectMetadata(objectKey + "/")); + callbacks.getObjectMetadata(objectKey + "/")); } // all user metadata Map rawHeaders = md.getUserMetadata(); @@ -443,7 +463,7 @@ public static Optional extractXAttrLongValue(byte[] data) { * @param source the {@link ObjectMetadata} to copy * @param dest the metadata to update; this is the return value. */ - public void cloneObjectMetadata(ObjectMetadata source, + public static void cloneObjectMetadata(ObjectMetadata source, ObjectMetadata dest) { // Possibly null attributes @@ -497,4 +517,16 @@ public void cloneObjectMetadata(ObjectMetadata source, .forEach(e -> dest.addUserMetadata(e.getKey(), e.getValue())); } + public interface HeaderProcessingCallbacks { + + /** + * Retrieve the object metadata. + * + * @param key key to retrieve. + * @return metadata + * @throws IOException IO and object access problems. + */ + @Retries.RetryTranslated + ObjectMetadata getObjectMetadata(String key) throws IOException; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index a5ce1f68ad3fc..d6142f49c94a2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -37,6 +37,16 @@ */ public final class InternalConstants { + /** + * This declared delete as idempotent. + * This is an "interesting" topic in past Hadoop FS work. + * Essentially: with a single caller, DELETE is idempotent + * but in a shared filesystem, it is is very much not so. + * Here, on the basis that isn't a filesystem with consistency guarantees, + * retryable results in files being deleted. + */ + public static final boolean DELETE_CONSIDERED_IDEMPOTENT = true; + private InternalConstants() { } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java index b0ebf1ca16975..99fb6809d93a3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java @@ -30,6 +30,7 @@ import org.apache.hadoop.fs.s3a.S3ListResult; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.store.audit.AuditSpan; /** * These are all the callbacks which @@ -46,13 +47,15 @@ public interface ListingOperationCallbacks { * Retry policy: retry untranslated. * @param request request to initiate * @param trackerFactory tracker with statistics to update + * @param span audit span for this operation * @return the results * @throws IOException if the retry invocation raises one (it shouldn't). */ @Retries.RetryRaw CompletableFuture listObjectsAsync( - S3ListRequest request, - DurationTrackerFactory trackerFactory) + S3ListRequest request, + DurationTrackerFactory trackerFactory, + AuditSpan span) throws IOException; /** @@ -61,14 +64,16 @@ CompletableFuture listObjectsAsync( * @param request last list objects request to continue * @param prevResult last paged result to continue from * @param trackerFactory tracker with statistics to update + * @param span audit span for the IO * @return the next result object * @throws IOException none, just there for retryUntranslated. */ @Retries.RetryRaw CompletableFuture continueListObjectsAsync( - S3ListRequest request, - S3ListResult prevResult, - DurationTrackerFactory trackerFactory) + S3ListRequest request, + S3ListResult prevResult, + DurationTrackerFactory trackerFactory, + AuditSpan span) throws IOException; /** @@ -82,16 +87,19 @@ S3ALocatedFileStatus toLocatedFileStatus( throws IOException; /** * Create a {@code ListObjectsRequest} request against this bucket, - * with the maximum keys returned in a query set by + * with the maximum keys returned in a query set in the FS config. + * The active span for the FS is handed the request to prepare it + * before this method returns. * {@link #getMaxKeys()}. * @param key key for request * @param delimiter any delimiter + * @param span span within which the request takes place. * @return the request */ S3ListRequest createListObjectsRequest( - String key, - String delimiter); - + String key, + String delimiter, + AuditSpan span); /** * Return the number of bytes that large input files should be optimally diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java new file mode 100644 index 0000000000000..1d824201abeae --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java @@ -0,0 +1,184 @@ +/* + * 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.hadoop.fs.s3a.impl; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.file.AccessDeniedException; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.s3a.S3AFileStatus; + +/** + * The mkdir operation. + * A walk up the ancestor list halting as soon as a directory (good) + * or file (bad) is found. + * Optimized with the expectation that there is a marker up the path + * or (ultimately) a sibling of the path being created. + * It performs the directory listing probe ahead of the simple object HEAD + * call for this reason -the object is the failure mode which SHOULD NOT + * be encountered on normal execution. + */ +public class MkdirOperation extends ExecutingStoreOperation { + + private static final Logger LOG = LoggerFactory.getLogger( + MkdirOperation.class); + + private final Path dir; + + private final MkdirCallbacks callbacks; + + public MkdirOperation( + final StoreContext storeContext, + final Path dir, + final MkdirCallbacks callbacks) { + super(storeContext); + this.dir = dir; + this.callbacks = callbacks; + } + + /** + * + * Make the given path and all non-existent parents into + * directories. + * @return true if a directory was created or already existed + * @throws FileAlreadyExistsException there is a file at the path specified + * @throws IOException other IO problems + */ + @Override + @Retries.RetryTranslated + public Boolean execute() throws IOException { + LOG.debug("Making directory: {}", dir); + if (dir.isRoot()) { + // fast exit for root. + return true; + } + + FileStatus fileStatus = getPathStatusExpectingDir(dir); + if (fileStatus != null) { + if (fileStatus.isDirectory()) { + return true; + } else { + throw new FileAlreadyExistsException("Path is a file: " + dir); + } + } + // dir, walk up tree + // Walk path to root, ensuring closest ancestor is a directory, not file + Path fPart = dir.getParent(); + try { + while (fPart != null && !fPart.isRoot()) { + fileStatus = getPathStatusExpectingDir(fPart); + if (fileStatus == null) { + // nothing at this path, so validate the parent + fPart = fPart.getParent(); + continue; + } + if (fileStatus.isDirectory()) { + // the parent dir exists. All is good. + break; + } + + // there's a file at the parent entry + throw new FileAlreadyExistsException(String.format( + "Can't make directory for path '%s' since it is a file.", + fPart)); + } + } catch (AccessDeniedException e) { + LOG.info("mkdirs({}}: Access denied when looking" + + " for parent directory {}; skipping checks", + dir, fPart); + LOG.debug("{}", e.toString(), e); + } + + // if we get here there is no directory at the destination. + // so create one. + String key = getStoreContext().pathToKey(dir); + // this will create the marker file, delete the parent entries + // and update S3Guard + callbacks.createFakeDirectory(key); + return true; + } + + /** + * Get the status of a path, downgrading FNFE to null result. + * @param path path to probe. + * @param probes probes to exec + * @return the status or null + * @throws IOException failure other than FileNotFound + */ + private S3AFileStatus probePathStatusOrNull(final Path path, + final Set probes) throws IOException { + try { + return callbacks.probePathStatus(path, probes); + } catch (FileNotFoundException fnfe) { + return null; + } + } + + /** + * Get the status of a path -optimized for paths + * where there is a directory marker or child entries. + * @param path path to probe. + * @return the status + * @throws IOException failure + */ + private S3AFileStatus getPathStatusExpectingDir(final Path path) + throws IOException { + S3AFileStatus status = probePathStatusOrNull(path, + StatusProbeEnum.DIRECTORIES); + if (status == null) { + status = probePathStatusOrNull(path, + StatusProbeEnum.FILE); + } + return status; + } + + /** + * Callbacks used by mkdir. + */ + public interface MkdirCallbacks { + + /** + * Get the status of a path. + * @param path path to probe. + * @param probes probes to exec + * @return the status + * @throws IOException failure + */ + @Retries.RetryTranslated + S3AFileStatus probePathStatus(Path path, + Set probes) throws IOException; + + /** + * Create a fake directory, always ending in "/". + * Retry policy: retrying; translated. + * @param key name of directory object. + * @throws IOException IO failure + */ + @Retries.RetryTranslated + void createFakeDirectory(String key) throws IOException; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java index 5890ac01a9e74..7b13d0d3c7c42 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java @@ -46,6 +46,7 @@ import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.OperationDuration; +import static org.apache.hadoop.fs.store.audit.AuditingFunctions.callableWithinAuditSpan; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_BLOCK_SIZE; import static org.apache.hadoop.fs.s3a.S3AUtils.objectRepresentsDirectory; @@ -381,7 +382,7 @@ protected Path renameFileToDest() throws IOException { * Execute a full recursive rename. * There is a special handling of directly markers here -only leaf markers * are copied. This reduces incompatibility "regions" across versions. -Are * @throws IOException failure + * @throws IOException failure */ protected void recursiveDirectoryRename() throws IOException { final StoreContext storeContext = getStoreContext(); @@ -596,15 +597,16 @@ protected CompletableFuture initiateCopy( source.getVersionId(), source.getLen()); // queue the copy operation for execution in the thread pool - return submit(getStoreContext().getExecutor(), () -> - copySourceAndUpdateTracker( - childSourcePath, - key, - sourceAttributes, - callbacks.createReadContext(source), - childDestPath, - newDestKey, - true)); + return submit(getStoreContext().getExecutor(), + callableWithinAuditSpan(getAuditSpan(), () -> + copySourceAndUpdateTracker( + childSourcePath, + key, + sourceAttributes, + callbacks.createReadContext(source), + childDestPath, + newDestKey, + true))); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java new file mode 100644 index 0000000000000..f9ff08a5f6542 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java @@ -0,0 +1,695 @@ +/* + * 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.hadoop.fs.s3a.impl; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; +import com.amazonaws.services.s3.model.CannedAccessControlList; +import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; +import com.amazonaws.services.s3.model.CopyObjectRequest; +import com.amazonaws.services.s3.model.DeleteObjectRequest; +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; +import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; +import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest; +import com.amazonaws.services.s3.model.ListObjectsRequest; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ObjectListing; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.PartETag; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; +import com.amazonaws.services.s3.model.SSECustomerKey; +import com.amazonaws.services.s3.model.SelectObjectContentRequest; +import com.amazonaws.services.s3.model.UploadPartRequest; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecretOperations; +import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; + +import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT; +import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument; +import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; + +/** + * The standard implementation of the request factory. + * This creates AWS SDK request classes for the specific bucket, + * with standard options/headers set. + * It is also where custom setting parameters can take place. + * + * All creation of AWS S3 requests MUST be through this class so that + * common options (encryption etc.) can be added here, + * and so that any chained transformation of requests can be applied. + * + * This is where audit span information is added to the requests, + * until it is done in the AWS SDK itself. + * + * All created requests will be passed through + * {@link PrepareRequest#prepareRequest(AmazonWebServiceRequest)} before + * being returned to the caller. + */ +public class RequestFactoryImpl implements RequestFactory { + + public static final Logger LOG = LoggerFactory.getLogger( + RequestFactoryImpl.class); + + /** + * Target bucket. + */ + private final String bucket; + + /** + * Encryption secrets. + */ + private EncryptionSecrets encryptionSecrets; + + /** + * ACL For new objects. + */ + private final CannedAccessControlList cannedACL; + + /** + * Max number of multipart entries allowed in a large + * upload. Tunable for testing only. + */ + private final long multipartPartCountLimit; + + /** + * Requester Pays. + * This is to be wired up in a PR with its + * own tests and docs. + */ + private final boolean requesterPays; + + /** + * Callback to prepare requests. + */ + private final PrepareRequest requestPreparer; + + /** + * Constructor. + * @param builder builder with all the configuration. + */ + protected RequestFactoryImpl( + final RequestFactoryBuilder builder) { + this.bucket = builder.bucket; + this.cannedACL = builder.cannedACL; + this.encryptionSecrets = builder.encryptionSecrets; + this.multipartPartCountLimit = builder.multipartPartCountLimit; + this.requesterPays = builder.requesterPays; + this.requestPreparer = builder.requestPreparer; + } + + /** + * Preflight preparation of AWS request. + * @param web service request + * @return prepared entry. + */ + @Retries.OnceRaw + private T prepareRequest(T t) { + return requestPreparer != null + ? requestPreparer.prepareRequest(t) + : t; + } + + /** + * Get the canned ACL of this FS. + * @return an ACL, if any + */ + @Override + public CannedAccessControlList getCannedACL() { + return cannedACL; + } + + /** + * Get the target bucket. + * @return the bucket. + */ + protected String getBucket() { + return bucket; + } + + /** + * Create the AWS SDK structure used to configure SSE, + * if the encryption secrets contain the information/settings for this. + * @return an optional set of KMS Key settings + */ + @Override + public Optional generateSSEAwsKeyParams() { + return EncryptionSecretOperations.createSSEAwsKeyManagementParams( + encryptionSecrets); + } + + /** + * Create the SSE-C structure for the AWS SDK, if the encryption secrets + * contain the information/settings for this. + * This will contain a secret extracted from the bucket/configuration. + * @return an optional customer key. + */ + @Override + public Optional generateSSECustomerKey() { + return EncryptionSecretOperations.createSSECustomerKey( + encryptionSecrets); + } + + /** + * Get the encryption algorithm of this endpoint. + * @return the encryption algorithm. + */ + @Override + public S3AEncryptionMethods getServerSideEncryptionAlgorithm() { + return encryptionSecrets.getEncryptionMethod(); + } + + /** + * Sets server side encryption parameters to the part upload + * request when encryption is enabled. + * @param request upload part request + */ + protected void setOptionalUploadPartRequestParameters( + UploadPartRequest request) { + generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + } + + /** + * Sets server side encryption parameters to the GET reuquest. + * request when encryption is enabled. + * @param request upload part request + */ + protected void setOptionalGetObjectMetadataParameters( + GetObjectMetadataRequest request) { + generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + } + + /** + * Set the optional parameters when initiating the request (encryption, + * headers, storage, etc). + * @param request request to patch. + */ + protected void setOptionalMultipartUploadRequestParameters( + InitiateMultipartUploadRequest request) { + generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams); + generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + } + + /** + * Set the optional parameters for a PUT request. + * @param request request to patch. + */ + protected void setOptionalPutRequestParameters(PutObjectRequest request) { + generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams); + generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + } + + /** + * Set the optional metadata for an object being created or copied. + * @param metadata to update. + */ + protected void setOptionalObjectMetadata(ObjectMetadata metadata) { + final S3AEncryptionMethods algorithm + = getServerSideEncryptionAlgorithm(); + if (S3AEncryptionMethods.SSE_S3 == algorithm) { + metadata.setSSEAlgorithm(algorithm.getMethod()); + } + } + + /** + * Create a new object metadata instance. + * Any standard metadata headers are added here, for example: + * encryption. + * + * @param length length of data to set in header; Ignored if negative + * @return a new metadata instance + */ + @Override + public ObjectMetadata newObjectMetadata(long length) { + final ObjectMetadata om = new ObjectMetadata(); + setOptionalObjectMetadata(om); + if (length >= 0) { + om.setContentLength(length); + } + return om; + } + + @Override + public CopyObjectRequest newCopyObjectRequest(String srcKey, + String dstKey, + ObjectMetadata srcom) { + CopyObjectRequest copyObjectRequest = + new CopyObjectRequest(getBucket(), srcKey, getBucket(), dstKey); + ObjectMetadata dstom = newObjectMetadata(srcom.getContentLength()); + HeaderProcessing.cloneObjectMetadata(srcom, dstom); + setOptionalObjectMetadata(dstom); + copyEncryptionParameters(srcom, copyObjectRequest); + copyObjectRequest.setCannedAccessControlList(cannedACL); + copyObjectRequest.setNewObjectMetadata(dstom); + Optional.ofNullable(srcom.getStorageClass()) + .ifPresent(copyObjectRequest::setStorageClass); + return prepareRequest(copyObjectRequest); + } + + /** + * Propagate encryption parameters from source file if set else use the + * current filesystem encryption settings. + * @param srcom source object metadata. + * @param copyObjectRequest copy object request body. + */ + protected void copyEncryptionParameters( + ObjectMetadata srcom, + CopyObjectRequest copyObjectRequest) { + String sourceKMSId = srcom.getSSEAwsKmsKeyId(); + if (isNotEmpty(sourceKMSId)) { + // source KMS ID is propagated + LOG.debug("Propagating SSE-KMS settings from source {}", + sourceKMSId); + copyObjectRequest.setSSEAwsKeyManagementParams( + new SSEAwsKeyManagementParams(sourceKMSId)); + } + switch (getServerSideEncryptionAlgorithm()) { + case SSE_S3: + /* no-op; this is set in destination object metadata */ + break; + + case SSE_C: + generateSSECustomerKey().ifPresent(customerKey -> { + copyObjectRequest.setSourceSSECustomerKey(customerKey); + copyObjectRequest.setDestinationSSECustomerKey(customerKey); + }); + break; + + case SSE_KMS: + generateSSEAwsKeyParams().ifPresent( + copyObjectRequest::setSSEAwsKeyManagementParams); + break; + default: + } + } + /** + * Create a putObject request. + * Adds the ACL and metadata + * @param key key of object + * @param metadata metadata header + * @param srcfile source file + * @return the request + */ + @Override + public PutObjectRequest newPutObjectRequest(String key, + ObjectMetadata metadata, File srcfile) { + Preconditions.checkNotNull(srcfile); + PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key, + srcfile); + setOptionalPutRequestParameters(putObjectRequest); + putObjectRequest.setCannedAcl(cannedACL); + putObjectRequest.setMetadata(metadata); + return prepareRequest(putObjectRequest); + } + + /** + * Create a {@link PutObjectRequest} request. + * The metadata is assumed to have been configured with the size of the + * operation. + * @param key key of object + * @param metadata metadata header + * @param inputStream source data. + * @return the request + */ + @Override + public PutObjectRequest newPutObjectRequest(String key, + ObjectMetadata metadata, + InputStream inputStream) { + Preconditions.checkNotNull(inputStream); + Preconditions.checkArgument(isNotEmpty(key), "Null/empty key"); + PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key, + inputStream, metadata); + setOptionalPutRequestParameters(putObjectRequest); + putObjectRequest.setCannedAcl(cannedACL); + return prepareRequest(putObjectRequest); + } + + @Override + public PutObjectRequest newDirectoryMarkerRequest(String directory) { + String key = directory.endsWith("/") + ? directory + : (directory + "/"); + // an input stream which is laways empty + final InputStream im = new InputStream() { + @Override + public int read() throws IOException { + return -1; + } + }; + // preparation happens in here + final ObjectMetadata md = newObjectMetadata(0L); + md.setContentType(HeaderProcessing.CONTENT_TYPE_X_DIRECTORY); + PutObjectRequest putObjectRequest = + newPutObjectRequest(key, md, im); + return putObjectRequest; + } + + @Override + public ListMultipartUploadsRequest + newListMultipartUploadsRequest(String prefix) { + ListMultipartUploadsRequest request = new ListMultipartUploadsRequest( + getBucket()); + if (prefix != null) { + request.setPrefix(prefix); + } + return prepareRequest(request); + } + + @Override + public AbortMultipartUploadRequest newAbortMultipartUploadRequest( + String destKey, + String uploadId) { + return prepareRequest(new AbortMultipartUploadRequest(getBucket(), + destKey, + uploadId)); + } + + @Override + public InitiateMultipartUploadRequest newMultipartUploadRequest( + String destKey) { + final InitiateMultipartUploadRequest initiateMPURequest = + new InitiateMultipartUploadRequest(getBucket(), + destKey, + newObjectMetadata(-1)); + initiateMPURequest.setCannedACL(getCannedACL()); + setOptionalMultipartUploadRequestParameters(initiateMPURequest); + return prepareRequest(initiateMPURequest); + } + + @Override + public CompleteMultipartUploadRequest newCompleteMultipartUploadRequest( + String destKey, + String uploadId, + List partETags) { + // a copy of the list is required, so that the AWS SDK doesn't + // attempt to sort an unmodifiable list. + return prepareRequest(new CompleteMultipartUploadRequest(bucket, + destKey, uploadId, new ArrayList<>(partETags))); + + } + + @Override + public GetObjectMetadataRequest newGetObjectMetadataRequest(String key) { + GetObjectMetadataRequest request = + new GetObjectMetadataRequest(getBucket(), key); + //SSE-C requires to be filled in if enabled for object metadata + setOptionalGetObjectMetadataParameters(request); + return prepareRequest(request); + } + + @Override + public GetObjectRequest newGetObjectRequest(String key) { + GetObjectRequest request = new GetObjectRequest(bucket, key); + generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + + return prepareRequest(request); + } + + @Override + public UploadPartRequest newUploadPartRequest( + String destKey, + String uploadId, + int partNumber, + int size, + InputStream uploadStream, + File sourceFile, + long offset) throws PathIOException { + checkNotNull(uploadId); + // exactly one source must be set; xor verifies this + checkArgument((uploadStream != null) ^ (sourceFile != null), + "Data source"); + checkArgument(size >= 0, "Invalid partition size %s", size); + checkArgument(partNumber > 0, + "partNumber must be between 1 and %s inclusive, but is %s", + DEFAULT_UPLOAD_PART_COUNT_LIMIT, partNumber); + + LOG.debug("Creating part upload request for {} #{} size {}", + uploadId, partNumber, size); + final String pathErrorMsg = "Number of parts in multipart upload exceeded." + + " Current part count = %s, Part count limit = %s "; + if (partNumber > multipartPartCountLimit) { + throw new PathIOException(destKey, + String.format(pathErrorMsg, partNumber, multipartPartCountLimit)); + } + UploadPartRequest request = new UploadPartRequest() + .withBucketName(getBucket()) + .withKey(destKey) + .withUploadId(uploadId) + .withPartNumber(partNumber) + .withPartSize(size); + if (uploadStream != null) { + // there's an upload stream. Bind to it. + request.setInputStream(uploadStream); + } else { + checkArgument(sourceFile.exists(), + "Source file does not exist: %s", sourceFile); + checkArgument(sourceFile.isFile(), + "Source is not a file: %s", sourceFile); + checkArgument(offset >= 0, "Invalid offset %s", offset); + long length = sourceFile.length(); + checkArgument(offset == 0 || offset < length, + "Offset %s beyond length of file %s", offset, length); + request.setFile(sourceFile); + request.setFileOffset(offset); + } + setOptionalUploadPartRequestParameters(request); + return prepareRequest(request); + } + + @Override + public SelectObjectContentRequest newSelectRequest(String key) { + SelectObjectContentRequest request = new SelectObjectContentRequest(); + request.setBucketName(bucket); + request.setKey(key); + generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + return prepareRequest(request); + } + + @Override + public ListObjectsRequest newListObjectsV1Request( + final String key, + final String delimiter, + final int maxKeys) { + ListObjectsRequest request = new ListObjectsRequest() + .withBucketName(bucket) + .withMaxKeys(maxKeys) + .withPrefix(key); + if (delimiter != null) { + request.setDelimiter(delimiter); + } + return prepareRequest(request); + } + + @Override + public ListNextBatchOfObjectsRequest newListNextBatchOfObjectsRequest( + ObjectListing prev) { + return prepareRequest(new ListNextBatchOfObjectsRequest(prev)); + } + + @Override + public ListObjectsV2Request newListObjectsV2Request( + final String key, + final String delimiter, + final int maxKeys) { + final ListObjectsV2Request request = new ListObjectsV2Request() + .withBucketName(bucket) + .withMaxKeys(maxKeys) + .withPrefix(key); + if (delimiter != null) { + request.setDelimiter(delimiter); + } + return prepareRequest(request); + } + + @Override + public DeleteObjectRequest newDeleteObjectRequest(String key) { + return prepareRequest(new DeleteObjectRequest(bucket, key)); + } + + @Override + public DeleteObjectsRequest newBulkDeleteRequest( + List keysToDelete, + boolean quiet) { + return prepareRequest( + new DeleteObjectsRequest(bucket) + .withKeys(keysToDelete) + .withQuiet(quiet)); + } + + @Override + public void setEncryptionSecrets(final EncryptionSecrets secrets) { + encryptionSecrets = secrets; + } + + /** + * Create a builder. + * @return new builder. + */ + public static RequestFactoryBuilder builder() { + return new RequestFactoryBuilder(); + } + + /** + * Builder. + */ + public static final class RequestFactoryBuilder { + + /** + * Target bucket. + */ + private String bucket; + + /** + * Encryption secrets. + */ + private EncryptionSecrets encryptionSecrets = new EncryptionSecrets(); + + /** + * ACL For new objects. + */ + private CannedAccessControlList cannedACL = null; + + /** Requester Pays flag. */ + private boolean requesterPays = false; + + /** + * Multipart limit. + */ + private long multipartPartCountLimit = DEFAULT_UPLOAD_PART_COUNT_LIMIT; + + /** + * Callback to prepare requests. + */ + private PrepareRequest requestPreparer; + + private RequestFactoryBuilder() { + } + + /** + * Build the request factory. + * @return the factory + */ + public RequestFactory build() { + return new RequestFactoryImpl(this); + } + + /** + * Target bucket. + * @param value new value + * @return the builder + */ + public RequestFactoryBuilder withBucket(final String value) { + bucket = value; + return this; + } + + /** + * Encryption secrets. + * @param value new value + * @return the builder + */ + public RequestFactoryBuilder withEncryptionSecrets( + final EncryptionSecrets value) { + encryptionSecrets = value; + return this; + } + + /** + * ACL For new objects. + * @param value new value + * @return the builder + */ + public RequestFactoryBuilder withCannedACL( + final CannedAccessControlList value) { + cannedACL = value; + return this; + } + + /** + * Requester Pays flag. + * @param value new value + * @return the builder + */ + public RequestFactoryBuilder withRequesterPays( + final boolean value) { + requesterPays = value; + return this; + } + + /** + * Multipart limit. + * @param value new value + * @return the builder + */ + public RequestFactoryBuilder withMultipartPartCountLimit( + final long value) { + multipartPartCountLimit = value; + return this; + } + + /** + * Callback to prepare requests. + * + * @param value new value + * @return the builder + */ + public RequestFactoryBuilder withRequestPreparer( + final PrepareRequest value) { + this.requestPreparer = value; + return this; + } + } + + /** + * This is a callback for anything to "prepare" every request + * after creation. The S3AFileSystem's Audit Manager is expected + * to be wired up via this call so can audit/prepare requests + * after their creation. + */ + @FunctionalInterface + public interface PrepareRequest { + + /** + * Post-creation preparation of AWS request. + * @param t request + * @param request type. + * @return prepared entry. + */ + @Retries.OnceRaw + T prepareRequest(T t); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java index 88231d8af9c04..ac29780dcbaec 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java @@ -32,6 +32,8 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AInputPolicy; @@ -40,6 +42,9 @@ import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; +import org.apache.hadoop.fs.store.audit.ActiveThreadSpanSource; +import org.apache.hadoop.fs.store.audit.AuditSpan; +import org.apache.hadoop.fs.store.audit.AuditSpanSource; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.LambdaUtils; import org.apache.hadoop.util.SemaphoredDelegatingExecutor; @@ -59,7 +64,7 @@ */ @InterfaceAudience.LimitedPrivate("S3A Filesystem and extensions") @InterfaceStability.Unstable -public class StoreContext { +public class StoreContext implements ActiveThreadSpanSource { /** Filesystem URI. */ private final URI fsURI; @@ -117,13 +122,17 @@ public class StoreContext { /** * Source of time. */ - private ITtlTimeProvider timeProvider; + + /** Time source for S3Guard TTLs. */ + private final ITtlTimeProvider timeProvider; + + /** Operation Auditor. */ + private final AuditSpanSource auditor; /** * Instantiate. - * @deprecated as public method: use {@link StoreContextBuilder}. */ - public StoreContext( + StoreContext( final URI fsURI, final String bucket, final Configuration configuration, @@ -140,13 +149,17 @@ public StoreContext( final MetadataStore metadataStore, final boolean useListV1, final ContextAccessors contextAccessors, - final ITtlTimeProvider timeProvider) { + final ITtlTimeProvider timeProvider, + final AuditSpanSource auditor) { this.fsURI = fsURI; this.bucket = bucket; this.configuration = configuration; this.username = username; this.owner = owner; - this.executor = MoreExecutors.listeningDecorator(executor); + // some mock tests have a null executor pool + this.executor = executor !=null + ? MoreExecutors.listeningDecorator(executor) + : null; this.executorCapacity = executorCapacity; this.invoker = invoker; this.instrumentation = instrumentation; @@ -158,11 +171,7 @@ public StoreContext( this.useListV1 = useListV1; this.contextAccessors = contextAccessors; this.timeProvider = timeProvider; - } - - @Override - protected Object clone() throws CloneNotSupportedException { - return super.clone(); + this.auditor = auditor; } public URI getFsURI() { @@ -391,4 +400,33 @@ public CompletableFuture submit( LambdaUtils.eval(future, call)); return future; } + + /** + * Get the auditor. + * @return auditor. + */ + public AuditSpanSource getAuditor() { + return auditor; + } + + /** + * Return the active audit span. + * This is thread local -it MUST be passed into workers. + * To ensure the correct span is used, it SHOULD be + * collected as early as possible, ideally during construction/ + * or service init/start. + * @return active audit span. + */ + @Override + public AuditSpan getActiveAuditSpan() { + return contextAccessors.getActiveAuditSpan(); + } + + /** + * Get the request factory. + * @return the factory for requests. + */ + public RequestFactory getRequestFactory() { + return contextAccessors.getRequestFactory(); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java index 13953f9c985f3..468af1bb77c8b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java @@ -25,9 +25,11 @@ import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AInputPolicy; import org.apache.hadoop.fs.s3a.S3AStorageStatistics; +import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; +import org.apache.hadoop.fs.store.audit.AuditSpanSource; import org.apache.hadoop.security.UserGroupInformation; /** @@ -69,6 +71,8 @@ public class StoreContextBuilder { private ITtlTimeProvider timeProvider; + private AuditSpanSource auditor; + public StoreContextBuilder setFsURI(final URI fsURI) { this.fsURI = fsURI; return this; @@ -165,6 +169,17 @@ public StoreContextBuilder setTimeProvider( return this; } + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public StoreContextBuilder setAuditor( + final AuditSpanSource value) { + auditor = value; + return this; + } + @SuppressWarnings("deprecation") public StoreContext build() { return new StoreContext(fsURI, @@ -183,6 +198,7 @@ public StoreContext build() { metadataStore, useListV1, contextAccessors, - timeProvider); + timeProvider, + auditor); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java index 20bd250da0c38..2a7cb4c1b7247 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java @@ -52,6 +52,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; import org.apache.hadoop.fs.s3a.S3ListRequest; +import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.service.Service; import org.apache.hadoop.service.launcher.LauncherExitCodes; import org.apache.hadoop.service.launcher.ServiceLaunchException; @@ -347,21 +348,26 @@ protected long listStatusFilesystem( protected long dumpRawS3ObjectStore( final CsvFile csv) throws IOException { S3AFileSystem fs = getFilesystem(); - Path rootPath = fs.qualify(new Path("/")); - Listing listing = fs.getListing(); - S3ListRequest request = listing.createListObjectsRequest("", null); long count = 0; - RemoteIterator st = - listing.createFileStatusListingIterator(rootPath, request, - ACCEPT_ALL, - new Listing.AcceptAllButSelfAndS3nDirs(rootPath)); - while (st.hasNext()) { - count++; - S3AFileStatus next = st.next(); - LOG.debug("[{}] {}", count, next); - csv.entry(next); + Path rootPath = fs.qualify(new Path("/")); + try (AuditSpan span = fs.createSpan("DumpS3GuardDynamoTable", + rootPath.toString(), null)) { + Listing listing = fs.getListing(); + S3ListRequest request = listing.createListObjectsRequest("", null, span); + count = 0; + RemoteIterator st = + listing.createFileStatusListingIterator(rootPath, request, + ACCEPT_ALL, + new Listing.AcceptAllButSelfAndS3nDirs(rootPath), + span); + while (st.hasNext()) { + count++; + S3AFileStatus next = st.next(); + LOG.debug("[{}] {}", count, next); + csv.entry(next); + } + LOG.info("entry count: {}", count); } - LOG.info("entry count: {}", count); return count; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index b963e7e2532e5..e4542eb1bcfa6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -63,6 +63,8 @@ import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription; import com.amazonaws.services.dynamodbv2.model.TableDescription; import com.amazonaws.services.dynamodbv2.model.WriteRequest; + +import org.apache.hadoop.fs.s3a.impl.InternalConstants; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; @@ -593,7 +595,7 @@ private void innerDelete(final Path path, } // the policy on whether repeating delete operations is based // on that of S3A itself - boolean idempotent = S3AFileSystem.DELETE_CONSIDERED_IDEMPOTENT; + boolean idempotent = InternalConstants.DELETE_CONSIDERED_IDEMPOTENT; if (tombstone) { Preconditions.checkArgument(ttlTimeProvider != null, "ttlTimeProvider " + "must not be null"); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java index 2acae76875a22..ef4756435533a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java @@ -30,7 +30,6 @@ import java.util.Date; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Locale; import java.util.Map; @@ -55,11 +54,11 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FilterFileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.StorageStatistics; import org.apache.hadoop.fs.s3a.MultipartUtils; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AUtils; +import org.apache.hadoop.fs.s3a.WriteOperationHelper; import org.apache.hadoop.fs.s3a.auth.RolePolicies; import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens; import org.apache.hadoop.fs.s3a.commit.CommitConstants; @@ -69,6 +68,8 @@ import org.apache.hadoop.fs.s3a.select.SelectTool; import org.apache.hadoop.fs.s3a.tools.MarkerTool; import org.apache.hadoop.fs.shell.CommandFormat; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.ExitCodeProvider; @@ -84,6 +85,9 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterConstants.FILESYSTEM_TEMP_PATH; import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStoreTableManager.SSE_DEFAULT_MASTER_KEY; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.MULTIPART_UPLOAD_ABORTED; import static org.apache.hadoop.service.launcher.LauncherExitCodes.*; /** @@ -531,16 +535,13 @@ protected void dumpFileSystemStatistics(PrintStream stream) { if (fs == null) { return; } - println(stream, "%nStorage Statistics for %s%n", fs.getUri()); - StorageStatistics st = fs.getStorageStatistics(); - Iterator it - = st.getLongStatistics(); - while (it.hasNext()) { - StorageStatistics.LongStatistic next = it.next(); - long value = next.getValue(); - if (value != 0) { - println(stream, "%s\t%s", next.getName(), value); - } + println(stream, "%nIO Statistics for %s%n", fs.getUri()); + final IOStatistics iostats = retrieveIOStatistics(fs); + if (iostats != null) { + println(stream, ioStatisticsToPrettyString(iostats)); + + } else { + println(stream, "FileSystem does not provide IOStatistics"); } println(stream, ""); } @@ -890,15 +891,18 @@ public int run(String[] args, PrintStream out) throws Exception { final CommandFormat commandFormat = getCommandFormat(); + final boolean verbose = commandFormat.getOpt(VERBOSE); final ImportOperation importer = new ImportOperation( getFilesystem(), getStore(), status, commandFormat.getOpt(AUTH_FLAG), - commandFormat.getOpt(VERBOSE)); + verbose); long items = importer.execute(); println(out, "Inserted %d items into Metadata Store", items); - + if (verbose) { + dumpFileSystemStatistics(out); + } return SUCCESS; } @@ -1584,9 +1588,12 @@ public int run(String[] args, PrintStream out) throw invalidArgs("No options specified"); } processArgs(paths, out); + println(out, "Listing uploads under path \"%s\"", prefix); promptBeforeAbort(out); processUploads(out); - + if (verbose) { + dumpFileSystemStatistics(out); + } out.flush(); return SUCCESS; } @@ -1605,8 +1612,15 @@ private void promptBeforeAbort(PrintStream out) throws IOException { } private void processUploads(PrintStream out) throws IOException { - MultipartUtils.UploadIterator uploads; - uploads = getFilesystem().listUploads(prefix); + final S3AFileSystem fs = getFilesystem(); + MultipartUtils.UploadIterator uploads = fs.listUploads(prefix); + // create a span so that the write operation helper + // is within one + AuditSpan span = + fs.createSpan(MULTIPART_UPLOAD_ABORTED, + prefix, null); + final WriteOperationHelper writeOperationHelper + = fs.getWriteOperationHelper(); int count = 0; while (uploads.hasNext()) { @@ -1620,18 +1634,20 @@ private void processUploads(PrintStream out) throws IOException { upload.getKey(), upload.getUploadId()); } if (mode == Mode.ABORT) { - getFilesystem().getWriteOperationHelper() + writeOperationHelper .abortMultipartUpload(upload.getKey(), upload.getUploadId(), true, LOG_EVENT); } } + span.deactivate(); if (mode != Mode.EXPECT || verbose) { println(out, "%s %d uploads %s.", TOTAL, count, mode == Mode.ABORT ? "deleted" : "found"); } if (mode == Mode.EXPECT) { if (count != expectedCount) { - throw badState("Expected %d uploads, found %d", expectedCount, count); + throw badState("Expected upload count under %s: %d, found %d", + prefix, expectedCount, count); } } } @@ -1643,6 +1659,9 @@ private void processUploads(PrintStream out) throws IOException { * @return true iff u was created at least age milliseconds ago. */ private boolean olderThan(MultipartUpload u, long msec) { + if (msec == 0) { + return true; + } Date ageDate = new Date(System.currentTimeMillis() - msec); return ageDate.compareTo(u.getInitiated()) >= 0; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java index 20dc00fbc06a1..150043aea9f1a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.Locale; -import java.util.Optional; import com.amazonaws.services.s3.model.CSVInput; import com.amazonaws.services.s3.model.CSVOutput; @@ -28,7 +27,6 @@ import com.amazonaws.services.s3.model.InputSerialization; import com.amazonaws.services.s3.model.OutputSerialization; import com.amazonaws.services.s3.model.QuoteFields; -import com.amazonaws.services.s3.model.SSECustomerKey; import com.amazonaws.services.s3.model.SelectObjectContentRequest; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; @@ -55,6 +53,7 @@ * This class is intended to be instantiated by the owning S3AFileSystem * instance to handle the construction of requests: IO is still done exclusively * in the filesystem. + * */ public class SelectBinding { @@ -70,12 +69,12 @@ public class SelectBinding { /** * Constructor. - * @param operations owning FS. + * @param operations callback to owner FS, with associated span. */ public SelectBinding(final WriteOperationHelper operations) { this.operations = checkNotNull(operations); Configuration conf = getConf(); - this.enabled = conf.getBoolean(FS_S3A_SELECT_ENABLED, true); + this.enabled = isSelectEnabled(conf); this.errorsIncludeSql = conf.getBoolean(SELECT_ERRORS_INCLUDE_SQL, false); } @@ -91,12 +90,20 @@ public boolean isEnabled() { return enabled; } + /** + * Static probe for select being enabled. + * @param conf configuration + * @return true iff select is enabled. + */ + public static boolean isSelectEnabled(Configuration conf) { + return conf.getBoolean(FS_S3A_SELECT_ENABLED, true); + } + /** * Build and execute a select request. * @param readContext the read context, which includes the source path. * @param expression the SQL expression. * @param builderOptions query options - * @param sseKey optional SSE customer key * @param objectAttributes object attributes from a HEAD request * @return an FSDataInputStream whose wrapped stream is a SelectInputStream * @throws IllegalArgumentException argument failure @@ -108,7 +115,6 @@ public FSDataInputStream select( final S3AReadOpContext readContext, final String expression, final Configuration builderOptions, - final Optional sseKey, final S3ObjectAttributes objectAttributes) throws IOException { return new FSDataInputStream( @@ -118,8 +124,8 @@ public FSDataInputStream select( buildSelectRequest( readContext.getPath(), expression, - builderOptions, - sseKey))); + builderOptions + ))); } /** @@ -127,7 +133,6 @@ public FSDataInputStream select( * @param path source path. * @param expression the SQL expression. * @param builderOptions config to extract other query options from - * @param sseKey optional SSE customer key * @return the request to serve * @throws IllegalArgumentException argument failure * @throws IOException problem building/validating the request @@ -135,16 +140,13 @@ public FSDataInputStream select( public SelectObjectContentRequest buildSelectRequest( final Path path, final String expression, - final Configuration builderOptions, - final Optional sseKey) + final Configuration builderOptions) throws IOException { Preconditions.checkState(isEnabled(), "S3 Select is not enabled for %s", path); SelectObjectContentRequest request = operations.newSelectRequest(path); buildRequest(request, expression, builderOptions); - // optionally set an SSE key in the input - sseKey.ifPresent(request::withSSECustomerKey); return request; } @@ -428,4 +430,5 @@ static String expandBackslashChars(String src) { .replace("\\\\", "\\"); } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/ForwardingIOStatisticsStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/ForwardingIOStatisticsStore.java new file mode 100644 index 0000000000000..612460835403e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/ForwardingIOStatisticsStore.java @@ -0,0 +1,186 @@ +/* + * 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.hadoop.fs.s3a.statistics.impl; + +import javax.annotation.Nullable; +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.MeanStatistic; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; + +/** + * This may seem odd having an IOStatisticsStore which does nothing + * but forward to a wrapped store, but it's designed to + * assist in subclassing of selective methods, such + * as those to increment counters, get durations etc. + */ +public class ForwardingIOStatisticsStore implements IOStatisticsStore { + + private final IOStatisticsStore innerStatistics; + + public ForwardingIOStatisticsStore( + final IOStatisticsStore innerStatistics) { + this.innerStatistics = innerStatistics; + } + + protected IOStatisticsStore getInnerStatistics() { + return innerStatistics; + } + + @Override + public Map counters() { + return getInnerStatistics().counters(); + } + + @Override + public Map gauges() { + return getInnerStatistics().gauges(); + } + + @Override + public Map minimums() { + return getInnerStatistics().minimums(); + } + + @Override + public Map maximums() { + return getInnerStatistics().maximums(); + } + + @Override + public Map meanStatistics() { + return getInnerStatistics().meanStatistics(); + } + + @Override + public boolean aggregate(@Nullable final IOStatistics statistics) { + return getInnerStatistics().aggregate(statistics); + } + + @Override + public long incrementCounter(final String key, final long value) { + return getInnerStatistics().incrementCounter(key, value); + } + + @Override + public void setCounter(final String key, final long value) { + getInnerStatistics().setCounter(key, value); + } + + @Override + public void setGauge(final String key, final long value) { + getInnerStatistics().setGauge(key, value); + } + + @Override + public long incrementGauge(final String key, final long value) { + return getInnerStatistics().incrementGauge(key, value); + } + + @Override + public void setMaximum(final String key, final long value) { + getInnerStatistics().setMaximum(key, value); + } + + @Override + public long incrementMaximum(final String key, final long value) { + return getInnerStatistics().incrementMaximum(key, value); + } + + @Override + public void setMinimum(final String key, final long value) { + getInnerStatistics().setMinimum(key, value); + + } + + @Override + public long incrementMinimum(final String key, final long value) { + return getInnerStatistics().incrementMinimum(key, value); + + } + + @Override + public void addMinimumSample(final String key, final long value) { + getInnerStatistics().addMinimumSample(key, value); + + } + + @Override + public void addMaximumSample(final String key, final long value) { + getInnerStatistics().addMaximumSample(key, value); + } + + @Override + public void setMeanStatistic(final String key, final MeanStatistic value) { + getInnerStatistics().setMeanStatistic(key, value); + + } + + @Override + public void addMeanStatisticSample(final String key, final long value) { + getInnerStatistics().addMeanStatisticSample(key, value); + + } + + @Override + public void reset() { + getInnerStatistics().reset(); + } + + @Override + public AtomicLong getCounterReference(final String key) { + return getInnerStatistics().getCounterReference(key); + } + + @Override + public AtomicLong getMaximumReference(final String key) { + return getInnerStatistics().getMaximumReference(key); + } + + @Override + public AtomicLong getMinimumReference(final String key) { + return getInnerStatistics().getMinimumReference(key); + } + + @Override + public AtomicLong getGaugeReference(final String key) { + return getInnerStatistics().getGaugeReference(key); + } + + @Override + public MeanStatistic getMeanStatistic(final String key) { + return getInnerStatistics().getMeanStatistic(key); + } + + @Override + public void addTimedOperation(final String prefix, + final long durationMillis) { + getInnerStatistics().addTimedOperation(prefix, durationMillis); + + } + + @Override + public void addTimedOperation(final String prefix, + final Duration duration) { + getInnerStatistics().addTimedOperation(prefix, duration); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index 6d386f250e6cf..a4d07c6c5f125 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -36,6 +36,7 @@ import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,7 +58,6 @@ import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; import org.apache.hadoop.fs.shell.CommandFormat; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.ExitUtil; @@ -363,7 +363,6 @@ ScanResult execute(final ScanArgs scanArgs) // extract the callbacks needed for the rest of the work storeContext = fs.createStoreContext(); - operations = fs.createMarkerToolOperations(); // filesystem policy. // if the -nonauth option is set, this is used to filter // out surplus markers from the results. @@ -417,13 +416,15 @@ ScanResult execute(final ScanArgs scanArgs) minMarkerCount = maxMarkerCount; maxMarkerCount = m; } - ScanResult result = scan(target, + // extract the callbacks needed for the rest of the work + operations = fs.createMarkerToolOperations( + target.toString()); + return scan(target, scanArgs.isDoPurge(), minMarkerCount, maxMarkerCount, scanArgs.getLimit(), filterPolicy); - return result; } /** diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md new file mode 100644 index 0000000000000..50d7e01de8458 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md @@ -0,0 +1,330 @@ + + +# Object Store Auditing + + +The S3A connector provides an extension point for auditing requests to S3. +The auditing can take place at the entry point to every FS operation, +and inside the AWS S3 SDK, immediately before the request is executed. + +The full architecture is covered in [Auditing Architecture](auditing_architecture.html); +this document covers its use. + +## Auditing workflow + +1. An _Auditor Service_ can be instantiated for each S3A FileSystem instance, + created during FS initialization and closed when the FS instance is closed. +1. The S3A FS will request from the Auditor Service an _Audit Span_ for each Hadoop FileSystem API call. +1. The audit span will have callbacks invoked during each of the S3 operations + invoked during the execution of the API call, *from within the AWS SDK* +1. This allows the Auditor Service to log requests made and associate with users and operations. +1. And/or reject operations. +1. The bundled "Logging Auditor" logs operations and attaches information about calls to the HTTP Referrer header. +1. So aiding debugging of issues related to performance, bucket load, S3 costs...etc. + +Thus: an Auditor Service can be plugged in to provide (best-effort) auditing as well +as hinted allow/deny security. + +* Why best effort: coverage is not complete. See limitations below. +* Why "hinted" security? Any custom code running in the JVM could retrieve the AWS + credential chain and so bypass this auditing mechanism. + +## Limitations + +This is not a means of controlling access to S3 resources. It is a best-effort +attempt at supporting logging of FileSystem operations API calls, and, in +particular, correlating S3 multiple object requests with a single FS API call, +ideally even identifying the process/job generating load. + +* Low-level code using public S3A methods intended only for internal use may not + create spans. +* Code which asks for the AWS S3 client may bypass span creation. +* Application code can also create a new S3 client (reusing any existing + credentials) + and so have unaudited access to S3. +* There's no tie-up with OpenTelemetry. +* Uploads and copy operations through the TransferManager do not pick up an + active span because work is executed in threads which the S3A code cannot + update. +* There's a limit to how long an http referer header can be; operations on long + paths may be incompletely logged. + +## Using Auditing + +The Logging Auditor is enabled by default; it annotates the S3 logs. + +### Auditor Options + +| Option | Meaning | Default Value | +|--------|---------|---------------| +| `fs.s3a.audit.service.classname` | Auditor classname | `org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor` | +| `fs.s3a.audit.request.handlers` | List of extra subclasses of AWS SDK RequestHandler2 to include in handler chain | `""` | +| `fs.s3a.audit.referrer.enabled` | Logging auditor to publish the audit information in the HTTP Referrer header | `true` | +| `fs.s3a.audit.referrer.filter` | List of audit fields to filter | `""` | +| `fs.s3a.audit.reject.out.of.span.operations` | Auditor to reject operations "outside of a span" | `false` | + + +### Disabling Auditing with the No-op Auditor + +The No-op auditor does not perform any logging of audit events. + +```xml + + fs.s3a.audit.service.classname + org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor + +``` + +### Auditing with the Logging Auditor + +The "Logging Auditor" is the default auditor. +It provides two forms of logging + +1. Logging of operations in the client via Log4J. +1. Dynamic generation of the HTTP Referrer header for S3 requests. + +The Logging Auditor is enabled by providing its classname in the option +`fs.s3a.audit.service.classname`. + +```xml + + fs.s3a.audit.service.classname + org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor + +``` + + +To print auditing events in the local client logs, set the associated Log4J log +to log at debug: + +``` +# Auditing +log4j.logger.org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor=DEBUG +``` + + +### Integration with S3 Server Access Logging + +An AWS S3 bucket can be configured to store logs of all HTTP requests made of a bucket +into a different S3 bucket, +[S3 Server Access Logging](https://docs.aws.amazon.com/AmazonS3/latest/userguide/ServerLogs.html) +In the logging auditor the HTTP `referer` field of every AWS S3 request is built +up into a URL which provides context and span information. As this field is +saved in the S3 logs, if S3 bucket logging is enabled, the logs will be able to +correlate access by S3 clients to the actual operations taking place. + +Note: this logging is described as "Best Effort". There's no guarantee as to +when logs arrive. + +### Rejecting out of span operations + +The logging auditor can be configured to raise an exception whenever +a request is made to S3 outside an audited span -that is: the thread +interacting with S3 through the `S3AFileSystem` instance which created +the auditor does not have any span activated. + +This is primarily for development, as it can be used to guarantee +spans are being entered through the public API calls. + +```xml + + fs.s3a.audit.reject.out.of.span.operations + true + +``` + +This rejection process is disabled for some AWS S3 Request classes, +which are created within the AWS SDK as part of larger operations +and for which spans cannot be attached. + +| AWS Request Always allowed | Reason | +|----------------------------|--------| +| `GetBucketLocationRequest` | Used in AWS SDK to determine S3 endpoint | +| `CopyPartRequest` | Used in AWS SDK during copy operations | +| `CompleteMultipartUploadRequest` | USed in AWS SDK to complete copy operations | + +The request to initiate a copy/multipart upload is always audited, +therefore the auditing process does have coverage of rename and multipart +IO. However, the AWS S3 logs will not include full trace information +in the referrer header of the associated copy/complete calls. + + +## Auditing and the HTTP Referrer header + +The HTTP referrer header is attached by the logging auditor. +If the S3 Bucket is configured to log requests to another bucket, then these logs +entries will include the audit information _as the referrer_. + +This can be parsed (consult AWS documentation for a regular expression) +and the http referrer header extracted. + +``` +https://audit.example.org/hadoop/1/op_rename/3c0d9b7e-2a63-43d9-a220-3c574d768ef3-3/ + ?op=op_rename + &p1=s3a://alice-london/path1 + &pr=alice + &p2=s3a://alice-london/path2 + &ps=235865a0-d399-4696-9978-64568db1b51c + &id=3c0d9b7e-2a63-43d9-a220-3c574d768ef3-3 + &t0=12 + &fs=af5943a9-b6f6-4eec-9c58-008982fc492a + &t1=12 + &ts=1617116985923 +``` + +Here are the fields which may be found in a request. +If any of the field values were `null`, the field is omitted. + +| Name | Meaning | Example | +|------|---------|---------| +| `cm` | Command | `S3GuardTool$BucketInfo` | +| `fs` | FileSystem ID | `af5943a9-b6f6-4eec-9c58-008982fc492a` | +| `id` | Span ID | `3c0d9b7e-2a63-43d9-a220-3c574d768ef3-3` | +| `ji` | Job ID | `(Generated by query engine)` | +| `op` | Filesystem API call | `op_rename` | +| `p1` | Path 1 of operation | `s3a://alice-london/path1` | +| `p2` | Path 2 of operation | `s3a://alice-london/path2` | +| `pr` | Principal | `alice` | +| `ps` | Unique process UUID | `235865a0-d399-4696-9978-64568db1b51c` | +| `t0` | Thread 0: thread span was created in | `100` | +| `t1` | Thread 1: thread this operation was executed in | `200` | +| `ts` | Timestamp (UTC epoch millis) | `1617116985923` | + + +Thread IDs are from the current thread in the JVM. + +```java +Long.toString(Thread.currentThread().getId()) +``` + +When `t0` and `t1` are different it means that the span +has been handed off to another thread for work on +behalf of the original operation. +This can be correlated with log entries on the client +to isolate work to specific threads. + +### Limitations of the HTTP Referrer header + +There is a size limit on the length of the header; +operations on long paths may exceed it. +In such situations the audit log is incomplete. + +This is why the span ID is always passed in as part of the URL, +rather than just an HTTP query parameter: even if +the header is chopped, the span ID will always be present. + +## Privacy Implications of HTTP Referrer auditing + +When the S3A client makes requests of an S3 bucket, the auditor +adds span information to the header, which is then +stored in the logs + +If the S3 bucket is owned by the same organization as the client, +this span information is internal to the organization. + +If the S3 bucket is owned/managed by a different entity, +then the span information is visible in any S3 bucket logs +collected by that entity. This includes the principal name +and the command executed if the application is launched via the `Tools` or +service launcher APIs. + +Sharing this information can be disabled by either filtering specific +headers, or by explicitly disabling referrer header generation entirely. + +Note: even when the HTTP Referrer is disabled by or the principal filtered, +AWS S3 logs include ARN of the user or IAM role making the request. + +### Filtering Referrer headers + +Specific fields can be filtered from the referrer header, and so are not +included in the S3A logs. + +```xml + + fs.s3a.audit.referrer.filter + pr, cm + Strip out principal and command from referrer headers + +``` + +### Disabling Referrer headers + +The logging auditor can be configured to not add the referrer header +by setting the option `fs.s3a.audit.referrer.enabled` to `false`, +either globally or for specific buckets: + +```xml + + + fs.s3a.audit.referrer.enabled + false + Disable referrer for all buckets + + + + fs.s3a.bucket.landsat-pds.audit.referrer.enabled + false + Do not add the referrer header to landsat operations + +``` + + + +## Parsing AWS S3 Logs to extract the referrer header + +The Java pattern regular expression used in the `hadoop-aws` test suites to +extract headers is defined in + +``` +(?[^ ]*) (?[^ ]*) (?\[(.*?)\]) (?[^ ]*) (?[^ ]*) (?[^ ]*) (?[^ ]*) (?[^ ]*) (?(-|"[^"]*")) (?(-|[0-9]*)) (?[^ ]*) (?[^ ]*) (?[^ ]*) (?[^ ]*) (?[^ ]*) (?(-|"[^"]*")) (?(-|"[^"]*")) (?[^ ]*) (?[^ ]*) (?[^ ]*) (?[^ ]*) (?[^ ]*) (?[^ ]*) (?[^ ]*)*$ +``` + +The class `org.apache.hadoop.fs.s3a.audit.S3LogParser` provides this pattern +as well as constants for each group. It is declared as `Public/Unstable`. + +## Debugging + +The `org.apache.hadoop.fs.s3a.audit` log context contains logs for the different +components implementing auditing. + +Logging of requests audited with the `LoggingAuditService` can be enabled by +setting that log to debug. + +``` +# Log before a request is made to S3 +log4j.logger.org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor=DEBUG +``` + +This adds one log line per request -and does provide some insight into +communications between the S3A client and AWS S3. + +For low-level debugging of the Auditing system, such as when when spans are +entered and exited, set the log to `TRACE`: + +``` +# log request creation, span lifecycle and other low-level details +log4j.logger.org.apache.hadoop.fs.s3a.audit=TRACE +``` + +This is very noisy and not recommended in normal operation. + +## Integration with S3A Committers + +Work submitted through the S3A committer will have the job (query) ID associated +with S3 operations taking place against all S3A filesystems in that thread. + +For this to be useful, the work performed in a task MUST be in the same thread +which called `jobSetup()` or `taskSetup()` on the committer. + diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing_architecture.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing_architecture.md new file mode 100644 index 0000000000000..50e8fe79fc669 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing_architecture.md @@ -0,0 +1,323 @@ + + +# Object Store Auditing: Architecture + +This the architecture document of the S3A [Auditing](auditing.html) component. + + +The S3A auditing subsystem is defined in the package `org.apache.hadoop.fs.s3a.audit`. + +This package is declared `LimitedPrivate`; some classes inside are explicitly +declared `@Public` (e.g `AuditConstants`) while others `@Private`. If declared +`@Private`, external auditing modules MUST NOT use them. + +```java +@InterfaceAudience.LimitedPrivate("auditing extensions") +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.audit; +``` + +The auditing implementation classes are all in the package +package `org.apache.hadoop.fs.s3a.audit.impl`. +These MUST NOT be subclassed or invoked directly by external code. + +Audit classes/interfaces which are intended to be used across object store +clients and manipulated from other parts of hadoop are in `hadoop-common` JAR in +the package `org.apache.hadoop.fs.store`. + +### Interface `org.apache.hadoop.fs.store.audit.AuditSpan` + +An AuditSpan audits a single Hadoop FileSystem API operation such as +`open(Path)`, `rename(Path, Path)` or `listFiles(Path, Boolean)`. + +```java +public interface AuditSpan extends Closeable { + + String getSpanId(); + + String getOperationName(); + + long getTimestamp(); + + AuditSpan activate(); + + void deactivate(); + + default void close() { + deactivate(); + } + + boolean isValidSpan(); + + void set(String key, String value); +} +``` + +Audit Spans are intended for use _within_ FileSystem clients; that is +not visible to applications invoking them. + +1. One `AuditSpan` is created per Hadoop FS API call. +1. Each span has the name of the operation and optionally source and destination paths. +1. A span may be `activate()`d or `deactivate()`d. Between these two operations a span is _active_. +1. Activation is on a per-thread basis. A single span can be active in multiple threads + simultaneously; other spans may be active in other threads. +1. A single filesystem can have only one active span per thread, but different filesystem + instances MAY have different active spans. +1. All store operations performed on a thread are considered _within_ + the active span. +1. Spans do not explicitly terminate; they just stop being invoked; eventually + Garbage Collection should dispose of them. +1. Every `AuditSpan` has an ID, which *must* be unique. A UUID and a counter is + the base implementation. +1. The `AuditSpan` class does extend `Closeable`; calling `close()` simply deactivates + the span _for that thread_. +1. All FS API calls which return objects which go on to perform FS operations + (`create()`, `open()`, incremental list calls which return `RemoteIterator` etc) pass + the span into the objects which they return. +1. As a result, any store IO performed by the returned streams and iterators MUST activate + the span before that IO and deactivate it afterwards. +1. There is also the "Unbonded Span" which is the effective span of an FS when there + is no active span. +1. Calling a store within the unbonded span is generally considered an bug. In the S3A codebase + this should never happen outside copy/rename operations, and will be logged at + warning level in the Logging Auditor. + +### interface `org.apache.hadoop.fs.store.audit.AuditSpanSource` + +This interface is implemented by sources of audit spans. + +```java +public interface AuditSpanSource { + + T createSpan(String operation, + @Nullable String path1, + @Nullable String path2) + throws IOException; +} +``` + +All S3 Auditors implement this interface, as does the `AuditManagerS3A`. +(Implementation note: so do `S3AFileSystem` and `WriteOperationHelper`) + +When a Hadoop FS API call is made of an `S3AFileSystem` instance, it +calls `startOperation` on its audit manager; this will relay it to +the auditor is bound to. + +The auditor then creates and returns a span for the specific operation. +The AuditManagerS3A will automatically activate the span returned by the auditor +(i.e. assign it the thread local variable tracking the active span in each thread) + +### Class `org.apache.hadoop.fs.audit.CommonAuditContext` + +This is a class in `hadoop-common` which provides a context to auditing operations +across all instrumented filesystems. + +It's Global Context values are a map of string keys and values, which are +constant across all threads. This is where global values such as a process +UUID and the class executed by `ToolRunner` are noted. + +The `CommonAuditContext.currentAuditContext()` call returns a thread local +`CommonAuditContext` which is a thread-local map of keys to string values. +It also supports a map of _evaluated entries_. +This is a map of type `Map<String, Supplier<String>>`. +supplier methods/lambda expressions set here are dynamically evaluated when +auditors retrieve the values. +Spans may be used on different thread from that which they were created. +Spans MUST always use the values from the `currentAuditContext()` in the creation +thread. + + +### class `NoopAuditor` + +This auditor creates spans which perform no auditing. +It is very efficient and reliable. + +### class `LoggingAuditor` + +The logging auditor logs operations to the console at DEBUG level (to keep the noise down), +and attaches the operation details in the HTTP "referer" header. + +It can be configured to raise an exception whenever an S3 API call is made +from within the unbonded span. +This option primarily for development, as it is how we can verify that all +calls are audited/identify where this is not possible. + + +### class `ActiveAuditManager` interface `ActiveAuditManager` + +The class `ActiveAuditManager` provides all the support needed for +`S3AFileSystem` to support spans, including +* Loading and starting the auditor declared in a Hadoop configuration. +* Maintaining a per-thread record of the active audit span +* Switching spans on `AuditSpan.activate()` and reverting to the + unbonded span in `deactivate()` and `close()`. +* Providing binding classes to be passed into the AWS SDK so as to + invoke audit operations prior to requests being issued. This is essential to + guarantee that all AWS S3 operations will be audited. + +It's a YARN composite service which follows the standard lifecycle. +The actual auditor is instantiated initialized and started in its service +start phase; closed when the Audit Manager is stopped. + +```java +public interface AuditManagerS3A extends Service, + AuditSpanSource, + AWSAuditEventCallbacks, + ActiveThreadSpanSource { + + /** + * Get the auditor; valid once initialized. + * @return the auditor. + */ + OperationAuditor getAuditor(); + + /** + * Create the request handler(s) for this audit service. + * The list returned is mutable; new handlers may be added. + * @return list of handlers for the SDK. + * @throws IOException failure. + */ + List createRequestHandlers() throws IOException; + + /** + * Return a transfer state change callback which + * fixes the active span context to be that in which + * the state change listener was created. + * This can be used to audit the creation of the multipart + * upload initiation request which the transfer manager + * makes when a file to be copied is split up. + * This must be invoked/used within the active span. + * @return a state change listener. + */ + TransferStateChangeListener createStateChangeListener(); + + /** + * Check for permission to access a path. + * The path is fully qualified and the status is the + * status of the path. + * This is called from the {@code FileSystem.access()} command + * and is a soft permission check used by Hive. + * @param path path to check + * @param status status of the path. + * @param mode access mode. + * @return true if access is allowed. + * @throws IOException failure + */ + boolean checkAccess(Path path, S3AFileStatus status, FsAction mode) + throws IOException; +} +``` + +## Using Audit Spans within the S3A Connector + +1. All public FS API calls must be marked as `@AuditEntryPoint` and initiate a span. +1. All interfaces which provided a subset of the store API to another class + (e.g. listing) MUST pick up the current span, store it, and activate/deactivate + the span when invoked. This ensures use across threads. +1. Methods/classes which operate across threads must store the audit span which + was active on their creation/invocation, and activate it in all threads which + interact with the FS. This should be automatic if callback interfaces + do this. +1. All S3 SDK request objects MUST be created in the request factory. Add new + methods if need be. + +## Implementing a custom `OperationAuditor` + + +_This extension point is `@Unstable`_ + +```java +@InterfaceAudience.LimitedPrivate("S3A auditing extensions") +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.audit; +``` + +A custom `OperationAuditor` auditor is a class which implements the interface +`org.apache.hadoop.fs.s3a.audit.OperationAuditor`. This SHOULD be done by +subclassing +`org.apache.hadoop.fs.s3a.audit.AbstractOperationAuditor`. + +It is a YARN service and follows the lifecycle: +configured in `serviceInit()`; start any worker threads/perform startup +operations in `serviceStart()` and shutdown in `serviceStop()`. + +In use, it will be instantiated in `S3AFileSystem.initialize()` +and shutdown when the FS instance is closed. + +It will be instantiated before the AWS S3 Client is built -it may provide a +request handler to be part of the handler chain of the S3 request pipeline. + +It will be closed in the `FileSystem.close()` operation, after the S3 Client is +itself closed. + + +### Design Decisions/Review questions + +### Why use https://audit.example.org/ as referrer host? + +IETF requires *.example.org to be unresolvable through DNS, so with a well configured DNS there's never any host to probe. + +It guarantees that there will never be real HTTP requests coming in from that host. + + +## And why `hadoop/1/` in the referrer path? + +Provenance and versioning. + + +### Why no explicit end to an AuditSpan? + +While most API calls have a bounded duration, e.g. `getFileStatus()`, +some calls have a very long lifespan (input and output streams). +List iterators are never formally terminated, they just "fall out of scope", +Thus, they'd never end. + +Having a uniform "Audit Spans are never explicitly terminated" design +means that it is consistent everywhere. + +### Can you activate an already active audit span? + +It's a no-op. + +It does mean that if you deactivate the span the first time, then the thread +reverts immediately to the unbonded span. + +### Why does `AuditSpan.deactivate()` switches to the unbound span, rather than the span which was active before + +Again, it gets complicated fast, especially when audit spans our shared across threads. + +Because of the sharing you cannot store the previous span in a field within the AuditSpan itself. + +Instead you need to have a thread local stack per FileSystem instance of active audit spans. + +And you had better be confident that audit spans are correctly activated and deactivated, +with no span deactivated more than once -else the stack will become confused. + +Having a simple "In Span" or "Out of Span" model avoids this problem. +However, it does prevent the S3A FileSystem implementation methods from +calling other methods which create new spans. +Hence the annotation of all span entry points as `@AuditEntryPoint` and +a need for rigorous review of the invocations. +As with the need to make sure that we never call retry() around a method tagged `@Retry`, +making sure that an audit entry point doesn't invoke another audit entry point +is going to become another piece of maintenance overhead. + + + +### History + + +* 2021-02 Creation. [HADOOP-17511](https://issues.apache.org/jira/browse/HADOOP-17511) _Add an Audit plugin point for S3A auditing/context_. + diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index c68c57d95617f..aff7694c52aa7 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -44,6 +44,8 @@ full details. * [Working with IAM Assumed Roles](./assumed_roles.html) * [S3A Delegation Token Support](./delegation_tokens.html) * [S3A Delegation Token Architecture](delegation_token_architecture.html). +* [Auditing](./auditing.html). +* [Auditing Architecture](./auditing_architecture.html). * [Testing](./testing.html) ## Overview diff --git a/hadoop-tools/hadoop-aws/src/site/resources/tools/hadoop-aws/audit-architecture.png b/hadoop-tools/hadoop-aws/src/site/resources/tools/hadoop-aws/audit-architecture.png new file mode 100644 index 0000000000000000000000000000000000000000..3d0adf61931cda905e601d70ef45f62945617250 GIT binary patch literal 55701 zcmc$`1yodB+dhscf+8RyA`K!Sozfj50-~fKFd`sGhjfe}B`O`#C`yNP4-!h3(%mRs z1I!FF^WTF$&*S^O-|t)BxBkCa%e7?AIeVX-_rCAzy6*Gx-dz=v^EBu2@bE}fZ!2lx z;SpHi;o*NF!UtEbcV3~#!+VaWswA)DVX{7T&gZf&?8nX{Jmo8=$d%~mX2Z`*Z03l0 z(GxfsofXyDzqY{Jel0)ac`0Ytr5)J>g`5%?`|g7 zIBZ&OqKsx5T@(y-Tq}O+xMt_?qWm!FKAhS4`I_l6#;(1Md36*p0?-URJTXW>$;lsh zhxbSccJe>o)hNDSf86?i`bf{h`dFDz|89Xn+1F7@T#H&+cL}T* zEc?>pa?L(BCR>T`cxJryIR);_URfmWmujaf|I7rzJy}hJz!xUlCE($2r^5Z_Re<63 zyzr4HzOaiMi07{&`Q-MPApQ6+eigs=-zeTOjc zcz%!JpijwAy@Hg|@+PbMG{Vd`U^bV1%VDCjXMiQSn=9mXUT3)?p;q_OUFGtsnesmy zn(re&ww3VI>9~qnkBFKbolJ~=#cJNZY; z3R2;#ToEXn@9_MMi5Owi=(@?2vIA*357o(O@&3ED~r_V*3G zlJX}r>`0CqM$=t_Y}kcfL2Ec`Xg(;h1OJw8EVAD zw;8mAe}!1#>-b_~Y0{j_`O|0j(E|})4r&iQ9?gtLB;2L(!G3PBSnD~ob5`<3xdr?& ztaRN}g9vv$B&`h5y;bao&Y96Iau4tEocy9y@hAJ+GQLIq=T+W#?SFwB8}VdsA#)tk zP1-mE;5#E(jr;STHmB+iR)^37Dw_kl1hm5q>}xsgokn?f*-)%Zg7L7jUQ9@2GTUoL z<{;Obb>meI+3_UI5t4qBWf4+7?%8)YQ1Aw4<4#*$nDx?r-DYixT0*#1#|XrGckyL# zpH)5o?%YcUY3yVu;nMx08f2cPOxh(sl;!F-3pAtASf*Lq!QrIu9!y$wd?DBe_Kmyr zcvr2Z(moN|wlMg;#vQ$DS;>P*`CJ8vmgET z{TZ}Atohxpf6lotY<4gq8P2Fk#K@VPp&SQQdkHltf#!x8VFRvVA9Dz6>h&Qd1&d3s=&cNN^D4(! z$;N5Bg$`9A3k+Y7V2q|A`mXB09bK7S2!s8|W;6A7NWl%+t=%)8Mh2d(Ws0&U5~`Ja zW%h7!{KOW`L@7awjRFH9NYwfE(?3TTMb=Gn7iXKQ4mW*0k2dD zhZ-O=Tz!KCveKq-AL3nEU$3J`n6DT7#{fJkWNUM*EV+}4ZnCk``>@FQkYMlYYienu z@q2~owVr9V*fC3nBddWx3>ww0&F>?@BKK=~QsS29vX4J9T+u+x$*quE$EQq7;mRTe zT^Q`sbPC(4jx|!e{mwMYnly2x%6jO|lNv$0&Edi_x4YBU7;cw!!r;kwog>-ZB0dbJ zGilcqVN8#9JkQi5bhl7Y@I`iD_xjN4Su=S?42MN^_Bvf0HkKXq`h_V)jhUwO1~A`Q zS8Gpr9(Ado=U6cRL_#7qLtZpzNi#t+4le4DlwV1)i#WSl+!zT}Y1j_so^e9KyW-WO zgPhFLquhj`Uro1XCkQY6$d9l3$P+8s32Bk|(^m9Sza_Vzi8?Nz1 zxvjFCMYc&lV^ZF{aWi(Qb0JsoK6LpfVNqidB!QRS0rr!oBHRznlf2x4v>+TYu!E#M zSsQ*?+<5SzR(Hs4dU7CmtA7r z*S0H}WNoy=%zqeG7uam8Lo}=T+>eob^SSm$UruJ?-C~R$uw!wuy6<-o0rD^GfAqhb z`s{!Dx)*$vcWA zLa;~9oL-DP+<2PFyXBN?np`#QykS>G!hiYBv(-VEW;$1z$FU602fbj|KMn-NbiN$4 zl*Y%?`{aTUCT6a+>W8jg6F1#^YP-imuVK``bcTeK67$Ve^zd%%J6)EmI=7REF1ACToE&bbl|tLl9DT2zfuPsavnv9fLR>`3wXpUz)8Crs;9=ce!Z5qG&h~MB1 zvB#t-@IWp;K^4~@KWGc5>Y_;4^*>Fx|6@Sc;z~i9gRaFmSbK|Xo>#_&qDPZz<%T^M z=jtpZ^Y!jRF*WJB>62_rTaAZa`Fe#<)S)Hywu17so~phj!LU@7*61RE4my&Y}lbwQ|%a(B;vk4ig`1VKDgY+ z`h^PxC&6$x2T@g0@2uY|^qC7NIk)mKnNG@~`0?suRD#L4!FkDNRtc#U>TGkxuw4(Z z<%W_HAk!Lb+lFyohp>8=m069_-Oj#gy(P6PjjmrP6DsUeM=NZyIbu{QZQS1>bH5NO zED<-`Htu??f>jl}a$~8%%XOv(t*C1c_ep1vQLUkSPXBs{oz-0iQ9B1t|^y}Xg1lc)aO zPe7hguWy9&!yVKitA4t$AE*2(+tRYS|M@?L`)JxnUX2*NLR?SthrN@BeZD9Q-A|*s zC1cBbs1f zE3FzI3L-OV3%M!^dqGj#WL$NlS%Pj&6nDmxFSX17e+w#VVLdJbH{ZRXGLefAdJ)|EdmO?BfgP5gDq7(Df# zT>4Ugt3Ca%+SfqsuMAQz=#$H4x8}0r8a4u~cGY>Pppmjv%BAd+?oEdJoT_i(DKkj) zpnWwT{qUq6Of+xF@Jg2BJ*)ru{aLaa_SwMxq@w3mk7V$+9J`L=PX;0s+D$l1Z=X1$ zb<+Kv2m~Iea_@Ipi~!?v_i772HEEn{-YUC%dYiPW+u>J5M{PcM(@k3}VqHA&GvM|` zl5WE14_vyWCja?8OK$|3<*qszGdW@Y|GG9Nd>7K*DC$_5L4xhrHcdp8)7N`*)rFR~ zm#rzz$X^8i0v|Su)haJqDKUkOYT}wy(k3=ZF8uh+;FxX3&c9yz;Sze;_bhlORQl^& zcm_F(T)N_iA41@Q+`0g%`R)z&fmHtrjStPzv82a*m zS;@Y*Qkcx?KMG~^F>CEpg9}UtS3@KNZoS6yC5?LD2QK{G#eeJ1=Pec{t@Uj1$nVCH zhh96-20#DqUlE_&hyQDX^4CcE__NsFoWJ0av;z+hbb4qt_8NT}um7=*j?UGH^KzL> zi;Gw5F87Nq64ROHlP^irZEvwjQ46j2?EIh&ydr|Sg!g%flatfD(^5v_V(a18pR137 z6Dhb$5dLW>wELVioydyEK8XTxvhO*(S?fjOP&PsY{wOd`Zs0 ze5de?-uRpvSOl)RWIL5uvmT2e`@Z97%EEvs5s&QwE8g?`C_bhnx<%qgpbC{yYo+Bt z2GV2~DdgFAK|QcDc#b!B36x?$T)EB!nI(Gorah&<#ndHSWrnHz(^vwq6}Mo|t(?JD zrT^8L9+zgm#v76RjT7faf_lW$Y8)=|i2SE7LZ@uM>f2k~O2-@p1#+h#EKHp(;*>Ou z;}JcYtx(X!C_es>MiF#tiH?h>i*om>*-3YL%n$)@>7`%dfgUTK^!R=<1!hkQ@~ z#KF&du@8#e)r2BT*Yo@Oj<>LWMaRW`beJ{1PHcT)ZSyh4U2SmiiH@G-+pg3wD~HAo z&>Jd2h~uh$azzf}alR*kWhOX~=1SqqyUStOY4weE6Cx@Rwt_R4U$jp)EcQx53mE5H z3uJyxQ^78Q00X5k&pM-YKIeB*`r85tl2>;x0uME@3w~c`dkh3pxwogJw!FJor<1lP zM@7>G95}zdm0lZydSaMeJk3CG`Fm`0(kar=tDmum2!~Y|sVx=`)j8h5+DYuecgCe& zBH9`~=C`r*`a5;+zkBz1pW?ZT6s1B^skH|nh9d<$eLLQM_(q|dxH?m!`Vm1`){}oz zoRd>%#g->Jlbf6eMq&wQ|Jd`$5(u? z+t3dY)&&}A94eDPjB_6tmKs*h`~gEmcqrTwUUfJbC?0hflSpQOoDjV*B)+G@|bJ`YPGi4448rU=az>q41nNS(d2VpiqEOZ z@Duy<9r->T7$)oJ~T{K?(_72bZ{b$FM1VnmEs-6VNVY@P8 z#z%T#TT?x(tRXLGcSbhSJ`GpZ@E?7=5KL49Q)+s*PUd!5wQzdOs9r|4F)M5CC;QTJ z*FsS*$J^3rQ5!XiVcYQ`*`YQ|F~se~ z*{NABgOBHz`X{P1ixVI;m9AaKlJ<=z;hi{aq!R?zej~=BZa=@3fH2Giy3rBCPE+)L zeWtO`BEdC^yhT?Z9xHkGs;oks1Bj}Q^RIEa<_F866$?W?^1;wdUYg#98PTuffOMHSiJ$WxIV#iAxTok~W8$fbmhXc%w+e1i z>1E<42F|!m;O!k0R%SM*P<`TL1RKhGt3h{_W8p@w<+oLLm^&)5E^b`@0`msiTSPu9&2&_?fecE=eIY}s=r*1lVenF{(Y5}V^AZ=w4&mZYg}YdX2^9g~B- zVU!`XFN%8%led0c;45CH{c4FLC-4=sZSPJwj9Kv%njmCO@eOUcmEp`NK&-@{*ti&; zOPrVCUksN+M`_yA?5y_#aB+2Hs}tD1T&~mFqJ3)W>ah!BDzDhuoz(2LMyRhWLXrF1 z%+8P21~T*M_`DC+vzA)~^WF-2ns<&Hls$o6Ya2nhwB-=qxtqkng6Byj9^YkJ@xkgN zr|dH(i>0uirAd?5hrV-)cn|0AR%@k;Z#6Q%xFmkHtJ^4l<&){5{kWJVKyg|IaEFZlDuSP4@NpTcT=E>;6tMamSQCFLvK=ARZ zlVl(lQNWr%5%&r|P6)AJlQKiTqFh=rPJ48dri8}$!JG3EZnk`F7U+202X-n!;(U*E zi-xq*Mk4$MQO(2?IX+=S2V|R&i3V?d>gyn6ebW77?%Q(SRJo78@LhoD@HGmn7s`5m zYiw^6+#d%95bOvX+@TnU54;qzUtUU!_+ncp2D7_^^+Jq7a|b^Qg}j(?&#^cU&SS!3 zE+MoF2@XOa5qN6+vfG1dgF9N)LoR_YGI6h&UfDXe{)M<;F|YKcBBSLw#uToEF<2}W z-TC;gb;IR32e!D*MX2jtfAz*83B9#7#l~=MzR}@NM)&k&`--sviuoHMxt8rd{z`_b zMaRi0w8yMn1QV~E4+!@NlF#pbE#SBb{VJcd)-d?EqQ3ZAH9f-S0sABGaLab890#H3 zAPUQoPK7U`T&(8un4uZ#tJK1U4&2B`^>ovDntq4mO)88vD_?1gNe3Abo3(4VbZ@kY z8)yXMmvXDjoH~una~8K!tvS}kcAiql(8HTqN__xy%lA-F75gCA?ksoFZJiPa&1lm= zW}yv$THWhP=S}CVvY!^#x7$1LAIaSv%71d(6F`A-=-VH4^YwC*S2~_GJgn*+Hz3X@ zkLcdeeXsP`x6`)JD5yr})GOq}A))@zgL9*sYIu11LMz0N#W{(DX9l!4;b4`%d`Z>* z0f|`gWBdM8*ZwBWK!{&zWF|iR9Pvgv#xO zZ?(~{eZ_%q({?>hPtWqTCa`tKu1&2?$m8I(g9v?#P9HH}haz21fKpZ4s&u-i)uln^ zM>}3?D0{|?B&a`F(`FR-WQcvH5v)F_|AeYO-H_WgIr+dCCif7kiI&2e!`&HA0Yoke z4qpD1jgVG;qM&s?Rk^HEc#&XVz|QQGeKPUhb&)H}U2dfZLDhHqm+AT<`R;X%mq6-v z`2hwBCe!DrNlg61&*E@zfb#&{GU+pdi%r@*vo!Q zShI8JkU`B((tY!Wd+gF&3!#m8cqB_&UZ=3Qt!C5>^;9wF7eFju!$+Za7cW(#dhShrUAV6jP6IyH;{0g6*t(LFifdhxxM}Hw89E;xHf zpW(o<9rG@M!^b1weHetFTU%k>`e%5TYEfa!Cw4tafh@SW&A=#^_?k8`k3%zUMnmhf6Ou;$7|O4U-Qpo3{iG1?vB+Dxjqxs~?JQi)Y&OF3!!JB2k6`{4L?!MK zl+aP0N&5NYY3OOxaA&K>T)=Eji?#6Mo3)>4D2=?HuDCy`H}~Vt-^k%#At>mCM|kI1 zdp~|WFZmsE;ei8o_+b$V!uLFG;$pG;(+?%xY||6=UShHUp9dI?P+JGxCLPs>z2ZIZ$Ej5)yYDYKgyb~7#zG6V z^8!bcCK==VPFhc8i*}q{yqM?P{VT%JNaqVh=VVYvq}|MRv+8Q&6zQ@u->20i;J(FT zdW(u#g`m`DtD(xoyN9eIb=X%U;=o>cMiK^GN>ux)ue6O!{ZEeD7EMr6EK8Oys%rPCRK+G?=8W z%4XzlvPdrnhY3eWE%0V@9ezZy%mCkXv>Jm>DTr`j^hl*EA4L9!(9QRmROH34bv{yv z>nhO*4;6eEPyYNMvlBSrP)7?gPa>`To^xAA#`+8%|@@0rOVppH?j{sYN`e_0oboUfOAIk5G=@cYd|;+KzrvgsI${v zHdr|{``M4tA!LPJea3$9p@xd6o>>&(9k_Wg-{~luV)Q#bdtJtS-0Cq$y>0pBPz`Od zSdkCimv^C;UfSkSjPi6*+1e?L&s_PGjxC$C>B^cNdc~~r*FrKfs#d7n>+jo)ep-LQ zW7>DcBC#Z7Ed0FZElnxE^w+P-w|vVv6WTT=YZiui_K5dn>(+{B`8K>>+Ol^a47t-u z?QT!)`hYM71Ga?r9&`gm`f*&}$Z4!%aZ3z73|#oA5+(-+w%N zfbIb7p-uO=H-#fop{Cjlh&i_g zA6?AlZl?N%fhw$3%a;SaCBgKB8g#3x3`ct8JeSy2sSh33jg~idFJ`U@k!y>dVc#|# zPH3M_7qTg1(wtYX1LFW1#oh!+^FxFQy(Wgy-oe;c?^VjR#s8*p#PQU3D=|H;Xk4l^Q zb}CW#Z)adAo+~m|?*X<(Rkhe;%H1A&&u-5D9?R+6V`=x90_m{U08g_;n4oYJ`3ssK z&wepE>SCg$QKuanVNHB)@$%iFmLb+ldhn9GMqTN`@wSV<04bUnzR>BMjGFK?!ONcT zxVE}y=Sam{mlE7j+b^69;}pVifgrF*t25<=#AiwAm=vivVo6{^3#^H?5A9UGWOrrf zL7?l?4f&X^cRfp5eYAeZnugM{sAq;pUU#~aC2O^_4@zO>+f%YNs>XLNc2iAKcT?S7 zT;WX^hJmGquJ{?lCR3~ze`DC@- zg!4>6y7%orSLb5e+q4%;Sdh=L~(Y*b?zHX<@8uH z>q?lU8syh)X1uHCeZU~7ozV5O~RONF0o5P>{U+*btpL%A^K%DAl`N% ztpmUw#Abic?OCb0DwaOc#Z%JeHJ9&5 zlW1Iif^v>|owc7By3M>%(1>ig__rt2%J0&gR9mpv$$8Fze}+c?9kyQ1?AkZDF%9rf zOp&*Qf-zw-*Pi@3TsFW_$-{H?jj+r=DA$fS`)lEN%gfi7M2`JT|1P>SChCXzAz)6F^f8X&>R1o|0Gho63+5ZLO)>tC$Rv`?qQd*4WlZ%Nu2_I5U zX$rxnqxkKKd6G+@czA}d!@|xl+&^U{{2NQpAosW8z}6ZyKLAWd%HGaE1a@TtD-d5x z2q%0W2o@}$SPX;h={Zfg0k=VlSYm`GipdJBxKyPDWMSm?L6mal4Uv*EVq~c_CVKV~ z;4ADoVMex;`Aqzz^A0-$@qtuyZ*{}_J93Hi(lxEQ4u5;cY4mG)9!G|>Jz z*8r+cL++U7=H!2#R{&6Gz+P0I7fqW7Z_oi90dEk~tgDtINVp1gVch5rJIYN7TuCc6 z0LY&KMc>ZN<5Lb1+Fp#41lKK>L|N}o-GUlHp#z61o%U4isTy}5Yp z$w9_Y|1-M;cQeROu?N-vRW;EbsM=l}SM}A~zpK`2|EutULF%L)IX1gGSN0<9xkcg> zITrlzb1r{1nk_$Of^!~=d-G$K-)|N$<>kHCgzNDrIcBjk%tHoO_gmFpb^oGil;S4H zDnNTKNo3CTT7K$-obw&Zkb(D&5P{x{zJhZh%4@F_p@Mzys3%fn7N zb|&Sr^H=_*=~ax@H88w?@2l27uX0AIp2l;!eOB&_{EwC+HYZ9n#r>AE2pf-pyK(8* z_FjIeGjregVMS@PLo1YK!q@Qd?3?k3AzcG>k9s?*jYKer02$j1s)X+9&pbuSKkofJx}Jx!~gdNnF-;w3E<)5iM8j4P!`J+ zU(NhA51*aZAK(d}THy!H{LgOGkv`b5p4t9u7yuJU*A`A7NAYP5bohYe-)s7B7MQOK z!V?QWb@4gdqmffwCL6RSz9s^vkfrrY3ELiyZi;fbX_C^EIi=jQ%|5xA@Qa}+fm-vhDY-Iw{h@Iy?{&m1u-v@pyAz(YDC{7OIfUBSaCI)bW zYL|c)Xg{_y_z?j@nEgCRaY7`)UZK+D*TmYXY)!xvCUZU3V+6hULo~a)OoD6F>>>H& z7eGlKNlA%Q;##JM2+(dEH^Mk{kzqF}&KX2Pd)AcrM5vbRxVuFY+5m8Pw3z##JA{^+ ztMvR=wgkO4^Dz5x#kSHb_I%&iiGII_6T;*z%VP~2wKid9xaM*gExC1E;&O80z#){XG@&!SE&=w9txzIo4Bqpxxg}(UzKOh6o?(8EZU$k>tyS( zkpu|+^Gj@dw2H4{Fg-z-FPBOm!EUN|-{oOw$lCurz+Gsr>NU_-N8Do*{Bl7zG5OW* zn)MqX@NM518QL(3GUf-o3p~A~#2hF9Y0K>gqi0DQ$J%GijPZU z&c%7!iH54}$iQ6a33dFblPVVNmw2TOO8dC=ieOT`EZ|H2Yawu7orLX7+J|!sDk(+N zMSR@6Mouje*q=Cg(-j=t)Y%oGs~bEqGyxVDm<1%hdr2pi2WneKt6XbywC-?ebp{BN zIvbq>zs+!wg7J}RkK_}>BwruKHIVWj}e_~Gc7MANV~t!OXHQ9Qfx2G&4n(02DpK{w~s z#^LOA5GyDQw>1$VYK7Ul5do3pZ;&y77XKwoCe1S!9&SHn5OWxUQzA8`c)6D>uPRz#7sJA z-%r!SMz_ouyGNGEx&!{Dy9Mxk(c+%pSE);7F2P#HwuTdK(@9{aUZY<_598n0|D;u0 zbfVh0Az?D)sI6rWi!JWuh%D?G*Glo8t?STgA5l&l8J`T^Z}K-Y*u9ax%6@FQ>Q>yt z^|rL>@Zi=AJ9UsD?DIdA2acO}Q-!gX9?GwlhIsEAoq$g%Qn8}BkU$GYV z9Z~T`?Mvv97eoNH!bhqQ*RAOazye3eI7f&uO?=vvHc(O4Th_9ZM57LC)=POylcWMC zfyAANqac552<9tUVN2vWxwDYw3-Dq|SL>PLt++Lj3jAxys`t1KT*;9*AVWmZvNemEbTd{6V|o&z75XTi^li}D$nRg@lcHS3DOqH z-{90IXquK-!dk7#|41H zo73O<5d{UEk*dYr+wNOEQ}-g-UA6b7BPV7cQLL=vDS8T7)oShp`5%U4fAfb254C9l zX3b(?7JWzGVx>zvQoL;PL(#X0@-*f(iUO$XEzCPQ0}cD=)u)sb21#5IT=209cFnB) z-7|eZ#yvfsm_GG9nBQ;ynS7_b?m+|ZxGN(NvVVxyYN)6|5QLO2sXqNPJ_n~xhzx{@ zg&*yjHm683QvRt-Hg@3U>kj-889KqTIigb34-W4PUj0kq&9R-jQft_#%xnIOLw z(mo;}-o+#OCRmVsWCCEd0}&}-{imMHh{?S^tdAPgZcF445@1e{zO*U5;v{BKV^51i zus!m32YdD|QBb7&F8fDZ(EuqQ5mE&3pDzQBHGv5914xIE_Iid}>Ps+?rg9Bvwyjq# z?V?DB?sjf}jTD=OB>qgdA;LjPM2VW=fD|aRO1yH{1Uu#VBr(D7Q|4`+tz4pBwV|iS za0DV|7dl@y+RTo#I|dneysNNxAaqDjWt5LP zg8_TokIa%3+#Div+ z!yd`EELHIg=VXd(U79L`cLPe12&k*$3saMXNCbZ3M848XB~{-q`{^qDBkYgvL-W1P z(hSJpppUi`zI%{stihxnOnb}{$PY84t*M4NUd*6srhv(3tJ z;aPK_nuR`Ur_b{_q2a!OWR2ic226tl>6e`OsPS0YOqeorR?<;pO=uh48mUcV_o|@NQ2TWtmn>*6ti|x*&7VTxy9=xk8#d4V1nPGPjwG1zL?G9Y z9rb9Sx{*4Zd;uipg%}h6b}~gI90wO3?1br?cp9Z888S#69pIS9CA+7@7LvRAXw*UZ z{oSn`Du$>fpp5Vvi&hQZ07o66!4fe(yfNEje=Y9P_!fts1MKq(gl*u6zQ6a|&cr0{JwvR5^$k0HR zWOyFO)`|UM>!h9&eFQ$*k{U=+azs8BH%gTkWXVVXs&2c) z9LZBs!VfLqT)6OiPOmPne5-e0Lky>*Xz*QakkgNR_u-S;vRGLI`ji#XWmk}tbF-pp zv3YntI3)mqBBn{?Bhv1Bu8g{hI|@M3&|qn${vhws1<(8|8qFFl2Wme^CoAAflJhWMSuKr3IHUv~BT$vwn__fVVcm&rA<(kaFd+}Tm z*_}i`6SEm31&sbZ$>9Z~#1K&B{qj*=U~)X(sBK}u>&sS`s<;i8wzjsmF`|Cb7Rz$D zpz@11=}#;>%yz?)j8v-&WEni0usc1tOQ~<<_FY3?6G&1XuP67lyU8FO>uY}kRvl>T zDVA3Pai+X^kr+RVOv%>)meHBciJP)!`XPUx?jA_J!w*qS6B>%=RSflWy`s8wu zR;F*$f$JGxsdbL(Tw+HA=`vpcHZ6fq?9#DEP$!izZo=@L6K2o$`Rcvz0CdDJstgB| z{Z8uWTy6ZOd?oyNBKuNl2CQ4TZpd2h%1?|V$YBC*q^RR8-b<5)+Acf`ho_E*w9ayD zW=sYi*oG03rSV}JaVDWwe_xx4@}~`sO{1R?5fAUf?r$D7H9tat1R{u=0TnfImVFxg zX>MwrT-W`zE{$uhIDE?Nt|QtYZ8x|i%{#>T6yRhhzWpks z5%t6tICl2u(|=KOi=;P>eEn5Ff(=%Ac_sI1f>CXcWlGy-AvWoG>B35n2&>4I#R$k+ zzO=g+#P|SWmn56KlBf%3Kd!J@*I&78wYRM)1UT=wye%4jfVGy=8yKK^m1{<-D`etE z`bQwf>BX?|ZS>&0C!FyCzlsw4LSSCRm1P7XmP95u+E9-I{=Rg!{)Rb>>3FSFxWUvn zjyK8xorqk+*Ly-B%W=_ae}!CO7)X`;?`@dj@V?*qDsQUx$-%gTd=-@G_k$21wY;OQ zP6v{PUWe%5@>C=V@ZqO3-jPk_>fxNJo7=l1SzR|)e$FZCc(U*v=`eEfAFsOhjHIB{Ai6J}dE%Y<-BQ5j|w_ zK5uo#&`xHuWuKSslietW;O2rxkX_fsyC&$~OZ!UJOAU2N>E&ASntNBzS2CF#AiR&i zalFL==&x;H?b;g+H=gaEB&cWX;?&lwfOXG_QZ03H>)01c z9x9)9{=WZ)6^NMDR^1q-l5PU6=e`EXOnx?uYYuqo7Hy^HNiDiA8+WM$PQq->zj;BP zdR7jquuW?*!W&*UA+wKsmc}gF%08rh7ZFx}=DyiMU&bbOSd`a0`Y@_uY4YZA)bRH_ znTm?02w6HYniK?8{8D@AxFHDa9&*S$b@VjZtD@4Fx~HW-V-Eye;LOtq^3>{x$rkL? zH18STojRo99n-xr6hn;}h)FEdYUHZQi_k?E0x#XS8wcI~%J1Pj5>^2-%;?8o1T?kC zozCA$J*E8i>q1j??$SP1#(tZ-z?2n|)SsP^{RCJ#n;6SZVGIzN<;3Zyt4jin(@5`Z zl~$nvI-J|4(TYd0gqY8EuXmryLW8;m?6CxLT_7b{?xAmJtCHX{`}sGA_VO~d2QH)W z&Du;~2l5D*Kvb2Zo%dnkoc1`|IfI zUw6`lG$|=*(fOu2sB_F@dDXL?-VMgiW+2|W&SrA)r|FGupoRim&gY@WB#Y}dwKPFf_BYEW_q2imz*O?iCL!VB&yNjg0JAy*5R4q`E#$d+gX~*{o*1 z)t6XVve6~DJ}g-zxr%RZ^kdz53k{Cg8Z;)8gfZnS({ z?L{3u6U5j=p3zxdK#UpyLJyZF_KVTWOE>gk8kEz(z6NTv=SL^;IG}~5C8atpK!JrU2|6uipXj<)FFXN0?R|UPK$z-lJ>h=B?|5-kgEuW>OHgE>B zJhb{N>xtg>V|EDHfOU~fwdHS9FBgRL4J<{bn+^CeGJj<*TU*d=>Y5^10(AS2KEJgV z2PZ0*l&GoJni+%0r+FA&=kGhK>S!`XD{^O4#ZBr}Gpu3G-hs8n#(5&1fWCYQZHx0T zQJg5~K4X4PZcl%#rPPbO2Y2Lp5b`L|l zJap|hwg+gW5~)l%>;#FDEzX-`f;c%XloIUuw)!tB{V|{d@tCy!mtZEy z7y4f#oj%b29-7IN_?TWc>CQz40G@tQ1Oyp6%u(36g9@()$iwS$eEgrf-)S||N$+NZ z;1wtSD-MhgK(#U!&(?J1}~wmE0Uk#bA&vkT4 zM`HFEv14WiWLvI49acQb0>PuY!H1P#HcdN|4a*F3JG*|=r4MNs$?74apqX^Hm%sCb zl?}6_-zYfE-21wlhC}lCLoNF`K-O7_$Ln#&r9xM}q)NMSPJ^Z$0Kbs{G%atrBn27H zIZpkF#jZ7!uy={*(&~&F4F?1<`{Ju)D}}3*JerirU+n^(=DZSD_n@jr#onP78DL+m zf!gijR(Z_y@A5xisPQ!@B(Q+w*bhGT*>``HK4JQ#IQ+Nxp1+hhSKO?YJ>P(=Pa=>c zZI#^kP<8&UugbZN3!}otu4}|=vccO^w7D`LlsZ?~*l4ycitbZUJ^#TyXJi3Yn^<{vVY?IMnal*_s9?y{7AG z?A~fVXg1;A7Xr@eR=n-EfI93KNDtJXJJ8+%G(t}(J>k55d(1Sa7<_|ioZ7J!0z+C4 zuqI4;B9=H+GQer(x-kLmFUg#=u2=NohZto9fRT#;JRlzZo~j3>y)5O{(8|#3Q70z9 zAgP~)0opU9?^91@AJ=|Z%vhkoZYW;Tfr?EYVB(#k(^e3uQ9Ci)a?8g}8k&_t7x&H*K0` z?s~~2$XoM^H{A0k-!4I@w&`83{Mo@O7As4pJBw&D?l7S_z7s0*xuv5s<((ri5UuMB+fw(aZLjSFJUZxHy!q|~Q;x+c%@koisn($naLr>$*&?{k*Pw|tMR z2t2MjI0AmF^sUSm;Eao!q(nr(1|kQxq?K}d_S5_jZ0;8=_;)$5Z9}TCD1 z`y`u_Q%|S0{!2Txga^beZ_7;VFi%v~+s)N!(d|An`BIhCkvB2^ddYinvpdsMtS@Vd&fF znw(UAd(!)l?A=tFkNBUzEcsq7`?`;R^Qo@>O44HS9EWEe-L?o&A?kStU(xa_qLxyY z)*fB#>GiLU`&?h5JnyPv*6lKPh8B?REmu>Rv)+nUs@HE?T>!EG-jo*#YI3q<5eGXA zCfX|eVv6bl4z`INQv)Ekx8JDVi-$KutW)cn7(=*+t}@z4Dj7&aE9>$AciIR^jnTEK9( zQl7Q^*JiF{^k3@lYsvJj4;dLGztDS8yPNhQn~A$H$X6Dol357*qq+=o@_)JeHP-6f zkCnLrndugrRj>#JzSNgk;6@aDfyTvD{pJ6eqA*2To>OQksqr>%4-NXJ*$aQX7IZO` z>pym)SNq_KdO^wx%Mwh;5Em59_RRf~9 z{M^ajyNm~UWtM$3$>*%fZ`urBt<%XIILxrj=JQp9eWo<$_CV@5fzJwsrbkkJ=^YIn z8ElcCVY)_wyfAn0X)L7S&a$nbx~M3XypMwq!;MOzw#+SZL!cR&iRN=8_!{7JRiy1! zeHH^L@+|q&eoLaNkRF+J-R`kawi9tq1WufTjQ8>9;=l5`<&(I`{7txHzi8;~7cz>~ z(zXYt#gBV$0Vz7r`FP_pTEA5RXUD(f5nXUk33+jD$P1Xf63sq!^M+{|nKz}2$a`EG zrpgQpE~ym83Yk6snqYz7HvEl;hBVzi$+d48eHZP?l6^<>-l^IoJ-Xgv@+IYsl`}5Y^$?fp`q80-sv6&{N$f(h8Oxpx za)}D$zY$(Dq6$AGY2K4hXP#sbL4DE>N-1-yW~ZS5LM&z!!+;>rXr6R4keYaS(1KN8 zPZgJLdF3jgjnZXOO?lm=Kr8o)e^RBdbj--2cZkm!i0ySLIMinec=OOu3*<+<@Kt2Q zEGTE6k4RN7jsqO*Nd4XVpx=qDH4`m6E0iB)Se(A5o50uLI7T+kQR;DJ`pYWZEOqi* zx}fG{rfc2nhuX$)DVynhkC2aukXjyKKtEe^R~OdKQ!PRLmbW3egp!-Z3man3xNYVR zuAwD;l=YfM7r(9kXk4)ie^WlPpI|A7IjEBL*_&X2V!kIpk>3+6AijRfQ0$@0l8J|g z*@0o;1+sk-p58%o-K-H^pM$No=*FT+WKAGge`eGqgp@Ol>}t^ILRyVUa|04%xx@itIGo!Wcw z>Bs)G6J?_}NP|s<9=}?s@Wg%I>CU@J^UWKEDf3;LnB(5?GtVuUVWW$cRf}FMA#>!J5gTuggfv#V_zCK>l?xX#7wA?9ta;<&tjiD+? z>jEF#sBS!{7n(pFT-UfNI}g5B;yVXEwQ*N$1{pgRyg_?y(e2tB@R=hWvJseX9l*j6 zP0c02@2Lb;M#Cm+@u;)KtoJVSX08!R+Q*JS!UXO@F`+}U@!=a=726=#O z)Trw5A5xxcgzBeZTeLC4ztj_N&A!!M@)T;Py@f_SiSFAGJsyZ@s!G4`c3Al+UZ`Ru%Ap< z_s-1qIJDmw0D6Y}U>evPV5;H?Vu)0iCC=FHCq{04XImtGzvDL;GIXFg+$dmbDZ6K4 zz0=LZisCG_kVAg3R-X-)9KtyUNN^hZsV(2Xa5|Z4FdfwN2&iQu6z4w2|IQc>nviDkh zuC?Zx^O?__tJ_y4QP)~-iT8GITk?rg&lETPvj33sUjuZ<8NKc7b>7u9rf_eobYIL! z*6^g>8LQKA@q76EtQvo2ZRHayWVP7wzcPfo7~OwFenPDXefaMZgj=TUa}yT($F-o@ zcS#w2vr;Ph5J#srJn2u?Z!x4(QDk)ahVsOmuW!y)T=*QLcGEmKxXOz8MsOsgjxssJ zK+>v7O(*R+WN7*8TmePkRApg-)?*!|Zp5tbZQoYxryYCnl#**rnJus(VUYocRH98MAe;kT*)NS)>7_!$F7uJ2I%vyVgxN*80R6vT*g+=Q%9k9a~rpX>4g0q4bS@=yqzd-szBSTAj^CmrT2V1TaR>qbh-sc(=($Y)(TDz|q zVnkxRUM#jhU3w}Qc{p3Qiv2eRnc;|5N6$s-<^NFmisDW^X>a7 zC&q06EIorcdaDH5C`~X0yKe!xCXDB>d2Zg*|F;rxRM#9zR&LlzQO%oW#QLX*&v<1M@5Flb3 zslNLW+M1fW733BiX~nMuN>5&hV^3NaP#v4KtRcL4O2~m_{mvdQs0bv!e$2vKmRT|4 zuic*iRkElUTLHhY3+wX8eZ8nZpu(F9YhB@`mVP&aDh6sE-U}LF|3-Yp`uXQfz`0pY zw#EYh|2i5gSmU9rz2V4jTd&klG^2=X`w1j|;@q^ma8EEBoN;ZhyYm2Oe)rN@{Pv=| zw0JWl{R)%y6I?CSK&JODr=^76#_0Y@ihn=tUGPVc2>lTm?YR_C^j%Z+T%_gCJ6HeP zB<5V+f}`Dz+X=#dXavZF;pBsX}hfhaZ=1WH*Jv4b%u6=wo5{#HYY(>$7YeWXTlgVu}BQvvi_>! zIPpV18~)U)(5)IePi3t=@H=^VcP0&%{lr5JNL@~z^*TLe!A`KpwXb;Z#xD8o&0gKT zB~XHMuQt^z@e9yS^Gl?62!0D)2j~+}o9S1Z!reMcIu-9%&j3M&C353*lS1R~Ry7&2 zVOxwNg1q;Fb7NMNGitYcBTw+mBD_eyi91N}z#lxdkW#=qKU zwp&(xW*0g%{yde}+0Q$$1jAb*WclQd^Ayt8RqXnr0&>{@iyi*OD(eVASwpq;&~g?? zc)}bd-OywHGsWMtnMOB&^rUP(=mkzvAn^|x8X3Y3q{;!*QpJMU+(yT`d4>ilF<2L( zeUMoGNyU2j`=gmLD11dek?9#gTT!Zu@lGnuH%2o!=Ds%L)Zp4;`1GsJ`xVna|g z5NHUB7K6K~cniyesDI9E4~o!kzPw+O;(1qY=3<`Q*91_JOhI-}*jmLlccBdOwzC#} zP1n?%G9fsl+Ol?p(Zu7HlBY;~(y+5!LdkuT8*ottQV{h!*vC6&#sjk8uNp*^ekv9+ zS_e?igtH%t+#xEWw*rmmB&{u3kZz=n4fB|qjc~?|fj!G~75$sx%61dQM(>YmqQ~3- z0>%cS{O;l82;@i7h{sPxt|5D#A%AYlnx(uOSp1cMvLgO9Lwkb_YwQw$ zN{z}v|UEH05;FUT?}_sjmr(d`l4SRIUTmP2jr(Nr+&~BGbTTQv{7Jw$k01&HcT2G2E?)IfE+fl#pH&H1pkY-vJ4gd-)6oCSD zA@7cxA2)DYHh=5}bN1r3rd9yDOY@K6cdh^fFBc=d@wg0k?#Tb*y#;UqgfE3HO5C%u zknNF^jPi*Rjq&Zixz(4eLC+1;*oZz#?mUCO>OV z>XNX!-v5;RQliGXLG)8#Zy;F{v3u;;UgEl|T{k9&n$=mxlm!XuY+Qc%&8hFi@S7cy ztHx(?6;Sh6{dP`!T@hPUkLAt-vr3vI6!Kwan;!47s3Of-q&1U{iVJl15*O`~fDjrk zmCaZK35jJrXI5GQtR!;Q1|0rby%q{+?%`JJryc=V>vKmz6-88=%(XOP>(}=s$O!EI zTa41mQfDyE@XGF1u9DdgY=DQBdLTjtP&tM;ft?&AabUow1d1o*7}WBNIk!fGzMBk5 zVJeW_t8?}?HO#nQkx{G#ro!TFZS-jNpF8h+oCgZ;gT&i)M z6jl84X0H9{beP{mZ7Tzyke_9br7n=zLO-(yE|noR<)oQAoQQG#_Ea4_lBwh zm2FLLjXYzJ_reZd zRo6MGU>4@SuvQ^eH2QqN9H@IaIodl=avn1r3@EaX2-<(%>U!m~5`6OBVSPhq(?B1p z2ktF{0lp2JTr~gF)=PX_-1OAl$9pGi8?QmPkE~-EvGpsiXOKv;^`I*|r*2pwG1FNkVTJ3fvy|h;&c>1up3tz%F0xl2 z$-kKnpgVW-{EJBC@@}ai*L`I*wS&dX+i@3NZ@Y>_vj&kHL2%cM$UVV@QA;E=v0sZv z8&ihPl+KSF+uQM9{83a?UXH?-;rv^Sk;v+ou0C<@6(wK2TF1jDZCt~_2Q+Y>Wdjwh z_4Y>gz0bSdQ^|L6;#H%0VTEqsMRool9)6uj}4#i*Yhwd|=$eY#2o2>@SYHp8RkhvGl zvJ*QxvtR7?o|UdJA+&%2Qq+sTopND1H!br)&DVVv72QMm{Er}HBJocDLfPw(Y70vr zn`L3~$QvfBdOiW{VjJNNUmHtpcrrM$6{sPq zW%9M*etV%#uuy_ty33$sya|Q-;4ptM^mR3QyP`n5NnLlxZ-c@6JBM+9%mBNaBzA2x zwYfDXB13xsGkjn$`l8ngm1-~T)k?l%0>C z3+|e!C&i^vWrTcv6DcULyHOs1oY0^vXJ>zZk4)81;;z@g6xfoa$G@AE zxq+6{cCgPr)*BGFe$3!gL@N9YFWhrqstj^O2lbemy^L$Dbf+lm6?>7uZF9{?r z8q8~^G@4siDD@hosf%4#xb|fAO{(0;9n5(Sq?_ye=VD~tBY6jUuqzEvHcpI+m7@1l z;^OxCh3sB_<)YrCn8Hx0c=|U+BD*gd_?+)(9DwxwwYBRqi}-jryDQwyE}48$Ql6hoO0}7TFTE4+MgtJHtPiTZ|8*kMNfuQe~@MIi#4Q<~CX% z2A@SWbf;Ad-8#dJ9lHfDBapD1b^rAOm6)Fsj(JvFd?meVtKSdwaX+}CWfNsrTVuam z^g5uGm@3?#d>Fx4u02$BEcwW)y-Db~2}1JqMHG1?LIK@LnXf{?-f)H>;CLrd>F{VM z7!nX8)`X6=my@>1esJ8uPt9HMd*lm+at4VI-Th;D&tZiXT$Z>fZ4#}S^17nn8F+Ri zMtmiFpCi09Ohn1I5RyA8G1?ig*|gRTM_5k3;X&TWsl-&llHHt`A!*;6{SGjoRd4Gr z`)$T@QBU)=ZjW^gKIV!XQ%u?z$1J^#%9!M_^xzf4NByuG| zsFwRjuJrsTp%$2S0K~t19p)o46K72MsN}TmC1K@grE6J?;CQ22vWU+mPSwx0j@}`{0*T7c%Kn#;?@UH zk6*uBjSDR5*(GUx$W}+3sq*ezWFo!xdGUmCv~zu-&YNV1;!}-AC|?t?}Kn@dEn+1pCyS zsAfD;mkvdigo6x*6LY_RafDP7Cblaga$U1KJKpPQ9XaCl(0C&-@K}OnCSEV@E!6W@`#}eKMUrZ- zOrTHlNtIryP}qDh{OV^n4qw-9Q%X-XbJf@6gYxo7EkWanZ-^2WXJ->ZWBj~SXL#Y36Y#4q=dMSd6y3#T179sA&bws7Zn84d^CsJcBM#H59l zHsl5!2((KS*u}c=&+O_wP#xkK+G1k_RfBD`48jK?aW3^GIsO*Q(yx**+;;A9WxhA} z=C}JdVN$_E zlCO4B$S#-fLj@~;s--Vx`uI6J_FdDImsvwsi?%iRM=Xj8pS_nErS&e%e=2ab&f0(D zu{L88TH{5oP$8q&BJ3*S_-crZM;4y0zCU zKztRLtH%$TvW~w6_h}~brMltJ1qyO?HCA~Hob}sZ_neQq$0A+_JXtID zBxvJouH-5mZ#Ww}E1SYBci(Ah$7lILi2d}laV9Wj_}p7p##P;}cx_%A!{sdSScDMa z0a#}GwfJoPyn)4e=%tQy9tdH~IqTIb__SyLF)=UcK{&0OyN)RY*=aRpMXYPas7%#K zf6QO2MH5&pMKDYw1t1! zao0}8iWFzj#Oc_!XvW?9GaZZHCHW>TWc1{v=04+{QL4+g4?POHUWC^8nkUCK#FbQ2 zRvJZh>B(=Ey83n}s46twW3hq&j`?Lzby_PcRv?38;Bz+mJ`p=)rl0G` zwQKi1roFbUv`3Z#j>n|cD%m4g_Cz|YU<_uAO!BjP7Vxxtb}%3I4(ILUA+VN2XnvTF zEn;weEyZfrf$`{Pau2$UFk|tO;6+Zm_ zb2XL7Y(Bs=`7O=wHRK%NGnge~sjJ>*q>_S9Ja8~f4OmWHrk!q^xF`D6-UIVN8NsV! z*8UM5o&Eb@br`xKUmdX{e?d<)bG71VgFp>e*JV4}_!=B3;z<`sjuJANo4swIOE}E0 zP)_&q0kkrr-~O@#&^wgM z6NtUx#YV5sd`%r*0igP7a&WvHyWrk|8UDoPJ$6?{Q7|m)*3CRmNv{!Qv)M?E0VR*9 z42Zlph?LV9|7F_@Y?dT;ox$73@Cm#3)Rn_l!HlDzYEm~Pw?hhBbHA4{@17r067fpC z;b21EOe5S{ITH;365k}LrqUcn@q6zkb9_k6IZ%70^WK+ihiIe6DQx~~nO64;$%bs$ z^VBlIl07Vfv414_k^g5au!SxppS>SX6>EP-_}f-@)kTUMU{RrE;F)sPudsV3=8e&T zw#g^ovM6UdvPas!c|38mAxP-)app-!e%P>EsQ#aI(}s(lEFm5&<(xAL|4oOBq|?Bj zSA^_*8X0%dsOQ)ZF_M359;2YGpo7Js6KOD7ooV>g`(}O z^6NP9G_y=^jW8#w!=HmV(! z67_7TIIz@rc{(WH+{8mVqm*=yGOh)Itrr87Qy`=IEmE();{M5^oWhcE?8%V!CT9u8 zNAJ#uIl(J`;KXDs8ZVotFMhzBXKN##hPDD7wB)U5A;MzTvh*x{ z_=f8IRwPEb4vofZY44=KU|RW#{1?UzM}bc9`Rw(XK3HKwio9;s*459tEVRyglMxWw z!`kgnVFF^tq~5VdDa0>%Nd{SSt}mucKHdu#E72y~OB9)UZ|3_GiZChj2P+b2za@!j8WCavxD4;f#hHF1miyICq3 z;!X?j9jaz~*|WLoDADl0yFpyZ%ydbG2b`0*_btS*1#i4|?NJcnjAh^s+08D0--T}k zT9d54<7vrfq&j^DnP^2JTvrYsY`og|sVhX|1! z<7-?3;v~QFkUs)Eec*75&3H*-hD7wt@SM@V#_2Z~tF&z7o?vv-3g$j4dkz@br%Q*{ zHK7%=C-S)JvSCQXje&@ZTr#_(bv+_Gp~3_3r5@g`mhDI7As6rzbT?4svfBjgw~%E+ zC-3zh-n_m$z{E3)N)2}O4Y>8%yUL)wz*)b__}q>1kj-(KHxhtAFUQztyK8)W5B%b* z%fEk7@YLvh@r}SkDQw zAf1XVq?7&dZo`1Cc7r^WzmT8?2=R-^4`yIdJ!wDn=(fYy*IzPzxB&r{;hEphz|M2_ z@~sz*G(o@vm{Z!3;x-=9y;5=OB%Wq^F8Jp1`Kg+_z{I+4sE*Hap#A#B{*wfjdhSE$ z>38zu?D*SBlI8zLkMyMcy?zHynYyY1t|oR{tyYCA_fBOZT=v}R*-7fGF>_qS;rUb63`F)PEwsAxA@8Dlj?yX0Xd2K9}jc*}zD*lhktDZt&zy z_)4a4zkyM7FjuD6S6En8N2RRj9g91)em!)G+*80c+g-NsxuR^?uo*v_1|1#fP2@gn zuI)>ZsO3fP2P3I!5@{9>pHW$bUzhMs6pYuXZ6Z504{Uq%kAa_3;;6}$GdCO}$>~Mn z1vSQ@p1xvxL(| z3%=VStN%z+|FSsIzpNMi{4qUyyK1J@#k_dOic_*)cTHqFE_YDQy(KBEwDWCvs(HNNqu%bS@a%Z##*I_wqTYtn?o$mXd1B-j0|*t;J>O+F zDlM+y_jf|aMmx`;$X~rc%Si6I#U0}Zoik&)0<#MY-oA$2QQ38{2|m{I+PJ=2gL%&F zKwowHZr_<$wOu}oJ9!SiK2!oRJIn0se}@o{ zBq#!30j;|usxp{kEwYu$3%-Bt02S<$B?>Aj0t~BJ-DQi)CVt!#_JgtXs}>}t)ndcv zL!Y2~>a9Uue|0Wv=j~yovE~E+qnb0Mn)|-m}z3*?_eaaguVA`V|Tl3SSwHs!Ek|irYyEE47r8A`5 z85jD_5DJvne-;~l=9VrhRY|;CCd54u+Yx7VL=GT|zohDEzO@B8SolI&R5m~?M~Ol7 zW%eJb7KClKQt<~OICj4fbvt_t73Gb+pf>e9DgyTQGh=31j=kJgCy#GDZV_XjuYJ3& z_G#HQ-Z&~>h7Dm__Q-V(r3d7Lr6p2Dq)@&(&(1JI7`>R=ZatyE#JB% zF#$`;^geu#1S{-mKP2*ATT&AWB&4!=7klCNQVTx*!{S_LJ@w{11KIN65l~6MSMd{UuIKHWucda)J9X3W=rGhEF z?_Wr}oH%k@Wg4(|TP!&o~bn)0aoR+Ci ztMGWh84G>T5We#^x~Eq6)J@3_Nt2sNeg|g*zWMjbn~Gw~A)A$XtChganp~-r6EnGc zPp?sM=AN(dBL-JQZM%fr%=7iFu+n%MJ8jt1%?5j84FQtCP#*ZMC-{*SWTm^0FH97% zt1H%Ft;4Z}9Jhnde>9A=biL(sfcrL?xGc`7S}Wg^YJTI#eC_fKJ7$+hPMH5qc7TDc z+JI47Jcr~}g_eeFfpHk-*Ia#K-6*kFK*f)K);1@L150AYuX-afJNa^UhE5gA{m5u} zCORi=0Cb;j;TPz;q~Be6Eu1BN1?b;CHYt>8m#tDMi-5?}_fll%Gu@o4w+m}ljJCLbk?f96Z*WL?#*lbo6$ zm!yep3l?>pfNR{MUWe4!97P_6rY34x&&l>1NKlV#Nu^82DUsy*qg~FHIM~g&7ry8o zJsggaxKfQ*ta5fAaxYLvN>lG~H@&e^&WrCmlLXF_paG^_P*VhSDhEHCFoAV@ zov?pUFup3(%*glx?)^dPf!zw~sf0D)}FRPIaj3`MhH_Y<#&ACTo^~q=# zg=f6v8R6!@U@=QcW5 zaGdC6f&_Na;rro27Ke+!-j6+p#DcKQ)`*bS8)bpiMNcS_zELD0PC^6nVcxD#h5n1H z{N@&LSx?w^l`-us4V{xQfB%4{sMoA@p`DV}vMbOb=weS!-=0{FVw97MciR37JCCP% zEo=ndqoLMvm^MZbZ9Ux}%GP8mo>43vgl~p!86w?~+*)-5ZGMU%2YRIhxcA2?U6{p- zaN4d*V$>IsJ1Mo|9*34l&UhD^^i%Khp=FdB4^oM#wQ|1${63PlJ0DMO#=1x(8LR1_qaFwCu3GHzn~` z6a=_z@oJ5mp^=o+yL`&&WYwsbZ`#qDbm1+ z50)8ET#k#J=BI{`3lz2pM35!7WU|!Nv z7jUNDQx(J+cKig~&v@cHe?hT0IPUEZn0NVpB9(YccOnoE&ezOrKHW<6e&^70vI56D zDc#nxdOiuP{LA=~1KMP=XfVrFrg{s3SXSP~pAA5Hwa&@HxYR`kyEIiQQ;pG-BTbBk zy5UBjmp(OnA-12YiXq|uD4_=Hx3_wY4jX-kBp%i=*|N@7a*8L@yCq>OU%z&Dkln`= z7=G$tB*#);o&-k4R<@7{9a$PBrjX-d@KggC_R`*urHqN+rvc^&{-rmU#0b7XXRIXjJq(6s91_Mj!FWW ze%R}ws?`^({}_eAVJp{v8&($Z4$91lDw9}n;Fs5MGqaC60$z-$Hd@JsVJikTG^t~1 z=zv~eGM2?W%OJ*q{%Dy8>2|}{Vl$p{BYXcrVMn5qxKdr7%w(OHy^+fpZ2t=cAQ1RR zQk~P+@_8@k#87Sed1Eck-?rfFJ8?DaaCd^V`Pw$XJP;FLPoX^CO-)SERv4=gvumU@DW~-e-If{JnTC`4mI%~de9x#diNIh9|eKpZWqEWr`mPkH8 zC~(X3%y+9|EpiOmuureb)>aK?l!M(F>`ZoFWi}~V$0m5suI87Ft7XH$EX$fOxt{Ul zG>T<9e9w}e_yBv?CAjNGH;B2uNr?ODK_!fa9m++4v$qNu zQ6)Hb0{cy5s*2B6P2(pG?4psD=q{3iTV>)qaO`Y>&>R$^wT>beSliM|>if%T-(dh9 zwx>4^cEcXd-8Z?J(jdw`_6VbAtUtn)Xn|*3{Hk9iY|z4G=eHJxIFWDv&Z81T)vd~X z5TgQ0kKr4SX3tdZ?O+KOYAdG*RjcMhMdRdPA!mnjh9ez*JXHV|qYyD2Xm>=ub$Tf6 z_TZajjTo&h-{dhtZdAwQNQB@)(EFqJTK|@DM{9emQ75HF3M0biH#uO# z7Kwf&hOETpDjFgIr-61=uW%6u4h}pM1L(C)qco$jK|W?-ab&k`755&}g}XrdSt7p2 zkI!vLd;QM_k7T!U=kPhGUzwp~|r>lj!+ z3CiIHqSAf%!gt7gCtm4Cim#dXY4XMr*g3veIsiNOqQO}sY*Z)!m0J-u+$Uls5&z*6 z((QQ#;974d-6xt2hOEj3G5(YPOUD=Pyajj~4Ud?@+oFV8I2Y%`{Vwa z^CNEK=?<}LO-4R-w{rk1Jc1_H$p-?#&8T9KNdAq8T|L4=CF;FS9g}`U_mboCiw{c@ z3t>W8j7GDAx52T1aU7^NI@|bTzg=F!z*>tVH|&;Uv#+qsPFeoiC|82*S5G4)>%=562ewXqbB*P-%@MDFO}Ff{O+B!p z(VvkD7hjd~M!!3-*@Pr^Zlrj+<<-@)&G6Yg70;&U5<37}&DIJ0%M3diC5OHKX4 znvMhJp7P`xf!DJg-zt0AL?N^Dqn2^k$7*|BYJr^?#ox_|j@w!-6myD>VfO7z-EUH& zc(AsKI41jG&C(RU?by9|N?~c+ClUTY%`Ge%1{y_cUKpXY3Y2 ztQn zDa^OhnAirHZHPa>#K~FMTJ)1ur5`o`)Xx7Xcu2nd zk;%%?dux~lhWqs|cd^hyaJfrCqt@QpI;lrBni%A><)*!Su%SJif#rdL^QP*60JCPH zQ&#X%{AP@hXzg+UqIr-xtk17&pWNVi@a`wv29%;TK>5Kz4N(m{_{|s0 zAo(SelYz{$HO;|^bf&P+;(yi?xxvPwTmwQu8o#}%U3!rE#NgCxRBMv9rtB}L^18&J z`CTV&Wh(>H)RgLs=Hx7E%&4~az5FyM(~rI#aG;{58|sU66TZ+x%Y@dp&kdXF3>?tD zF2zteU0*n(r!LIstY3K1N_w@31wkWTo-A-;+FFnvwg(naCZPpJxU`E;_JS!?=U2f< zykZ^TXt*kNsi9l%SYW|-@19ULtqm@GvL!8fV=J6WtP@{=BD7=;#H}ZH=d} zM2-E51FBY9G+(KFbK-Rw;i$RQG6meaYjBGXAfnv>^l@mBotA0;)lab?e2EqBbz1N* zpyhEljd@=#|SuZ357CjTC zklh`N-0^>JPShQgU36c)P8)rP+%W@g>dMzum>nKt{#O_2KsPN;u%(Bj>^H#U^i|%> z2@-l1n(=RCYzBW5&IXJOIU$&g^EY(G&_l=V@o!552&f1J{ZBjM!T|cYIz8xJpL&A1 z0d~TAdr&xA6O!tv`oDwk#3N{@slv_RrydMKKGV=HvSk6F8rj6h@a*wj?tcMa_#Ltj zTv^9aO}<%GswXGM`bC z!m#V}G$+T#{&N9<9CgBJEw{oFufC7cSa;efM-!0ch{d zz>nXZDyKU+Mp}0JPapr!|5zgrYf06!!SGKdE-|iA0-Zl4O`AR{^Tyopdg%_?+U=8X zI9qKAXYPo0oc#OE_BquxR@q8Hlav3vlY3=AIu!~KJ%A54CEpI=DAXN*cqj0hzw+xg z7Vwh~=c4ASiKnjv4jJ%4*6W`F4E4wHU$O#_)B9zRQ|%pICkGc_X8HepQ@;@WMZ@0+ zKJZ9pfNr?_4>L2fXPYkrBV$?c$yv+W=TCyOKY#v=F1NM&oan{X>0?NLGIicS0Kxuq z8b!_mz@{fZ1ejcJZkoZr1OloNcr^oZy8KT#t=HXOQvL!E6i&T;vqMH5^lJzX0TQ$9 z<-34J1XKm^G6lqcjtn4|q6WU*#`^PdtNarB8-U$<>Yo5B_*_as+Tv`I|6ceLmWxs& z8Z#s8XvV@iPkV5DcEF@X=PfLH=I@T&jiQwl0nQ>BF>N|L&Q3CYKpa%+oKlmN+Rp-r zRIR$r-g60zKq=b1y0)1rjFQf1CudR}W?9Pe`xq;Fwa=3+=kI>%ZGyZuzs<~&HroSe zH4L)|#+}YYsQ=IR06N{h&~MZ1G1rpsb}Z!YkkkaR{~M&nDwoJRIffZ<>2NY)^<-|v z*Z^3PmZ%stm0p2OR%AJhJZ&MaXVMA$No7<$adgbIG~mpy={t8qVR&f92pSjO-{p;B zC295zm=ZBt{+Q{Irj9I+0}dM&|9RdGQq$+X`lc}6|AO-BxeRnXd8&yEyQ(Ad4S=!S zSR?jX!sLVf2IkM=dpY{Q$d;9=!JF7-GG}D=iVdwCJw5LMB<8XDx}8>L{#{*s)dj#o z28QNOPmUzNvYHSu(BZ64Er)wV)8@egqpgw(q9SK;3ow{heQ9-f_h&EzQngxAiN65f zM84T8rq73BdoD6fzZp~zj>F)<4HMdn%lRQyj zkVLZ3 zfP_lWRuPo|{K?;)Kcnf)tB%k|Ubdf}y6^dTt-2={m;Tm?;9g;W>8};}nUuMpJ4wRl z_sh(A|x(m(MX=4J0^VUM#XOB}22Emd@~bDRW#PTWS~&RczUXFQ{L9;^Qy0>?S=aN6YM zs5-G}okXK*4zS#p;ZRGZWyB?ZZ)ky&i@u@d*fyA|!^I~mzmxJ-=VCoCVdx1a+W0Z^wye^@CG0e&D2@ zqm(XarerDIWnq<~NzsTgVpM`Sw8GP$EPI5sA>9SmDtwJy|JJ%-xY>)9*Z5Nb;*mhf z$xpZ|dt$T~qiD5c%(I4K8%H_v2KX}eh{UekUbU;$o}<)Mqx~jHUe)DKebD$zV*`bX zy(U?7J-xGItP_`=GXaFrmrv+B#6=wD?*;hb7G$e7v<8!mOO-x}PQowy-2$ZRX40pjJ{+mcKsW|HamGbeN}ww~4B;4oaM8qKE=G8#2K zfGz*Mi{QNi{=?6dyI}e0xLc^IM-w@fiNuZ%h)J100YYPdgAH}F#~qO6%7CJ5%9NCS zK9$Mrl{fE?EVa}na1_{Vu4WlzYRPEV7fXBCTkAdiBz@)ME(r>Iejt#E^O4B-IRP4qvot=9ZO`9;Kxh0^j;5AEP@lEk)Ny%ov}H~!LzbJ8Z4zy9s2EX+?Pre**HrMYA|_DbkWTo^Sa-Cq}c z@d}`5`(+sU0VdYXa{wFB?60ZxKQt2nZQ0*8@=Ym3=$EZPG6!shw~>*Njq_5!jf<9F zZ2@}~V49y;6Q@qB`d@!f?D$`AzJ&f-%Yd2F{;Qv-PJIS$^XmoJ?Z4juhWPI{|9hMN znQRiqyjqcOg7pz=_bbb3qABgY8MsDA$l%`$Ogcl{Yu@ab@h73a!3<;tN|(-~cxHFr zo-jUG!ae&%w5|y)tB0Sb2i`miS7(+#wUA+c5r{Nj?e$tQdMv_baj88SAZLiMeLFW2 z@XyT|?hbXpYi<31|I!PRCa;cDlE3-So#%cHcP+2fz4izio)ji%!)sT8wW5KK8 z!K;y*`W+JObc(G2{_YJ^AcE0i1M-Wl-!=IGZggyOfRC^E&1S?Oc8A+Qhr|j4o@404 z@TKDOuS3MbDrX0NF#aYZ9&kjie_91ZI;abt`bm%9qok=DF`O+4a|xk_J~o# zT>uQHZr@`(?2-DAYqgUr)D{TA28!%#NaSGY{bC4mt4U4~X%y%FSjae=t}j3rJ85#C ztiB(m8 zgn0~d!-^@;cuMa9bP|y~RML(`<2w0Q`GdKj#MY%>)N~5~+ZCd{gw~^L4vV=GXWX8V(&g>k)1Tqa*9#ik57c#!gqICjD?FNUg5-EWSMb?u)OH8;Es?OE=D)a#26{ zQmJshmd-TZS%$c328r;lMM@w}66Y#s%9cY+iGw7ud95a74xE3zKK;|P)Hjs0)3;(q z@aQv|{19Zl{|`CoT_TW&cL&n67#p^X^sNG)Kgv~~N*i#2>#KwVaX(@cWM8K-syM_A zqgHeqV7=~8W>xdevLJSmVXq%TS1TL&*7UW#v%t0H-(I&A-QQgCTj*Q8E1(-;o<-=0 z9@HJsn#Tq@9W`*KEqN8tJ?5VCYRv>3H}B)aEIWVVZUy9kS6n2Wj~?8e5^Q>?h=+jI zC7nRmV*eYOiK`2I?q}tP>h%bKO%&kfIp$A-JHaeiBEQi%+e%GqCrN()43*|o3$KeB z+)CK2?$v2TOkQKGVA?F#Y-$$i>%+FdzDb+R(RC8IKzxJ^tecBOV(m9)|!@L`9ozd=(7MYpUG^;M>P(qz(I%KFkcd zC_g=t5=vXTG;jt#c6*_unbb7UG9NqE9ca1m8h@=yU)9!SImJfrfx%QAzXXs2+e|21 zv*?p4iITjB9%b~0IiwOo{u(L04oYZ4!xz($ClR!KjSxFc6g$rmJi->#@xG@gt1lK4 zulS;L$=6A_W&2i?#^zo zE~JSMQ}7c+gp-mg_NB(~t0gV5A*&S~4YKCI2>oWCh>cWJNe>GD)Cmd)Jde#1rwsjs zc+yOhw4f*0{4ru%5{9-?sr;Mta(Cup(xpM@2LI}72xP<2s=FtDep3!G_`mkOMo04@ zX_ERwQtvQZjP#kz4py(Jw@DBArbX^mINBzTL|gUrD%ExrM8cO!(lXavZl_&)KB1TL zqCZH-!phQ@_fs1q^gzBz^5A8Dr^r2ZUT{~6eVk{Rk~BcfNh$51%!S++15SL$OQ=QO z{E009;K;Qq?83kN(cMyCv04Ua^|eG)z44%DLdY^BP42Zuk@>r_s0`PLJn|x*w+OF? zFG>`8is*j1!Epjp2M)?znX}%7{#)}t-Zwd37(5-k|IhNG0dwyCGAD;jlZ5mpfl7fy z>G$GJ?-?628{nmRz*2vZ9#LNO^z@A;NS z0c=_0_-a0&L8iljE=9Ia9S`!b@O7YEYLli#)+n~TI;{pr8+LmT5g{^%t+Ws?RMIdi zm05;R!aM7e%TCxFGK2I*`5!Kwq|Tlsh?yRL<%&ogofO%#WJ$omg2R{KTVUPhorsF0 ziA9b@wV`}}pbbq@@}fI6IA<}s&h~bGPY6Zj*j5OPm^#+9`4$0!rjXSve{kxj?+#%%3W%<5SOSBj!0VPsmYM~h@){|G>W z>SRpKV`s~t@eW-D<}OQCB_}c|U~JF=$io>bMXynW%4nt+?&u zuQfKPRP-`*Lp*)+%L(u4_)+b_{N((G(5&1|2~8il!K65rc)sc6nGr_YnOVKG1~Ff? z&hCeok;LH$+CsuHkk<MG3QbStG9|$bh2rU4s3UJ`R0aO!nt+;qi z!8=i+1GB>q?yF0jATZqk0;5rz=L~}I*oP~eo&oq%ebYK-x5rjLgKhR}p_UD0_kXRa zkunu1Q%Zef&I{LDI6dUHt9v`=Qgzd+J9F49z|+&%HKe}hU>~MnyfE5l?JK<1R@0x}Gc|#Z84i+C_q)|z~zhgKH-z-4nE7&XOC?Mq+?1>X%PW3vI z%Fh0K9D{xJ>rf@_V7WR!Tb2Iwou-~p->s&VPG|s>ZMg>OS)I#PZ${)_=H@v-;+>(= zv&2!%uC8skqLOOOs#}GG1&$<*3J-zq8j6AvIJd@{ar7d_bxknLFu!RX6LQU9q0i3z zo21G1N;y60N1dXJpII1K^~S9BHKbQSIo;313Ug2g3^S(9YV1A~LF+J!`+_dWs-@I2S2pU`KMDq_zVyjhWW@ z8K0~eqzm6K%lGrnTdjO{H_a7R=LfrT+8ZOZ4fHmrkqu^BbH%XM_rFDcGVN0E*P+MS0d| z%Sn(C2!(K#+-E<>0L1YiDTb6NH-wDhFhmuf$v;^UxmhyYd!sw*%Z)7IVrK5lwE$|u zIw>eA6*ED-4Dea7xm!E6`@PHuzy=d7J@ttn?eHPX56NK|bl_SlRdpQWJnU+us#s+V zWgD#0&JghCV=l|q%*s1i4Nd=7d*2z>)YkoZuZpOsC|r$FL_|P9h|)U>A|({1_b63* z?+`AcAiWdmMM4WjdQU{ER4D;zK{|vI>5!0Q4(fg1|IB}$dFI2E4>O`^2Ck0@~*v#o&%vF~#Q`I1R8>$4zVMse=^a1LXYlGXD$8gv>P=BpauOrgY;1f)% zT#9-5lhB3fHcE4ASFcSgTG~~;puV4WKkO+}c;x(ko>6$_)t$me?b0R>Y_=fLjfu6@ zo|nh|n)W~lwolT5LGv0J8;G}e?S-u2ia|?55U?;d_G@)j?w;#(HH+pjk{d6Ohg`?% zTUuKqjPNmTOyI*649l}g~7B!dFKt&efs7-YT zGJzd#RORXU*8h3bq-~V9lzYz@Z|9Iy+d4!b{0|0D>Z=_crHzi58qb6C6d(By^Ni`+_9fdbzQM37r&O78AM^DsA9 zgcVSP9n=5O?b^@wGD7}rDS*ZV&gILP(zm4W73Tcq2Bz@~`ji%x&ZmSoPxIT$TSvz- zH!W@+*nUH!Z;t^Sf`D!VrMZn(9&6Zidh(*@!DaA_vlv&O2eS`9DQ~l1CEph2j&ycy zWCx&2grku2{j)Y93J1ddlOx>J{zp-DTI};BzF%TDy9i$KyS4e+cQGY*#I!@QRrZVk zCY#{W+=AZm36t#t_Jqk+`)?+jFtB^x#;#iuqoCL*LQ(Mh1a^HZed?VdfMvDW?ud^i z0G#n9+OCxv9t0OQ=~;yK(zNGJI|wTJ6;A$!1Dc5Rze*Rasq$$MhE%Qt{a|WUM3Ry$ zAU*GZHP~uk90SL3GO&E}Fp8>5lHSesgq(iPe^leB_Q3x`s+(88_?n8fvSp>8wTn{p zO6pAArlZV{(lInycJ=6erp4;3jY&;bb)#yRx=zz49`iS?R1!+-2xSwD;n%S#L+H}p zonVLwXAfmJzclW8uD5m6B+^iQdfUCn8Td_-hN@a4crG#kWzCwtAE7ia*RrFKfDiIL zld_tE55Vtm+cZ7Wy=at_S#fPX!IFdtU`9rlHWY5XmT;hyNB+F z<0MT)0H5Y&vhVQBu__@;9sRS&aTf@QyG^@j5Ptd|V?TY5`E5-CE@nvUA5}V+{fnZP zwEQ}OymO9ARRN%lO~tNArI@JQ9nKPQ_>_blOt)2sbrXOzOWLK9p?ZGP~t*o`OD?WY`1)# z)U8uQ4+Q_g8(qP6063+Z8?v+g>&!75uYTZ~UIp0krbGsW(HE#l)CdP#MF^sm>5E^0 zl^W({mW=V$qV(WQrf|TIwy8mp6*o#(PWK6?B1AAXC|+| zZaw8~K&UG$q=SZm8U(430o6r-3Zk#=P7pGqhWnV$yU$k<)c8LHjbP%*bamR7T3HM- zpwmU0Q#GB=0PhcG9Ma zy5xa;D*Zlvarr9K!7>ntW`YC2h%XUf&w7?Y0Jii5{;UXKB>O~l=#3xVV0t&DGwQKC zvCxpT6qLSN+lCmJssbB3P^-u!b75?h0kSYhjIDbNY@01#hsUhxlgY7Bl39O)-VWd! zP!tC%K#bG-d4la_oUSslTgh)Rk>z$BG;Zzqa>Dzxgb0n2xeDWkIs41)dM<}w38D0y zSf|$I-0VD;ele>VDPl;WO3xBuO}f<5u+cKZ=J}=lu{7t7HpeNal9Y_a^fVm#WoKq( zKhXBVSuB7U=&3|MEZP3fETq%`|3Rw z)O0a!n)qsQROw=Uttz3zdpUi0vRR%qhHBT!rE^D`hL zQ9+-WrnmdreN`<3YS?vvFKmijBf`4!eexuY{)4*d-dfLR1OdebyqpLzm3h4r7mcg$ z|Q^66gu%|2P4X_4o1MCQLKHb>`f?ru^OfbZGSZ^ zp$3m?`iJu|rX9B6{@AkG8)XaxP8^hjgoy(LczW`IPz z17u@_+fuDcRl5z61LRCZr1BR>Hq>TtcAHh;nOL&$pWYu)2qnu+X$G-Fb5b*-D)dS1 z2aBiLcVJnwPM*`v$(d!k!$jgR%kZd~ zQ+(Il)enJknrWa7kcs$Mgqe^t*I8Wi0YPez+^g_>p*$f#j1R~U129FQfo($p$5H~J zsC4wJln+#kca@_62kt^^V1EOVAN@HiEWY$`^&h4s6@H~O9w$~}dRTcD5as!b=BRe{ z41S>FELnUz?IVt)LcZRa#RAa1?10p7zk17gk(HnhG@^(HU$^kiD zZY2C8j0bB<&Z@Ip1wcqL8#7~sZe*yi& zM>8JY?yHmn)0#q`&|b>hfdJW}FF^Dn;6Q>>9E^19fzR<`?15744nhc`&ctIJ(ZMaY zDKZV~s`fFM8^=BCMn2{4tju&UdL&|8o0hQjAku8F*IH&}bjISS0hfi;X;{=d$$B7S z=BeU-RYO|K&~yP6kj;;B`DGGZUVJR}cdEX6RHjljV6EiSbbD6PB&OSM?LDFTwRNJ> zd}Ak5K14xQ{Jdv`N`wX0LN$HjaU5SakUs~s zH)J`*xtN2m{l)6&uE^E(zGGazn?FK*at0KuR-Ev#P~WC@2H?(1IqkI>A%p(P7fa5R zkCFM3jmF%wOqpA*V@kfqygtb@OeLoS^wZPXGtPZ_zrDl4*S)E5DNh4GXIh0SUO}yUD;U~?7 zgVS6OqH~=`G5Y=_cNVz+J$i&tlmFUjzA}k#P&U<1fcF6MmRcamOGX? zNHN394EqbpY}#+4Q<7TZd0{0ikhs0^uk(-8=@(wo zwVZLE$4*~c&I~BMuOsD2tg)yVju?o9WGW;+Nu-!g^759O&EiytoN=O3UcpI*|dD#R?X; zHGG9%C+@pR2^0KZaHzm;b)#McN7nKwL`pVS z*7o1vx*=5GUk1re8pfn*CGV2tG;&{@LtyTTZNqT1`?;ga{n)1fT!?j9>^P6K>~I^h|zNx$-Mk* zLC=wvc%*v^EPgO$MSp&+_|dax>nCJYyjjx=Ng-`rNZIPIPD3q;`RO&guW7s*uLV9S zrv`5E3*36WCAezu*yxa3dG~tqt(oI~PvIGt@V)bYQ=b1E1^NOQrB9r++oPJ!?%KKP z2v}7sU89%&r@jaLnh*$>kopdXil~IE%zln0n|P3T`T^J`Ej z&^BjBe(T)bFe%`!?=%shh)OH^P9jgu;#O z^lCXIZiLeJ)lO}HOVs5{c>ufyq_e3#?9;CsZgqMEhA4J^5kG%U{MX2;RU=YwqUdGc znbHCSc+vB^(`L`Vxsvm1yeE)pxfM~?H~02|5C0Y!)Ht{zu6B;~|9|!Wv1J;seG%wR zcg5|OETqbcm^g`av^B6iEG-V9#=ekJJ2@ViZ=IWN3+X{zE-LpX1P&T zTpX>+=eWjvlV5a1yEZ2ecnuD#eWnwDp%8}iR5xE`u`LL}?sveH1_m1d9w1_Jj&`&S zC=8#l`uf%e;OqZQK{rbT#^s^JTr;p8A$>uP$)@V(SCO)(4P--Cfilm5zPu(^8Ri@H2ClyYtSS!s{9f z$UnzS3w*jKKXBj*3(++0K^G-MGd0mrU2UuPv6J8!uMNyZx+BOIjJGi~sN#$?& z`6|5a^$&=M2<$=i;w9*SkE?c;yh8C;MqgQ3j$TN7$GojSJ@lNa536A9gO9zlh4!lY z8*8mID_!o5#2gY>qGA2oS{fa{7ADwchhXz2g{k88UouPb3XFfr>bRN?AKq-#)qt4d z!oJVBWL64W(7|l3ZdQQ^Q-fVs1*+#q?zk*mP@4bZQsyVgGb%B5#u%fpGkW;`WJ_Bk ze!w29&g>fxIAV3Ty7y~6o7mmqHLv6xe`JjEiU|9Of}6&TZYZ{yE|z|qpDw36pBw{U za_=oK7^$u6og8sE5E8wfaF|zeZvSZ%iAdoZ-=%q4K%d&7J}yZ!dT2Pg(ekvjf;&1l zd7;v~T3x^|Qn`k)G;ejK=F4MCiRnt6vlcnkW8?!LPk)={y5W?WX0z zK8^6+`2IS~*V!*~2Txp^`Gz4HjBQRkeo=8R)On@!p5}DutKwoQoh-EwQ+3adu)ekT zWw!QZ%$mcA=Iuc8;KrIO# zqYPgiDtr&Ew7n+E2t%O42=#ljqgLx%}r$(}tm~Bd_ zO8XwvPObzR4PmCxc9=&5wsC;vamop7h#l?y<9mg0|x{*x7ZS z8X^H5g?5$b=qu5b^y@vpT{FUBm=i2e^A+SPYnd`8_+}Iw!%r?$F4MY=JLcr^+tx@Q zg@V3fEnLTRbvYcQ3alSun_g^!iuW7M^G04|_;gOmqJ{e#mWq_dvzM72^r2xq`nlNY zuOkjnlf_uNHLP}EK2LL{F4YPs?JqSuhBYG9rCDwz0WSP};c&5Cx~q%FgU*Dc!Xasl zXgdrZ!6qP5C8~U?e|eoWNnj|V@#7me;tqf|f2bzC^TAGmV@TcI1K*k}hB>*LTrUu0 zg4Gj_EK9TqmxoophDqvmOjxsU7~Ca(A%u; zBk8lvo;4fVwnc`FiPY`Z>^eoVNa32PTd7v(lC4dKuY@c6Yr4n^=$z%W3d#KluTV(O zEOOFdo3dsxML!BlW?v-DpP|!zg`8J>zUx#DO$SRP9!-=YHsj!8YKZXik0bflVZmAd zIu9BG@Uv+evIG4zQ;pmO(xBVgDZC{=q)|2K{n31jqAGPEwG6|0#g}FkWe4%t=&Ywh zu$Iq4q|SM3#r0ZbK&##y>`bc}&C>NV*Do=ld7Sm16^MNyf&5f}-xTxmsSotybY_$YdHzblp{7nLU{^}TCi+o9(bCc$6#8~Ms- zqDslpsrAs?t{hg>J5!|2GT05CIH~V{N1q3N0)OwjO^{uf))^K){bwWWtn}!DN&C`@ zA&^0OxoK!N$bhg6(B&sNEJ7~$DSMyngxNdNsX}jap$*XY-TA3>M!&4Dl(cE&+kLqR z_CJ@`o5*YBnpD3gKDw!(X8>$BW;W#VUDe80PYOSU$5)qYApD1%M=(BYHQjstoudR` zF}iPg?LR*DB2e{plm4En z^coF!VRal#%9ih9l`K-c=8@^j+awh9w)W^q+6$d(jUB~5D|2(uf1k(kebWKv{TkH< z%I-lDI>(@gB*pRom=c8w8=M7o;ukNsF5aV=kgy-tJp-2@X!2>ND3K_;kNh3#SXiwm z@);CZj&2=PU29Qv3<&y^`K@VUvjOoZFChmr;;LP-GoDsA?&w4kWD~ZU^>cQohYnq(Ed_kBCu8m|@ zRj4hoHp2wThp?Ojh2D#nEVkEHW^k0NR#^*pNp3M+l=!xJ%Yw(fKYkIb@BJs2wSmNX zg%)EzAkjlK*+P$Xt>E_l5beQF@aE>LhS>wWjY+aSKPLAGzomk(%$#)AIL;vZ2}8cJ zfYRPgo}f+f5OcDe{GYtMnh5_Q=L+LnlRo)({U9O!vkWu4*IM-E#NN>vhi)2Lf8t_0 z!@vx@^vGhT=5neXyD$9wd3W;QK8?%tW}|D+{-yrlnj$I$%2$^v=QI&R{m-Uh!7-s5 z0!wH~l9Fq-&_8DPiu&;dW3&5>tVxKsO$o_MLYK-5wCWcPP4|mFReD6{qbRr5lRO#? zz^Vh6mb?=fYcQXcirC!!dHo_o949?6+DI8T&w)>;mvf_13=2lgx)W(GM+>V#wFasRkG6#Mlmh(QX= zKrHu$W+dC{w1gAmYu4NeTF;62D39Yo#`0ttTf%p<6OZenfD4got1W0QcXYxO%F0R~ zYvQZsoBza7@aQ0fP59oM;-v*uGgtI&v)BJt18wxs0hE#MeOCIVQ-41TlI^ll!wQyK zm!u@;OM+XQR?KOCY4Qhb&Co{|*?~cV1#4dXws+aUrCWgeBgbI);P+4d3ch`o7FcZ7 zNZuR2_2mlaN)R=m9AY}IU;I7TF3r1^lV>TK(fpS18w9Gh?~hr{g>uWOf__g=aUJx= zFg*0bh5vfIr>5vXIzJs!-qCj-E(Z$Tpqx1ApT5|&?fohr_>{HsyP$xL@Vk!S9qa^B z|MACz9O%!#e?6!FfG>k{x0KO^assn=(5s~@=*XQ{29wv{BDlw2oA!v1_H z`7&L=YluH6NRVw#M;QnU{N4K${r8QGkNr0#t&>|Gr(AXBL7ezumLGs~sjjztF{7&Y z`|ZAWb%{g37`H12!MB4si(SvE8(2EUSU#CjwgxW8j0e|4KOV71zd72C>N@<-gE7{? zS|H$tg&BK1>Mkq2e`=H1Mmk;dAotFA2Gs|!!T7M5z`(%Bjr{}L|1>{gNz!Xo|C;H> znF&Je(#E8)<77?!ECxix&II@lml`s9l1iUYE^Avjytz8%IjT@TNslx3EHDJjxESG=Rp z0E?*}=g*a#{IN@O*TlElV>zm1kauec?pf7DbbdUOqs$H& z-4)C8o^4e$Q#N%5ijlhPZNv*jn+H=d`z(m2YTo=D|v)w8(5F|4!BL zQ>f>L1>^V70AGQ8O+3mONFD;q)XA{uuSqUw09@u#;DTIGa?*8 zdYyvWXv~;tDw|8(aKw8-)UvAO8GV8dsvPReXIlZ2y(AzLs#iR9&9_NH=MF3V_E3}B z(5sD)5Nkmxk-DJ`VX!Lw(&ziuhYX3Bi=a22Zj%O+8{=vR#fguTW6T~niY7RpjeMmw z4liq&wJi_^Gb=Mq!lHSBqb9>YT#<))XfOYuXV%)LR-{V1WV^0Nn8|D+H!PPZ6)c+J ztSG=M8+gBPc|e-jD|0;1w}g<3paD01XPi02rNqfsGwMiIn=iSKnT&pm0flQ#!H8(- zgZ1_P{Wy#~O~1HpQ&U1>W#z3HDV_18SE*2c3}TFY$iMS3=q%~PD#U*V&cXix!g+Mu ze<{5a|E{BZ!i);%c5mO!O67>p_TFnum{Fv3)06>BOP%|9wv=Ahzba8*lX&7|-@}wK?j0B!8V*`` z1=r@HeI8%SmAc$8wU)|E1M$~jN(xTnpV7p7*_Ajhb^m?Lxlljd+)Scu6z-?uNc-*F zi-*e#)C*qM%l6(%5J3nZXfPnu$6YEt@(41O?Pa`L_yHak#JFS8 zE<^}c`YT56z2K8DiLvp<*z%zJoHtEP{VoAA2&tsq1? zc&)5UTOA&s@en}gr^*rJZ*)Rw8(KRvw9<=^qF6{wA8gbEhVWH6hr(87?ZgWQDxY{30PY#3c&K~ zx;n6q&BS!+=dhVAq;$MODQ<_2Ev#oPECgRyQ2N9*{vL)d90Fqw3pi>n28zz)88kZE zZevbUO{{&x@9EN%+kA8>ir~BTB$14vklG0oQ=eaZe|Q>+!`(|3+L2M0u;0xDsY5j@ zR#4-fQ+~tIn8)NbnE;N&Ba$m@W+=fCD-W&8G2i>_Jl`Q*^D0byIhayM_)G(SmY?ZO z$arX`MpMhJU}z*7u)8vdM(uG{&=l^Y$$MpOcdqfz%)|C|YmEY0j-cevAdcf>D;Y_k zDx?|~$h2>dCZpA5+y_KZTbN;Wmf_Kysqk2w=%ttGf`IYrvTFqb2r=|?60N>?yGdjQ z6HtH-Y`=uAntO*4qoG4|pTL`5!7}S}Tm=B)iYqe2W$Jr}pbJ#A-z$|@|@?9>~qaK9kH#>=2qhHK2WQy}*Q2a1mO0ZIA>>b?f92xuu@igw*0yH_Jdi#g_ zdc8w!laM`q;Ro|-rhJ$Bmd_ec_sA^?J+a`<(R~&<$#xj#0eS`q_Q;=}Y`_&+SQqGv zWeDOSEGGsn(7F)&C+lJ@Oh_nde;{SqW1yGDfvN7|W0oZngKQ?Oumiy8DI{EwpymgZ zu;)m@a}5MUDPXgHxnS&^Vql4htk0ll4wAjZy?&ELd1%Mw9AQpw360Nz!5h87m+o??rap1!Op=`vT7WLSfYK zm4EC^FX{E;R==hE^oP`D5cXhLhtYMHe!ZI!?5KF?(vMfwoe16J z*6WrAcQ-(KrQbGGnKlnaEAF6nPsHQu z`=pI@;pF*J5iGaM#Q+_BtI_W73F}`odzv(s*AxWsA{pW&jmS)!G(iBoS?6`M(voJ= zT>GQ;xoIWp7L(RS45U2UfgTW1s~aW~4pZOo#)Oh&0T|ojL3HU6>i)z>KB6nVw1j1^ z^)~<75nGgzsqq8bi4ka~@1_`_2TgpvUf$OzZ+O)*J?scSW#aop*3)YBrd33Ivx0A; z!{^Ub@t7khVTeN5^hAsKPuSCGa?BKAD1f?q6xIEMNCU(px^AgUxsOCN2GFyti&lSB zV*pOSxiHNhjksAmot%uBNtI6hPQc?r^*NN^V$_P$+|@snXaowi&2#L&o35Ji^;X@; zg?=>>dIgl0Is6EerMP;OU9U*2a0!yLX*#k^D6`2gIp^a}D{=q4sMQ*u$g}y-3wSB- z$Aw}RJBVvovo+9@4~ZIe*N+G^x<{GM$&GZh-_(DvFW~HG*p;1UC=(H@k+F4b)2YXD zt#VFU(n`9rW|*ERpT0w-&d_{pnY5yS5bQ@#_<*?Wu?5`t-WE_dV1nD(%Ldtq6h8y= z#3-9|J92de5FB-}?qjMrj`R)olj9BN0pAp|83Ep=VYT$o!j)?T#Ji5(dlarj@0k2x zR$vgzzlXT!i=hScb_KWr8fL=yVnyf90A;98grpZf_xT8yL$sXPY6tuEKgdHtzfS9O z%Sk<0KEU-njuvo0{8A;?aUiYE>xvh*BI=Qy!dwzny|Y}~jU(?<=nd%C!EchZc0t@N zfiz7-I%pN2x}@j>=>-}zDHPM7@>ccu9+66^(>wQ@40_>5^=Shy9J9vX+QqIq7}4td z&cc_QAU6*#=zUvIvz4~*F6nhHQM>gRwz$z{_$(h2rbaRj{zo$;Ti4f39XyLIhyNvm zlCec~N&(Ijf6SXDQl1i8a6Yy3;hUFe!tK7h2B|c;pHRT*wCq2OWSH;Z8o2$A6X#Kg z0i)M*ikI>5Q)J++D0;_wl!Idfm5g1aK>v*~OgTq}*PCo%bP2*~s;-=#vI}V?09IWX zIJ?Ho-T3<*)VB|g3mJR-OdLS1H#+ekw;?7%zaY$x>gUJTAscS-4Q zrYC}8;tH!<$uG8Mk7D1s40zikHHN$xi!2O?7G7M|c}r&?r`7Z|ZN&C~kTP4f`?m_? z3?Od9&zdduh)+8(!7Z)AyU+*^T$U{%EGt39_O+UVYBkw#MmEzB zn-Y=7oPAM|b=HU{zUYyl^K<`{ z0|f0_T1#|qm7R6%K-Z;iXc4OsX6PmOQbR^HkNrEqhf+439F=JFwd3#lEF`TYzlQ}} zTGjXy-7T(0IMLnlUdQEyfZ7;zV2Y{R?}YnPh%-s&k36U9hVh3R19E}In(EB#Z~f(u zGA+o=p2x(Tx0(|M7fzEG;ZTpkWUHv?{Ui{mKNC3OJIks@px&Yi-uhXY(xiswf)}Wn z#`R3pQMJWD|K-~27FJSDSnp+<0j_tPC<@%TDI@J+%lHhwyUTbkCc^z3K3icF?%tJBv*+P0 zj{gZl?L~=Hbf}CyO+)O1Or9PUeRJuJa88(7@x!F)D?LN9CfzDKDTEjs^YYCW7+vC8 z;_@d5)t$HT2GPGFS<9yisNRpTy3!=Fa3vV9_skA1K@#~NZuPu1P~6J9)Hs1psn)13 zGI|=Xtg@u&ZmuUo_Z-*CzKGSL(in0ka2pELq(4Novi)T(-N6TP{8>`j4g+IC}vu$nB zqN7#&0r}}Mr>ArKZ%`kOE0xu*9~sBFY-^qDeIYiOMF*4qWKYXx$>8LZG+)1+MRy z5Kz=DDZ*Shdh2)_u(%H?GM>}Bv=pa(uJh32x%y}sVQsM@>^1;mrHFPGq;j)PBjS9| z=aTMd(5tQZwXz#hgprP5Nqe=b+VxTi$Am-+`bAt2)C9Ep9NS*zP;bI)B*ks-?@a@u z?8s!eMpdj;_A^kXDl6whIXQrRbo~bxKRBQn(Hq8WH|A~N>MK;bAUoojZdIP+Bbl6n zPZSPj@zxVUq%evr5UmpDA5PJHcP?YvhkP?=$9|QzS~dw!eNvD;@;J8EjvAU33)q5T7xcXBI>LCy-?Y){Zs1>?=8S3D4t9NyM z-H*6YYax5VF8OFF)v{R#^I#?bTI7&iZ$DY1 zcmH~HCoOxJRWcjc!XV6^tKCAjdldxrdtNgQti*(~Xu`eEz@nPJNtf`j;UltKmGRHnIc+m&ptFEEkrQ1=)ng{{Vw1IwV>9osTn-@;2qpofy`?2f zZ?0iuV`B~3p4~jvRPr_)t+>Ivh9+({N_cK2h{Y;CnSM6vy_l#wvYNZLK3S`7Sv37d z&Kj=}<6@kiWZ&4>Xkj{#AyrYLMdqI#9kr};y1lC59nG(0_&QrPOYN>yR}eL=u&{84 zX?e6XK@t5ca;#<|hXb|EJ9EWh%BI@a%FuikJHW<`z-@xQx6;ZaBm?`LG~@$7cXxMp z+%6H1rkrf z=IYs?SX#b<6@6l4$*$&9<3`OwCmbgrR5KkzpbF+T=r*es%0k&WI_l!3W9SSxCg?dQ=DarXY{nQrO8N_Jx{ z6nyvBpCh-lNUvWBa#OPzez`p{xCv}blxocc%AXi(co%8|%MXS<9tzT;(3=keOTu~D zM)&2vkHjaxWKxa&Bp}I%Z%G&Q?XNH?OHM!IL1V0#3S{_hd#ryO939Ooof#i5>>oLx z(J1*Kt^T}ieoJr+w7a`IGB(y@Gd(>`Fr>|`akRI`l=z-$9Rl38kT~WAK<*aNvL?WW z125P>68e71v8d7|ELd^=L$tKG*N-e`2b&6VCrf}`n(y5m@$I4G-9M%DxkH@4fBJKS z`mzwvVi7065U-3DytMPUyEGbyK50q`^emDng$1@ZQ^-U>n-n@+QAGcBa{2Gr`|(67 zU`dn+3#YURh?8r7M3^T}{ccv&8T`K_Be6m`O=;b8C buildDeleteRequest( @@ -119,6 +123,9 @@ public void testMultiObjectDeleteNoPermissions() throws Throwable { Path csvPath = maybeGetCsvPath(); S3AFileSystem fs = (S3AFileSystem) csvPath.getFileSystem( getConfiguration()); + // create a span, expect it to be activated. + fs.getAuditSpanSource().createSpan(StoreStatisticNames.OP_DELETE, + csvPath.toString(), null); List keys = buildDeleteRequest( new String[]{ @@ -163,7 +170,9 @@ public void testMultiObjectDeleteMissingEntriesSucceeds() throws Throwable { S3AFileSystem fs = getFileSystem(); List keys = keysToDelete( Lists.newArrayList(new Path(base, "1"), new Path(base, "2"))); - fs.removeKeys(keys, false, null); + try (AuditSpan span = span()) { + fs.removeKeys(keys, false, null); + } } private String join(final Iterable iterable) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java index e6ebfba922d5f..496226caf3da1 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java @@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.fs.store.EtagChecksum; import org.apache.hadoop.test.LambdaTestUtils; @@ -111,16 +112,18 @@ public void testCreateNonRecursiveParentIsFile() throws IOException { @Test public void testPutObjectDirect() throws Throwable { final S3AFileSystem fs = getFileSystem(); - ObjectMetadata metadata = fs.newObjectMetadata(-1); - metadata.setContentLength(-1); - Path path = path("putDirect"); - final PutObjectRequest put = new PutObjectRequest(fs.getBucket(), - path.toUri().getPath(), - new ByteArrayInputStream("PUT".getBytes()), - metadata); - LambdaTestUtils.intercept(IllegalStateException.class, - () -> fs.putObjectDirect(put)); - assertPathDoesNotExist("put object was created", path); + try (AuditSpan span = span()) { + ObjectMetadata metadata = fs.newObjectMetadata(-1); + metadata.setContentLength(-1); + Path path = path("putDirect"); + final PutObjectRequest put = new PutObjectRequest(fs.getBucket(), + path.toUri().getPath(), + new ByteArrayInputStream("PUT".getBytes()), + metadata); + LambdaTestUtils.intercept(IllegalStateException.class, + () -> fs.putObjectDirect(put)); + assertPathDoesNotExist("put object was created", path); + } } private FSDataOutputStream createNonRecursive(Path path) throws IOException { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java index 4746ad5588bd0..818d2fc889c17 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java @@ -21,6 +21,8 @@ import com.amazonaws.services.s3.model.MultipartUpload; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.store.audit.AuditSpan; + import org.junit.Test; import java.io.IOException; @@ -57,7 +59,7 @@ protected Configuration createConfiguration() { public void testListMultipartUploads() throws Exception { S3AFileSystem fs = getFileSystem(); Set keySet = new HashSet<>(); - try { + try (AuditSpan span = span()) { // 1. Create NUM_KEYS pending upload parts for (int i = 0; i < NUM_KEYS; i++) { Path filePath = getPartFilename(i); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java index adcf578b05862..770a99599f508 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java @@ -1352,7 +1352,7 @@ private void skipIfVersionPolicyAndNoVersionId(Path testpath) // enabled Assume.assumeTrue( "Target filesystem does not support versioning", - fs.getObjectMetadata(fs.pathToKey(testpath)).getVersionId() != null); + fs.getObjectMetadata(testpath).getVersionId() != null); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java index bd69ef28669bb..d603d38432285 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java @@ -20,14 +20,12 @@ import java.io.FileNotFoundException; import java.io.IOException; -import java.io.InputStream; import java.util.stream.Stream; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.ListObjectsV2Request; import com.amazonaws.services.s3.model.ListObjectsV2Result; import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.S3ObjectSummary; import org.assertj.core.api.Assertions; import org.junit.Test; @@ -40,9 +38,11 @@ import org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore; +import org.apache.hadoop.fs.store.audit.AuditSpan; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_MKDIRS; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeFilesystemHasMetadatastore; @@ -73,7 +73,9 @@ public void testRenameEmptyDir() throws Throwable { String destDirMarker = fs.pathToKey(destDir) + "/"; // set things up. mkdirs(sourceDir); - // there's source directory marker + // create a span for all the low level operations + span(); + // there's source directory marker+ fs.getObjectMetadata(sourceDirMarker); S3AFileStatus srcStatus = getEmptyDirStatus(sourceDir); assertEquals("Must be an empty dir: " + srcStatus, Tristate.TRUE, @@ -89,6 +91,7 @@ public void testRenameEmptyDir() throws Throwable { // and verify that there's no dir marker hidden under a tombstone intercept(FileNotFoundException.class, () -> Invoker.once("HEAD", sourceDirMarker, () -> { + span(); ObjectMetadata md = fs.getObjectMetadata(sourceDirMarker); return String.format("Object %s of length %d", sourceDirMarker, md.getInstanceLength()); @@ -99,11 +102,14 @@ public void testRenameEmptyDir() throws Throwable { assertEquals("Must not be an empty dir: " + baseStatus, Tristate.FALSE, baseStatus.isEmptyDirectory()); // and verify the dest dir has a marker + span(); fs.getObjectMetadata(destDirMarker); } private S3AFileStatus getEmptyDirStatus(Path dir) throws IOException { - return getFileSystem().innerGetFileStatus(dir, true, StatusProbeEnum.ALL); + try (AuditSpan span = span()) { + return getFileSystem().innerGetFileStatus(dir, true, StatusProbeEnum.ALL); + } } @Test @@ -128,7 +134,7 @@ public void testEmptyDirs() throws Exception { fs.setMetadataStore(configuredMs); // "start cluster" Path newFile = path("existing-dir/new-file"); touch(fs, newFile); - + span(); S3AFileStatus status = fs.innerGetFileStatus(existingDir, true, StatusProbeEnum.ALL); assertEquals("Should not be empty dir", Tristate.FALSE, @@ -137,6 +143,7 @@ public void testEmptyDirs() throws Exception { // 3. Assert that removing the only file the MetadataStore witnessed // being created doesn't cause it to think the directory is now empty. fs.delete(newFile, false); + span(); status = fs.innerGetFileStatus(existingDir, true, StatusProbeEnum.ALL); assertEquals("Should not be empty dir", Tristate.FALSE, status.isEmptyDirectory()); @@ -144,6 +151,7 @@ public void testEmptyDirs() throws Exception { // 4. Assert that removing the final file, that existed "before" // MetadataStore started, *does* cause the directory to be marked empty. fs.delete(existingFile, false); + span(); status = fs.innerGetFileStatus(existingDir, true, StatusProbeEnum.ALL); assertEquals("Should be empty dir now", Tristate.TRUE, status.isEmptyDirectory()); @@ -198,11 +206,9 @@ public void testTombstonesAndEmptyDirectories() throws Throwable { createEmptyObject(fs, childKey); // Do a list - ListObjectsV2Request listReq = new ListObjectsV2Request() - .withBucketName(bucket) - .withPrefix(baseKey) - .withMaxKeys(10) - .withDelimiter("/"); + span(); + ListObjectsV2Request listReq = ctx.getRequestFactory() + .newListObjectsV2Request(baseKey, "/", 10); ListObjectsV2Result listing = s3.listObjectsV2(listReq); // the listing has the first path as a prefix, because of the child @@ -243,6 +249,7 @@ public void testTombstonesAndEmptyDirectories() throws Throwable { } finally { // try to recover from the defective state. + span(); s3.deleteObject(bucket, childKey); fs.delete(lastPath, true); ddbMs.forgetMetadata(firstPath); @@ -272,19 +279,13 @@ private DynamoDBMetadataStore getRequiredDDBMetastore(S3AFileSystem fs) { * @param fs filesystem * @param key key */ - private void createEmptyObject(S3AFileSystem fs, String key) { - final InputStream im = new InputStream() { - @Override - public int read() { - return -1; - } - }; + private void createEmptyObject(S3AFileSystem fs, String key) + throws IOException { - PutObjectRequest putObjectRequest = fs.newPutObjectRequest(key, - fs.newObjectMetadata(0L), - im); - AmazonS3 s3 = fs.getAmazonS3ClientForTesting("PUT"); - s3.putObject(putObjectRequest); + try (AuditSpan span = fs.getAuditSpanSource() + .createSpan(INVOCATION_MKDIRS.getSymbol(), key, null)) { + fs.createMkdirOperationCallbacks().createFakeDirectory(key); + } } @Test diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java index e2915884cefa3..624bfd6c94501 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java @@ -22,9 +22,8 @@ import java.net.URI; import com.amazonaws.AmazonClientException; +import com.amazonaws.AmazonWebServiceRequest; import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; -import com.amazonaws.services.s3.model.UploadPartRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,14 +36,18 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditTestSupport; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase; +import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.util.Progressable; +import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.noopAuditor; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTrackerFactory; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; @@ -81,6 +84,16 @@ public class MockS3AFileSystem extends S3AFileSystem { private final Path root; + /** + * This is a request factory whose preparation is a no-op. + */ + public static final RequestFactory REQUEST_FACTORY = + RequestFactoryImpl.builder() + .withRequestPreparer(MockS3AFileSystem::prepareRequest) + .withBucket(BUCKET) + .withEncryptionSecrets(new EncryptionSecrets()) + .build(); + /** * This can be edited to set the log level of events through the * mock FS. @@ -99,6 +112,15 @@ public MockS3AFileSystem(S3AFileSystem mock, root = new Path(FS_URI.toString()); } + private static T prepareRequest(T t) { + return t; + } + + @Override + public RequestFactory getRequestFactory() { + return REQUEST_FACTORY; + } + public Pair getOutcome() { return outcome; @@ -147,8 +169,11 @@ public Path qualify(final Path path) { public void initialize(URI name, Configuration originalConf) throws IOException { conf = originalConf; - writeHelper = new WriteOperationHelper(this, conf, - new EmptyS3AStatisticsContext()); + writeHelper = new WriteOperationHelper(this, + conf, + new EmptyS3AStatisticsContext(), + noopAuditor(conf), + AuditTestSupport.NOOP_SPAN); } @Override @@ -305,17 +330,6 @@ public void incrementPutCompletedStatistics(boolean success, long bytes) { public void incrementPutProgressStatistics(String key, long bytes) { } - @Override - protected void setOptionalMultipartUploadRequestParameters( - InitiateMultipartUploadRequest req) { -// no-op - } - - @Override - protected void setOptionalUploadPartRequestParameters( - UploadPartRequest request) { - } - @Override @SuppressWarnings("deprecation") public long getDefaultBlockSize() { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java index 861824277aca9..04c2b2a09bda2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java @@ -22,6 +22,9 @@ import com.amazonaws.services.s3.model.PartETag; import com.amazonaws.services.s3.model.UploadPartRequest; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.store.audit.AuditSpan; +import org.apache.hadoop.io.IOUtils; + import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,7 +57,9 @@ private MultipartTestUtils() { } static void cleanupParts(S3AFileSystem fs, Set keySet) { boolean anyFailure = false; for (IdKey ik : keySet) { - try { + try (AuditSpan span = + fs.createSpan("multipart", ik.key, null)) { + LOG.debug("aborting upload id {}", ik.getUploadId()); fs.abortMultipartUpload(ik.getKey(), ik.getUploadId()); } catch (Exception e) { @@ -69,31 +74,39 @@ static void cleanupParts(S3AFileSystem fs, Set keySet) { public static IdKey createPartUpload(S3AFileSystem fs, String key, int len, int partNo) throws IOException { - WriteOperationHelper writeHelper = fs.getWriteOperationHelper(); - byte[] data = dataset(len, 'a', 'z'); - InputStream in = new ByteArrayInputStream(data); - String uploadId = writeHelper.initiateMultiPartUpload(key); - UploadPartRequest req = writeHelper.newUploadPartRequest(key, uploadId, - partNo, len, in, null, 0L); - PartETag partEtag = fs.uploadPart(req).getPartETag(); - LOG.debug("uploaded part etag {}, upid {}", partEtag.getETag(), uploadId); - return new IdKey(key, uploadId); + try (AuditSpan span = fs.createSpan("multipart", key, null)) { + WriteOperationHelper writeHelper = fs.getWriteOperationHelper(); + byte[] data = dataset(len, 'a', 'z'); + InputStream in = new ByteArrayInputStream(data); + String uploadId = writeHelper.initiateMultiPartUpload(key); + UploadPartRequest req = writeHelper.newUploadPartRequest(key, uploadId, + partNo, len, in, null, 0L); + PartETag partEtag = writeHelper.uploadPart(req).getPartETag(); + LOG.debug("uploaded part etag {}, upid {}", partEtag.getETag(), uploadId); + return new IdKey(key, uploadId); + } } /** Delete any uploads under given path (recursive). Silent on failure. */ public static void clearAnyUploads(S3AFileSystem fs, Path path) { + String key = fs.pathToKey(path); + AuditSpan span = null; try { - String key = fs.pathToKey(path); MultipartUtils.UploadIterator uploads = fs.listUploads(key); + span = fs.createSpan("multipart", path.toString(), null); + final WriteOperationHelper helper + = fs.getWriteOperationHelper(); while (uploads.hasNext()) { MultipartUpload upload = uploads.next(); - fs.getWriteOperationHelper().abortMultipartUpload(upload.getKey(), - upload.getUploadId(), true, LOG_EVENT); LOG.debug("Cleaning up upload: {} {}", upload.getKey(), truncatedUploadId(upload.getUploadId())); + helper.abortMultipartUpload(upload.getKey(), + upload.getUploadId(), true, LOG_EVENT); } } catch (IOException ioe) { LOG.info("Ignoring exception: ", ioe); + } finally { + IOUtils.closeStream(span); } } @@ -131,13 +144,15 @@ public static int countUploadsAt(S3AFileSystem fs, Path path) throws public static List listMultipartUploads(S3AFileSystem fs, String prefix) throws IOException { - return fs - .listMultipartUploads(prefix).stream() - .map(upload -> String.format("Upload to %s with ID %s; initiated %s", - upload.getKey(), - upload.getUploadId(), - S3ATestUtils.LISTING_FORMAT.format(upload.getInitiated()))) - .collect(Collectors.toList()); + try (AuditSpan span = fs.createSpan("multipart", prefix, null)) { + return fs + .listMultipartUploads(prefix).stream() + .map(upload -> String.format("Upload to %s with ID %s; initiated %s", + upload.getKey(), + upload.getUploadId(), + S3ATestUtils.LISTING_FORMAT.format(upload.getInitiated()))) + .collect(Collectors.toList()); + } } @@ -146,7 +161,7 @@ private static String truncatedUploadId(String fullId) { } /** Struct of object key, upload ID. */ - static class IdKey { + public static class IdKey { private String key; private String uploadId; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index 599f18cd9ad62..6a920972bab78 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -67,7 +67,6 @@ import java.io.Closeable; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; @@ -91,7 +90,6 @@ import static org.apache.hadoop.fs.s3a.S3ATestConstants.*; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3AUtils.propagateBucketOptions; -import static org.apache.hadoop.test.LambdaTestUtils.eventually; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.junit.Assert.*; @@ -853,21 +851,6 @@ public static void callQuietly(final Logger log, } } - /** - * Call a void operation; any exception raised is logged at info. - * This is for test teardowns. - * @param log log to use. - * @param operation operation to invoke - */ - public static void callQuietly(final Logger log, - final Invoker.VoidOperation operation) { - try { - operation.execute(); - } catch (Exception e) { - log.info(e.toString(), e); - } - } - /** * Deploy a hadoop service: init and start it. * @param conf configuration to use @@ -1449,35 +1432,6 @@ public static void checkListingContainsPath(S3AFileSystem fs, Path filePath) listStatusHasIt); } - /** - * Wait for a deleted file to no longer be visible. - * @param fs filesystem - * @param testFilePath path to query - * @throws Exception failure - */ - public static void awaitDeletedFileDisappearance(final S3AFileSystem fs, - final Path testFilePath) throws Exception { - eventually( - STABILIZATION_TIME, PROBE_INTERVAL_MILLIS, - () -> intercept(FileNotFoundException.class, - () -> fs.getFileStatus(testFilePath))); - } - - /** - * Wait for a file to be visible. - * @param fs filesystem - * @param testFilePath path to query - * @return the file status. - * @throws Exception failure - */ - public static S3AFileStatus awaitFileStatus(S3AFileSystem fs, - final Path testFilePath) - throws Exception { - return (S3AFileStatus) eventually( - STABILIZATION_TIME, PROBE_INTERVAL_MILLIS, - () -> fs.getFileStatus(testFilePath)); - } - /** * This creates a set containing all current threads and some well-known * thread names whose existence should not fail test runs. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java index de27411a41ab8..9ae24c19f3da5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java @@ -20,6 +20,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.audit.AuditTestSupport; import org.apache.hadoop.fs.s3a.commit.PutTracker; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; import org.apache.hadoop.util.Progressable; @@ -30,6 +31,7 @@ import java.io.IOException; import java.util.concurrent.ExecutorService; +import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.noopAuditor; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -86,9 +88,14 @@ public void testFlushNoOpWhenStreamClosed() throws Exception { public void testWriteOperationHelperPartLimits() throws Throwable { S3AFileSystem s3a = mock(S3AFileSystem.class); when(s3a.getBucket()).thenReturn("bucket"); + when(s3a.getRequestFactory()) + .thenReturn(MockS3AFileSystem.REQUEST_FACTORY); + final Configuration conf = new Configuration(); WriteOperationHelper woh = new WriteOperationHelper(s3a, - new Configuration(), - new EmptyS3AStatisticsContext()); + conf, + new EmptyS3AStatisticsContext(), + noopAuditor(conf), + AuditTestSupport.NOOP_SPAN); ByteArrayInputStream inputStream = new ByteArrayInputStream( "a".getBytes()); // first one works diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java new file mode 100644 index 0000000000000..c76e3fa968f92 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java @@ -0,0 +1,213 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import java.io.IOException; +import java.util.Map; + +import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import org.junit.After; +import org.junit.Before; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl; +import org.apache.hadoop.fs.statistics.IOStatisticAssertions; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.fs.store.audit.AuditSpan; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_GET_FILE_STATUS; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.UNAUDITED_OPERATION; +import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.createIOStatisticsStoreForAuditing; +import static org.apache.hadoop.service.ServiceOperations.stopQuietly; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Abstract class for auditor unit tests. + */ +public abstract class AbstractAuditingTest extends AbstractHadoopTestBase { + + protected static final String OPERATION + = INVOCATION_GET_FILE_STATUS.getSymbol(); + + /** + * Logging. + */ + private static final Logger LOG = + LoggerFactory.getLogger(AbstractAuditingTest.class); + + public static final String PATH_1 = "/path1"; + + public static final String PATH_2 = "/path2"; + + /** + * Statistics store with the auditor counters wired up. + */ + private final IOStatisticsStore ioStatistics = + createIOStatisticsStoreForAuditing(); + + private RequestFactory requestFactory; + + private AuditManagerS3A manager; + + @Before + public void setup() throws Exception { + requestFactory = RequestFactoryImpl.builder() + .withBucket("bucket") + .build(); + manager = AuditIntegration.createAndStartAuditManager( + createConfig(), + ioStatistics); + } + + /** + * Create config. + * @return config to use when creating a manager + */ + protected abstract Configuration createConfig(); + + @After + public void teardown() { + stopQuietly(manager); + } + + protected IOStatisticsStore getIOStatistics() { + return ioStatistics; + } + + protected RequestFactory getRequestFactory() { + return requestFactory; + } + + protected AuditManagerS3A getManager() { + return manager; + } + + /** + * Assert that a specific span is active. + * This matches on the wrapped spans. + * @param span span to assert over. + */ + protected void assertActiveSpan(final AuditSpan span) { + assertThat(activeSpan()) + .isSameAs(span); + } + + /** + * Assert a span is unbound/invalid. + * @param span span to assert over. + */ + protected void assertUnbondedSpan(final AuditSpan span) { + assertThat(span.isValidSpan()) + .describedAs("Validity of %s", span) + .isFalse(); + } + + protected AuditSpanS3A activeSpan() { + return manager.getActiveAuditSpan(); + } + + /** + * Create a head request and pass it through the manager's beforeExecution() + * callback. + * @return a processed request. + */ + protected GetObjectMetadataRequest head() { + return manager.beforeExecution( + requestFactory.newGetObjectMetadataRequest("/")); + } + + /** + * Assert a head request fails as there is no + * active span. + */ + protected void assertHeadUnaudited() throws Exception { + intercept(AuditFailureException.class, + UNAUDITED_OPERATION, this::head); + } + + /** + * Assert that the audit failure is of a given value. + * Returns the value to assist in chaining, + * @param expected expected value + * @return the expected value. + */ + protected long verifyAuditFailureCount( + final long expected) { + return verifyCounter(Statistic.AUDIT_FAILURE, expected); + } + + /** + * Assert that the audit execution count + * is of a given value. + * Returns the value to assist in chaining, + * @param expected expected value + * @return the expected value. + */ + protected long verifyAuditExecutionCount( + final long expected) { + return verifyCounter(Statistic.AUDIT_REQUEST_EXECUTION, expected); + } + + /** + * Assert that a statistic counter is of a given value. + * Returns the value to assist in chaining, + * @param statistic statistic to check + * @param expected expected value + * @return the expected value. + */ + protected long verifyCounter(final Statistic statistic, + final long expected) { + IOStatisticAssertions.assertThatStatisticCounter( + ioStatistics, + statistic.getSymbol()) + .isEqualTo(expected); + return expected; + } + + /** + * Create and switch to a span. + * @return a span + */ + protected AuditSpanS3A span() throws IOException { + AuditSpanS3A span = manager.createSpan(OPERATION, PATH_1, PATH_2); + assertThat(span) + .matches(AuditSpan::isValidSpan); + return span; + } + + /** + * Assert the map contains the expected (key, value). + * @param params map of params + * @param key key + * @param expected expected value. + */ + protected void assertMapContains(final Map params, + final String key, final String expected) { + assertThat(params.get(key)) + .describedAs(key) + .isEqualTo(expected); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AccessCheckingAuditor.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AccessCheckingAuditor.java new file mode 100644 index 0000000000000..e8505016c0b31 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AccessCheckingAuditor.java @@ -0,0 +1,53 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import java.io.IOException; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor; + +/** + * Noop auditor which lets access checks be enabled/disabled. + */ +public class AccessCheckingAuditor extends NoopAuditor { + + public static final String CLASS = + "org.apache.hadoop.fs.s3a.audit.AccessCheckingAuditor"; + + /** Flag to enable/disable access. */ + private boolean accessAllowed = true; + + public AccessCheckingAuditor() { + } + + public void setAccessAllowed(final boolean accessAllowed) { + this.accessAllowed = accessAllowed; + } + + @Override + public boolean checkAccess(final Path path, + final S3AFileStatus status, + final FsAction mode) + throws IOException { + return accessAllowed; + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java new file mode 100644 index 0000000000000..9519c394ce23d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java @@ -0,0 +1,123 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditManagerS3A; +import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.fs.store.audit.AuditSpan; + +import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_ACCESS_CHECK_FAILURE; +import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_FAILURE; +import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_REQUEST_EXECUTION; +import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_SPAN_CREATION; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_SERVICE_CLASSNAME; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.LOGGING_AUDIT_SERVICE; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.NOOP_AUDIT_SERVICE; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_ENABLED; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REJECT_OUT_OF_SPAN_OPERATIONS; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; + +/** + * Support for auditing in testing. + */ +public final class AuditTestSupport { + + private AuditTestSupport() { + } + + /** + * Reusable no-op span instance. + */ + public static final AuditSpan NOOP_SPAN = NoopAuditManagerS3A + .createNewSpan("noop", null, null); + + /** + * Create, init and start a no-op auditor instance. + * @param conf configuration. + * @return a started instance. + */ + public static OperationAuditor noopAuditor(Configuration conf) { + return NoopAuditor.createAndStartNoopAuditor(conf, null); + } + + /** + * Create config for no-op auditor. + * @return config with nothing but the no-op audit service set up. + */ + public static Configuration noopAuditConfig() { + final Configuration conf = new Configuration(false); + conf.set( + AUDIT_SERVICE_CLASSNAME, NOOP_AUDIT_SERVICE); + return conf; + } + + /** + * Create config for logging auditor which + * rejects out of span operations. + * @return config + */ + public static Configuration loggingAuditConfig() { + return enableLoggingAuditor(new Configuration(false)); + } + + /** + * Patch the configuration to support the logging auditor and + * rejects out of span operations. + * @param conf config to patch. + * @return the config + */ + public static Configuration enableLoggingAuditor(final Configuration conf) { + conf.set(AUDIT_SERVICE_CLASSNAME, LOGGING_AUDIT_SERVICE); + conf.setBoolean(REJECT_OUT_OF_SPAN_OPERATIONS, true); + return conf; + } + + /** + * Create IOStatistics store with the auditor counters wired up. + * @return an IOStatistics store to pass to audit managers. + */ + public static IOStatisticsStore createIOStatisticsStoreForAuditing() { + return iostatisticsStore() + .withCounters( + AUDIT_ACCESS_CHECK_FAILURE.getSymbol(), + AUDIT_FAILURE.getSymbol(), + AUDIT_REQUEST_EXECUTION.getSymbol(), + AUDIT_SPAN_CREATION.getSymbol()) + .build(); + } + + /** + * Remove all overridden values for + * the test bucket/global in the given config. + * @param conf configuration to patch + * @return the configuration. + */ + public static Configuration resetAuditOptions(Configuration conf) { + S3ATestUtils.removeBaseAndBucketOverrides(conf, + REFERRER_HEADER_ENABLED, + REJECT_OUT_OF_SPAN_OPERATIONS, + AUDIT_REQUEST_HANDLERS, + AUDIT_SERVICE_CLASSNAME); + return conf; + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditAccessChecks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditAccessChecks.java new file mode 100644 index 0000000000000..bd145fa9c7403 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditAccessChecks.java @@ -0,0 +1,201 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import java.io.FileNotFoundException; +import java.io.IOException; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; +import org.apache.hadoop.security.AccessControlException; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_ACCESS_CHECK_FAILURE; +import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_REQUEST_EXECUTION; +import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_ACCESS; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_REQUEST; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_SERVICE_CLASSNAME; +import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.resetAuditOptions; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.FILE_STATUS_ALL_PROBES; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.FILE_STATUS_FILE_PROBE; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.ROOT_FILE_STATUS_PROBE; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; + +/** + * Test S3A FS Access permit/deny is passed through all the way to the + * auditor. + * Uses {@link AccessCheckingAuditor} to enable/disable access. + * There are not currently any contract tests for this; behaviour + * based on base FileSystem implementation. + */ +public class ITestAuditAccessChecks extends AbstractS3ACostTest { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestAuditAccessChecks.class); + + private AccessCheckingAuditor auditor; + + public ITestAuditAccessChecks() { + super(true); + } + + @Override + public Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + resetAuditOptions(conf); + conf.set(AUDIT_SERVICE_CLASSNAME, AccessCheckingAuditor.CLASS); + return conf; + } + + @Override + public void setup() throws Exception { + super.setup(); + auditor = (AccessCheckingAuditor) getFileSystem().getAuditor(); + } + + @Test + public void testFileAccessAllowed() throws Throwable { + describe("Enable checkaccess and verify it works with expected" + + " statistics"); + auditor.setAccessAllowed(true); + Path path = methodPath(); + S3AFileSystem fs = getFileSystem(); + touch(fs, path); + verifyMetrics( + () -> access(fs, path), + with(INVOCATION_ACCESS, 1), + whenRaw(FILE_STATUS_FILE_PROBE)); + } + + @Test + public void testDirAccessAllowed() throws Throwable { + describe("Enable checkaccess and verify it works with a dir"); + auditor.setAccessAllowed(true); + Path path = methodPath(); + S3AFileSystem fs = getFileSystem(); + mkdirs(path); + verifyMetrics( + () -> access(fs, path), + with(INVOCATION_ACCESS, 1), + whenRaw(FILE_STATUS_ALL_PROBES)); + } + + @Test + public void testRootDirAccessAllowed() throws Throwable { + describe("Enable checkaccess and verify root dir access"); + auditor.setAccessAllowed(true); + Path path = new Path("/"); + S3AFileSystem fs = getFileSystem(); + mkdirs(path); + verifyMetrics( + () -> access(fs, path), + with(INVOCATION_ACCESS, 1), + whenRaw(ROOT_FILE_STATUS_PROBE)); + } + + /** + * If the test auditor blocks access() calls, then + * the audit will fall after checking to see if the file + * exists. + */ + @Test + public void testFileAccessDenied() throws Throwable { + describe("Disable checkaccess and verify it fails"); + auditor.setAccessAllowed(false); + Path path = methodPath(); + S3AFileSystem fs = getFileSystem(); + touch(fs, path); + verifyMetricsIntercepting( + AccessControlException.class, + "\"" + path + "\"", + () -> access(fs, path), + with(INVOCATION_ACCESS, 1), + with(AUDIT_ACCESS_CHECK_FAILURE, 1), + // one S3 request: a HEAD. + with(AUDIT_REQUEST_EXECUTION, 1), + whenRaw(FILE_STATUS_FILE_PROBE)); + } + + /** + * If the test auditor blocks access() calls, then + * the audit will fall after checking to see if the directory + * exists. + */ + @Test + public void testDirAccessDenied() throws Throwable { + describe("Disable checkaccess and verify it dir access denied"); + auditor.setAccessAllowed(false); + Path path = methodPath(); + S3AFileSystem fs = getFileSystem(); + mkdirs(path); + verifyMetricsIntercepting( + AccessControlException.class, + "\"" + path + "\"", + () -> access(fs, path), + with(INVOCATION_ACCESS, 1), + // two S3 requests: a HEAD and a LIST. + with(AUDIT_REQUEST_EXECUTION, 2), + with(STORE_IO_REQUEST, 2), + with(AUDIT_ACCESS_CHECK_FAILURE, 1), + whenRaw(FILE_STATUS_ALL_PROBES)); + } + + /** + * Missing path will fail with FNFE irrespective of + * the access permission. + */ + @Test + public void testMissingPathAccessFNFE() throws Throwable { + describe("access() on missing path goes to S3 and fails with FNFE"); + auditor.setAccessAllowed(false); + Path path = methodPath(); + S3AFileSystem fs = getFileSystem(); + verifyMetricsIntercepting( + FileNotFoundException.class, + path.toString(), + () -> access(fs, path), + with(INVOCATION_ACCESS, 1), + // two S3 requests: a HEAD and a LIST. + with(AUDIT_REQUEST_EXECUTION, 2), + with(AUDIT_ACCESS_CHECK_FAILURE, 0), + whenRaw(FILE_STATUS_ALL_PROBES)); + } + + /** + * Call {@link S3AFileSystem#access(Path, FsAction)}. + * @param fs filesystem + * @param path path to check access + * @return the IOStatistics + * @throws AccessControlException access denied + * @throws IOException failure, including permission failure. + */ + private String access(final S3AFileSystem fs, final Path path) + throws AccessControlException, IOException { + fs.access(path, FsAction.ALL); + return ioStatisticsToPrettyString(fs.getIOStatistics()); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java new file mode 100644 index 0000000000000..287fe51b5ea29 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java @@ -0,0 +1,135 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import java.nio.file.AccessDeniedException; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.WriteOperationHelper; +import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; +import org.apache.hadoop.fs.statistics.IOStatistics; + +import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_FAILURE; +import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_REQUEST_EXECUTION; +import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.enableLoggingAuditor; +import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.resetAuditOptions; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.UNAUDITED_OPERATION; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupCounterStatistic; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Test audit manager invocation by making assertions + * about the statistics of audit request execution + * {@link org.apache.hadoop.fs.s3a.Statistic#AUDIT_REQUEST_EXECUTION} + * and + * {@link org.apache.hadoop.fs.s3a.Statistic#AUDIT_FAILURE}. + */ +public class ITestAuditManager extends AbstractS3ACostTest { + + public ITestAuditManager() { + super(true); + } + + @Override + public Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + resetAuditOptions(conf); + enableLoggingAuditor(conf); + conf.set(AUDIT_REQUEST_HANDLERS, + SimpleAWSRequestHandler.CLASS); + return conf; + } + + /** + * Get the FS IOStatistics. + * @return the FS live IOSTats. + */ + private IOStatistics iostats() { + return getFileSystem().getIOStatistics(); + } + + /** + * Verify that operations outside a span are rejected + * by ensuring that the thread is outside a span, create + * a write operation helper, then + * reject it. + */ + @Test + public void testInvokeOutOfSpanRejected() throws Throwable { + describe("Operations against S3 will be rejected outside of a span"); + final S3AFileSystem fs = getFileSystem(); + final long failures0 = lookupCounterStatistic(iostats(), + AUDIT_FAILURE.getSymbol()); + final long exec0 = lookupCounterStatistic(iostats(), + AUDIT_REQUEST_EXECUTION.getSymbol()); + // API call + // create and close a span, so the FS is not in a span. + fs.createSpan("span", null, null).close(); + + // this will be out of span + final WriteOperationHelper writer + = fs.getWriteOperationHelper(); + + // which can be verified + Assertions.assertThat(writer.getAuditSpan()) + .matches(s -> !s.isValidSpan(), "Span is not valid"); + + // an S3 API call will fail and be mapped to access denial. + final AccessDeniedException ex = intercept( + AccessDeniedException.class, UNAUDITED_OPERATION, () -> + writer.listMultipartUploads("/")); + + // verify the type of the inner cause, throwing the outer ex + // if it is null or a different class + if (!(ex.getCause() instanceof AuditFailureException)) { + throw ex; + } + + assertThatStatisticCounter(iostats(), AUDIT_REQUEST_EXECUTION.getSymbol()) + .isGreaterThan(exec0); + assertThatStatisticCounter(iostats(), AUDIT_FAILURE.getSymbol()) + .isGreaterThan(failures0); + } + + @Test + public void testRequestHandlerBinding() throws Throwable { + describe("Verify that extra request handlers can be added and that they" + + " will be invoked during request execution"); + final long baseCount = SimpleAWSRequestHandler.getInvocationCount(); + final S3AFileSystem fs = getFileSystem(); + final long exec0 = lookupCounterStatistic(iostats(), + AUDIT_REQUEST_EXECUTION.getSymbol()); + // API call + fs.getBucketLocation(); + // which MUST have ended up calling the extension request handler + Assertions.assertThat(SimpleAWSRequestHandler.getInvocationCount()) + .describedAs("Invocation count of plugged in request handler") + .isGreaterThan(baseCount); + assertThatStatisticCounter(iostats(), AUDIT_REQUEST_EXECUTION.getSymbol()) + .isGreaterThan(exec0); + assertThatStatisticCounter(iostats(), AUDIT_FAILURE.getSymbol()) + .isZero(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/SimpleAWSRequestHandler.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/SimpleAWSRequestHandler.java new file mode 100644 index 0000000000000..6f5a0445a92f7 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/SimpleAWSRequestHandler.java @@ -0,0 +1,54 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import java.util.concurrent.atomic.AtomicLong; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.handlers.RequestHandler2; + +/** + * Simple AWS handler to verify dynamic loading of extra request + * handlers during auditing setup. + * The invocation counter tracks the count of calls to + * {@link #beforeExecution(AmazonWebServiceRequest)}. + */ +public final class SimpleAWSRequestHandler extends RequestHandler2 { + + public static final String CLASS + = "org.apache.hadoop.fs.s3a.audit.SimpleAWSRequestHandler"; + + /** Count of invocations. */ + private static final AtomicLong INVOCATIONS = new AtomicLong(0); + + @Override + public AmazonWebServiceRequest beforeExecution( + final AmazonWebServiceRequest request) { + INVOCATIONS.incrementAndGet(); + return request; + } + + /** + * Get the count of invocations. + * @return a natural number. + */ + public static long getInvocationCount() { + return INVOCATIONS.get(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditIntegration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditIntegration.java new file mode 100644 index 0000000000000..bd552b91aadcb --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditIntegration.java @@ -0,0 +1,216 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import java.io.IOException; +import java.nio.file.AccessDeniedException; +import java.util.List; + +import com.amazonaws.DefaultRequest; +import com.amazonaws.handlers.RequestHandler2; +import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor; +import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.service.Service; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.s3a.S3AUtils.translateException; +import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.attachSpanToRequest; +import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.retrieveAttachedSpan; +import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.createIOStatisticsStoreForAuditing; +import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.noopAuditConfig; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_SERVICE_CLASSNAME; +import static org.apache.hadoop.service.ServiceAssert.assertServiceStateStarted; +import static org.apache.hadoop.service.ServiceAssert.assertServiceStateStopped; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Unit tests for auditing. + */ +public class TestAuditIntegration extends AbstractHadoopTestBase { + + private final IOStatisticsStore ioStatistics = + createIOStatisticsStoreForAuditing(); + + /** + * AuditFailureException is mapped to AccessDeniedException. + */ + @Test + public void testExceptionTranslation() throws Throwable { + intercept(AccessDeniedException.class, + () -> { + throw translateException("test", "/", + new AuditFailureException("should be translated")); + }); + } + + /** + * Create a no-op auditor. + */ + @Test + public void testNoOpAuditorInstantiation() throws Throwable { + OperationAuditor auditor = createAndStartNoopAuditor( + ioStatistics); + assertThat(auditor) + .describedAs("No-op auditor") + .isInstanceOf(NoopAuditor.class) + .satisfies(o -> o.isInState(Service.STATE.STARTED)); + } + + /** + * Create a no-op auditor through AuditIntegration, just as + * the audit manager does. + * @param store stats store. + * @return a started auditor + */ + private NoopAuditor createAndStartNoopAuditor( + final IOStatisticsStore store) + throws IOException { + Configuration conf = noopAuditConfig(); + OperationAuditorOptions options = + OperationAuditorOptions.builder() + .withConfiguration(conf) + .withIoStatisticsStore(store); + OperationAuditor auditor = + AuditIntegration.createAndInitAuditor(conf, + AUDIT_SERVICE_CLASSNAME, + options); + assertThat(auditor) + .describedAs("No-op auditor") + .isInstanceOf(NoopAuditor.class) + .satisfies(o -> o.isInState(Service.STATE.INITED)); + auditor.start(); + return (NoopAuditor) auditor; + } + + /** + * The auditor class has to exist. + */ + @Test + public void testCreateNonexistentAuditor() throws Throwable { + final Configuration conf = new Configuration(); + OperationAuditorOptions options = + OperationAuditorOptions.builder() + .withConfiguration(conf) + .withIoStatisticsStore(ioStatistics); + conf.set(AUDIT_SERVICE_CLASSNAME, "not.a.known.class"); + intercept(RuntimeException.class, () -> + AuditIntegration.createAndInitAuditor(conf, + AUDIT_SERVICE_CLASSNAME, + options)); + } + + /** + * The audit manager creates the auditor the config tells it to; + * this will have the same lifecycle as the manager. + */ + @Test + public void testAuditManagerLifecycle() throws Throwable { + AuditManagerS3A manager = AuditIntegration.createAndStartAuditManager( + noopAuditConfig(), + ioStatistics); + OperationAuditor auditor = manager.getAuditor(); + assertServiceStateStarted(auditor); + manager.close(); + assertServiceStateStopped(auditor); + } + + @Test + public void testSingleRequestHandler() throws Throwable { + AuditManagerS3A manager = AuditIntegration.createAndStartAuditManager( + noopAuditConfig(), + ioStatistics); + List handlers + = manager.createRequestHandlers(); + assertThat(handlers) + .hasSize(1); + RequestHandler2 handler = handlers.get(0); + RequestFactory requestFactory = RequestFactoryImpl.builder() + .withBucket("bucket") + .build(); + // test the basic pre-request sequence while avoiding + // the complexity of recreating the full sequence + // (and probably getting it wrong) + GetObjectMetadataRequest r + = requestFactory.newGetObjectMetadataRequest("/"); + DefaultRequest dr = new DefaultRequest(r, "S3"); + assertThat(handler.beforeMarshalling(r)) + .isNotNull(); + assertThat(handler.beforeExecution(r)) + .isNotNull(); + handler.beforeRequest(dr); + + } + + /** + * Register a second handler, verify it makes it to the list. + */ + @Test + public void testRequestHandlerLoading() throws Throwable { + Configuration conf = noopAuditConfig(); + conf.setClassLoader(this.getClass().getClassLoader()); + conf.set(AUDIT_REQUEST_HANDLERS, + SimpleAWSRequestHandler.CLASS); + AuditManagerS3A manager = AuditIntegration.createAndStartAuditManager( + conf, + ioStatistics); + assertThat(manager.createRequestHandlers()) + .hasSize(2) + .hasAtLeastOneElementOfType(SimpleAWSRequestHandler.class); + } + + @Test + public void testLoggingAuditorBinding() throws Throwable { + AuditManagerS3A manager = AuditIntegration.createAndStartAuditManager( + AuditTestSupport.loggingAuditConfig(), + ioStatistics); + OperationAuditor auditor = manager.getAuditor(); + assertServiceStateStarted(auditor); + manager.close(); + assertServiceStateStopped(auditor); + } + + @Test + public void testNoopAuditManager() throws Throwable { + AuditManagerS3A manager = AuditIntegration.stubAuditManager(); + assertThat(manager.createStateChangeListener()) + .describedAs("transfer state change listener") + .isNotNull(); + } + + @Test + public void testSpanAttachAndRetrieve() throws Throwable { + AuditManagerS3A manager = AuditIntegration.stubAuditManager(); + + AuditSpanS3A span = manager.createSpan("op", null, null); + GetObjectMetadataRequest request = + new GetObjectMetadataRequest("bucket", "key"); + attachSpanToRequest(request, span); + AWSAuditEventCallbacks callbacks = retrieveAttachedSpan(request); + assertThat(callbacks).isSameAs(span); + + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditSpanLifecycle.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditSpanLifecycle.java new file mode 100644 index 0000000000000..608667d9dfed8 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditSpanLifecycle.java @@ -0,0 +1,133 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import java.util.List; + +import com.amazonaws.handlers.RequestHandler2; +import org.junit.Before; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.store.audit.AuditSpan; + +import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.noopAuditConfig; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Unit tests related to span lifecycle. + */ +public class TestAuditSpanLifecycle extends AbstractAuditingTest { + + private AuditSpan resetSpan; + + @Before + public void setup() throws Exception { + super.setup(); + resetSpan = getManager().getActiveAuditSpan(); + } + + protected Configuration createConfig() { + return noopAuditConfig(); + } + + /** + * Core lifecycle (remember: the service has already been started). + */ + @Test + public void testStop() throws Throwable { + getManager().stop(); + } + + @Test + public void testCreateRequestHandlers() throws Throwable { + List handlers + = getManager().createRequestHandlers(); + assertThat(handlers).isNotEmpty(); + } + + @Test + public void testInitialSpanIsInvalid() throws Throwable { + assertThat(resetSpan) + .matches(f -> !f.isValidSpan(), "is invalid"); + } + + @Test + public void testCreateCloseSpan() throws Throwable { + AuditSpan span = getManager().createSpan("op", null, null); + assertThat(span) + .matches(AuditSpan::isValidSpan, "is valid"); + assertActiveSpan(span); + // activation when already active is no-op + span.activate(); + assertActiveSpan(span); + // close the span + span.close(); + // the original span is restored. + assertActiveSpan(resetSpan); + } + + @Test + public void testSpanActivation() throws Throwable { + // real activation switches spans in the current thead. + + AuditSpan span1 = getManager().createSpan("op1", null, null); + AuditSpan span2 = getManager().createSpan("op2", null, null); + assertActiveSpan(span2); + // switch back to span 1 + span1.activate(); + assertActiveSpan(span1); + // then to span 2 + span2.activate(); + assertActiveSpan(span2); + span2.close(); + + assertActiveSpan(resetSpan); + span1.close(); + assertActiveSpan(resetSpan); + } + + @Test + public void testSpanDeactivation() throws Throwable { + AuditSpan span1 = getManager().createSpan("op1", null, null); + AuditSpan span2 = getManager().createSpan("op2", null, null); + assertActiveSpan(span2); + + // this doesn't close as it is not active + span1.close(); + assertActiveSpan(span2); + span2.close(); + assertActiveSpan(resetSpan); + } + + @Test + public void testResetSpanCannotBeClosed() throws Throwable { + + assertThat(resetSpan) + .matches(f -> !f.isValidSpan(), "is invalid"); + // create a new span + AuditSpan span1 = getManager().createSpan("op1", null, null); + // switch to the reset span and then close it. + resetSpan.activate(); + resetSpan.close(); + assertActiveSpan(resetSpan); + span1.close(); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java new file mode 100644 index 0000000000000..822e8aebb2861 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java @@ -0,0 +1,323 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import java.net.URISyntaxException; +import java.util.Map; +import java.util.regex.Matcher; + +import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor; +import org.apache.hadoop.fs.store.audit.AuditSpan; +import org.apache.hadoop.fs.audit.CommonAuditContext; +import org.apache.hadoop.fs.store.audit.HttpReferrerAuditHeader; +import org.apache.hadoop.security.UserGroupInformation; + +import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.loggingAuditConfig; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_FILTER; +import static org.apache.hadoop.fs.s3a.audit.S3LogParser.*; +import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.HEADER_REFERRER; +import static org.apache.hadoop.fs.store.audit.HttpReferrerAuditHeader.maybeStripWrappedQuotes; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_FILESYSTEM_ID; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_ID; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_OP; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PATH; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PATH2; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PRINCIPAL; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD0; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD1; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_TIMESTAMP; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Tests for referrer audit header generation/parsing. + */ +public class TestHttpReferrerAuditHeader extends AbstractAuditingTest { + + /** + * Logging. + */ + private static final Logger LOG = + LoggerFactory.getLogger(TestHttpReferrerAuditHeader.class); + + private LoggingAuditor auditor; + + @Before + public void setup() throws Exception { + super.setup(); + + auditor = (LoggingAuditor) getManager().getAuditor(); + } + + /** + * Create the config from {@link AuditTestSupport#loggingAuditConfig()} + * and patch in filtering for fields x1, x2, x3. + * @return a logging configuration. + */ + protected Configuration createConfig() { + final Configuration conf = loggingAuditConfig(); + conf.set(REFERRER_HEADER_FILTER, "x1, x2, x3"); + return conf; + } + + /** + * This verifies that passing a request through the audit manager + * causes the http referrer header to be added, that it can + * be split to query parameters, and that those parameters match + * those of the active wrapped span. + */ + @Test + public void testHttpReferrerPatchesTheRequest() throws Throwable { + AuditSpan span = span(); + long ts = span.getTimestamp(); + GetObjectMetadataRequest request = head(); + Map headers + = request.getCustomRequestHeaders(); + assertThat(headers) + .describedAs("Custom headers") + .containsKey(HEADER_REFERRER); + String header = headers.get(HEADER_REFERRER); + LOG.info("Header is {}", header); + Map params + = HttpReferrerAuditHeader.extractQueryParameters(header); + assertMapContains(params, PARAM_PRINCIPAL, + UserGroupInformation.getCurrentUser().getUserName()); + assertMapContains(params, PARAM_FILESYSTEM_ID, auditor.getAuditorId()); + assertMapContains(params, PARAM_OP, OPERATION); + assertMapContains(params, PARAM_PATH, PATH_1); + assertMapContains(params, PARAM_PATH2, PATH_2); + String threadID = CommonAuditContext.currentThreadID(); + assertMapContains(params, PARAM_THREAD0, threadID); + assertMapContains(params, PARAM_THREAD1, threadID); + assertMapContains(params, PARAM_ID, span.getSpanId()); + assertThat(span.getTimestamp()) + .describedAs("Timestamp of " + span) + .isEqualTo(ts); + + assertMapContains(params, PARAM_TIMESTAMP, + Long.toString(ts)); + } + + /** + * Test that a header with complext paths including spaces + * and colons can be converted to a URI and back again + * without the path getting corrupted. + */ + @Test + public void testHeaderComplexPaths() throws Throwable { + String p1 = "s3a://dotted.bucket/path: value/subdir"; + String p2 = "s3a://key/"; + AuditSpan span = getManager().createSpan(OPERATION, p1, p2); + long ts = span.getTimestamp(); + Map params = issueRequestAndExtractParameters(); + assertMapContains(params, PARAM_PRINCIPAL, + UserGroupInformation.getCurrentUser().getUserName()); + assertMapContains(params, PARAM_FILESYSTEM_ID, auditor.getAuditorId()); + assertMapContains(params, PARAM_OP, OPERATION); + assertMapContains(params, PARAM_PATH, p1); + assertMapContains(params, PARAM_PATH2, p2); + String threadID = CommonAuditContext.currentThreadID(); + assertMapContains(params, PARAM_THREAD0, threadID); + assertMapContains(params, PARAM_THREAD1, threadID); + assertMapContains(params, PARAM_ID, span.getSpanId()); + assertThat(span.getTimestamp()) + .describedAs("Timestamp of " + span) + .isEqualTo(ts); + + assertMapContains(params, PARAM_TIMESTAMP, + Long.toString(ts)); + } + + /** + * Issue a request, then get the header field and parse it to the parameter. + * @return map of query params on the referrer header. + * @throws URISyntaxException failure to parse the header as a URI. + */ + private Map issueRequestAndExtractParameters() + throws URISyntaxException { + head(); + return HttpReferrerAuditHeader.extractQueryParameters( + auditor.getLastHeader()); + } + + + /** + * Test that headers are filtered out if configured. + */ + @Test + public void testHeaderFiltering() throws Throwable { + // add two attributes, x2 will be filtered. + AuditSpan span = getManager().createSpan(OPERATION, null, null); + auditor.addAttribute("x0", "x0"); + auditor.addAttribute("x2", "x2"); + final Map params + = issueRequestAndExtractParameters(); + assertThat(params) + .doesNotContainKey("x2"); + + } + + /** + * A real log entry. + * This is derived from a real log entry on a test run. + * If this needs to be updated, please do it from a real log. + * Splitting this up across lines has a tendency to break things, so + * be careful making changes. + */ + public static final String SAMPLE_LOG_ENTRY = + "183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000" + + " bucket-london" + + " [13/May/2021:11:26:06 +0000]" + + " 109.157.171.174" + + " arn:aws:iam::152813717700:user/dev" + + " M7ZB7C4RTKXJKTM9" + + " REST.PUT.OBJECT" + + " fork-0001/test/testParseBrokenCSVFile" + + " \"PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1\"" + + " 200" + + " -" + + " -" + + " 794" + + " 55" + + " 17" + + " \"https://audit.example.org/op_create/" + + "e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/" + + "?op=op_create" + + "&p1=fork-0001/test/testParseBrokenCSVFile" + + "&pr=alice" + + "&ps=2eac5a04-2153-48db-896a-09bc9a2fd132" + + "&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154" + + "&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&" + + "ts=1620905165700\"" + + " \"Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK\"" + + " -" + + " TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0=" + + " SigV4" + + " ECDHE-RSA-AES128-GCM-SHA256" + + " AuthHeader" + + " bucket-london.s3.eu-west-2.amazonaws.com" + + " TLSv1.2"; + + private static final String DESCRIPTION = String.format( + "log entry %s split by %s", SAMPLE_LOG_ENTRY, + LOG_ENTRY_PATTERN); + + /** + * Match the log entry and validate the results. + */ + @Test + public void testMatchAWSLogEntry() throws Throwable { + + LOG.info("Matcher pattern is\n'{}'", LOG_ENTRY_PATTERN); + LOG.info("Log entry is\n'{}'", SAMPLE_LOG_ENTRY); + final Matcher matcher = LOG_ENTRY_PATTERN.matcher(SAMPLE_LOG_ENTRY); + + // match the pattern against the entire log entry. + assertThat(matcher.matches()) + .describedAs("matches() " + DESCRIPTION) + .isTrue(); + final int groupCount = matcher.groupCount(); + assertThat(groupCount) + .describedAs("Group count of " + DESCRIPTION) + .isGreaterThanOrEqualTo(AWS_LOG_REGEXP_GROUPS.size()); + + // now go through the groups + + for (String name : AWS_LOG_REGEXP_GROUPS) { + try { + final String group = matcher.group(name); + LOG.info("[{}]: '{}'", name, group); + } catch (IllegalStateException e) { + // group failure + throw new AssertionError("No match for group <" + name + ">: " + + e, e); + } + } + // if you print out the groups as integers, there is duplicate matching + // for some fields. Why? + for (int i = 1; i <= groupCount; i++) { + try { + final String group = matcher.group(i); + LOG.info("[{}]: '{}'", i, group); + } catch (IllegalStateException e) { + // group failure + throw new AssertionError("No match for group " + i + +": "+ e, e); + } + } + + // verb + assertThat(nonBlankGroup(matcher, VERB_GROUP)) + .describedAs("HTTP Verb") + .isEqualTo(S3LogVerbs.PUT); + + // referrer + final String referrer = nonBlankGroup(matcher, REFERRER_GROUP); + Map params + = HttpReferrerAuditHeader.extractQueryParameters(referrer); + LOG.info("Parsed referrer"); + for (Map.Entry entry : params.entrySet()) { + LOG.info("{} = \"{}\"", entry.getKey(), entry.getValue()); + } + } + + /** + * Get a group entry which must be non-blank. + * @param matcher matcher + * @param group group name + * @return value + */ + private String nonBlankGroup(final Matcher matcher, + final String group) { + final String g = matcher.group(group); + assertThat(g) + .describedAs("Value of group %s", group) + .isNotBlank(); + return g; + } + + /** + * Verify the header quote stripping works. + */ + @Test + public void testStripWrappedQuotes() throws Throwable { + expectStrippedField("", ""); + expectStrippedField("\"UA\"", "UA"); + expectStrippedField("\"\"\"\"", ""); + expectStrippedField("\"\"\"b\"", "b"); + } + + /** + * Expect a field with quote stripping to match the expected value. + * @param str string to strip + * @param ex expected value. + */ + private void expectStrippedField(final String str, + final String ex) { + assertThat(maybeStripWrappedQuotes(str)) + .describedAs("Stripped <%s>", str) + .isEqualTo(ex); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestLoggingAuditor.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestLoggingAuditor.java new file mode 100644 index 0000000000000..8d37b432acb56 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestLoggingAuditor.java @@ -0,0 +1,203 @@ +/* + * 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.hadoop.fs.s3a.audit; + +import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; +import com.amazonaws.services.s3.model.CopyPartRequest; +import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor; +import org.apache.hadoop.fs.store.audit.AuditSpan; + +import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.loggingAuditConfig; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Logging auditor tests. + * By setting the auditor to raise an exception on unaudited spans, + * it is straightforward to determine if an operation was invoked + * outside a span: call it, and if it does not raise an exception, + * all is good. + */ +public class TestLoggingAuditor extends AbstractAuditingTest { + + /** + * Logging. + */ + private static final Logger LOG = + LoggerFactory.getLogger(TestLoggingAuditor.class); + + private LoggingAuditor auditor; + + @Before + public void setup() throws Exception { + super.setup(); + auditor = (LoggingAuditor) getManager().getAuditor(); + } + + /** + * Config has logging auditing and adds SimpleAWSRequestHandler + * too, for testing of that being added to the chain. + * @return a config + */ + protected Configuration createConfig() { + return loggingAuditConfig(); + } + + @Test + public void testToStringRobustness() throws Throwable { + // force in the toString calls so if there are NPE problems + // they will surface irrespective of log settings + LOG.info(getManager().toString()); + LOG.info(auditor.toString()); + } + + /** + * Test span activity with a span being activated/deactivated + * and verification that calls to head() succeed in the span + * and fail outside of it. + */ + @Test + public void testLoggingSpan() throws Throwable { + long executionCount = 0; + long failureCount = 0; + + // create a span + AuditSpan span = span(); + + // which is active + assertActiveSpan(span); + // so requests are allowed + verifyAuditExecutionCount(0); + head(); + verifyAuditExecutionCount(++executionCount); + + // now leave the span + span.deactivate(); + + // head calls are no longer allowed. + verifyAuditFailureCount(failureCount); + assertHeadUnaudited(); + verifyAuditFailureCount(++failureCount); + verifyAuditExecutionCount(++executionCount); + + // spans can be reactivated and used. + span.activate(); + head(); + verifyAuditExecutionCount(++executionCount); + + // its a no-op if the span is already active. + span.activate(); + assertActiveSpan(span); + + // closing a span deactivates it. + span.close(); + + // IO on unaudited spans + assertHeadUnaudited(); + verifyAuditFailureCount(++failureCount); + verifyAuditExecutionCount(++executionCount); + + // and it is harmless to deactivate a span repeatedly. + span.deactivate(); + span.deactivate(); + } + + /** + * Some request types are allowed to execute outside of + * a span. + * Required as the transfer manager runs them in its threads. + */ + @Test + public void testCopyOutsideSpanAllowed() throws Throwable { + getManager().beforeExecution(new CopyPartRequest()); + getManager().beforeExecution(new CompleteMultipartUploadRequest()); + } + + /** + * Outside a span, the transfer state change setup works but + * the call is unaudited. + */ + @Test + public void testTransferStateListenerOutsideSpan() throws Throwable { + TransferStateChangeListener listener + = getManager().createStateChangeListener(); + listener.transferStateChanged(null, null); + assertHeadUnaudited(); + } + + /** + * Outside a span, the transfer state change setup works but + * the call is unaudited. + */ + @Test + public void testTransferStateListenerInSpan() throws Throwable { + + assertHeadUnaudited(); + AuditSpan span = span(); + + // create the listener in the span + TransferStateChangeListener listener + = getManager().createStateChangeListener(); + span.deactivate(); + + // head calls fail + assertHeadUnaudited(); + + // until the state change switches this thread back to the span + listener.transferStateChanged(null, null); + + // which can be probed + assertActiveSpan(span); + + // and executed within + head(); + } + + /** + * You cannot deactivate the unbonded span. + */ + @Test + public void testUnbondedSpanWillNotDeactivate() throws Throwable { + AuditSpan span = activeSpan(); + // the active span is unbonded + assertUnbondedSpan(span); + // deactivate it. + span.deactivate(); + // it is still the active span. + assertActiveSpan(span); + } + + /** + * Spans have a different ID. + * This is clearly not an exhaustive test. + */ + @Test + public void testSpanIdsAreDifferent() throws Throwable { + AuditSpan s1 = span(); + AuditSpan s2 = span(); + assertThat(s1.getSpanId()) + .doesNotMatch(s2.getSpanId()); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index 771578001235e..814292c45d83b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -255,7 +255,8 @@ public void testAssumeRoleBadInnerAuth() throws Exception { conf.set(SECRET_KEY, "not secret"); expectFileSystemCreateFailure(conf, AWSBadRequestException.class, - "IncompleteSignature"); + "not a valid " + + "key=value pair (missing equal-sign) in Authorization header"); } @Test diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java index 1df4f9b9cc15c..faef79c1f6379 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java @@ -40,6 +40,7 @@ import org.apache.hadoop.fs.s3a.InconsistentAmazonS3Client; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.WriteOperationHelper; +import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.RecordWriter; @@ -289,10 +290,13 @@ protected int abortMultipartUploadsUnderPath(Path path) throws IOException { S3AFileSystem fs = getFileSystem(); if (fs != null && path != null) { String key = fs.pathToKey(path); - WriteOperationHelper writeOps = fs.getWriteOperationHelper(); - int count = writeOps.abortMultipartUploadsUnderPath(key); - if (count > 0) { - log().info("Multipart uploads deleted: {}", count); + int count = 0; + try (AuditSpan span = span()) { + WriteOperationHelper writeOps = fs.getWriteOperationHelper(); + count = writeOps.abortMultipartUploadsUnderPath(key); + if (count > 0) { + log().info("Multipart uploads deleted: {}", count); + } } return count; } else { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java index 14207e8359788..1e5a9582b2c2d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java @@ -44,6 +44,7 @@ import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.MapFile; @@ -177,7 +178,7 @@ public void teardown() throws Exception { describe("teardown"); abortInTeardown.forEach(this::abortJobQuietly); if (outDir != null) { - try { + try (AuditSpan span = span()) { abortMultipartUploadsUnderPath(outDir); cleanupDestDir(); } catch (IOException e) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java index ab2f9a2d316ee..e8c5d888d10dc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java @@ -195,7 +195,8 @@ private void fullThrottle() { setThrottling(FULL_THROTTLE, STANDARD_FAILURE_LIMIT); } - private CommitOperations newCommitOperations() { + private CommitOperations newCommitOperations() + throws IOException { return new CommitOperations(getFileSystem()); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java index aa3cecaf1eb7a..415dcba0f575f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java @@ -38,6 +38,7 @@ import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_XATTR_GET_NAMED; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_OCTET_STREAM; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_APPLICATION_XML; +import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_X_DIRECTORY; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_CONTENT_LENGTH; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_CONTENT_TYPE; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_STANDARD_HEADERS; @@ -163,7 +164,7 @@ public void testXAttrDir() throws Throwable { assertHeader(XA_CONTENT_LENGTH, bytes) .isEqualTo("0"); assertHeaderEntry(xAttrs, XA_CONTENT_TYPE) - .isEqualTo(CONTENT_TYPE_OCTET_STREAM); + .isEqualTo(CONTENT_TYPE_X_DIRECTORY); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java index e0c6feeb256cc..15c7ae917dfc5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java @@ -33,8 +33,12 @@ import org.junit.Test; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.MockS3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditTestSupport; import org.apache.hadoop.fs.s3a.test.OperationTrackingStore; +import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.test.HadoopTestBase; import static java.lang.System.currentTimeMillis; @@ -50,8 +54,8 @@ /** * Unit tests of header processing logic in {@link HeaderProcessing}. * Builds up a context accessor where the path - * defined in {@link #MAGIC_PATH} exists and returns object metadata. - * + * defined in {@link #MAGIC_PATH} exists and returns object metadata + * through the HeaderProcessingCallbacks. */ public class TestHeaderProcessing extends HadoopTestBase { @@ -95,7 +99,7 @@ public void setup() throws Exception { Long.toString(MAGIC_LEN)); context = S3ATestUtils.createMockStoreContext(true, new OperationTrackingStore(), CONTEXT_ACCESSORS); - headerProcessing = new HeaderProcessing(context); + headerProcessing = new HeaderProcessing(context, CONTEXT_ACCESSORS); } @Test @@ -203,7 +207,7 @@ public void testMetadataCopySkipsMagicAttribute() throws Throwable { final String owner = "x-header-owner"; final String root = "root"; CONTEXT_ACCESSORS.userHeaders.put(owner, root); - final ObjectMetadata source = context.getContextAccessors() + final ObjectMetadata source = CONTEXT_ACCESSORS .getObjectMetadata(MAGIC_KEY); final Map sourceUserMD = source.getUserMetadata(); Assertions.assertThat(sourceUserMD.get(owner)) @@ -254,9 +258,11 @@ protected void assertAttributeHasValue(final String key, /** * Context accessor with XAttrs returned for the {@link #MAGIC_PATH} * path. + * It also implements the Header Processing Callbacks, + * so those calls are mapped to the same data. */ private static final class XAttrContextAccessor - implements ContextAccessors { + implements ContextAccessors, HeaderProcessing.HeaderProcessingCallbacks { private final Map userHeaders = new HashMap<>(); @@ -291,6 +297,16 @@ public Path makeQualified(final Path path) { return path; } + @Override + public AuditSpan getActiveAuditSpan() { + return AuditTestSupport.NOOP_SPAN; + } + + @Override + public RequestFactory getRequestFactory() { + return MockS3AFileSystem.REQUEST_FACTORY; + } + @Override public ObjectMetadata getObjectMetadata(final String key) throws IOException { @@ -303,6 +319,7 @@ public ObjectMetadata getObjectMetadata(final String key) } else { throw new FileNotFoundException(key); } + } public void setHeader(String key, String val) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java index 42714cb1555e4..0d9ba1d304490 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java @@ -29,7 +29,6 @@ import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; -import com.amazonaws.services.s3.model.ObjectMetadata; import org.assertj.core.api.Assertions; import org.junit.Before; import org.junit.Test; @@ -37,8 +36,12 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.lang3.tuple.Triple; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.MockS3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditTestSupport; import org.apache.hadoop.fs.s3a.test.OperationTrackingStore; +import org.apache.hadoop.fs.store.audit.AuditSpan; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.ACCESS_DENIED; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.removeUndeletedPaths; @@ -257,10 +260,15 @@ public Path makeQualified(final Path path) { } @Override - public ObjectMetadata getObjectMetadata(final String key) - throws IOException { - return new ObjectMetadata(); + public AuditSpan getActiveAuditSpan() { + return AuditTestSupport.NOOP_SPAN; } + + @Override + public RequestFactory getRequestFactory() { + return MockS3AFileSystem.REQUEST_FACTORY; + } + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java new file mode 100644 index 0000000000000..c28eb5a9628d3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java @@ -0,0 +1,185 @@ +/* + * 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.hadoop.fs.s3a.impl; + +import java.io.ByteArrayInputStream; +import java.io.File; +import java.util.ArrayList; +import java.util.concurrent.atomic.AtomicLong; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.services.s3.model.ObjectListing; +import com.amazonaws.services.s3.model.ObjectMetadata; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AWSRequestAnalyzer; +import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Test that the request factory creates requests; factory + * is is built with different options on different test cases. + * Everything goes through {@link AWSRequestAnalyzer} to + * verify it handles every example, and logged so that a manual + * review of the output can show it is valid. + */ +public class TestRequestFactory extends AbstractHadoopTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TestRequestFactory.class); + + private final AWSRequestAnalyzer analyzer = new AWSRequestAnalyzer(); + + /** + * Count of requests analyzed via the {@link #a(AmazonWebServiceRequest)} + * call. + */ + private int requestsAnalyzed; + + /** + * No preparer; encryption is set. + */ + @Test + public void testRequestFactoryWithEncryption() throws Throwable { + RequestFactory factory = RequestFactoryImpl.builder() + .withBucket("bucket") + .withEncryptionSecrets( + new EncryptionSecrets(S3AEncryptionMethods.SSE_KMS, + "kms:key")) + .build(); + createFactoryObjects(factory); + } + + /** + * Now add a processor and verify that it was invoked for + * exactly as many requests as were analyzed. + */ + @Test + public void testRequestFactoryWithProcessor() throws Throwable { + CountRequests countRequests = new CountRequests(); + RequestFactory factory = RequestFactoryImpl.builder() + .withBucket("bucket") + .withRequestPreparer(countRequests) + .build(); + + createFactoryObjects(factory); + assertThat(countRequests.counter.get()) + .describedAs("request preparation count") + .isEqualTo(requestsAnalyzed); + } + + private final class CountRequests + implements RequestFactoryImpl.PrepareRequest { + + private final AtomicLong counter = new AtomicLong(); + + @Override + public T prepareRequest(final T t) { + counter.addAndGet(1); + return t; + } + } + + /** + * Analyze the request, log the output, return the info. + * @param request request. + * @param type of request. + * @return value + */ + private AWSRequestAnalyzer.RequestInfo + a(T request) { + AWSRequestAnalyzer.RequestInfo info = analyzer.analyze(request); + LOG.info("{}", info); + requestsAnalyzed++; + return info; + } + + /** + * Create objects through the factory. + * @param factory factory + */ + private void createFactoryObjects(RequestFactory factory) { + String path = "path"; + String path2 = "path2"; + String id = "1"; + ObjectMetadata md = factory.newObjectMetadata(128); + a(factory.newAbortMultipartUploadRequest(path, id)); + a(factory.newCompleteMultipartUploadRequest(path, id, + new ArrayList<>())); + a(factory.newCopyObjectRequest(path, path2, md)); + a(factory.newDeleteObjectRequest(path)); + a(factory.newBulkDeleteRequest(new ArrayList<>(), true)); + a(factory.newDirectoryMarkerRequest(path)); + a(factory.newGetObjectRequest(path)); + a(factory.newGetObjectMetadataRequest(path)); + a(factory.newListMultipartUploadsRequest(path)); + a(factory.newListObjectsV1Request(path, "/", 1)); + a(factory.newListNextBatchOfObjectsRequest(new ObjectListing())); + a(factory.newListObjectsV2Request(path, "/", 1)); + a(factory.newMultipartUploadRequest(path)); + File srcfile = new File("/tmp/a"); + a(factory.newPutObjectRequest(path, + factory.newObjectMetadata(-1), srcfile)); + ByteArrayInputStream stream = new ByteArrayInputStream(new byte[0]); + a(factory.newPutObjectRequest(path, md, stream)); + a(factory.newSelectRequest(path)); + } + + /** + * Multiparts are special so test on their own. + */ + @Test + public void testMultipartUploadRequest() throws Throwable { + CountRequests countRequests = new CountRequests(); + + RequestFactory factory = RequestFactoryImpl.builder() + .withBucket("bucket") + .withRequestPreparer(countRequests) + .build(); + + String path = "path"; + String path2 = "path2"; + String id = "1"; + File srcfile = File.createTempFile("file", ""); + try { + ByteArrayInputStream stream = new ByteArrayInputStream(new byte[0]); + + a(factory.newUploadPartRequest(path, id, 1, 0, stream, null, 0)); + a(factory.newUploadPartRequest(path, id, 2, 128_000_000, + null, srcfile, 0)); + // offset is past the EOF + intercept(IllegalArgumentException.class, () -> + factory.newUploadPartRequest(path, id, 3, 128_000_000, + null, srcfile, 128)); + } finally { + srcfile.delete(); + } + assertThat(countRequests.counter.get()) + .describedAs("request preparation count") + .isEqualTo(requestsAnalyzed); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index c4f8db71937d0..99a64e6a94268 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -40,6 +40,7 @@ import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum; +import org.apache.hadoop.fs.store.audit.AuditSpan; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; @@ -108,6 +109,18 @@ public AbstractS3ACostTest( this.authoritative = authoritative; } + /** + * Constructor for tests which don't include + * any for S3Guard. + * @param keepMarkers should markers be tested. + */ + public AbstractS3ACostTest( + final boolean keepMarkers) { + this.s3guard = false; + this.keepMarkers = keepMarkers; + this.authoritative = false; + } + @Override public Configuration createConfiguration() { Configuration conf = super.createConfiguration(); @@ -183,6 +196,8 @@ public void setup() throws Exception { deleteMarkerStatistic = isBulkDelete() ? OBJECT_BULK_DELETE_REQUEST : OBJECT_DELETE_REQUEST; + + setSpanSource(fs); } public void assumeUnguarded() { @@ -357,6 +372,7 @@ private void resetStatistics() { protected T verifyMetrics( Callable eval, OperationCostValidator.ExpectedProbe... expected) throws Exception { + span(); return costValidator.exec(eval, expected); } @@ -379,6 +395,7 @@ protected E verifyMetricsIntercepting( String text, Callable eval, OperationCostValidator.ExpectedProbe... expected) throws Exception { + span(); return costValidator.intercepting(clazz, text, eval, expected); } @@ -476,6 +493,8 @@ protected OperationCostValidator.ExpectedProbe whenDeleting( /** * Execute a closure expecting a specific number of HEAD/LIST calls * on raw S3 stores only. The operation is always evaluated. + * A span is always created prior to the invocation; saves trouble + * in tests that way. * @param cost expected cost * @param eval closure to evaluate * @param return type of closure @@ -525,12 +544,14 @@ public void interceptRawGetFileStatusFNFE( boolean needEmptyDirectoryFlag, Set probes, OperationCost cost) throws Exception { - interceptRaw(FileNotFoundException.class, "", - cost, () -> - innerGetFileStatus(getFileSystem(), - path, - needEmptyDirectoryFlag, - probes)); + try (AuditSpan span = span()) { + interceptRaw(FileNotFoundException.class, "", + cost, () -> + innerGetFileStatus(getFileSystem(), + path, + needEmptyDirectoryFlag, + probes)); + } } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java index ed56802ddfec1..a08d77367b18f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java @@ -46,6 +46,7 @@ import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AUtils; +import org.apache.hadoop.fs.store.audit.AuditSpan; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; @@ -58,6 +59,7 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.apache.hadoop.util.functional.RemoteIterators.foreach; /** * This is a test suite designed to verify that directory markers do @@ -223,7 +225,6 @@ public void setup() throws Exception { assume("unguarded FS only", !fs.hasMetadataStore()); s3client = fs.getAmazonS3ClientForTesting("markers"); - bucket = fs.getBucket(); Path base = new Path(methodPath(), "base"); @@ -653,7 +654,9 @@ private void head404(final String key) throws Exception { } /** - * Execute an operation; transate AWS exceptions. + * Execute an operation; translate AWS exceptions. + * Wraps the operation in an audit span, so that low-level + * calls can be safely made. * @param op operation * @param call call to make * @param returned type @@ -662,7 +665,7 @@ private void head404(final String key) throws Exception { */ private T exec(String op, Callable call) throws Exception { ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); - try { + try (AuditSpan span = getSpanSource().createSpan(op, null, null)) { return call.call(); } catch (AmazonClientException ex) { throw S3AUtils.translateException(op, "", ex); @@ -749,9 +752,7 @@ private List toList( RemoteIterator status) throws IOException { List l = new ArrayList<>(); - while (status.hasNext()) { - l.add(status.next()); - } + foreach(status, st -> l.add(st)); return dump(l); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMiscOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMiscOperationCost.java new file mode 100644 index 0000000000000..6449d2a5e44c9 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMiscOperationCost.java @@ -0,0 +1,143 @@ +/* + * 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.hadoop.fs.s3a.performance; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileSystem; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_SPAN_CREATION; +import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_GET_CONTENT_SUMMARY; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_METADATA_REQUESTS; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.FILESTATUS_DIR_PROBE_L; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.FILE_STATUS_FILE_PROBE; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.LIST_OPERATION; + +/** + * Use metrics to assert about the cost of misc operations. + * Parameterized on directory marker keep vs delete + */ +@RunWith(Parameterized.class) +public class ITestS3AMiscOperationCost extends AbstractS3ACostTest { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestS3AMiscOperationCost.class); + + /** + * Parameterization. + */ + @Parameterized.Parameters(name = "{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {"keep-markers", true}, + {"delete-markers", false} + }); + } + + public ITestS3AMiscOperationCost(final String name, + final boolean keepMarkers) { + super(false, keepMarkers, false); + } + + /** + * Common operation which should be low cost as possible. + */ + @Test + public void testMkdirOverDir() throws Throwable { + describe("create a dir over a dir"); + S3AFileSystem fs = getFileSystem(); + // create base dir with marker + Path baseDir = dir(methodPath()); + + // create the child; only assert on HEAD/GET IO + verifyMetrics(() -> fs.mkdirs(baseDir), + with(AUDIT_SPAN_CREATION, 1), + // full probe on dest plus list only on parent. + with(OBJECT_METADATA_REQUESTS, 0), + with(OBJECT_LIST_REQUEST, FILESTATUS_DIR_PROBE_L)); + } + + @Test + public void testGetContentSummaryRoot() throws Throwable { + describe("getContentSummary on Root"); + S3AFileSystem fs = getFileSystem(); + + Path root = new Path("/"); + verifyMetrics(() -> getContentSummary(root), + with(INVOCATION_GET_CONTENT_SUMMARY, 1)); + } + + @Test + public void testGetContentSummaryDir() throws Throwable { + describe("getContentSummary on test dir with children"); + S3AFileSystem fs = getFileSystem(); + Path baseDir = methodPath(); + Path childDir = new Path(baseDir, "subdir/child"); + touch(fs, childDir); + + final ContentSummary summary = verifyMetrics( + () -> getContentSummary(baseDir), + with(INVOCATION_GET_CONTENT_SUMMARY, 1), + with(AUDIT_SPAN_CREATION, 1), + whenRaw(FILE_STATUS_FILE_PROBE // look at path to see if it is a file + .plus(LIST_OPERATION) // it is not: so LIST + .plus(LIST_OPERATION))); // and a LIST on the child dir + Assertions.assertThat(summary.getDirectoryCount()) + .as("Summary " + summary) + .isEqualTo(2); + Assertions.assertThat(summary.getFileCount()) + .as("Summary " + summary) + .isEqualTo(1); + } + + @Test + public void testGetContentMissingPath() throws Throwable { + describe("getContentSummary on a missing path"); + Path baseDir = methodPath(); + verifyMetricsIntercepting(FileNotFoundException.class, + "", () -> getContentSummary(baseDir), + with(INVOCATION_GET_CONTENT_SUMMARY, 1), + with(AUDIT_SPAN_CREATION, 1), + whenRaw(FILE_STATUS_FILE_PROBE + .plus(FILE_STATUS_FILE_PROBE) + .plus(LIST_OPERATION) + .plus(LIST_OPERATION))); + } + + private ContentSummary getContentSummary(final Path baseDir) + throws IOException { + S3AFileSystem fs = getFileSystem(); + return fs.getContentSummary(baseDir); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMkdirCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMkdirCost.java new file mode 100644 index 0000000000000..639e1dddbae74 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMkdirCost.java @@ -0,0 +1,185 @@ +/* + * 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.hadoop.fs.s3a.performance; + +import java.util.Arrays; +import java.util.Collection; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileSystem; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_METADATA_REQUESTS; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.FILESTATUS_DIR_PROBE_L; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.FILESTATUS_FILE_PROBE_H; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.FILESTATUS_FILE_PROBE_L; + +/** + * Use metrics to assert about the cost of mkdirs. + * Parameterized directory marker keep vs delete + */ +@RunWith(Parameterized.class) +public class ITestS3AMkdirCost extends AbstractS3ACostTest { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestS3AMkdirCost.class); + + /** + * Parameterization. + */ + @Parameterized.Parameters(name = "{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {"keep-markers", true}, + {"delete-markers", false} + }); + } + + public ITestS3AMkdirCost(final String name, + final boolean keepMarkers) { + super(false, true, false); + } + + /** + * Common operation which should be low cost as possible. + */ + @Test + public void testMkdirOverDir() throws Throwable { + describe("create a dir over a dir"); + S3AFileSystem fs = getFileSystem(); + // create base dir with marker + Path baseDir = dir(methodPath()); + + // create the child; only assert on HEAD/GET IO + verifyMetrics(() -> fs.mkdirs(baseDir), + // full probe on dest plus list only on parent. + with(OBJECT_METADATA_REQUESTS, + 0), + with(OBJECT_LIST_REQUEST, FILESTATUS_DIR_PROBE_L)); + } + + /** + * Mkdir with a parent dir will check dest (list+HEAD) + * then do a list on the parent to find the marker. + * Once the dir is created, creating a sibling will + * have the same cost. + */ + @Test + public void testMkdirWithParent() throws Throwable { + describe("create a dir under a dir with a parent"); + S3AFileSystem fs = getFileSystem(); + // create base dir with marker + Path baseDir = dir(methodPath()); + Path childDir = new Path(baseDir, "child"); + + // create the child; only assert on HEAD/GET IO + verifyMetrics(() -> fs.mkdirs(childDir), + // full probe on dest plus list only on parent. + with(OBJECT_METADATA_REQUESTS, + FILESTATUS_FILE_PROBE_H), + + with(OBJECT_LIST_REQUEST, + FILESTATUS_FILE_PROBE_L + 2 * FILESTATUS_DIR_PROBE_L)); + + // now include a sibling; cost will be the same. + Path sibling = new Path(baseDir, "sibling"); + verifyMetrics(() -> fs.mkdirs(sibling), + // full probe on dest plus list only on parent. + with(OBJECT_METADATA_REQUESTS, + FILESTATUS_FILE_PROBE_H), + + with(OBJECT_LIST_REQUEST, + FILESTATUS_FILE_PROBE_L + 2 * FILESTATUS_DIR_PROBE_L)); + } + + /** + * Mkdir with a grandparent dir will check dest (list+HEAD) + * then do a list + HEAD on the parent and ultimately find the + * marker with a list of the parent. + * That's three list calls and two head requsts. + * Once the dir is created, creating a sibling will + * cost less as the list of the parent path will find + * a directory. + */ + @Test + public void testMkdirWithGrandparent() throws Throwable { + describe("create a dir under a dir with a parent"); + S3AFileSystem fs = getFileSystem(); + // create base dir with marker + Path baseDir = dir(methodPath()); + Path subDir = new Path(baseDir, "child/grandchild"); + + // create the child; only assert on HEAD/GET IO + verifyMetrics(() -> fs.mkdirs(subDir), + // full probe on dest plus list only on parent. + with(OBJECT_METADATA_REQUESTS, + 2 * FILESTATUS_FILE_PROBE_H), + + with(OBJECT_LIST_REQUEST, + 3 * FILESTATUS_DIR_PROBE_L)); + + + // now include a sibling; cost will be less because + // now the immediate parent check will succeed on the list call. + Path sibling = new Path(baseDir, "child/sibling"); + + verifyMetrics(() -> fs.mkdirs(sibling), + + // full probe on dest plus list only on parent. + with(OBJECT_METADATA_REQUESTS, + FILESTATUS_FILE_PROBE_H), + + with(OBJECT_LIST_REQUEST, + FILESTATUS_FILE_PROBE_L + 2 * FILESTATUS_DIR_PROBE_L)); + } + + + /** + * When calling mkdir over a file, the list happens first, so + * is always billed for. + * @throws Throwable failure. + */ + @Test + public void testMkdirOverFile() throws Throwable { + describe("create a dir over a file; expect dir and file probes"); + S3AFileSystem fs = getFileSystem(); + // create base dir with marker + Path baseDir = dir(methodPath()); + Path childDir = new Path(baseDir, "child"); + touch(fs, childDir); + + // create the child; only assert on HEAD/GET IO + verifyMetricsIntercepting( + FileAlreadyExistsException.class, "", + () -> fs.mkdirs(childDir), + // full probe on dest plus list only on parent. + with(OBJECT_METADATA_REQUESTS, + FILESTATUS_FILE_PROBE_H), + with(OBJECT_LIST_REQUEST, FILESTATUS_DIR_PROBE_L)); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java index 3f03abbabc3cc..72e51ee3b9958 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java @@ -41,6 +41,7 @@ import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUEST; import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_METADATA_REQUESTS; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** @@ -169,7 +170,7 @@ public T exec( LOG.info("{}", text); LOG.info("state {}", this.toString()); LOG.info("probes {}", expected); - LOG.info("IOStatistics {}", ioStatistics); + LOG.info("IOStatistics {}", ioStatisticsToPrettyString(ioStatistics)); for (ExpectedProbe ed : expected) { ed.verify(this, text); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java index c4e903df7390f..28eb52ff41ada 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java @@ -44,7 +44,6 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.awaitFileStatus; import static org.apache.hadoop.fs.s3a.S3ATestUtils.metadataStorePersistsAuthoritativeBit; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; @@ -207,7 +206,6 @@ public void testIDetectDirInS3FileInMs() throws Exception { try { // create a file with guarded fs mkdirs(cwd); - awaitFileStatus(guardedFs, cwd); // modify the cwd metadata and set that it's not a directory final S3AFileStatus newParentFile = MetadataStoreTestBase .basicFileStatus(cwd, 1, false, 1); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java index 798bdd693081f..ecddbbd820872 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java @@ -272,35 +272,28 @@ public void testUploads() throws Throwable { Path path = path(UPLOAD_PREFIX + "/" + UPLOAD_NAME); describe("Cleaning up any leftover uploads from previous runs."); - // 1. Make sure key doesn't already exist - clearAnyUploads(fs, path); + final String key = fs.pathToKey(path); + try { + // 1. Make sure key doesn't already exist + clearAnyUploads(fs, path); - // 2. Confirm no uploads are listed via API - assertNoUploadsAt(fs, path.getParent()); + // 2. Confirm no uploads are listed via API + assertNoUploadsAt(fs, path.getParent()); - // 3. Confirm no uploads are listed via CLI - describe("Confirming CLI lists nothing."); - assertNumUploads(path, 0); + // 3. Confirm no uploads are listed via CLI + describe("Confirming CLI lists nothing."); + assertNumUploads(path, 0); - // 4. Create a upload part - describe("Uploading single part."); - createPartUpload(fs, fs.pathToKey(path), 128, 1); + // 4. Create a upload part + describe("Uploading single part."); + createPartUpload(fs, key, 128, 1); - try { - // 5. Confirm it exists via API.. - LambdaTestUtils.eventually(5000, /* 5 seconds until failure */ - 1000, /* one second retry interval */ - () -> { - assertEquals("Should be one upload", 1, countUploadsAt(fs, path)); - }); + assertEquals("Should be one upload", 1, countUploadsAt(fs, path)); // 6. Confirm part exists via CLI, direct path and parent path describe("Confirming CLI lists one part"); - LambdaTestUtils.eventually(5000, 1000, - () -> { assertNumUploads(path, 1); }); - LambdaTestUtils.eventually(5000, 1000, - () -> { assertNumUploads(path.getParent(), 1); }); - + assertNumUploads(path, 1); + assertNumUploads(path.getParent(), 1); // 7. Use CLI to delete part, assert it worked describe("Deleting part via CLI"); assertNumDeleted(fs, path, 1); @@ -331,15 +324,15 @@ public void testUploadListByAge() throws Throwable { // 2. Create a upload part describe("Uploading single part."); - createPartUpload(fs, fs.pathToKey(path), 128, 1); + final String key = fs.pathToKey(path); + createPartUpload(fs, key, 128, 1); + //try (AuditSpan span = fs.startOperation("multipart", key, null)) { try { + // 3. Confirm it exists via API.. may want to wrap with // LambdaTestUtils.eventually() ? - LambdaTestUtils.eventually(5000, 1000, - () -> { - assertEquals("Should be one upload", 1, countUploadsAt(fs, path)); - }); + assertEquals("Should be one upload", 1, countUploadsAt(fs, path)); // 4. Confirm part does appear in listing with long age filter describe("Confirming CLI older age doesn't list"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index 2b3043f39ac73..3e00917a2dbe5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java @@ -128,8 +128,7 @@ protected Configuration createScaleConfiguration() { @Test public void test_010_CreateHugeFile() throws IOException { - assertFalse("Please run this test sequentially to avoid timeouts" + - " and bandwidth problems", isParallelExecution()); + long filesizeMB = filesize / _1MB; // clean up from any previous attempts diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java index 61f6ef3c76473..d5e4788036ebe 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java @@ -28,7 +28,11 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.WriteOperationHelper; +import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.store.audit.AuditSpan; +import org.apache.hadoop.util.functional.RemoteIterators; import org.junit.Test; import org.assertj.core.api.Assertions; @@ -39,7 +43,7 @@ import java.io.InputStream; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -47,13 +51,17 @@ import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.PutObjectResult; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupCounterStatistic; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; -import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; @@ -164,6 +172,13 @@ public void testListOperations() throws Throwable { } } + /** + * This is quite a big test; it PUTs up a number of + * files and then lists them in a filesystem set to ask for a small number + * of files on each listing. + * The standard listing API calls are all used, and then + * delete() is invoked to verify that paged deletion works correctly too. + */ @Test public void testMultiPagesListingPerformanceAndCorrectness() throws Throwable { @@ -180,45 +195,59 @@ public void testMultiPagesListingPerformanceAndCorrectness() .isEqualTo(0); final Configuration conf = getConfigurationWithConfiguredBatchSize(batchSize); - removeBaseAndBucketOverrides(conf, S3_METADATA_STORE_IMPL); - final InputStream im = new InputStream() { - @Override - public int read() throws IOException { - return -1; - } - }; + + removeBaseAndBucketOverrides(conf, + S3_METADATA_STORE_IMPL, + DIRECTORY_MARKER_POLICY); + // force directory markers = keep to save delete requests on every + // file created. + conf.set(DIRECTORY_MARKER_POLICY, DIRECTORY_MARKER_POLICY_KEEP); + S3AFileSystem fs = (S3AFileSystem) FileSystem.get(dir.toUri(), conf); + final List originalListOfFiles = new ArrayList<>(); - List> putObjectRequests = new ArrayList<>(); ExecutorService executorService = Executors .newFixedThreadPool(numOfPutThreads); NanoTimer uploadTimer = new NanoTimer(); - S3AFileSystem fs = (S3AFileSystem) FileSystem.get(dir.toUri(), conf); try { assume("Test is only for raw fs", !fs.hasMetadataStore()); fs.create(dir); + + // create a span for the write operations + final AuditSpan span = fs.getAuditSpanSource() + .createSpan(OBJECT_PUT_REQUESTS.getSymbol(), dir.toString(), null); + final WriteOperationHelper writeOperationHelper + = fs.getWriteOperationHelper(); + final RequestFactory requestFactory + = writeOperationHelper.getRequestFactory(); + List> futures = + new ArrayList<>(numOfPutRequests); + for (int i=0; i - fs.getWriteOperationHelper().putObject(put)); + PutObjectRequest put = requestFactory + .newPutObjectRequest(fs.pathToKey(file), om, + new FailingInputStream()); + futures.add(submit(executorService, () -> + writeOperationHelper.putObject(put))); } - executorService.invokeAll(putObjectRequests); + LOG.info("Waiting for PUTs to complete"); + waitForCompletion(futures); uploadTimer.end("uploading %d files with a parallelism of %d", numOfPutRequests, numOfPutThreads); RemoteIterator resIterator = fs.listFiles(dir, true); List listUsingListFiles = new ArrayList<>(); NanoTimer timeUsingListFiles = new NanoTimer(); - while(resIterator.hasNext()) { - listUsingListFiles.add(resIterator.next().getPath().toString()); - Thread.sleep(eachFileProcessingTime); - } + RemoteIterators.foreach(resIterator, st -> { + listUsingListFiles.add(st.getPath().toString()); + sleep(eachFileProcessingTime); + }); + LOG.info("Listing Statistics: {}", ioStatisticsToPrettyString( + retrieveIOStatistics(resIterator))); + timeUsingListFiles.end("listing %d files using listFiles() api with " + "batch size of %d including %dms of processing time" + " for each file", @@ -226,7 +255,7 @@ public int read() throws IOException { Assertions.assertThat(listUsingListFiles) .describedAs("Listing results using listFiles() must" + - "match with original list of files") + " match with original list of files") .hasSameElementsAs(originalListOfFiles) .hasSize(numOfPutRequests); List listUsingListStatus = new ArrayList<>(); @@ -234,7 +263,7 @@ public int read() throws IOException { FileStatus[] fileStatuses = fs.listStatus(dir); for(FileStatus fileStatus : fileStatuses) { listUsingListStatus.add(fileStatus.getPath().toString()); - Thread.sleep(eachFileProcessingTime); + sleep(eachFileProcessingTime); } timeUsingListStatus.end("listing %d files using listStatus() api with " + "batch size of %d including %dms of processing time" + @@ -247,12 +276,12 @@ public int read() throws IOException { .hasSize(numOfPutRequests); // Validate listing using listStatusIterator(). NanoTimer timeUsingListStatusItr = new NanoTimer(); - RemoteIterator lsItr = fs.listStatusIterator(dir); List listUsingListStatusItr = new ArrayList<>(); - while (lsItr.hasNext()) { - listUsingListStatusItr.add(lsItr.next().getPath().toString()); - Thread.sleep(eachFileProcessingTime); - } + RemoteIterator lsItr = fs.listStatusIterator(dir); + RemoteIterators.foreach(lsItr, st -> { + listUsingListStatusItr.add(st.getPath().toString()); + sleep(eachFileProcessingTime); + }); timeUsingListStatusItr.end("listing %d files using " + "listStatusIterator() api with batch size of %d " + "including %dms of processing time for each file", @@ -265,24 +294,68 @@ public int read() throws IOException { // now validate the statistics returned by the listing // to be non-null and containing list and continue counters. IOStatistics lsStats = retrieveIOStatistics(lsItr); - String statsReport = ioStatisticsToString(lsStats); + String statsReport = ioStatisticsToPrettyString(lsStats); LOG.info("Listing Statistics: {}", statsReport); verifyStatisticCounterValue(lsStats, OBJECT_LIST_REQUEST, 1); long continuations = lookupCounterStatistic(lsStats, OBJECT_CONTINUE_LIST_REQUEST); // calculate expected #of continuations - int expectedContinuations = numOfPutRequests / batchSize -1; + int expectedContinuations = numOfPutRequests / batchSize - 1; Assertions.assertThat(continuations) .describedAs("%s in %s", OBJECT_CONTINUE_LIST_REQUEST, statsReport) .isEqualTo(expectedContinuations); + + List listUsingListLocatedStatus = new ArrayList<>(); + + RemoteIterator it = fs.listLocatedStatus(dir); + RemoteIterators.foreach(it, st -> { + listUsingListLocatedStatus.add(st.getPath().toString()); + sleep(eachFileProcessingTime); + }); + final IOStatistics llsStats = retrieveIOStatistics(it); + LOG.info("Listing Statistics: {}", ioStatisticsToPrettyString( + llsStats)); + verifyStatisticCounterValue(llsStats, OBJECT_CONTINUE_LIST_REQUEST, + expectedContinuations); + Assertions.assertThat(listUsingListLocatedStatus) + .describedAs("Listing results using listLocatedStatus() must" + + "match with original list of files") + .hasSameElementsAs(originalListOfFiles); + // delete in this FS so S3Guard is left out of it. + // and so that the incremental listing is tested through + // the delete operation. + fs.delete(dir, true); } finally { executorService.shutdown(); - // delete in this FS so S3Guard is left out of it. + // in case the previous delete was not reached. fs.delete(dir, true); + LOG.info("FS statistics {}", + ioStatisticsToPrettyString(fs.getIOStatistics())); fs.close(); } } + /** + * Input stream which always returns -1. + */ + private static final class FailingInputStream extends InputStream { + @Override + public int read() throws IOException { + return -1; + } + } + + /** + * Sleep briefly. + * @param eachFileProcessingTime time to sleep. + */ + private void sleep(final int eachFileProcessingTime) { + try { + Thread.sleep(eachFileProcessingTime); + } catch (InterruptedException ignored) { + } + } + private Configuration getConfigurationWithConfiguredBatchSize(int batchSize) { Configuration conf = new Configuration(getFileSystem().getConf()); S3ATestUtils.disableFilesystemCaching(conf); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalListingOperationCallbacks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalListingOperationCallbacks.java index f683a0a008caf..678e5d482d1a5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalListingOperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalListingOperationCallbacks.java @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.s3a.impl.ListingOperationCallbacks; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.store.audit.AuditSpan; /** * Stub implementation of {@link ListingOperationCallbacks}. @@ -39,7 +40,8 @@ public class MinimalListingOperationCallbacks @Override public CompletableFuture listObjectsAsync( final S3ListRequest request, - final DurationTrackerFactory trackerFactory) throws IOException { + final DurationTrackerFactory trackerFactory, AuditSpan span) + throws IOException { return null; } @@ -47,7 +49,8 @@ public CompletableFuture listObjectsAsync( public CompletableFuture continueListObjectsAsync( final S3ListRequest request, final S3ListResult prevResult, - final DurationTrackerFactory trackerFactory) throws IOException { + final DurationTrackerFactory trackerFactory, AuditSpan span) + throws IOException { return null; } @@ -60,7 +63,8 @@ public S3ALocatedFileStatus toLocatedFileStatus( @Override public S3ListRequest createListObjectsRequest( String key, - String delimiter) { + String delimiter, + AuditSpan span) { return null; } diff --git a/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml b/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml index a90edbe24fc4a..21e30b6d66cb1 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml +++ b/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml @@ -57,6 +57,12 @@ Let's postpone existence checks to the first IO operation + + fs.s3a.bucket.landsat-pds.audit.add.referrer.header + false + Do not add the referrer header to landsat operations + + s3guard.null @@ -166,6 +172,13 @@ 0 + + + fs.s3a.audit.reject.out.of.span.operations + true + +