[ 
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?
   </description>
 </property>
+```
+## <a name="glacier"></a> 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;
+
+
+/**
+ * <pre>
+ * {@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.
+ * </pre>
+ */
+public enum S3ObjectStorageClassFilter {
+  READ_ALL(o -> true),
+  SKIP_ALL_GLACIER(S3ObjectStorageClassFilter::isNotGlacierObject),
+  
READ_RESTORED_GLACIER_OBJECTS(S3ObjectStorageClassFilter::isCompletedRestoredObject);
+
+  private static final Set<ObjectStorageClass> GLACIER_STORAGE_CLASSES = 
Sets.newHashSet(
+      ObjectStorageClass.GLACIER, ObjectStorageClass.DEEP_ARCHIVE);
+
+  private final Function<S3Object, Boolean> filter;
+
+  S3ObjectStorageClassFilter(Function<S3Object, Boolean> 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 not an object with a storage class of 
glacier/deep_archive
+   */
+  private static boolean isNotGlacierObject(S3Object object) {
+    return !GLACIER_STORAGE_CLASSES.contains(object.storageClass());
+  }
+
+  /**
+   * Checks if the s3 object is an object with a storage class of 
glacier/deep_archive.
+   * @param object s3 object
+   * @return if the s3 object is an object with a storage class of 
glacier/deep_archive
+   */
+  private static boolean isGlacierObject(S3Object object) {
+    return GLACIER_STORAGE_CLASSES.contains(object.storageClass());
+  }
+
+  /**
+   * Checks if the s3 object is completely restored.
+   * @param object s3 object
+   * @return if the s3 object is completely restored
+   */
+  private static boolean isCompletedRestoredObject(S3Object object) {
+    if(isGlacierObject(object)) {
+      return object.restoreStatus() != null && 
!object.restoreStatus().isRestoreInProgress();
+    }
+    return true;
+  }
+
+  /**
+   * Returns the filter function set as part of the enum definition

Review Comment:
   needs a trailing .



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java:
##########
@@ -117,6 +117,8 @@ public class StoreContext implements 
ActiveThreadSpanSource<AuditSpan> {
   /** Is client side encryption enabled? */
   private final boolean isCSEEnabled;
 
+  private final S3ObjectStorageClassFilter s3ObjectStorageClassFilter;

Review Comment:
   nit: javadocs



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+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;
+
+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.api.S3ObjectStorageClassFilter;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+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;
+
+
+/**
+ * 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<Object[]> data(){
+    return Arrays.asList(new Object[][] {
+        {STORAGE_CLASS_GLACIER}, {STORAGE_CLASS_DEEP_ARCHIVE},
+    });
+  }
+
+  private static final int MAX_RETRIES = 100;
+  private static final int RETRY_DELAY_MS = 5000;
+  private final String glacierClass;
+
+  public ITestS3AReadRestoredGlacierObjects(String glacierClass) {
+    this.glacierClass = glacierClass;
+  }
+
+  private FileSystem createFiles(String s3ObjectStorageClassFilter) throws 
Throwable {
+    FileSystem fs = createFileSystem(s3ObjectStorageClassFilter);
+    Path path = new Path(methodPath(), "glaciated");
+    ContractTestUtils.touch(fs, path);
+    return fs;
+  }
+
+  private FileSystem createFileSystem(String s3ObjectStorageClassFilter) 
throws Throwable {
+    Configuration conf = createConfiguration();
+    conf.set(READ_RESTORED_GLACIER_OBJECTS, s3ObjectStorageClassFilter);

Review Comment:
   insert line
   ```
       skipIfStorageClassTestsDisabled(conf);
   ```
   
   (and matching import
   ```
   import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfStorageClassTestsDisabled;
   
   ```
   
   See ITestS3AStorageClass for an example.



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/list/ITestS3AReadRestoredGlacierObjects.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.apache.hadoop.fs.s3a.S3AFileSystem;

Review Comment:
   move to the apache group





> 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: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to