hadoop/hadoop-aws/tools/hadoop-aws/troubleshooting_s3a.html

2245 lines
166 KiB
HTML

<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN" "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
<!--
| Generated by Apache Maven Doxia at 2023-03-02
| Rendered using Apache Maven Stylus Skin 1.5
-->
<html xmlns="http://www.w3.org/1999/xhtml">
<head>
<title>Apache Hadoop Amazon Web Services support &#x2013; Troubleshooting</title>
<style type="text/css" media="all">
@import url("../../css/maven-base.css");
@import url("../../css/maven-theme.css");
@import url("../../css/site.css");
</style>
<link rel="stylesheet" href="../../css/print.css" type="text/css" media="print" />
<meta name="Date-Revision-yyyymmdd" content="20230302" />
<meta http-equiv="Content-Type" content="text/html; charset=UTF-8" />
</head>
<body class="composite">
<div id="banner">
<a href="http://hadoop.apache.org/" id="bannerLeft">
<img src="http://hadoop.apache.org/images/hadoop-logo.jpg" alt="" />
</a>
<a href="http://www.apache.org/" id="bannerRight">
<img src="http://www.apache.org/images/asf_logo_wide.png" alt="" />
</a>
<div class="clear">
<hr/>
</div>
</div>
<div id="breadcrumbs">
<div class="xright"> <a href="http://wiki.apache.org/hadoop" class="externalLink">Wiki</a>
|
<a href="https://gitbox.apache.org/repos/asf/hadoop.git" class="externalLink">git</a>
&nbsp;| Last Published: 2023-03-02
&nbsp;| Version: 3.4.0-SNAPSHOT
</div>
<div class="clear">
<hr/>
</div>
</div>
<div id="leftColumn">
<div id="navcolumn">
<h5>General</h5>
<ul>
<li class="none">
<a href="../../../index.html">Overview</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/SingleCluster.html">Single Node Setup</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/ClusterSetup.html">Cluster Setup</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/CommandsManual.html">Commands Reference</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/FileSystemShell.html">FileSystem Shell</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/Compatibility.html">Compatibility Specification</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/DownstreamDev.html">Downstream Developer's Guide</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/AdminCompatibilityGuide.html">Admin Compatibility Guide</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/InterfaceClassification.html">Interface Classification</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/filesystem/index.html">FileSystem Specification</a>
</li>
</ul>
<h5>Common</h5>
<ul>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/CLIMiniCluster.html">CLI Mini Cluster</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/FairCallQueue.html">Fair Call Queue</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/NativeLibraries.html">Native Libraries</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/Superusers.html">Proxy User</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/RackAwareness.html">Rack Awareness</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/SecureMode.html">Secure Mode</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/ServiceLevelAuth.html">Service Level Authorization</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/HttpAuthentication.html">HTTP Authentication</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/CredentialProviderAPI.html">Credential Provider API</a>
</li>
<li class="none">
<a href="../../../hadoop-kms/index.html">Hadoop KMS</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/Tracing.html">Tracing</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/UnixShellGuide.html">Unix Shell Guide</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/registry/index.html">Registry</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/AsyncProfilerServlet.html">Async Profiler</a>
</li>
</ul>
<h5>HDFS</h5>
<ul>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HdfsDesign.html">Architecture</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HdfsUserGuide.html">User Guide</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HDFSCommands.html">Commands Reference</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithQJM.html">NameNode HA With QJM</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html">NameNode HA With NFS</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/ObserverNameNode.html">Observer NameNode</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/Federation.html">Federation</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/ViewFs.html">ViewFs</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/ViewFsOverloadScheme.html">ViewFsOverloadScheme</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HdfsSnapshots.html">Snapshots</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HdfsEditsViewer.html">Edits Viewer</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HdfsImageViewer.html">Image Viewer</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HdfsPermissionsGuide.html">Permissions and HDFS</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HdfsQuotaAdminGuide.html">Quotas and HDFS</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/LibHdfs.html">libhdfs (C API)</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/WebHDFS.html">WebHDFS (REST API)</a>
</li>
<li class="none">
<a href="../../../hadoop-hdfs-httpfs/index.html">HttpFS</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/ShortCircuitLocalReads.html">Short Circuit Local Reads</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/CentralizedCacheManagement.html">Centralized Cache Management</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HdfsNfsGateway.html">NFS Gateway</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HdfsRollingUpgrade.html">Rolling Upgrade</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/ExtendedAttributes.html">Extended Attributes</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/TransparentEncryption.html">Transparent Encryption</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HdfsMultihoming.html">Multihoming</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/ArchivalStorage.html">Storage Policies</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/MemoryStorage.html">Memory Storage Support</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/SLGUserGuide.html">Synthetic Load Generator</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html">Erasure Coding</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HDFSDiskbalancer.html">Disk Balancer</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HdfsUpgradeDomain.html">Upgrade Domain</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HdfsDataNodeAdminGuide.html">DataNode Admin</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs-rbf/HDFSRouterFederation.html">Router Federation</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/HdfsProvidedStorage.html">Provided Storage</a>
</li>
</ul>
<h5>MapReduce</h5>
<ul>
<li class="none">
<a href="../../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html">Tutorial</a>
</li>
<li class="none">
<a href="../../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapredCommands.html">Commands Reference</a>
</li>
<li class="none">
<a href="../../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduce_Compatibility_Hadoop1_Hadoop2.html">Compatibility with 1.x</a>
</li>
<li class="none">
<a href="../../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/EncryptedShuffle.html">Encrypted Shuffle</a>
</li>
<li class="none">
<a href="../../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/PluggableShuffleAndPluggableSort.html">Pluggable Shuffle/Sort</a>
</li>
<li class="none">
<a href="../../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/DistributedCacheDeploy.html">Distributed Cache Deploy</a>
</li>
<li class="none">
<a href="../../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/SharedCacheSupport.html">Support for YARN Shared Cache</a>
</li>
</ul>
<h5>MapReduce REST APIs</h5>
<ul>
<li class="none">
<a href="../../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapredAppMasterRest.html">MR Application Master</a>
</li>
<li class="none">
<a href="../../../hadoop-mapreduce-client/hadoop-mapreduce-client-hs/HistoryServerRest.html">MR History Server</a>
</li>
</ul>
<h5>YARN</h5>
<ul>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/YARN.html">Architecture</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/YarnCommands.html">Commands Reference</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/CapacityScheduler.html">Capacity Scheduler</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/FairScheduler.html">Fair Scheduler</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/ResourceManagerRestart.html">ResourceManager Restart</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/ResourceManagerHA.html">ResourceManager HA</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/ResourceModel.html">Resource Model</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/NodeLabel.html">Node Labels</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/NodeAttributes.html">Node Attributes</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/WebApplicationProxy.html">Web Application Proxy</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/TimelineServer.html">Timeline Server</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html">Timeline Service V.2</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/WritingYarnApplications.html">Writing YARN Applications</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/YarnApplicationSecurity.html">YARN Application Security</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/NodeManager.html">NodeManager</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/DockerContainers.html">Running Applications in Docker Containers</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/RuncContainers.html">Running Applications in runC Containers</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/NodeManagerCgroups.html">Using CGroups</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/SecureContainer.html">Secure Containers</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/ReservationSystem.html">Reservation System</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/GracefulDecommission.html">Graceful Decommission</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/OpportunisticContainers.html">Opportunistic Containers</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/Federation.html">YARN Federation</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/SharedCache.html">Shared Cache</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/UsingGpus.html">Using GPU</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/UsingFPGA.html">Using FPGA</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/PlacementConstraints.html">Placement Constraints</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/YarnUI2.html">YARN UI2</a>
</li>
</ul>
<h5>YARN REST APIs</h5>
<ul>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/WebServicesIntro.html">Introduction</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/ResourceManagerRest.html">Resource Manager</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/NodeManagerRest.html">Node Manager</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/TimelineServer.html#Timeline_Server_REST_API_v1">Timeline Server</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html#Timeline_Service_v.2_REST_API">Timeline Service V.2</a>
</li>
</ul>
<h5>YARN Service</h5>
<ul>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/yarn-service/Overview.html">Overview</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/yarn-service/QuickStart.html">QuickStart</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/yarn-service/Concepts.html">Concepts</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/yarn-service/YarnServiceAPI.html">Yarn Service API</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/yarn-service/ServiceDiscovery.html">Service Discovery</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-site/yarn-service/SystemServices.html">System Services</a>
</li>
</ul>
<h5>Hadoop Compatible File Systems</h5>
<ul>
<li class="none">
<a href="../../../hadoop-aliyun/tools/hadoop-aliyun/index.html">Aliyun OSS</a>
</li>
<li class="none">
<a href="../../../hadoop-aws/tools/hadoop-aws/index.html">Amazon S3</a>
</li>
<li class="none">
<a href="../../../hadoop-azure/index.html">Azure Blob Storage</a>
</li>
<li class="none">
<a href="../../../hadoop-azure-datalake/index.html">Azure Data Lake Storage</a>
</li>
<li class="none">
<a href="../../../hadoop-cos/cloud-storage/index.html">Tencent COS</a>
</li>
<li class="none">
<a href="../../../hadoop-huaweicloud/cloud-storage/index.html">Huaweicloud OBS</a>
</li>
</ul>
<h5>Auth</h5>
<ul>
<li class="none">
<a href="../../../hadoop-auth/index.html">Overview</a>
</li>
<li class="none">
<a href="../../../hadoop-auth/Examples.html">Examples</a>
</li>
<li class="none">
<a href="../../../hadoop-auth/Configuration.html">Configuration</a>
</li>
<li class="none">
<a href="../../../hadoop-auth/BuildingIt.html">Building</a>
</li>
</ul>
<h5>Tools</h5>
<ul>
<li class="none">
<a href="../../../hadoop-streaming/HadoopStreaming.html">Hadoop Streaming</a>
</li>
<li class="none">
<a href="../../../hadoop-archives/HadoopArchives.html">Hadoop Archives</a>
</li>
<li class="none">
<a href="../../../hadoop-archive-logs/HadoopArchiveLogs.html">Hadoop Archive Logs</a>
</li>
<li class="none">
<a href="../../../hadoop-distcp/DistCp.html">DistCp</a>
</li>
<li class="none">
<a href="../../../hadoop-federation-balance/HDFSFederationBalance.html">HDFS Federation Balance</a>
</li>
<li class="none">
<a href="../../../hadoop-gridmix/GridMix.html">GridMix</a>
</li>
<li class="none">
<a href="../../../hadoop-rumen/Rumen.html">Rumen</a>
</li>
<li class="none">
<a href="../../../hadoop-resourceestimator/ResourceEstimator.html">Resource Estimator Service</a>
</li>
<li class="none">
<a href="../../../hadoop-sls/SchedulerLoadSimulator.html">Scheduler Load Simulator</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/Benchmarking.html">Hadoop Benchmarking</a>
</li>
<li class="none">
<a href="../../../hadoop-dynamometer/Dynamometer.html">Dynamometer</a>
</li>
</ul>
<h5>Reference</h5>
<ul>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/release/">Changelog and Release Notes</a>
</li>
<li class="none">
<a href="../../../api/index.html">Java API docs</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/UnixShellAPI.html">Unix Shell API</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/Metrics.html">Metrics</a>
</li>
</ul>
<h5>Configuration</h5>
<ul>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/core-default.xml">core-default.xml</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs/hdfs-default.xml">hdfs-default.xml</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-hdfs-rbf/hdfs-rbf-default.xml">hdfs-rbf-default.xml</a>
</li>
<li class="none">
<a href="../../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/mapred-default.xml">mapred-default.xml</a>
</li>
<li class="none">
<a href="../../../hadoop-yarn/hadoop-yarn-common/yarn-default.xml">yarn-default.xml</a>
</li>
<li class="none">
<a href="../../../hadoop-kms/kms-default.html">kms-default.xml</a>
</li>
<li class="none">
<a href="../../../hadoop-hdfs-httpfs/httpfs-default.html">httpfs-default.xml</a>
</li>
<li class="none">
<a href="../../../hadoop-project-dist/hadoop-common/DeprecatedProperties.html">Deprecated Properties</a>
</li>
</ul>
<a href="http://maven.apache.org/" title="Built by Maven" class="poweredBy">
<img alt="Built by Maven" src="../../images/logos/maven-feather.png"/>
</a>
</div>
</div>
<div id="bodyColumn">
<div id="contentBox">
<!---
Licensed 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. See accompanying LICENSE file.
-->
<h1>Troubleshooting</h1>
<ul>
<li><a href="#Introduction"> Introduction</a></li>
<li><a href="#Classpath_Setup"> Classpath Setup</a>
<ul>
<li><a href="#ClassNotFoundException:_org.apache.hadoop.fs.s3a.S3AFileSystem">ClassNotFoundException: org.apache.hadoop.fs.s3a.S3AFileSystem</a></li>
<li><a href="#ClassNotFoundException:_com.amazonaws.services.s3.AmazonS3Client">ClassNotFoundException: com.amazonaws.services.s3.AmazonS3Client</a></li>
<li><a href="#java.lang.NoSuchMethodError_referencing_a_com.amazonaws_class">java.lang.NoSuchMethodError referencing a com.amazonaws class</a></li>
<li><a href="#java.lang.NoSuchMethodError_referencing_an_org.apache.hadoop_class">java.lang.NoSuchMethodError referencing an org.apache.hadoop class</a></li>
<li><a href="#java.lang.NoClassDefFoundError:_org.2Fwildfly.2Fopenssl.2FOpenSSLProvider">java.lang.NoClassDefFoundError: org/wildfly/openssl/OpenSSLProvider</a></li></ul></li>
<li><a href="#Authentication_Failure"> Authentication Failure</a>
<ul>
<li><a href="#Authentication_failure_.E2.80.9CThe_Filesystem_URI_contains_login_details..E2.80.9D">Authentication failure &#x201c;The Filesystem URI contains login details.&#x201d;</a></li>
<li><a href="#Authentication_failure_due_to_clock_skew">Authentication failure due to clock skew</a></li>
<li><a href="#a.E2.80.9CBad_Request.E2.80.9D_exception_when_working_with_AWS_S3_Frankfurt.2C_Seoul.2C_or_other_.E2.80.9CV4.E2.80.9D_endpoint"> &#x201c;Bad Request&#x201d; exception when working with AWS S3 Frankfurt, Seoul, or other &#x201c;V4&#x201d; endpoint</a></li>
<li><a href="#a.E2.80.9CAuthorization_Header_is_Malformed.E2.80.9D.28400.29_exception_when_PrivateLink_URL_is_used_in_.E2.80.9Cfs.s3a.endpoint.E2.80.9D"> &#x201c;Authorization Header is Malformed&#x201d;(400) exception when PrivateLink URL is used in &#x201c;fs.s3a.endpoint&#x201d;</a></li>
<li><a href="#Class_does_not_implement_AWSCredentialsProvider">Class does not implement AWSCredentialsProvider</a></li></ul></li>
<li><a href="#a.E2.80.9CThe_security_token_included_in_the_request_is_invalid.E2.80.9D"> &#x201c;The security token included in the request is invalid&#x201d;</a>
<ul>
<li><a href="#AccessDeniedException_.E2.80.9CThe_AWS_Access_Key_Id_you_provided_does_not_exist_in_our_records..E2.80.9D"> AccessDeniedException &#x201c;The AWS Access Key Id you provided does not exist in our records.&#x201d;</a></li>
<li><a href="#AccessDeniedException_All_access_to_this_object_has_been_disabled"> AccessDeniedException All access to this object has been disabled</a></li>
<li><a href="#AccessDeniedException_.E2.80.9CAccess_denied.E2.80.9D_when_trying_to_manipulate_data"> AccessDeniedException &#x201c;Access denied&#x201d; when trying to manipulate data</a></li>
<li><a href="#AccessDeniedException_in_rename.2C_.E2.80.9CMultiObjectDeleteException:_One_or_more_objects_could_not_be_deleted.E2.80.9D"> AccessDeniedException in rename, &#x201c;MultiObjectDeleteException: One or more objects could not be deleted&#x201d;</a></li>
<li><a href="#AccessDeniedException_when_using_SSE-KMS"> AccessDeniedException when using SSE-KMS</a></li>
<li><a href="#AccessDeniedException_when_using_a_.E2.80.9CRequester_Pays.E2.80.9D_enabled_bucket">AccessDeniedException when using a &#x201c;Requester Pays&#x201d; enabled bucket</a></li>
<li><a href="#AccessDeniedException_.E2.80.9CInvalidObjectState.E2.80.9D_when_trying_to_read_files">AccessDeniedException &#x201c;InvalidObjectState&#x201d; when trying to read files</a></li>
<li><a href="#a.E2.80.9CUnable_to_find_a_region_via_the_region_provider_chain..E2.80.9D_when_using_session_credentials."> &#x201c;Unable to find a region via the region provider chain.&#x201d; when using session credentials.</a></li></ul></li>
<li><a href="#Connectivity_Problems"> Connectivity Problems</a>
<ul>
<li><a href="#Error_.E2.80.9CThe_bucket_you_are_attempting_to_access_must_be_addressed_using_the_specified_endpoint.E2.80.9D"> Error &#x201c;The bucket you are attempting to access must be addressed using the specified endpoint&#x201d;</a></li>
<li><a href="#Unable_to_find_a_region_via_the_region_provider_chain"> Unable to find a region via the region provider chain</a></li>
<li><a href="#a.E2.80.9CTimeout_waiting_for_connection_from_pool.E2.80.9D_when_writing_data"> &#x201c;Timeout waiting for connection from pool&#x201d; when writing data</a></li>
<li><a href="#a.E2.80.9CTimeout_waiting_for_connection_from_pool.E2.80.9D_when_reading_data">&#x201c;Timeout waiting for connection from pool&#x201d; when reading data</a></li>
<li><a href="#NoHttpResponseException">NoHttpResponseException</a></li>
<li><a href="#Out_of_heap_memory_when_writing_with_via_Fast_Upload">Out of heap memory when writing with via Fast Upload</a></li>
<li><a href="#MultiObjectDeleteException_during_delete_or_rename_of_files">MultiObjectDeleteException during delete or rename of files</a></li>
<li><a href="#a.E2.80.9CFailed_to_Sanitize_XML_document.E2.80.9D">&#x201c;Failed to Sanitize XML document&#x201d;</a></li>
<li><a href="#JSON_Parse_Error_from_AWS_SDK">JSON Parse Error from AWS SDK</a></li></ul></li>
<li><a href="#File_System_Semantics">File System Semantics</a>
<ul>
<li><a href="#File_not_visible.2Fsaved">File not visible/saved</a></li>
<li><a href="#File_flush.28.29_calls_do_not_save_data_to_S3">File flush() calls do not save data to S3</a></li>
<li><a href="#UnsupportedOperationException_.E2.80.9CS3A_streams_are_not_Syncable._See_HADOOP-17597..E2.80.9D"> UnsupportedOperationException &#x201c;S3A streams are not Syncable. See HADOOP-17597.&#x201d;</a></li>
<li><a href="#RemoteFileChangedException_and_read-during-overwrite">RemoteFileChangedException and read-during-overwrite</a></li>
<li><a href="#Rename_not_behaving_as_.E2.80.9Cexpected.E2.80.9D">Rename not behaving as &#x201c;expected&#x201d;</a></li></ul></li>
<li><a href="#S3_Server_Side_Encryption"> S3 Server Side Encryption</a>
<ul>
<li><a href="#AWSS3IOException_KMS.NotFoundException_.E2.80.9CInvalid_arn.E2.80.9D_when_using_SSE-KMS">AWSS3IOException KMS.NotFoundException &#x201c;Invalid arn&#x201d; when using SSE-KMS</a></li>
<li><a href="#Using_SSE-C_.E2.80.9CBad_Request.E2.80.9D">Using SSE-C &#x201c;Bad Request&#x201d;</a></li></ul></li>
<li><a href="#S3_Client_Side_Encryption"> S3 Client Side Encryption</a>
<ul>
<li><a href="#Instruction_file_not_found_for_S3_object">Instruction file not found for S3 object</a></li>
<li><a href="#CSE-KMS_method_requires_KMS_key_ID">CSE-KMS method requires KMS key ID</a></li>
<li><a href="#com.amazonaws.services.kms.model.IncorrectKeyException_The_key_ID_in_the_request_does_not_identify_a_CMK_that_can_perform_this_operation.">com.amazonaws.services.kms.model.IncorrectKeyException The key ID in the request does not identify a CMK that can perform this operation.</a></li>
<li><a href="#com.amazonaws.services.kms.model.NotFoundException_key.2F.3CKMS_KEY_ID.3E_does_not_exist">com.amazonaws.services.kms.model.NotFoundException key/&lt;KMS_KEY_ID&gt; does not exist</a></li>
<li><a href="#Unable_to_perform_range_get_request:_Range_get_support_has_been_disabled">Unable to perform range get request: Range get support has been disabled</a></li>
<li><a href="#WARNING:_Range_gets_do_not_provide_authenticated_encryption_properties_even_when_used_with_an_authenticated_mode_.28AES-GCM.29.">WARNING: Range gets do not provide authenticated encryption properties even when used with an authenticated mode (AES-GCM).</a></li>
<li><a href="#WARNING:_If_you_don.E2.80.99t_have_objects_encrypted_with_these_legacy_modes.2C_you_should_disable_support_for_them_to_enhance_security.">WARNING: If you don&#x2019;t have objects encrypted with these legacy modes, you should disable support for them to enhance security.</a></li>
<li><a href="#com.amazonaws.services.kms.model.InvalidKeyUsageException:_You_cannot_generate_a_data_key_with_an_asymmetric_CMK">com.amazonaws.services.kms.model.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK</a></li>
<li><a href="#com.amazonaws.services.kms.model.NotFoundException:_Invalid_keyId">com.amazonaws.services.kms.model.NotFoundException: Invalid keyId</a></li>
<li><a href="#com.amazonaws.services.kms.model.AWSKMSException:_User:_.3CUser_ARN.3E_is_not_authorized_to_perform_:_kms_:GenerateDataKey_on_resource:_.3CKEY_ID.3E">com.amazonaws.services.kms.model.AWSKMSException: User: &lt;User_ARN&gt; is not authorized to perform : kms :GenerateDataKey on resource: &lt;KEY_ID&gt;</a></li>
<li><a href="#Message_appears_in_logs_.E2.80.9CNot_all_bytes_were_read_from_the_S3ObjectInputStream.E2.80.9D"> Message appears in logs &#x201c;Not all bytes were read from the S3ObjectInputStream&#x201d;</a></li>
<li><a href="#PathIOException_Number_of_parts_in_multipart_upload_exceeded."> PathIOException Number of parts in multipart upload exceeded.</a></li>
<li><a href="#UnknownStoreException_Bucket_does_not_exist."> UnknownStoreException Bucket does not exist.</a></li></ul></li>
<li><a href="#S3Guard_Errors"> S3Guard Errors</a>
<ul>
<li><a href="#S3Guard_is_no_longer_needed.2Fsupported"> S3Guard is no longer needed/supported</a></li></ul></li>
<li><a href="#Other_Errors"> Other Errors</a>
<ul>
<li><a href="#SdkClientException_Unable_to_verify_integrity_of_data_upload"> SdkClientException Unable to verify integrity of data upload</a></li>
<li><a href="#AWSS3IOException_The_Content-MD5_you_specified_did_not_match_what_we_received">AWSS3IOException The Content-MD5 you specified did not match what we received</a></li>
<li><a href="#When_writing_data:_.E2.80.9Cjava.io.FileNotFoundException:_Completing_multi-part_upload.E2.80.9D">When writing data: &#x201c;java.io.FileNotFoundException: Completing multi-part upload&#x201d;</a></li>
<li><a href="#Application_hangs_after_reading_a_number_of_files">Application hangs after reading a number of files</a></li>
<li><a href="#Issue:_when_writing_data.2C_HTTP_Exceptions_logged_at_info_from_AmazonHttpClient">Issue: when writing data, HTTP Exceptions logged at info from AmazonHttpClient</a></li>
<li><a href="#AWSBadRequestException_IllegalLocationConstraintException.2FThe_unspecified_location_constraint_is_incompatible">AWSBadRequestException IllegalLocationConstraintException/The unspecified location constraint is incompatible</a></li></ul></li>
<li><a href="#Best_Practises"> Best Practises</a>
<ul>
<li><a href="#Enabling_low-level_logging"> Enabling low-level logging</a></li>
<li><a href="#Enable_S3_Server-side_Logging"> Enable S3 Server-side Logging</a></li>
<li><a href="#Reducing_failures_by_configuring_retry_policy"> Reducing failures by configuring retry policy</a></li>
<li><a href="#Tuning_AWS_request_timeouts"> Tuning AWS request timeouts</a></li></ul></li>
<li><a href="#SDK_Upgrade_Warnings"> SDK Upgrade Warnings</a>
<ul>
<li><a href="#Directly_referencing_AWS_SDK_V1_credential_provider"> Directly referencing AWS SDK V1 credential provider</a></li>
<li><a href="#getAmazonS3ClientForTesting.28.29_will_be_removed"> getAmazonS3ClientForTesting() will be removed</a></li>
<li><a href="#a"></a></li>
<li><a href="#Custom_credential_providers_used_in_delegation_tokens_binding_classes_will_need_to_be_updated">Custom credential providers used in delegation tokens binding classes will need to be updated</a></li>
<li><a href="#a"></a></li>
<li><a href="#The_signer_interface_has_changed_in_AWS_SDK_V2.2C_custom_signers_will_need_to_be_updated">The signer interface has changed in AWS SDK V2, custom signers will need to be updated</a></li>
<li><a href="#a"></a></li>
<li><a href="#getObjectMetadata.28.29_called._This_operation_and_it.27s_response_will_be_changed">getObjectMetadata() called. This operation and it's response will be changed</a></li></ul></li></ul>
<section>
<h2><a name="Introduction"></a><a name="introduction"></a> Introduction</h2>
<p>Common problems working with S3 are:</p>
<ol style="list-style-type: decimal">
<li><a href="#classpath">Classpath setup</a></li>
<li><a href="#authentication">Authentication</a></li>
<li><a href="#access_denied">Access Denial</a></li>
<li><a href="#connectivity">Connectivity Problems</a></li>
<li><a href="#semantics">File System Semantics</a></li>
<li><a href="#encryption">Encryption</a></li>
<li><a href="#other">Other Errors</a></li>
<li><a href="#upgrade_warnings">SDK Upgrade Warnings</a></li>
</ol>
<p>This document also includes some <a href="#best">best pactises</a> to aid troubleshooting.</p>
<p>Troubleshooting IAM Assumed Roles is covered in its <a href="assumed_roles.html#troubleshooting">specific documentation</a>.</p></section><section>
<h2><a name="Classpath_Setup"></a><a name="classpath"></a> Classpath Setup</h2>
<p>Classpath is usually the first problem. For the S3A filesystem client, you need the Hadoop-specific filesystem clients, the very same AWS SDK library which Hadoop was built against, and any dependent libraries compatible with Hadoop and the specific JVM.</p>
<p>The classpath must be set up for the process talking to S3: if this is code running in the Hadoop cluster, the JARs must be on that classpath. That includes <code>distcp</code> and the <code>hadoop fs</code> command.</p>
<p><b>Critical:</b> <i>Do not attempt to &#x201c;drop in&#x201d; a newer version of the AWS SDK than that which the Hadoop version was built with</i> Whatever problem you have, changing the AWS SDK version will not fix things, only change the stack traces you see.</p>
<p>Similarly, don&#x2019;t try and mix a <code>hadoop-aws</code> JAR from one Hadoop release with that of any other. The JAR must be in sync with <code>hadoop-common</code> and some other Hadoop JARs.</p>
<p><i>Randomly changing hadoop- and aws- JARs in the hope of making a problem &#x201c;go away&#x201d; or to gain access to a feature you want, will not lead to the outcome you desire.</i></p>
<p>Tip: you can use <a class="externalLink" href="http://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws">mvnrepository</a> to determine the dependency version requirements of a specific <code>hadoop-aws</code> JAR published by the ASF.</p><section>
<h3><a name="ClassNotFoundException:_org.apache.hadoop.fs.s3a.S3AFileSystem"></a><code>ClassNotFoundException: org.apache.hadoop.fs.s3a.S3AFileSystem</code></h3>
<p>These are Hadoop filesystem client classes, found in the <code>hadoop-aws</code> JAR. An exception reporting this class as missing means that this JAR is not on the classpath.</p></section><section>
<h3><a name="ClassNotFoundException:_com.amazonaws.services.s3.AmazonS3Client"></a><code>ClassNotFoundException: com.amazonaws.services.s3.AmazonS3Client</code></h3>
<p>(or other <code>com.amazonaws</code> class.)</p>
<p>This means that the <code>aws-java-sdk-bundle.jar</code> JAR is not on the classpath: add it.</p></section><section>
<h3><a name="java.lang.NoSuchMethodError_referencing_a_com.amazonaws_class"></a><code>java.lang.NoSuchMethodError</code> referencing a <code>com.amazonaws</code> class</h3>
<p>This can be triggered by incompatibilities between the AWS SDK on the classpath and the version which Hadoop was compiled with.</p>
<p>The AWS SDK JARs change their signature enough between releases that the only way to safely update the AWS SDK version is to recompile Hadoop against the later version.</p>
<p>The sole fix is to use the same version of the AWS SDK with which Hadoop was built.</p>
<p>This can also be caused by having more than one version of an AWS SDK JAR on the classpath. If the full <code>aws-java-sdk-bundle&lt;</code> JAR is on the classpath, do not add any of the <code>aws-sdk-</code> JARs.</p></section><section>
<h3><a name="java.lang.NoSuchMethodError_referencing_an_org.apache.hadoop_class"></a><code>java.lang.NoSuchMethodError</code> referencing an <code>org.apache.hadoop</code> class</h3>
<p>This happens if the <code>hadoop-aws</code> and <code>hadoop-common</code> JARs are out of sync. You can&#x2019;t mix them around: they have to have exactly matching version numbers.</p></section><section>
<h3><a name="java.lang.NoClassDefFoundError:_org.2Fwildfly.2Fopenssl.2FOpenSSLProvider"></a><code>java.lang.NoClassDefFoundError: org/wildfly/openssl/OpenSSLProvider</code></h3>
<p>This happens when <a href="performance.html#openssl">OpenSSL performance acceleration</a> has been configured by setting <code>fs.s3a.ssl.channel.mode</code> to <code>openssl</code> but the wildfly JAR is not on the classpath.</p>
<p>Fixes: * Add it to the classpath * Use a different channel mode, including <code>default</code>, which will revert to the JVM SSL implementation when the wildfly or native openssl libraries cannot be loaded.</p></section></section><section>
<h2><a name="Authentication_Failure"></a><a name="authentication"></a> Authentication Failure</h2>
<p>If Hadoop cannot authenticate with the S3 service endpoint, the client retries a number of times before eventually failing. When it finally gives up, it will report a message about signature mismatch:</p>
<div class="source">
<div class="source">
<pre>com.amazonaws.services.s3.model.AmazonS3Exception:
The request signature we calculated does not match the signature you provided.
Check your key and signing method.
(Service: Amazon S3; Status Code: 403; Error Code: SignatureDoesNotMatch,
</pre></div></div>
<p>The likely cause is that you either have the wrong credentials or somehow the credentials were not readable on the host attempting to read or write the S3 Bucket.</p>
<p>Enabling debug logging for the package <code>org.apache.hadoop.fs.s3a</code> can help provide more information.</p>
<p>The most common cause is that you have the wrong credentials for any of the current authentication mechanism(s) &#x2014;or somehow the credentials were not readable on the host attempting to read or write the S3 Bucket. However, there are a couple of system configuration problems (JVM version, system clock) which also need to be checked.</p>
<p>Most common: there&#x2019;s an error in the configuration properties.</p>
<ol style="list-style-type: decimal">
<li>
<p>Make sure that the name of the bucket is the correct one. That is: check the URL.</p>
</li>
<li>
<p>If using a private S3 server, make sure endpoint in <code>fs.s3a.endpoint</code> has been set to this server -and that the client is not accidentally trying to authenticate with the public Amazon S3 service.</p>
</li>
<li>
<p>Make sure the property names are correct. For S3A, they are <code>fs.s3a.access.key</code> and <code>fs.s3a.secret.key</code> &#x2014;you cannot just copy the S3N properties and replace <code>s3n</code> with <code>s3a</code>.</p>
</li>
<li>
<p>Make sure the properties are visible to the process attempting to talk to the object store. Placing them in <code>core-site.xml</code> is the standard mechanism.</p>
</li>
<li>
<p>If using session authentication, the session may have expired. Generate a new session token and secret.</p>
</li>
<li>
<p>If using environment variable-based authentication, make sure that the relevant variables are set in the environment in which the process is running.</p>
</li>
</ol>
<p>The standard first step is: try to use the AWS command line tools with the same credentials, through a command such as:</p>
<div class="source">
<div class="source">
<pre>hadoop fs -ls s3a://my-bucket/
</pre></div></div>
<p>Note the trailing &#x201c;/&#x201d; here; without that the shell thinks you are trying to list your home directory under the bucket, which will only exist if explicitly created.</p>
<p>Finally, if you set the environment variables, you can take advantage of S3A&#x2019;s support of environment-variable authentication by attempting the same ls operation. That is: unset the <code>fs.s3a</code> secrets and rely on the environment variables.</p><section>
<h3><a name="Authentication_failure_.E2.80.9CThe_Filesystem_URI_contains_login_details..E2.80.9D"></a>Authentication failure &#x201c;The Filesystem URI contains login details.&#x201d;</h3>
<div class="source">
<div class="source">
<pre>The Filesystem URI contains login details. This authentication mechanism is no longer supported.
</pre></div></div>
<p>The S3A connector no longer supports the dangerously insecure mechanism of passing login details within the S3A URLs.</p>
<p>Fix: use a more secure mechanism to pass down the secrets.</p></section><section>
<h3><a name="Authentication_failure_due_to_clock_skew"></a>Authentication failure due to clock skew</h3>
<p>The timestamp is used in signing to S3, so as to defend against replay attacks. If the system clock is too far behind <i>or ahead</i> of Amazon&#x2019;s, requests will be rejected.</p>
<p>This can surface as the situation where read requests are allowed, but operations which write to the bucket are denied.</p>
<p>Check the system clock.</p></section><section>
<h3><a name="a.E2.80.9CBad_Request.E2.80.9D_exception_when_working_with_AWS_S3_Frankfurt.2C_Seoul.2C_or_other_.E2.80.9CV4.E2.80.9D_endpoint"></a><a name="bad_request"></a> &#x201c;Bad Request&#x201d; exception when working with AWS S3 Frankfurt, Seoul, or other &#x201c;V4&#x201d; endpoint</h3>
<p>S3 Frankfurt and Seoul <i>only</i> support <a class="externalLink" href="http://docs.aws.amazon.com/AmazonS3/latest/API/sig-v4-authenticating-requests.html">the V4 authentication API</a>.</p>
<p>Requests using the V2 API will be rejected with 400 <code>Bad Request</code></p>
<div class="source">
<div class="source">
<pre>$ bin/hadoop fs -ls s3a://frankfurt/
WARN s3a.S3AFileSystem: Client: Amazon S3 error 400: 400 Bad Request; Bad Request (retryable)
com.amazonaws.services.s3.model.AmazonS3Exception: Bad Request (Service: Amazon S3;
Status Code: 400; Error Code: 400 Bad Request; Request ID: 923C5D9E75E44C06),
S3 Extended Request ID: HDwje6k+ANEeDsM6aJ8+D5gUmNAMguOk2BvZ8PH3g9z0gpH+IuwT7N19oQOnIr5CIx7Vqb/uThE=
at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182)
at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770)
at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
at com.amazonaws.services.s3.AmazonS3Client.headBucket(AmazonS3Client.java:1107)
at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1070)
at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:307)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:284)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2793)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:101)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2830)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2812)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:389)
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:356)
at org.apache.hadoop.fs.shell.PathData.expandAsGlob(PathData.java:325)
at org.apache.hadoop.fs.shell.Command.expandArgument(Command.java:235)
at org.apache.hadoop.fs.shell.Command.expandArguments(Command.java:218)
at org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:103)
at org.apache.hadoop.fs.shell.Command.run(Command.java:165)
at org.apache.hadoop.fs.FsShell.run(FsShell.java:315)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90)
at org.apache.hadoop.fs.FsShell.main(FsShell.java:373)
ls: doesBucketExist on frankfurt-new: com.amazonaws.services.s3.model.AmazonS3Exception:
Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request;
</pre></div></div>
<p>This happens when trying to work with any S3 service which only supports the &#x201c;V4&#x201d; signing API &#x2014;but the client is configured to use the default S3 service endpoint.</p>
<p>The S3A client needs to be given the endpoint to use via the <code>fs.s3a.endpoint</code> property.</p>
<p>As an example, the endpoint for S3 Frankfurt is <code>s3.eu-central-1.amazonaws.com</code>:</p>
<div class="source">
<div class="source">
<pre>&lt;property&gt;
&lt;name&gt;fs.s3a.endpoint&lt;/name&gt;
&lt;value&gt;s3.eu-central-1.amazonaws.com&lt;/value&gt;
&lt;/property&gt;
</pre></div></div>
</section><section>
<h3><a name="a.E2.80.9CAuthorization_Header_is_Malformed.E2.80.9D.28400.29_exception_when_PrivateLink_URL_is_used_in_.E2.80.9Cfs.s3a.endpoint.E2.80.9D"></a><a name="AuthorizationHeaderMalformed"></a> &#x201c;Authorization Header is Malformed&#x201d;(400) exception when PrivateLink URL is used in &#x201c;fs.s3a.endpoint&#x201d;</h3>
<p>When <a class="externalLink" href="https://docs.aws.amazon.com/AmazonS3/latest/userguide/privatelink-interface-endpoints.html">PrivateLink</a> URL is used instead of standard s3a endpoint, it returns &#x201c;authorization header is malformed&#x201d; exception. So, if we set fs.s3a.endpoint=bucket.vpce -&lt;some_string&gt;.s3.ca-central-1.vpce.amazonaws.com and make s3 calls we get:</p>
<div class="source">
<div class="source">
<pre>com.amazonaws.services.s3.model.AmazonS3Exception: The authorization header is malformed; the region 'vpce' is wrong; expecting 'ca-central-1'
(Service: Amazon S3; Status Code: 400; Error Code: AuthorizationHeaderMalformed; Request ID: req-id; S3 Extended Request ID: req-id-2), S3 Extended Request ID: req-id-2:AuthorizationHeaderMalformed: The authorization
header is malformed; the region 'vpce' is wrong; expecting 'ca-central-1' (Service: Amazon S3; Status Code: 400; Error Code: AuthorizationHeaderMalformed; Request ID: req-id;
</pre></div></div>
<p>Cause:</p>
<p>Since, endpoint parsing is done in a way that it assumes the AWS S3 region would be the 2nd component of the <code>fs.s3a.endpoint</code> URL delimited by &#x201c;.&#x201d;, in case of PrivateLink URL, it can&#x2019;t figure out the region and throws an authorization exception. Thus, to add support to using PrivateLink URLs we use <code>fs.s3a.endpoint.region</code> to set the region and bypass this parsing of <code>fs.s3a.endpoint</code>, in the case shown above to make it work we&#x2019;ll set the AWS S3 region as <code>ca-central-1</code>.</p>
<div class="source">
<div class="source">
<pre>&lt;property&gt;
&lt;name&gt;fs.s3a.endpoint.region&lt;/name&gt;
&lt;value&gt;ca-central-1&lt;/value&gt;
&lt;/property&gt;
</pre></div></div>
</section><section>
<h3><a name="Class_does_not_implement_AWSCredentialsProvider"></a><code>Class does not implement AWSCredentialsProvider</code></h3>
<p>A credential provider listed in <code>fs.s3a.aws.credentials.provider</code> does not implement the interface <code>com.amazonaws.auth.AWSCredentialsProvider</code>.</p>
<div class="source">
<div class="source">
<pre> Cause: java.lang.RuntimeException: java.io.IOException: Class class com.amazonaws.auth.EnvironmentVariableCredentialsProvider does not implement AWSCredentialsProvider
at org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:686)
at org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:621)
at org.apache.spark.sql.hive.client.HiveClientImpl.newState(HiveClientImpl.scala:219)
at org.apache.spark.sql.hive.client.HiveClientImpl.&lt;init&gt;(HiveClientImpl.scala:126)
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at org.apache.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:306)
at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:433)
...
Cause: java.io.IOException: Class class com.amazonaws.auth.EnvironmentVariableCredentialsProvider does not implement AWSCredentialsProvider
at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:722)
at org.apache.hadoop.fs.s3a.S3AUtils.buildAWSProviderList(S3AUtils.java:687)
at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:620)
at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:673)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:414)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3462)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:171)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3522)
at org.apache.hadoop.fs.FileSystem$Cache.getUnique(FileSystem.java:3496)
at org.apache.hadoop.fs.FileSystem.newInstance(FileSystem.java:591)
</pre></div></div>
<p>There&#x2019;s two main causes</p>
<ol style="list-style-type: decimal">
<li>A class listed there is not an implementation of the interface. Fix: review the settings and correct as appropriate.</li>
<li>A class listed there does implement the interface, but it has been loaded in a different classloader, so the JVM does not consider it to be an implementation. Fix: learn the entire JVM classloader model and see if you can then debug it. Tip: having both the AWS Shaded SDK and individual AWS SDK modules on your classpath may be a cause of this.</li>
</ol>
<p>If you see this and you are trying to use the S3A connector with Spark, then the cause can be that the isolated classloader used to load Hive classes is interfering with the S3A connector&#x2019;s dynamic loading of <code>com.amazonaws</code> classes. To fix this, declare that the classes in the aws SDK are loaded from the same classloader which instantiated the S3A FileSystem instance:</p>
<div class="source">
<div class="source">
<pre>spark.sql.hive.metastore.sharedPrefixes com.amazonaws.
</pre></div></div>
</section></section><section>
<h2><a name="a.E2.80.9CThe_security_token_included_in_the_request_is_invalid.E2.80.9D"></a><a name="access_denied"></a> &#x201c;The security token included in the request is invalid&#x201d;</h2>
<p>You are trying to use session/temporary credentials and the session token supplied is considered invalid.</p>
<p>This can surface if your configuration is setting the <code>fs.s3a.secret.key</code>, <code>fs.s3a.access.key</code> and <code>fs.s3a.session.key</code> correctly, but the AWS credential provider list set in <code>AWS_CREDENTIALS_PROVIDER</code> does not include <code>org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider</code>.</p><section>
<h3><a name="AccessDeniedException_.E2.80.9CThe_AWS_Access_Key_Id_you_provided_does_not_exist_in_our_records..E2.80.9D"></a><a name="access_denied_unknown-ID"></a> AccessDeniedException &#x201c;The AWS Access Key Id you provided does not exist in our records.&#x201d;</h3>
<p>The value of <code>fs.s3a.access.key</code> does not match a known access key ID. It may be mistyped, or the access key may have been deleted by one of the account managers.</p>
<div class="source">
<div class="source">
<pre>java.nio.file.AccessDeniedException: bucket: doesBucketExist on bucket:
com.amazonaws.services.s3.model.AmazonS3Exception:
The AWS Access Key Id you provided does not exist in our records.
(Service: Amazon S3; Status Code: 403; Error Code: InvalidAccessKeyId;
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:214)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111)
at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:260)
at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:314)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:256)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:231)
at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:366)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:302)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:124)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3403)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3371)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:477)
at org.apache.hadoop.fs.contract.AbstractBondedFSContract.init(AbstractBondedFSContract.java:72)
at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.setup(AbstractFSContractTestBase.java:177)
at org.apache.hadoop.fs.s3a.commit.AbstractCommitITest.setup(AbstractCommitITest.java:163)
at org.apache.hadoop.fs.s3a.commit.AbstractITCommitMRJob.setup(AbstractITCommitMRJob.java:129)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:24)
at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception:
The AWS Access Key Id you provided does not exist in our records.
(Service: Amazon S3; Status Code: 403; Error Code: InvalidAccessKeyId;
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1055)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4229)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4176)
at com.amazonaws.services.s3.AmazonS3Client.getAcl(AmazonS3Client.java:3381)
at com.amazonaws.services.s3.AmazonS3Client.getBucketAcl(AmazonS3Client.java:1160)
at com.amazonaws.services.s3.AmazonS3Client.getBucketAcl(AmazonS3Client.java:1150)
at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1266)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$verifyBucketExists$1(S3AFileSystem.java:367)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:109)
... 27 more
</pre></div></div>
</section><section>
<h3><a name="AccessDeniedException_All_access_to_this_object_has_been_disabled"></a><a name="access_denied_disabled"></a> <code>AccessDeniedException</code> All access to this object has been disabled</h3>
<p>Caller has no permission to access the bucket at all.</p>
<div class="source">
<div class="source">
<pre>doesBucketExist on fdsd: java.nio.file.AccessDeniedException: fdsd: doesBucketExist on fdsd:
com.amazonaws.services.s3.model.AmazonS3Exception: All access to this object has been disabled
(Service: Amazon S3; Status Code: 403; Error Code: AllAccessDisabled; Request ID: E6229D7F8134E64F;
S3 Extended Request ID: 6SzVz2t4qa8J2Wxo/oc8yBuB13Mgrn9uMKnxVY0hsBd2kU/YdHzW1IaujpJdDXRDCQRX3f1RYn0=),
S3 Extended Request ID: 6SzVz2t4qa8J2Wxo/oc8yBuB13Mgrn9uMKnxVY0hsBd2kU/YdHzW1IaujpJdDXRDCQRX3f1RYn0=:AllAccessDisabled
All access to this object has been disabled (Service: Amazon S3; Status Code: 403;
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:205)
at org.apache.hadoop.fs.s3a.S3ALambda.once(S3ALambda.java:122)
at org.apache.hadoop.fs.s3a.S3ALambda.lambda$retry$2(S3ALambda.java:233)
at org.apache.hadoop.fs.s3a.S3ALambda.retryUntranslated(S3ALambda.java:288)
at org.apache.hadoop.fs.s3a.S3ALambda.retry(S3ALambda.java:228)
at org.apache.hadoop.fs.s3a.S3ALambda.retry(S3ALambda.java:203)
at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:357)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:293)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3288)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:123)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3337)
at org.apache.hadoop.fs.FileSystem$Cache.getUnique(FileSystem.java:3311)
at org.apache.hadoop.fs.FileSystem.newInstance(FileSystem.java:529)
at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool$BucketInfo.run(S3GuardTool.java:997)
at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.run(S3GuardTool.java:309)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.run(S3GuardTool.java:1218)
at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.main(S3GuardTool.java:1227)
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: All access to this object has been disabled
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1055)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4229)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4176)
at com.amazonaws.services.s3.AmazonS3Client.getAcl(AmazonS3Client.java:3381)
at com.amazonaws.services.s3.AmazonS3Client.getBucketAcl(AmazonS3Client.java:1160)
at com.amazonaws.services.s3.AmazonS3Client.getBucketAcl(AmazonS3Client.java:1150)
at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1266)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$verifyBucketExists$1(S3AFileSystem.java:360)
at org.apache.hadoop.fs.s3a.S3ALambda.once(S3ALambda.java:120)
</pre></div></div>
<p>Check the name of the bucket is correct, and validate permissions for the active user/role.</p></section><section>
<h3><a name="AccessDeniedException_.E2.80.9CAccess_denied.E2.80.9D_when_trying_to_manipulate_data"></a><a name="access_denied_writing"></a> <code>AccessDeniedException</code> &#x201c;Access denied&#x201d; when trying to manipulate data</h3>
<p>Data can be read, but attempts to write data or manipulate the store fail with 403/Access denied.</p>
<p>The bucket may have an access policy which the request does not comply with. or the caller does not have the right to access the data.</p>
<div class="source">
<div class="source">
<pre>java.nio.file.AccessDeniedException: test/: PUT 0-byte object on test/:
com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service: Amazon S3; Status Code: 403;
Error Code: AccessDenied; Request ID: EDC662AD2EEEA33C;
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:210)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:110)
at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:259)
at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:313)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:255)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:230)
at org.apache.hadoop.fs.s3a.S3AFileSystem.createEmptyObject(S3AFileSystem.java:2691)
at org.apache.hadoop.fs.s3a.S3AFileSystem.createFakeDirectory(S3AFileSystem.java:2666)
at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:2030)
at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1965)
at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2305)
</pre></div></div>
<p>In the AWS S3 management console, select the &#x201c;permissions&#x201d; tab for the bucket, then &#x201c;bucket policy&#x201d;.</p>
<p>If there is a bucket access policy, e.g. required encryption headers, then the settings of the s3a client must guarantee the relevant headers are set (e.g. the encryption options match). Note: S3 Default Encryption options are not considered here: if the bucket policy requires AES256 as the encryption policy on PUT requests, then the encryption option must be set in the hadoop client so that the header is set.</p>
<p>Otherwise, the problem will likely be that the user does not have full access to the operation. Check what they were trying to (read vs write) and then look at the permissions of the user/role.</p>
<p>If the client using <a href="assumed_roles.html">assumed roles</a>, and a policy is set in <code>fs.s3a.assumed.role.policy</code>, then that policy declares <i>all</i> the rights which the caller has.</p></section><section>
<h3><a name="AccessDeniedException_in_rename.2C_.E2.80.9CMultiObjectDeleteException:_One_or_more_objects_could_not_be_deleted.E2.80.9D"></a><a name="access_denied_renaming"></a> <code>AccessDeniedException</code> in rename, &#x201c;MultiObjectDeleteException: One or more objects could not be deleted&#x201d;</h3>
<div class="source">
<div class="source">
<pre>mv: rename s3a://london/dest to s3a://london/src on
s3a://london/dest:
com.amazonaws.services.s3.model.MultiObjectDeleteException: One or more objects
could not be deleted (Service: null; Status Code: 200; Error Code: null; Request
ID: 5C9018EF245F02C5; S3 Extended Request ID:
5fQ2RVCPF0rdvADRv2XY3U4yb2J0gHRID/4jm1eqCXp7RxpU0dH9DliChYsCUD1aVCFtbwfWJWY=),
S3 Extended Request ID:
5fQ2RVCPF0rdvADRv2XY3U4yb2J0gHRID/4jm1eqCXp7RxpU0dH9DliChYsCUD1aVCFtbwfWJWY=:null:
AccessDenied: dest/file10: Access Denied
</pre></div></div>
<p>The S3A connector&#x2019;s emulation of file and directory rename is implemented by copying each file, then deleting the originals. This delete process is done in batches, by default in a single &#x201c;multiple object delete request&#x201d;. If one or more of the objects listed in the request cannot be deleted, an error is returned in S3 listing which objects were not deleted. If the cause was &#x201c;access denied&#x201d;, it is translated into an <code>AccessDeniedException</code>.</p>
<p>The rename is halted at this point: files may be present in both the source and destination directories. Those files which could not be deleted from the source directory will also have been copied into the destination directory. Files which were successfully deleted from the source directory will <i>only</i> be in the destination. And files for which the rename operation had yet to commence -they will only be in the source tree.</p>
<p>The user has to recover from this themselves. Be assured: no data will have been deleted, it is just that the data may now be scattered across two directories. Note: this is one reason why any application which tries to atomically commit work via rename (classic Hadoop output committers, distcp with the <code>-atomic</code> option) are not safe to use with S3. It is not a file system.</p>
<p>For an &#x2018;AccessDenied&#x2019; failure, the root cause is IAM permissions. The user/role/bucket must have the permission <code>s3:DeleteObject</code> on the source path. It is safest to grant <code>s3:Delete*</code> so that if a future version of the S3A connector supported extra operations (explicit deletion of versioned files, get/set/delete object tagging, &#x2026;), the client will have the permission to use them.</p></section><section>
<h3><a name="AccessDeniedException_when_using_SSE-KMS"></a><a name="kms_access_denied"></a> <code>AccessDeniedException</code> when using SSE-KMS</h3>
<p>When trying to write or read SEE-KMS-encrypted data, the client gets a <code>java.nio.AccessDeniedException</code> with the error 403/Forbidden.</p>
<p>The caller does not have the permissions to access the key with which the data was encrypted.</p></section><section>
<h3><a name="AccessDeniedException_when_using_a_.E2.80.9CRequester_Pays.E2.80.9D_enabled_bucket"></a><a name="access_denied_requester_pays"></a><code>AccessDeniedException</code> when using a &#x201c;Requester Pays&#x201d; enabled bucket</h3>
<p>When making cross-account requests to a requester pays enabled bucket, all calls must acknowledge via a header that the requester will be billed.</p>
<p>If you don&#x2019;t enable this acknowledgement within S3A, then you will see a message similar to this:</p>
<div class="source">
<div class="source">
<pre>java.nio.file.AccessDeniedException: s3a://my-bucket/my-object: getFileStatus on s3a://my-bucket/my-object:
com.amazonaws.services.s3.model.AmazonS3Exception: Forbidden (Service: Amazon S3; Status Code: 403;
Error Code: 403 Forbidden; Request ID: myshortreqid; S3 Extended Request ID: mylongreqid):403 Forbidden
</pre></div></div>
<p>To enable requester pays, set <code>fs.s3a.requester.pays.enabled</code> property to <code>true</code>.</p></section><section>
<h3><a name="AccessDeniedException_.E2.80.9CInvalidObjectState.E2.80.9D_when_trying_to_read_files"></a><a name="access_denied_archive_storage_class"></a><code>AccessDeniedException</code> &#x201c;InvalidObjectState&#x201d; when trying to read files</h3>
<div class="source">
<div class="source">
<pre>java.nio.file.AccessDeniedException: file1: copyFile(file1, file2) on file1: com.amazonaws.services.s3.model.AmazonS3Exception: Operation is not valid for the source object's storage class (Service: Amazon S3; Status Code: 403; Error Code: InvalidObjectState; Request ID: SK9EMPC1YRX75VZR; S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw=; Proxy: null), S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw=:InvalidObjectState
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Operation is not valid for the source object's storage class (Service: Amazon S3; Status Code: 403; Error Code: InvalidObjectState; Request ID: SK9EMPC1YRX75VZR; S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw=; Proxy: null), S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw=
</pre></div></div>
<p>This happens when you&#x2019;re trying to read or copy files that have archive storage class such as Glacier.</p>
<p>If you want to access the file with S3A after writes, do not set <code>fs.s3a.create.storage.class</code> to <code>glacier</code> or <code>deep_archive</code>.</p></section><section>
<h3><a name="a.E2.80.9CUnable_to_find_a_region_via_the_region_provider_chain..E2.80.9D_when_using_session_credentials."></a><a name="no_region_session_credentials"></a> &#x201c;Unable to find a region via the region provider chain.&#x201d; when using session credentials.</h3>
<p>Region must be provided when requesting session credentials, or an exception will be thrown with the message:</p>
<div class="source">
<div class="source">
<pre>com.amazonaws.SdkClientException: Unable to find a region via the region provider
chain. Must provide an explicit region in the builder or setup environment to supply a region.
</pre></div></div>
<p>In this case you have to set the <code>fs.s3a.assumed.role.sts.endpoint</code> property to a valid S3 sts endpoint and region like the following:</p>
<div class="source">
<div class="source">
<pre>&lt;property&gt;
&lt;name&gt;fs.s3a.assumed.role.sts.endpoint&lt;/name&gt;
&lt;value&gt;${sts.endpoint}&lt;/value&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;fs.s3a.assumed.role.sts.endpoint.region&lt;/name&gt;
&lt;value&gt;${sts.region}&lt;/value&gt;
&lt;/property&gt;
</pre></div></div>
</section></section><section>
<h2><a name="Connectivity_Problems"></a><a name="connectivity"></a> Connectivity Problems</h2><section>
<h3><a name="Error_.E2.80.9CThe_bucket_you_are_attempting_to_access_must_be_addressed_using_the_specified_endpoint.E2.80.9D"></a><a name="bad_endpoint"></a> Error &#x201c;The bucket you are attempting to access must be addressed using the specified endpoint&#x201d;</h3>
<p>This surfaces when <code>fs.s3a.endpoint</code> is configured to use an S3 service endpoint which is neither the original AWS one, <code>s3.amazonaws.com</code> , nor the one where the bucket is hosted. The error message contains the redirect target returned by S3, which can be used to determine the correct value for <code>fs.s3a.endpoint</code>.</p>
<div class="source">
<div class="source">
<pre>org.apache.hadoop.fs.s3a.AWSS3IOException: Received permanent redirect response
to bucket.s3-us-west-2.amazonaws.com. This likely indicates that the S3
endpoint configured in fs.s3a.endpoint does not match the AWS region
containing the bucket.: The bucket you are attempting to access must be
addressed using the specified endpoint. Please send all future requests to
this endpoint. (Service: Amazon S3; Status Code: 301;
Error Code: PermanentRedirect; Request ID: 7D39EC1021C61B11)
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:132)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initMultipartUploads(S3AFileSystem.java:287)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:203)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2895)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:102)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2932)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2914)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:390)
</pre></div></div>
<ol style="list-style-type: decimal">
<li>Use the <a class="externalLink" href="http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region">Specific endpoint of the bucket&#x2019;s S3 service</a></li>
<li>If not using &#x201c;V4&#x201d; authentication (see above), the original S3 endpoint can be used:</li>
</ol>
<div class="source">
<div class="source">
<pre>&lt;property&gt;
&lt;name&gt;fs.s3a.endpoint&lt;/name&gt;
&lt;value&gt;s3.amazonaws.com&lt;/value&gt;
&lt;/property&gt;
</pre></div></div>
<p>Using the explicit endpoint for the region is recommended for speed and to use the V4 signing API.</p></section><section>
<h3><a name="Unable_to_find_a_region_via_the_region_provider_chain"></a><a name="NoRegion"></a> <code>Unable to find a region via the region provider chain</code></h3>
<p>S3A client creation fails, possibly after a pause of some seconds.</p>
<p>This failure surfaces when <i>all</i> the following conditions are met:</p>
<ol style="list-style-type: decimal">
<li>Deployment outside EC2.</li>
<li><code>fs.s3a.endpoint</code> is unset.</li>
<li><code>fs.s3a.endpoint.region</code> is set to <code>&quot;&quot;</code>. (Hadoop 3.3.2+ only)</li>
<li>Without the file <code>~/.aws/config</code> existing or without a region set in it.</li>
<li>Without the JVM system property <code>aws.region</code> declaring a region.</li>
<li>Without the environment variable <code>AWS_REGION</code> declaring a region.</li>
</ol>
<p>Stack trace (Hadoop 3.3.1):</p>
<div class="source">
<div class="source">
<pre>Caused by: com.amazonaws.SdkClientException: Unable to find a region via the region provider chain.
Must provide an explicit region in the builder or setup environment to supply a region.
at com.amazonaws.client.builder.AwsClientBuilder.setRegion(AwsClientBuilder.java:462)
at com.amazonaws.client.builder.AwsClientBuilder.configureMutableProperties(AwsClientBuilder.java:424)
at com.amazonaws.client.builder.AwsSyncClientBuilder.build(AwsSyncClientBuilder.java:46)
at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.buildAmazonS3Client(DefaultS3ClientFactory.java:145)
at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:97)
at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:788)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:478)
</pre></div></div>
<p>Log and stack trace on later releases, with &#x201c;S3A filesystem client is using the SDK region resolution chain.&#x201d; warning that the SDK resolution chain is in use:</p>
<div class="source">
<div class="source">
<pre>2021-06-23 19:56:55,971 [main] WARN s3a.DefaultS3ClientFactory (LogExactlyOnce.java:warn(39)) -
S3A filesystem client is using the SDK region resolution chain.
2021-06-23 19:56:56,073 [main] WARN fs.FileSystem (FileSystem.java:createFileSystem(3464)) -
Failed to initialize fileystem s3a://osm-pds/planet:
org.apache.hadoop.fs.s3a.AWSClientIOException: creating AWS S3 client on s3a://osm-pds:
com.amazonaws.SdkClientException: Unable to find a region via the region provider chain.
Must provide an explicit region in the builder or setup environment to supply a region.:
Unable to find a region via the region provider chain.
Must provide an explicit region in the builder or setup environment to supply a region.
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:208)
at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:122)
at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:788)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:478)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3460)
at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:172)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3565)
at org.apache.hadoop.fs.FileSystem$Cache.getUnique(FileSystem.java:3518)
at org.apache.hadoop.fs.FileSystem.newInstance(FileSystem.java:592)
Caused by: com.amazonaws.SdkClientException: Unable to find a region via the region provider chain.
Must provide an explicit region in the builder or setup environment to supply a region.
at com.amazonaws.client.builder.AwsClientBuilder.setRegion(AwsClientBuilder.java:462)
at com.amazonaws.client.builder.AwsClientBuilder.configureMutableProperties(AwsClientBuilder.java:424)
at com.amazonaws.client.builder.AwsSyncClientBuilder.build(AwsSyncClientBuilder.java:46)
at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.buildAmazonS3Client(DefaultS3ClientFactory.java:185)
at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:117)
... 21 more
</pre></div></div>
<p>Due to changes in S3 client construction in Hadoop 3.3.1 this option surfaces in non-EC2 deployments where no AWS endpoint was declared: <a class="externalLink" href="https://issues.apache.org/jira/browse/HADOOP-17771">HADOOP-17771</a>. On Hadoop 3.3.2 and later it takes active effort to create this stack trace.</p>
<p><b>Fix: set <code>fs.s3a.endpoint</code> to <code>s3.amazonaws.com</code></b></p>
<p>Set <code>fs.s3a.endpoint</code> to the endpoint where the data is stored (best), or to <code>s3.amazonaws.com</code> (second-best).</p>
<div class="source">
<div class="source">
<pre>&lt;property&gt;
&lt;name&gt;fs.s3a.endpoint&lt;/name&gt;
&lt;value&gt;s3.amazonaws.com&lt;/value&gt;
&lt;/property&gt;
</pre></div></div>
<p>For Apache Spark, this can be done in <code>spark-defaults.conf</code></p>
<div class="source">
<div class="source">
<pre>spark.hadoop.fs.s3a.endpoint s3.amazonaws.com
</pre></div></div>
<p>Or in Scala by editing the spark configuration during setup.</p>
<div class="source">
<div class="source">
<pre>sc.hadoopConfiguration.set(&quot;fs.s3a.endpoint&quot;, &quot;s3.amazonaws.com&quot;)
</pre></div></div>
<p>Tip: set the logging of <code>org.apache.hadoop.fs.s3a.DefaultS3ClientFactory</code> to <code>DEBUG</code> to see how the endpoint and region configuration is determined.</p>
<div class="source">
<div class="source">
<pre>log4j.logger.org.apache.hadoop.fs.s3a.DefaultS3ClientFactory=DEBUG
</pre></div></div>
</section><section>
<h3><a name="a.E2.80.9CTimeout_waiting_for_connection_from_pool.E2.80.9D_when_writing_data"></a><a name="timeout_from_pool"></a> &#x201c;Timeout waiting for connection from pool&#x201d; when writing data</h3>
<p>This happens when using the output stream thread pool runs out of capacity.</p>
<div class="source">
<div class="source">
<pre>[s3a-transfer-shared-pool1-t20] INFO http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496))
- Unable to execute HTTP request:
Timeout waiting for connection from poolorg.apache.http.conn.ConnectionPoolTimeoutException:
Timeout waiting for connection from pool
at org.apache.http.impl.conn.PoolingClientConnectionManager.leaseConnection(PoolingClientConnectionManager.java:230)
at org.apache.http.impl.conn.PoolingClientConnectionManager$1.getConnection(PoolingClientConnectionManager.java:199)
at sun.reflect.GeneratedMethodAccessor13.invoke(Unknown Source)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at com.amazonaws.http.conn.ClientConnectionRequestFactory$Handler.invoke(ClientConnectionRequestFactory.java:70)
at com.amazonaws.http.conn.$Proxy10.getConnection(Unknown Source)
at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:424)
at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
at com.amazonaws.services.s3.AmazonS3Client.doUploadPart(AmazonS3Client.java:2921)
at com.amazonaws.services.s3.AmazonS3Client.uploadPart(AmazonS3Client.java:2906)
at org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:1025)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:360)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:355)
at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
</pre></div></div>
<p>Make sure that <code>fs.s3a.connection.maximum</code> is at least larger than <code>fs.s3a.threads.max</code>.</p>
<div class="source">
<div class="source">
<pre>&lt;property&gt;
&lt;name&gt;fs.s3a.threads.max&lt;/name&gt;
&lt;value&gt;20&lt;/value&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;fs.s3a.connection.maximum&lt;/name&gt;
&lt;value&gt;30&lt;/value&gt;
&lt;/property&gt;
</pre></div></div>
</section><section>
<h3><a name="a.E2.80.9CTimeout_waiting_for_connection_from_pool.E2.80.9D_when_reading_data"></a>&#x201c;Timeout waiting for connection from pool&#x201d; when reading data</h3>
<p>This happens when more threads are trying to read from an S3A system than the maximum number of allocated HTTP connections.</p>
<p>Set <code>fs.s3a.connection.maximum</code> to a larger value (and at least as large as <code>fs.s3a.threads.max</code>)</p></section><section>
<h3><a name="NoHttpResponseException"></a><code>NoHttpResponseException</code></h3>
<p>The HTTP Server did not respond.</p>
<div class="source">
<div class="source">
<pre>2017-02-07 10:01:07,950 INFO [s3a-transfer-shared-pool1-t7] com.amazonaws.http.AmazonHttpClient:
Unable to execute HTTP request: bucket.s3.amazonaws.com:443 failed to respond
org.apache.http.NoHttpResponseException: bucket.s3.amazonaws.com:443 failed to respond
at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:143)
at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:57)
at org.apache.http.impl.io.AbstractMessageParser.parse(AbstractMessageParser.java:261)
at org.apache.http.impl.AbstractHttpClientConnection.receiveResponseHeader(AbstractHttpClientConnection.java:283)
at org.apache.http.impl.conn.DefaultClientConnection.receiveResponseHeader(DefaultClientConnection.java:259)
at org.apache.http.impl.conn.ManagedClientConnectionImpl.receiveResponseHeader(ManagedClientConnectionImpl.java:209)
at org.apache.http.protocol.HttpRequestExecutor.doReceiveResponse(HttpRequestExecutor.java:272)
at com.amazonaws.http.protocol.SdkHttpRequestExecutor.doReceiveResponse(SdkHttpRequestExecutor.java:66)
at org.apache.http.protocol.HttpRequestExecutor.execute(HttpRequestExecutor.java:124)
at org.apache.http.impl.client.DefaultRequestDirector.tryExecute(DefaultRequestDirector.java:686)
at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:488)
at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
at com.amazonaws.services.s3.AmazonS3Client.copyPart(AmazonS3Client.java:1731)
at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:41)
at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:28)
at org.apache.hadoop.fs.s3a.SemaphoredDelegatingExecutor$CallableWithPermitRelease.call(SemaphoredDelegatingExecutor.java:222)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
</pre></div></div>
<p>Probably network problems, unless it really is an outage of S3.</p></section><section>
<h3><a name="Out_of_heap_memory_when_writing_with_via_Fast_Upload"></a>Out of heap memory when writing with via Fast Upload</h3>
<p>This can happen when using the upload buffering mechanism uses memory (either <code>fs.s3a.fast.upload.buffer=array</code> or <code>fs.s3a.fast.upload.buffer=bytebuffer</code>).</p>
<p>More data is being generated than in the JVM than it can upload to S3 &#x2014;and so much data has been buffered that the JVM has run out of memory.</p>
<ol style="list-style-type: decimal">
<li>
<p>Consult <a href="./index.html#fast_upload_thread_tuning">S3A Fast Upload Thread Tuning</a> for detail on this issue and options to address it.</p>
</li>
<li>
<p>Switch to buffering to disk, rather than memory.</p>
</li>
</ol>
<p>This surfaces if, while a multipart upload was taking place, all outstanding multipart uploads were garbage collected. The upload operation cannot complete because the data uploaded has been deleted.</p>
<p>Consult <a href="./index.html#multipart_purge">Cleaning up After Incremental Upload Failures</a> for details on how the multipart purge timeout can be set. If multipart uploads are failing with the message above, it may be a sign that this value is too low.</p></section><section>
<h3><a name="MultiObjectDeleteException_during_delete_or_rename_of_files"></a><code>MultiObjectDeleteException</code> during delete or rename of files</h3>
<div class="source">
<div class="source">
<pre>Exception in thread &quot;main&quot; com.amazonaws.services.s3.model.MultiObjectDeleteException:
Status Code: 0, AWS Service: null, AWS Request ID: null, AWS Error Code: null,
AWS Error Message: One or more objects could not be deleted, S3 Extended Request ID: null
at com.amazonaws.services.s3.AmazonS3Client.deleteObjects(AmazonS3Client.java:1745)
</pre></div></div>
<p>This happens when trying to delete multiple objects, and one of the objects could not be deleted. It <i>should not occur</i> just because the object is missing. More specifically: at the time this document was written, we could not create such a failure.</p>
<p>It will occur if the caller lacks the permission to delete any of the objects.</p>
<p>Consult the log to see the specifics of which objects could not be deleted. Do you have permission to do so?</p>
<p>If this operation is failing for reasons other than the caller lacking permissions:</p>
<ol style="list-style-type: decimal">
<li>Try setting <code>fs.s3a.multiobjectdelete.enable</code> to <code>false</code>.</li>
<li>Consult <a class="externalLink" href="https://issues.apache.org/jira/browse/HADOOP-11572">HADOOP-11572</a> for up to date advice.</li>
</ol></section><section>
<h3><a name="a.E2.80.9CFailed_to_Sanitize_XML_document.E2.80.9D"></a>&#x201c;Failed to Sanitize XML document&#x201d;</h3>
<div class="source">
<div class="source">
<pre>org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on test/testname/streaming/:
com.amazonaws.AmazonClientException: Failed to sanitize XML document
destined for handler class com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler:
Failed to sanitize XML document destined for handler class
com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:105)
at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1462)
at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListStatus(S3AFileSystem.java:1227)
at org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:1203)
at org.apache.hadoop.fs.s3a.S3AGlobber.listStatus(S3AGlobber.java:69)
at org.apache.hadoop.fs.s3a.S3AGlobber.doGlob(S3AGlobber.java:210)
at org.apache.hadoop.fs.s3a.S3AGlobber.glob(S3AGlobber.java:125)
at org.apache.hadoop.fs.s3a.S3AFileSystem.globStatus(S3AFileSystem.java:1853)
at org.apache.hadoop.fs.s3a.S3AFileSystem.globStatus(S3AFileSystem.java:1841)
</pre></div></div>
<p>We believe this is caused by the connection to S3 being broken. See <a class="externalLink" href="https://issues.apache.org/jira/browse/HADOOP-13811">HADOOP-13811</a>.</p>
<p>It may go away if the operation is retried.</p></section><section>
<h3><a name="JSON_Parse_Error_from_AWS_SDK"></a>JSON Parse Error from AWS SDK</h3>
<p>Sometimes a JSON Parse error is reported with the stack trace in the <code>com.amazonaws</code>,</p>
<p>Again, we believe this is caused by the connection to S3 being broken.</p>
<p>It may go away if the operation is retried.</p></section></section><section>
<h2><a name="File_System_Semantics"></a><a name="semantics"></a>File System Semantics</h2>
<p>These are the issues where S3 does not appear to behave the way a filesystem &#x201c;should&#x201d;. That&#x2019;s because it &#x201c;isn&#x2019;t&#x201d;.</p><section>
<h3><a name="File_not_visible.2Fsaved"></a>File not visible/saved</h3>
<p>The files in an object store are not visible until the write has been completed. In-progress writes are simply saved to a local file/cached in RAM and only uploaded. at the end of a write operation. If a process terminated unexpectedly, or failed to call the <code>close()</code> method on an output stream, the pending data will have been lost.</p></section><section>
<h3><a name="File_flush.28.29_calls_do_not_save_data_to_S3"></a>File <code>flush()</code> calls do not save data to S3</h3>
<p>Again, this is due to the fact that the data is cached locally until the <code>close()</code> operation. The S3A filesystem cannot be used as a store of data if it is required that the data is persisted durably after every <code>Syncable.hflush()</code> or <code>Syncable.hsync()</code> call. This includes resilient logging, HBase-style journaling and the like. The standard strategy here is to save to HDFS and then copy to S3.</p></section><section>
<h3><a name="UnsupportedOperationException_.E2.80.9CS3A_streams_are_not_Syncable._See_HADOOP-17597..E2.80.9D"></a><a name="syncable"></a> <code>UnsupportedOperationException</code> &#x201c;S3A streams are not Syncable. See HADOOP-17597.&#x201d;</h3>
<p>The application has tried to call either the <code>Syncable.hsync()</code> or <code>Syncable.hflush()</code> methods on an S3A output stream. This has been rejected because the connector isn&#x2019;t saving any data at all. The <code>Syncable</code> API, especially the <code>hsync()</code> call, are critical for applications such as HBase to safely persist data.</p>
<p>When configured to do so, the S3A connector throws an <code>UnsupportedOperationException</code> when these API calls are made, because the API guarantees absolutely cannot be met: <i>nothing is being flushed or saved</i>.</p>
<ul>
<li>Applications which intend to invoke the Syncable APIs should call <code>hasCapability(&quot;hsync&quot;)</code> on the stream to see if they are supported.</li>
<li>Or catch and downgrade <code>UnsupportedOperationException</code>.</li>
</ul>
<p>These recommendations <i>apply to all filesystems</i>.</p>
<p>For consistency with other filesystems, S3A output streams do not by default reject the <code>Syncable</code> calls -instead they print a warning of its use.</p>
<p>The count of invocations of the two APIs are collected in the S3A filesystem Statistics/IOStatistics and so their use can be monitored.</p>
<p>To switch the S3A connector to rejecting all use of <code>hsync()</code> or <code>hflush()</code> calls, set the option <code>fs.s3a.downgrade.syncable.exceptions</code> to <code>false</code>.</p>
<div class="source">
<div class="source">
<pre>&lt;property&gt;
&lt;name&gt;fs.s3a.downgrade.syncable.exceptions&lt;/name&gt;
&lt;value&gt;false&lt;/value&gt;
&lt;/property&gt;
</pre></div></div>
<p>Regardless of the setting, the <code>Syncable</code> API calls do not work. Telling the store to <i>not</i> downgrade the calls is a way to 1. Prevent applications which require Syncable to work from being deployed against S3. 2. Identify applications which are making the calls even though they don&#x2019;t need to. These applications can then be fixed -something which may take time.</p>
<p>Put differently: it is safest to disable downgrading syncable exceptions. However, enabling the downgrade stops applications unintentionally using the API from breaking.</p>
<p><i>Tip</i>: try turning it on in staging environments to see what breaks.</p></section><section>
<h3><a name="RemoteFileChangedException_and_read-during-overwrite"></a><code>RemoteFileChangedException</code> and read-during-overwrite</h3>
<div class="source">
<div class="source">
<pre>org.apache.hadoop.fs.s3a.RemoteFileChangedException: re-open `s3a://my-bucket/test/file.txt':
Change reported by S3 while reading at position 1949.
ETag f9c186d787d4de9657e99f280ba26555 was unavailable
at org.apache.hadoop.fs.s3a.impl.ChangeTracker.processResponse(ChangeTracker.java:137)
at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:200)
at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$lazySeek$1(S3AInputStream.java:346)
at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$2(Invoker.java:195)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:109)
at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:265)
at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:322)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:261)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:193)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:215)
at org.apache.hadoop.fs.s3a.S3AInputStream.lazySeek(S3AInputStream.java:339)
at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:372)
</pre></div></div>
<p>If an S3 object is updated while an S3A filesystem reader has an open <code>InputStream</code> on it, the reader may encounter <code>RemoteFileChangedException</code>. This occurs if the S3A <code>InputStream</code> needs to re-open the object (e.g. during a seek()) and detects the change.</p>
<p>If the change detection mode is configured to &#x2018;warn&#x2019;, a warning like the following will be seen instead of <code>RemoteFileChangedException</code>:</p>
<div class="source">
<div class="source">
<pre>WARN - ETag change detected on re-open s3a://my-bucket/test/readFileToChange.txt at 1949.
Expected f9c186d787d4de9657e99f280ba26555 got 043abff21b7bd068d2d2f27ccca70309
</pre></div></div>
<p>Using a third-party S3 implementation that doesn&#x2019;t support eTags might result in the following error.</p>
<div class="source">
<div class="source">
<pre>org.apache.hadoop.fs.s3a.NoVersionAttributeException: `s3a://my-bucket/test/file.txt':
Change detection policy requires ETag
at org.apache.hadoop.fs.s3a.impl.ChangeTracker.processResponse(ChangeTracker.java:153)
at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:200)
at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$lazySeek$1(S3AInputStream.java:346)
at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$2(Invoker.java:195)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:109)
at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:265)
at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:322)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:261)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:193)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:215)
at org.apache.hadoop.fs.s3a.S3AInputStream.lazySeek(S3AInputStream.java:339)
at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:372)
</pre></div></div>
<p>If the change policy is <code>versionid</code> there are a number of possible causes</p>
<ul>
<li>The bucket does not have object versioning enabled.</li>
<li>The bucket does have versioning enabled, but the object being read was created before versioning was enabled.</li>
<li>The bucket is on a third-party store which does not support object versioning.</li>
</ul>
<p>See <a href="./index.html#handling_read-during-overwrite">Handling Read-During-Overwrite</a> for more information.</p></section><section>
<h3><a name="Rename_not_behaving_as_.E2.80.9Cexpected.E2.80.9D"></a>Rename not behaving as &#x201c;expected&#x201d;</h3>
<p>S3 is not a filesystem. The S3A connector mimics file and directory rename by</p>
<ul>
<li>HEAD then LIST of source path. The source MUST exist, else a <code>FileNotFoundException</code> is raised.</li>
<li>HEAD then LIST of the destination path. This SHOULD NOT exist. If it does and if the source is a directory, the destination MUST be an empty directory. If the source is a file, the destination MAY be a directory, empty or not. If the destination exists and relevant conditions are not met, a <code>FileAlreadyExistsException</code> is raised.</li>
<li>If the destination path does not exist, a HEAD request of the parent path to verify that there is no object there. Directory markers are not checked for, nor that the path has any children,</li>
<li>File-by-file copy of source objects to destination. Parallelized, with page listings of directory objects and issuing of DELETE requests.</li>
<li>Post-delete recreation of source parent directory marker, if needed.</li>
</ul>
<p>This is slow (<code>O(data)</code>) and can cause timeouts on code which is required to send regular progress reports/heartbeats -for example, distCp. It is <i>very unsafe</i> if the calling code expects atomic renaming as part of any commit algorithm. This is why the <a href="committers.html">S3A Committers</a> or similar are needed to safely commit output.</p>
<p>There is also the risk of race conditions arising if many processes/threads are working with the same directory tree <a class="externalLink" href="https://issues.apache.org/jira/browse/HADOOP-16721">HADOOP-16721</a>.</p>
<p>To reduce this risk, since Hadoop 3.3.1, the S3A connector no longer verifies the parent directory of the destination of a rename is a directory -only that it is <i>not</i> a file. You can rename a directory or file deep under a file if you try -after which there is no guarantee of the files being found in listings. Try not to do that.</p></section></section><section>
<h2><a name="S3_Server_Side_Encryption"></a><a name="encryption"></a> S3 Server Side Encryption</h2><section>
<h3><a name="AWSS3IOException_KMS.NotFoundException_.E2.80.9CInvalid_arn.E2.80.9D_when_using_SSE-KMS"></a><code>AWSS3IOException</code> <code>KMS.NotFoundException</code> &#x201c;Invalid arn&#x201d; when using SSE-KMS</h3>
<p>When performing file operations, the user may run into an issue where the KMS key arn is invalid.</p>
<div class="source">
<div class="source">
<pre>org.apache.hadoop.fs.s3a.AWSS3IOException: innerMkdirs on /test:
com.amazonaws.services.s3.model.AmazonS3Exception:
Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException;
Request ID: CA89F276B3394565),
S3 Extended Request ID: ncz0LWn8zor1cUO2fQ7gc5eyqOk3YfyQLDn2OQNoe5Zj/GqDLggUYz9QY7JhdZHdBaDTh+TL5ZQ=:
Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: CA89F276B3394565)
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:194)
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:117)
at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1541)
at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2230)
at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.mkdirs(AbstractFSContractTestBase.java:338)
at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.setup(AbstractFSContractTestBase.java:193)
at org.apache.hadoop.fs.s3a.scale.S3AScaleTestBase.setup(S3AScaleTestBase.java:90)
at org.apache.hadoop.fs.s3a.scale.AbstractSTestS3AHugeFiles.setup(AbstractSTestS3AHugeFiles.java:77)
at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:24)
at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception:
Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: CA89F276B3394565)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1588)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1258)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1030)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:742)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:716)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4221)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4168)
at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1718)
at com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:133)
at com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:125)
at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:143)
at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:48)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
</pre></div></div>
<p>Possible causes:</p>
<ul>
<li>the KMS key ARN is entered incorrectly, or</li>
<li>the KMS key referenced by the ARN is in a different region than the S3 bucket being used.</li>
</ul></section><section>
<h3><a name="Using_SSE-C_.E2.80.9CBad_Request.E2.80.9D"></a>Using SSE-C &#x201c;Bad Request&#x201d;</h3>
<p>When performing file operations the user may run into an unexpected 400/403 error such as</p>
<div class="source">
<div class="source">
<pre>org.apache.hadoop.fs.s3a.AWSS3IOException: getFileStatus on fork-4/:
com.amazonaws.services.s3.model.AmazonS3Exception:
Bad Request (Service: Amazon S3; Status Code: 400;
Error Code: 400 Bad Request; Request ID: 42F9A1987CB49A99),
S3 Extended Request ID: jU2kcwaXnWj5APB14Cgb1IKkc449gu2+dhIsW/+7x9J4D+VUkKvu78mBo03oh9jnOT2eoTLdECU=:
Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: 42F9A1987CB49A99)
</pre></div></div>
<p>This can happen in the cases of not specifying the correct SSE-C encryption key. Such cases can be as follows: 1. An object is encrypted using SSE-C on S3 and either the wrong encryption type is used, no encryption is specified, or the SSE-C specified is incorrect. 2. A directory is encrypted with a SSE-C keyA and the user is trying to move a file using configured SSE-C keyB into that structure.</p></section></section><section>
<h2><a name="S3_Client_Side_Encryption"></a><a name="client-side-encryption"></a> S3 Client Side Encryption</h2><section>
<h3><a name="Instruction_file_not_found_for_S3_object"></a>Instruction file not found for S3 object</h3>
<p>Reading an unencrypted file would fail when read through CSE enabled client.</p>
<div class="source">
<div class="source">
<pre>java.lang.SecurityException: Instruction file not found for S3 object with bucket name: ap-south-cse, key: unencryptedData.txt
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleAE.decipher(S3CryptoModuleAE.java:190)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleAE.getObjectSecurely(S3CryptoModuleAE.java:136)
at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.getObject(AmazonS3EncryptionClientV2.java:241)
at org.apache.hadoop.fs.s3a.S3AFileSystem$InputStreamCallbacksImpl.getObject(S3AFileSystem.java:1462)
at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$reopen$0(S3AInputStream.java:217)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:216)
at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$lazySeek$1(S3AInputStream.java:382)
at org.apache.hadoop.fs.s3a.Invoker.lambda$maybeRetry$3(Invoker.java:230)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
at org.apache.hadoop.fs.s3a.Invoker.lambda$maybeRetry$5(Invoker.java:354)
at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:414)
at org.apache.hadoop.fs.s3a.Invoker.maybeRetry(Invoker.java:350)
at org.apache.hadoop.fs.s3a.Invoker.maybeRetry(Invoker.java:228)
at org.apache.hadoop.fs.s3a.Invoker.maybeRetry(Invoker.java:272)
at org.apache.hadoop.fs.s3a.S3AInputStream.lazySeek(S3AInputStream.java:374)
at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:493)
at java.io.DataInputStream.read(DataInputStream.java:100)
at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:94)
at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:68)
at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:129)
at org.apache.hadoop.fs.shell.Display$Cat.printToStdout(Display.java:101)
at org.apache.hadoop.fs.shell.Display$Cat.processPath(Display.java:96)
at org.apache.hadoop.fs.shell.Command.processPathInternal(Command.java:370)
at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:333)
at org.apache.hadoop.fs.shell.Command.processPathArgument(Command.java:306)
at org.apache.hadoop.fs.shell.Command.processArgument(Command.java:288)
at org.apache.hadoop.fs.shell.Command.processArguments(Command.java:272)
at org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:121)
at org.apache.hadoop.fs.shell.Command.run(Command.java:179)
at org.apache.hadoop.fs.FsShell.run(FsShell.java:327)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:81)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:95)
at org.apache.hadoop.fs.FsShell.main(FsShell.java:390)
</pre></div></div>
<p>CSE enabled client should read encrypted data only.</p></section><section>
<h3><a name="CSE-KMS_method_requires_KMS_key_ID"></a>CSE-KMS method requires KMS key ID</h3>
<p>KMS key ID is required for CSE-KMS to encrypt data, not providing one leads to failure.</p>
<div class="source">
<div class="source">
<pre>2021-07-07 11:33:04,550 WARN fs.FileSystem: Failed to initialize fileystem
s3a://ap-south-cse/: java.lang.IllegalArgumentException: CSE-KMS
method requires KMS key ID. Use fs.s3a.encryption.key property to set it.
-ls: CSE-KMS method requires KMS key ID. Use fs.s3a.encryption.key property to
set it.
</pre></div></div>
<p>set <code>fs.s3a.encryption.key=&lt;KMS_KEY_ID&gt;</code> generated through AWS console.</p></section><section>
<h3><a name="com.amazonaws.services.kms.model.IncorrectKeyException_The_key_ID_in_the_request_does_not_identify_a_CMK_that_can_perform_this_operation."></a><code>com.amazonaws.services.kms.model.IncorrectKeyException</code> The key ID in the request does not identify a CMK that can perform this operation.</h3>
<p>KMS key ID used to PUT(encrypt) the data, must be the one used to GET the data.</p>
<div class="source">
<div class="source">
<pre>cat: open s3a://ap-south-cse/encryptedData.txt at 0 on
s3a://ap-south-cse/encryptedData.txt:
com.amazonaws.services.kms.model.IncorrectKeyException: The key ID in the
request does not identify a CMK that can perform this operation. (Service: AWSKMS;
Status Code: 400; ErrorCode: IncorrectKeyException;
Request ID: da21aa8a-f00d-467c-94a0-32b627d32bc0; Proxy: null):IncorrectKeyException:
The key ID in the request does not identify a CMK that can perform this
operation. (Service: AWSKMS ; Status Code: 400; Error Code: IncorrectKeyException;
Request ID: da21aa8a-f00d-467c-94a0-32b627d32bc0; Proxy: null)
</pre></div></div>
<p>Use the same KMS key ID used to upload data to download and read it as well.</p></section><section>
<h3><a name="com.amazonaws.services.kms.model.NotFoundException_key.2F.3CKMS_KEY_ID.3E_does_not_exist"></a><code>com.amazonaws.services.kms.model.NotFoundException</code> key/&lt;KMS_KEY_ID&gt; does not exist</h3>
<p>Using a KMS key ID from a different region than the bucket used to store data would lead to failure while uploading.</p>
<div class="source">
<div class="source">
<pre>mkdir: PUT 0-byte object on testmkdir:
com.amazonaws.services.kms.model.NotFoundException: Key
'arn:aws:kms:ap-south-1:152813717728:key/&lt;KMS_KEY_ID&gt;'
does not exist (Service: AWSKMS; Status Code: 400; Error Code: NotFoundException;
Request ID: 279db85d-864d-4a38-9acd-d892adb504c0; Proxy: null):NotFoundException:
Key 'arn:aws:kms:ap-south-1:152813717728:key/&lt;KMS_KEY_ID&gt;'
does not exist(Service: AWSKMS; Status Code: 400; Error Code: NotFoundException;
Request ID: 279db85d-864d-4a38-9acd-d892adb504c0; Proxy: null)
</pre></div></div>
<p>While generating the KMS Key ID make sure to generate it in the same region as your bucket.</p></section><section>
<h3><a name="Unable_to_perform_range_get_request:_Range_get_support_has_been_disabled"></a>Unable to perform range get request: Range get support has been disabled</h3>
<p>If Range get is not supported for a CSE algorithm or is disabled:</p>
<div class="source">
<div class="source">
<pre>java.lang.SecurityException: Unable to perform range get request: Range get support has been disabled. See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryptography.html
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleAE.assertCanGetPartialObject(S3CryptoModuleAE.java:446)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleAE.getObjectSecurely(S3CryptoModuleAE.java:117)
at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.getObject(AmazonS3EncryptionClientV2.java:241)
at org.apache.hadoop.fs.s3a.S3AFileSystem$InputStreamCallbacksImpl.getObject(S3AFileSystem.java:1462)
at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$reopen$0(S3AInputStream.java:217)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:216)
at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$lazySeek$1(S3AInputStream.java:382)
at org.apache.hadoop.fs.s3a.Invoker.lambda$maybeRetry$3(Invoker.java:230)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
at org.apache.hadoop.fs.s3a.Invoker.lambda$maybeRetry$5(Invoker.java:354)
at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:414)
at org.apache.hadoop.fs.s3a.Invoker.maybeRetry(Invoker.java:350)
at org.apache.hadoop.fs.s3a.Invoker.maybeRetry(Invoker.java:228)
at org.apache.hadoop.fs.s3a.Invoker.maybeRetry(Invoker.java:272)
at org.apache.hadoop.fs.s3a.S3AInputStream.lazySeek(S3AInputStream.java:374)
at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:408)
at java.io.DataInputStream.readByte(DataInputStream.java:265)
</pre></div></div>
<p>Range gets must be enabled for CSE to work.</p></section><section>
<h3><a name="WARNING:_Range_gets_do_not_provide_authenticated_encryption_properties_even_when_used_with_an_authenticated_mode_.28AES-GCM.29."></a>WARNING: Range gets do not provide authenticated encryption properties even when used with an authenticated mode (AES-GCM).</h3>
<p>The S3 Encryption Client is configured to support range get requests. This warning would be shown everytime S3-CSE is used.</p>
<div class="source">
<div class="source">
<pre>2021-07-14 12:54:09,525 [main] WARN s3.AmazonS3EncryptionClientV2
(AmazonS3EncryptionClientV2.java:warnOnRangeGetsEnabled(401)) - The S3
Encryption Client is configured to support range get requests. Range gets do
not provide authenticated encryption properties even when used with an
authenticated mode (AES-GCM). See https://docs.aws.amazon.com/general/latest
/gr/aws_sdk_cryptography.html
</pre></div></div>
<p>We can Ignore this warning since, range gets must be enabled for S3-CSE to get data.</p></section><section>
<h3><a name="WARNING:_If_you_don.E2.80.99t_have_objects_encrypted_with_these_legacy_modes.2C_you_should_disable_support_for_them_to_enhance_security."></a>WARNING: If you don&#x2019;t have objects encrypted with these legacy modes, you should disable support for them to enhance security.</h3>
<p>The S3 Encryption Client is configured to read encrypted data with legacy encryption modes through the CryptoMode setting, and we would see this warning for all S3-CSE request.</p>
<div class="source">
<div class="source">
<pre>2021-07-14 12:54:09,519 [main] WARN s3.AmazonS3EncryptionClientV2
(AmazonS3EncryptionClientV2.java:warnOnLegacyCryptoMode(409)) - The S3
Encryption Client is configured to read encrypted data with legacy
encryption modes through the CryptoMode setting. If you don't have objects
encrypted with these legacy modes, you should disable support for them to
enhance security. See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryptography.html
</pre></div></div>
<p>We can ignore this, since this CryptoMode setting(CryptoMode.AuthenticatedEncryption) is required for range gets to work.</p></section><section>
<h3><a name="com.amazonaws.services.kms.model.InvalidKeyUsageException:_You_cannot_generate_a_data_key_with_an_asymmetric_CMK"></a>com.amazonaws.services.kms.model.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK</h3>
<p>If you generated an Asymmetric CMK from AWS console then CSE-KMS won&#x2019;t be able to generate unique data key for encryption.</p>
<div class="source">
<div class="source">
<pre>Caused by: com.amazonaws.services.kms.model.InvalidKeyUsageException:
You cannot generate a data key with an asymmetric CMK
(Service: AWSKMS; Status Code: 400; Error Code: InvalidKeyUsageException; Request ID: 93609c15-e490-4035-8390-f4396f0d90bf; Proxy: null)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223)
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190)
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179)
at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482)
at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156)
at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792)
at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
... 49 more
</pre></div></div>
<p>Generate a Symmetric Key in the same region as your S3 storage for CSE-KMS to work.</p></section><section>
<h3><a name="com.amazonaws.services.kms.model.NotFoundException:_Invalid_keyId"></a>com.amazonaws.services.kms.model.NotFoundException: Invalid keyId</h3>
<p>If the value in <code>fs.s3a.encryption.key</code> property, does not exist /valid in AWS KMS CMK(Customer managed keys), then this error would be seen.</p>
<div class="source">
<div class="source">
<pre>Caused by: com.amazonaws.services.kms.model.NotFoundException: Invalid keyId abc
(Service: AWSKMS; Status Code: 400; Error Code: NotFoundException; Request ID: 9d53552a-3d1b-47c8-984c-9a599d5c2391; Proxy: null)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223)
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190)
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179)
at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482)
at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156)
at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792)
at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
... 49 more
</pre></div></div>
<p>Check if <code>fs.s3a.encryption.key</code> is set correctly and matches the same on AWS console.</p></section><section>
<h3><a name="com.amazonaws.services.kms.model.AWSKMSException:_User:_.3CUser_ARN.3E_is_not_authorized_to_perform_:_kms_:GenerateDataKey_on_resource:_.3CKEY_ID.3E"></a>com.amazonaws.services.kms.model.AWSKMSException: User: &lt;User_ARN&gt; is not authorized to perform : kms :GenerateDataKey on resource: &lt;KEY_ID&gt;</h3>
<p>User doesn&#x2019;t have authorization to the specific AWS KMS Key ID.</p>
<div class="source">
<div class="source">
<pre>Caused by: com.amazonaws.services.kms.model.AWSKMSException:
User: arn:aws:iam::152813717728:user/&lt;user&gt; is not authorized to perform: kms:GenerateDataKey on resource: &lt;key_ID&gt;
(Service: AWSKMS; Status Code: 400; Error Code: AccessDeniedException; Request ID: 4ded9f1f-b245-4213-87fc-16cba7a1c4b9; Proxy: null)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223)
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190)
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179)
at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482)
at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156)
at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792)
at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
... 49 more
</pre></div></div>
<p>The user trying to use the KMS Key ID should have the right permissions to access (encrypt/decrypt) using the AWS KMS Key used via <code>fs.s3a.encryption.key</code>. If not, then add permission(or IAM role) in &#x201c;Key users&#x201d; section by selecting the AWS-KMS CMK Key on AWS console.</p></section><section>
<h3><a name="Message_appears_in_logs_.E2.80.9CNot_all_bytes_were_read_from_the_S3ObjectInputStream.E2.80.9D"></a><a name="not_all_bytes_were_read"></a> Message appears in logs &#x201c;Not all bytes were read from the S3ObjectInputStream&#x201d;</h3>
<p>This is a message which can be generated by the Amazon SDK when the client application calls <code>abort()</code> on the HTTP input stream, rather than reading to the end of the file/stream and causing <code>close()</code>. The S3A client does call <code>abort()</code> when seeking round large files, <a class="externalLink" href="https://github.com/aws/aws-sdk-java/issues/1211">so leading to the message</a>.</p>
<p>No ASF Hadoop releases have shipped with an SDK which prints this message when used by the S3A client. However third party and private builds of Hadoop may cause the message to be logged.</p>
<p>Ignore it. The S3A client does call <code>abort()</code>, but that&#x2019;s because our benchmarking shows that it is generally more efficient to abort the TCP connection and initiate a new one than read to the end of a large file.</p>
<p>Note: the threshold when data is read rather than the stream aborted can be tuned by <code>fs.s3a.readahead.range</code>; seek policy in <code>fs.s3a.experimental.input.fadvise</code>.</p></section><section>
<h3><a name="PathIOException_Number_of_parts_in_multipart_upload_exceeded."></a><a name="upload_failure"></a> <code>PathIOException</code> Number of parts in multipart upload exceeded.</h3>
<p>Number of parts in multipart upload exceeded</p>
<div class="source">
<div class="source">
<pre>org.apache.hadoop.fs.PathIOException: `test/testMultiPartUploadFailure': Number of parts in multipart upload exceeded. Current part count = X, Part count limit = Y
at org.apache.hadoop.fs.s3a.WriteOperationHelper.newUploadPartRequest(WriteOperationHelper.java:432)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.uploadBlockAsync(S3ABlockOutputStream.java:627)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$000(S3ABlockOutputStream.java:532)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.uploadCurrentBlock(S3ABlockOutputStream.java:316)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.write(S3ABlockOutputStream.java:301)
</pre></div></div>
<p>This is a known issue where upload fails if number of parts is more than 10000 (specified by aws SDK). You can configure <code>fs.s3a.multipart.size</code> to reduce the number of parts.</p></section><section>
<h3><a name="UnknownStoreException_Bucket_does_not_exist."></a><a name="no_such_bucket"></a> <code>UnknownStoreException</code> Bucket does not exist.</h3>
<p>The bucket does not exist.</p>
<div class="source">
<div class="source">
<pre>org.apache.hadoop.fs.s3a.UnknownStoreException:
Bucket random-bucket-33013fb8-f7f7-4edb-9c26-16a6ed019184 does not exist
at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:537)
at org.apache.hadoop.fs.s3a.S3AFileSystem.doBucketProbing(S3AFileSystem.java:471)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:387)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3422)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:502)
</pre></div></div>
<p>Check the URI is correct, and that the bucket actually exists.</p>
<p>If using a third-party store, verify that you&#x2019;ve configured the client to talk to the specific server in <code>fs.s3a.endpoint</code>. Forgetting to update this value and asking the AWS S3 endpoint for a bucket is not an unusual occurrence.</p>
<p>This can surface during filesystem API calls if the bucket is deleted while you are using it, -or the startup check for bucket existence has been disabled by setting <code>fs.s3a.bucket.probe</code> to 0.</p>
<div class="source">
<div class="source">
<pre>org.apache.hadoop.fs.s3a.UnknownStoreException: s3a://random-bucket-7d9217b0-b426-4344-82ea-25d6cbb316f1/
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:254)
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:167)
at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListFiles(S3AFileSystem.java:4149)
at org.apache.hadoop.fs.s3a.S3AFileSystem.listFiles(S3AFileSystem.java:3983)
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception:
The specified bucket does not exist
(Service: Amazon S3; Status Code: 404; Error Code: NoSuchBucket
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1712)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1367)
</pre></div></div>
</section></section><section>
<h2><a name="S3Guard_Errors"></a><a name="s3guard"></a> S3Guard Errors</h2>
<p>S3Guard has been completely cut from the s3a connector [HADOOP-17409 Remove S3Guard - no longer needed](HADOOP-17409 Remove S3Guard - no longer needed).</p>
<p>To avoid consistency problems with older releases, if an S3A filesystem is configured to use DynamoDB the filesystem will fail to initialize.</p><section>
<h3><a name="S3Guard_is_no_longer_needed.2Fsupported"></a><a name="s3guard_unsupported"></a> S3Guard is no longer needed/supported</h3>
<p>The option <code>fs.s3a.metadatastore.impl</code> or the per-bucket version has a value of <code>org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore</code></p>
<div class="source">
<div class="source">
<pre>org.apache.hadoop.fs.PathIOException: `s3a://production-london': S3Guard is no longer needed/supported,
yet s3a://production-london is configured to use DynamoDB as the S3Guard metadata store.
This is no longer needed or supported.
Origin of setting is fs.s3a.bucket.production-london.metadatastore.impl via [core-site.xml]
at org.apache.hadoop.fs.s3a.s3guard.S3Guard.checkNoS3Guard(S3Guard.java:111)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:540)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3459)
at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:171)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3564)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3511)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:538)
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:366)
at org.apache.hadoop.fs.shell.PathData.expandAsGlob(PathData.java:342)
at org.apache.hadoop.fs.shell.Command.expandArgument(Command.java:252)
at org.apache.hadoop.fs.shell.Command.expandArguments(Command.java:235)
at org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:105)
at org.apache.hadoop.fs.shell.Command.run(Command.java:179)
at org.apache.hadoop.fs.FsShell.run(FsShell.java:327)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:81)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:95)
at org.apache.hadoop.fs.FsShell.main(FsShell.java:390)
ls: `s3a://production-london': S3Guard is no longer needed/supported,
yet s3a://production-london is configured to use DynamoDB as the S3Guard metadata store.
This is no longer needed or supported.
Origin of setting is fs.s3a.bucket.production-london.metadatastore.impl via [core-site.xml]
</pre></div></div>
<p>The error message will state the property from where it came, here <code>fs.s3a.bucket.production-london.metadatastore.impl</code> and which file the option was set if known, here <code>core-site.xml</code>.</p>
<p>Fix: remove the configuration options enabling S3Guard.</p>
<p>Consult the <a href="s3guard.html">S3Guard documentation</a> for more details.</p></section></section><section>
<h2><a name="Other_Errors"></a><a name="other"></a> Other Errors</h2><section>
<h3><a name="SdkClientException_Unable_to_verify_integrity_of_data_upload"></a><a name="integrity"></a> <code>SdkClientException</code> Unable to verify integrity of data upload</h3>
<p>Something has happened to the data as it was uploaded.</p>
<div class="source">
<div class="source">
<pre>Caused by: org.apache.hadoop.fs.s3a.AWSClientIOException: saving output on dest/_task_tmp.-ext-10000/_tmp.000000_0:
com.amazonaws.AmazonClientException: Unable to verify integrity of data upload.
Client calculated content hash (contentMD5: L75PalQk0CIhTp04MStVOA== in base 64)
didn't match hash (etag: 37ace01f2c383d6b9b3490933c83bb0f in hex) calculated by Amazon S3.
You may need to delete the data stored in Amazon S3.
(metadata.contentMD5: L75PalQk0CIhTp04MStVOA==, md5DigestStream: null,
bucketName: ext2, key: dest/_task_tmp.-ext-10000/_tmp.000000_0):
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:144)
at org.apache.hadoop.fs.s3a.S3AOutputStream.close(S3AOutputStream.java:121)
at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
at org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat$1.close(HiveIgnoreKeyTextOutputFormat.java:99)
at org.apache.hadoop.hive.ql.exec.FileSinkOperator$FSPaths.closeWriters(FileSinkOperator.java:190)
... 22 more
Caused by: com.amazonaws.AmazonClientException: Unable to verify integrity of data upload.
Client calculated content hash (contentMD5: L75PalQk0CIhTp04MStVOA== in base 64)
didn't match hash (etag: 37ace01f2c383d6b9b3490933c83bb0f in hex) calculated by Amazon S3.
You may need to delete the data stored in Amazon S3.
(metadata.contentMD5: L75PalQk0CIhTp04MStVOA==, md5DigestStream: null,
bucketName: ext2, key: dest/_task_tmp.-ext-10000/_tmp.000000_0)
at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1492)
at com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:131)
at com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:123)
at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:139)
at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:47)
... 4 more
</pre></div></div>
<p>As it uploads data to S3, the AWS SDK builds up an MD5 checksum of what was PUT/POSTed. When S3 returns the checksum of the uploaded data, that is compared with the local checksum. If there is a mismatch, this error is reported.</p>
<p>The uploaded data is already on S3 and will stay there, though if this happens during a multipart upload, it may not be visible (but still billed: clean up your multipart uploads via the <code>hadoop s3guard uploads</code> command).</p>
<p>Possible causes for this</p>
<ol style="list-style-type: decimal">
<li>A (possibly transient) network problem, including hardware faults.</li>
<li>A proxy server is doing bad things to the data.</li>
<li>Some signing problem, especially with third-party S3-compatible object stores.</li>
</ol>
<p>This is a very, very rare occurrence.</p>
<p>If the problem is a signing one, try changing the signature algorithm.</p>
<div class="source">
<div class="source">
<pre>&lt;property&gt;
&lt;name&gt;fs.s3a.signing-algorithm&lt;/name&gt;
&lt;value&gt;S3SignerType&lt;/value&gt;
&lt;/property&gt;
</pre></div></div>
<p>We cannot make any promises that it will work, only that it has been known to make the problem go away &#x201c;once&#x201d;</p></section><section>
<h3><a name="AWSS3IOException_The_Content-MD5_you_specified_did_not_match_what_we_received"></a><code>AWSS3IOException</code> The Content-MD5 you specified did not match what we received</h3>
<p>Reads work, but writes, even <code>mkdir</code>, fail:</p>
<div class="source">
<div class="source">
<pre>org.apache.hadoop.fs.s3a.AWSS3IOException: copyFromLocalFile(file:/tmp/hello.txt, s3a://bucket/hello.txt)
on file:/tmp/hello.txt:
The Content-MD5 you specified did not match what we received.
(Service: Amazon S3; Status Code: 400; Error Code: BadDigest; Request ID: 4018131225),
S3 Extended Request ID: null
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:127)
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:69)
at org.apache.hadoop.fs.s3a.S3AFileSystem.copyFromLocalFile(S3AFileSystem.java:1494)
at org.apache.hadoop.tools.cloudup.Cloudup.uploadOneFile(Cloudup.java:466)
at org.apache.hadoop.tools.cloudup.Cloudup.access$000(Cloudup.java:63)
at org.apache.hadoop.tools.cloudup.Cloudup$1.call(Cloudup.java:353)
at org.apache.hadoop.tools.cloudup.Cloudup$1.call(Cloudup.java:350)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:748)
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception:
The Content-MD5 you specified did not match what we received.
(Service: Amazon S3; Status Code: 400; Error Code: BadDigest; Request ID: 4018131225),
S3 Extended Request ID: null
at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1307)
at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:894)
at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:597)
at com.amazonaws.http.AmazonHttpClient.doExecute(AmazonHttpClient.java:363)
at com.amazonaws.http.AmazonHttpClient.executeWithTimer(AmazonHttpClient.java:329)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:308)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3659)
at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1422)
at com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:131)
at com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:123)
at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:139)
at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:47)
at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
... 4 more
</pre></div></div>
<p>This stack trace was seen when interacting with a third-party S3 store whose expectations of headers related to the AWS V4 signing mechanism was not compatible with that of the specific AWS SDK Hadoop was using.</p>
<p>Workaround: revert to V2 signing.</p>
<div class="source">
<div class="source">
<pre>&lt;property&gt;
&lt;name&gt;fs.s3a.signing-algorithm&lt;/name&gt;
&lt;value&gt;S3SignerType&lt;/value&gt;
&lt;/property&gt;
</pre></div></div>
</section><section>
<h3><a name="When_writing_data:_.E2.80.9Cjava.io.FileNotFoundException:_Completing_multi-part_upload.E2.80.9D"></a>When writing data: &#x201c;java.io.FileNotFoundException: Completing multi-part upload&#x201d;</h3>
<p>A multipart upload was trying to complete, but failed as there was no upload with that ID.</p>
<div class="source">
<div class="source">
<pre>java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multipart/1c397ca6-9dfb-4ac1-9cf7-db666673246b:
com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist.
The upload ID may be invalid, or the upload may have been aborted or completed.
(Service: Amazon S3; Status Code: 404; Error Code: NoSuchUpload;
at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182)
at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770)
at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2705)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:473)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:382)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:272)
at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
</pre></div></div>
<p>This can happen when all outstanding uploads have been aborted, including the active ones.</p>
<p>If the bucket has a lifecycle policy of deleting multipart uploads, make sure that the expiry time of the deletion is greater than that required for all open writes to complete the write, <i>and for all jobs using the S3A committers to commit their work.</i></p></section><section>
<h3><a name="Application_hangs_after_reading_a_number_of_files"></a>Application hangs after reading a number of files</h3>
<p>The pool of https client connections and/or IO threads have been used up, and none are being freed.</p>
<ol style="list-style-type: decimal">
<li>The pools aren&#x2019;t big enough. See <a href="#timeout_from_pool">&#x201c;Timeout waiting for connection from pool&#x201d;</a></li>
<li>Likely root cause: whatever code is reading files isn&#x2019;t calling <code>close()</code> on the input streams. Make sure your code does this! And if it&#x2019;s someone else&#x2019;s: make sure you have a recent version; search their issue trackers to see if its a known/fixed problem. If not, it&#x2019;s time to work with the developers, or come up with a workaround (i.e closing the input stream yourself).</li>
</ol></section><section>
<h3><a name="Issue:_when_writing_data.2C_HTTP_Exceptions_logged_at_info_from_AmazonHttpClient"></a>Issue: when writing data, HTTP Exceptions logged at info from <code>AmazonHttpClient</code></h3>
<div class="source">
<div class="source">
<pre>[s3a-transfer-shared-pool4-t6] INFO http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496))
- Unable to execute HTTP request: hwdev-steve-ireland-new.s3.amazonaws.com:443 failed to respond
org.apache.http.NoHttpResponseException: bucket.s3.amazonaws.com:443 failed to respond
at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:143)
at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:57)
at org.apache.http.impl.io.AbstractMessageParser.parse(AbstractMessageParser.java:261)
at org.apache.http.impl.AbstractHttpClientConnection.receiveResponseHeader(AbstractHttpClientConnection.java:283)
at org.apache.http.impl.conn.DefaultClientConnection.receiveResponseHeader(DefaultClientConnection.java:259)
at org.apache.http.impl.conn.ManagedClientConnectionImpl.receiveResponseHeader(ManagedClientConnectionImpl.java:209)
at org.apache.http.protocol.HttpRequestExecutor.doReceiveResponse(HttpRequestExecutor.java:272)
at com.amazonaws.http.protocol.SdkHttpRequestExecutor.doReceiveResponse(SdkHttpRequestExecutor.java:66)
at org.apache.http.protocol.HttpRequestExecutor.execute(HttpRequestExecutor.java:124)
at org.apache.http.impl.client.DefaultRequestDirector.tryExecute(DefaultRequestDirector.java:686)
at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:488)
at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
at com.amazonaws.services.s3.AmazonS3Client.copyPart(AmazonS3Client.java:1731)
at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:41)
at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:28)
at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
</pre></div></div>
<p>These are HTTP I/O exceptions caught and logged inside the AWS SDK. The client will attempt to retry the operation; it may just be a transient event. If there are many such exceptions in logs, it may be a symptom of connectivity or network problems.</p></section><section>
<h3><a name="AWSBadRequestException_IllegalLocationConstraintException.2FThe_unspecified_location_constraint_is_incompatible"></a><code>AWSBadRequestException</code> IllegalLocationConstraintException/The unspecified location constraint is incompatible</h3>
<div class="source">
<div class="source">
<pre> Cause: org.apache.hadoop.fs.s3a.AWSBadRequestException: put on :
com.amazonaws.services.s3.model.AmazonS3Exception:
The unspecified location constraint is incompatible for the region specific
endpoint this request was sent to.
(Service: Amazon S3; Status Code: 400; Error Code: IllegalLocationConstraintException;
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:178)
at org.apache.hadoop.fs.s3a.S3ALambda.execute(S3ALambda.java:64)
at org.apache.hadoop.fs.s3a.WriteOperationHelper.uploadObject(WriteOperationHelper.java:451)
at org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker.aboutToComplete(MagicCommitTracker.java:128)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:373)
at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101)
at org.apache.hadoop.hive.ql.io.orc.WriterImpl.close(WriterImpl.java:2429)
at org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat$OrcRecordWriter.close(OrcOutputFormat.java:106)
at org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat$OrcRecordWriter.close(OrcOutputFormat.java:91)
...
Cause: com.amazonaws.services.s3.model.AmazonS3Exception:
The unspecified location constraint is incompatible for the region specific endpoint
this request was sent to. (Service: Amazon S3; Status Code: 400; Error Code: IllegalLocationConstraintException;
Request ID: EEBC5A08BCB3A645)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1588)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1258)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1030)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:742)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:716)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4221)
...
</pre></div></div>
<p>Something has been trying to write data to &#x201c;/&#x201d;.</p></section></section><section>
<h2><a name="Best_Practises"></a><a name="best"></a> Best Practises</h2><section>
<h3><a name="Enabling_low-level_logging"></a><a name="logging"></a> Enabling low-level logging</h3>
<p>The AWS SDK and the Apache S3 components can be configured to log at more detail, as can S3A itself.</p>
<div class="source">
<div class="source">
<pre>log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
log4j.logger.com.amazonaws.request=DEBUG
log4j.logger.com.amazonaws.thirdparty.apache.http=DEBUG
</pre></div></div>
<p>If using the &#x201c;unshaded&#x201d; JAR, then the Apache HttpClient can be directly configured:</p>
<div class="source">
<div class="source">
<pre>log4j.logger.org.apache.http=DEBUG
</pre></div></div>
<p>This produces a log such as this, which is for a V4-authenticated PUT of a 0-byte file used as an empty directory marker</p>
<div class="source">
<div class="source">
<pre>execchain.MainClientExec (MainClientExec.java:execute(255)) - Executing request PUT /test/ HTTP/1.1
execchain.MainClientExec (MainClientExec.java:execute(266)) - Proxy auth state: UNCHALLENGED
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(135)) - http-outgoing-0 &gt;&gt; PUT /test/ HTTP/1.1
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 &gt;&gt; Host: ireland-new.s3-eu-west-1.amazonaws.com
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 &gt;&gt; x-amz-content-sha256: UNSIGNED-PAYLOAD
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 &gt;&gt; Authorization: AWS4-HMAC-SHA256 Credential=AKIAIYZ5JEEEER/20170904/eu-west-1/s3/aws4_request, ...
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 &gt;&gt; X-Amz-Date: 20170904T172929Z
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 &gt;&gt; User-Agent: Hadoop 3.0.0-beta-1, aws-sdk-java/1.11.134 ...
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 &gt;&gt; amz-sdk-invocation-id: 75b530f8-ad31-1ad3-13db-9bd53666b30d
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 &gt;&gt; amz-sdk-retry: 0/0/500
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 &gt;&gt; Content-Type: application/octet-stream
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 &gt;&gt; Content-Length: 0
http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 &gt;&gt; Connection: Keep-Alive
http.wire (Wire.java:wire(72)) - http-outgoing-0 &gt;&gt; &quot;PUT /test/ HTTP/1.1[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &gt;&gt; &quot;Host: ireland-new.s3-eu-west-1.amazonaws.com[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &gt;&gt; &quot;x-amz-content-sha256: UNSIGNED-PAYLOAD[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &gt;&gt; &quot;Authorization: AWS4-HMAC-SHA256 Credential=AKIAIYZ5JEEEER/20170904/eu-west-1/s3/aws4_request, ,,,
http.wire (Wire.java:wire(72)) - http-outgoing-0 &gt;&gt; &quot;X-Amz-Date: 20170904T172929Z[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &gt;&gt; &quot;User-Agent: 3.0.0-beta-1, aws-sdk-java/1.11.134 ...
http.wire (Wire.java:wire(72)) - http-outgoing-0 &gt;&gt; &quot;amz-sdk-invocation-id: 75b530f8-ad31-1ad3-13db-9bd53666b30d[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &gt;&gt; &quot;amz-sdk-retry: 0/0/500[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &gt;&gt; &quot;Content-Type: application/octet-stream[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &gt;&gt; &quot;Content-Length: 0[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &gt;&gt; &quot;Connection: Keep-Alive[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &gt;&gt; &quot;[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &lt;&lt; &quot;HTTP/1.1 200 OK[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &lt;&lt; &quot;x-amz-id-2: mad9GqKztzlL0cdnCKAj9GJOAs+DUjbSC5jRkO7W1E7Nk2BUmFvt81bhSNPGdZmyyKqQI9i/B/A=[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &lt;&lt; &quot;x-amz-request-id: C953D2FE4ABF5C51[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &lt;&lt; &quot;Date: Mon, 04 Sep 2017 17:29:30 GMT[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &lt;&lt; &quot;ETag: &quot;d41d8cd98f00b204e9800998ecf8427e&quot;[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &lt;&lt; &quot;Content-Length: 0[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &lt;&lt; &quot;Server: AmazonS3[\r][\n]&quot;
http.wire (Wire.java:wire(72)) - http-outgoing-0 &lt;&lt; &quot;[\r][\n]&quot;
http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(124)) - http-outgoing-0 &lt;&lt; HTTP/1.1 200 OK
http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 &lt;&lt; x-amz-id-2: mad9GqKztzlL0cdnCKAj9GJOAs+DUjbSC5jRkO7W1E7Nk2BUmFvt81bhSNPGdZmyyKqQI9i/B/A=
http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 &lt;&lt; x-amz-request-id: C953D2FE4ABF5C51
http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 &lt;&lt; Date: Mon, 04 Sep 2017 17:29:30 GMT
http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 &lt;&lt; ETag: &quot;d41d8cd98f00b204e9800998ecf8427e&quot;
http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 &lt;&lt; Content-Length: 0
http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 &lt;&lt; Server: AmazonS3
execchain.MainClientExec (MainClientExec.java:execute(284)) - Connection can be kept alive for 60000 MILLISECONDS
</pre></div></div>
</section><section>
<h3><a name="Enable_S3_Server-side_Logging"></a><a name="audit-logging"></a> Enable S3 Server-side Logging</h3>
<p>The <a href="auditing">Auditing</a> feature of the S3A connector can be used to generate S3 Server Logs with information which can be used to debug problems working with S3, such as throttling events.</p>
<p>Consult the <a href="auditing">auditing documentation</a> documentation. As auditing is enabled by default, enabling S3 Logging for a bucket should be sufficient to collect these logs.</p></section><section>
<h3><a name="Reducing_failures_by_configuring_retry_policy"></a><a name="retries"></a> Reducing failures by configuring retry policy</h3>
<p>The S3A client can ba configured to retry those operations which are considered retryable. That can be because they are idempotent, or because the failure happened before the request was processed by S3.</p>
<p>The number of retries and interval between each retry can be configured:</p>
<div class="source">
<div class="source">
<pre>&lt;property&gt;
&lt;name&gt;fs.s3a.retry.limit&lt;/name&gt;
&lt;value&gt;7&lt;/value&gt;
&lt;description&gt;
Number of times to retry any repeatable S3 client request on failure,
excluding throttling requests.
&lt;/description&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;fs.s3a.retry.interval&lt;/name&gt;
&lt;value&gt;500ms&lt;/value&gt;
&lt;description&gt;
Initial retry interval when retrying operations for any reason other
than S3 throttle errors.
&lt;/description&gt;
&lt;/property&gt;
</pre></div></div>
<p>Not all failures are retried. Specifically excluded are those considered unrecoverable:</p>
<ul>
<li>Low-level networking: <code>UnknownHostException</code>, <code>NoRouteToHostException</code>.</li>
<li>302 redirects.</li>
<li>Missing resources, 404/<code>FileNotFoundException</code>.</li>
<li>HTTP 416 response/<code>EOFException</code>. This can surface if the length of a file changes while another client is reading it.</li>
<li>Failures during execution or result processing of non-idempotent operations where it is considered likely that the operation has already taken place.</li>
</ul>
<p>In future, others may be added to this list.</p>
<p>When one of these failures arises in the S3/S3A client, the retry mechanism is bypassed and the operation will fail.</p>
<p><i>Warning</i>: the S3A client considers DELETE, PUT and COPY operations to be idempotent, and will retry them on failure. These are only really idempotent if no other client is attempting to manipulate the same objects, such as: renaming() the directory tree or uploading files to the same location. Please don&#x2019;t do that. Given that the emulated directory rename and delete operations are not atomic, even without retries, multiple S3 clients working with the same paths can interfere with each other</p></section><section>
<h3><a name="Tuning_AWS_request_timeouts"></a><a name="aws-timeouts"></a> Tuning AWS request timeouts</h3>
<p>It is possible to configure a global timeout for AWS service calls using following property:</p>
<div class="source">
<div class="source">
<pre>&lt;property&gt;
&lt;name&gt;fs.s3a.connection.request.timeout&lt;/name&gt;
&lt;value&gt;0&lt;/value&gt;
&lt;description&gt;
Time out on HTTP requests to the AWS service; 0 means no timeout.
Measured in seconds; the usual time suffixes are all supported
Important: this is the maximum duration of any AWS service call,
including upload and copy operations. If non-zero, it must be larger
than the time to upload multi-megabyte blocks to S3 from the client,
and to rename many-GB files. Use with care.
Values that are larger than Integer.MAX_VALUE milliseconds are
converged to Integer.MAX_VALUE milliseconds
&lt;/description&gt;
&lt;/property&gt;
</pre></div></div>
<p>If this value is configured too low, user may encounter <code>SdkClientException</code>s due to many requests timing-out.</p>
<div class="source">
<div class="source">
<pre>com.amazonaws.SdkClientException: Unable to execute HTTP request:
Request did not complete before the request timeout configuration.:
Unable to execute HTTP request: Request did not complete before the request timeout configuration.
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:205)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:112)
at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$4(Invoker.java:315)
at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:407)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:311)
</pre></div></div>
<p>When this happens, try to set <code>fs.s3a.connection.request.timeout</code> to a larger value or disable it completely by setting it to <code>0</code>.</p></section></section><section>
<h2><a name="SDK_Upgrade_Warnings"></a><a name="upgrade_warnings"></a> SDK Upgrade Warnings</h2>
<p>S3A will soon be upgraded to <a class="externalLink" href="https://github.com/aws/aws-sdk-java-v2">AWS&#x2019;s Java SDK V2</a>. For more information on the upgrade and what&#x2019;s changing, see <a href="./aws_sdk_upgrade.html">Upcoming upgrade to AWS Java SDK V2</a>.</p>
<p>S3A logs the following warnings for things that will be changing in the upgrade. To disable these logs, comment out <code>log4j.logger.org.apache.hadoop.fs.s3a.SDKV2Upgrade</code> in log4j.properties.</p><section>
<h3><a name="Directly_referencing_AWS_SDK_V1_credential_provider"></a><a name="ProviderReferenced"></a> <code>Directly referencing AWS SDK V1 credential provider</code></h3>
<p>This will be logged when an AWS credential provider is referenced directly in <code>fs.s3a.aws.credentials.provider</code>. For example, <code>com.amazonaws.auth.AWSSessionCredentialsProvider</code></p>
<p>To stop this warning, remove any AWS credential providers from <code>fs.s3a.aws.credentials.provider</code>. Instead, use S3A&#x2019;s credential providers.</p></section><section>
<h3><a name="getAmazonS3ClientForTesting.28.29_will_be_removed"></a><a name="ClientRequested"></a> <code>getAmazonS3ClientForTesting() will be removed</code></h3>
<p>This will be logged when <code>getAmazonS3ClientForTesting()</code> is called to get the S3 Client. With V2, the S3 client will change from type <code>com.amazonaws.services.s3.AmazonS3</code> to <code>software.amazon.awssdk.services.s3.S3Client</code>, and so this method will be removed.</p></section><section>
<h3><a name="DelegationTokenProvider"></a></h3></section><section>
<h3><a name="Custom_credential_providers_used_in_delegation_tokens_binding_classes_will_need_to_be_updated"></a><code>Custom credential providers used in delegation tokens binding classes will need to be updated</code></h3>
<p>This will be logged when delegation tokens are used. Delegation tokens allow the use of custom binding classes which can implement custom credential providers. These credential providers will currently be implementing <code>com.amazonaws.auth.AWSCredentialsProvider</code> and will need to be updated to implement <code>software.amazon.awssdk.auth.credentials.AwsCredentialsProvider</code>.</p></section><section>
<h3><a name="CustomSignerUsed"></a></h3></section><section>
<h3><a name="The_signer_interface_has_changed_in_AWS_SDK_V2.2C_custom_signers_will_need_to_be_updated"></a><code>The signer interface has changed in AWS SDK V2, custom signers will need to be updated</code></h3>
<p>This will be logged when a custom signer is used. Custom signers will currently be implementing <code>com.amazonaws.auth.Signer</code> and will need to be updated to implement <code>software.amazon.awssdk.core.signer.Signer</code>.</p></section><section>
<h3><a name="GetObjectMetadataCalled"></a></h3></section><section>
<h3><a name="getObjectMetadata.28.29_called._This_operation_and_it.27s_response_will_be_changed"></a><code>getObjectMetadata() called. This operation and it's response will be changed</code></h3>
<p>This will be logged when <code>getObjectMetadata</code> is called. In SDK V2, this operation has changed to <code>headObject()</code> and will return a response of the type <code>HeadObjectResponse</code>.</p></section></section>
</div>
</div>
<div class="clear">
<hr/>
</div>
<div id="footer">
<div class="xright">
&#169; 2008-2023
Apache Software Foundation
- <a href="http://maven.apache.org/privacy-policy.html">Privacy Policy</a>.
Apache Maven, Maven, Apache, the Apache feather logo, and the Apache Maven project logos are trademarks of The Apache Software Foundation.
</div>
<div class="clear">
<hr/>
</div>
</div>
</body>
</html>