[
https://issues.apache.org/jira/browse/HADOOP-16202?focusedWorklogId=755841&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-755841
]
ASF GitHub Bot logged work on HADOOP-16202:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 12/Apr/22 15:38
Start Date: 12/Apr/22 15:38
Worklog Time Spent: 10m
Work Description: steveloughran commented on code in PR #2584:
URL: https://github.com/apache/hadoop/pull/2584#discussion_r848584564
##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileSupport.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * 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.impl;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.assertj.core.api.Assertions;
+import org.assertj.core.api.ObjectAssert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.impl.OpenFileParameters;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3AInputPolicy;
+import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
+import org.apache.hadoop.test.HadoopTestBase;
+
+import static java.util.Collections.singleton;
+import static
org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static
org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE;
+import static
org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
+import static
org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static
org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE;
+import static
org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT;
+import static
org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_RANDOM;
+import static
org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
+import static
org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END;
+import static
org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START;
+import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ASYNC_DRAIN_THRESHOLD;
+import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE;
+import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Unit tests for {@link OpenFileSupport} and the associated
+ * seek policy lookup in {@link S3AInputPolicy}.
+ */
+public class TestOpenFileSupport extends HadoopTestBase {
+
+ private static final ChangeDetectionPolicy CHANGE_POLICY =
+ ChangeDetectionPolicy.createPolicy(
+ ChangeDetectionPolicy.Mode.Server,
+ ChangeDetectionPolicy.Source.None,
+ false);
+
+ private static final long READ_AHEAD_RANGE = 16;
+
+ private static final String USERNAME = "hadoop";
+
+ public static final S3AInputPolicy INPUT_POLICY = S3AInputPolicy.Sequential;
+
+ public static final String TESTFILE = "s3a://bucket/name";
+
+ private static final Path TESTPATH = new Path(TESTFILE);
+
+ /**
+ * Create a OpenFileSupport instance.
+ */
+ private static final OpenFileSupport PREPARE =
+ new OpenFileSupport(
+ CHANGE_POLICY,
+ READ_AHEAD_RANGE,
+ USERNAME,
+ IO_FILE_BUFFER_SIZE_DEFAULT,
+ DEFAULT_ASYNC_DRAIN_THRESHOLD,
+ INPUT_POLICY);
+
+ @Test
+ public void testSimpleFile() throws Throwable {
+ ObjectAssert<OpenFileSupport.OpenFileInformation>
+ asst = assertFileInfo(
+ PREPARE.openSimpleFile(1024));
+
+ asst.extracting(f -> f.getChangePolicy())
+ .isEqualTo(CHANGE_POLICY);
+ asst.extracting(f -> f.getInputPolicy())
+ .isEqualTo(INPUT_POLICY);
+ asst.extracting(f -> f.getReadAheadRange())
+ .isEqualTo(READ_AHEAD_RANGE);
+ }
+
+ /**
+ * Initiate an assert from an open file information instance.
+ * @param fi file info
+ * @return an assert stream.
+ */
+ private ObjectAssert<OpenFileSupport.OpenFileInformation> assertFileInfo(
+ final OpenFileSupport.OpenFileInformation fi) {
+ return Assertions.assertThat(fi)
+ .describedAs("File Information %s", fi);
+ }
+
+ /**
+ * Create an assertion about the openFile information from a configuration
+ * with the given key/value option.
+ * @param key key to set.
+ * @param option option value.
+ * @return the constructed OpenFileInformation.
+ */
+ public ObjectAssert<OpenFileSupport.OpenFileInformation> assertOpenFile(
+ final String key,
+ final String option) throws IOException {
+ return assertFileInfo(prepareToOpenFile(params(key, option)));
+ }
+
+ @Test
+ public void testUnknownMandatoryOption() throws Throwable {
+
+ String key = "unknown";
+ intercept(IllegalArgumentException.class, key, () ->
+ prepareToOpenFile(params(key, "undefined")));
+ }
+
+ @Test
+ public void testSeekRandomIOPolicy() throws Throwable {
+
+ // ask for random IO
+ String option = FS_OPTION_OPENFILE_READ_POLICY_RANDOM;
+
+ // is picked up
+ assertOpenFile(INPUT_FADVISE, option)
+ .extracting(f -> f.getInputPolicy())
+ .isEqualTo(S3AInputPolicy.Random);
+ // and as neither status nor length was set: no file status
+ assertOpenFile(INPUT_FADVISE, option)
+ .extracting(f -> f.getStatus())
+ .isNull();
+ }
+
+ /**
+ * There's a standard policy name. 'adaptive',
+ * meaning 'whatever this stream does to adapt to the client's use'.
+ * On the S3A connector that is mapped to {@link S3AInputPolicy#Normal}.
+ */
+ @Test
+ public void testSeekPolicyAdaptive() throws Throwable {
+
+ // when caller asks for adaptive, they get "normal"
+ assertOpenFile(FS_OPTION_OPENFILE_READ_POLICY,
+ FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE)
+ .extracting(f -> f.getInputPolicy())
+ .isEqualTo(S3AInputPolicy.Normal);
+ }
+
+ /**
+ * Verify that an unknown seek policy falls back to
+ * {@link S3AInputPolicy#Normal}. Delete
Review Comment:
or macos speech recognition joining in coding
Issue Time Tracking
-------------------
Worklog Id: (was: 755841)
Time Spent: 18h 20m (was: 18h 10m)
> Enhance openFile() for better read performance against object stores
> ---------------------------------------------------------------------
>
> Key: HADOOP-16202
> URL: https://issues.apache.org/jira/browse/HADOOP-16202
> Project: Hadoop Common
> Issue Type: Bug
> Components: fs, fs/s3, tools/distcp
> Affects Versions: 3.3.0
> Reporter: Steve Loughran
> Assignee: Steve Loughran
> Priority: Major
> Labels: pull-request-available
> Time Spent: 18h 20m
> Remaining Estimate: 0h
>
> The {{openFile()}} builder API lets us add new options when reading a file
> Add an option {{"fs.s3a.open.option.length"}} which takes a long and allows
> the length of the file to be declared. If set, *no check for the existence of
> the file is issued when opening the file*
> Also: withFileStatus() to take any FileStatus implementation, rather than
> only S3AFileStatus -and not check that the path matches the path being
> opened. Needed to support viewFS-style wrapping and mounting.
> and Adopt where appropriate to stop clusters with S3A reads switched to
> random IO from killing download/localization
> * fs shell copyToLocal
> * distcp
> * IOUtils.copy
--
This message was sent by Atlassian Jira
(v8.20.1#820001)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]