Skip to content

HDFS-15484. Add new method batchRename for DistributedFileSystem and W… #2235

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 5 commits into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -139,4 +139,11 @@ private CommonPathCapabilities() {
public static final String FS_MULTIPART_UPLOADER =
"fs.capability.multipart.uploader";

/**
* Does the store support batch rename?
* Value: {@value}.
*/
public static final String FS_BATCH_RENAME =
"fs.capability.batch.rename";

}
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl;
import org.apache.hadoop.fs.impl.FutureDataInputStreamBuilderImpl;
import org.apache.hadoop.fs.impl.FutureRenameBuilderImpl;
import org.apache.hadoop.fs.impl.OpenFileParameters;
import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus;
Expand Down Expand Up @@ -1609,6 +1610,18 @@ protected void rename(final Path src, final Path dst,
}
}

public IORenameStatistic batchRename(final List<String> srcs, final List<String> dsts,
final Rename... options) throws IOException {
if (srcs.size() != dsts.size()) {
throw new InvalidPathException("mismatch batch path src: " +
String.join(",", srcs) + " dst: " + String.join(",", dsts));
}
for(int i = 0; i < srcs.size(); i++) {
rename(new Path(srcs.get(i)), new Path(dsts.get(i)), options);
}
return new IORenameStatistic();
}

/**
* Truncate the file in the indicated path to the indicated size.
* <ul>
Expand Down Expand Up @@ -4670,4 +4683,63 @@ public MultipartUploaderBuilder createMultipartUploader(Path basePath)
methodNotSupported();
return null;
}

/**
* Builder returned for {@code #openFile(Path)}
* and {@code #openFile(PathHandle)}.
*/
private static class FSRenameBuilder
extends FutureRenameBuilderImpl
implements FutureRenameBuilder {

/**
* Path Constructor. *
* @param srcs path to open.
*/
protected FSRenameBuilder(
@Nonnull final FileSystem fs,
@Nonnull final List<String> srcs,
@Nonnull final List<String> dsts,
Rename... options) {
super(fs, srcs, dsts, options);
}

/**
* Perform the open operation.
* Returns a future which, when get() or a chained completion
* operation is invoked, will supply the input stream of the file
* referenced by the path/path handle.
* @return a future to the input stream.
* @throws IOException early failure to open
* @throws UnsupportedOperationException if the specific operation
* is not supported.
* @throws IllegalArgumentException if the parameters are not valid.
*/
@Override
public CompletableFuture<IORenameStatistic> build() throws IOException {
return LambdaUtils.eval(new CompletableFuture<>(),
() ->getFS().batchRename(getSrcs(), getDsts(), getOptions()));
}
}

/**
* rename one or batch file through a builder API.
* Ultimately calls {@link #rename(Path, Path)} unless a subclass
* executes the open command differently.
*
* The semantics of this call are therefore the same as that of
* {@link #rename(Path, int)} with one special point: it is in
* {@code FSDataInputStreamBuilder.build()} in which the open operation
* takes place -it is there where all preconditions to the operation
* are checked.
* @param path file path
* @return a FSDataInputStreamBuilder object to build the input stream
* @throws IOException if some early checks cause IO failures.
* @throws UnsupportedOperationException if support is checked early.
*/
@InterfaceStability.Unstable
public FutureRenameBuilder renameFile(List<String> srcs, List<String> dsts, Rename... options)
throws IOException, UnsupportedOperationException {
return new FSRenameBuilder(this, srcs, dsts, options);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* 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;

import com.sun.tools.javac.util.Pair;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

import java.io.IOException;
import java.util.concurrent.CompletableFuture;

/**
* Builder for input streams and subclasses whose return value is
* actually a completable future: this allows for better asynchronous
* operation.
*
* To be more generic, {@link #opt(String, int)} and {@link #must(String, int)}
* variants provide implementation-agnostic way to customize the builder.
* Each FS-specific builder implementation can interpret the FS-specific
* options accordingly, for example:
*
* If the option is not related to the file system, the option will be ignored.
* If the option is must, but not supported by the file system, a
* {@link IllegalArgumentException} will be thrown.
*
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public interface FutureRenameBuilder {
public FutureRenameBuilder rename(Pair<String, String> src2dst);
public FutureRenameBuilder option(Options.Rename... options);

CompletableFuture<IORenameStatistic> build()
throws IllegalArgumentException, UnsupportedOperationException,
IOException;
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/**
*
* 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;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

@InterfaceAudience.Public
@InterfaceStability.Unstable
public class IORenameStatistic {
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
/*
* 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 com.sun.tools.javac.util.Pair;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FutureRenameBuilder;
import org.apache.hadoop.fs.Options;

import javax.annotation.Nonnull;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;

import static java.util.Objects.requireNonNull;

/**
* Builder for input streams and subclasses whose return value is
* actually a completable future: this allows for better asynchronous
* operation.
*
* To be more generic, {@link #opt(String, int)} and {@link #must(String, int)}
* variants provide implementation-agnostic way to customize the builder.
* Each FS-specific builder implementation can interpret the FS-specific
* options accordingly, for example:
*
* If the option is not related to the file system, the option will be ignored.
* If the option is must, but not supported by the file system, a
* {@link IllegalArgumentException} will be thrown.
*
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract class FutureRenameBuilderImpl
implements FutureRenameBuilder {

private final FileSystem fileSystem;
private List<String> srcs = new ArrayList<>();
private List<String> dsts = new ArrayList<>();
private List<Options.Rename> options = new ArrayList<>();
/**
* Constructor.
* @param srcs owner FS.
* @param dsts path
* @param dsts options
*/
protected FutureRenameBuilderImpl(@Nonnull FileSystem fileSystem,
@Nonnull List<String> srcs,
@Nonnull List<String> dsts,
Options.Rename... options) {
this.fileSystem = requireNonNull(fileSystem, "fileSystem");
this.srcs.addAll(srcs);
this.dsts.addAll(dsts);
for (Options.Rename opt : options) {
this.options.add(opt);
}
}

public FileSystem getFS() {
return fileSystem;
}

public List<String> getSrcs() {
return srcs;
}

public List<String> getDsts() {
return dsts;
}

public Options.Rename[] getOptions() {
return options.toArray(new Options.Rename[options.size()]);
}

public FutureRenameBuilder rename(Pair<String, String> src2dst) {
srcs.add(src2dst.fst);
dsts.add(src2dst.snd);
return this;
}

public FutureRenameBuilder option(Options.Rename... options) {
for (Options.Rename opt : options) {
this.options.add(opt);
}
return this;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,7 @@
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.hdfs.protocol.AclException;
import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
import org.apache.hadoop.hdfs.protocol.BatchRenameException;
import org.apache.hadoop.hdfs.protocol.BatchedDirectoryListing;
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
Expand Down Expand Up @@ -1609,6 +1610,27 @@ public void rename(String src, String dst, Options.Rename... options)
}
}

