소스 검색

HADOOP-17817. S3A to raise IOE if both S3-CSE and S3Guard enabled (#3239)

Contributed by Mehakmeet Singh
Mehakmeet Singh 3 년 전
부모
커밋
b19dae8db3

+ 4 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -539,6 +539,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       if (hasMetadataStore()) {
         LOG.debug("Using metadata store {}, authoritative store={}, authoritative path={}",
             getMetadataStore(), allowAuthoritativeMetadataStore, allowAuthoritativePaths);
+        if (isCSEEnabled) {
+          throw new PathIOException(uri.toString(), "S3-CSE cannot be used "
+              + "with S3Guard");
+        }
       }
 
       // LOG if S3Guard is disabled on the warn level set in config

+ 1 - 0
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md

@@ -601,6 +601,7 @@ clients where S3-CSE has not been enabled.
 
 ### Limitations
 
+- S3Guard is not supported with S3-CSE.
 - Performance will be reduced. All encrypt/decrypt is now being done on the
  client.
 - Writing files may be slower, as only a single block can be encrypted and

+ 25 - 0
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md

@@ -1435,6 +1435,31 @@ The user trying to use the KMS Key ID should have the right permissions to acces
 If not, then add permission(or IAM role) in "Key users" section by selecting the
 AWS-KMS CMK Key on AWS console.
 
+### S3-CSE cannot be used with S3Guard
+
+S3-CSE not supported for S3Guard enabled buckets.
+```
+org.apache.hadoop.fs.PathIOException: `s3a://test-bucket': S3-CSE cannot be used with S3Guard
+    at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:543)
+    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.get(FileSystem.java:3512)
+    at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:539)
+    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)
+```
+If you want to use S3Guard then disable S3-CSE or disable S3Guard if you want
+to use S3-CSE.
+
 ### <a name="not_all_bytes_were_read"></a> Message appears in logs "Not all bytes were read from the S3ObjectInputStream"