[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-05-16 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17847028#comment-17847028
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

steveloughran commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1603436149


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##
@@ -585,6 +592,18 @@ public void initialize(URI name, Configuration 
originalConf)
 
   s3aInternals = createS3AInternals();
 
+  try {
+s3ObjectStorageClassFilter = 
Optional.of(conf.getTrimmed(READ_RESTORED_GLACIER_OBJECTS,
+DEFAULT_READ_RESTORED_GLACIER_OBJECTS))
+.map(String::toUpperCase)
+.map(S3ObjectStorageClassFilter::valueOf).get();
+  } catch (IllegalArgumentException e) {
+LOG.warn("Invalid value for the config {} is set. Valid values are:" +
+"READ_ALL, SKIP_ALL_GLACIER, READ_RESTORED_GLACIER_OBJECTS. 
Defaulting to READ_ALL",
+READ_RESTORED_GLACIER_OBJECTS);
+s3ObjectStorageClassFilter = S3ObjectStorageClassFilter.READ_ALL;

Review Comment:
   lets fall to the default. maybe pull the conf.getTrimmed() out of the try {} 
so it's value can be printed too.
   
   FWIW in #6789 I'm doing a `getEnumSet()` which is case independent too.



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##
@@ -585,6 +592,18 @@ public void initialize(URI name, Configuration 
originalConf)
 
   s3aInternals = createS3AInternals();
 
+  try {
+s3ObjectStorageClassFilter = 
Optional.of(conf.getTrimmed(READ_RESTORED_GLACIER_OBJECTS,
+DEFAULT_READ_RESTORED_GLACIER_OBJECTS))
+.map(String::toUpperCase)

Review Comment:
   (Locale.ROOT)



##
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md:
##
@@ -927,8 +927,34 @@ The switch to turn S3A auditing on or off.
 Should auditing of S3A requests be enabled?
   
 
+```
+##  Glacier Object Support

Review Comment:
   can you add a newline. thanks



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/S3ObjectStorageClassFilter.java:
##
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.api;
+
+import java.util.Set;
+import java.util.function.Function;
+
+import software.amazon.awssdk.services.s3.model.ObjectStorageClass;
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.util.Sets;
+
+
+/**
+ * 
+ * {@link S3ObjectStorageClassFilter} will filter the S3 files based on the
+ * {@code fs.s3a.glacier.read.restored.objects} configuration set in {@link 
S3AFileSystem}
+ * The config can have 3 values:
+ * {@code READ_ALL}: Retrieval of Glacier files will fail with 
InvalidObjectStateException:
+ * The operation is not valid for the object's storage class.
+ * {@code SKIP_ALL_GLACIER}: If this value is set then this will ignore any S3 
Objects which are
+ * tagged with Glacier storage classes and retrieve the others.
+ * {@code READ_RESTORED_GLACIER_OBJECTS}: If this value is set then restored 
status of the Glacier
+ * object will be checked, if restored the objects would be read like normal 
S3 objects
+ * else they will be ignored as the objects would not have been retrieved from 
the S3 Glacier.
+ * 
+ */
+public enum S3ObjectStorageClassFilter {
+  READ_ALL(o -> true),
+  SKIP_ALL_GLACIER(S3ObjectStorageClassFilter::isNotGlacierObject),
+  
READ_RESTORED_GLACIER_OBJECTS(S3ObjectStorageClassFilter::isCompletedRestoredObject);
+
+  private static final Set GLACIER_STORAGE_CLASSES = 
Sets.newHashSet(
+  ObjectStorageClass.GLACIER, ObjectStorageClass.DEEP_ARCHIVE);
+
+  private final Function filter;
+
+  S3ObjectStorageClassFilter(Function filter) {
+this.filter = filter;
+  }
+
+  /**
+   * Checks if the s3 object is not an object with a storage class of 
glacier/deep_archive.
+   * @param object s3 object
+   * @return if the s3 object is n

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-05-16 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17846891#comment-17846891
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-2114733132

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 23s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  
|
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  36m 30s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  trunk passed with JDK 
Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1  |
   | +1 :green_heart: |  compile  |   0m 24s |  |  trunk passed with JDK 
Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06  |
   | +1 :green_heart: |  checkstyle  |   0m 22s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 25s |  |  trunk passed  |
   | -1 :x: |  javadoc  |   0m 24s | 
[/branch-javadoc-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/16/artifact/out/branch-javadoc-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1.txt)
 |  hadoop-aws in trunk failed with JDK 
Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1.  |
   | -1 :x: |  javadoc  |   0m 27s | 
[/branch-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/16/artifact/out/branch-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06.txt)
 |  hadoop-aws in trunk failed with JDK Private 
Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06.  |
   | -1 :x: |  spotbugs  |   0m 26s | 
[/branch-spotbugs-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/16/artifact/out/branch-spotbugs-hadoop-tools_hadoop-aws.txt)
 |  hadoop-aws in trunk failed.  |
   | +1 :green_heart: |  shadedclient  |   3m  0s |  |  branch has no errors 
when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |   3m 24s |  |  Used diff version of patch file. 
Binary files and potentially other changes not applied. Please rebase and 
squash commits if necessary.  |
    _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 21s | 
[/patch-mvninstall-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/16/artifact/out/patch-mvninstall-hadoop-tools_hadoop-aws.txt)
 |  hadoop-aws in the patch failed.  |
   | -1 :x: |  compile  |   0m 22s | 
[/patch-compile-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/16/artifact/out/patch-compile-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1.txt)
 |  hadoop-aws in the patch failed with JDK 
Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1.  |
   | -1 :x: |  javac  |   0m 22s | 
[/patch-compile-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/16/artifact/out/patch-compile-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1.txt)
 |  hadoop-aws in the patch failed with JDK 
Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1.  |
   | -1 :x: |  compile  |   0m 22s | 
[/patch-compile-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/16/artifact/out/patch-compile-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06.txt)
 |  hadoop-aws in the patch failed with JDK Private 
Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06.  |
   | -1 :x: |  javac  |   0m 22s | 
[/patch-compile-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/16/artifact/out/patch-compile-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06.txt)
 |  hadoop-aws in the patch failed with JDK Private 
Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06.  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The pa

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-05-16 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17846883#comment-17846883
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-2114600740

   @steveloughran , I have pushed the changes for 
`ITestS3AReadRestoredGlacierObjects.java` , and it now has only 1 test with all 
the test cases. In total , the setup happens twice, once for Glacier Storage 
class and once more for Glacier Deep Archive Storage class. The number of 
setups happening has been reduced , where earlier it created the file for each 
test case. 
   Please review whenever you get some time. 
   Let me know if any more changes are needed here. 
   
   
   Thanks :) 




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-05-16 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17846882#comment-17846882
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1602914173


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/S3ObjectStorageClassFilter.java:
##
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.api;
+
+import java.util.Set;
+import java.util.function.Function;
+
+import software.amazon.awssdk.services.s3.model.ObjectStorageClass;
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+
+
+/**
+ * 
+ * {@link S3ObjectStorageClassFilter} will filter the S3 files based on the
+ * {@code fs.s3a.glacier.read.restored.objects} configuration set in {@link 
S3AFileSystem}
+ * The config can have 3 values:
+ * {@code READ_ALL}: Retrieval of Glacier files will fail with 
InvalidObjectStateException:
+ * The operation is not valid for the object's storage class.
+ * {@code SKIP_ALL_GLACIER}: If this value is set then this will ignore any S3 
Objects which are
+ * tagged with Glacier storage classes and retrieve the others.
+ * {@code READ_RESTORED_GLACIER_OBJECTS}: If this value is set then restored 
status of the Glacier
+ * object will be checked, if restored the objects would be read like normal 
S3 objects
+ * else they will be ignored as the objects would not have been retrieved from 
the S3 Glacier.
+ * 
+ */
+public enum S3ObjectStorageClassFilter {
+  READ_ALL(o -> true),
+  SKIP_ALL_GLACIER(S3ObjectStorageClassFilter::isNotGlacierObject),
+  
READ_RESTORED_GLACIER_OBJECTS(S3ObjectStorageClassFilter::isCompletedRestoredObject);
+
+  private static final Set GLACIER_STORAGE_CLASSES = 
Sets.newHashSet(
+  ObjectStorageClass.GLACIER, ObjectStorageClass.DEEP_ARCHIVE);
+
+  private final Function filter;
+
+  S3ObjectStorageClassFilter(Function filter) {

Review Comment:
   Implicit should be okay





> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-04-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17837509#comment-17837509
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-2058206967

   Sure @steveloughran, will prioritize this




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-04-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17837430#comment-17837430
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

steveloughran commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-2057762347

   @bpahuja can you address the review comments? Otherwise I'm going to forget 
about it.
   
   Get it done now and we can target 3.4.1




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-03-19 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17828513#comment-17828513
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

steveloughran commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1530919362


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/S3ObjectStorageClassFilter.java:
##
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.api;
+
+import java.util.Set;
+import java.util.function.Function;
+
+import software.amazon.awssdk.services.s3.model.ObjectStorageClass;
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+
+
+/**
+ * 
+ * {@link S3ObjectStorageClassFilter} will filter the S3 files based on the
+ * {@code fs.s3a.glacier.read.restored.objects} configuration set in {@link 
S3AFileSystem}
+ * The config can have 3 values:
+ * {@code READ_ALL}: Retrieval of Glacier files will fail with 
InvalidObjectStateException:
+ * The operation is not valid for the object's storage class.
+ * {@code SKIP_ALL_GLACIER}: If this value is set then this will ignore any S3 
Objects which are
+ * tagged with Glacier storage classes and retrieve the others.
+ * {@code READ_RESTORED_GLACIER_OBJECTS}: If this value is set then restored 
status of the Glacier
+ * object will be checked, if restored the objects would be read like normal 
S3 objects
+ * else they will be ignored as the objects would not have been retrieved from 
the S3 Glacier.
+ * 
+ */
+public enum S3ObjectStorageClassFilter {
+  READ_ALL(o -> true),
+  SKIP_ALL_GLACIER(S3ObjectStorageClassFilter::isNotGlacierObject),
+  
READ_RESTORED_GLACIER_OBJECTS(S3ObjectStorageClassFilter::isCompletedRestoredObject);
+
+  private static final Set GLACIER_STORAGE_CLASSES = 
Sets.newHashSet(
+  ObjectStorageClass.GLACIER, ObjectStorageClass.DEEP_ARCHIVE);
+
+  private final Function filter;
+
+  S3ObjectStorageClassFilter(Function filter) {

Review Comment:
   should this be private, or it implicit with enums?



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/S3ObjectStorageClassFilter.java:
##
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.api;
+
+import java.util.Set;
+import java.util.function.Function;
+
+import software.amazon.awssdk.services.s3.model.ObjectStorageClass;
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;

Review Comment:
   could you use org.apache.hadoop.util.Sets here. its part of our attempt to 
isolate ourselves better from guava changes and the pain that causes downstream



##
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:
##
@@ -0,0 +1,193 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Versio

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-03-18 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17827934#comment-17827934
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

steveloughran commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-2003669976

   I am catching up now, as warned giving priority to people who helped 
validate the RC. the core project is a community project, and everyone gets to 
participate




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-03-17 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17827847#comment-17827847
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-2002993100

   Hello @steveloughran, Just a gentle reminder to review the PR :) , Do take a 
look whenever you get some time 
   
   Thanks




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-26 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17820759#comment-17820759
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

steveloughran commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1964529624

   I'm doing nothing but helping it get Hadoop 3.4.0 out the door this week. No 
code of my except related to packaging; no reviews of other people except 
related to the release at all. Sorry.
   
   Well you are waiting for a review on your code from myself or someone else 
-why not getting involved? There is no release engineering team doing this and 
it is up to all of us developers in the community to get our hands dirty. We 
all have different deployment environments and we all have different things we 
want to test. And, given this is the first public release with the AWS V2 SDK: 
it matters a lot that this thing ships. It will be the way we actually find 
real world bugs.
   
   Look on the hadoop common-dev list for the announcement of the next release 
candidate: once the announcement is made we have five days to test and vote on 
the RC. That is why we are under so much time pressure here.
   
   Note I've created a project, 
https://github.com/apache/hadoop-release-support , to assist in qualifying. One 
thing which would be good it would be some extra scripts we could run to help 
validate storage operations -operations which we can then execute against cloud 
storage from either the local host or a remote one -I am qualifying the Arm64 
build on a raspberry pi5 under my television and would like to have that 
testing fully automated.
   
   Anything you can do here will be very much appreciated. And like I said: I'm 
unlikely to be looking at any other code right now.




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-25 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17820598#comment-17820598
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1963476220

   Hello @steveloughran, Just a gentle reminder to review the PR. Thanks 😄




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-22 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17819609#comment-17819609
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1959287298

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 26s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  1s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  markdownlint  |   0m  1s |  |  markdownlint was not available.  
|
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  36m 54s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   0m 23s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 22s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 48s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  22m 54s |  |  branch has no errors 
when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  23m  7s |  |  Used diff version of patch file. 
Binary files and potentially other changes not applied. Please rebase and 
squash commits if necessary.  |
    _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 20s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 20s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |   0m 20s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 15s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 15s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | -0 :warning: |  checkstyle  |   0m 14s | 
[/results-checkstyle-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/15/artifact/out/results-checkstyle-hadoop-tools_hadoop-aws.txt)
 |  hadoop-tools/hadoop-aws: The patch generated 2 new + 6 unchanged - 0 fixed 
= 8 total (was 6)  |
   | +1 :green_heart: |  mvnsite  |   0m 19s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 11s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 19s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 47s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  22m  6s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 15s |  |  hadoop-aws in the patch passed. 
 |
   | +1 :green_heart: |  asflicense  |   0m 26s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   |  93m 44s |  |  |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/15/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/6407 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets markdownlint 
|
   | uname | Linux fe484a3a2b53 5.15.0-94-generic #104-Ubuntu SMP Tue Jan 9 
15:25:40 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / cbb8580f8732c7bd6cf5902b024d5545b4950295 |
   | Default Java | Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   | Multi-JDK versions | 
/usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 
/usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   |  Test Results | 
https://ci-hadoop.apache.org/job/hadoop-multibranch/

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-20 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17819101#comment-17819101
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1956027400

   Hello @steveloughran, Just a gentle reminder to review the PR. Thanks 😄 




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-16 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17817932#comment-17817932
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1948249298

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 23s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  
|
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 13s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 23s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   0m 19s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 20s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 23s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 17s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 44s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m 11s |  |  branch has no errors 
when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  19m 23s |  |  Used diff version of patch file. 
Binary files and potentially other changes not applied. Please rebase and 
squash commits if necessary.  |
    _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 19s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 19s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |   0m 19s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 14s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 14s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | -0 :warning: |  checkstyle  |   0m 11s | 
[/results-checkstyle-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/14/artifact/out/results-checkstyle-hadoop-tools_hadoop-aws.txt)
 |  hadoop-tools/hadoop-aws: The patch generated 2 new + 6 unchanged - 0 fixed 
= 8 total (was 6)  |
   | +1 :green_heart: |  mvnsite  |   0m 19s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m  9s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 16s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 40s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  18m 42s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 13s |  |  hadoop-aws in the patch passed. 
 |
   | +1 :green_heart: |  asflicense  |   0m 23s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   |  80m 59s |  |  |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/14/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/6407 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets markdownlint 
|
   | uname | Linux 7c86bd58d065 5.15.0-88-generic #98-Ubuntu SMP Mon Oct 2 
15:18:56 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / df48a67dcfcd9ef4578c2133660ced4924986e92 |
   | Default Java | Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   | Multi-JDK versions | 
/usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 
/usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   |  Test Results | 
https://ci-hadoop.apache.org/job/hadoop-multibranch/j

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-16 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17817922#comment-17817922
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1948198682

   > @bpahuja have you tested this with any of: s3 express, third party stores, 
gcs? Just want to make sure things work and the tests are skipped?
   
   
   Hello @steveloughran , wanted to confirm the requirement here ? Is it that 
you wanted me to ensure that this test is skipped when ran in other 
environments ? 
   
   If that's the case then IMO, it should work as other Integration tests do in 
this package as it is being inheriting `AbstractS3ATestBase`. 
   
   Do let me know if something other then this is needed from me :) 
   
   Thanks




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17817659#comment-17817659
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

steveloughran commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1946014245

   @bpahuja have you tested this with any of: s3 express, third party stores, 
gcs? Just want to make sure things work and the tests are skipped?
   
   I'll inevitably do the test runs anyway, but it'd save me time and effort, 
and as I don't run those regularly enough, multiple commits may be the source 
of regressions




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17817658#comment-17817658
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

steveloughran commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1490936359


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java:
##
@@ -37,7 +37,7 @@
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AInputPolicy;
 import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
-import org.apache.hadoop.fs.s3a.S3ObjectStorageClassFilter;
+import org.apache.hadoop.fs.s3a.api.S3ObjectStorageClassFilter;

Review Comment:
   this has to move down to L42 now, as its in a sub package.



##
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:
##
@@ -89,7 +92,7 @@ private FileSystem createFiles(String 
s3ObjectStorageClassFilter) throws Throwab
 FileSystem fs = contract.getTestFileSystem();
 Path dir = methodPath();
 fs.mkdirs(dir);

Review Comment:
   skip this for a marginal increase in performance. create() assumes there's a 
dir and just creates the object in the destination path without checks.



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##
@@ -52,6 +52,7 @@
 import java.util.concurrent.atomic.AtomicBoolean;
 import javax.annotation.Nullable;
 
+import org.apache.hadoop.fs.s3a.api.S3ObjectStorageClassFilter;

Review Comment:
   same



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java:
##
@@ -1525,6 +1526,11 @@ private Constants() {
*/
   public static final String READ_RESTORED_GLACIER_OBJECTS = 
"fs.s3a.glacier.read.restored.objects";
 
+  /**
+   * Default value of Read Restored Glacier objects config.
+   */
+  public static final String DEFAULT_READ_RESTORED_GLACIER_OBJECTS = 
S3ObjectStorageClassFilter.READ_ALL.toString();

Review Comment:
   split onto a new line to keep checkstyle happy



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java:
##
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import org.apache.hadoop.fs.s3a.api.S3ObjectStorageClassFilter;

Review Comment:
   new org.apache imports MUST go into that block for new classes. set your IDE 
up for this and life gets simpler for all



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/S3ObjectStorageClassFilter.java:
##
@@ -16,8 +16,9 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a;
+package org.apache.hadoop.fs.s3a.api;
 
+import org.apache.hadoop.fs.s3a.S3AFileSystem;

Review Comment:
   now, this is a branch new class. in which case the L22 import can go into 
the org.apache block and the S3aFilesystem one with it. the only reason it is 
in the wrong place in existing code is that the move to repackaged classes was 
a big search and replace only: no re-ordering





> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-13 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17817209#comment-17817209
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1943092389

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 21s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  
|
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 55s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   0m 18s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 21s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 25s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 18s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 44s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m 52s |  |  branch has no errors 
when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  20m  5s |  |  Used diff version of patch file. 
Binary files and potentially other changes not applied. Please rebase and 
squash commits if necessary.  |
    _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 18s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 19s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |   0m 19s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 16s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 16s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | -0 :warning: |  checkstyle  |   0m 13s | 
[/results-checkstyle-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/13/artifact/out/results-checkstyle-hadoop-tools_hadoop-aws.txt)
 |  hadoop-tools/hadoop-aws: The patch generated 3 new + 6 unchanged - 0 fixed 
= 9 total (was 6)  |
   | +1 :green_heart: |  mvnsite  |   0m 19s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m  9s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 17s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 39s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  19m 41s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 14s |  |  hadoop-aws in the patch passed. 
 |
   | +1 :green_heart: |  asflicense  |   0m 23s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   |  84m 10s |  |  |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/13/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/6407 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets markdownlint 
|
   | uname | Linux e4e0405a30ee 5.15.0-88-generic #98-Ubuntu SMP Mon Oct 2 
15:18:56 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 6742e8d0e79057623ea74c1d040be93653f3f2cd |
   | Default Java | Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   | Multi-JDK versions | 
/usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 
/usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   |  Test Results | 
https://ci-hadoop.apache.org/job/hadoop-multibranch/j

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-13 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17816958#comment-17816958
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1941195431

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   7m 18s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  
|
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  31m 16s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 23s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   0m 17s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 19s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 17s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 19s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 41s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  18m 49s |  |  branch has no errors 
when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  19m  2s |  |  Used diff version of patch file. 
Binary files and potentially other changes not applied. Please rebase and 
squash commits if necessary.  |
    _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 18s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 20s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |   0m 20s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 16s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 16s |  |  the patch passed  |
   | -1 :x: |  blanks  |   0m  0s | 
[/blanks-eol.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/12/artifact/out/blanks-eol.txt)
 |  The patch has 2 line(s) that end in blanks. Use git apply --whitespace=fix 
<>. Refer https://git-scm.com/docs/git-apply  |
   | -0 :warning: |  checkstyle  |   0m 10s | 
[/results-checkstyle-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/12/artifact/out/results-checkstyle-hadoop-tools_hadoop-aws.txt)
 |  hadoop-tools/hadoop-aws: The patch generated 3 new + 6 unchanged - 0 fixed 
= 9 total (was 6)  |
   | +1 :green_heart: |  mvnsite  |   0m 18s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m  9s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | -1 :x: |  javadoc  |   0m 16s | 
[/patch-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/12/artifact/out/patch-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt)
 |  hadoop-aws in the patch failed with JDK Private 
Build-1.8.0_392-8u392-ga-1~20.04-b08.  |
   | +1 :green_heart: |  spotbugs  |   0m 39s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  18m 50s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 22s |  |  hadoop-aws in the patch passed. 
 |
   | +1 :green_heart: |  asflicense  |   0m 23s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   |  86m 21s |  |  |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/12/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/6407 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets markdownlint 
|
   | uname | Linux 92c6e03fa6e0 5.15.0-88-generic #98-Ubuntu SMP Mon Oct 2 
15:18:56 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build to

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-13 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17816892#comment-17816892
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1487377659


##
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:
##
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.list;
+
+import static org.apache.hadoop.fs.s3a.Constants.READ_RESTORED_GLACIER_OBJECTS;

Review Comment:
   Sure will update in this test





> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-13 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17816890#comment-17816890
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1487376391


##
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md:
##
@@ -1309,6 +1309,20 @@ The switch to turn S3A auditing on or off.
   
 
 
+
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-13 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17816887#comment-17816887
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1487363104


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##
@@ -577,6 +583,11 @@ public void initialize(URI name, Configuration 
originalConf)
 
   s3aInternals = createS3AInternals();
 
+  s3ObjectStorageClassFilter = 
Optional.of(conf.getTrimmed(READ_RESTORED_GLACIER_OBJECTS,

Review Comment:
   By Default, we read all objects coming as part of listObjects API call and 
when an engine tries to access the archived file it fails with 
   
   
   software.amazon.awssdk.services.s3.model.InvalidObjectStateException: The 
operation is not valid for the object's storage class (Service: S3, Status 
Code: 403, Request ID: X05JDR633AAK4TBQ, Extended Request ID: 
uOxWdN4giUAuB9a4YWvnyrXPYCi2U35P5BrHhFO3aLSLLe4GtWhXGXCEJ/Ld5EyGr5b6VezTzeI=):InvalidObjectState
at 
org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:243)



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##
@@ -577,6 +583,11 @@ public void initialize(URI name, Configuration 
originalConf)
 
   s3aInternals = createS3AInternals();
 
+  s3ObjectStorageClassFilter = 
Optional.of(conf.getTrimmed(READ_RESTORED_GLACIER_OBJECTS,

Review Comment:
   By Default, we read all objects coming as part of listObjects API call and 
when an engine tries to access the archived file it fails with 
   
   ```
   software.amazon.awssdk.services.s3.model.InvalidObjectStateException: The 
operation is not valid for the object's storage class (Service: S3, Status 
Code: 403, Request ID: X05JDR633AAK4TBQ, Extended Request ID: 
uOxWdN4giUAuB9a4YWvnyrXPYCi2U35P5BrHhFO3aLSLLe4GtWhXGXCEJ/Ld5EyGr5b6VezTzeI=):InvalidObjectState
at 
org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:243)
   ```





> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-12 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17816885#comment-17816885
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1487359319


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##
@@ -577,6 +583,11 @@ public void initialize(URI name, Configuration 
originalConf)
 
   s3aInternals = createS3AInternals();
 
+  s3ObjectStorageClassFilter = 
Optional.of(conf.getTrimmed(READ_RESTORED_GLACIER_OBJECTS,
+  S3ObjectStorageClassFilter.READ_ALL.toString()))

Review Comment:
   Sure
   





> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-12 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17816882#comment-17816882
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1487351924


##
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:
##
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.list;
+
+import static org.apache.hadoop.fs.s3a.Constants.READ_RESTORED_GLACIER_OBJECTS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_DEEP_ARCHIVE;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_GLACIER;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfStorageClassTestsDisabled;
+import static 
org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.fs.s3a.S3ListRequest;
+import org.apache.hadoop.fs.s3a.S3ObjectStorageClassFilter;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.model.RestoreObjectRequest;
+import software.amazon.awssdk.services.s3.model.S3Object;
+import software.amazon.awssdk.services.s3.model.Tier;
+
+/**
+ * Tests of various cases related to Glacier/Deep Archive Storage class.
+ */
+@RunWith(Parameterized.class)
+public class ITestS3AReadRestoredGlacierObjects extends AbstractS3ATestBase {
+
+  enum Type { GLACIER_AND_DEEP_ARCHIVE, GLACIER }
+
+  @Parameterized.Parameters(name = "storage-class-{1}")
+  public static Collection data(){
+return Arrays.asList(new Object[][] {
+{Type.GLACIER_AND_DEEP_ARCHIVE, STORAGE_CLASS_GLACIER},
+{Type.GLACIER_AND_DEEP_ARCHIVE, STORAGE_CLASS_DEEP_ARCHIVE},
+{Type.GLACIER, STORAGE_CLASS_GLACIER}
+});
+  }
+
+  private final int maxRetries = 100;
+  private final int retryDelayMs = 5000;
+
+  private final Type type;
+  private final String glacierClass;
+
+  public ITestS3AReadRestoredGlacierObjects(Type type, String glacierClass) {
+this.type = type;
+this.glacierClass = glacierClass;
+  }
+
+  private FileSystem createFiles(String s3ObjectStorageClassFilter) throws 
Throwable {
+Configuration conf = this.createConfiguration();
+conf.set(READ_RESTORED_GLACIER_OBJECTS, s3ObjectStorageClassFilter);
+// Create Glacier objects:Storage Class:DEEP_ARCHIVE/GLACIER
+conf.set(STORAGE_CLASS, glacierClass);
+S3AContract contract = (S3AContract) createContract(conf);
+contract.init();
+
+FileSystem fs = contract.getTestFileSystem();
+Path dir = methodPath();
+fs.mkdirs(dir);
+Path path = new Path(dir, "file1");

Review Comment:
   Sure





> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy som

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-12 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17816881#comment-17816881
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1487349936


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:
##
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;

Review Comment:
   Sure will move this





> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-12 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17816880#comment-17816880
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1487348994


##
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:
##
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.list;
+
+import static org.apache.hadoop.fs.s3a.Constants.READ_RESTORED_GLACIER_OBJECTS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_DEEP_ARCHIVE;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_GLACIER;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfStorageClassTestsDisabled;
+import static 
org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.fs.s3a.S3ListRequest;
+import org.apache.hadoop.fs.s3a.S3ObjectStorageClassFilter;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.model.RestoreObjectRequest;
+import software.amazon.awssdk.services.s3.model.S3Object;
+import software.amazon.awssdk.services.s3.model.Tier;
+
+/**
+ * Tests of various cases related to Glacier/Deep Archive Storage class.
+ */
+@RunWith(Parameterized.class)
+public class ITestS3AReadRestoredGlacierObjects extends AbstractS3ATestBase {
+
+  enum Type { GLACIER_AND_DEEP_ARCHIVE, GLACIER }
+
+  @Parameterized.Parameters(name = "storage-class-{1}")
+  public static Collection data(){
+return Arrays.asList(new Object[][] {
+{Type.GLACIER_AND_DEEP_ARCHIVE, STORAGE_CLASS_GLACIER},
+{Type.GLACIER_AND_DEEP_ARCHIVE, STORAGE_CLASS_DEEP_ARCHIVE},
+{Type.GLACIER, STORAGE_CLASS_GLACIER}
+});
+  }
+
+  private final int maxRetries = 100;
+  private final int retryDelayMs = 5000;
+
+  private final Type type;
+  private final String glacierClass;
+
+  public ITestS3AReadRestoredGlacierObjects(Type type, String glacierClass) {
+this.type = type;
+this.glacierClass = glacierClass;
+  }
+
+  private FileSystem createFiles(String s3ObjectStorageClassFilter) throws 
Throwable {
+Configuration conf = this.createConfiguration();
+conf.set(READ_RESTORED_GLACIER_OBJECTS, s3ObjectStorageClassFilter);
+// Create Glacier objects:Storage Class:DEEP_ARCHIVE/GLACIER
+conf.set(STORAGE_CLASS, glacierClass);
+S3AContract contract = (S3AContract) createContract(conf);
+contract.init();
+
+FileSystem fs = contract.getTestFileSystem();
+Path dir = methodPath();
+fs.mkdirs(dir);
+Path path = new Path(dir, "file1");
+ContractTestUtils.touch(fs, path);
+return fs;
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+Configuration newConf = super.createConfiguration();
+skipIfStorageClassTestsDisabled(newConf);
+disableFilesystemCaching(newConf);
+removeBaseAndBucketOverrides(newConf, STORAGE_CLASS);
+return newConf;
+  }
+
+  @Test
+  public void testIgnoreGlacierObject() throws Throwable {
+Assume.assumeTrue(type == Type.GLACIER_AND_DEEP_ARCHIVE);
+try (FileSystem fs = 
createFiles(S3ObjectStor

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-12 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17816879#comment-17816879
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1487348795


##
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:
##
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.list;
+
+import static org.apache.hadoop.fs.s3a.Constants.READ_RESTORED_GLACIER_OBJECTS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_DEEP_ARCHIVE;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_GLACIER;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfStorageClassTestsDisabled;
+import static 
org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.fs.s3a.S3ListRequest;
+import org.apache.hadoop.fs.s3a.S3ObjectStorageClassFilter;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.model.RestoreObjectRequest;
+import software.amazon.awssdk.services.s3.model.S3Object;
+import software.amazon.awssdk.services.s3.model.Tier;
+
+/**
+ * Tests of various cases related to Glacier/Deep Archive Storage class.
+ */
+@RunWith(Parameterized.class)
+public class ITestS3AReadRestoredGlacierObjects extends AbstractS3ATestBase {
+
+  enum Type { GLACIER_AND_DEEP_ARCHIVE, GLACIER }
+
+  @Parameterized.Parameters(name = "storage-class-{1}")
+  public static Collection data(){
+return Arrays.asList(new Object[][] {
+{Type.GLACIER_AND_DEEP_ARCHIVE, STORAGE_CLASS_GLACIER},
+{Type.GLACIER_AND_DEEP_ARCHIVE, STORAGE_CLASS_DEEP_ARCHIVE},
+{Type.GLACIER, STORAGE_CLASS_GLACIER}
+});
+  }
+
+  private final int maxRetries = 100;
+  private final int retryDelayMs = 5000;
+
+  private final Type type;
+  private final String glacierClass;
+
+  public ITestS3AReadRestoredGlacierObjects(Type type, String glacierClass) {
+this.type = type;
+this.glacierClass = glacierClass;
+  }
+
+  private FileSystem createFiles(String s3ObjectStorageClassFilter) throws 
Throwable {
+Configuration conf = this.createConfiguration();

Review Comment:
   Doing this to get access to the conf object to set the 
READ_RESTORED_GLACIER_OBJECTS config test wise



##
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:
##
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-09 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17816203#comment-17816203
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

steveloughran commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1484732749


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:
##
@@ -27,19 +27,25 @@
 
 /**
  * 
- * {@link S3ObjectStorageClassFilter} will filter the S3 files based on the 
{@code fs.s3a.glacier.read.restored.objects} configuration set in {@link 
S3AFileSystem}
+ * {@link S3ObjectStorageClassFilter} will filter the S3 files based on the
+ * {@code fs.s3a.glacier.read.restored.objects} configuration set in {@link 
S3AFileSystem}
  * The config can have 3 values:
- * {@code READ_ALL}: Retrieval of Glacier files will fail with 
InvalidObjectStateException: The operation is not valid for the object's 
storage class.
- * {@code SKIP_ALL_GLACIER}: If this value is set then this will ignore any S3 
Objects which are tagged with Glacier storage classes and retrieve the others.
- * {@code READ_RESTORED_GLACIER_OBJECTS}: If this value is set then restored 
status of the Glacier object will be checked, if restored the objects would be 
read like normal S3 objects else they will be ignored as the objects would not 
have been retrieved from the S3 Glacier.
+ * {@code READ_ALL}: Retrieval of Glacier files will fail with 
InvalidObjectStateException:
+ * The operation is not valid for the object's storage class.
+ * {@code SKIP_ALL_GLACIER}: If this value is set then this will ignore any S3 
Objects which are
+ * tagged with Glacier storage classes and retrieve the others.
+ * {@code READ_RESTORED_GLACIER_OBJECTS}: If this value is set then restored 
status of the Glacier
+ * object will be checked, if restored the objects would be read like normal 
S3 objects
+ * else they will be ignored as the objects would not have been retrieved from 
the S3 Glacier.
  * 
  */
 public enum S3ObjectStorageClassFilter {

Review Comment:
   I like this design you know: enum based mapping to closures.



##
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:
##
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.list;
+
+import static org.apache.hadoop.fs.s3a.Constants.READ_RESTORED_GLACIER_OBJECTS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_DEEP_ARCHIVE;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_GLACIER;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfStorageClassTestsDisabled;
+import static 
org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.fs.s3a.S3ListRequest;
+import org.apache.hadoop.fs.s3a.S3ObjectStorageClassFilter;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.model.RestoreObjectRequest;
+import software.amazon.awssdk.services.s3.model.S3Object;
+import software.amazon.awssdk.services.s3.model.Tier;
+
+/**
+ * Tests of various cases related to Glacier/Deep Archive Storage class.
+ */
+@RunWith(Parameterized.class)
+

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-08 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17815565#comment-17815565
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1933545491

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 20s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  
|
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  31m 18s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   0m 18s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 20s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 19s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 42s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  18m 51s |  |  branch has no errors 
when building and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 19s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |   0m 19s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 15s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 15s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | -0 :warning: |  checkstyle  |   0m 10s | 
[/results-checkstyle-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/11/artifact/out/results-checkstyle-hadoop-tools_hadoop-aws.txt)
 |  hadoop-tools/hadoop-aws: The patch generated 2 new + 6 unchanged - 0 fixed 
= 8 total (was 6)  |
   | +1 :green_heart: |  mvnsite  |   0m 18s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m  9s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 16s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 40s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  18m 49s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 26s |  |  hadoop-aws in the patch passed. 
 |
   | +1 :green_heart: |  asflicense  |   0m 21s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   |  79m 33s |  |  |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/11/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/6407 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets markdownlint 
|
   | uname | Linux cc61e9a0a8ab 5.15.0-88-generic #98-Ubuntu SMP Mon Oct 2 
15:18:56 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 2e6562a3ba9691060f7b76fd51dcfa554469652c |
   | Default Java | Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   | Multi-JDK versions | 
/usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 
/usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   |  Test Results | 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/11/testReport/ |
   | Max. process+thread count | 626 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws |
   | Console output | 
http

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-07 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17815535#comment-17815535
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1933450335

   Hi @steveloughran, Have addressed the checkstyle violations in the latest 
commit. 




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-07 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17815330#comment-17815330
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

steveloughran commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1932206238

   @bpahuja can you not do rebase and force push unless some other change has 
broken the merge? breaks the "changes since your last review" feature. If 
someone has stepped on the code, don't be afraid to rebase -I generally squash 
all changes first there to make the rebase easier




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-07 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17815328#comment-17815328
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

steveloughran commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1932200850

   checkstyle failures, we all get those. and sometimes we have to say "better 
if broken"
   ```
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java:256:
  /**: First sentence should end with a period. [JavadocStyle]
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java:617:
.optionalObjectAttributes(OptionalObjectAttributes.RESTORE_STATUS) // 
Optional Attribute to get the Restored Status of the Glacier Objects: Line is 
longer than 100 characters (found 146). [LineLength]
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java:419:
  /**: First sentence should end with a period. [JavadocStyle]
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java:420:
   * Return the S3ObjectStorageClassFilter object for S3A, whose value is set 
according to the config {@code fs.s3a.glacier.read.restored.objects}: Line is 
longer than 100 characters (found 146). [LineLength]
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java:467:
// Handle Glacier Storage Class objects based on the config 
fs.s3a.glacier.read.restored.objects value set: Line is longer than 100 
characters (found 114). [LineLength]
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java:468:
if ( s3ObjectStorageClassFilter.getFilter().apply(s3Object) &&  
acceptor.accept(keyPath, s3Object) && filter.accept(keyPath)) {: Line is longer 
than 100 characters (found 135). [LineLength]
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java:468:
if ( s3ObjectStorageClassFilter.getFilter().apply(s3Object) &&  
acceptor.accept(keyPath, s3Object) && filter.accept(keyPath)) {:12: '(' is 
followed by whitespace. [ParenPad]
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:30:
 * {@link S3ObjectStorageClassFilter} will filter the S3 files based on the 
{@code fs.s3a.glacier.read.restored.objects} configuration set in {@link 
S3AFileSystem}: Line is longer than 100 characters (found 163). [LineLength]
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:32:
 * {@code READ_ALL}: Retrieval of Glacier files will fail with 
InvalidObjectStateException: The operation is not valid for the object's 
storage class.: Line is longer than 100 characters (found 150). [LineLength]
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:33:
 * {@code SKIP_ALL_GLACIER}: If this value is set then this will ignore any S3 
Objects which are tagged with Glacier storage classes and retrieve the others.: 
Line is longer than 100 characters (found 157). [LineLength]
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:34:
 * {@code READ_RESTORED_GLACIER_OBJECTS}: If this value is set then restored 
status of the Glacier object will be checked, if restored the objects would be 
read like normal S3 objects else they will be ignored as the objects would not 
have been retrieved from the S3 Glacier.: Line is longer than 100 characters 
(found 275). [LineLength]
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:42:
  private static final Set GLACIER_STORAGE_CLASSES = 
Sets.newHashSet(ObjectStorageClass.GLACIER, ObjectStorageClass.DEEP_ARCHIVE);: 
Line is longer than 100 characters (found 150). [LineLength]
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:50:
  /**: First sentence should end with a period. [JavadocStyle]
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:59:
  /**: First sentence should end with a period. [JavadocStyle]
   
./hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:68:
  /**: First sentence should end with a period. [JavadocStyle]
   
./hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:70:
  private final int MAX_RETRIES = 100;:21: Name 'MAX_RETRIES' must match 
pattern '^[a-z][a-zA-Z0-9]*$'. [MemberName]
   
./hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:71:
  private final int RETRY_DELAY_MS = 5000;:21: Name 'RETRY_DELAY_MS' must match 
pattern '^[a-z][a-zA-Z0-9]*$'. [MemberName]
   
./hadoop-tools/hadoop-aws/src/test/java/org/apa

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-07 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17815249#comment-17815249
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1931817110

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 20s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  1s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  markdownlint  |   0m  1s |  |  markdownlint was not available.  
|
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  31m 45s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   0m 20s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 21s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 25s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 18s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 43s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m 36s |  |  branch has no errors 
when building and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 19s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |   0m 19s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 16s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 16s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | -0 :warning: |  checkstyle  |   0m 12s | 
[/results-checkstyle-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/10/artifact/out/results-checkstyle-hadoop-tools_hadoop-aws.txt)
 |  hadoop-tools/hadoop-aws: The patch generated 21 new + 6 unchanged - 0 fixed 
= 27 total (was 6)  |
   | +1 :green_heart: |  mvnsite  |   0m 18s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m  9s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 17s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 42s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  19m  1s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 28s |  |  hadoop-aws in the patch passed. 
 |
   | +1 :green_heart: |  asflicense  |   0m 23s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   |  81m 32s |  |  |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/10/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/6407 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets markdownlint 
|
   | uname | Linux 8e4ff0d8df99 5.15.0-88-generic #98-Ubuntu SMP Mon Oct 2 
15:18:56 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / c5dcd5c4426135b7b054ed19f4bcf8e68d955d77 |
   | Default Java | Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   | Multi-JDK versions | 
/usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 
/usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   |  Test Results | 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/10/testReport/ |
   | Max. process+thread count | 559 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws |
   | Console output | 
ht

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-07 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17815211#comment-17815211
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1931661739

   Build failed due to an OOM Error, 
   
   build passing on local 
   
   ```
   [INFO] 

   [INFO] BUILD FAILURE
   [INFO] 

   [INFO] Total time:  01:33 min
   [INFO] Finished at: 2024-02-07T08:13:12Z
   [INFO] 

   [ERROR] unable to create new native thread -> [Help 1]
   [ERROR] 
   [ERROR] To see the full stack trace of the errors, re-run Maven with the -e 
switch.
   [ERROR] Re-run Maven using the -X switch to enable full debug logging.
   [ERROR] 
   [ERROR] For more information about the errors and possible solutions, please 
read the following articles:
   [ERROR] [Help 1] 
http://cwiki.apache.org/confluence/display/MAVEN/OutOfMemoryError
   
   ```




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-07 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17815153#comment-17815153
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1931517312

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 23s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  
|
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | -1 :x: |  mvninstall  |   1m 40s | 
[/branch-mvninstall-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/9/artifact/out/branch-mvninstall-root.txt)
 |  root in trunk failed.  |
   | -1 :x: |  compile  |   0m 22s | 
[/branch-compile-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/9/artifact/out/branch-compile-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt)
 |  hadoop-aws in trunk failed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.  |
   | -1 :x: |  compile  |   3m 21s | 
[/branch-compile-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/9/artifact/out/branch-compile-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt)
 |  hadoop-aws in trunk failed with JDK Private 
Build-1.8.0_392-8u392-ga-1~20.04-b08.  |
   | -0 :warning: |  checkstyle  |   0m 19s | 
[/buildtool-branch-checkstyle-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/9/artifact/out/buildtool-branch-checkstyle-hadoop-tools_hadoop-aws.txt)
 |  The patch fails to run checkstyle in hadoop-aws  |
   | -1 :x: |  mvnsite  |   0m 21s | 
[/branch-mvnsite-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/9/artifact/out/branch-mvnsite-hadoop-tools_hadoop-aws.txt)
 |  hadoop-aws in trunk failed.  |
   | -1 :x: |  javadoc  |   0m 22s | 
[/branch-javadoc-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/9/artifact/out/branch-javadoc-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt)
 |  hadoop-aws in trunk failed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.  |
   | -1 :x: |  javadoc  |   0m  5s | 
[/branch-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/9/artifact/out/branch-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt)
 |  hadoop-aws in trunk failed with JDK Private 
Build-1.8.0_392-8u392-ga-1~20.04-b08.  |
   | +1 :green_heart: |  spotbugs  |   1m 45s |  |  trunk passed  |
   | -1 :x: |  shadedclient  |   4m 27s |  |  branch has errors when building 
and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 20s | 
[/patch-mvninstall-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/9/artifact/out/patch-mvninstall-hadoop-tools_hadoop-aws.txt)
 |  hadoop-aws in the patch failed.  |
   | -1 :x: |  compile  |   0m 20s | 
[/patch-compile-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/9/artifact/out/patch-compile-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt)
 |  hadoop-aws in the patch failed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.  |
   | -1 :x: |  javac  |   0m 20s | 
[/patch-compile-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/9/artifact/out/patch-compile-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt)
 |  hadoop-aws in the patch failed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s 

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-06 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17814712#comment-17814712
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1929172260

   > I 'd go for getTrimmed(READ_RESTORED_GLACIER_OBJECTS, ""); if empty string 
map to empty optional, otherwise .toupper and valueof. one thing to consider: 
meaningful failure if the value doesn't map.
   
   > or we just go for "upper case is required" and use what you've proposed. 
more brittle but simpler?
   
   
   Regarding the above, I will update the config read to getTrimmed and set the 
config values of defualt to READ_ALL instead of "". 
   
   > meaningful failure if the value doesn't map
   
   the valueof method will throw the illegalArgs exception if an invalid value 
is set in the config




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-06 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17814710#comment-17814710
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1479512215


##
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:
##
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.list;
+
+import static org.apache.hadoop.fs.s3a.Constants.READ_RESTORED_GLACIER_OBJECTS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_DEEP_ARCHIVE;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_GLACIER;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfStorageClassTestsDisabled;
+import static 
org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REJECT_OUT_OF_SPAN_OPERATIONS;
+
+import java.util.Arrays;
+import java.util.Collection;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.fs.s3a.S3ListRequest;
+import org.apache.hadoop.fs.s3a.S3ObjectStorageClassFilter;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.model.GlacierJobParameters;
+import software.amazon.awssdk.services.s3.model.ObjectStorageClass;
+import software.amazon.awssdk.services.s3.model.RestoreObjectRequest;
+import software.amazon.awssdk.services.s3.model.RestoreRequest;
+import software.amazon.awssdk.services.s3.model.S3Object;
+import software.amazon.awssdk.services.s3.model.Tier;
+
+@RunWith(Parameterized.class)
+public class ITestS3AReadRestoredGlacierObjects extends AbstractS3ATestBase {
+
+  enum Type { GLACIER_AND_DEEP_ARCHIVE, GLACIER }
+
+  @Parameterized.Parameters

Review Comment:
   Sure will update



##
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:
##
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.list;
+
+import static org.apache.hadoop.fs.s3a.Constants.READ_RESTORED_GLACIER_OBJECTS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_DEEP_ARCHIVE;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_GLACIER;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfStorageClassTestsDisabled;
+import static 
org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REJECT_OUT_OF_SPAN_OPERATIONS;
+
+imp

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-06 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17814709#comment-17814709
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1479511520


##
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:
##
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.list;
+
+import static org.apache.hadoop.fs.s3a.Constants.READ_RESTORED_GLACIER_OBJECTS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_DEEP_ARCHIVE;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_GLACIER;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfStorageClassTestsDisabled;
+import static 
org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REJECT_OUT_OF_SPAN_OPERATIONS;
+
+import java.util.Arrays;
+import java.util.Collection;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.fs.s3a.S3ListRequest;
+import org.apache.hadoop.fs.s3a.S3ObjectStorageClassFilter;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.model.GlacierJobParameters;
+import software.amazon.awssdk.services.s3.model.ObjectStorageClass;
+import software.amazon.awssdk.services.s3.model.RestoreObjectRequest;
+import software.amazon.awssdk.services.s3.model.RestoreRequest;
+import software.amazon.awssdk.services.s3.model.S3Object;
+import software.amazon.awssdk.services.s3.model.Tier;
+
+@RunWith(Parameterized.class)
+public class ITestS3AReadRestoredGlacierObjects extends AbstractS3ATestBase {
+
+  enum Type { GLACIER_AND_DEEP_ARCHIVE, GLACIER }
+
+  @Parameterized.Parameters
+  public static Collection data(){
+return Arrays.asList(new Object[][] {
+{Type.GLACIER_AND_DEEP_ARCHIVE, STORAGE_CLASS_GLACIER},
+{Type.GLACIER_AND_DEEP_ARCHIVE, STORAGE_CLASS_DEEP_ARCHIVE},
+{Type.GLACIER, STORAGE_CLASS_GLACIER}
+});
+  }
+
+  private int retryCount = 0;
+  private final int MAX_RETRIES = 100;
+  private final int RETRY_DELAY_MS = 5000;
+
+  private Type type;
+  private String glacierClass;
+
+  public ITestS3AReadRestoredGlacierObjects(Type type, String glacierClass) {
+this.type = type;
+this.glacierClass = glacierClass;
+  }
+
+  private FileSystem createFiles(String s3ObjectStorageClassFilter) throws 
Throwable {
+Configuration conf = this.createConfiguration();
+conf.set(READ_RESTORED_GLACIER_OBJECTS, s3ObjectStorageClassFilter);
+conf.set(STORAGE_CLASS, glacierClass); // Create Glacier objects:Storage 
Class:DEEP_ARCHIVE/GLACIER
+S3AContract contract = (S3AContract) createContract(conf);
+contract.init();
+
+FileSystem fs = contract.getTestFileSystem();
+Path dir = methodPath();
+fs.mkdirs(dir);
+Path path = new Path(dir, "file1");
+ContractTestUtils.touch(fs, path);
+return fs;
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+Configuration newConf = super.createConfiguration();
+skipIfStorageClassTestsDisabled(newConf);
+disableFilesystemCaching(newConf);
+removeBaseAndBucketOverrides(newConf, STORAGE_CLASS);
+newConf.set(REJECT_OUT_OF_SPAN_OPERATIONS, "false");
+return newConf;
+  }
+
+  @Test
+  public void testIgnoreGlacierObject() throws Throwable {
+Assume.assumeTrue(type == Type.GLACIER_AND_DEEP_ARCHIVE);
+   

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-06 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17814708#comment-17814708
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1479510851


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:
##
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import java.util.Set;
+import java.util.function.Function;
+import software.amazon.awssdk.services.s3.model.ObjectStorageClass;
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+
+/**
+ * 
+ * {@link S3ObjectStorageClassFilter} will filter the S3 files based on the 
{@code fs.s3a.glacier.read.restored.objects} configuration set in {@link 
S3AFileSystem}
+ * The config can have 3 values:
+ * {@code READ_ALL}: Retrieval of Glacier files will fail with 
InvalidObjectStateException: The operation is not valid for the object's 
storage class.
+ * {@code SKIP_ALL_GLACIER}: If this value is set then this will ignore any S3 
Objects which are tagged with Glacier storage classes and retrieve the others.
+ * {@code READ_RESTORED_GLACIER_OBJECTS}: If this value is set then restored 
status of the Glacier object will be checked, if restored the objects would be 
read like normal S3 objects else they will be ignored as the objects would not 
have been retrieved from the S3 Glacier.
+ * 
+ */
+public enum S3ObjectStorageClassFilter {
+  READ_ALL(o -> true),
+  SKIP_ALL_GLACIER(S3ObjectStorageClassFilter::isNotGlacierObject),
+  
READ_RESTORED_GLACIER_OBJECTS(S3ObjectStorageClassFilter::isCompletedRestoredObject);
+
+  private static final Set GLACIER_STORAGE_CLASSES = 
Sets.newHashSet(ObjectStorageClass.GLACIER, ObjectStorageClass.DEEP_ARCHIVE);
+
+  private final Function filter;
+
+  S3ObjectStorageClassFilter(Function filter) {
+this.filter = filter;
+  }
+
+  private static boolean isNotGlacierObject(S3Object object) {

Review Comment:
   Sure will add for methods in the class



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java:
##
@@ -25,6 +25,7 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 
+import org.apache.hadoop.fs.s3a.S3ObjectStorageClassFilter;

Review Comment:
   Sure





> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-05 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17814523#comment-17814523
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

steveloughran commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1478954399


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java:
##
@@ -25,6 +25,7 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 
+import org.apache.hadoop.fs.s3a.S3ObjectStorageClassFilter;

Review Comment:
   can you move down to the rest of the org.apache. 
   these guava things are in the wrong block due to the big search and replace 
which created them



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##
@@ -441,6 +441,8 @@ public class S3AFileSystem extends FileSystem implements 
StreamCapabilities,
*/
   private boolean isCSEEnabled;
 
+  private S3ObjectStorageClassFilter s3ObjectStorageClassFilter;

Review Comment:
   nit: add a javadoc -and remember a "." at the end to keep all javadoc 
versions happy



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:
##
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import java.util.Set;
+import java.util.function.Function;
+import software.amazon.awssdk.services.s3.model.ObjectStorageClass;
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+
+/**
+ * 
+ * {@link S3ObjectStorageClassFilter} will filter the S3 files based on the 
{@code fs.s3a.glacier.read.restored.objects} configuration set in {@link 
S3AFileSystem}
+ * The config can have 3 values:
+ * {@code READ_ALL}: Retrieval of Glacier files will fail with 
InvalidObjectStateException: The operation is not valid for the object's 
storage class.
+ * {@code SKIP_ALL_GLACIER}: If this value is set then this will ignore any S3 
Objects which are tagged with Glacier storage classes and retrieve the others.
+ * {@code READ_RESTORED_GLACIER_OBJECTS}: If this value is set then restored 
status of the Glacier object will be checked, if restored the objects would be 
read like normal S3 objects else they will be ignored as the objects would not 
have been retrieved from the S3 Glacier.
+ * 
+ */
+public enum S3ObjectStorageClassFilter {
+  READ_ALL(o -> true),
+  SKIP_ALL_GLACIER(S3ObjectStorageClassFilter::isNotGlacierObject),
+  
READ_RESTORED_GLACIER_OBJECTS(S3ObjectStorageClassFilter::isCompletedRestoredObject);
+
+  private static final Set GLACIER_STORAGE_CLASSES = 
Sets.newHashSet(ObjectStorageClass.GLACIER, ObjectStorageClass.DEEP_ARCHIVE);
+
+  private final Function filter;
+
+  S3ObjectStorageClassFilter(Function filter) {
+this.filter = filter;
+  }
+
+  private static boolean isNotGlacierObject(S3Object object) {

Review Comment:
   add javadocs all the way down here, thanks



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##
@@ -581,6 +583,12 @@ public void initialize(URI name, Configuration 
originalConf)
 
   s3aInternals = createS3AInternals();
 
+  s3ObjectStorageClassFilter = 
Optional.ofNullable(conf.get(READ_RESTORED_GLACIER_OBJECTS))

Review Comment:
   @ahmarsuhail but doing the the way it is does handle case differences.
   
   I'd go for getTrimmed(READ_RESTORED_GLACIER_OBJECTS, ""); if empty string 
map to empty optional, otherwise .toupper and valueof. one thing to consider: 
meaningful failure if the value doesn't map.
   
   I'd change Configuration to do that case mapping if it wasn't such a 
critical class



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##
@@ -581,6 +583,12 @@ public void initialize(URI name, Configuration 
originalConf)
 
   s3aInternals = createS3AInternals();
 
+  s3ObjectStorageClassFilter = 
Optional.ofNullable(conf.get(READ_RESTORED_GLACIER_OBJECTS)

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-05 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17814321#comment-17814321
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1926752080

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 21s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  
|
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  31m 31s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 23s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   0m 19s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 19s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 23s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 17s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 41s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m 21s |  |  branch has no errors 
when building and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 16s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 20s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |   0m 20s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 16s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 16s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | -0 :warning: |  checkstyle  |   0m 13s | 
[/results-checkstyle-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/8/artifact/out/results-checkstyle-hadoop-tools_hadoop-aws.txt)
 |  hadoop-tools/hadoop-aws: The patch generated 21 new + 6 unchanged - 0 fixed 
= 27 total (was 6)  |
   | +1 :green_heart: |  mvnsite  |   0m 18s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 10s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 17s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 41s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  19m  5s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 27s |  |  hadoop-aws in the patch passed. 
 |
   | +1 :green_heart: |  asflicense  |   0m 24s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   |  81m  4s |  |  |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/8/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/6407 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets markdownlint 
|
   | uname | Linux 6f1c70e86dea 5.15.0-88-generic #98-Ubuntu SMP Mon Oct 2 
15:18:56 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / d035d1561d64e8d1279a7c6240bff6f09494b64c |
   | Default Java | Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   | Multi-JDK versions | 
/usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 
/usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   |  Test Results | 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/8/testReport/ |
   | Max. process+thread count | 663 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws |
   | Console output | 
https

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-02-05 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17814283#comment-17814283
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1926503914

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m  0s |  |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m 16s |  |  
https://github.com/apache/hadoop/pull/6407 does not apply to trunk. Rebase 
required? Wrong Branch? See 
https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute for help.  
|
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | GITHUB PR | https://github.com/apache/hadoop/pull/6407 |
   | Console output | 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/7/console |
   | versions | git=2.34.1 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-01-29 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17811860#comment-17811860
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

ahmarsuhail commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1469491275


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##
@@ -581,6 +583,12 @@ public void initialize(URI name, Configuration 
originalConf)
 
   s3aInternals = createS3AInternals();
 
+  s3ObjectStorageClassFilter = 
Optional.ofNullable(conf.get(READ_RESTORED_GLACIER_OBJECTS))

Review Comment:
   would prefer `conf.get(READ_RESTORED_GLACIER_OBJECTS, READ_ALL)` , meaning 
READ_ALL is the default. and then you can get rid of the `orElse()`



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java:
##
@@ -411,4 +416,8 @@ public RequestFactory getRequestFactory() {
   public boolean isCSEEnabled() {
 return isCSEEnabled;
   }
+
+  public S3ObjectStorageClassFilter s3ObjectsStorageClassFilter() {

Review Comment:
   nit: rename to getS3ObjectStorageClassFilter(), and add java docs for the 
method



##
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:
##
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.list;
+
+import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER;
+import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER_ARRAY;
+import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER_DISK;
+import static org.apache.hadoop.fs.s3a.Constants.READ_RESTORED_GLACIER_OBJECTS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS;
+import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_GLACIER;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfStorageClassTestsDisabled;
+import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_STORAGE_CLASS;
+import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.decodeBytes;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+import java.io.File;
+import java.nio.file.AccessDeniedException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3ObjectStorageClassFilter;
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class ITestS3AReadRestoredGlacierObjects extends AbstractS3ATestBase {
+
+  @Parameterized.Parameters(name = "fast-upload-buffer-{0}")
+  public static Collection params() {
+return Arrays.asList(new Object[][]{
+{FAST_UPLOAD_BUFFER_DISK},
+{FAST_UPLOAD_BUFFER_ARRAY}
+});
+  }
+
+  private final String fastUploadBufferType;
+
+  public ITestS3AReadRestoredGlacierObjects(String fastUploadBufferType) {
+this.fastUploadBufferType = fastUploadBufferType;
+  }
+
+  @Test
+  public void testIgnoreGlacierObject() throws Throwable {
+try (FileSystem fs = 
createFiles(S3ObjectStorageClassFilter.SKIP_ALL_GLACIER.name())) {
+  Assertions.assertThat(
+  fs.listStatus(methodPath()))
+.describedAs("FileStatus List of %s", methodPath()).isEmpty();
+}
+  }
+
+  @Test
+  public void testIgnoreRestoringGlacierObject() throws Throwable {
+try (FileSystem fs = 
createFiles(S3ObjectStorageClassFilter.READ_RESTORED_GLACIER_OBJECTS.name())) {
+  Assertions.assertThat(
+  fs.listStatus

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-01-24 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17810477#comment-17810477
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1908429821

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 23s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  
|
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 31s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   0m 17s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 20s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 18s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 44s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m 20s |  |  branch has no errors 
when building and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 20s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 21s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |   0m 21s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 15s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 15s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | -0 :warning: |  checkstyle  |   0m 10s | 
[/results-checkstyle-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/6/artifact/out/results-checkstyle-hadoop-tools_hadoop-aws.txt)
 |  hadoop-tools/hadoop-aws: The patch generated 13 new + 6 unchanged - 0 fixed 
= 19 total (was 6)  |
   | +1 :green_heart: |  mvnsite  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m  9s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 17s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 45s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  19m 17s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 28s |  |  hadoop-aws in the patch passed. 
 |
   | +1 :green_heart: |  asflicense  |   0m 22s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   |  82m 58s |  |  |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/6/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/6407 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets markdownlint 
|
   | uname | Linux 732160df4277 5.15.0-88-generic #98-Ubuntu SMP Mon Oct 2 
15:18:56 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 4e9b673f0ace4ebaa8c1e467617b6fe5d8ab93b4 |
   | Default Java | Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   | Multi-JDK versions | 
/usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 
/usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   |  Test Results | 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/6/testReport/ |
   | Max. process+thread count | 551 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws |
   | Console output | 
https

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-01-24 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17810291#comment-17810291
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1907733303

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 21s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  1s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  markdownlint  |   0m  1s |  |  markdownlint was not available.  
|
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  31m 57s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   0m 21s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 21s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 19s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 45s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m 28s |  |  branch has no errors 
when building and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 19s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 20s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |   0m 20s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 16s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 16s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | -0 :warning: |  checkstyle  |   0m 12s | 
[/results-checkstyle-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/5/artifact/out/results-checkstyle-hadoop-tools_hadoop-aws.txt)
 |  hadoop-tools/hadoop-aws: The patch generated 18 new + 6 unchanged - 0 fixed 
= 24 total (was 6)  |
   | +1 :green_heart: |  mvnsite  |   0m 18s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 10s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 16s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 43s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  19m 22s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 26s |  |  hadoop-aws in the patch passed. 
 |
   | +1 :green_heart: |  asflicense  |   0m 23s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   |  81m 44s |  |  |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/5/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/6407 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets markdownlint 
|
   | uname | Linux 4dfd5fc0d4f0 5.15.0-88-generic #98-Ubuntu SMP Mon Oct 2 
15:18:56 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / f0144a759ad6604af682fd3c28c60d303e7036d5 |
   | Default Java | Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   | Multi-JDK versions | 
/usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 
/usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   |  Test Results | 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/5/testReport/ |
   | Max. process+thread count | 552 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws |
   | Console output | 
https

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-01-22 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17809654#comment-17809654
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1904868824

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 24s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  xmllint  |   0m  0s |  |  xmllint was not available.  |
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  
|
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 2 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  13m 44s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  22m 14s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   9m 50s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   8m 59s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   2m 29s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 20s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 59s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 46s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m  0s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 15s |  |  branch has no errors 
when building and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   0m 15s | 
[/patch-mvninstall-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/3/artifact/out/patch-mvninstall-hadoop-tools_hadoop-aws.txt)
 |  hadoop-aws in the patch failed.  |
   | -1 :x: |  compile  |   8m 45s | 
[/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/3/artifact/out/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt)
 |  root in the patch failed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.  |
   | -1 :x: |  javac  |   8m 45s | 
[/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/3/artifact/out/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt)
 |  root in the patch failed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.  |
   | -1 :x: |  compile  |   8m  9s | 
[/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/3/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt)
 |  root in the patch failed with JDK Private 
Build-1.8.0_392-8u392-ga-1~20.04-b08.  |
   | -1 :x: |  javac  |   8m  9s | 
[/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/3/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt)
 |  root in the patch failed with JDK Private 
Build-1.8.0_392-8u392-ga-1~20.04-b08.  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | -0 :warning: |  checkstyle  |   2m  9s | 
[/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/3/artifact/out/results-checkstyle-root.txt)
 |  root: The patch generated 22 new + 6 unchanged - 0 fixed = 28 total (was 6) 
 |
   | -1 :x: |  mvnsite  |   0m 24s | 
[/patch-mvnsite-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/3/artifact/out/patch-mvnsite-hadoop-tools_hadoop-aws.txt)
 |  hadoop-aws in the patch failed.  |
   | +1 :green_heart: |  javadoc  |   0m 50s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 47s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | -1 :x: |  spotbugs  |   0m 25s | 
[/patch-spotbugs-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-64

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-01-22 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17809655#comment-17809655
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1904870927

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 22s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  xmllint  |   0m  0s |  |  xmllint was not available.  |
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  
|
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 2 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  13m 42s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  22m 18s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   9m 43s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   9m  4s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   2m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 18s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 57s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 48s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m  3s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 11s |  |  branch has no errors 
when building and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   0m 16s | 
[/patch-mvninstall-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/4/artifact/out/patch-mvninstall-hadoop-tools_hadoop-aws.txt)
 |  hadoop-aws in the patch failed.  |
   | -1 :x: |  compile  |   8m 39s | 
[/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/4/artifact/out/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt)
 |  root in the patch failed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.  |
   | -1 :x: |  javac  |   8m 39s | 
[/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/4/artifact/out/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt)
 |  root in the patch failed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.  |
   | -1 :x: |  compile  |   8m  7s | 
[/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/4/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt)
 |  root in the patch failed with JDK Private 
Build-1.8.0_392-8u392-ga-1~20.04-b08.  |
   | -1 :x: |  javac  |   8m  7s | 
[/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/4/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt)
 |  root in the patch failed with JDK Private 
Build-1.8.0_392-8u392-ga-1~20.04-b08.  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | -0 :warning: |  checkstyle  |   2m 12s | 
[/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/4/artifact/out/results-checkstyle-root.txt)
 |  root: The patch generated 22 new + 6 unchanged - 0 fixed = 28 total (was 6) 
 |
   | -1 :x: |  mvnsite  |   0m 23s | 
[/patch-mvnsite-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/4/artifact/out/patch-mvnsite-hadoop-tools_hadoop-aws.txt)
 |  hadoop-aws in the patch failed.  |
   | +1 :green_heart: |  javadoc  |   0m 50s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 50s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | -1 :x: |  spotbugs  |   0m 24s | 
[/patch-spotbugs-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-64

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-01-18 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17808469#comment-17808469
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1899812388

   > most of my comments are on the basic stuff, especially those test 
assertions and the need to have a single factored out assertion() method.
   
   Am converting the current test into an ITest, will update assertions 
according to the recommendations 
   
   > Now, can we have the storage class an attribute in S3AFileStatus? 
populated in listings and from HEAD requests? included in .toString()? that 
could be useful in future
   
   This is something that can be done, do we want this as part of this PR ? Or 
a separate one adding the storage class to S3AFileStatus ? 
   




> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Assignee: Bhavay Pahuja
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-01-18 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17808249#comment-17808249
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

steveloughran commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1457522341


##
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/TestS3AReadRestoredGlacierObjects.java:
##
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.list;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.AbstractS3AMockTest;
+import org.apache.hadoop.fs.s3a.S3ObjectStorageClassFilter;
+import org.apache.hadoop.fs.s3a.Listing;
+import org.apache.hadoop.fs.s3a.Listing.GlacierStatusAcceptor;
+import org.junit.Assert;
+import org.junit.Test;
+import software.amazon.awssdk.services.s3.model.ObjectStorageClass;
+import software.amazon.awssdk.services.s3.model.RestoreStatus;
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+public class TestS3AReadRestoredGlacierObjects extends AbstractS3AMockTest {
+
+  String s3Path = "s3://test_bucket/path";
+
+  @Test
+  public void testIgnoreGlacierObject() {
+S3ObjectStorageClassFilter restoredObjectsType = 
S3ObjectStorageClassFilter.SKIP_ALL_GLACIER;
+
+Listing.GlacierStatusAcceptor acceptor = new 
GlacierStatusAcceptor(restoredObjectsType);
+
+boolean result = acceptor.accept(new Path(s3Path),
+getS3ObjectWithStorageClassAndRestoreStatus(
+s3Path,
+RestoreStatus.builder().isRestoreInProgress(false).build(),
+ObjectStorageClass.GLACIER));
+
+Assert.assertFalse(result);
+
+result = acceptor.accept(new Path(s3Path),
+getS3ObjectWithStorageClassAndRestoreStatus(
+s3Path,
+RestoreStatus.builder().isRestoreInProgress(false).build(),
+ObjectStorageClass.DEEP_ARCHIVE));
+
+Assert.assertFalse(result);

Review Comment:
   same. you could actually have an assertAcceptance(object, outcome) method 
instead of all this duplication



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:
##
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import java.util.Set;
+import java.util.function.Function;
+import software.amazon.awssdk.services.s3.model.ObjectStorageClass;
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+
+/**
+ * S3ObjectStorageClassFilter will filter the S3 files based on the 
fs.s3a.glacier.read.restored.objects configuration set in S3AFileSystem

Review Comment:
   be good to use  on lines and {@code } around formatted code so that 
javadocs and IDEs render better



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java:
##
@@ -1486,6 +1486,17 @@ private Constants() {
*/
   public static final int DEFAULT_PREFETCH_MAX_BLOCKS_COUNT = 4;
 
+  /**
+   * Read Restored Glacier objects config.
+   * Value = {@value}
+   */
+  public static final String READ_RESTORED_GLACIER_OBJECTS = 
"fs.s3a.glacier.read.restored.objects";
+
+  /**
+   * Default value of Read Restored Glacier objects config.
+   */
+  public static final

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-01-17 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17808029#comment-17808029
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1456959404


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java:
##
@@ -1486,6 +1486,17 @@ private Constants() {
*/
   public static final int DEFAULT_PREFETCH_MAX_BLOCKS_COUNT = 4;
 
+  /**
+   * Read Restored Glacier objects config.
+   * Value = {@value}
+   */
+  public static final String READ_RESTORED_GLACIER_OBJECTS = 
"fs.s3a.glacier.read.restored.objects";
+
+  /**
+   * Default value of Read Restored Glacier objects config.
+   */
+  public static final S3ObjectStorageClassFilter 
DEFAULT_READ_RESTORED_GLACIER_OBJECTS = S3ObjectStorageClassFilter.READ_ALL;

Review Comment:
   Sure will update



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:
##
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import java.util.Set;
+import java.util.function.Function;
+import software.amazon.awssdk.services.s3.model.ObjectStorageClass;
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+
+/**
+ * S3ObjectStorageClassFilter will filter the S3 files based on the 
fs.s3a.glacier.read.restored.objects configuration set in S3AFileSystem
+ * The config can have 3 values:
+ * READ_ALL: This would conform to the current default behavior of not taking 
into account the storage classes retrieved from S3. This will be done to keep 
the current behavior for the customers and not changing the experience for them.

Review Comment:
   Sure will update





> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-01-17 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17808028#comment-17808028
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1456959208


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##
@@ -2466,8 +2474,8 @@ public RemoteIterator 
listFilesAndDirectoryMarkers(
   path,
   true,
   includeSelf
-  ? Listing.ACCEPT_ALL_BUT_S3N
-  : new Listing.AcceptAllButSelfAndS3nDirs(path),
+  ? Lists.newArrayList(new 
GlacierStatusAcceptor(s3ObjectStorageClassFilter), Listing.ACCEPT_ALL_BUT_S3N)

Review Comment:
   Sure will update





> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-01-17 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17808027#comment-17808027
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1456958842


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:
##
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import java.util.Set;
+import java.util.function.Function;
+import software.amazon.awssdk.services.s3.model.ObjectStorageClass;
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+
+/**
+ * S3ObjectStorageClassFilter will filter the S3 files based on the 
fs.s3a.glacier.read.restored.objects configuration set in S3AFileSystem
+ * The config can have 3 values:
+ * READ_ALL: This would conform to the current default behavior of not taking 
into account the storage classes retrieved from S3. This will be done to keep 
the current behavior for the customers and not changing the experience for them.
+ * SKIP_ALL_GLACIER: If this value is set then this will ignore any S3 Objects 
which are tagged with Glacier storage classes and retrieve the others.
+ * READ_RESTORED_GLACIER_OBJECTS: If this value is set then restored status of 
the Glacier object will be checked, if restored the objects would be read like 
normal S3 objects else they will be ignored as the objects would not have been 
retrieved from the S3 Glacier.
+ */
+public enum S3ObjectStorageClassFilter {
+  READ_ALL(o -> true),
+  SKIP_ALL_GLACIER(S3ObjectStorageClassFilter::isNotGlacierObject),
+  
READ_RESTORED_GLACIER_OBJECTS(S3ObjectStorageClassFilter::isCompletedRestoredObject);
+
+  private static final Set GLACIER_STORAGE_CLASSES = 
Sets.newHashSet(ObjectStorageClass.GLACIER, ObjectStorageClass.DEEP_ARCHIVE);

Review Comment:
   GLACIER_IR files, are instantly available so no failure is observed. S3A 
will be able to access the same. 



##
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml:
##
@@ -2191,6 +2191,18 @@
   
 
 
+

Review Comment:
   Sure will do





> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Priority: Minor
>  Labels: pull-request-available
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-01-17 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17807799#comment-17807799
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

ahmarsuhail commented on code in PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#discussion_r1455953276


##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:
##
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import java.util.Set;
+import java.util.function.Function;
+import software.amazon.awssdk.services.s3.model.ObjectStorageClass;
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+
+/**
+ * S3ObjectStorageClassFilter will filter the S3 files based on the 
fs.s3a.glacier.read.restored.objects configuration set in S3AFileSystem
+ * The config can have 3 values:
+ * READ_ALL: This would conform to the current default behavior of not taking 
into account the storage classes retrieved from S3. This will be done to keep 
the current behavior for the customers and not changing the experience for them.

Review Comment:
   Instead of saying current behaviour, can you specify what that current 
behaviour is. It errors right?



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java:
##
@@ -1486,6 +1486,17 @@ private Constants() {
*/
   public static final int DEFAULT_PREFETCH_MAX_BLOCKS_COUNT = 4;
 
+  /**
+   * Read Restored Glacier objects config.
+   * Value = {@value}
+   */
+  public static final String READ_RESTORED_GLACIER_OBJECTS = 
"fs.s3a.glacier.read.restored.objects";
+
+  /**
+   * Default value of Read Restored Glacier objects config.
+   */
+  public static final S3ObjectStorageClassFilter 
DEFAULT_READ_RESTORED_GLACIER_OBJECTS = S3ObjectStorageClassFilter.READ_ALL;

Review Comment:
   I'd update this to just READ_ALL, and then when you do conf.get(), pass in 
the default there ... conf.get(READ_RESTORED_GLACIER_OBJECTS, 
DEFAULT_READ_RESTORED_GLACIER_OBJECTS)



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##
@@ -2466,8 +2474,8 @@ public RemoteIterator 
listFilesAndDirectoryMarkers(
   path,
   true,
   includeSelf
-  ? Listing.ACCEPT_ALL_BUT_S3N
-  : new Listing.AcceptAllButSelfAndS3nDirs(path),
+  ? Lists.newArrayList(new 
GlacierStatusAcceptor(s3ObjectStorageClassFilter), Listing.ACCEPT_ALL_BUT_S3N)

Review Comment:
   don't think you need an acceptor for this, so just update if clause in the 
Listing



##
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectStorageClassFilter.java:
##
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import java.util.Set;
+import java.util.function.Function;
+import software.amazon.awssdk.services.s3.model.ObjectStorageClass;
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+
+/**
+ * S3ObjectStorageClassFilter will filter the S3 files based on the 
fs.s3a.glacier.read.restored.objects configuration set in S3AFileSystem
+ * The config can have 3 va

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-01-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17803443#comment-17803443
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1878267213

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 19s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  xmllint  |   0m  0s |  |  xmllint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  30m 41s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  19m 40s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   8m 13s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   7m 29s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   1m 59s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 24s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   3m 14s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 56s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m  7s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m 51s |  |  branch has no errors 
when building and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 19s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   0m 50s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   7m 54s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |   7m 54s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   7m 33s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   7m 33s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | -0 :warning: |  checkstyle  |   2m  0s | 
[/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/2/artifact/out/results-checkstyle-root.txt)
 |  root: The patch generated 29 new + 6 unchanged - 0 fixed = 35 total (was 6) 
 |
   | +1 :green_heart: |  mvnsite  |   1m 25s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 57s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   1m  1s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m 21s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  20m  9s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | +1 :green_heart: |  unit  |  16m 14s |  |  hadoop-common in the patch 
passed.  |
   | +1 :green_heart: |  unit  |   2m 16s |  |  hadoop-aws in the patch passed. 
 |
   | +1 :green_heart: |  asflicense  |   0m 36s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   | 163m 18s |  |  |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/2/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/6407 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient codespell detsecrets xmllint spotbugs checkstyle |
   | uname | Linux 03a16e3a2176 5.15.0-88-generic #98-Ubuntu SMP Mon Oct 2 
15:18:56 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 5d53b5f97e688b4f71bf84df15bee340f71fbefb |
   | Default Java | Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   | Multi-JDK versions | 
/usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 
/usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_392-8u392-ga-1~20.04-b08 |
   |  Test Results | 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/2

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-01-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17803112#comment-17803112
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

hadoop-yetus commented on PR #6407:
URL: https://github.com/apache/hadoop/pull/6407#issuecomment-1877019109

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 21s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  
|
   | +0 :ok: |  xmllint  |   0m  0s |  |  xmllint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  14m  6s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  19m 33s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   8m 18s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   7m 35s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   2m  2s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 24s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 59s |  |  trunk passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 58s |  |  trunk passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m  7s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m 37s |  |  branch has no errors 
when building and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   0m 50s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   7m 56s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |   7m 56s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   7m 28s |  |  the patch passed with JDK 
Private Build-1.8.0_392-8u392-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   7m 28s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | -0 :warning: |  checkstyle  |   1m 56s | 
[/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/1/artifact/out/results-checkstyle-root.txt)
 |  root: The patch generated 29 new + 6 unchanged - 0 fixed = 35 total (was 6) 
 |
   | +1 :green_heart: |  mvnsite  |   1m 19s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 57s |  |  the patch passed with JDK 
Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04  |
   | -1 :x: |  javadoc  |   0m 30s | 
[/patch-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/1/artifact/out/patch-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt)
 |  hadoop-aws in the patch failed with JDK Private 
Build-1.8.0_392-8u392-ga-1~20.04-b08.  |
   | +1 :green_heart: |  spotbugs  |   2m 24s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  19m 44s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | +1 :green_heart: |  unit  |  16m 29s |  |  hadoop-common in the patch 
passed.  |
   | +1 :green_heart: |  unit  |   2m 19s |  |  hadoop-aws in the patch passed. 
 |
   | -1 :x: |  asflicense  |   0m 36s | 
[/results-asflicense.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/1/artifact/out/results-asflicense.txt)
 |  The patch generated 1 ASF License warnings.  |
   |  |   | 144m 20s |  |  |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-6407/1/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/6407 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient codespell detsecrets xmllint spotbugs checkstyle |
   | uname | Linux 467f4ec5d796 5.15.0-88-generic #98-Ubuntu SMP Mon Oct 2 
15:18:56 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 550bf

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2024-01-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17802877#comment-17802877
 ] 

ASF GitHub Bot commented on HADOOP-14837:
-

bpahuja opened a new pull request, #6407:
URL: https://github.com/apache/hadoop/pull/6407

   ### Description of PR
   
   Currently S3A does not distinguish Glacier and Glacier Deep Archive files, 
as it doesn't examine the storage class or verify if an object is in the 
process of restoration from Glacier. Attempting to access an in-progress 
Glacier file via S3A results in an AmazonS3Exception, indicating the 
operation's invalidity for the object's storage class. 
   
   As part of this change, Users will be able to successfully read restored 
glacier objects from the s3 location of the table using S3A. It will ignore any 
Glacier archived files if they are in process of being restored asynchronously. 
There will be no change to the existing behavior and additional configuration 
will be needed to enable the above mentioned flow.
   
   The config which would control the behavior of the S3AFileSystem with 
respect to glacier storage classes will be 
`fs.s3a.glacier.read-restored-objects`
   
   The config can have 3 values:
   
   - `READ_ALL` : This would conform to the current default behavior of not 
taking into account the storage classes retrieved from S3. This will be done to 
keep the current behavior for the users and not changing the experience for 
them.
   
   - `SKIP_ALL_GLACIER`: If this value is set then this will ignore any S3 
Objects which are tagged with Glacier storage classes and retrieve the others.
   
   - `READ_RESTORED_GLACIER_OBJECTS`: If this value is set then restored status 
of the Glacier object will be checked, if restored the objects would be read 
like normal S3 objects else they will be ignored as the objects would not have 
been retrieved from the S3 Glacier. ( To check the restored status, newly 
introduced RestoredStatus will be used which is present in the S3 Object). This 
wasn't previously possible as ListObjects did not return any information about 
the restore status of an object, only it's storage class.
   
   
   A new `FileStatusAcceptor` is created which will use the `RestoreStatus` 
attribute from the `S3Object` and will filter out or include the glacier 
objects from the list as defined by the config. `FileStatusAcceptor` is an 
interface with 3 overloaded predicates, which filter the files based on the 
conditions defined in the said predicates. A new attribute `RestoreStatus` of 
will be used from the response of  `ListObjects` . This field will indicate 
whether the object is unrestored, restoring, or restored, and also when the 
expiration of that restore is.
   
   ### How was this patch tested?
   
    Integration Tests (hadoop-aws)
   
   All the Integration tests are passing, the tests were run in accordance with 
https://hadoop.apache.org/docs/current2/hadoop-aws/tools/hadoop-aws/testing.html.
 The tests were executed in the region `us-east-1`.
   
   There were 2 failures observed which seem intermittent and unrelated to the 
change introduced in this CR. As the default behavior of S3AFileSystem was not 
changed. 
   
   Failures observed : 
   ```
   ITestS3ACommitterFactory.testEverything
   ITestS3AConfiguration.testRequestTimeout
   ```
   
    Manual Testing 
   
   Manual testing of the change was done with Spark v3.5
   
   A Parquet table was created using the following in Spark-SQL. 
   
   ```
   CREATE DATABASE IF NOT EXISTS glacier_test  location 
"s3a:///data/glacier_test";
   
   USE glacier_test;
   
   CREATE TABLE IF NOT EXISTS parquet_glacier_test (id int, data string) using 
parquet location "s3a:///data/glacier_test/parquet_glacier_test";
   
   INSERT INTO parquet_glacier_test  VALUES (1, 'a'), (2, 'b'), (3, 'c');
   
   INSERT INTO parquet_glacier_test  VALUES (4, 'a'), (5, 'b'), (6, 'c');
   
   INSERT INTO parquet_glacier_test  VALUES (7, 'a'), (8, 'b'), (9, 'c');
   ```
   Was able to successfully retrieve the data using the following. 
   
   ```
   SELECT * FROM parquet_glacier_test;
   
   +---++
   | id|data|
   +---++
   |  7|   a|
   |  8|   b|
   |  9|   c|
   |  4|   a|
   |  5|   b|
   |  6|   c|
   |  1|   a|
   |  2|   b|
   |  3|   c|
   +---++
   
   ```
   
   The storage class of the file 
`s3:///data/glacier_test/parquet_glacier_test/part-0-f9cb400e-35b2-41f7-9c39-8e34cd830fed-c000.snappy.parquet`
 was changed to `Glacier Flexible Retrieval (formerly Glacier)`  from 
`Standard`. 
   
   When trying to retrieve the data again form the same table, the following 
exception was observed. 
   
   ```
   software.amazon.awssdk.services.s3.model.InvalidObjectStateException: The 
operation is not valid for the object's storage class (Service: S3, Status 
Code: 403, Request ID: X05JDR633AAK4TBQ, Extended Request ID: 
uOxWdN4giUAuB9

[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2022-04-05 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17517548#comment-17517548
 ] 

Steve Loughran commented on HADOOP-14837:
-

implementing getBlocklocations with a head request would kill perf of 
liststatus too, which is the fast recursive listing option in the api, so no, 
not going to work.

> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Priority: Minor
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2022-04-04 Thread Daniel Carl Jones (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17516689#comment-17516689
 ] 

Daniel Carl Jones commented on HADOOP-14837:


FYI, Glacier recently introduced a new 'instant' storage class offering 
millisecond retrieval: 
[https://aws.amazon.com/about-aws/whats-new/2021/11/amazon-s3-glacier-instant-retrieval-storage-class/]

 

We may instead want some way to indicate if a file is available immediately or 
needs requesting, and this might influence our choice of design.

> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Priority: Minor
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2022-03-31 Thread Ahmar Suhail (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17515405#comment-17515405
 ] 

Ahmar Suhail commented on HADOOP-14837:
---

[~ste...@apache.org] Object summaries include the storage class, which means we 
can filter without any additional HEAD calls. 

For getBlockLocations(), I was looking at how it's used in Spark, and found 
that it's called[ 
here|[https://github.com/apache/spark/blob/0494dc90af48ce7da0625485a4dc6917a244d580/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala#L307].|https://github.com/apache/spark/blob/0494dc90af48ce7da0625485a4dc6917a244d580/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala#L307]]

If we implement getBlockLocations() in S3FS to return storage type, we would 
have to do an HEAD call which would slow down the above usage, not sure if 
that's something we should do? 

If we do want to implement getBlockLocations(), we could have a configuration 
option like `fs.s3a.get.file.locations` which when enabled would make the head 
call, otherwise just return the default location. 

> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Priority: Minor
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2022-03-22 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17510508#comment-17510508
 ] 

Steve Loughran commented on HADOOP-14837:
-

good questions, -I have no idea what the right answers are

bq. For reporting better, do we want to add in a new statistic, something like 
`objects_in_glacier` which will have the count of objects currently in glacier?

why not?

bq. In listings, we can add in a new option to filter out glacier files by 
doing something like `!summary.getStorageClass().equals("GLACIER")` in the 
acceptor here? After we do this and call `getContentSummary()` it won't return 
glacier files in the fileCount. 

I'm not worried about that. is the storage type returned in the list call. 
allowing it to be filtered there? i wouldn't want to do any HEAD requests here

bq. getBlockLocations()

there's special handling in spark for that location, which says "run your work 
anywnere". we doin't want to break that.

I think the best tactic here is to work out what people I want to do here and 
provide the bare minimum. Looking at some of the JIRAs there's no consensus as 
to what people want. Do they want glaciated files to be skipped in queries? or 
for recovery to be triggered (somehow). Returning the storage type ARCHIVE 
would be enough for anyone who wants to identify these files (distcp?) and at 
least then know there's a cost in accessing them. 

> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Priority: Minor
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2022-02-22 Thread Ahmar Suhail (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17496047#comment-17496047
 ] 

Ahmar Suhail commented on HADOOP-14837:
---

[~ste...@apache.org] I've been looking at this and had a few questions: 
 * For reporting better, do we want to add in a new statistic, something like 
`objects_in_glacier` which will have the count of objects currently in glacier?
 * In listings, we can add in a new option to filter out glacier files by doing 
something like `!summary.getStorageClass().equals("GLACIER")` in the acceptor 
[here|https://github.com/apache/hadoop/blob/365375412fe5eea82549630ee8c5598502b95caf/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java#L770]?
 After we do this and call `getContentSummary()` it won't return glacier files 
in the fileCount. 
 * To return StorageType.Archive for a file, I was looking at 
getBlockLocations, it'll currently return something like `BlockLocation( \{ 
"localhost:9866" }, \{ "localhost" }, 0, file.getLen())` , so not sure how we 
want it to behave when implemented in S3AFS? Will it be something like 
`BlockLocation( \{ filepath }, \{ StorageType.Archive.toString() }, 0, 
file.getLen())` ?
 * Do we want implement retrieval in open()? If yes, will the behaviour be:
 ** If fs.s3a.open.glacier.retrieve is enabled, check if file is in glacier, if 
yes, initiate restore
 ** If restore has not complete and .read() is called, throw "cannot read yet 
-retrieval requested"
 ** If restore has not been initiated (can happen when 
fs.s3a.open.glacier.retrieve is false) and .read() is called throw "cannot read 
data in glacier" 

> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Priority: Minor
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2019-10-02 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16942667#comment-16942667
 ] 

Steve Loughran commented on HADOOP-14837:
-

we might also want a way to request that glacier files should be retrieved.

I could sneak this into our existing open() API with

-an S3A specific option to say "retrieve glaciated data"

explicit exceptions
* "cannot read data in glacier"'
* "cannot read yet -retrieval requested" -what you get on an open with 
fs.s3a.open.glacier.retrieve



> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Priority: Minor
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2019-02-14 Thread Steve Loughran (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16768461#comment-16768461
 ] 

Steve Loughran commented on HADOOP-14837:
-

Update: use getBlockLocations to return this value; S3AFileSystem could be 
extended to add it maybe

> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Priority: Minor
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14837) Handle S3A "glacier" data

2017-10-12 Thread Steve Loughran (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-14837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16201882#comment-16201882
 ] 

Steve Loughran commented on HADOOP-14837:
-

The way to make this visible will be to support 
{{FileSystem.getContentSummary()}} and return {{StorageType.ARCHIVE}} for data 
in Glacier. Then callers can opt to filter on glaciated data, with an API which 
has been around for a long time.

> Handle S3A "glacier" data
> -
>
> Key: HADOOP-14837
> URL: https://issues.apache.org/jira/browse/HADOOP-14837
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-beta1
>Reporter: Steve Loughran
>Priority: Minor
>
> SPARK-21797 covers how if you have AWS S3 set to copy some files to glacier, 
> they appear in the listing but GETs fail, and so does everything else
> We should think about how best to handle this.
> # report better
> # if listings can identify files which are glaciated then maybe we could have 
> an option to filter them out
> # test & see what happens



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org