[
https://issues.apache.org/jira/browse/SAMZA-138?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13973605#comment-13973605
]
Yan Fang commented on SAMZA-138:
--------------------------------
Since SAMZA-235 needs file reader, I pick up this ticket. Thank [~poltak], he
made my work easier.
For the improvements:
{quote}
handle reading from multiple files (suggested alternative input specification-
point 2)
{quote}
Done.
{quote}
use of filepos for IncomingMessageEnvelope offset (more info here
{quote}
Hi [~criccomini] , I need some help here. In order to have the samza job pick
up from offset, I also need to implement a new SystemAdmin in
FileReaderSystemFactory. It seems that I can not reuse the KafkaSystemAdmin
directly. And reading SinglePartitionWithoutOffsetsSystemAdmin does not help a
lot. So my questions are:
* what is the exactly goal of SystemAdmin?
* Does getSystemStreamMetadata method need to talk directly with Kafka to get
the metadata? I am a little confused with CheckPointManager here.
{quote}
come up with a reasonable bounded queue threshold (the value of 100 was
arbitrary, as I was unsure of a reasonable value here)
{quote}
It is 100 by default. I plan not to change it. Will SAMZA-220 affect this?
{quote}
better handling for the exceptions encountered (I wasn't 100% sure about some
of them)
{quote}
I wrap all the exceptions to SamzaException. Whenever there is a
SamzaException, will the Samza job be killed?
> 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
> Priority: Minor
> Labels: feature, newbie, patch
> Attachments: FileReaderConsumer.java, FileReaderSystemFactory.java
>
>
> 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)