[
https://issues.apache.org/jira/browse/HADOOP-17981?focusedWorklogId=673471&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-673471
]
ASF GitHub Bot logged work on HADOOP-17981:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 02/Nov/21 18:00
Start Date: 02/Nov/21 18:00
Worklog Time Spent: 10m
Work Description: mukund-thakur commented on a change in pull request
#3597:
URL: https://github.com/apache/hadoop/pull/3597#discussion_r740782689
##########
File path:
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/ResilientCommitByRename.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.impl;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import javax.annotation.Nullable;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+
+/**
+ * This is something internal to make our rename-based job committers
+ * more resilient to failures.
+ * If you are in the hive team: do not use this as it lacks
+ * spec, tests, stability, etc. if we find you using it we will change
+ * the signature just to stop your code compiling.
+ * View this as a proof of concept of the functionality we'd want from a
+ * "modern" rename call, but not the API (which would be builder based,
+ * return a future, etc).
+ */
[email protected]({"Filesystems",
"hadoop-mapreduce-client-core"})
[email protected]
+public interface ResilientCommitByRename {
+
+ /**
+ * Path capability.
+ * FS Instances which support the operation MUST return
+ * true and implement the method; FileSystem instances which do not
+ * MUST return false.
+ * There's a risk wrapper filesystems may pass the probe
+ * through.
+ * Clients MUST check for both the interface and this
+ * cqpability.
+ */
+ String RESILIENT_COMMIT_BY_RENAME_PATH_CAPABILITY =
+ "org.apache.hadoop.fs.impl.resilient.commit.by.rename";
+
+ /**
+ * Rename source file to dest path *Exactly*; no subdirectory games here.
+ * if the op does not raise an exception,then
+ * the data at dest is the data which was at source.
+ *
+ * Requirements
+ *
+ * <pre>
+ * exists(FS, source) else raise FileNotFoundException
+ * source != dest else raise PathIOException
+ * not exists(FS, dest)
+ * isDir(FS, dest.getParent)
+ * </pre>
+ * <ol>
+ * <li>supported in this instance else raise PathIOException</li>
+ * <li>source != dest else raise PathIOException</li>
+ * <li>source must exist else raise FileNotFoundException</li>
+ * <li>source must exist and be a file</li>
+ * <li>dest must not exist; </li>
+ * <li>dest.getParent() must be a dir</li>
+ * <li>if sourceEtag is non-empty, it MAY be used to qualify/validate the
rename.</li>
+ * </ol>
+ *
+ * The outcome of the operation is undefined if source is not a file, dest
exists,
+ * dest.getParent() doesn't exist/is a file.
+ * That is: implementations SHOULD assume that the code calling this method
has
+ * set up the destination directory tree and is only invoking this call on a
file.
+ * Accordingly: <i>implementations MAY skip validation checks</i>
+ *
+ * If sourceStatus is not null, its contents MAY be used to qualify the
rename.
+ * <ol>
+ * <li>Values extracted from sourceStatus SHALL take priority over
+ * sourceEtag/sourceLastModified parameter.</li>
+ * <li>sourceStatus.getPath().getName() MUST equal source.getName()</li>
+ * <li>If store has a subclass of FileStatus and it is sourceStatus is of
this type,
+ * custom information MAY be used to qualify/validate the request.
+ * This MAY include etag or S3 version ID extraction,</li>
+ * </ol>
+ *
+ * Filesystems MAY support this call on an instance-by-instance basis,
depending on
+ * the nature of the remote store.
+ * If not available the implementation MUST {@code
ResilientCommitByRenameUnsupported}.
+ * Callers SHOULD use a check of
+ * {@code hasPathCapability(source,
RESILIENT_COMMIT_BY_RENAME_PATH_CAPABILITY}
+ * before trying to use this call.
+ *
+ * PostConditions on a successful operation:
+ * <pre>
+ * FS' where:
+ * not exists(FS', source)
+ * and exists(FS', dest)
+ * and data(FS', dest) == data (FS, source)
+ * </pre>
+ * This is exactly the same outcome as `FileSystem.rename()` when the same
preconditions
+ * are met. This API call simply restricts the operation to file rename with
strict
+ * conditions, (no need to be 'clever' about dest path calculation) and the
ability
+ * to pass in etags, modtimes and file status values.
+ *
+ * @param source path to source file
+ * @param dest destination of rename.
+ * @param sourceEtag etag of source file. may be null or empty
+ * @param sourceStatus nullable FileStatus of source.
+ * @param options rename flags
+ * @throws FileNotFoundException source file not found
+ * @throws ResilientCommitByRenameUnsupported not available on this store.
+ * @throws PathIOException failure, including source and dest being the same
path
+ * @throws IOException any other exception
+ */
+ default CommitByRenameOutcome commitSingleFileByRename(
+ Path source,
+ Path dest,
+ @Nullable String sourceEtag,
+ @Nullable FileStatus sourceStatus,
+ ResilientCommitByRename.CommitFlqgs... options)
+ throws FileNotFoundException,
+ ResilientCommitByRenameUnsupported,
+ PathIOException,
+ IOException {
+ throw new ResilientCommitByRenameUnsupported(source.toString());
+ }
+
+ /**
+ * The outcome. This is always a success, but it
+ * may include some information about what happened.
+ */
+ class CommitByRenameOutcome {
+
+ /* Throttling encountered and recovered from. */
+ private boolean throttlingEncountered;
+
+ /* The new commit operation has been rejected; falling back. */
+ private boolean commitRejected;
+
+ /* Classic rename was used. */
+ private boolean classicRenameUsed;
+
+ public CommitByRenameOutcome() {
+ }
+
+ public CommitByRenameOutcome(
+ final boolean throttlingEncountered,
+ final boolean commitRejected,
+ final boolean classicRenameUsed) {
+ this.throttlingEncountered = throttlingEncountered;
+ this.commitRejected = commitRejected;
+ this.classicRenameUsed = classicRenameUsed;
+ }
+
+ public boolean isThrottlingEncountered() {
+ return throttlingEncountered;
+ }
+
+ public void setThrottlingEncountered(final boolean throttlingEncountered) {
+ this.throttlingEncountered = throttlingEncountered;
+ }
+
+ public boolean isCommitRejected() {
+ return commitRejected;
+ }
+
+ public void setCommitRejected(final boolean commitRejected) {
+ this.commitRejected = commitRejected;
+ }
+
+ public boolean isClassicRenameUsed() {
+ return classicRenameUsed;
+ }
+
+ public void setClassicRenameUsed(final boolean classicRenameUsed) {
+ this.classicRenameUsed = classicRenameUsed;
+ }
+ }
+
+ final class ResilientCommitByRenameUnsupported extends PathIOException {
+ public ResilientCommitByRenameUnsupported(final String path) {
+ super(path, "ResilientCommit operations not supported");
+ }
+ }
+
+ /**
+ * Enum of options.
+ */
+ enum CommitFlqgs {
Review comment:
nit : typo.. Flags.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 673471)
Time Spent: 2h 50m (was: 2h 40m)
> Support etag-assisted renames in FileOutputCommitter
> ----------------------------------------------------
>
> Key: HADOOP-17981
> URL: https://issues.apache.org/jira/browse/HADOOP-17981
> Project: Hadoop Common
> Issue Type: New Feature
> Components: fs, fs/azure
> Affects Versions: 3.4.0
> Reporter: Steve Loughran
> Assignee: Steve Loughran
> Priority: Major
> Labels: pull-request-available
> Time Spent: 2h 50m
> Remaining Estimate: 0h
>
> To deal with some throttling/retry issues in object stores,
> pass the FileStatus entries retrieved during listing
> into a private interface ResilientCommitByRename which filesystems
> may implement to use extra attributes in the listing (etag, version)
> to constrain and validate the operation.
> Although targeting azure, GCS and others could use. no point in S3A as they
> shouldn't use this committer.
> # And we are not going to do any changes to FileSystem as there are explicit
> guarantees of public use and stability.
> I am not going to make that change as the hive thing that will suddenly start
> expecting it to work forever.
> # I'm not planning to merge this in, as the manifest committer is going to
> include this and more (MAPREDUCE-7341)
> However, I do need to get this in on a branch, so am doing this work on trunk
> for dev & test and for others to review
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]