[ 
https://issues.apache.org/jira/browse/HADOOP-5438?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12735950#action_12735950
 ] 

He Yongqiang commented on HADOOP-5438:
--------------------------------------

{quote}
If there weren't so many different create methods, I would suggest having one 
each for just the Enum value and and equivalent one that takes an enumset, so 
that if you just wanted to overwrite you could just use the one that takes the 
enum, and the convenience method would jam it into an enumset. It's too bad 
Java won't automatically convert a single enum into an enumset when necessary.
{quote}
Agree. It will be better we can provide a create method which takes one 
CreateFlag enum value and equivalent one that takes an enumset (i think it's 
already there). But how about the other already existing create methods? Remove 
them? And this will be a incompatible change.

>>We could use a variable arity parameter, e.g.: FSDataOutputStream create(Path 
>>file, CreateFlag... flags);
This is a very good suggestion. some programmer(like me) does not use 
EnumSet.of() very often. But the situation can be mitigated if we provide a 
method that accept one CreateFlag parameter.

> Merge FileSystem.create and FileSystem.append
> ---------------------------------------------
>
>                 Key: HADOOP-5438
>                 URL: https://issues.apache.org/jira/browse/HADOOP-5438
>             Project: Hadoop Common
>          Issue Type: Improvement
>          Components: fs
>            Reporter: He Yongqiang
>            Assignee: He Yongqiang
>             Fix For: 0.21.0
>
>         Attachments: Hadoop-5438(2009-04-06).patch, 
> Hadoop-5438-2009-03-30.patch, Hadoop-5438-2009-03-31-2.patch, 
> Hadoop-5438-2009-03-31.patch, Hadoop-5438-2009-05-10.patch, 
> Hadoop-5438-2009-05-15.patch, Hadoop-5438-2009-05-19.patch, 
> Hadoop-5438-2009-05-5.patch
>
>
> Currently, when a user wants to modify a file, the user first calls exists() 
> to know if this file is already there. And then uses create() or append() 
> according to whether the file exists or not.
> the code looks like:
> {code}
> FSDataOutputStream out_1 = null;
> if (fs.exists(path_1))
>    out_1 = fs.append(path_1);
> else
>    out_1 = fs.create(path_1);
> {code}
> . On the performace side,It involes two RPCs. On the easy-of-use side, it is 
> not very convient in contrast to the traditional open interface.
> It will more complicate if there is a overwrite parameter specified. I donot 
> know whether there is a bug about 'overwrite' in 0.19, some times it takes a 
> long time for overwrite creates to reture. So i make the write file code with 
> overwrite param works like:
> {code}
> boolean exists = fs.exists(name);
> if (overwrite) {
>     if (exists)
>        fs.delete(name, true);
>      this.out = fs.create(name, overwrite, bufferSize, replication,
>                                   blockSize, progress);
>      this.currentRowID = 0;
>  } else {
>    if (!exists)
>       this.out = fs.create(name, overwrite, bufferSize,
>                                       replication, blockSize, progress);
>    else
>       this.out = fs.append(name, bufferSize, progress);
> {code}
> Some code statements there are really redundant and not needed, especialy 
> with the delete(). But without deleting first, the overwrite takes a long 
> time to reture.
> BTW, i will create another issue about the overwrite problem. If it is not a 
> bug at all or a duplicate, someone please close it.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to