> On April 30, 2014, 6:27 p.m., Chris Riccomini wrote:
> > 1. Stylistic: white space should be spaces, not tabs.
> > 2. Can we put this in the main code base, rather than in hello-samza? 
> > Hello-samza can still use it, but this seems like a really useful consumer. 
> > I think samza-core is appropriate for it.
> > 3. If we do (2), would you feel comfortable converting this to Scala? Happy 
> > to provide Scala guidance, if you need it.
> > 4. Need tests.
> > 5. I see that your implementation right now treats an offset as a line 
> > (e.g. 0 means the first line, 1 means the second, etc). This makes things a 
> > lot easier to implement internally, but means we have to read the entire 
> > file every time we start up. If we use the actual file offset for the \n 
> > location, we can do an fseek, which is much faster.

3. If we want to put them in core base, I can convert this to Scala. It may 
take a little effort, but it's fine to me.
4. Will add tests.
5. agree. Will update this.


> On April 30, 2014, 6:27 p.m., Chris Riccomini wrote:
> > samza-wikipedia/src/main/java/samza/examples/filereader/system/FileReaderSystemAdmin.java,
> >  line 40
> > <https://reviews.apache.org/r/20869/diff/2/?file=571202#file571202line40>
> >
> >     This is not quite correct. Rather than use a static 
> > FilePartitionMetadata and hard code made-up offsets, you'll need to 
> > uniquely determine the oldest, newest, and upcoming offsets for every 
> > stream (file).
> >     
> >     The oldest offset will always be 0.
> >     
> >     The newest offset will always be the offset of second-to-last \n in the 
> > file + 1. The upcoming offset will be the offset of the last \n in the file 
> > + 1.
> >     
> >     msg1\n <-- oldest offset
> >     msg2\n <-- newest offset
> >     msg3\n <-- upcoming offset
> >     not-yet-complete-msg-that-has-no-newline-after-it-yet

oldest offset makes sense to me.
In order to get the newest offset, do I have to create a kafka consumer to get 
current last \n? Otherwise, it seems no way I can get the file pos. (Read the 
file in this class?)


- Yan


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/20869/#review41856
-----------------------------------------------------------


On April 30, 2014, 12:42 a.m., Yan Fang wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/20869/
> -----------------------------------------------------------
> 
> (Updated April 30, 2014, 12:42 a.m.)
> 
> 
> Review request for samza.
> 
> 
> Repository: samza-hello-samza
> 
> 
> Description
> -------
> 
> a filereader system which reads specified file contents onto stream.
> 
> 1. FileReaderSystemAdmin implements SystemAdmin and is similar to 
> SinglePartitionWithoutOffsetsSystemAdmin except it updates offset.
> 2. FileReaderSystemFactory implements SystemFactory. It throws a Samza 
> exception for getProducer because this system does not suppose to write to 
> files.
> 3. FileReaderSystemConsumer gets file path from stream names, creates threads 
> for each file and then read files line-by-line. It skips certain lines when 
> its startingOffset is not zero, meaning it has checkpoint.
> 
> 
> Diffs
> -----
> 
>   
> samza-wikipedia/src/main/java/samza/examples/filereader/system/FileReaderSystemAdmin.java
>  PRE-CREATION 
>   
> samza-wikipedia/src/main/java/samza/examples/filereader/system/FileReaderSystemConsumer.java
>  PRE-CREATION 
>   
> samza-wikipedia/src/main/java/samza/examples/filereader/system/FileReaderSystemFactory.java
>  PRE-CREATION 
> 
> Diff: https://reviews.apache.org/r/20869/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Yan Fang
> 
>

Reply via email to