[ 
https://issues.apache.org/jira/browse/IGNITE-2730?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15215292#comment-15215292
 ] 

Roman Shtykh edited comment on IGNITE-2730 at 3/29/16 3:42 AM:
---------------------------------------------------------------

Denis,

1) I wouldn't like to lose any events but enable a user to implement dedup 
logic, if needed. So I was thinking about at-least-once guarantee.

I am confused with "a guarantee when an even can be missed"... Does it mean 
duplicate events will be ignored when we have a primary node check? Or does it 
mean that with a primary node check events can be lost? If we receive events 
both from primary and backup nodes, are there chances that events can be lost?

To be sure my understanding is correct: with the current design, both primary 
and backup nodes will send events. To implement at-least-once guarantee, you 
recommend not to use _remoteFilter_ and make sure events come from primary 
nodes. Is it correct?
Unfortunately, I couldn't find any other way how to check if events come from 
primary nodes (except using _ContinuousQuery_, but that limits types of events 
a user can receive).

On using _IgniteQueue_ : batching is needed to efficiently send data to Kafka. 
If we batch it in memory locally, the accumulated data can be lost if something 
goes wrong with the _IgniteSourceTask_. Therefore I chose to use _IgniteQueue_ 
to make sure the data is safe.
It won't guarantee 100% safety though, so I can use a local queue -- it will be 
faster.

2. Connector requires a de/serializer (Please see _makeWorkerProps()_). I can 
implement it but _JdkMarshaller_ has already the functionality.


was (Author: roman_s):
Denis,

1) I wouldn't like to lose any events but enable a user to implement dedup 
logic, if needed. So I was thinking about at-least-once guarantee.

I am confused with "a guarantee when an even can be missed"... Does it mean 
duplicate events will be ignored when we have a primary node check? Or does it 
mean that with a primary node check events can be lost? If we receive events 
both from primary and backup nodes, are there chances that events can be lost?

To be sure my understanding is correct: with the current design, both primary 
and backup nodes will send events. To implement at-least-once guarantee, you 
recommend not to use _remoteFilter_ and make sure events come from primary 
nodes. Is it correct?
Unfortunately, I couldn't find any other way how to check if events come from 
primary nodes (except using _ContinuousQuery_, but that limits types of events 
a user can receive).

On using _IgniteQueue_ : batching is needed to efficiently send data to Kafka. 
If we batch it in memory locally, the accumulated data can be lost if something 
goes wrong with the _IgniteSourceTask_. Therefore I chose to use _IgniteQueue_ 
to make sure the data is safe.

2. Connector requires a de/serializer (Please see _makeWorkerProps()_). I can 
implement it but _JdkMarshaller_ has already the functionality.

> Ignite Events Source Streaming to Kafka
> ---------------------------------------
>
>                 Key: IGNITE-2730
>                 URL: https://issues.apache.org/jira/browse/IGNITE-2730
>             Project: Ignite
>          Issue Type: New Feature
>          Components: streaming
>            Reporter: Roman Shtykh
>            Assignee: Roman Shtykh
>              Labels: community
>
> Streaming specified Ignite events 
> (https://apacheignite.readme.io/docs/events) to Kafka via Kafka Connect.
> It has to be added to org.apache.ignite.stream.kafka.connect package.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to