/**
* Rename a batch files or directories.
* @see ClientProtocol#batchRename(List<String>, List<String>,
* Options.Rename...)
*/
public void batchRename(List<String> srcs, List<String> dsts,
Options.Rename... options) throws IOException {
checkOpen();
try {
namenode.batchRename(srcs, dsts, options);
} catch(RemoteException re) {
throw re.unwrapRemoteException(AccessControlException.class,
NSQuotaExceededException.class,
DSQuotaExceededException.class,
UnresolvedPathException.class,
SnapshotAccessControlException.class,
BatchRenameException.class);
}
}


/**
* Truncate a file to an indicated size
* See {@link ClientProtocol#truncate}.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,8 @@ public enum OpType {
GET_SNAPSHOT_LIST("op_get_snapshot_list"),
TRUNCATE(CommonStatisticNames.OP_TRUNCATE),
UNSET_EC_POLICY("op_unset_ec_policy"),
UNSET_STORAGE_POLICY("op_unset_storage_policy");
UNSET_STORAGE_POLICY("op_unset_storage_policy"),
BATCH_RENAME("op_batch_rename");

private static final Map<String, OpType> SYMBOL_MAP =
new HashMap<>(OpType.values().length);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,8 @@
import org.apache.hadoop.fs.FsStatus;
import org.apache.hadoop.fs.GlobalStorageStatistics;
import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
import org.apache.hadoop.fs.IORenameStatistic;
import org.apache.hadoop.fs.InvalidPathException;
import org.apache.hadoop.fs.InvalidPathHandleException;
import org.apache.hadoop.fs.PartialListing;
import org.apache.hadoop.fs.MultipartUploaderBuilder;
Expand Down Expand Up @@ -965,6 +967,26 @@ public Void next(final FileSystem fs, final Path p)
}
}

protected List<String> getBatchPathName(List<String> files) {
List<String> ret = new ArrayList<>();
for(String f : files) {
ret.add(getPathName(new Path(f)));
}
return ret;
}

@Override
public IORenameStatistic batchRename(List<String> srcs, List<String> dsts,
final Options.Rename... options) throws IOException {
if (srcs.size() != dsts.size()) {
throw new InvalidPathException("mismatch batch path src: " +
String.join(",", srcs) + " dst: " + String.join(",", dsts));
}
statistics.incrementWriteOps(1);
dfs.batchRename(getBatchPathName(srcs), getBatchPathName(dsts));
return new IORenameStatistic();
}

@Override
public boolean truncate(Path f, final long newLength) throws IOException {
statistics.incrementWriteOps(1);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@ public static Optional<Boolean> hasPathCapability(final Path path,
case CommonPathCapabilities.FS_SNAPSHOTS:
case CommonPathCapabilities.FS_STORAGEPOLICY:
case CommonPathCapabilities.FS_XATTRS:
case CommonPathCapabilities.FS_BATCH_RENAME:
return Optional.of(true);
case CommonPathCapabilities.FS_SYMLINKS:
return Optional.of(FileSystem.areSymlinksEnabled());
Expand Down
Loading