[ https://issues.apache.org/jira/browse/SAMZA-138?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13987498#comment-13987498 ]
Martin Kleppmann commented on SAMZA-138: ---------------------------------------- bq. Does getSystemStreamMetadata method actually have nothing to do with checkpoint? The system stream metadata is information about what messages are available to be read from a stream. It isn't specific to any particular job. A checkpoint, on the other hand, has information on how far a particular job has consumed a stream. Different jobs may be consuming the same stream at different offsets, and thus have different checkpoints. bq. Is the Metadata only used for bootstrap stream? There are ideas floating around for using Metadata in new ways. For example, SAMZA-179 explored exposing the metadata to tasks (although that isn't yet committed, as we weren't sure that this was the best approach). SAMZA-234 may also use metadata. So the metadata should be general, not tied to bootstrap streams in particular. bq. newest/upcoming offset is actually just the status of the stream(file in this context) when we start the container, right? That's the way bootstrap streams use it (only read it once when the container is started), but in general, it would make more sense for it to return the latest value whenever it is called. There's already a cache in place (StreamMetadataCache) to avoid fetching the same metadata several times in quick succession. > System that places specified file contents onto stream > ------------------------------------------------------ > > Key: SAMZA-138 > URL: https://issues.apache.org/jira/browse/SAMZA-138 > Project: Samza > Issue Type: New Feature > Affects Versions: 0.7.0 > Environment: RHELinux 2.6.18-371.4.1.el5 > Reporter: Jonathan Poltak Samosir > Assignee: Yan Fang > Priority: Minor > Labels: feature, newbie, patch > Attachments: FileReaderConsumer.java, FileReaderSystemFactory.java, > SAMZA-138.patch > > > A fairly straightforward Samza System that reads from a specified file, and > places that file's contents onto a SystemStreamPartition for use as input for > a StreamTask. > Roughly based off how the hello-samza example project's WikipediaSystem works > (more the SystemConsumerFactory rather than SystemConsumer class). > Probably needs a bit of work, but basic functionality works as intended. > Hopefully useful to some, either as a functioning system or as a base for a > more robust and functionally-promising system that you wish to implement. > Some suggested improvements (not yet implemented): > * handle reading from multiple files ([suggested alternative input > specification|https://mail-archives.apache.org/mod_mbox/incubator-samza-dev/201401.mbox/%3C1B43C7411DB20E47AB0FB62E7262B80179BA7465%40ESV4-MBX01.linkedin.biz%3E]- > point 2) > * use of filepos for IncomingMessageEnvelope offset ([more info > here|https://mail-archives.apache.org/mod_mbox/incubator-samza-dev/201401.mbox/%3C1B43C7411DB20E47AB0FB62E7262B80179BA749D%40ESV4-MBX01.linkedin.biz%3E] > * come up with a reasonable bounded queue threshold (the value of 100 was > arbitrary, as I was unsure of a reasonable value here) > * better handling for the exceptions encountered (I wasn't 100% sure about > some of them) -- This message was sent by Atlassian JIRA (v6.2#6252)