Hi Jason, If the bufferSize is set when the stream is created, when the size is reached, will it automatically write itself out to HDFS? What happens when the buffer size is exceeded?
-sasha On Mon, May 18, 2009 at 3:04 AM, jason hadoop <jason.had...@gmail.com> wrote: > When you open a file you have the option, blockSize > /** > * Opens an FSDataOutputStream at the indicated Path with write-progress > * reporting. > * @param f the file name to open > * @param permission > * @param overwrite if a file with this name already exists, then if true, > * the file will be overwritten, and if false an error will be thrown. > * @param bufferSize the size of the buffer to be used. > * @param replication required block replication for the file. > * @param blockSize > * @param progress > * @throws IOException > * @see #setPermission(Path, FsPermission) > */ > public abstract FSDataOutputStream create(Path f, > FsPermission permission, > boolean overwrite, > int bufferSize, > short replication, > long blockSize, > Progressable progress) throws IOException; > > On Fri, May 15, 2009 at 12:44 PM, Sasha Dolgy <sdo...@gmail.com> wrote: > >> Hi Todd, >> Reading through the JIRA, my impression is that data will be written out to >> hdfs only once it has reached a certain size in the buffer. Is it possible >> to define the size of that buffer? Or is this a future enhancement? >> >> -sasha >> >> On Fri, May 15, 2009 at 6:14 PM, Todd Lipcon <t...@cloudera.com> wrote: >> >> > Hi Sasha, >> > >> > What version are you running? Up until very recent versions, sync() was >> not >> > implemented. Even in the newest releases, sync isn't completely finished, >> > and you may find unreliable behavior. >> > >> > For now, if you need this kind of behavior, your best bet is to close >> each >> > file and then open the next every N minutes. For example, if you're >> > processing logs every 5 minutes, simply close log file log.00223 and >> round >> > robin to log.00224 right before you need the data to be available to >> > readers. If you're collecting data at a low rate, these files may end up >> > being rather small, and you should probably look into doing merges on the >> > hour/day/etc to avoid small-file proliferation. >> > >> > If you want to track the work being done around append and sync, check >> out >> > HADOOP-5744 and the issues referenced therein: >> > >> > http://issues.apache.org/jira/browse/HADOOP-5744 >> > >> > Hope that helps, >> > -Todd >> > >> > On Fri, May 15, 2009 at 6:35 AM, Sasha Dolgy <sdo...@gmail.com> wrote: >> > >> > > Hi there, forgive the repost: >> > > >> > > Right now data is received in parallel and is written to a queue, then >> a >> > > single thread reads the queue and writes those messages to a >> > > FSDataOutputStream which is kept open, but the messages never get >> > flushed. >> > > Tried flush() and sync() with no joy. >> > > >> > > 1. >> > > outputStream.writeBytes(rawMessage.toString()); >> > > >> > > 2. >> > > >> > > log.debug("Flushing stream, size = " + s.getOutputStream().size()); >> > > s.getOutputStream().sync(); >> > > log.debug("Flushed stream, size = " + s.getOutputStream().size()); >> > > >> > > or >> > > >> > > log.debug("Flushing stream, size = " + s.getOutputStream().size()); >> > > s.getOutputStream().flush(); >> > > log.debug("Flushed stream, size = " + s.getOutputStream().size()); >> > > >> > > The size() remains the same after performing this action. >> > > >> > > 2009-05-12 12:42:17,470 DEBUG [Thread-7] (FSStreamManager.java:28) >> > > hdfs.HdfsQueueConsumer: Thread 19 getting an output stream >> > > 2009-05-12 12:42:17,470 DEBUG [Thread-7] (FSStreamManager.java:49) >> > > hdfs.HdfsQueueConsumer: Re-using existing stream >> > > 2009-05-12 12:42:17,472 DEBUG [Thread-7] (FSStreamManager.java:63) >> > > hdfs.HdfsQueueConsumer: Flushing stream, size = 1986 >> > > 2009-05-12 12:42:17,472 DEBUG [Thread-7] (DFSClient.java:3013) >> > > hdfs.DFSClient: DFSClient flush() : saveOffset 1613 bytesCurBlock 1986 >> > > lastFlushOffset 1731 >> > > 2009-05-12 12:42:17,472 DEBUG [Thread-7] (FSStreamManager.java:66) >> > > hdfs.HdfsQueueConsumer: Flushed stream, size = 1986 >> > > 2009-05-12 12:42:19,586 DEBUG [Thread-7] (HdfsQueueConsumer.java:39) >> > > hdfs.HdfsQueueConsumer: Consumer writing event >> > > 2009-05-12 12:42:19,587 DEBUG [Thread-7] (FSStreamManager.java:28) >> > > hdfs.HdfsQueueConsumer: Thread 19 getting an output stream >> > > 2009-05-12 12:42:19,588 DEBUG [Thread-7] (FSStreamManager.java:49) >> > > hdfs.HdfsQueueConsumer: Re-using existing stream >> > > 2009-05-12 12:42:19,589 DEBUG [Thread-7] (FSStreamManager.java:63) >> > > hdfs.HdfsQueueConsumer: Flushing stream, size = 2235 >> > > 2009-05-12 12:42:19,589 DEBUG [Thread-7] (DFSClient.java:3013) >> > > hdfs.DFSClient: DFSClient flush() : saveOffset 2125 bytesCurBlock 2235 >> > > lastFlushOffset 1986 >> > > 2009-05-12 12:42:19,590 DEBUG [Thread-7] (FSStreamManager.java:66) >> > > hdfs.HdfsQueueConsumer: Flushed stream, size = 2235 >> > > >> > > So although the Offset is changing as expected, the output stream isn't >> > > being flushed or cleared out and isn't being written to file unless the >> > > stream is closed() ... is this the expected behaviour? >> > > >> > > -sd