[
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 <p> 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 S3ObjectStorageClassFilter
DEFAULT_READ_RESTORED_GLACIER_OBJECTS = S3ObjectStorageClassFilter.READ_ALL;
Review Comment:
+make sure that the option is supported in lower case as well as upper case,
and trimmed.
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##########
@@ -52,6 +52,8 @@
import java.util.concurrent.atomic.AtomicBoolean;
import javax.annotation.Nullable;
+import org.apache.hadoop.fs.s3a.Listing.AcceptAllButSelfAndS3nDirs;
Review Comment:
nit, import ordering
##########
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);
Review Comment:
i require a meaningful error message on all failures. ideally using assertj
whose describedAs() class can do string formatting.
```
o = getS3ObjectWithStorageClassAndRestoreStatus(...)
Assertions.assertThat(acceptor.accept(...))
.describedAs("accept %s", o)
.isFalse()
```
just imagine you've seen a test failure: what information would you want in
the assertion message to begin debugging it?
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java:
##########
@@ -831,6 +860,56 @@ public boolean accept(FileStatus status) {
}
}
+
+ /**
+ * Accept all entries except unrestored Glacier objects if the config
fs.s3a.glacier.read.restored.objects,
+ * is set to READ_RESTORED_GLACIER_OBJECTS
+ * Accept all entries except Glacier objects if the config
fs.s3a.glacier.read.restored.objects,
+ * is set to SKIP_ALL_GLACIER
+ */
+ public static class GlacierStatusAcceptor implements FileStatusAcceptor {
Review Comment:
does this storage class come in on a v2 LIST request? guess it must...
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##########
@@ -5213,7 +5223,7 @@ private RemoteIterator<S3ALocatedFileStatus>
innerListFiles(
RemoteIterator<S3ALocatedFileStatus> listFilesAssumingDir =
listing.getListFilesAssumingDir(path,
recursive,
- acceptor,
+ acceptors,
Review Comment:
nit: restore consistent indentation
##########
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.util.Lists;
Review Comment:
should go with the rest of the hadoop imports. it's in the wrong place in a
lot of files as the move off guava lists was a search and replace without
reordering
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##########
@@ -576,6 +580,10 @@ public void initialize(URI name, Configuration
originalConf)
s3aInternals = createS3AInternals();
+ s3ObjectStorageClassFilter =
Optional.ofNullable(conf.get(READ_RESTORED_GLACIER_OBJECTS))
Review Comment:
use getTrimmed(key, ""), toUpper() and then do the matching. fail
meaningfully if the value isn't recognised.
> 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: [email protected]
For additional commands, e-mail: [email protected]