HADOOP-6138. Eliminate the depracate warnings introduced by H-5438. Contributed by He Yongqiang

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@795172 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2009-07-17 18:01:15 +00:00
parent 5478e9a188
commit 391112f6ce
4 changed files with 11 additions and 5 deletions

View File

@ -880,6 +880,9 @@ Trunk (unreleased changes)
HADOOP-6137. Fix project specific test-patch requirements
(Giridharan Kesavan)
HADOOP-6138. Eliminate the depracate warnings introduced by H-5438.
(He Yongqiang via szetszwo)
Release 0.20.1 - Unreleased
INCOMPATIBLE CHANGES

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.fs;
import java.io.*;
import java.util.Arrays;
import java.util.EnumSet;
import java.util.zip.CRC32;
import org.apache.commons.logging.Log;
@ -360,14 +361,14 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
/** {@inheritDoc} */
@Override
public FSDataOutputStream create(Path f, FsPermission permission,
boolean overwrite, int bufferSize, short replication, long blockSize,
EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize,
Progressable progress) throws IOException {
Path parent = f.getParent();
if (parent != null && !mkdirs(parent)) {
throw new IOException("Mkdirs failed to create " + parent);
}
final FSDataOutputStream out = new FSDataOutputStream(
new ChecksumFSOutputSummer(this, f, overwrite, bufferSize, replication,
new ChecksumFSOutputSummer(this, f, flag.contains(CreateFlag.OVERWRITE), bufferSize, replication,
blockSize, progress), null);
if (permission != null) {
setPermission(f, permission);

View File

@ -500,8 +500,9 @@ public abstract class FileSystem extends Configured implements Closeable {
long blockSize,
Progressable progress
) throws IOException {
return this.create(f, FsPermission.getDefault(),
overwrite, bufferSize, replication, blockSize, progress);
return this.create(f, FsPermission.getDefault(), overwrite ? EnumSet
.of(CreateFlag.OVERWRITE) : EnumSet.of(CreateFlag.CREATE), bufferSize,
replication, blockSize, progress);
}
/**

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
@ -580,7 +581,7 @@ public class HarFileSystem extends FilterFileSystem {
public FSDataOutputStream create(Path f,
FsPermission permission,
boolean overwrite,
EnumSet<CreateFlag> flag,
int bufferSize,
short replication,
long blockSize,