On Fri, Sep 19, 2014 at 9:41 AM, Vinayakumar B <vinayakum...@apache.org> wrote: > Thanks Colin for the detailed explanation. > > On Fri, Sep 19, 2014 at 9:38 PM, Colin McCabe <cmcc...@alumni.cmu.edu> > wrote: >> >> On Thu, Sep 18, 2014 at 11:06 AM, Vinayakumar B <vinayakum...@apache.org> > wrote: >> > bq. I don't know about the merits of this, but I do know that native >> > filesystems >> > implement this by not raising the EOF exception on the seek() but only > on >> > the read ... some of the non-HDFS filesystems Hadoop support work this > way. >> >> Pretty much all of them should. POSIX specifies that seeking past the >> end of a file is not an error. Reading past the end of the file gives >> an EOF, but the seek always succeeds. >> >> It would be nice if HDFS had this behavior as well. It seems like >> this would have to be a 3.0 thing, since it's a potential >> incompatibility. >> >> > I agree with you steve. read only will throw EOF. But when we know that >> > file is being written and it can have fresh data, then polling can be > done >> > by calling available(). later we can continue read or call seek. >> > > Yes, I too agree that, if we are changing seek() behaviour, then definitely > that is a 3.0 thing. > >> InputStream#available() has a really specific function in Java: >> telling you approximately how much data is currently buffered by the >> stream. >> >> As a side note, InputStream#available seems to be one of the most >> misunderstood APIs in Java. It's pretty common for people to assume >> that it means "how much data is left in the stream" or something like >> that. I think I made that mistake at least once when getting started >> with Java. I guess the JavaDoc is kind of vague-- it specifies that >> available returns "an estimate of the number of bytes that can be read >> (or skipped over) from this input stream without blocking." But in >> practice, that means how much is buffered (for a file-backed stream, >> to pull more bytes from the OS would require a syscall, which is >> "blocking." Similarly for network-backed streams.) > > Yes, InputStream#available() javadoc says its the data which can be read > non-blocking, > It also says, impls can chose to return total number of bytes available in > the stream, which is done in DFSInputStream
I think DFSInputStream#available would be a lot more useful if it told users how much data could be read without doing network I/O. Right now, this is something that users have no way of figuring out. Plus, available() returns an int, and HDFS files are often longer than 2 gigs. We have an API for getting file length and current offset (DFSInputStream#getFileLength)... we don't need to make available() do that stuff. > >> In any case, we certainly could create a new API to refresh >> inputstream data. I guess the idea would be to check if the last >> block we knew about had reached full length-- if so, we would ask the >> NameNode for any new block locations. So it would be a DN operation >> in most cases, but sometimes a NN operation. > > Correct, we can anyway have new API to refresh. But if clients uses just > InputStream interface, then IMO its better to do this in available() > itself. This will be inline with native FileInputStream. > If the file is closed, then we can chose to return -1, else if no new data > available then can return 0 as its doing now. > As you mentioned, refresh can be done only from DNs, and if the block is > full, then refresh from NN again. But also needs to think how we can handle > this, if the proposed "variable length blocks" comes to HDFS. Returning the complete file length minus the current position might technically be within the bounds of the JavaDoc (although a poor implementation, I would argue), but going over the network and contacting the NameNode is definitely way outside it. In C++ terms, available() is intended to be const... it's not supposed to mutate the state of the stream. In my opinion, at least... > >> Have you looked at https://issues.apache.org/jira/browse/HDFS-6633: >> Support reading new data in a being written file until the file is >> closed? That patch seems to take the approach of turning reading past >> the end of the file into an operation that blocks until there is new >> data. (when dfs.client.read.tail-follow is set.) I think I prefer >> the idea of a new refresh API, just because it puts more control in >> the hands of the user. > > Just now saw the Jira. Intention of the Jira is same as this discussion. > Before seeing this mail, I had raised > https://issues.apache.org/jira/browse/HDFS-7099, anyway we can duplicate > this. > But patch proposed in HDFS-6633, is a blocking call on read(), which may > not be feasible for the clients. > I feel polling for new data, instead of blocking read call(). Anyway this > can be discussed more on the jira itself. I agree, having a call to poll for new data would be better. best, Colin > >> Another thing to consider is how this all interacts with the proposed >> HDFS truncate operation (see HDFS-3107). > > I haven't seen this in detail. I will check it soon. > >> best, >> Colin >> >> >> > >> > One simple example use case is tailing a file. >> > >> > Regards, >> > Vinay >> > >> > On Thu, Sep 18, 2014 at 3:35 PM, Steve Loughran <ste...@hortonworks.com> >> > wrote: >> > >> >> I don't know about the merits of this, but I do know that native >> >> filesystems implement this by not raising the EOF exception on the > seek() >> >> but only on the read ... some of the non-HDFS filesystems Hadoop > support >> >> work this way. >> >> >> >> -I haven't ever looked to see what code assumes that it is the seek > that >> >> fails, not the read. >> >> -PositionedReadable had better handle this too, even if it isn't done > via a >> >> seek()-read()-seek() sequence >> >> >> >> >> >> On 18 September 2014 08:48, Vinayakumar B <vinayakum...@apache.org> > wrote: >> >> >> >> > Hi all, >> >> > >> >> > Currently *DFSInputStream *doen't allow reading a write-inprogress > file, >> >> > once all written bytes, by the time of opening an input stream, are > read. >> >> > >> >> > To read further update on the same file, needs to be read by opening >> >> > another stream to the same file again. >> >> > >> >> > Instead how about refreshing length of such open files if the current >> >> > position is at earlier EOF. >> >> > >> >> > May be this could be done in *available() *method, So that clients > who >> >> > knows that original writer will not close then read can continuously > poll >> >> > for new data using the same stream? >> >> > >> >> > PS: This is possible in local disk read using FileInputStream >> >> > >> >> > Regards, >> >> > Vinay >> >> > >> >> >> >> -- >> >> CONFIDENTIALITY NOTICE >> >> NOTICE: This message is intended for the use of the individual or > entity to >> >> which it is addressed and may contain information that is confidential, >> >> privileged and exempt from disclosure under applicable law. If the > reader >> >> of this message is not the intended recipient, you are hereby notified > that >> >> any printing, copying, dissemination, distribution, disclosure or >> >> forwarding of this communication is strictly prohibited. If you have >> >> received this communication in error, please contact the sender > immediately >> >> and delete it from your system. Thank You. >> >> > > Regards, > Vinay