[ 
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<Object[]> 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);
+    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 {
+    Assume.assumeTrue(type == Type.GLACIER_AND_DEEP_ARCHIVE);
+    try (FileSystem fs = 
createFiles(S3ObjectStorageClassFilter.READ_RESTORED_GLACIER_OBJECTS.name())) {
+      Assertions.assertThat(
+              fs.listStatus(
+                  methodPath()))
+          .describedAs("FileStatus List of %s", methodPath()).isEmpty();
+    }
+  }
+
+  @Test
+  public void testRestoredGlacierObject() throws Throwable {
+    Assume.assumeTrue(type == Type.GLACIER); // Skipping this test for Deep 
Archive as expedited retrieval is not supported
+    try (FileSystem fs = 
createFiles(S3ObjectStorageClassFilter.READ_RESTORED_GLACIER_OBJECTS.name())) {
+      restoreGlacierObject(methodPath().toUri().getHost(), 
getFilePrefixForListObjects() + "file1", 2);
+      Assertions.assertThat(
+              fs.listStatus(
+                  methodPath()))
+          .describedAs("FileStatus List of %s", methodPath()).isNotEmpty();
+    }
+  }
+
+  @Test
+  public void testDefault() throws Throwable {
+    Assume.assumeTrue(type == Type.GLACIER_AND_DEEP_ARCHIVE);
+    try (FileSystem fs = 
createFiles(S3ObjectStorageClassFilter.READ_ALL.name())) {
+      Assertions.assertThat(
+              fs.listStatus(methodPath()))
+          .describedAs("FileStatus List of %s", methodPath()).isNotEmpty();
+    }
+  }
+
+
+  private void restoreGlacierObject(String bucketName, String 
glacierObjectKey, int expirationDays) {
+
+    S3Client s3Client = 
getFileSystem().getS3AInternals().getAmazonS3Client("test");
+
+    // Create a restore object request
+    RestoreObjectRequest requestRestore = RestoreObjectRequest.builder()
+        .bucket(bucketName)
+        .key(glacierObjectKey)
+        .restoreRequest(
+            RestoreRequest.builder().glacierJobParameters(
+                GlacierJobParameters.builder()
+                    .tier(Tier.EXPEDITED)
+                    .build()).days(expirationDays)
+                .build())
+        .build();
+
+    s3Client.restoreObject(requestRestore);
+
+    // fetch the glacier object
+    S3ListRequest s3ListRequest = getFileSystem().createListObjectsRequest(
+        getFilePrefixForListObjects(), "/");
+    S3Object s3GlacierObject =  getS3GlacierObject(s3Client, s3ListRequest);
+
+    while ((s3GlacierObject != null && 
s3GlacierObject.restoreStatus().isRestoreInProgress()) && retryCount < 
MAX_RETRIES) {
+      // Wait for few seconds before checking again
+      try {
+        Thread.sleep(RETRY_DELAY_MS);
+        retryCount++;
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+      s3GlacierObject =  getS3GlacierObject(s3Client, s3ListRequest);;
+    }
+
+    if (retryCount >= MAX_RETRIES){
+      throw new RuntimeException("The restore process exceeded the maximum 
allowed time.");
+    }
+  }
+
+
+  private String getFilePrefixForListObjects() {
+    return getContract().getTestPath().getName() + "/" + 
methodName.getMethodName() + "/";

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

Reply via email to