Re: kafka 0.9 support

2019-04-03 Thread Raghu Angadi
On Wed, Apr 3, 2019 at 5:46 AM David Morávek 
wrote:

> I'd say that APIs we use in KafkaIO are pretty much stable since 0.10
> release, all reflection based compatibility adapters seem to be aimed for
> 0.9 release (which is 8 major releases behind current Kafka release).
>
> We may take an inspiration from Flink's kafka connector
> <https://ci.apache.org/projects/flink/flink-docs-stable/dev/connectors/kafka.html>,
> they maintain separate maven artifact for all supported Kafka APIs. This
> may be the best approach as we can still share most of the codebase between
> versions, have compile time checks and also run tests against all of the
> supported versions.
>

>From that page, Flink also moved to single Kafka connector for versions
10.x and newer. Kafka itself seems to have improved compatibility between
client and broker versions starting 0.11. Not sure if there is any need now
to make multiple versions of KafkaIO versions for 0.9.x etc. Are you
suggesting we should?

>From Flink's page:
"Starting with Flink 1.7, there is a new universal Kafka connector that
does not track a specific Kafka major version. Rather, it tracks the latest
version of Kafka at the time of the Flink release.

If your Kafka broker version is 1.0.0 or newer, you should use this Kafka
connector. If you use an older version of Kafka (0.11, 0.10, 0.9, or 0.8),
you should use the connector corresponding to the broker version."



>
> I'm not really comfortable with reflection based adapters
> <https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConsumerSpEL.java>
> as they seem fragile and don't provide compile time checks.
>
> On Tue, Apr 2, 2019 at 11:27 PM Austin Bennett <
> whatwouldausti...@gmail.com> wrote:
>
>> I withdraw my concern -- checked on info on the cluster I will eventually
>> access.  It is on 0.8, so I was speaking too soon.  Can't speak to rest of
>> user base.
>>
>> On Tue, Apr 2, 2019 at 11:03 AM Raghu Angadi  wrote:
>>
>>> Thanks to David Morávek for pointing out possible improvement to KafkaIO
>>> for dropping support for 0.9 since it avoids having a second consumer just
>>> to fetch latest offsets for backlog.
>>>
>>> Ideally we should be dropping 0.9 support for next major release, in
>>> fact better to drop versions before 0.10.1 at the same time. This would
>>> further reduce reflection based calls for supporting multiple versions. If
>>> the users still on 0.9 could stay on current stable release of Beam,
>>> dropping would not affect them. Otherwise, it would be good to hear from
>>> them about how long we need to keep support for old versions.
>>>
>>> I don't think it is good idea to have multiple forks of KafkaIO in the
>>> same repo. If we do go that route, we should fork the entire kafka
>>> directory and rename the main class KafkaIO_Unmaintained :).
>>>
>>> IMHO, so far, additional complexity for supporting these versions is not
>>> that bad. Most of it is isolated to ConsumerSpEL.java & ProducerSpEL.java.
>>> My first preference is dropping support for deprecated versions (and a
>>> deprecate a few more versions, may be till the version that added
>>> transactions around 0.11.x I think).
>>>
>>> I haven't looked into what's new in Kafka 2.x. Are there any features
>>> that KafkaIO should take advantage of? I have not noticed our existing code
>>> breaking. We should certainly certainly support latest releases of Kafka.
>>>
>>> Raghu.
>>>
>>> On Tue, Apr 2, 2019 at 10:27 AM Mingmin Xu  wrote:
>>>
>>>>
>>>> We're still using Kafka 0.10 a lot, similar as 0.9 IMO. To expand
>>>> multiple versions in KafkaIO is quite complex now, and it confuses users
>>>> which is supported / which is not. I would prefer to support Kafka 2.0+
>>>> only in the latest version. For old versions, there're some options:
>>>> 1). document Kafka-Beam support versions, like what we do in
>>>> FlinkRunner;
>>>> 2). maintain separated KafkaIOs for old versions;
>>>>
>>>> 1) would be easy to maintain, and I assume there should be no issue to
>>>> use Beam-Core 3.0 together with KafkaIO 2.0.
>>>>
>>>> Any thoughts?
>>>>
>>>> Mingmin
>>>>
>>>> On Tue, Apr 2, 2019 at 9:56 AM Reuven Lax  wrote:
>>>>
>>>>> KafkaIO is marked as Experimental, and the comment already warns that
>>>>> 0.9 support might be removed. I think that if users still rely on

Re: kafka 0.9 support

2019-04-02 Thread Raghu Angadi
Thanks to David Morávek for pointing out possible improvement to KafkaIO
for dropping support for 0.9 since it avoids having a second consumer just
to fetch latest offsets for backlog.

Ideally we should be dropping 0.9 support for next major release, in fact
better to drop versions before 0.10.1 at the same time. This would further
reduce reflection based calls for supporting multiple versions. If the
users still on 0.9 could stay on current stable release of Beam, dropping
would not affect them. Otherwise, it would be good to hear from them about
how long we need to keep support for old versions.

I don't think it is good idea to have multiple forks of KafkaIO in the same
repo. If we do go that route, we should fork the entire kafka directory and
rename the main class KafkaIO_Unmaintained :).

IMHO, so far, additional complexity for supporting these versions is not
that bad. Most of it is isolated to ConsumerSpEL.java & ProducerSpEL.java.
My first preference is dropping support for deprecated versions (and a
deprecate a few more versions, may be till the version that added
transactions around 0.11.x I think).

I haven't looked into what's new in Kafka 2.x. Are there any features that
KafkaIO should take advantage of? I have not noticed our existing code
breaking. We should certainly certainly support latest releases of Kafka.

Raghu.

On Tue, Apr 2, 2019 at 10:27 AM Mingmin Xu  wrote:

>
> We're still using Kafka 0.10 a lot, similar as 0.9 IMO. To expand multiple
> versions in KafkaIO is quite complex now, and it confuses users which is
> supported / which is not. I would prefer to support Kafka 2.0+ only in the
> latest version. For old versions, there're some options:
> 1). document Kafka-Beam support versions, like what we do in FlinkRunner;
> 2). maintain separated KafkaIOs for old versions;
>
> 1) would be easy to maintain, and I assume there should be no issue to use
> Beam-Core 3.0 together with KafkaIO 2.0.
>
> Any thoughts?
>
> Mingmin
>
> On Tue, Apr 2, 2019 at 9:56 AM Reuven Lax  wrote:
>
>> KafkaIO is marked as Experimental, and the comment already warns that 0.9
>> support might be removed. I think that if users still rely on Kafka 0.9 we
>> should leave a fork (renamed) of the IO in the tree for 0.9, but we can
>> definitely remove 0.9 support from the main IO if we want, especially if
>> it's complicated changes to that IO. If we do though, we should fail with a
>> clear error message telling users to use the Kafka 0.9 IO.
>>
>> On Tue, Apr 2, 2019 at 9:34 AM Alexey Romanenko 
>> wrote:
>>
>>> > How are multiple versions of Kafka supported? Are they all in one
>>> client, or is there a case for forks like ElasticSearchIO?
>>>
>>> They are supported in one client but we have additional “ConsumerSpEL”
>>> adapter which unifies interface difference among different Kafka client
>>> versions (mostly to support old ones 0.9-0.10.0).
>>>
>>> On the other hand, we warn user in Javadoc of KafkaIO (which is
>>> Unstable, btw) by the following:
>>> *“KafkaIO relies on kafka-clients for all its interactions with the
>>> Kafka cluster.**kafka-clients versions 0.10.1 and newer are supported
>>> at runtime. The older versions 0.9.x **- 0.10.0.0 are also supported,
>>> but are deprecated and likely be removed in near future.”*
>>>
>>> Despite the fact that, personally, I’d prefer to have only one unified
>>> client interface but, since people still use Beam with old Kafka instances,
>>> we, likely, should stick with it till Beam 3.0.
>>>
>>> WDYT?
>>>
>>> On 2 Apr 2019, at 02:27, Austin Bennett 
>>> wrote:
>>>
>>> FWIW --
>>>
>>> On my (desired, not explicitly job-function) roadmap is to tap into a
>>> bunch of our corporate Kafka queues to ingest that data to places I can
>>> use.  Those are 'stuck' 0.9, with no upgrade in sight (am told the upgrade
>>> path isn't trivial, is very critical flows, and they are scared for it to
>>> break, so it just sits behind firewalls, etc).  But, I wouldn't begin that
>>> for probably at least another quarter.
>>>
>>> I don't contribute to nor understand the burden of maintaining the
>>> support for the older version, so can't reasonably lobby for that continued
>>> pain.
>>>
>>> Anecdotally, this could be a place many enterprises are at (though I
>>> also wonder whether many of the people that would be 'stuck' on such
>>> versions would also have Beam on their current radar).
>>>
>>>
>>> On Mon, Apr 1, 2019 at 2:29 PM Kenneth Knowles  wrote:
>>>
 This could be a backward-incompatible change, though that notion has
 many interpretations. What matters is user pain. Technically if we don't
 break the core SDK, users should be able to use Java SDK >=2.11.0 with
 KafkaIO 2.11.0 forever.

 How are multiple versions of Kafka supported? Are they all in one
 client, or is there a case for forks like ElasticSearchIO?

 Kenn

 On Mon, Apr 1, 2019 at 10:37 AM Jean-Baptiste Onofré 
 wrote:

> +1 to remove 0.9 support.
>

Re: Some questions about ensuring correctness with windowing and triggering

2019-02-20 Thread Raghu Angadi
On Tue, Feb 12, 2019 at 10:28 AM Robert Bradshaw 
wrote:

> Correct, even within the same key there's no promise of event time
> ordering mapping of panes to real time ordering because the downstream
> operations *may* happen on a different machine. Multiply triggered
> windows add an element of non-determinism to the process.
>

For clarification, the stage immediately after GBK itself processes fired
panes in order, correct? Of course, any more stages downstream of that may
see them out of order.

Raghu.

>
> You're also correct that triggering with multiple panes requires lots of
> care, especially when it comes to operations with side effects (like
> sinks). Most safe is to only write the final pane to the sink, and handle
> early triggering in a different way.
> https://s.apache.org/beam-sink-triggers is a proposal to make this easier
> to reason about.
>
>
> On Tue, Feb 12, 2019 at 7:19 PM Steve Niemitz  wrote:
>
>> Also to clarify here (I re-read this and realized it could be slightly
>> unclear).  My question is only about in-order delivery of panes.  ie: will
>> pane P always be delivered before P+1.
>>
>> I realize the use of "in-order" before could be confusing, I don't care
>> about the ordering of the elements per-se, just the ordering of the pane
>> delivery.
>>
>> I want to make sure that given a GBK that produces 3 panes (P0, P1, P2)
>> for a key, a downstream PCollection could never see P0, P2, P1.  OR at
>> least, the final firing is always guaranteed to be delivered after all
>> early-firings (eg we could have P0, P2, P1, but then always PLast).
>>
>> On Tue, Feb 12, 2019 at 11:48 AM Steve Niemitz 
>> wrote:
>>
>>> Are you also saying also that even in the first example (Source ->
>>> CombineByKey (Sum) -> Sink) there's no guarantee that events would be
>>> delivered in-order from the Combine -> Sink transforms?  This seems like a
>>> pretty big "got-cha" for correctness if you ever use accumulating
>>> triggering.
>>>
>>> I'd also like to point out I'm not talking about a global ordering
>>> across the entire PCollection, I'm talking about within the same key after
>>> a GBK transform.
>>>
>>> On Tue, Feb 12, 2019 at 11:35 AM Robert Bradshaw 
>>> wrote:
>>>
 Due to the nature of distributed processing, order is not preserved.
 You can, however, inspect the PaneInfo to determine if an element was
 early, on-time, or late and act accordingly.

 On Tue, Feb 12, 2019 at 5:15 PM Juan Carlos Garcia 
 wrote:

> In my experience ordering is not guaranteed, you may need apply a
> transformation that sort the elements and then dispatch them sorted out.
>
> Or uses the Sorter extension for this:
>
> https://github.com/apache/beam/tree/master/sdks/java/extensions/sorter
>
> Steve Niemitz  schrieb am Di., 12. Feb. 2019,
> 16:31:
>
>> Hi everyone, I have some questions I want to ask about how windowing,
>> triggering, and panes work together, and how to ensure correctness
>> throughout a pipeline.
>>
>> Lets assume I have a very simple streaming pipeline that looks like:
>> Source -> CombineByKey (Sum) -> Sink
>>
>> Given fixed windows of 1 hour, early firings every minute, and
>> accumulating panes, this is pretty straight forward.  However, this can 
>> get
>> more complicated if we add steps after the CombineByKey, for instance
>> (using the same windowing strategy):
>>
>> Say I want to buffer the results of the CombineByKey into batches of
>> N elements.  I can do this with the built-in GroupIntoBatches [1]
>> transform, now my pipeline looks like:
>>
>> Source -> CombineByKey (Sum) -> GroupIntoBatches -> Sink
>>
>> *This leads to my main question:*
>> Is ordering preserved somehow here?  ie: is it possible that the
>> result from early firing F+1 now comes BEFORE the firing F (because it 
>> was
>> re-ordered in the GroupIntoBatches).  This would mean that the sink then
>> gets F+1 before F, which means my resulting store has incorrect data
>> (possibly forever if F+1 was the final firing).
>>
>> If ordering is not preserved, it seems as if I'd need to introduce my
>> own ordering back in after GroupIntoBatches.  GIB is an example here, 
>> but I
>> imagine this could happen with any GBK type operation.
>>
>> Am I thinking about this the correct way?  Thanks!
>>
>> [1]
>> https://beam.apache.org/releases/javadoc/2.10.0/org/apache/beam/sdk/transforms/GroupIntoBatches.html
>>
>


Re: kafkaIO Consumer Rebalance with Spark Runner

2019-01-25 Thread Raghu Angadi
You have 32 partitions. Reading can not be distributed to more than 32
parallel tasks.
If you have a log of processing for each record after reading, you can
reshuffle the messages before processing them, that way the processing
could be distributed to more tasks. Search for previous threads about
reshuffle in Beam lists.

On Thu, Jan 24, 2019 at 7:23 PM  wrote:

> Dear all,
>
>
>
> I am using the kafkaIO sdk in my project (Beam with Spark runner).
>
>
>
> The problem about task skew is shown as the following figure.
>
>
>
> My running environment is:
>
> OS: Ubuntn 14.04.4 LTS
>
> The version of related tools is:
>
> java version: "1.8.0_151"
>
> Beam version: 2.9.0 (Spark runner with Standalone mode)
>
> Spark version: 2.3.1 Standalone mode
>
> Execution condition:
>
> Master/Driver node: ubuntu7
>
> Worker nodes: ubuntu8 (4 Executors); ubuntu9 (4 Executors)
>
> The number of executors is 8
>
>
>
> Kafka Broker: 2.10-0.10.1.1
>
> Broker node at ubuntu7
>
> Kafka Client:
>
> The topic: kafkasink32
>
> kafkasink32 Partitions: 32
>
>
>
> The programming of my project for kafkaIO SDK is as:
>
>
> ==
>
> Map map = ImmutableMap.*builder*()
>
>.put("group.id", (Object)"test-consumer-group")
>
>.build();
>
> List topicPartitions = *new** ArrayList()*;
>
>*for*(*int* i = 0; i < 32; i++) {
>
>  topicPartitions.add(*new* TopicPartition(
> "kafkasink32",i));
>
> }
>
> PCollection> readKafkaData = p.apply(KafkaIO. String>*read*()
>
>  .withBootstrapServers("ubuntu7:9092")
>
>.updateConsumerProperties(map)
>
>.withKeyDeserializer(LongDeserializer.*class*)
>
>.withValueDeserializer(StringDeserializer.*class*)
>
>.withTopicPartitions(topicPartitions)
>
>.withoutMetadata()
>
>);
>
>
> ==
>
> Here I have two directions to solve this problem:
>
>
>
> 1.  Using the following sdk from spark streaming
>
>
> https://jaceklaskowski.gitbooks.io/spark-streaming/spark-streaming-kafka-LocationStrategy.html
>
> LocationStrategies.PreferConsistent: Use in most cases as it consistently
> distributes partitions across all executors.
>
>
>
> If we would like to use this feature, we have not idea to set this in
> kafkaIO SDK.
>
>
>
> 2.  Setting the related configurations of kafka to perform the
> consumer rebalance
>
> set consumer group? Set group.id?
>
>
>
> If we need to do No2., could someone give me some ideas to set
> configurations?
>
>
>
> If anyone provides any direction to help us to overcome this problem, we
> would appreciate it.
>
>
>
> Thanks.
>
>
>
> Sincerely yours,
>
>
>
> Rick
>
>
>
>
> --
> 本信件可能包含工研院機密資訊,非指定之收件者,請勿使用或揭露本信件內容,並請銷毀此信件。 This email may contain
> confidential information. Please do not use or disclose it in any way and
> delete it if you are not the intended recipient.
>


Re: KafkaIO not commiting offsets when using withMaxReadTime

2019-01-18 Thread Raghu Angadi
On Thu, Jan 10, 2019 at 7:57 AM Alexey Romanenko 
wrote:

> Don’t you think that we could have some race condition there since,
> according to initial issue description, sometimes offset was committed and
> sometimes not?
>

Yeah, there is a timing issue. 'finalizeCheckpoint()' does not wait until
checkpoint is committed by the IO thread. See comment at
https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L613

This is best suited for unbounded case, since we don't want to block on
each call to finalize checkpoint. There could be lots of these calls per
second in a streaming pipeline and we only need to commit the latest
checkpoint. But that does not work well when this is used in bounded reader
context.

Fix: KafkaIO could store a flag that it is being read by a bounded wrapper
(see expand() where the bounded wrapper is added). When this flag is set it
could wake up the IO thread and wait for offsets to be committed.





>
>
> On 9 Jan 2019, at 19:48, Raghu Angadi  wrote:
>
> Oh, the generic bounded source wrapper over an unbounded source does not
> seem to call finalize when it is done with a split. I think it should.
>
> Could you file a bug for the wrapper?
> Mean while, this check could be added sanity checks in
> KafkaIO.Read.expand().
>
>
>
> On Wed, Jan 9, 2019 at 10:37 AM André Missaglia <
> andre.missag...@arquivei.com.br> wrote:
>
>> Hi Juan,
>>
>> After researching a bit, I found this issue, which is open since 2017:
>> https://issues.apache.org/jira/browse/BEAM-2185
>>
>> I guess KafkaIO isn't intended to provide a bounded source. Maybe I
>> should write my own code that fetches messages from kafka, even if it means
>> giving up on some processing guarantees from beam...
>>
>>
>> Em qua, 9 de jan de 2019 às 14:24, Juan Carlos Garcia <
>> jcgarc...@gmail.com> escreveu:
>>
>>> Just for you to have a look where this happen:
>>>
>>>
>>> https://github.com/apache/beam/blob/dffe2c1a2bd95f78869b266d3e1ea3f8ad8c323d/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L584
>>>
>>> Cheers
>>>
>>> On Wed, Jan 9, 2019 at 5:09 PM Juan Carlos Garcia 
>>> wrote:
>>>
>>>> I also experience the same, as per the documentation **withMaxReadTime**
>>>> and **withMaxNumRecords** are mainly used for Demo purposes, so i
>>>> guess is beyond the scope of the current KafkaIO to behave as Bounded with
>>>> offset management or just something is missing in the current
>>>> implementation (Watermarking).
>>>>
>>>>
>>>>
>>>> On Wed, Jan 9, 2019 at 2:28 PM André Missaglia <
>>>> andre.missag...@arquivei.com.br> wrote:
>>>>
>>>>> Hello everyone,
>>>>>
>>>>> I need to do some batch processing that uses messages in a Kafka
>>>>> topic. So I tried the "withMaxReadTime" KafkaIO setting:
>>>>>
>>>>> ---
>>>>> val properties = new Properties()
>>>>> properties.setProperty("bootstrap.servers", "...")
>>>>> properties.setProperty("group.id", "mygroup")
>>>>> properties.setProperty("sasl.jaas.config", "...")
>>>>> properties.setProperty("security.protocol", "SASL_PLAINTEXT")
>>>>> properties.setProperty("sasl.mechanism", "SCRAM-SHA-256")
>>>>> properties.setProperty("enable.auto.commit", "false")
>>>>>
>>>>> sc.customInput("Read From Kafka",
>>>>>   KafkaIO
>>>>> .read[String, String]()
>>>>> .withTopic("mytopic")
>>>>> .withKeyDeserializer(classOf[StringDeserializer])
>>>>> .withValueDeserializer(classOf[StringDeserializer])
>>>>> .updateConsumerProperties(properties)
>>>>> .withMaxReadTime(Duration.standardSeconds(20))
>>>>> .withMaxNumRecords(100)
>>>>> .commitOffsetsInFinalize()
>>>>> .withoutMetadata()
>>>>> )
>>>>> .count.debug() // prints something between 1 and 2
>>>>> ---
>>>>> I can see that it was able to read the messages and process them. But
>>>>> in the end, no offset was commited:
>>>>>
>>>>> TOPICPARTITION
>>>>

Re: KafkaIO not commiting offsets when using withMaxReadTime

2019-01-09 Thread Raghu Angadi
Oh, the generic bounded source wrapper over an unbounded source does not
seem to call finalize when it is done with a split. I think it should.

Could you file a bug for the wrapper?
Mean while, this check could be added sanity checks in
KafkaIO.Read.expand().



On Wed, Jan 9, 2019 at 10:37 AM André Missaglia <
andre.missag...@arquivei.com.br> wrote:

> Hi Juan,
>
> After researching a bit, I found this issue, which is open since 2017:
> https://issues.apache.org/jira/browse/BEAM-2185
>
> I guess KafkaIO isn't intended to provide a bounded source. Maybe I should
> write my own code that fetches messages from kafka, even if it means giving
> up on some processing guarantees from beam...
>
>
> Em qua, 9 de jan de 2019 às 14:24, Juan Carlos Garcia 
> escreveu:
>
>> Just for you to have a look where this happen:
>>
>>
>> https://github.com/apache/beam/blob/dffe2c1a2bd95f78869b266d3e1ea3f8ad8c323d/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L584
>>
>> Cheers
>>
>> On Wed, Jan 9, 2019 at 5:09 PM Juan Carlos Garcia 
>> wrote:
>>
>>> I also experience the same, as per the documentation **withMaxReadTime**
>>> and **withMaxNumRecords** are mainly used for Demo purposes, so i guess
>>> is beyond the scope of the current KafkaIO to behave as Bounded with offset
>>> management or just something is missing in the current implementation
>>> (Watermarking).
>>>
>>>
>>>
>>> On Wed, Jan 9, 2019 at 2:28 PM André Missaglia <
>>> andre.missag...@arquivei.com.br> wrote:
>>>
 Hello everyone,

 I need to do some batch processing that uses messages in a Kafka topic.
 So I tried the "withMaxReadTime" KafkaIO setting:

 ---
 val properties = new Properties()
 properties.setProperty("bootstrap.servers", "...")
 properties.setProperty("group.id", "mygroup")
 properties.setProperty("sasl.jaas.config", "...")
 properties.setProperty("security.protocol", "SASL_PLAINTEXT")
 properties.setProperty("sasl.mechanism", "SCRAM-SHA-256")
 properties.setProperty("enable.auto.commit", "false")

 sc.customInput("Read From Kafka",
   KafkaIO
 .read[String, String]()
 .withTopic("mytopic")
 .withKeyDeserializer(classOf[StringDeserializer])
 .withValueDeserializer(classOf[StringDeserializer])
 .updateConsumerProperties(properties)
 .withMaxReadTime(Duration.standardSeconds(20))
 .withMaxNumRecords(100)
 .commitOffsetsInFinalize()
 .withoutMetadata()
 )
 .count.debug() // prints something between 1 and 2
 ---
 I can see that it was able to read the messages and process them. But
 in the end, no offset was commited:

 TOPICPARTITION  CURRENT-OFFSET
 LOG-END-OFFSET  LAG CONSUMER-ID HOSTCLIENT-ID
 mytopic 0
 0   3094751 3094751 -
 -   -

 But it is a strange behavior: sometimes it commits the offset,
 sometimes not. I'm not sure if it is a bug, or I'm using the wrong configs.

 Has anyone used Bounded KafkaIO before? is there anything I can do?

 Best Regards,

 --
 *André Badawi Missaglia*
 Data Engineer
 (16) 3509-5515 *|* www.arquivei.com.br
 
 [image: Arquivei.com.br – Inteligência em Notas Fiscais]
 
 [image: Google seleciona Arquivei para imersão e mentoria no Vale do
 Silício]
 
 
 
 

>>>
>>>
>>> --
>>>
>>> JC
>>>
>>>
>>
>> --
>>
>> JC
>>
>>
>
> --
> *André Badawi Missaglia*
> Data Engineer
> (16) 3509-5515 *|* www.arquivei.com.br
> 
> [image: Arquivei.com.br – Inteligência em Notas Fiscais]
> 
> [image: Google seleciona Arquivei para imersão e mentoria no Vale do
> Silício]
> 
> 
> 
> 
>


Re: Kafka LogAppendTimePolicy not throwing Exception when log.message.timestamp.type = CreateTime for the Kafka topic

2019-01-04 Thread Raghu Angadi
On Fri, Jan 4, 2019 at 12:54 PM rahul patwari 
wrote:

> Hi Raghu,
> Thanks for the response.
>
> I used withTopics() and withCreateTime() to read records from multiple
> topics, "topic1" with message.timestamp.type=CreateTime and "topic2"
> with message.timestamp.type=LogAppendTime.
> And I got the Exception: java.lang.IllegalArgumentException: Kafka
> record's timestamp is not 'CREATE_TIME' (topic: topic2, partition 0, offset
> 0, timestamp type 'LogAppendTime').
>

If you are reading 'withCreateTime()', all the records from all the topics
are expected to have CREATE_TIME timestamp. I don't think it makes much
logical sense for built in timestamp factories to support mixed special
cases like this. Couple of options for you depending on your exact use case:

   - Read topic1 and topic2 as two separate PCollections (i.e. two
   instances of KafkaIO.read()).
   - Or implement a timestamp factory that handles the mixed case like you
   prefer (as you described below). It is pretty simple to implement them. You
   can start with one of the built in factories.


> So, my understanding is that, only when withLogAppendTime() is used, we
> don't want users to get stuck if some topics have multiple timestamp types.
> In this case, where two topics have two different timestamp types, the
> watermark will be calculated only based on the records which belong to the
> topic with timestamp as LogAppendTime. I am thinking that we can calculate
> the watermark more accurately if we also consider the records in the topic
> with timestamp type CreateTime. So, instead of directly returning
> currentWatermark, we can update currentWatermark with the record's
> timestamp(either CreateTime (or) LogAppendTime) always and then return
> currentWatermark.
>
> I would like to contribute if a fix is needed.
>
> Regards,
> Rahul
>
> On Sat, Jan 5, 2019 at 12:40 AM Raghu Angadi  wrote:
>
>> The intention was to assert on 'timestamp_type' on the first record only.
>> I was not entirely sure if there are situations in Kafka where a timestamp
>> type could be different or timestamp itself could be missing for some
>> records. The assertion on the first record was just to sanity check common
>> misconfiguration. The way this policy checked for first record itself is
>> incorrect in the case of idle partitions since the watermark advances even
>> with out any records read.. this is the issue you encountered. When the
>> timestamp type does not match, it's timestamp is not used to watermark.
>>
>> As you suggested, simpler fix might just be require every record's
>> timestamp_type to be LOG_APPEND_TIME (i.e. replace 'else if' with 'else').
>> Is that safe? We don't want users to get stuck if some topics are expected
>> to have multiple timestamp types.
>>
>> Raghu.
>> On Thu, Jan 3, 2019 at 11:22 PM rahul patwari 
>> wrote:
>>
>>> Hi,
>>>
>>> We are using KafkaIO.read() with LogAppendTimePolicy. When the topic is
>>> idle at the beginning of the pipeline, IllegalStateException is  NOT thrown
>>> even when log.message.timestamp.type = CreateTime.
>>>
>>> This happens due to the statement:
>>> else if (currentWatermark.equals(BoundedWindow.TIMESTAMP_MIN_VALUE)) in
>>> getTimestampForRecord() method in TimestampPolicyFactory Interface.
>>>
>>> As the topic is idle at the beginning of the pipeline, the
>>> currentWatermark is advanced (backlog==0), because of which
>>> currentWatermark.equals(BoundedWindow.TIMESTAMP_MIN_VALUE) is False and
>>> the timestamp of the records are taken as currentWatermark.
>>>
>>> If we change else if() to else, IllegalStateException is thrown when the
>>> first record from the Kafka topic is read, which is expected.
>>> Is there any specific reason behind using else if() instead of else?
>>>
>>> Thanks and Regards,
>>> Rahul
>>>
>>>


Re: Kafka LogAppendTimePolicy not throwing Exception when log.message.timestamp.type = CreateTime for the Kafka topic

2019-01-04 Thread Raghu Angadi
The intention was to assert on 'timestamp_type' on the first record only. I
was not entirely sure if there are situations in Kafka where a timestamp
type could be different or timestamp itself could be missing for some
records. The assertion on the first record was just to sanity check common
misconfiguration. The way this policy checked for first record itself is
incorrect in the case of idle partitions since the watermark advances even
with out any records read.. this is the issue you encountered. When the
timestamp type does not match, it's timestamp is not used to watermark.

As you suggested, simpler fix might just be require every record's
timestamp_type to be LOG_APPEND_TIME (i.e. replace 'else if' with 'else').
Is that safe? We don't want users to get stuck if some topics are expected
to have multiple timestamp types.

Raghu.
On Thu, Jan 3, 2019 at 11:22 PM rahul patwari 
wrote:

> Hi,
>
> We are using KafkaIO.read() with LogAppendTimePolicy. When the topic is
> idle at the beginning of the pipeline, IllegalStateException is  NOT thrown
> even when log.message.timestamp.type = CreateTime.
>
> This happens due to the statement:
> else if (currentWatermark.equals(BoundedWindow.TIMESTAMP_MIN_VALUE)) in
> getTimestampForRecord() method in TimestampPolicyFactory Interface.
>
> As the topic is idle at the beginning of the pipeline, the
> currentWatermark is advanced (backlog==0), because of which
> currentWatermark.equals(BoundedWindow.TIMESTAMP_MIN_VALUE) is False and
> the timestamp of the records are taken as currentWatermark.
>
> If we change else if() to else, IllegalStateException is thrown when the
> first record from the Kafka topic is read, which is expected.
> Is there any specific reason behind using else if() instead of else?
>
> Thanks and Regards,
> Rahul
>
>


Re: KafkaIO and added partitions

2019-01-02 Thread Raghu Angadi
+1, we should do it.
The implementation could be something on these line:

   - While assigning Kafka partitions to each source split during the first
   run, assign them deterministically.
  - Current round-robin assignment works fine for single topic. But is
  not deterministic while reading from more than one topic. We
need to tweak
  the assignment to work well in that case.
   - On the worker, each reader should check the partitions for input topic
   (this can be part of existing periodic threads that checks backlog)
   - When partitions are added:
  - The readers (source splits) that new partitions belong to will
  start consuming from it. This is straight forward.
  - What if the new partition's watermark is older the current
  watermark? Can't do much about it since a watermark can not go back.
   - When the partitions are deleted:
  - This is a bit more tricky.
  - We need to handle the case a source split might not have any
  partitions assigned.
 - What should the watermark be? I think current wall time makes
 sense. Note that there could be new partitions added later.


On Wed, Jan 2, 2019 at 7:59 AM Alexey Romanenko 
wrote:

> I just wanted to mention that there is quite old open issue about that:
> https://issues.apache.org/jira/browse/BEAM-727
>
>  Fell free to take this one if anyone is interested.
>
> On 2 Jan 2019, at 15:22, Juan Carlos Garcia  wrote:
>
> +1
>
> Am Mi., 2. Jan. 2019, 14:34 hat Abdul Qadeer 
> geschrieben:
>
>> +1
>>
>> On Tue, 1 Jan 2019 at 12:45,  wrote:
>>
>>> +1 from my side too :-)
>>> And ideally I would want to have some hooks to let me know the extra
>>> partitions have been picked up (or a way to query it).
>>>
>>> Although if that can't be provided I can work around it myself by
>>> sending some specific message to the partition that somewhere results in a
>>> visible state change in the pipeline.
>>>
>>> Also, as a quick (semi related) heads up: I will very likely soon
>>> contribute a change to the LogAppendTimePolicy so that the idle partition
>>> behavior (automatic watermark generation) can be disabled.
>>>
>>> (of course all related to my streamy-db project)
>>>
>>> Kind regards,
>>> Jan
>>>
>>>
>>> On Tue, 1 Jan 2019 at 08:19, Ramesh Nethi 
>>> wrote:
>>>
>>>> +1 for this capability.  This would enable pipelines to continue to run
>>>> when such changes need to be made.
>>>>
>>>> regards
>>>> Ramesh
>>>>
>>>> On Fri, 23 Nov 2018 at 00:40 Raghu Angadi  wrote:
>>>>
>>>>> On Thu, Nov 22, 2018 at 10:10 AM Raghu Angadi 
>>>>> wrote:
>>>>>
>>>>>> - New partitions will be ignored during runtime.
>>>>>> - Update will not succeed either. Error message on the workers should
>>>>>> explain the mismatch.
>>>>>>
>>>>>
>>>>> This is the current state. Supporting changes to number of partition
>>>>> is quite doable if there is enough user interested (even in the current
>>>>> UnnoundedSource API framework).
>>>>>
>>>>>>
>>>>>> On Thu, Nov 22, 2018 at 2:15 AM Jozef Vilcek 
>>>>>> wrote:
>>>>>>
>>>>>>> Hello,
>>>>>>> just wanted to check how does Beam KafkaIO behaves when partitions
>>>>>>> are added to the topic.
>>>>>>> Will they be picked up or ignored during the runtime?
>>>>>>> Will they be picked up on restart with state restore?
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Jozef
>>>>>>>
>>>>>>
>


Re: [Dataflow] Delaying PubSubIO acks until FileIO completes writes

2018-12-06 Thread Raghu Angadi
On Thu, Dec 6, 2018 at 6:43 AM Jeff Klukas  wrote:

> Thanks for the response, Raghu. Does this sound like a use case that would
> be worth supporting in the future, or does that fall outside of
> Beam/Dataflow's goals?
>

Beam does not hinder runners from providing these. Beam itself has stayed
above state checkpointing and associated guarantees so far, letting runners
define those semantics. What Beam should do especially in lieu of being a
widely applicable data processing API will be an important aspect. I am
sure all the proposals are welcome ('DoFn.RequiresStableInput' is an
example). Often, one of the tricky issues at Beam is how it affects batch,
and in streaming, how it affects orthogonal checkpointing semantics in
runners like Flink and runners like Dataflow & Spark.

Coming to one of its runners Dataflow, yes, it should certainly aim to make
these use cases easier to do. Currently it has 'update' (gives same
exactly-once guarantees across a job restart as within in a single job) &
'drain' (supports graceful shutdown). While these two address a large range
of consistency requirements for customers, you want something in
between:  ability to save the state and be able to restart from that. Flink
has support for this, I think Dataflow will support it too.


> Is there a general design principle of optimizing to make complex
> pipelines possible in favor of allowing detailed control for a simple
> pipeline like this?
>

'save points' / snapshots has been the generic way of supporting this. Note
that resuming from a snapshot has the same caveats as an 'update'
(compatibility of coders for serialized state etc). Another approach is
what Kstreams does, where the streaming engines state, and input and output
are all Kafka topics that can be atomically checkpointed. Beam needs to
support wide range of input sources


> In effect, would you recommend we write a custom application using the
> Pub/Sub and Storage SDKs directly rather than trying to use Beam's
> abstractions?
>

Certainly not :). These are important use cases and we love users to raise
these, like you did. I think the current approach taken by Beam is to let
the runners define consistency semantics.

Raghu.


> On Wed, Dec 5, 2018 at 2:24 PM Raghu Angadi  wrote:
>
>>
>> On Wed, Dec 5, 2018 at 11:13 AM Jeff Klukas  wrote:
>>
>>> We are attempting to build a Beam pipeline running on Dataflow that
>>> reads from a Cloud Pub/Sub subscription and writes 10 minute windows to
>>> files on GCS via FileIO. At-least-once completeness is critical in our use
>>> case, so we are seeking the simplest possible solution with obvious and
>>> verifiable delivery guarantees.
>>>
>>> We had assumed we'd be able to avoid intermediate storage, but it
>>> appears that it's necessary to specify sharding for FileIO when reading
>>> from Pub/Sub which implicitly calls GroupByKey [1] and that PubsubIO will
>>> ack messages at the first GroupByKey operation [0]. Thus, we become
>>> vulnerable to data loss scenarios if the Dataflow service becomes
>>> unavailable or if we improperly drain a pipeline on deletion or update.
>>>
>>> Is this assessment correct?
>>>
>> Yes.
>>
>>
>>> Would it be possible to delay pubsub acks until after the FileIO writes
>>> complete?
>>>
>>
>> There is no option to delay the acks for downstream stages.
>>
>>
>>> Other workarounds we could use to avoid Dataflow checkpoint data
>>> becoming critical state for the completeness of our pipeline?
>>>
>>
>> I don't see a good way not to depend on Dataflow checkpoint for
>> guarantees on a Dataflow pipeline. Others might have ideas here. If you are
>> willing to use external storage (or another pubsub topic) you could achieve
>> at least once. In that case, check other threads on the mailing list about
>> Wait() transform that lets you take an action after certain operation is
>> done (in this case FileIO).
>>
>>
>>> [0]
>>> https://beam.apache.org/releases/javadoc/2.8.0/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.html
>>> [1]
>>> https://beam.apache.org/releases/javadoc/2.8.0/org/apache/beam/sdk/io/FileIO.html
>>>
>>


Re: [Dataflow] Delaying PubSubIO acks until FileIO completes writes

2018-12-05 Thread Raghu Angadi
On Wed, Dec 5, 2018 at 11:13 AM Jeff Klukas  wrote:

> We are attempting to build a Beam pipeline running on Dataflow that reads
> from a Cloud Pub/Sub subscription and writes 10 minute windows to files on
> GCS via FileIO. At-least-once completeness is critical in our use case, so
> we are seeking the simplest possible solution with obvious and verifiable
> delivery guarantees.
>
> We had assumed we'd be able to avoid intermediate storage, but it appears
> that it's necessary to specify sharding for FileIO when reading from
> Pub/Sub which implicitly calls GroupByKey [1] and that PubsubIO will ack
> messages at the first GroupByKey operation [0]. Thus, we become vulnerable
> to data loss scenarios if the Dataflow service becomes unavailable or if we
> improperly drain a pipeline on deletion or update.
>
> Is this assessment correct?
>
Yes.


> Would it be possible to delay pubsub acks until after the FileIO writes
> complete?
>

There is no option to delay the acks for downstream stages.


> Other workarounds we could use to avoid Dataflow checkpoint data becoming
> critical state for the completeness of our pipeline?
>

I don't see a good way not to depend on Dataflow checkpoint for guarantees
on a Dataflow pipeline. Others might have ideas here. If you are willing to
use external storage (or another pubsub topic) you could achieve at least
once. In that case, check other threads on the mailing list about Wait()
transform that lets you take an action after certain operation is done (in
this case FileIO).


> [0]
> https://beam.apache.org/releases/javadoc/2.8.0/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.html
> [1]
> https://beam.apache.org/releases/javadoc/2.8.0/org/apache/beam/sdk/io/FileIO.html
>


Re: KafkaIO and added partitions

2018-11-22 Thread Raghu Angadi
On Thu, Nov 22, 2018 at 10:10 AM Raghu Angadi  wrote:

> - New partitions will be ignored during runtime.
> - Update will not succeed either. Error message on the workers should
> explain the mismatch.
>

This is the current state. Supporting changes to number of partition is
quite doable if there is enough user interested (even in the current
UnnoundedSource API framework).

>
> On Thu, Nov 22, 2018 at 2:15 AM Jozef Vilcek 
> wrote:
>
>> Hello,
>> just wanted to check how does Beam KafkaIO behaves when partitions are
>> added to the topic.
>> Will they be picked up or ignored during the runtime?
>> Will they be picked up on restart with state restore?
>>
>> Thanks,
>> Jozef
>>
>


Re: Experience with KafkaIO -> BigQueryIO

2018-11-08 Thread Raghu Angadi
>
>
>> That is fine. We can ignore the timestamp as possible suspect for
>> debugging this. Using custom timestamps from records is normal.
>>
>>
> Could you clarify of what you meant with "withTimestampFn() was never
> meant to
> be public"? I am using it to attach the right timestamp to an element to be
> able to window into days with the local time zone in the windowing
> function. If
> this is not used in the correct way could you tell me how I can do it
> better?
>

The problem with watermark part of the policy. A source needs to provide
both a timestamp for a record as well as a watermark for the stream. A
TimestampPolicy in KafkaIO is responsible for both of these for each Kafka
partition.

`withTimestampFn()` lets user provide a function to extract timestamp. But
for watermark, it just uses most recent record's timestamp. Say record A
has timestamp 9:00:01 and arrives at 9:00:05, and B has a timestamp of
8:59:58 and arrives at 9:00:15.
That implies once is A is processed at 9:00:05, your pipelines watermark
could jump to 9:00:01, that implies a hourly window for [8:00, 9:00) will
close. When B arrives 10 seconds later, it would be considered late. The
problem is not just that such watermark policy is too brittle, it is the
fact that users have no idea that is happening.

Deprecation documentation for this API[1] suggests using
`CustomTimestampPolicyWithLimitedDelay()` [2] in stead.

[1]:
https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/TimestampPolicyFactory.java#L100
[2]:
https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/CustomTimestampPolicyWithLimitedDelay.java#L29

Raghu.

After the rollback I am busy making the existing pipeline to GCS so robust
> that
> it never fails to deliver all files so that there is always a backup. As I
> am
> under a lot of pressure right now I don't want to fuck it up with
> easy-to-avoid
> mistakes and the GCS pipeline has the same logic, but just a different sink
> that uses a FileIO to write out different days to different "folders".
>
> Thank you,
>
> Tobi
>
>
>
>> Raghu.
>>
>>
>>> I could also not fiddle with the timestamp at all and let the system
>>> decide and
>>> then in the BigQuery.IO partitioning step parse it and assign it to a
>>> partition. Is this better?
>>>
>>>
>>>
 On Tue, Nov 6, 2018 at 3:44 AM Kaymak, Tobias 
 wrote:

> Hi,
>
> I am sharing my experience with you after trying to use the following
> pipeline
> logic (with Beam 2.6.0 - running on Flink 1.5):
>
> 1. Reading from KafkaIO, attaching a timestamp from each parsed element
> 2. Filtering bad records
> 3. Writing to a partitioned table in BigQuery with FILE_LOADS (batch
> jobs)
> every 15 minutes
>
> I had a working pipeline that does not write to BigQuery directly, but
> to
> Cloud Storage, so it's 3rd step was
>
> 3. Writing files to GCS in daily "subdirectories"
>
> I tried to rewrite the pipeline to reduce complexity: Resetting it's
> state
> should no longer be tied to thinking about what to delete on GCS, also
> configurable refresh times directly from within the Pipeline was
> something I
> was looking for. The thing that I needed to change was the output in
> the end,
> so knew my parsing logic would not change and that should reduce the
> risk.
>
> I tested the pipeline within our testcluster and it looked promising.
> When I
> deployed it last week everything seemed to go smoothly. On Friday I
> noticed
> that I had holes in the data: in the BigQuery tables there were
> missing days
> (tricky was that the recent days looked fine). (To be sure I reset the
> pipeline
> and read from the very beginning of each topic from Kafka. Within
> different
> runs, different days were missing.) I spent the weekend rolling back
> the
> changes and trying to figure out what was going on.
>
> I didn't see any error in the logs (the log level was on WARNING for
> most
> parts), but I thought, well maybe it's because there are too many
> partitions
> and BigQuery has a limit of 1000 partition operations per day. So I
> started
> reading from just 90 days in the past, but I still had holes (whole
> days).
>
> I had a windowing step that I needed for the GCS pipeline, I became
> aware that I
> wouldn't need this anymore with BigQueryIO so I commented it out and
> tested
> again, without luck.
>
> What struck me was that the Flink Cluster didn't do any checkpoints
> for the
> pipeline that was using BigQueryIO - it does so when writing to GCS
> and I
> tested it's failure logic there. Additionally the graph in Flink with
> BigQueryIO becomes very complex, but this is something I expected.
>
> Here is the Pipeline code with 

Re: Experience with KafkaIO -> BigQueryIO

2018-11-07 Thread Raghu Angadi
On Wed, Nov 7, 2018 at 5:04 AM Kaymak, Tobias 
wrote:

>
> On Tue, Nov 6, 2018 at 6:58 PM Raghu Angadi  wrote:
>
>> You seem to be reading from multiple topics and your timestamp policy is
>> too simplistic (withTimestampFn() was never meant to be public API, I am
>> sending a PR to deprecate it first and then will make it package private).
>> So if you have two topics of different sizes, smaller topic might be
>> consumed really fast pushing your watermark way ahead. This might or might
>> be happening, but this is one of the dangers of using record timestamp for
>> watermark (we should never do that).
>>
>>
> To clarify: Test was done consuming from a single topic. I am using a field
> inside the element's JSON to get the element's timestamp. Data in a topic
> can
> go way back to let's say 2017, but that data was pushed to Kafka in one go
> and
> the timestamp when it arrived is for example wednesday last week.
> Sometimes the
> producing side does not set the element's timestamp for Kafka (since it's
> using
> a library that does not support that yet), so it has to be parsed.
>

That is fine. We can ignore the timestamp as possible suspect for debugging
this. Using custom timestamps from records is normal.

Raghu.


> I could also not fiddle with the timestamp at all and let the system
> decide and
> then in the BigQuery.IO partitioning step parse it and assign it to a
> partition. Is this better?
>
>
>
>> On Tue, Nov 6, 2018 at 3:44 AM Kaymak, Tobias 
>> wrote:
>>
>>> Hi,
>>>
>>> I am sharing my experience with you after trying to use the following
>>> pipeline
>>> logic (with Beam 2.6.0 - running on Flink 1.5):
>>>
>>> 1. Reading from KafkaIO, attaching a timestamp from each parsed element
>>> 2. Filtering bad records
>>> 3. Writing to a partitioned table in BigQuery with FILE_LOADS (batch
>>> jobs)
>>> every 15 minutes
>>>
>>> I had a working pipeline that does not write to BigQuery directly, but
>>> to
>>> Cloud Storage, so it's 3rd step was
>>>
>>> 3. Writing files to GCS in daily "subdirectories"
>>>
>>> I tried to rewrite the pipeline to reduce complexity: Resetting it's
>>> state
>>> should no longer be tied to thinking about what to delete on GCS, also
>>> configurable refresh times directly from within the Pipeline was
>>> something I
>>> was looking for. The thing that I needed to change was the output in the
>>> end,
>>> so knew my parsing logic would not change and that should reduce the
>>> risk.
>>>
>>> I tested the pipeline within our testcluster and it looked promising.
>>> When I
>>> deployed it last week everything seemed to go smoothly. On Friday I
>>> noticed
>>> that I had holes in the data: in the BigQuery tables there were missing
>>> days
>>> (tricky was that the recent days looked fine). (To be sure I reset the
>>> pipeline
>>> and read from the very beginning of each topic from Kafka. Within
>>> different
>>> runs, different days were missing.) I spent the weekend rolling back the
>>> changes and trying to figure out what was going on.
>>>
>>> I didn't see any error in the logs (the log level was on WARNING for most
>>> parts), but I thought, well maybe it's because there are too many
>>> partitions
>>> and BigQuery has a limit of 1000 partition operations per day. So I
>>> started
>>> reading from just 90 days in the past, but I still had holes (whole
>>> days).
>>>
>>> I had a windowing step that I needed for the GCS pipeline, I became
>>> aware that I
>>> wouldn't need this anymore with BigQueryIO so I commented it out and
>>> tested
>>> again, without luck.
>>>
>>> What struck me was that the Flink Cluster didn't do any checkpoints for
>>> the
>>> pipeline that was using BigQueryIO - it does so when writing to GCS and I
>>> tested it's failure logic there. Additionally the graph in Flink with
>>> BigQueryIO becomes very complex, but this is something I expected.
>>>
>>> Here is the Pipeline code with the commented out windowing part:
>>>
>>>   pipeline
>>> .apply(
>>> KafkaIO.read()
>>> .withBootstrapServers(bootstrap)
>>> .withTopics(topics)
>>> .withKeyDeserializer(StringDeserializer.class)
>>> .withValueDe

Re: Experience with KafkaIO -> BigQueryIO

2018-11-06 Thread Raghu Angadi
You seem to be reading from multiple topics and your timestamp policy is
too simplistic (withTimestampFn() was never meant to be public API, I am
sending a PR to deprecate it first and then will make it package private).
So if you have two topics of different sizes, smaller topic might be
consumed really fast pushing your watermark way ahead. This might or might
be happening, but this is one of the dangers of using record timestamp for
watermark (we should never do that).

On Tue, Nov 6, 2018 at 3:44 AM Kaymak, Tobias 
wrote:

> Hi,
>
> I am sharing my experience with you after trying to use the following
> pipeline
> logic (with Beam 2.6.0 - running on Flink 1.5):
>
> 1. Reading from KafkaIO, attaching a timestamp from each parsed element
> 2. Filtering bad records
> 3. Writing to a partitioned table in BigQuery with FILE_LOADS (batch jobs)
> every 15 minutes
>
> I had a working pipeline that does not write to BigQuery directly, but to
> Cloud Storage, so it's 3rd step was
>
> 3. Writing files to GCS in daily "subdirectories"
>
> I tried to rewrite the pipeline to reduce complexity: Resetting it's state
> should no longer be tied to thinking about what to delete on GCS, also
> configurable refresh times directly from within the Pipeline was something
> I
> was looking for. The thing that I needed to change was the output in the
> end,
> so knew my parsing logic would not change and that should reduce the risk.
>
> I tested the pipeline within our testcluster and it looked promising. When
> I
> deployed it last week everything seemed to go smoothly. On Friday I noticed
> that I had holes in the data: in the BigQuery tables there were missing
> days
> (tricky was that the recent days looked fine). (To be sure I reset the
> pipeline
> and read from the very beginning of each topic from Kafka. Within different
> runs, different days were missing.) I spent the weekend rolling back the
> changes and trying to figure out what was going on.
>
> I didn't see any error in the logs (the log level was on WARNING for most
> parts), but I thought, well maybe it's because there are too many
> partitions
> and BigQuery has a limit of 1000 partition operations per day. So I started
> reading from just 90 days in the past, but I still had holes (whole days).
>
> I had a windowing step that I needed for the GCS pipeline, I became aware
> that I
> wouldn't need this anymore with BigQueryIO so I commented it out and
> tested
> again, without luck.
>
> What struck me was that the Flink Cluster didn't do any checkpoints for the
> pipeline that was using BigQueryIO - it does so when writing to GCS and I
> tested it's failure logic there. Additionally the graph in Flink with
> BigQueryIO becomes very complex, but this is something I expected.
>
> Here is the Pipeline code with the commented out windowing part:
>
>   pipeline
> .apply(
> KafkaIO.read()
> .withBootstrapServers(bootstrap)
> .withTopics(topics)
> .withKeyDeserializer(StringDeserializer.class)
> .withValueDeserializer(ConfigurableDeserializer.class)
> .updateConsumerProperties(
>
> ImmutableMap.of(InputMessagesConfig.CONFIG_PROPERTY_NAME,
> inputMessagesConfig))
>
> .updateConsumerProperties(ImmutableMap.of("auto.offset.reset", "earliest"))
> .updateConsumerProperties(ImmutableMap.of("group.id",
> "di-beam-consumers"))
>
> .updateConsumerProperties(ImmutableMap.of("enable.auto.commit", "true"))
> .withTimestampPolicyFactory(
> TimestampPolicyFactory.withTimestampFn(
> new
> MessageTimestampExtractor(inputMessagesConfig)))
> .withReadCommitted()
> .commitOffsetsInFinalize())
> .apply(ParDo.of(new ToEventFn()))
> //.apply(
> //Window.into(new ZurichTimePartitioningWindowFn())
> //.triggering(
> //Repeatedly.forever(
> //AfterFirst.of(
> //
> AfterPane.elementCountAtLeast(bundleSize),
> //
> AfterProcessingTime.pastFirstElementInPane()
> //.plusDelayOf(refreshFrequency
> //.withAllowedLateness(Duration.standardDays(1))
> //.discardingFiredPanes())
> .apply(
> BigQueryIO.write()
> .withMethod(BigQueryIO.Write.Method.FILE_LOADS)
> .withTriggeringFrequency(refreshFrequency)
> .withNumFileShards(1)
> .to(partitionedTableDynamicDestinations)
> .withFormatFunction(
> (SerializableFunction)
> KafkaToBigQuery::convertUserEventToTableRow)
>
> .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
>
> .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
>
>
> I 

Re: KafkaIO - Deadletter output

2018-10-29 Thread Raghu Angadi
On Mon, Oct 29, 2018 at 5:26 AM Jozef Vilcek  wrote:

> Yes, state in timestamp policy but simple, not like ParDo state.
> TimestampPolicy appears to be long lived instance. Take an inspiration from
> already existing policies, e.g. like the one here:
>
> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/TimestampPolicyFactory.java#L135
>

Correct. Looks like we need to improve JavaDoc[1] for TimestampPolicy to
explain the contract better. The policy has same life cycle as the the
reader, with one instance for each of the Kafka partition read by the
reader. It is created when the reader is created. When the reader is
resumed from a checkpoint, the new policy's constructor has access to the
previous watermark set, but not the previous record timestamp. In that
sense, Jozef's solution has one corner case about unknown timestamp for the
first record if it is not parsable (which probably fine in practice). I had
left some comments about allowing user to remember more state for each
PartitionContext [2] when we added TimestampPolicy to KafkaIO.

State API is not available in UnboundedSource API, essentially
'CheckpointMark' is the state that can be stored with each reader.

[1]:
https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/TimestampPolicy.java#L29
[2]
https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/TimestampPolicy.java#L35


> On Mon, Oct 29, 2018 at 11:57 AM Kaymak, Tobias 
> wrote:
>
>> Sorry for not replying, I was sick with a flu.
>>
>> On Thu, Oct 25, 2018 at 9:56 AM Jozef Vilcek 
>> wrote:
>>
>>> what I ended up doing, when I could not for any reasono rely on kafka
>>> timestamps, but need to parse them form message is:
>>>
>>> * have a cusom kafka deserializer which never throws but returns message
>>> which is either a success with parsed data structure plus timestamp or
>>> failure with original kafka bytes payload
>>> * timestamp policy than can extract timestamp in case of a success
>>> deserialize result and in case of failure result, I am returning timestamp
>>> of a last success message  ( in my case messages are not terribly out of
>>> order and failures are rather rare )
>>>
>>
>> It's possible that my stream contains data that is very old (when
>> rewinding a topic, lets say it goes back to 2012). If I get the logic here
>> correctly this means I need to remember the last successfully parsed
>> timestamp. Do you solve this via StatefulProcessing?
>>
>>
>>> * following ParDo then side output failures to dead letters
>>>
>>>
>>
>>
>>> On Thu, Oct 25, 2018 at 8:54 AM Reuven Lax  wrote:
>>>
>>>>
>>>>
>>>> On Wed, Oct 24, 2018, 10:26 PM Raghu Angadi  wrote:
>>>>
>>>>> Well, if every input record's timestamp is X, watermark staying at X
>>>>> is the right answer, no? But I am not sure where the disagreement is,
>>>>> actually. I might be mistaken.
>>>>>
>>>>> KafkaIO has a few in-built policies for watermark and timestamp that
>>>>> cover most use cases (including server time, which has a benefit of
>>>>> providing perfect watermark). It also gives fairly complete control on
>>>>> these to users if they chose to. I think it looks like reasonable for a
>>>>> policy to base its watermark only only on parsable records, and ignore
>>>>> unparsable records w.r.t watermark calculation.
>>>>>
>>>>
>>>> But then doesn't that force the user to set max allowed lateness to
>>>> infinity, otherwise these records will be dropped?
>>>>
>>>> It could even assign a timestamp that makes more logical sense in a
>>>>> particular application.
>>>>>
>>>>> On Wed, Oct 24, 2018 at 8:30 PM Kenneth Knowles 
>>>>> wrote:
>>>>>
>>>>>> Forgive me if this is naive or missing something, but here are my
>>>>>> thoughts on these alternatives:
>>>>>>
>>>>>> (0) Timestamp has to be pulled out in the source to control the
>>>>>> watermark. Luke's point is imortant.
>>>>>>
>>>>>> (1) If bad records get min_timestamp, and they occur infrequently
>>>>>> enough, then watermark will advance and they will all be dropped. That 
>>>>>> will
>>>>>> not allow output to a dead-letter queue.
>>>

Re: write to a kafka topic that is set in data

2018-10-26 Thread Raghu Angadi
On Fri, Oct 26, 2018 at 7:50 AM Alexey Romanenko 
wrote:

> Perhaps, the optimal solution would be to have a common way to transfer a
>> meta data along with key/values to KafkaIO transform. Don’t you think the
>> we could use KafkaRecord (used for reading currently) for this purpose?
>>
>
> I agree, I think this is the Right Thing to do. It addresses a few other
> issues too (Support for Kafka headers while writing requires this :
> BEAM-4038 <https://issues.apache.org/jira/browse/BEAM-4038>).
>
>
> Thank you for pointing to this Jira. I see that some work had been done
> there but it has not been finished yet. The idea was to use
> PCollection> instead of current PCollection>,
> right?
>
> It sounds good for me since perfectly fits the goal of this thread
> (dynamic topics) but it seems that we need to change a contract of
> KafkaIO.Write to use ProducerRecord<> instead of KV<>. So, it will be the
> breaking changes for user API. Do you have an idea how we can make it
> back-compatible?
>

Interface would be pretty straight forward. See 'Write.values()'
which takes PCollection instead of PCollection>. I think
similar technique would wor. All the existing code works without any
changes and will be a one line change for users who want to write
ProducerRecords.

Of course, internal implementation will have more changes since we will be
carrying ProducerRecords rather than KV. I think that is fine and
safe. We can discuss more details in a follow up Jira or PR. Thanks for
proposing this solution.

Raghu.



>
>
>
>
>> > Actually I take this back. It I don't think it coupled with output
>> topic and partitions. It might just work (assuming Kafka can handle
>> individual transactions spanning many topics well).
>>
>> Do you mean that we can just take a topic name based on KV (using
>> Serialisable function or other way discussed above) and use it instead of
>> current spec.getTopic() ?
>>
>
> Yes, something on those lines. In practice it might hit practical
> limitations on Kafka transaction support. E.g. if a bundle has 1000 records
> going to 200 distinct topics, all of those will be written in a single
> transaction. Not sure how well that would work in practice, but
> theoretically it should.
>
> Raghu.
>
>>
>>
>> On 24 Oct 2018, at 20:01, Raghu Angadi  wrote:
>>
>> On Wed, Oct 24, 2018 at 10:47 AM Raghu Angadi  wrote:
>>
>>> My bad Alexey, I will review today. I had skimmed through the patch on
>>> my phone. You are right, exactly-once sink support is not required for now.
>>>
>>
>>
>>
>>> It is a quite a different beast and necessarily coupled with
>>> transactions on a specific topic-partitions for correctness.
>>>
>> Actually I take this back. It I don't think it coupled with output topic
>> and partitions. It might just work (assuming Kafka can handle individual
>> transactions spanning many topics well). As you mentioned, we would still
>> need to plumb it through. As such we don't know if exactly-once sink is
>> being used much... (I would love to hear about it if anyone is using it).
>>
>>
>>>
>>> The primary concern is with the API. The user provides a function to map
>>> an output record to its topic. We have found that such an API is usually
>>> problematic. E.g. what if the record does not encode enough information
>>> about topic? Say we want to select topic name based on aggregation window.
>>> It might be bit more code, but simpler to let the user decide topic for
>>> each record _before_ writing to the sink. E.g. it could be
>>> KafkaIO.Writer>.
>>> I wanted to think a little bit more about this, but didn't get around to
>>> it. I will comment on the PR today.
>>>
>>> thanks for the initiative and the PR.
>>> Raghu.
>>> On Wed, Oct 24, 2018 at 7:03 AM Alexey Romanenko <
>>> aromanenko@gmail.com> wrote:
>>>
>>>> I added a simple support of this for usual type of Kafka sink (PR:
>>>> https://github.com/apache/beam/pull/6776 , welcomed for review, btw :)
>>>> )
>>>>
>>>> In the same time, there is another, more complicated, type of sink -
>>>> EOS (Exactly Once Sink). In this case the data is partitioned among fixed
>>>> number of shards and it creates one ShardWriter per shard. In its
>>>> order, ShardWriter depends on Kafka topic. So, seems that in case of
>>>> multiple and dynamic sink topics, we need to create new ShardWriter for
>>>> every new topic per shard,
>>>>
>&

Re: KafkaIO - Deadletter output

2018-10-25 Thread Raghu Angadi
On Thu, Oct 25, 2018 at 10:28 AM Chamikara Jayalath 
wrote:

> Not sure if I understand why this would require Kafka to behave as two
> independent sources.
>



> Won't setting a non-negative-infinity timestamp (for example processing
> time) for failed records be enough ?
>

Well that's what I suggested and that is what a user seems to have done.
The question what if that is not what we want and don't want to mix these
two together (at least from my reading of Luke's and Kenn's comments, which
could be off).


>
> Also (at least at some point) there were discussions on supporting SDFs to
> report different watermarks for different outputs. More details are
> available here:
> https://docs.google.com/document/d/1BGc8pM1GOvZhwR9SARSVte-20XEoBUxrGJ5gTWXdv3c/edit
>
> - Cham
>
>
>>
>> Raghu.
>>
>>>
>>> It could even assign a timestamp that makes more logical sense in a
>>>> particular application.
>>>>
>>>> On Wed, Oct 24, 2018 at 8:30 PM Kenneth Knowles 
>>>> wrote:
>>>>
>>>>> Forgive me if this is naive or missing something, but here are my
>>>>> thoughts on these alternatives:
>>>>>
>>>>> (0) Timestamp has to be pulled out in the source to control the
>>>>> watermark. Luke's point is imortant.
>>>>>
>>>>> (1) If bad records get min_timestamp, and they occur infrequently
>>>>> enough, then watermark will advance and they will all be dropped. That 
>>>>> will
>>>>> not allow output to a dead-letter queue.
>>>>>
>>>>> (2) If you have always min_timestamp records, or if bad records are
>>>>> frequent, the watermark will never advance. So windows/aggregations would
>>>>> never be considered complete. Triggers could be used to get output anyhow,
>>>>> but it would never be a final answer. I think it is not in the spirit of
>>>>> Beam to work this way. Pragmatically, no state could ever be freed by a
>>>>> runner.
>>>>>
>>>>> In SQL there is an actual "dead letter" option when creating a table
>>>>> that parses from a bytes source. If, for example, a JSON record cannot be
>>>>> parsed to the expected schema - like maybe an avro record got in the
>>>>> stream, or the JSON doesn't match the expected schema - it is output as-is
>>>>> to a user-specified dead letter queue. I think this same level of support
>>>>> is also required for records that cannot have timestamps extracted in an
>>>>> unbounded source.
>>>>>
>>>>> In an SDF I think the function has enough control to do it all in
>>>>> "userland", so Cham is right on here.
>>>>>
>>>>> Kenn
>>>>>
>>>>> On Wed, Oct 24, 2018 at 6:54 PM Lukasz Cwik  wrote:
>>>>>
>>>>>> That depends on the users pipeline and how watermark advancement of
>>>>>> the source may impact elements becoming droppably late if they are 
>>>>>> emitted
>>>>>> with the minimum timestamp.
>>>>>>
>>>>>> On Wed, Oct 24, 2018 at 4:42 PM Raghu Angadi 
>>>>>> wrote:
>>>>>>
>>>>>>> I see.
>>>>>>>
>>>>>>> What I meant was to return min_timestamp for bad records in the
>>>>>>> timestamp handler passed to KafkaIO itself, and correct timestamp for
>>>>>>> parsable records. That should work too, right?
>>>>>>>
>>>>>>> On Wed, Oct 24, 2018 at 4:21 PM Lukasz Cwik 
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Yes, that would be fine.
>>>>>>>>
>>>>>>>> The user could then use a ParDo which outputs to a DLQ for things
>>>>>>>> it can't parse the timestamp for and use outputWithTimestamp[1] for
>>>>>>>> everything else.
>>>>>>>>
>>>>>>>> 1:
>>>>>>>> https://beam.apache.org/releases/javadoc/2.7.0/org/apache/beam/sdk/transforms/DoFn.WindowedContext.html#outputWithTimestamp-org.apache.beam.sdk.values.TupleTag-T-org.joda.time.Instant-
>>>>>>>>
>>>>>>>> On Wed, Oct 24, 2018 at 1:21 PM Raghu Angadi 
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Thanks. So re

Re: write to a kafka topic that is set in data

2018-10-25 Thread Raghu Angadi
On Thu, Oct 25, 2018 at 8:31 AM Alexey Romanenko 
wrote:

> > The primary concern is with the E.g. what if the record does not
> encode enough information about topic? Say we want to select topic name
> based on aggregation window.
> > E.g. it could be KafkaIO.Writer>.
>
> Well, I think even with this API, using user provided function, this
> information (topic name) can be incapsulated into key or value but then it
> should be filtered out (if needed) by user Serializer… Not a nice solution.
>
Interesting. It works :).


> Perhaps, the optimal solution would be to have a common way to transfer a
> meta data along with key/values to KafkaIO transform. Don’t you think the
> we could use KafkaRecord (used for reading currently) for this purpose?
>

I agree, I think this is the Right Thing to do. It addresses a few other
issues too (Support for Kafka headers while writing requires this :
BEAM-4038 <https://issues.apache.org/jira/browse/BEAM-4038>).


> > Actually I take this back. It I don't think it coupled with output topic
> and partitions. It might just work (assuming Kafka can handle individual
> transactions spanning many topics well).
>
> Do you mean that we can just take a topic name based on KV (using
> Serialisable function or other way discussed above) and use it instead of
> current spec.getTopic() ?
>

Yes, something on those lines. In practice it might hit practical
limitations on Kafka transaction support. E.g. if a bundle has 1000 records
going to 200 distinct topics, all of those will be written in a single
transaction. Not sure how well that would work in practice, but
theoretically it should.

Raghu.

>
>
> On 24 Oct 2018, at 20:01, Raghu Angadi  wrote:
>
> On Wed, Oct 24, 2018 at 10:47 AM Raghu Angadi  wrote:
>
>> My bad Alexey, I will review today. I had skimmed through the patch on my
>> phone. You are right, exactly-once sink support is not required for now.
>>
>
>
>
>> It is a quite a different beast and necessarily coupled with transactions
>> on a specific topic-partitions for correctness.
>>
> Actually I take this back. It I don't think it coupled with output topic
> and partitions. It might just work (assuming Kafka can handle individual
> transactions spanning many topics well). As you mentioned, we would still
> need to plumb it through. As such we don't know if exactly-once sink is
> being used much... (I would love to hear about it if anyone is using it).
>
>
>>
>> The primary concern is with the API. The user provides a function to map
>> an output record to its topic. We have found that such an API is usually
>> problematic. E.g. what if the record does not encode enough information
>> about topic? Say we want to select topic name based on aggregation window.
>> It might be bit more code, but simpler to let the user decide topic for
>> each record _before_ writing to the sink. E.g. it could be
>> KafkaIO.Writer>.
>> I wanted to think a little bit more about this, but didn't get around to
>> it. I will comment on the PR today.
>>
>> thanks for the initiative and the PR.
>> Raghu.
>> On Wed, Oct 24, 2018 at 7:03 AM Alexey Romanenko <
>> aromanenko@gmail.com> wrote:
>>
>>> I added a simple support of this for usual type of Kafka sink (PR:
>>> https://github.com/apache/beam/pull/6776 , welcomed for review, btw :) )
>>>
>>> In the same time, there is another, more complicated, type of sink - EOS
>>> (Exactly Once Sink). In this case the data is partitioned among fixed
>>> number of shards and it creates one ShardWriter per shard. In its
>>> order, ShardWriter depends on Kafka topic. So, seems that in case of
>>> multiple and dynamic sink topics, we need to create new ShardWriter for
>>> every new topic per shard,
>>>
>>> Is my assumption correct or I missed/misunderstood something?
>>>
>>> On 20 Oct 2018, at 01:21, Lukasz Cwik  wrote:
>>>
>>> Thanks Raghu, added starter and newbie labels to the issue.
>>>
>>> On Fri, Oct 19, 2018 at 4:20 PM Raghu Angadi  wrote:
>>>
>>>> It will be a good starter feature for someone interested in Beam &
>>>> Kafka. Writer is very simple in Beam. It is little more than a ParDo.
>>>>
>>>> On Fri, Oct 19, 2018 at 3:37 PM Dmitry Minaev 
>>>> wrote:
>>>>
>>>>> Lukasz, I appreciate the quick response and filing the JIRA ticket.
>>>>> Thanks for the suggestion, unfortunately, I don't have a fixed number of
>>>>> topics. Still, we'll probably use your approach for a limited number of
>>>>> topi

Re: KafkaIO - Deadletter output

2018-10-24 Thread Raghu Angadi
Well, if every input record's timestamp is X, watermark staying at X is the
right answer, no? But I am not sure where the disagreement is, actually. I
might be mistaken.

KafkaIO has a few in-built policies for watermark and timestamp that cover
most use cases (including server time, which has a benefit of providing
perfect watermark). It also gives fairly complete control on these to users
if they chose to. I think it looks like reasonable for a policy to base its
watermark only only on parsable records, and ignore unparsable records
w.r.t watermark calculation. It could even assign a timestamp that makes
more logical sense in a particular application.

On Wed, Oct 24, 2018 at 8:30 PM Kenneth Knowles  wrote:

> Forgive me if this is naive or missing something, but here are my thoughts
> on these alternatives:
>
> (0) Timestamp has to be pulled out in the source to control the watermark.
> Luke's point is imortant.
>
> (1) If bad records get min_timestamp, and they occur infrequently enough,
> then watermark will advance and they will all be dropped. That will not
> allow output to a dead-letter queue.
>
> (2) If you have always min_timestamp records, or if bad records are
> frequent, the watermark will never advance. So windows/aggregations would
> never be considered complete. Triggers could be used to get output anyhow,
> but it would never be a final answer. I think it is not in the spirit of
> Beam to work this way. Pragmatically, no state could ever be freed by a
> runner.
>
> In SQL there is an actual "dead letter" option when creating a table that
> parses from a bytes source. If, for example, a JSON record cannot be parsed
> to the expected schema - like maybe an avro record got in the stream, or
> the JSON doesn't match the expected schema - it is output as-is to a
> user-specified dead letter queue. I think this same level of support is
> also required for records that cannot have timestamps extracted in an
> unbounded source.
>
> In an SDF I think the function has enough control to do it all in
> "userland", so Cham is right on here.
>
> Kenn
>
> On Wed, Oct 24, 2018 at 6:54 PM Lukasz Cwik  wrote:
>
>> That depends on the users pipeline and how watermark advancement of the
>> source may impact elements becoming droppably late if they are emitted with
>> the minimum timestamp.
>>
>> On Wed, Oct 24, 2018 at 4:42 PM Raghu Angadi  wrote:
>>
>>> I see.
>>>
>>> What I meant was to return min_timestamp for bad records in the
>>> timestamp handler passed to KafkaIO itself, and correct timestamp for
>>> parsable records. That should work too, right?
>>>
>>> On Wed, Oct 24, 2018 at 4:21 PM Lukasz Cwik  wrote:
>>>
>>>> Yes, that would be fine.
>>>>
>>>> The user could then use a ParDo which outputs to a DLQ for things it
>>>> can't parse the timestamp for and use outputWithTimestamp[1] for everything
>>>> else.
>>>>
>>>> 1:
>>>> https://beam.apache.org/releases/javadoc/2.7.0/org/apache/beam/sdk/transforms/DoFn.WindowedContext.html#outputWithTimestamp-org.apache.beam.sdk.values.TupleTag-T-org.joda.time.Instant-
>>>>
>>>> On Wed, Oct 24, 2018 at 1:21 PM Raghu Angadi 
>>>> wrote:
>>>>
>>>>> Thanks. So returning  min timestamp is OK, right (assuming application
>>>>> fine is with what it means)?
>>>>>
>>>>> On Wed, Oct 24, 2018 at 1:17 PM Lukasz Cwik  wrote:
>>>>>
>>>>>> All records in Apache Beam have a timestamp. The default timestamp is
>>>>>> the min timestamp defined here:
>>>>>> https://github.com/apache/beam/blob/279a05604b83a54e8e5a79e13d8761f94841f326/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java#L48
>>>>>>
>>>>>>
>>>>>> On Wed, Oct 24, 2018 at 12:51 PM Raghu Angadi 
>>>>>> wrote:
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Oct 24, 2018 at 12:33 PM Lukasz Cwik 
>>>>>>> wrote:
>>>>>>>
>>>>>>>> You would have to return min timestamp for all records otherwise
>>>>>>>> the watermark may have advanced and you would be outputting records 
>>>>>>>> that
>>>>>>>> are droppably late.
>>>>>>>>
>>>>>>>
>>>>>>> That would be fine I guess. What’s the timestamp for a record that
>>>>>>> doesn’t have o

Re: KafkaIO - Deadletter output

2018-10-24 Thread Raghu Angadi
I see.

What I meant was to return min_timestamp for bad records in the timestamp
handler passed to KafkaIO itself, and correct timestamp for parsable
records. That should work too, right?

On Wed, Oct 24, 2018 at 4:21 PM Lukasz Cwik  wrote:

> Yes, that would be fine.
>
> The user could then use a ParDo which outputs to a DLQ for things it can't
> parse the timestamp for and use outputWithTimestamp[1] for everything else.
>
> 1:
> https://beam.apache.org/releases/javadoc/2.7.0/org/apache/beam/sdk/transforms/DoFn.WindowedContext.html#outputWithTimestamp-org.apache.beam.sdk.values.TupleTag-T-org.joda.time.Instant-
>
> On Wed, Oct 24, 2018 at 1:21 PM Raghu Angadi  wrote:
>
>> Thanks. So returning  min timestamp is OK, right (assuming application
>> fine is with what it means)?
>>
>> On Wed, Oct 24, 2018 at 1:17 PM Lukasz Cwik  wrote:
>>
>>> All records in Apache Beam have a timestamp. The default timestamp is
>>> the min timestamp defined here:
>>> https://github.com/apache/beam/blob/279a05604b83a54e8e5a79e13d8761f94841f326/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java#L48
>>>
>>>
>>> On Wed, Oct 24, 2018 at 12:51 PM Raghu Angadi 
>>> wrote:
>>>
>>>>
>>>>
>>>> On Wed, Oct 24, 2018 at 12:33 PM Lukasz Cwik  wrote:
>>>>
>>>>> You would have to return min timestamp for all records otherwise the
>>>>> watermark may have advanced and you would be outputting records that are
>>>>> droppably late.
>>>>>
>>>>
>>>> That would be fine I guess. What’s the timestamp for a record that
>>>> doesn’t have one?
>>>>
>>>>
>>>>> On Wed, Oct 24, 2018 at 12:25 PM Raghu Angadi 
>>>>> wrote:
>>>>>
>>>>>> To be clear, returning min_timestamp for unparsable records shound
>>>>>> not affect the watermark.
>>>>>>
>>>>>> On Wed, Oct 24, 2018 at 10:32 AM Raghu Angadi 
>>>>>> wrote:
>>>>>>
>>>>>>> How about returning min_timestamp? The would be dropped or
>>>>>>> redirected by the ParDo after that.
>>>>>>> Btw, TimestampPolicyFactory.withTimestampFn() is not a public API,
>>>>>>> is this pipeline defined under kafkaio package?
>>>>>>>
>>>>>>> On Wed, Oct 24, 2018 at 10:19 AM Lukasz Cwik 
>>>>>>> wrote:
>>>>>>>
>>>>>>>> In this case, the user is attempting to handle errors when parsing
>>>>>>>> the timestamp. The timestamp controls the watermark for the
>>>>>>>> UnboundedSource, how would they control the watermark in a downstream 
>>>>>>>> ParDo?
>>>>>>>>
>>>>>>>> On Wed, Oct 24, 2018 at 9:48 AM Raghu Angadi 
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> On Wed, Oct 24, 2018 at 7:19 AM Chamikara Jayalath <
>>>>>>>>> chamik...@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> Ah nice. Yeah, if user can return full bytes instead of applying
>>>>>>>>>> a function that would result in an exception,  this can be extracted 
>>>>>>>>>> by a
>>>>>>>>>> ParDo down the line.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> KafkaIO does return bytes, and I think most sources should, unless
>>>>>>>>> there is a good reason not to.
>>>>>>>>> Given that, do we think Beam should provide a tranform that makes
>>>>>>>>> to simpler to handle deadletter output? I think there was a thread 
>>>>>>>>> about it
>>>>>>>>> in the past.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Tue, Oct 23, 2018 at 11:14 PM Juan Carlos Garcia <
>>>>>>>>>> jcgarc...@gmail.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> As Raghu said,
>>>>>>>>>>>
>>>>>>>>>>> Just apply a regular ParDo and return a PCollectionTuple afert
>>>>>>>>>>> that you can ext

Re: KafkaIO - Deadletter output

2018-10-24 Thread Raghu Angadi
Thanks. So returning  min timestamp is OK, right (assuming application fine
is with what it means)?

On Wed, Oct 24, 2018 at 1:17 PM Lukasz Cwik  wrote:

> All records in Apache Beam have a timestamp. The default timestamp is the
> min timestamp defined here:
> https://github.com/apache/beam/blob/279a05604b83a54e8e5a79e13d8761f94841f326/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java#L48
>
>
> On Wed, Oct 24, 2018 at 12:51 PM Raghu Angadi  wrote:
>
>>
>>
>> On Wed, Oct 24, 2018 at 12:33 PM Lukasz Cwik  wrote:
>>
>>> You would have to return min timestamp for all records otherwise the
>>> watermark may have advanced and you would be outputting records that are
>>> droppably late.
>>>
>>
>> That would be fine I guess. What’s the timestamp for a record that
>> doesn’t have one?
>>
>>
>>> On Wed, Oct 24, 2018 at 12:25 PM Raghu Angadi 
>>> wrote:
>>>
>>>> To be clear, returning min_timestamp for unparsable records shound not
>>>> affect the watermark.
>>>>
>>>> On Wed, Oct 24, 2018 at 10:32 AM Raghu Angadi 
>>>> wrote:
>>>>
>>>>> How about returning min_timestamp? The would be dropped or redirected
>>>>> by the ParDo after that.
>>>>> Btw, TimestampPolicyFactory.withTimestampFn() is not a public API, is
>>>>> this pipeline defined under kafkaio package?
>>>>>
>>>>> On Wed, Oct 24, 2018 at 10:19 AM Lukasz Cwik  wrote:
>>>>>
>>>>>> In this case, the user is attempting to handle errors when parsing
>>>>>> the timestamp. The timestamp controls the watermark for the
>>>>>> UnboundedSource, how would they control the watermark in a downstream 
>>>>>> ParDo?
>>>>>>
>>>>>> On Wed, Oct 24, 2018 at 9:48 AM Raghu Angadi 
>>>>>> wrote:
>>>>>>
>>>>>>> On Wed, Oct 24, 2018 at 7:19 AM Chamikara Jayalath <
>>>>>>> chamik...@google.com> wrote:
>>>>>>>
>>>>>>>> Ah nice. Yeah, if user can return full bytes instead of applying a
>>>>>>>> function that would result in an exception,  this can be extracted by a
>>>>>>>> ParDo down the line.
>>>>>>>>
>>>>>>>
>>>>>>> KafkaIO does return bytes, and I think most sources should, unless
>>>>>>> there is a good reason not to.
>>>>>>> Given that, do we think Beam should provide a tranform that makes to
>>>>>>> simpler to handle deadletter output? I think there was a thread about 
>>>>>>> it in
>>>>>>> the past.
>>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>> On Tue, Oct 23, 2018 at 11:14 PM Juan Carlos Garcia <
>>>>>>>> jcgarc...@gmail.com> wrote:
>>>>>>>>
>>>>>>>>> As Raghu said,
>>>>>>>>>
>>>>>>>>> Just apply a regular ParDo and return a PCollectionTuple afert
>>>>>>>>> that you can extract your Success Records (TupleTag) and your 
>>>>>>>>> DeadLetter
>>>>>>>>> records(TupleTag) and do whatever you want with them.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Raghu Angadi  schrieb am Mi., 24. Okt. 2018,
>>>>>>>>> 05:18:
>>>>>>>>>
>>>>>>>>>> User can read serialized bytes from KafkaIO and deserialize
>>>>>>>>>> explicitly in a ParDo, which gives complete control on how to handle 
>>>>>>>>>> record
>>>>>>>>>> errors. This is I would do if I need to in my pipeline.
>>>>>>>>>>
>>>>>>>>>> If there is a transform in Beam that does this, it could be
>>>>>>>>>> convenient for users in many such scenarios. This is simpler than 
>>>>>>>>>> each
>>>>>>>>>> source supporting it explicitly.
>>>>>>>>>>
>>>>>>>>>> On Tue, Oct 23, 2018 at 8:03 PM Chamikara Jayalath <
>>>>>>>>>> chamik...@google.com> wrote:
>>>>>>>&

Re: KafkaIO - Deadletter output

2018-10-24 Thread Raghu Angadi
On Wed, Oct 24, 2018 at 12:33 PM Lukasz Cwik  wrote:

> You would have to return min timestamp for all records otherwise the
> watermark may have advanced and you would be outputting records that are
> droppably late.
>

That would be fine I guess. What’s the timestamp for a record that doesn’t
have one?


> On Wed, Oct 24, 2018 at 12:25 PM Raghu Angadi  wrote:
>
>> To be clear, returning min_timestamp for unparsable records shound not
>> affect the watermark.
>>
>> On Wed, Oct 24, 2018 at 10:32 AM Raghu Angadi  wrote:
>>
>>> How about returning min_timestamp? The would be dropped or redirected by
>>> the ParDo after that.
>>> Btw, TimestampPolicyFactory.withTimestampFn() is not a public API, is
>>> this pipeline defined under kafkaio package?
>>>
>>> On Wed, Oct 24, 2018 at 10:19 AM Lukasz Cwik  wrote:
>>>
>>>> In this case, the user is attempting to handle errors when parsing the
>>>> timestamp. The timestamp controls the watermark for the UnboundedSource,
>>>> how would they control the watermark in a downstream ParDo?
>>>>
>>>> On Wed, Oct 24, 2018 at 9:48 AM Raghu Angadi 
>>>> wrote:
>>>>
>>>>> On Wed, Oct 24, 2018 at 7:19 AM Chamikara Jayalath <
>>>>> chamik...@google.com> wrote:
>>>>>
>>>>>> Ah nice. Yeah, if user can return full bytes instead of applying a
>>>>>> function that would result in an exception,  this can be extracted by a
>>>>>> ParDo down the line.
>>>>>>
>>>>>
>>>>> KafkaIO does return bytes, and I think most sources should, unless
>>>>> there is a good reason not to.
>>>>> Given that, do we think Beam should provide a tranform that makes to
>>>>> simpler to handle deadletter output? I think there was a thread about it 
>>>>> in
>>>>> the past.
>>>>>
>>>>>
>>>>>>
>>>>>> On Tue, Oct 23, 2018 at 11:14 PM Juan Carlos Garcia <
>>>>>> jcgarc...@gmail.com> wrote:
>>>>>>
>>>>>>> As Raghu said,
>>>>>>>
>>>>>>> Just apply a regular ParDo and return a PCollectionTuple afert that
>>>>>>> you can extract your Success Records (TupleTag) and your DeadLetter
>>>>>>> records(TupleTag) and do whatever you want with them.
>>>>>>>
>>>>>>>
>>>>>>> Raghu Angadi  schrieb am Mi., 24. Okt. 2018,
>>>>>>> 05:18:
>>>>>>>
>>>>>>>> User can read serialized bytes from KafkaIO and deserialize
>>>>>>>> explicitly in a ParDo, which gives complete control on how to handle 
>>>>>>>> record
>>>>>>>> errors. This is I would do if I need to in my pipeline.
>>>>>>>>
>>>>>>>> If there is a transform in Beam that does this, it could be
>>>>>>>> convenient for users in many such scenarios. This is simpler than each
>>>>>>>> source supporting it explicitly.
>>>>>>>>
>>>>>>>> On Tue, Oct 23, 2018 at 8:03 PM Chamikara Jayalath <
>>>>>>>> chamik...@google.com> wrote:
>>>>>>>>
>>>>>>>>> Given that KafkaIO uses UnboundeSource framework, this is probably
>>>>>>>>> not something that can easily be supported. We might be able to 
>>>>>>>>> support
>>>>>>>>> similar features when we have Kafka on top of Splittable DoFn though.
>>>>>>>>>
>>>>>>>> So feel free to create a feature request JIRA for this.
>>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> Cham
>>>>>>>>>
>>>>>>>>> On Tue, Oct 23, 2018 at 7:43 PM Kenneth Knowles 
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> This is a great question. I've added the dev list to be sure it
>>>>>>>>>> gets noticed by whoever may know best.
>>>>>>>>>>
>>>>>>>>>> Kenn
>>>>>>>>>>
>>>>>>>>>> On Tue, Oct 23, 2018 at 2:05 AM Kaymak, Tobias <
>>>

Re: KafkaIO - Deadletter output

2018-10-24 Thread Raghu Angadi
To be clear, returning min_timestamp for unparsable records shound not
affect the watermark.

On Wed, Oct 24, 2018 at 10:32 AM Raghu Angadi  wrote:

> How about returning min_timestamp? The would be dropped or redirected by
> the ParDo after that.
> Btw, TimestampPolicyFactory.withTimestampFn() is not a public API, is
> this pipeline defined under kafkaio package?
>
> On Wed, Oct 24, 2018 at 10:19 AM Lukasz Cwik  wrote:
>
>> In this case, the user is attempting to handle errors when parsing the
>> timestamp. The timestamp controls the watermark for the UnboundedSource,
>> how would they control the watermark in a downstream ParDo?
>>
>> On Wed, Oct 24, 2018 at 9:48 AM Raghu Angadi  wrote:
>>
>>> On Wed, Oct 24, 2018 at 7:19 AM Chamikara Jayalath 
>>> wrote:
>>>
>>>> Ah nice. Yeah, if user can return full bytes instead of applying a
>>>> function that would result in an exception,  this can be extracted by a
>>>> ParDo down the line.
>>>>
>>>
>>> KafkaIO does return bytes, and I think most sources should, unless there
>>> is a good reason not to.
>>> Given that, do we think Beam should provide a tranform that makes to
>>> simpler to handle deadletter output? I think there was a thread about it in
>>> the past.
>>>
>>>
>>>>
>>>> On Tue, Oct 23, 2018 at 11:14 PM Juan Carlos Garcia <
>>>> jcgarc...@gmail.com> wrote:
>>>>
>>>>> As Raghu said,
>>>>>
>>>>> Just apply a regular ParDo and return a PCollectionTuple afert that
>>>>> you can extract your Success Records (TupleTag) and your DeadLetter
>>>>> records(TupleTag) and do whatever you want with them.
>>>>>
>>>>>
>>>>> Raghu Angadi  schrieb am Mi., 24. Okt. 2018,
>>>>> 05:18:
>>>>>
>>>>>> User can read serialized bytes from KafkaIO and deserialize
>>>>>> explicitly in a ParDo, which gives complete control on how to handle 
>>>>>> record
>>>>>> errors. This is I would do if I need to in my pipeline.
>>>>>>
>>>>>> If there is a transform in Beam that does this, it could be
>>>>>> convenient for users in many such scenarios. This is simpler than each
>>>>>> source supporting it explicitly.
>>>>>>
>>>>>> On Tue, Oct 23, 2018 at 8:03 PM Chamikara Jayalath <
>>>>>> chamik...@google.com> wrote:
>>>>>>
>>>>>>> Given that KafkaIO uses UnboundeSource framework, this is probably
>>>>>>> not something that can easily be supported. We might be able to support
>>>>>>> similar features when we have Kafka on top of Splittable DoFn though.
>>>>>>>
>>>>>> So feel free to create a feature request JIRA for this.
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Cham
>>>>>>>
>>>>>>> On Tue, Oct 23, 2018 at 7:43 PM Kenneth Knowles 
>>>>>>> wrote:
>>>>>>>
>>>>>>>> This is a great question. I've added the dev list to be sure it
>>>>>>>> gets noticed by whoever may know best.
>>>>>>>>
>>>>>>>> Kenn
>>>>>>>>
>>>>>>>> On Tue, Oct 23, 2018 at 2:05 AM Kaymak, Tobias <
>>>>>>>> tobias.kay...@ricardo.ch> wrote:
>>>>>>>>
>>>>>>>>>
>>>>>>>>> Hi,
>>>>>>>>>
>>>>>>>>> Is there a way to get a Deadletter Output from a pipeline that
>>>>>>>>> uses a KafkaIO
>>>>>>>>> connector for it's input? As
>>>>>>>>> `TimestampPolicyFactory.withTimestampFn()` takes
>>>>>>>>> only a SerializableFunction and not a ParDo, how would I be able
>>>>>>>>> to produce a
>>>>>>>>> Deadletter output from it?
>>>>>>>>>
>>>>>>>>> I have the following pipeline defined that reads from a KafkaIO
>>>>>>>>> input:
>>>>>>>>>
>>>>>>>>> pipeline.apply(
>>>>>>>>>   KafkaIO.read()
>>>>>>>>> .withBootstrapServers(bootstrap)
>>>>>>>>> .withTopics(topics)
>>>>>>>>> .withKeyDeserializer(StringDeserializer.class)
>>>>>>>>> .withValueDeserializer(ConfigurableDeserializer.class)
>>>>>>>>> .updateConsumerProperties(
>>>>>>>>> ImmutableMap.of(InputMessagesConfig.CONFIG_PROPERTY_NAME,
>>>>>>>>> inputMessagesConfig))
>>>>>>>>> .updateConsumerProperties(ImmutableMap.of("auto.offset.reset",
>>>>>>>>> "earliest"))
>>>>>>>>> .updateConsumerProperties(ImmutableMap.of("group.id",
>>>>>>>>> "beam-consumers"))
>>>>>>>>>
>>>>>>>>> .updateConsumerProperties(ImmutableMap.of("enable.auto.commit", 
>>>>>>>>> "true"))
>>>>>>>>> .withTimestampPolicyFactory(
>>>>>>>>> TimestampPolicyFactory.withTimestampFn(
>>>>>>>>> new MessageTimestampExtractor(inputMessagesConfig)))
>>>>>>>>> .withReadCommitted()
>>>>>>>>> .commitOffsetsInFinalize())
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> and I like to get deadletter outputs when my timestamp extraction
>>>>>>>>> fails.
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> Tobi
>>>>>>>>>
>>>>>>>>>


Re: write to a kafka topic that is set in data

2018-10-24 Thread Raghu Angadi
On Wed, Oct 24, 2018 at 10:47 AM Raghu Angadi  wrote:

> My bad Alexey, I will review today. I had skimmed through the patch on my
> phone. You are right, exactly-once sink support is not required for now.
>



> It is a quite a different beast and necessarily coupled with transactions
> on a specific topic-partitions for correctness.
>
Actually I take this back. It I don't think it coupled with output topic
and partitions. It might just work (assuming Kafka can handle individual
transactions spanning many topics well). As you mentioned, we would still
need to plumb it through. As such we don't know if exactly-once sink is
being used much... (I would love to hear about it if anyone is using it).


>
> The primary concern is with the API. The user provides a function to map
> an output record to its topic. We have found that such an API is usually
> problematic. E.g. what if the record does not encode enough information
> about topic? Say we want to select topic name based on aggregation window.
> It might be bit more code, but simpler to let the user decide topic for
> each record _before_ writing to the sink. E.g. it could be
> KafkaIO.Writer>.
> I wanted to think a little bit more about this, but didn't get around to
> it. I will comment on the PR today.
>
> thanks for the initiative and the PR.
> Raghu.
> On Wed, Oct 24, 2018 at 7:03 AM Alexey Romanenko 
> wrote:
>
>> I added a simple support of this for usual type of Kafka sink (PR:
>> https://github.com/apache/beam/pull/6776 , welcomed for review, btw :) )
>>
>> In the same time, there is another, more complicated, type of sink - EOS
>> (Exactly Once Sink). In this case the data is partitioned among fixed
>> number of shards and it creates one ShardWriter per shard. In its
>> order, ShardWriter depends on Kafka topic. So, seems that in case of
>> multiple and dynamic sink topics, we need to create new ShardWriter for
>> every new topic per shard,
>>
>> Is my assumption correct or I missed/misunderstood something?
>>
>> On 20 Oct 2018, at 01:21, Lukasz Cwik  wrote:
>>
>> Thanks Raghu, added starter and newbie labels to the issue.
>>
>> On Fri, Oct 19, 2018 at 4:20 PM Raghu Angadi  wrote:
>>
>>> It will be a good starter feature for someone interested in Beam &
>>> Kafka. Writer is very simple in Beam. It is little more than a ParDo.
>>>
>>> On Fri, Oct 19, 2018 at 3:37 PM Dmitry Minaev  wrote:
>>>
>>>> Lukasz, I appreciate the quick response and filing the JIRA ticket.
>>>> Thanks for the suggestion, unfortunately, I don't have a fixed number of
>>>> topics. Still, we'll probably use your approach for a limited number of
>>>> topics until the functionality is added, thank you!
>>>>
>>>> Thanks,
>>>> Dmitry
>>>>
>>>> On Fri, Oct 19, 2018 at 2:53 PM Lukasz Cwik  wrote:
>>>>
>>>>> If there are a fixed number of topics, you could partition your write
>>>>> by structuring your pipeline as such:
>>>>> ParDo(PartitionByTopic) > KafkaIO.write(topicA)
>>>>> \---> KafkaIO.write(topicB)
>>>>> \---> KafkaIO.write(...)
>>>>>
>>>>> There is no support currently for writing to Kafka dynamically based
>>>>> upon a destination that is part of the data.
>>>>> I filed https://issues.apache.org/jira/browse/BEAM-5798 for the issue.
>>>>>
>>>>> On Fri, Oct 19, 2018 at 2:05 PM mina...@gmail.com 
>>>>> wrote:
>>>>>
>>>>>> Hi guys!!
>>>>>>
>>>>>> I'm trying to find a way to write to a Kafka topic using
>>>>>> KafkaIO.write() But I need to be able to get topic name dynamically based
>>>>>> on the data received. For example, I would like to send data for one 
>>>>>> tenant
>>>>>> to topic "data_feed_1" and for another tenant to "topic data_feed_999".
>>>>>> I'm coming from Flink where it's possible via
>>>>>> KeyedSerializationSchema.getTargetTopic().
>>>>>> Is there anything similar in KafkaIO?
>>>>>>
>>>>>> Thanks,
>>>>>> Dmitry
>>>>>>
>>>>> --
>>>>
>>>> --
>>>> Dmitry
>>>>
>>>
>>


Re: write to a kafka topic that is set in data

2018-10-24 Thread Raghu Angadi
My bad Alexey, I will review today. I had skimmed through the patch on my
phone. You are right, exactly-once sink support is not required for now. It
is a quite a different beast and necessarily coupled with transactions on a
specific topic-partitions for correctness.

The primary concern is with the API. The user provides a function to map an
output record to its topic. We have found that such an API is usually
problematic. E.g. what if the record does not encode enough information
about topic? Say we want to select topic name based on aggregation window.
It might be bit more code, but simpler to let the user decide topic for
each record _before_ writing to the sink. E.g. it could be
KafkaIO.Writer>.
I wanted to think a little bit more about this, but didn't get around to
it. I will comment on the PR today.

thanks for the initiative and the PR.
Raghu.
On Wed, Oct 24, 2018 at 7:03 AM Alexey Romanenko 
wrote:

> I added a simple support of this for usual type of Kafka sink (PR:
> https://github.com/apache/beam/pull/6776 , welcomed for review, btw :) )
>
> In the same time, there is another, more complicated, type of sink - EOS
> (Exactly Once Sink). In this case the data is partitioned among fixed
> number of shards and it creates one ShardWriter per shard. In its
> order, ShardWriter depends on Kafka topic. So, seems that in case of
> multiple and dynamic sink topics, we need to create new ShardWriter for
> every new topic per shard,
>
> Is my assumption correct or I missed/misunderstood something?
>
> On 20 Oct 2018, at 01:21, Lukasz Cwik  wrote:
>
> Thanks Raghu, added starter and newbie labels to the issue.
>
> On Fri, Oct 19, 2018 at 4:20 PM Raghu Angadi  wrote:
>
>> It will be a good starter feature for someone interested in Beam & Kafka.
>> Writer is very simple in Beam. It is little more than a ParDo.
>>
>> On Fri, Oct 19, 2018 at 3:37 PM Dmitry Minaev  wrote:
>>
>>> Lukasz, I appreciate the quick response and filing the JIRA ticket.
>>> Thanks for the suggestion, unfortunately, I don't have a fixed number of
>>> topics. Still, we'll probably use your approach for a limited number of
>>> topics until the functionality is added, thank you!
>>>
>>> Thanks,
>>> Dmitry
>>>
>>> On Fri, Oct 19, 2018 at 2:53 PM Lukasz Cwik  wrote:
>>>
>>>> If there are a fixed number of topics, you could partition your write
>>>> by structuring your pipeline as such:
>>>> ParDo(PartitionByTopic) > KafkaIO.write(topicA)
>>>> \---> KafkaIO.write(topicB)
>>>> \---> KafkaIO.write(...)
>>>>
>>>> There is no support currently for writing to Kafka dynamically based
>>>> upon a destination that is part of the data.
>>>> I filed https://issues.apache.org/jira/browse/BEAM-5798 for the issue.
>>>>
>>>> On Fri, Oct 19, 2018 at 2:05 PM mina...@gmail.com 
>>>> wrote:
>>>>
>>>>> Hi guys!!
>>>>>
>>>>> I'm trying to find a way to write to a Kafka topic using
>>>>> KafkaIO.write() But I need to be able to get topic name dynamically based
>>>>> on the data received. For example, I would like to send data for one 
>>>>> tenant
>>>>> to topic "data_feed_1" and for another tenant to "topic data_feed_999".
>>>>> I'm coming from Flink where it's possible via
>>>>> KeyedSerializationSchema.getTargetTopic().
>>>>> Is there anything similar in KafkaIO?
>>>>>
>>>>> Thanks,
>>>>> Dmitry
>>>>>
>>>> --
>>>
>>> --
>>> Dmitry
>>>
>>
>


Re: KafkaIO - Deadletter output

2018-10-24 Thread Raghu Angadi
How about returning min_timestamp? The would be dropped or redirected by
the ParDo after that.
Btw, TimestampPolicyFactory.withTimestampFn() is not a public API, is this
pipeline defined under kafkaio package?

On Wed, Oct 24, 2018 at 10:19 AM Lukasz Cwik  wrote:

> In this case, the user is attempting to handle errors when parsing the
> timestamp. The timestamp controls the watermark for the UnboundedSource,
> how would they control the watermark in a downstream ParDo?
>
> On Wed, Oct 24, 2018 at 9:48 AM Raghu Angadi  wrote:
>
>> On Wed, Oct 24, 2018 at 7:19 AM Chamikara Jayalath 
>> wrote:
>>
>>> Ah nice. Yeah, if user can return full bytes instead of applying a
>>> function that would result in an exception,  this can be extracted by a
>>> ParDo down the line.
>>>
>>
>> KafkaIO does return bytes, and I think most sources should, unless there
>> is a good reason not to.
>> Given that, do we think Beam should provide a tranform that makes to
>> simpler to handle deadletter output? I think there was a thread about it in
>> the past.
>>
>>
>>>
>>> On Tue, Oct 23, 2018 at 11:14 PM Juan Carlos Garcia 
>>> wrote:
>>>
>>>> As Raghu said,
>>>>
>>>> Just apply a regular ParDo and return a PCollectionTuple afert that you
>>>> can extract your Success Records (TupleTag) and your DeadLetter
>>>> records(TupleTag) and do whatever you want with them.
>>>>
>>>>
>>>> Raghu Angadi  schrieb am Mi., 24. Okt. 2018, 05:18:
>>>>
>>>>> User can read serialized bytes from KafkaIO and deserialize explicitly
>>>>> in a ParDo, which gives complete control on how to handle record errors.
>>>>> This is I would do if I need to in my pipeline.
>>>>>
>>>>> If there is a transform in Beam that does this, it could be convenient
>>>>> for users in many such scenarios. This is simpler than each source
>>>>> supporting it explicitly.
>>>>>
>>>>> On Tue, Oct 23, 2018 at 8:03 PM Chamikara Jayalath <
>>>>> chamik...@google.com> wrote:
>>>>>
>>>>>> Given that KafkaIO uses UnboundeSource framework, this is probably
>>>>>> not something that can easily be supported. We might be able to support
>>>>>> similar features when we have Kafka on top of Splittable DoFn though.
>>>>>>
>>>>> So feel free to create a feature request JIRA for this.
>>>>>>
>>>>>> Thanks,
>>>>>> Cham
>>>>>>
>>>>>> On Tue, Oct 23, 2018 at 7:43 PM Kenneth Knowles 
>>>>>> wrote:
>>>>>>
>>>>>>> This is a great question. I've added the dev list to be sure it gets
>>>>>>> noticed by whoever may know best.
>>>>>>>
>>>>>>> Kenn
>>>>>>>
>>>>>>> On Tue, Oct 23, 2018 at 2:05 AM Kaymak, Tobias <
>>>>>>> tobias.kay...@ricardo.ch> wrote:
>>>>>>>
>>>>>>>>
>>>>>>>> Hi,
>>>>>>>>
>>>>>>>> Is there a way to get a Deadletter Output from a pipeline that uses
>>>>>>>> a KafkaIO
>>>>>>>> connector for it's input? As
>>>>>>>> `TimestampPolicyFactory.withTimestampFn()` takes
>>>>>>>> only a SerializableFunction and not a ParDo, how would I be able to
>>>>>>>> produce a
>>>>>>>> Deadletter output from it?
>>>>>>>>
>>>>>>>> I have the following pipeline defined that reads from a KafkaIO
>>>>>>>> input:
>>>>>>>>
>>>>>>>> pipeline.apply(
>>>>>>>>   KafkaIO.read()
>>>>>>>> .withBootstrapServers(bootstrap)
>>>>>>>> .withTopics(topics)
>>>>>>>> .withKeyDeserializer(StringDeserializer.class)
>>>>>>>> .withValueDeserializer(ConfigurableDeserializer.class)
>>>>>>>> .updateConsumerProperties(
>>>>>>>> ImmutableMap.of(InputMessagesConfig.CONFIG_PROPERTY_NAME,
>>>>>>>> inputMessagesConfig))
>>>>>>>> .updateConsumerProperties(ImmutableMap.of("auto.offset.reset",
>>>>>>>> "earliest"))
>>>>>>>> .updateConsumerProperties(ImmutableMap.of("group.id",
>>>>>>>> "beam-consumers"))
>>>>>>>> .updateConsumerProperties(ImmutableMap.of("enable.auto.commit",
>>>>>>>> "true"))
>>>>>>>> .withTimestampPolicyFactory(
>>>>>>>> TimestampPolicyFactory.withTimestampFn(
>>>>>>>> new MessageTimestampExtractor(inputMessagesConfig)))
>>>>>>>> .withReadCommitted()
>>>>>>>> .commitOffsetsInFinalize())
>>>>>>>>
>>>>>>>>
>>>>>>>> and I like to get deadletter outputs when my timestamp extraction
>>>>>>>> fails.
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> Tobi
>>>>>>>>
>>>>>>>>


Re: KafkaIO - Deadletter output

2018-10-24 Thread Raghu Angadi
On Wed, Oct 24, 2018 at 7:19 AM Chamikara Jayalath 
wrote:

> Ah nice. Yeah, if user can return full bytes instead of applying a
> function that would result in an exception,  this can be extracted by a
> ParDo down the line.
>

KafkaIO does return bytes, and I think most sources should, unless there is
a good reason not to.
Given that, do we think Beam should provide a tranform that makes to
simpler to handle deadletter output? I think there was a thread about it in
the past.


>
> On Tue, Oct 23, 2018 at 11:14 PM Juan Carlos Garcia 
> wrote:
>
>> As Raghu said,
>>
>> Just apply a regular ParDo and return a PCollectionTuple afert that you
>> can extract your Success Records (TupleTag) and your DeadLetter
>> records(TupleTag) and do whatever you want with them.
>>
>>
>> Raghu Angadi  schrieb am Mi., 24. Okt. 2018, 05:18:
>>
>>> User can read serialized bytes from KafkaIO and deserialize explicitly
>>> in a ParDo, which gives complete control on how to handle record errors.
>>> This is I would do if I need to in my pipeline.
>>>
>>> If there is a transform in Beam that does this, it could be convenient
>>> for users in many such scenarios. This is simpler than each source
>>> supporting it explicitly.
>>>
>>> On Tue, Oct 23, 2018 at 8:03 PM Chamikara Jayalath 
>>> wrote:
>>>
>>>> Given that KafkaIO uses UnboundeSource framework, this is probably not
>>>> something that can easily be supported. We might be able to support similar
>>>> features when we have Kafka on top of Splittable DoFn though.
>>>>
>>> So feel free to create a feature request JIRA for this.
>>>>
>>>> Thanks,
>>>> Cham
>>>>
>>>> On Tue, Oct 23, 2018 at 7:43 PM Kenneth Knowles  wrote:
>>>>
>>>>> This is a great question. I've added the dev list to be sure it gets
>>>>> noticed by whoever may know best.
>>>>>
>>>>> Kenn
>>>>>
>>>>> On Tue, Oct 23, 2018 at 2:05 AM Kaymak, Tobias <
>>>>> tobias.kay...@ricardo.ch> wrote:
>>>>>
>>>>>>
>>>>>> Hi,
>>>>>>
>>>>>> Is there a way to get a Deadletter Output from a pipeline that uses a
>>>>>> KafkaIO
>>>>>> connector for it's input? As
>>>>>> `TimestampPolicyFactory.withTimestampFn()` takes
>>>>>> only a SerializableFunction and not a ParDo, how would I be able to
>>>>>> produce a
>>>>>> Deadletter output from it?
>>>>>>
>>>>>> I have the following pipeline defined that reads from a KafkaIO input:
>>>>>>
>>>>>> pipeline.apply(
>>>>>>   KafkaIO.read()
>>>>>> .withBootstrapServers(bootstrap)
>>>>>> .withTopics(topics)
>>>>>> .withKeyDeserializer(StringDeserializer.class)
>>>>>> .withValueDeserializer(ConfigurableDeserializer.class)
>>>>>> .updateConsumerProperties(
>>>>>> ImmutableMap.of(InputMessagesConfig.CONFIG_PROPERTY_NAME,
>>>>>> inputMessagesConfig))
>>>>>> .updateConsumerProperties(ImmutableMap.of("auto.offset.reset",
>>>>>> "earliest"))
>>>>>> .updateConsumerProperties(ImmutableMap.of("group.id",
>>>>>> "beam-consumers"))
>>>>>> .updateConsumerProperties(ImmutableMap.of("enable.auto.commit",
>>>>>> "true"))
>>>>>> .withTimestampPolicyFactory(
>>>>>> TimestampPolicyFactory.withTimestampFn(
>>>>>> new MessageTimestampExtractor(inputMessagesConfig)))
>>>>>> .withReadCommitted()
>>>>>> .commitOffsetsInFinalize())
>>>>>>
>>>>>>
>>>>>> and I like to get deadletter outputs when my timestamp extraction
>>>>>> fails.
>>>>>>
>>>>>> Best,
>>>>>> Tobi
>>>>>>
>>>>>>


Re: KafkaIO - Deadletter output

2018-10-23 Thread Raghu Angadi
User can read serialized bytes from KafkaIO and deserialize explicitly in a
ParDo, which gives complete control on how to handle record errors. This is
I would do if I need to in my pipeline.

If there is a transform in Beam that does this, it could be convenient for
users in many such scenarios. This is simpler than each source supporting
it explicitly.

On Tue, Oct 23, 2018 at 8:03 PM Chamikara Jayalath 
wrote:

> Given that KafkaIO uses UnboundeSource framework, this is probably not
> something that can easily be supported. We might be able to support similar
> features when we have Kafka on top of Splittable DoFn though.
>
So feel free to create a feature request JIRA for this.
>
> Thanks,
> Cham
>
> On Tue, Oct 23, 2018 at 7:43 PM Kenneth Knowles  wrote:
>
>> This is a great question. I've added the dev list to be sure it gets
>> noticed by whoever may know best.
>>
>> Kenn
>>
>> On Tue, Oct 23, 2018 at 2:05 AM Kaymak, Tobias 
>> wrote:
>>
>>>
>>> Hi,
>>>
>>> Is there a way to get a Deadletter Output from a pipeline that uses a
>>> KafkaIO
>>> connector for it's input? As `TimestampPolicyFactory.withTimestampFn()`
>>> takes
>>> only a SerializableFunction and not a ParDo, how would I be able to
>>> produce a
>>> Deadletter output from it?
>>>
>>> I have the following pipeline defined that reads from a KafkaIO input:
>>>
>>> pipeline.apply(
>>>   KafkaIO.read()
>>> .withBootstrapServers(bootstrap)
>>> .withTopics(topics)
>>> .withKeyDeserializer(StringDeserializer.class)
>>> .withValueDeserializer(ConfigurableDeserializer.class)
>>> .updateConsumerProperties(
>>> ImmutableMap.of(InputMessagesConfig.CONFIG_PROPERTY_NAME,
>>> inputMessagesConfig))
>>> .updateConsumerProperties(ImmutableMap.of("auto.offset.reset",
>>> "earliest"))
>>> .updateConsumerProperties(ImmutableMap.of("group.id",
>>> "beam-consumers"))
>>> .updateConsumerProperties(ImmutableMap.of("enable.auto.commit",
>>> "true"))
>>> .withTimestampPolicyFactory(
>>> TimestampPolicyFactory.withTimestampFn(
>>> new MessageTimestampExtractor(inputMessagesConfig)))
>>> .withReadCommitted()
>>> .commitOffsetsInFinalize())
>>>
>>>
>>> and I like to get deadletter outputs when my timestamp extraction fails.
>>>
>>> Best,
>>> Tobi
>>>
>>>


Re: Problem with KafkaIO

2018-10-22 Thread Raghu Angadi
Thanks for confirming the fix.

On Mon, Oct 22, 2018 at 7:48 AM Eduardo Soldera <
eduardo.sold...@arquivei.com.br> wrote:

> Hi Raghu, Dataflow throws an exception if Kafka fails now and it recovers
> after Kafka is available.
>
> Regards
>
> Em sex, 19 de out de 2018 às 14:01, Raghu Angadi 
> escreveu:
>
>> On Fri, Oct 19, 2018 at 6:54 AM Eduardo Soldera <
>> eduardo.sold...@arquivei.com.br> wrote:
>>
>>> Hi Raghu, just a quick update. We were waiting for Spotify's Scio to
>>> update to Beam 2.7. We've just deployed the pipeling sucessfully. Just for
>>> letting you know, I tried to use the workaround code snipped, but Dataflow
>>> wouldn't recover after a Kafka unavailability.
>>>
>>
>> Thanks for the update. The workaround helps only if KafkaClient itself
>> can recover when try to read again. I guess some of those exceptions are
>> are not recoverable.
>>
>> Please let us know how the actual fix works.
>>
>> Thanks.
>> Raghu.
>>
>>
>>>
>>> Thanks for your help.
>>>
>>> Regards
>>>
>>> Em qua, 19 de set de 2018 às 15:37, Raghu Angadi 
>>> escreveu:
>>>
>>>>
>>>>
>>>> On Wed, Sep 19, 2018 at 11:24 AM Juan Carlos Garcia <
>>>> jcgarc...@gmail.com> wrote:
>>>>
>>>>> Sorry I hit the send button to fast... The error occurs in the worker.
>>>>>
>>>>
>>>> Np. Just one more comment on it: it is a very important
>>>> design/correctness decision to for runner to decide how to handle
>>>> persistent errors in a streaming pipeline. Dataflow keeps failing since
>>>> there is no solution to restart a pipeline from scratch without losing
>>>> exactly-once guarantees. It lets user decide if the pipeline needs to be
>>>> 'upgraded'.
>>>>
>>>> Raghu.
>>>>
>>>>>
>>>>> Juan Carlos Garcia  schrieb am Mi., 19. Sep.
>>>>> 2018, 20:22:
>>>>>
>>>>>> Sorry for hijacking the thread, we are running Spark on top of Yarn,
>>>>>> yarn retries multiple times until it reachs it max attempt and then gives
>>>>>> up.
>>>>>>
>>>>>> Raghu Angadi  schrieb am Mi., 19. Sep. 2018,
>>>>>> 18:58:
>>>>>>
>>>>>>> On Wed, Sep 19, 2018 at 7:26 AM Juan Carlos Garcia <
>>>>>>> jcgarc...@gmail.com> wrote:
>>>>>>>
>>>>>>>> Don't know if its related, but we have seen our pipeline dying
>>>>>>>> (using SparkRunner) when there is problem with Kafka  (network
>>>>>>>> interruptions), errors like:
>>>>>>>> org.apache.kafka.common.errors.TimeoutException: Timeout expired while
>>>>>>>> fetching topic metadata
>>>>>>>>
>>>>>>>> Maybe this will fix it as well, thanks Raghu for the hint about
>>>>>>>> *withConsumerFactoryFn.*
>>>>>>>>
>>>>>>>
>>>>>>> Wouldn't that be retried by the SparkRunner if it happens on the
>>>>>>> worker? or does it happen while launching the pipeline on the client?
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Sep 19, 2018 at 3:29 PM Eduardo Soldera <
>>>>>>>> eduardo.sold...@arquivei.com.br> wrote:
>>>>>>>>
>>>>>>>>> Hi Raghu, thank you.
>>>>>>>>>
>>>>>>>>> I'm not sure though what to pass as an argument:
>>>>>>>>>
>>>>>>>>> KafkaIO.read[String,String]()
>>>>>>>>>   .withBootstrapServers(server)
>>>>>>>>>   .withTopic(topic)
>>>>>>>>>   .withKeyDeserializer(classOf[StringDeserializer])
>>>>>>>>>   .withValueDeserializer(classOf[StringDeserializer])
>>>>>>>>>   .withConsumerFactoryFn(new 
>>>>>>>>> KafkaExecutor.ConsumerFactoryFn())
>>>>>>>>>   .updateConsumerProperties(properties)
>>>>>>>>>   .commitOffsetsInFinalize()
>>>>>>>>>   .withoutMetadata()

Re: write to a kafka topic that is set in data

2018-10-19 Thread Raghu Angadi
It will be a good starter feature for someone interested in Beam & Kafka.
Writer is very simple in Beam. It is little more than a ParDo.

On Fri, Oct 19, 2018 at 3:37 PM Dmitry Minaev  wrote:

> Lukasz, I appreciate the quick response and filing the JIRA ticket. Thanks
> for the suggestion, unfortunately, I don't have a fixed number of topics.
> Still, we'll probably use your approach for a limited number of topics
> until the functionality is added, thank you!
>
> Thanks,
> Dmitry
>
> On Fri, Oct 19, 2018 at 2:53 PM Lukasz Cwik  wrote:
>
>> If there are a fixed number of topics, you could partition your write by
>> structuring your pipeline as such:
>> ParDo(PartitionByTopic) > KafkaIO.write(topicA)
>> \---> KafkaIO.write(topicB)
>> \---> KafkaIO.write(...)
>>
>> There is no support currently for writing to Kafka dynamically based upon
>> a destination that is part of the data.
>> I filed https://issues.apache.org/jira/browse/BEAM-5798 for the issue.
>>
>> On Fri, Oct 19, 2018 at 2:05 PM mina...@gmail.com 
>> wrote:
>>
>>> Hi guys!!
>>>
>>> I'm trying to find a way to write to a Kafka topic using KafkaIO.write()
>>> But I need to be able to get topic name dynamically based on the data
>>> received. For example, I would like to send data for one tenant to topic
>>> "data_feed_1" and for another tenant to "topic data_feed_999".
>>> I'm coming from Flink where it's possible via
>>> KeyedSerializationSchema.getTargetTopic().
>>> Is there anything similar in KafkaIO?
>>>
>>> Thanks,
>>> Dmitry
>>>
>> --
>
> --
> Dmitry
>


Re: Ingesting from KafkaIO into HDFS, problems with checkpoint and data lost on top of yarn + spark

2018-10-02 Thread Raghu Angadi
> I am curious about what you mentioned (30min of 1hr window would be
lost), just a noob question, why?

Say you have 1 hour windowing in your pipeline. The aggregation is emitted
at the end of the window. 30 minutes into the window, there might many
incoming messages processed already. Where should the information about
this partially processed window be stored? Managing this kind of state is
an important part of runner. It is checkpointed on some persistent storage.
If you restart the pipeline at that time, new job may not have access to
stage from previous job, so you would lose 30 minutes worth of messages.

The reliability has to be provided by the runner. I don't know much about
details on SparkRunner, but you can certainly try another runner like Flink
or Dataflow.

Raghu.
On Tue, Oct 2, 2018 at 8:09 AM Juan Carlos Garcia 
wrote:

> Thanks for you inputs on this matter, by data loss i meant data that is on
> Kafka but was not written in HDFS due to restarting the pipeline
> (SparkRunner) or because it failed due to connectivity and was kill by Yarn
> and when we restarted the pipeline those records were skipped.
>
> I am curious about what you mentioned (30min of 1hr window would be lost),
> just a noob question, why?
>
> > read from any source and write on any supported sink.
>
> I have no doubt about it.
>
> With a multi stage pipeline (where we sort, manipulate, group the data) my
> purpose is to reliable sink data to HDFS, regardless of any interruption on
> the pipeline, like other ingestion library do in "batch" (Camus / gobbling
> from LinkedIn)
>
> During this period we are using HDFS as sink with Spark with a window to
> avoid hitting HDFS badly, do you recommend me to use Flink instead (at
> least for this requirement)?
>
> Thanks in advanced
>
> Raghu Angadi  schrieb am Di., 2. Okt. 2018, 08:25:
>
>> This is mostly a question about SparkRunner and to certain extent FileIO.
>> You might want to elaborate a bit more what you mean by data loss. In most
>> cases, restarting a pipeline from scratch loses checkpointed state from
>> previous job (e.g. first 30 minutes of a 1 hour window would be lost),
>> unless you have a way to restart from a 'snapshot' of the pipeline (i.e.
>> starting from 'savepoint' in Flink or 'updating' a pipeline in Dataflow).
>>
>> Regd 'commitOnFinalize()' in KafkaIO, it runs 'soon after' the
>> corresponding messages are processed/checkpointed. In the case of Spark and
>> Dataflow, it would be after the messages pass through the first stage of
>> the pipeline.
>>
>> > Please advice if this usecase (data ingestion to hdfs) is something
>> beam could achieve without lossing data from KafkaIO.
>> Yes, reading from any supported source and writing to any supported sink
>> is supported. Otherwise, it would be a bug.
>>
>> On Mon, Oct 1, 2018 at 10:25 PM Juan Carlos Garcia 
>> wrote:
>>
>>> Hi folks we are running a pipeline which as the subject says the we are
>>> having issues with data lost.
>>>
>>> Using KafkaIO (2.0.4 due to the version of our brokers) with
>>> commitOnFinalize, we would like to understand how this finalize work
>>> together with a FileIO.
>>>
>>> I studied the KafkaIO and saw that the records are committed to kafka
>>> inside the consumerPollLoop method only when a checkpoint is produced, but
>>> when is this checkpoint produced?, how does it cope with windowed data and
>>> a FileIO to produces files?
>>>
>>> When running with spark our batchInterval is 30secs, and the pipeline
>>> have a fixed-window of 1hr for FileIO to write to HDFS and we are
>>> constantly restarting the pipeline (1 or 3 times a day, or yarn reach it
>>> maximum restart attempt and then it kill it completely due to networks
>>> interruption ), however we have detected we have missing data on HDFS.
>>>
>>> Initially we were running without specifying a checkpoint directory
>>> (SparkRunner) , and we found that on each deployment a random directory was
>>> generated under /tmp, recently we started to uses a fixed directory for
>>> checkpoint (via - - checkpointDir on the spark runner), but still we have
>>> doubts that this will completely solve our data lost problems when
>>> restarting the pipeline multiple times a day (or is it our assumption
>>> incorrect? ).
>>>
>>> Please advice if this usecase (data ingestion to hdfs) is something beam
>>> could achieve without lossing data from KafkaIO.
>>>
>>> Thanks
>>> JC
>>>
>>>
>>>
>>>


Re: Ingesting from KafkaIO into HDFS, problems with checkpoint and data lost on top of yarn + spark

2018-10-02 Thread Raghu Angadi
This is mostly a question about SparkRunner and to certain extent FileIO.
You might want to elaborate a bit more what you mean by data loss. In most
cases, restarting a pipeline from scratch loses checkpointed state from
previous job (e.g. first 30 minutes of a 1 hour window would be lost),
unless you have a way to restart from a 'snapshot' of the pipeline (i.e.
starting from 'savepoint' in Flink or 'updating' a pipeline in Dataflow).

Regd 'commitOnFinalize()' in KafkaIO, it runs 'soon after' the
corresponding messages are processed/checkpointed. In the case of Spark and
Dataflow, it would be after the messages pass through the first stage of
the pipeline.

> Please advice if this usecase (data ingestion to hdfs) is something beam
could achieve without lossing data from KafkaIO.
Yes, reading from any supported source and writing to any supported sink is
supported. Otherwise, it would be a bug.

On Mon, Oct 1, 2018 at 10:25 PM Juan Carlos Garcia 
wrote:

> Hi folks we are running a pipeline which as the subject says the we are
> having issues with data lost.
>
> Using KafkaIO (2.0.4 due to the version of our brokers) with
> commitOnFinalize, we would like to understand how this finalize work
> together with a FileIO.
>
> I studied the KafkaIO and saw that the records are committed to kafka
> inside the consumerPollLoop method only when a checkpoint is produced, but
> when is this checkpoint produced?, how does it cope with windowed data and
> a FileIO to produces files?
>
> When running with spark our batchInterval is 30secs, and the pipeline have
> a fixed-window of 1hr for FileIO to write to HDFS and we are constantly
> restarting the pipeline (1 or 3 times a day, or yarn reach it maximum
> restart attempt and then it kill it completely due to networks interruption
> ), however we have detected we have missing data on HDFS.
>
> Initially we were running without specifying a checkpoint directory
> (SparkRunner) , and we found that on each deployment a random directory was
> generated under /tmp, recently we started to uses a fixed directory for
> checkpoint (via - - checkpointDir on the spark runner), but still we have
> doubts that this will completely solve our data lost problems when
> restarting the pipeline multiple times a day (or is it our assumption
> incorrect? ).
>
> Please advice if this usecase (data ingestion to hdfs) is something beam
> could achieve without lossing data from KafkaIO.
>
> Thanks
> JC
>
>
>
>


Re: Kafka Avro Schema Registry Support

2018-09-28 Thread Raghu Angadi
Looks like your producer writing a Avro specfic records.

Can you read the records using bundled console consumer? I think it will be
simpler for you to get it returning valid records and use the same
deserializer config with your KafkaIO reader.

On Fri, Sep 28, 2018 at 9:33 AM Vishwas Bm  wrote:

> Hi Raghu,
>
> Thanks for the response.  We are now trying with GenericAvroDeserializer
> but still seeing issues.
> We have a producer which sends messages to kafka in format
> .
>
> Below is the code snippet, we have used at Beam KafkaIo.
>
>  org.apache.avro.Schema schema = null;
> try {
> schema = new org.apache.avro.Schema.Parser().parse(new
> File("Schema path"));
> } catch (Exception e) {
> e.printStackTrace();
> }
> KafkaIO.Read kafkaIoRead = KafkaIO. GenericRecord>read()
>
> .withBootstrapServers(bootstrapServerUrl).withTopic(topicName)
> .withKeyDeserializer(StringDeserializer.class)
>
> .withValueDeserializerAndCoder(GenericAvroDeserializer.class,
> AvroCoder.of(schema))
>
> .updateConsumerProperties(ImmutableMap.of("schema.registry.url", schemaUrl))
> .withTimestampPolicyFactory((tp, prevWatermark) -> new
> KafkaCustomTimestampPolicy(maxDelay,
> timestampInfo, prevWatermark));
>
> Below is the error seen,
>
> Caused by:
> avro.shaded.com.google.common.util.concurrent.UncheckedExecutionException:
> org.apache.avro.AvroRuntimeException: Not a Specific class: interface
> org.apache.avro.generic.GenericRecord
> at
> avro.shaded.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2234)
> at
> avro.shaded.com.google.common.cache.LocalCache.get(LocalCache.java:3965)
> at
> avro.shaded.com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3969)
> at
> avro.shaded.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4829)
> at
> org.apache.avro.specific.SpecificData.getSchema(SpecificData.java:225)
> ... 8 more
> Caused by: org.apache.avro.AvroRuntimeException: Not a Specific class:
> interface org.apache.avro.generic.GenericRecord
> at
> org.apache.avro.specific.SpecificData.createSchema(SpecificData.java:285)
> at
> org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:594)
> at
> org.apache.avro.specific.SpecificData$2.load(SpecificData.java:218)
> at
> org.apache.avro.specific.SpecificData$2.load(SpecificData.java:215)
> at
> avro.shaded.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3568)
> at
> avro.shaded.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2350)
> at
> avro.shaded.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2313)
>     at
> avro.shaded.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2228)
>
>
> Can you provide some pointers on this.
>
>
> *Thanks & Regards,*
>
> *Vishwas *
>
>
>
> On Fri, Sep 28, 2018 at 3:12 AM Raghu Angadi  wrote:
>
>> It is a compilation error due to type mismatch for value type.
>>
>> Please match key and value types for KafkaIO reader. I.e. if you have
>> KafkaIO.read().,  'withValueDeserializer()' needs a
>> class object which extends 'Deserializer'. Since
>> KafkaAvroDeserializer extends 'Deserializer', so your ValueType
>> needs to be Object, instead of String.
>>
>> Btw, it might be better to use GenericAvroDeseiralizer or
>> SpecificAvroDeserializer from the same package.
>>
>>
>> On Thu, Sep 27, 2018 at 10:31 AM Vishwas Bm  wrote:
>>
>>>
>>> Hi Raghu,
>>>
>>> The deserializer is provided by confluent
>>> *io.confluent.kafka.serializers* package.
>>>
>>> When we set valueDeserializer as  KafkaAvroDeserializer.  We are getting
>>> below error:
>>>The method withValueDeserializer(Class>> Deserializer>) in the type KafkaIO.Read is not
>>> applicable for the arguments
>>>  (Class)
>>>
>>> From the error, it looks like beam does not support this deserializer.
>>> Also we wanted to use schemaRegistry from confluent, is this supported
>>> in Beam ?
>>>
>>>
>>> *Thanks & Regards,*
>>> *Vishwas *
>>>
>>>
>>> On Thu, Sep 27, 2018 at 10:28 PM Raghu Angadi 
>>> wrote:
>>>
>>>> You can set key/value deserializers :
>>>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/

Re: Kafka Avro Schema Registry Support

2018-09-27 Thread Raghu Angadi
It is a compilation error due to type mismatch for value type.

Please match key and value types for KafkaIO reader. I.e. if you have
KafkaIO.read().,  'withValueDeserializer()' needs a
class object which extends 'Deserializer'. Since
KafkaAvroDeserializer extends 'Deserializer', so your ValueType
needs to be Object, instead of String.

Btw, it might be better to use GenericAvroDeseiralizer or
SpecificAvroDeserializer from the same package.


On Thu, Sep 27, 2018 at 10:31 AM Vishwas Bm  wrote:

>
> Hi Raghu,
>
> The deserializer is provided by confluent *io.confluent.kafka.serializers*
> package.
>
> When we set valueDeserializer as  KafkaAvroDeserializer.  We are getting
> below error:
>The method withValueDeserializer(Class>)
> in the type KafkaIO.Read is not applicable for the arguments
>  (Class)
>
> From the error, it looks like beam does not support this deserializer.
> Also we wanted to use schemaRegistry from confluent, is this supported in
> Beam ?
>
>
> *Thanks & Regards,*
> *Vishwas *
>
>
> On Thu, Sep 27, 2018 at 10:28 PM Raghu Angadi  wrote:
>
>> You can set key/value deserializers :
>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java#L101
>> What are the errors you see?
>>
>> Also note that Beam includes AvroCoder for handling Avro records in Beam.
>>
>> On Thu, Sep 27, 2018 at 6:05 AM rahul patwari 
>> wrote:
>>
>>> Hi,
>>>
>>> We have a usecase to read data from Kafka serialized with
>>> KafkaAvroSerializer and schema is present in Schema Registry.
>>>
>>> When we are trying to use ValueDeserializer as
>>> io.confluent.kafka.serializers.KafkaAvroDeserializer to get GenericRecord,
>>> we are seeing errors.
>>>
>>> Does KafkaIO.read() supports reading from schema registry and using
>>> confluent KafkaAvroSerDe?
>>>
>>> Regards,
>>> Rahul
>>>
>>


Re: KafkaIO needs access to the brokers even before the pipeline reach the worker

2018-09-19 Thread Raghu Angadi
This access is needed in order to determine number of partitions for a
topic. If you know number of partitions already, you can provide a list of
partitions manually and that avoids accessing Kafka cluster on the client.
See example usage at [1].

It is feasible to avoid this access with some deterministic assignment of
partitions on the workers for each input split. We might look into it if
'withTopicPartitions()' does not help in most of these cases.

[1]:
https://github.com/apache/beam/blob/60e0a22ea95921636c392b5aae77cb48196dd700/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java#L380

On Wed, Sep 19, 2018 at 11:59 AM Juan Carlos Garcia 
wrote:

> Hi folks, we have a pipeline for Dataflow and our Google cloud environment
> has a private network (where the pipeline should run, this network
> interconnect via an IP-sec to an AWS environment where the Kafka brokers
> are running).
>
> We have found that in order to be able to submit the pipeline we have to
> do it from a machine that has access to the Kafka brokers.
>
> Is there a way to avoid that?
>
> Why KafkaIO cannot defer the communication to the brokers after the
> pipeline its on the worker node?
>
> Thanks and regards,
> JC
>


Re: Problem with KafkaIO

2018-09-19 Thread Raghu Angadi
On Wed, Sep 19, 2018 at 11:24 AM Juan Carlos Garcia 
wrote:

> Sorry I hit the send button to fast... The error occurs in the worker.
>

Np. Just one more comment on it: it is a very important design/correctness
decision to for runner to decide how to handle persistent errors in a
streaming pipeline. Dataflow keeps failing since there is no solution to
restart a pipeline from scratch without losing exactly-once guarantees. It
lets user decide if the pipeline needs to be 'upgraded'.

Raghu.

>
> Juan Carlos Garcia  schrieb am Mi., 19. Sep. 2018,
> 20:22:
>
>> Sorry for hijacking the thread, we are running Spark on top of Yarn, yarn
>> retries multiple times until it reachs it max attempt and then gives up.
>>
>> Raghu Angadi  schrieb am Mi., 19. Sep. 2018, 18:58:
>>
>>> On Wed, Sep 19, 2018 at 7:26 AM Juan Carlos Garcia 
>>> wrote:
>>>
>>>> Don't know if its related, but we have seen our pipeline dying (using
>>>> SparkRunner) when there is problem with Kafka  (network interruptions),
>>>> errors like:  org.apache.kafka.common.errors.TimeoutException: Timeout
>>>> expired while fetching topic metadata
>>>>
>>>> Maybe this will fix it as well, thanks Raghu for the hint about
>>>> *withConsumerFactoryFn.*
>>>>
>>>
>>> Wouldn't that be retried by the SparkRunner if it happens on the worker?
>>> or does it happen while launching the pipeline on the client?
>>>
>>>
>>>
>>>>
>>>>
>>>>
>>>> On Wed, Sep 19, 2018 at 3:29 PM Eduardo Soldera <
>>>> eduardo.sold...@arquivei.com.br> wrote:
>>>>
>>>>> Hi Raghu, thank you.
>>>>>
>>>>> I'm not sure though what to pass as an argument:
>>>>>
>>>>> KafkaIO.read[String,String]()
>>>>>   .withBootstrapServers(server)
>>>>>   .withTopic(topic)
>>>>>   .withKeyDeserializer(classOf[StringDeserializer])
>>>>>   .withValueDeserializer(classOf[StringDeserializer])
>>>>>   .withConsumerFactoryFn(new 
>>>>> KafkaExecutor.ConsumerFactoryFn())
>>>>>   .updateConsumerProperties(properties)
>>>>>   .commitOffsetsInFinalize()
>>>>>   .withoutMetadata()
>>>>>
>>>>>
>>>>> Regards
>>>>>
>>>>>
>>>>> Em ter, 18 de set de 2018 às 21:15, Raghu Angadi 
>>>>> escreveu:
>>>>>
>>>>>> Hi Eduardo,
>>>>>>
>>>>>> There another work around you can try without having to wait for
>>>>>> 2.7.0 release: Use a wrapper to catch exception from KafkaConsumer#poll()
>>>>>> and pass the wrapper to withConsumerFactoryFn() for KafkIO reader [1].
>>>>>>
>>>>>> Using something like (such a wrapper is used in KafkasIO tests [2]):
>>>>>> private static class ConsumerFactoryFn
>>>>>> implements SerializableFunction,
>>>>>> Consumer> {
>>>>>>   @Override
>>>>>> public Consumer apply(Map config)
>>>>>> {
>>>>>>   return new KafkaConsumer(config) {
>>>>>>   @Override
>>>>>>   public ConsumerRecords poll(long timeout) {
>>>>>>   // work around for BEAM-5375
>>>>>>   while (true) {
>>>>>>   try {
>>>>>> return super.poll(timeout);
>>>>>>  } catch (Exception e) {
>>>>>> // LOG & sleep for sec
>>>>>> }
>>>>>>   }
>>>>>>}
>>>>>> }
>>>>>> }
>>>>>>
>>>>>> [1]:
>>>>>> https://github.com/apache/beam/blob/release-2.4.0/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java#L417
>>>>>> [2]:
>>>>>> https://github.com/apache/beam/blob/release-2.4.0/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java#L261
>>>>>>
>>>>>> On Tue, Sep 18, 2018 at 5:49 AM Eduardo Soldera <
>>>>>> eduardo.sold...@arquivei.com.br> wrote:
>>>>>>
>>>>>>> Hi Raghu, we're not sure how long the network was down. According to
>>>>>

Re: Problem with KafkaIO

2018-09-19 Thread Raghu Angadi
On Wed, Sep 19, 2018 at 7:26 AM Juan Carlos Garcia 
wrote:

> Don't know if its related, but we have seen our pipeline dying (using
> SparkRunner) when there is problem with Kafka  (network interruptions),
> errors like:  org.apache.kafka.common.errors.TimeoutException: Timeout
> expired while fetching topic metadata
>
> Maybe this will fix it as well, thanks Raghu for the hint about
> *withConsumerFactoryFn.*
>

Wouldn't that be retried by the SparkRunner if it happens on the worker? or
does it happen while launching the pipeline on the client?



>
>
>
> On Wed, Sep 19, 2018 at 3:29 PM Eduardo Soldera <
> eduardo.sold...@arquivei.com.br> wrote:
>
>> Hi Raghu, thank you.
>>
>> I'm not sure though what to pass as an argument:
>>
>> KafkaIO.read[String,String]()
>>   .withBootstrapServers(server)
>>   .withTopic(topic)
>>   .withKeyDeserializer(classOf[StringDeserializer])
>>   .withValueDeserializer(classOf[StringDeserializer])
>>   .withConsumerFactoryFn(new 
>> KafkaExecutor.ConsumerFactoryFn())
>>   .updateConsumerProperties(properties)
>>   .commitOffsetsInFinalize()
>>   .withoutMetadata()
>>
>>
>> Regards
>>
>>
>> Em ter, 18 de set de 2018 às 21:15, Raghu Angadi 
>> escreveu:
>>
>>> Hi Eduardo,
>>>
>>> There another work around you can try without having to wait for 2.7.0
>>> release: Use a wrapper to catch exception from KafkaConsumer#poll() and
>>> pass the wrapper to withConsumerFactoryFn() for KafkIO reader [1].
>>>
>>> Using something like (such a wrapper is used in KafkasIO tests [2]):
>>> private static class ConsumerFactoryFn
>>> implements SerializableFunction,
>>> Consumer> {
>>>   @Override
>>> public Consumer apply(Map config) {
>>>   return new KafkaConsumer(config) {
>>>   @Override
>>>   public ConsumerRecords poll(long timeout) {
>>>   // work around for BEAM-5375
>>>   while (true) {
>>>   try {
>>> return super.poll(timeout);
>>>  } catch (Exception e) {
>>> // LOG & sleep for sec
>>> }
>>>   }
>>>}
>>> }
>>> }
>>>
>>> [1]:
>>> https://github.com/apache/beam/blob/release-2.4.0/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java#L417
>>> [2]:
>>> https://github.com/apache/beam/blob/release-2.4.0/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java#L261
>>>
>>> On Tue, Sep 18, 2018 at 5:49 AM Eduardo Soldera <
>>> eduardo.sold...@arquivei.com.br> wrote:
>>>
>>>> Hi Raghu, we're not sure how long the network was down. According to
>>>> the logs no longer than one minute. A 30 second shutdown would work for the
>>>> tests.
>>>>
>>>> Regards
>>>>
>>>> Em sex, 14 de set de 2018 às 21:41, Raghu Angadi 
>>>> escreveu:
>>>>
>>>>> Thanks. I could repro myself as well. How long was the network down?
>>>>>
>>>>> Trying to get the fix into 2.7 RC2.
>>>>>
>>>>> On Fri, Sep 14, 2018 at 12:25 PM Eduardo Soldera <
>>>>> eduardo.sold...@arquivei.com.br> wrote:
>>>>>
>>>>>> Just to make myself clear, I'm not sure how to use the patch but if
>>>>>> you could send us some guidance would be great.
>>>>>>
>>>>>> Em sex, 14 de set de 2018 às 16:24, Eduardo Soldera <
>>>>>> eduardo.sold...@arquivei.com.br> escreveu:
>>>>>>
>>>>>>> Hi Raghu, yes, it is feasible, would you do that for us? I'm not
>>>>>>> sure how we'd use the patch. We're using SBT and Spotify's Scio with 
>>>>>>> Scala.
>>>>>>>
>>>>>>> Thanks
>>>>>>>
>>>>>>> Em sex, 14 de set de 2018 às 16:07, Raghu Angadi 
>>>>>>> escreveu:
>>>>>>>
>>>>>>>> Is is feasible for you to verify the fix in your dev job? I can
>>>>>>>> make a patch against Beam 2.4 branch if you like.
>>>>>>>>
>>>>>>>> Raghu.
>>>>>>>>
>>>>>>>> On Fri, Sep 14, 2018 at 11:14 AM Eduardo Solder

Re: Problem with KafkaIO

2018-09-19 Thread Raghu Angadi
You would pass a SerializableFunction (with whatever args required to
create the object of this functor, if any). E.g. ConsumerFactoryFn()
defined in my pseudo code below does not need to any args. So it would be :

KafkaIO.read()
   ...
   .withConsumerFactoryFn(new ConsumerFactoryFn()).
---






On Wed, Sep 19, 2018 at 6:29 AM Eduardo Soldera <
eduardo.sold...@arquivei.com.br> wrote:

> Hi Raghu, thank you.
>
> I'm not sure though what to pass as an argument:
>
> KafkaIO.read[String,String]()
>   .withBootstrapServers(server)
>   .withTopic(topic)
>   .withKeyDeserializer(classOf[StringDeserializer])
>   .withValueDeserializer(classOf[StringDeserializer])
>   .withConsumerFactoryFn(new 
> KafkaExecutor.ConsumerFactoryFn())
>   .updateConsumerProperties(properties)
>   .commitOffsetsInFinalize()
>   .withoutMetadata()
>
>
> Regards
>
>
> Em ter, 18 de set de 2018 às 21:15, Raghu Angadi 
> escreveu:
>
>> Hi Eduardo,
>>
>> There another work around you can try without having to wait for 2.7.0
>> release: Use a wrapper to catch exception from KafkaConsumer#poll() and
>> pass the wrapper to withConsumerFactoryFn() for KafkIO reader [1].
>>
>> Using something like (such a wrapper is used in KafkasIO tests [2]):
>> private static class ConsumerFactoryFn
>> implements SerializableFunction,
>> Consumer> {
>>   @Override
>> public Consumer apply(Map config) {
>>   return new KafkaConsumer(config) {
>>   @Override
>>   public ConsumerRecords poll(long timeout) {
>>   // work around for BEAM-5375
>>   while (true) {
>>   try {
>> return super.poll(timeout);
>>  } catch (Exception e) {
>> // LOG & sleep for sec
>> }
>>   }
>>}
>> }
>> }
>>
>> [1]:
>> https://github.com/apache/beam/blob/release-2.4.0/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java#L417
>> [2]:
>> https://github.com/apache/beam/blob/release-2.4.0/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java#L261
>>
>> On Tue, Sep 18, 2018 at 5:49 AM Eduardo Soldera <
>> eduardo.sold...@arquivei.com.br> wrote:
>>
>>> Hi Raghu, we're not sure how long the network was down. According to the
>>> logs no longer than one minute. A 30 second shutdown would work for the
>>> tests.
>>>
>>> Regards
>>>
>>> Em sex, 14 de set de 2018 às 21:41, Raghu Angadi 
>>> escreveu:
>>>
>>>> Thanks. I could repro myself as well. How long was the network down?
>>>>
>>>> Trying to get the fix into 2.7 RC2.
>>>>
>>>> On Fri, Sep 14, 2018 at 12:25 PM Eduardo Soldera <
>>>> eduardo.sold...@arquivei.com.br> wrote:
>>>>
>>>>> Just to make myself clear, I'm not sure how to use the patch but if
>>>>> you could send us some guidance would be great.
>>>>>
>>>>> Em sex, 14 de set de 2018 às 16:24, Eduardo Soldera <
>>>>> eduardo.sold...@arquivei.com.br> escreveu:
>>>>>
>>>>>> Hi Raghu, yes, it is feasible, would you do that for us? I'm not sure
>>>>>> how we'd use the patch. We're using SBT and Spotify's Scio with Scala.
>>>>>>
>>>>>> Thanks
>>>>>>
>>>>>> Em sex, 14 de set de 2018 às 16:07, Raghu Angadi 
>>>>>> escreveu:
>>>>>>
>>>>>>> Is is feasible for you to verify the fix in your dev job? I can make
>>>>>>> a patch against Beam 2.4 branch if you like.
>>>>>>>
>>>>>>> Raghu.
>>>>>>>
>>>>>>> On Fri, Sep 14, 2018 at 11:14 AM Eduardo Soldera <
>>>>>>> eduardo.sold...@arquivei.com.br> wrote:
>>>>>>>
>>>>>>>> Hi Raghu, thank you very much for the pull request.
>>>>>>>> We'll wait for the 2.7 Beam release.
>>>>>>>>
>>>>>>>> Regards!
>>>>>>>>
>>>>>>>> Em qui, 13 de set de 2018 às 18:19, Raghu Angadi <
>>>>>>>> rang...@google.com> escreveu:
>>>>>>>>
>>>>>>>>> Fix: https://github.com/apache/beam/pull/6391
>>>>>>>>>

Re: Problem with KafkaIO

2018-09-18 Thread Raghu Angadi
Hi Eduardo,

There another work around you can try without having to wait for 2.7.0
release: Use a wrapper to catch exception from KafkaConsumer#poll() and
pass the wrapper to withConsumerFactoryFn() for KafkIO reader [1].

Using something like (such a wrapper is used in KafkasIO tests [2]):
private static class ConsumerFactoryFn
implements SerializableFunction,
Consumer> {
  @Override
public Consumer apply(Map config) {
  return new KafkaConsumer(config) {
  @Override
  public ConsumerRecords poll(long timeout) {
  // work around for BEAM-5375
  while (true) {
  try {
return super.poll(timeout);
 } catch (Exception e) {
// LOG & sleep for sec
}
  }
   }
}
}

[1]:
https://github.com/apache/beam/blob/release-2.4.0/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java#L417
[2]:
https://github.com/apache/beam/blob/release-2.4.0/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java#L261

On Tue, Sep 18, 2018 at 5:49 AM Eduardo Soldera <
eduardo.sold...@arquivei.com.br> wrote:

> Hi Raghu, we're not sure how long the network was down. According to the
> logs no longer than one minute. A 30 second shutdown would work for the
> tests.
>
> Regards
>
> Em sex, 14 de set de 2018 às 21:41, Raghu Angadi 
> escreveu:
>
>> Thanks. I could repro myself as well. How long was the network down?
>>
>> Trying to get the fix into 2.7 RC2.
>>
>> On Fri, Sep 14, 2018 at 12:25 PM Eduardo Soldera <
>> eduardo.sold...@arquivei.com.br> wrote:
>>
>>> Just to make myself clear, I'm not sure how to use the patch but if you
>>> could send us some guidance would be great.
>>>
>>> Em sex, 14 de set de 2018 às 16:24, Eduardo Soldera <
>>> eduardo.sold...@arquivei.com.br> escreveu:
>>>
>>>> Hi Raghu, yes, it is feasible, would you do that for us? I'm not sure
>>>> how we'd use the patch. We're using SBT and Spotify's Scio with Scala.
>>>>
>>>> Thanks
>>>>
>>>> Em sex, 14 de set de 2018 às 16:07, Raghu Angadi 
>>>> escreveu:
>>>>
>>>>> Is is feasible for you to verify the fix in your dev job? I can make a
>>>>> patch against Beam 2.4 branch if you like.
>>>>>
>>>>> Raghu.
>>>>>
>>>>> On Fri, Sep 14, 2018 at 11:14 AM Eduardo Soldera <
>>>>> eduardo.sold...@arquivei.com.br> wrote:
>>>>>
>>>>>> Hi Raghu, thank you very much for the pull request.
>>>>>> We'll wait for the 2.7 Beam release.
>>>>>>
>>>>>> Regards!
>>>>>>
>>>>>> Em qui, 13 de set de 2018 às 18:19, Raghu Angadi 
>>>>>> escreveu:
>>>>>>
>>>>>>> Fix: https://github.com/apache/beam/pull/6391
>>>>>>>
>>>>>>> On Wed, Sep 12, 2018 at 3:30 PM Raghu Angadi 
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Filed BEAM-5375 <https://issues.apache.org/jira/browse/BEAM-5375>.
>>>>>>>> I will fix it later this week.
>>>>>>>>
>>>>>>>> On Wed, Sep 12, 2018 at 12:16 PM Raghu Angadi 
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Sep 12, 2018 at 12:11 PM Raghu Angadi 
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Thanks for the job id, I looked at the worker logs (following
>>>>>>>>>> usual support oncall access protocol that provides temporary access 
>>>>>>>>>> to
>>>>>>>>>> things like logs in GCP):
>>>>>>>>>>
>>>>>>>>>> Root issue looks like consumerPollLoop() mentioned earlier needs
>>>>>>>>>> to handle unchecked exception. In your case it is clear that poll 
>>>>>>>>>> thread
>>>>>>>>>> exited with a runtime exception. The reader does not check for it and
>>>>>>>>>> continues to wait for poll thread to enqueue messages. A fix should 
>>>>>>>>>> result
>>>>>>>>>> in an IOException for read from the source. The runners will handle 
>>>>&g

Re: Problem with KafkaIO

2018-09-14 Thread Raghu Angadi
Thanks. I could repro myself as well. How long was the network down?

Trying to get the fix into 2.7 RC2.

On Fri, Sep 14, 2018 at 12:25 PM Eduardo Soldera <
eduardo.sold...@arquivei.com.br> wrote:

> Just to make myself clear, I'm not sure how to use the patch but if you
> could send us some guidance would be great.
>
> Em sex, 14 de set de 2018 às 16:24, Eduardo Soldera <
> eduardo.sold...@arquivei.com.br> escreveu:
>
>> Hi Raghu, yes, it is feasible, would you do that for us? I'm not sure how
>> we'd use the patch. We're using SBT and Spotify's Scio with Scala.
>>
>> Thanks
>>
>> Em sex, 14 de set de 2018 às 16:07, Raghu Angadi 
>> escreveu:
>>
>>> Is is feasible for you to verify the fix in your dev job? I can make a
>>> patch against Beam 2.4 branch if you like.
>>>
>>> Raghu.
>>>
>>> On Fri, Sep 14, 2018 at 11:14 AM Eduardo Soldera <
>>> eduardo.sold...@arquivei.com.br> wrote:
>>>
>>>> Hi Raghu, thank you very much for the pull request.
>>>> We'll wait for the 2.7 Beam release.
>>>>
>>>> Regards!
>>>>
>>>> Em qui, 13 de set de 2018 às 18:19, Raghu Angadi 
>>>> escreveu:
>>>>
>>>>> Fix: https://github.com/apache/beam/pull/6391
>>>>>
>>>>> On Wed, Sep 12, 2018 at 3:30 PM Raghu Angadi 
>>>>> wrote:
>>>>>
>>>>>> Filed BEAM-5375 <https://issues.apache.org/jira/browse/BEAM-5375>. I
>>>>>> will fix it later this week.
>>>>>>
>>>>>> On Wed, Sep 12, 2018 at 12:16 PM Raghu Angadi 
>>>>>> wrote:
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Sep 12, 2018 at 12:11 PM Raghu Angadi 
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Thanks for the job id, I looked at the worker logs (following usual
>>>>>>>> support oncall access protocol that provides temporary access to things
>>>>>>>> like logs in GCP):
>>>>>>>>
>>>>>>>> Root issue looks like consumerPollLoop() mentioned earlier needs to
>>>>>>>> handle unchecked exception. In your case it is clear that poll thread
>>>>>>>> exited with a runtime exception. The reader does not check for it and
>>>>>>>> continues to wait for poll thread to enqueue messages. A fix should 
>>>>>>>> result
>>>>>>>> in an IOException for read from the source. The runners will handle 
>>>>>>>> that
>>>>>>>> appropriately after that.  I will file a jira.
>>>>>>>>
>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L678
>>>>>>>>
>>>>>>>
>>>>>>> Ignore the link.. was pasted here by mistake.
>>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>> From the logs (with a comment below each one):
>>>>>>>>
>>>>>>>>- 2018-09-12 06:13:07.345 PDT Reader-0: reading from
>>>>>>>>kafka_topic-0 starting at offset 2
>>>>>>>>   - Implies the reader is initialized and poll thread is
>>>>>>>>   started.
>>>>>>>>- 2018-09-12 06:13:07.780 PDT Reader-0: first record offset 2
>>>>>>>>   - The reader actually got a message received by the poll
>>>>>>>>   thread from Kafka.
>>>>>>>>- 2018-09-12 06:16:48.771 PDT Reader-0: exception while
>>>>>>>>fetching latest offset for partition kafka_topic-0. will be retried.
>>>>>>>>   - This must have happened around the time when network was
>>>>>>>>   disrupted. This is from. Actual log is from another periodic 
>>>>>>>> task that
>>>>>>>>   fetches latest offsets for partitions.
>>>>>>>>
>>>>>>>> The poll thread must have died around the time network was
>>>>>>>> disrupted.
>>>>>>>>
>>>>>>>> The following log comes from kafka client itself and is printed
>>>>>>>> every secon

Re: Problem with KafkaIO

2018-09-14 Thread Raghu Angadi
Is is feasible for you to verify the fix in your dev job? I can make a
patch against Beam 2.4 branch if you like.

Raghu.

On Fri, Sep 14, 2018 at 11:14 AM Eduardo Soldera <
eduardo.sold...@arquivei.com.br> wrote:

> Hi Raghu, thank you very much for the pull request.
> We'll wait for the 2.7 Beam release.
>
> Regards!
>
> Em qui, 13 de set de 2018 às 18:19, Raghu Angadi 
> escreveu:
>
>> Fix: https://github.com/apache/beam/pull/6391
>>
>> On Wed, Sep 12, 2018 at 3:30 PM Raghu Angadi  wrote:
>>
>>> Filed BEAM-5375 <https://issues.apache.org/jira/browse/BEAM-5375>. I
>>> will fix it later this week.
>>>
>>> On Wed, Sep 12, 2018 at 12:16 PM Raghu Angadi 
>>> wrote:
>>>
>>>>
>>>>
>>>> On Wed, Sep 12, 2018 at 12:11 PM Raghu Angadi 
>>>> wrote:
>>>>
>>>>> Thanks for the job id, I looked at the worker logs (following usual
>>>>> support oncall access protocol that provides temporary access to things
>>>>> like logs in GCP):
>>>>>
>>>>> Root issue looks like consumerPollLoop() mentioned earlier needs to
>>>>> handle unchecked exception. In your case it is clear that poll thread
>>>>> exited with a runtime exception. The reader does not check for it and
>>>>> continues to wait for poll thread to enqueue messages. A fix should result
>>>>> in an IOException for read from the source. The runners will handle that
>>>>> appropriately after that.  I will file a jira.
>>>>>
>>>>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L678
>>>>>
>>>>
>>>> Ignore the link.. was pasted here by mistake.
>>>>
>>>>
>>>>>
>>>>> From the logs (with a comment below each one):
>>>>>
>>>>>- 2018-09-12 06:13:07.345 PDT Reader-0: reading from kafka_topic-0
>>>>>starting at offset 2
>>>>>   - Implies the reader is initialized and poll thread is started.
>>>>>- 2018-09-12 06:13:07.780 PDT Reader-0: first record offset 2
>>>>>   - The reader actually got a message received by the poll thread
>>>>>   from Kafka.
>>>>>- 2018-09-12 06:16:48.771 PDT Reader-0: exception while fetching
>>>>>latest offset for partition kafka_topic-0. will be retried.
>>>>>   - This must have happened around the time when network was
>>>>>   disrupted. This is from. Actual log is from another periodic task 
>>>>> that
>>>>>   fetches latest offsets for partitions.
>>>>>
>>>>> The poll thread must have died around the time network was disrupted.
>>>>>
>>>>> The following log comes from kafka client itself and is printed every
>>>>> second when KafkaIO fetches latest offset. This log seems to be added in
>>>>> recent versions. It is probably an unintentional log. I don't think there
>>>>> is any better to fetch latest offsets than how KafkaIO does now. This is
>>>>> logged inside consumer.position() called at [1].
>>>>>
>>>>>- 2018-09-12 06:13:11.786 PDT [Consumer clientId=consumer-2,
>>>>>groupId=Reader-0_offset_consumer_1735388161_genericPipe] Resetting 
>>>>> offset
>>>>>for partition com.arquivei.dataeng.andre-0 to offset 3.
>>>>>
>>>>> [1]:
>>>>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L678
>>>>>
>>>>
>>>> This 'Resetting offset' is harmless, but is quite annoying to see in
>>>> the worker logs. One way to avoid is to set kafka consumer's log level to
>>>> WARNING. Ideally KafkaIO itself should do something to avoid it without
>>>> user option.
>>>>
>>>>
>>>>
>>>>
>>>>> On Wed, Sep 12, 2018 at 10:27 AM Eduardo Soldera <
>>>>> eduardo.sold...@arquivei.com.br> wrote:
>>>>>
>>>>>> Hi Raghu! The job_id of our dev job is
>>>>>> 2018-09-12_06_11_48-5600553605191377866.
>>>>>>
>>>>>> Thanks!
>>>>>>
>>>>>> Em qua, 12 de set de 2018 às 14:18, Raghu Angadi 
>>>>>&g

Re: Problem with KafkaIO

2018-09-13 Thread Raghu Angadi
Fix: https://github.com/apache/beam/pull/6391

On Wed, Sep 12, 2018 at 3:30 PM Raghu Angadi  wrote:

> Filed BEAM-5375 <https://issues.apache.org/jira/browse/BEAM-5375>. I will
> fix it later this week.
>
> On Wed, Sep 12, 2018 at 12:16 PM Raghu Angadi  wrote:
>
>>
>>
>> On Wed, Sep 12, 2018 at 12:11 PM Raghu Angadi  wrote:
>>
>>> Thanks for the job id, I looked at the worker logs (following usual
>>> support oncall access protocol that provides temporary access to things
>>> like logs in GCP):
>>>
>>> Root issue looks like consumerPollLoop() mentioned earlier needs to
>>> handle unchecked exception. In your case it is clear that poll thread
>>> exited with a runtime exception. The reader does not check for it and
>>> continues to wait for poll thread to enqueue messages. A fix should result
>>> in an IOException for read from the source. The runners will handle that
>>> appropriately after that.  I will file a jira.
>>>
>>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L678
>>>
>>
>> Ignore the link.. was pasted here by mistake.
>>
>>
>>>
>>> From the logs (with a comment below each one):
>>>
>>>- 2018-09-12 06:13:07.345 PDT Reader-0: reading from kafka_topic-0
>>>starting at offset 2
>>>   - Implies the reader is initialized and poll thread is started.
>>>- 2018-09-12 06:13:07.780 PDT Reader-0: first record offset 2
>>>   - The reader actually got a message received by the poll thread
>>>   from Kafka.
>>>- 2018-09-12 06:16:48.771 PDT Reader-0: exception while fetching
>>>latest offset for partition kafka_topic-0. will be retried.
>>>   - This must have happened around the time when network was
>>>   disrupted. This is from. Actual log is from another periodic task that
>>>   fetches latest offsets for partitions.
>>>
>>> The poll thread must have died around the time network was disrupted.
>>>
>>> The following log comes from kafka client itself and is printed every
>>> second when KafkaIO fetches latest offset. This log seems to be added in
>>> recent versions. It is probably an unintentional log. I don't think there
>>> is any better to fetch latest offsets than how KafkaIO does now. This is
>>> logged inside consumer.position() called at [1].
>>>
>>>- 2018-09-12 06:13:11.786 PDT [Consumer clientId=consumer-2,
>>>groupId=Reader-0_offset_consumer_1735388161_genericPipe] Resetting offset
>>>for partition com.arquivei.dataeng.andre-0 to offset 3.
>>>
>>> [1]:
>>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L678
>>>
>>
>> This 'Resetting offset' is harmless, but is quite annoying to see in the
>> worker logs. One way to avoid is to set kafka consumer's log level to
>> WARNING. Ideally KafkaIO itself should do something to avoid it without
>> user option.
>>
>>
>>
>>
>>> On Wed, Sep 12, 2018 at 10:27 AM Eduardo Soldera <
>>> eduardo.sold...@arquivei.com.br> wrote:
>>>
>>>> Hi Raghu! The job_id of our dev job is
>>>> 2018-09-12_06_11_48-5600553605191377866.
>>>>
>>>> Thanks!
>>>>
>>>> Em qua, 12 de set de 2018 às 14:18, Raghu Angadi 
>>>> escreveu:
>>>>
>>>>> Thanks for debugging.
>>>>> Can you provide the job_id of your dev job? The stacktrace shows that
>>>>> there is no thread running 'consumerPollLoop()' which can explain stuck
>>>>> reader. You will likely find a logs at line 594 & 587 [1].  Dataflow 
>>>>> caches
>>>>> its readers and DirectRunner may not. That can explain DirectRunner resume
>>>>> reads. The expectation in KafkaIO is that Kafka client library takes care
>>>>> of retrying in case of connection problems (as documented). It is possible
>>>>> that in some cases poll() throws and we need to restart the client in
>>>>> KafkaIO.
>>>>>
>>>>> [1]:
>>>>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L594
>>>>>
>>>>> On Wed, Sep 12, 2018 at 9:59 AM Eduardo Soldera <
>>>>> eduardo.sold...@arquivei.com.br> wrote:
&

Re: Problem with KafkaIO

2018-09-12 Thread Raghu Angadi
Filed BEAM-5375 <https://issues.apache.org/jira/browse/BEAM-5375>. I will
fix it later this week.

On Wed, Sep 12, 2018 at 12:16 PM Raghu Angadi  wrote:

>
>
> On Wed, Sep 12, 2018 at 12:11 PM Raghu Angadi  wrote:
>
>> Thanks for the job id, I looked at the worker logs (following usual
>> support oncall access protocol that provides temporary access to things
>> like logs in GCP):
>>
>> Root issue looks like consumerPollLoop() mentioned earlier needs to
>> handle unchecked exception. In your case it is clear that poll thread
>> exited with a runtime exception. The reader does not check for it and
>> continues to wait for poll thread to enqueue messages. A fix should result
>> in an IOException for read from the source. The runners will handle that
>> appropriately after that.  I will file a jira.
>>
>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L678
>>
>
> Ignore the link.. was pasted here by mistake.
>
>
>>
>> From the logs (with a comment below each one):
>>
>>- 2018-09-12 06:13:07.345 PDT Reader-0: reading from kafka_topic-0
>>starting at offset 2
>>   - Implies the reader is initialized and poll thread is started.
>>- 2018-09-12 06:13:07.780 PDT Reader-0: first record offset 2
>>   - The reader actually got a message received by the poll thread
>>   from Kafka.
>>- 2018-09-12 06:16:48.771 PDT Reader-0: exception while fetching
>>latest offset for partition kafka_topic-0. will be retried.
>>   - This must have happened around the time when network was
>>   disrupted. This is from. Actual log is from another periodic task that
>>   fetches latest offsets for partitions.
>>
>> The poll thread must have died around the time network was disrupted.
>>
>> The following log comes from kafka client itself and is printed every
>> second when KafkaIO fetches latest offset. This log seems to be added in
>> recent versions. It is probably an unintentional log. I don't think there
>> is any better to fetch latest offsets than how KafkaIO does now. This is
>> logged inside consumer.position() called at [1].
>>
>>- 2018-09-12 06:13:11.786 PDT [Consumer clientId=consumer-2,
>>groupId=Reader-0_offset_consumer_1735388161_genericPipe] Resetting offset
>>for partition com.arquivei.dataeng.andre-0 to offset 3.
>>
>> [1]:
>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L678
>>
>
> This 'Resetting offset' is harmless, but is quite annoying to see in the
> worker logs. One way to avoid is to set kafka consumer's log level to
> WARNING. Ideally KafkaIO itself should do something to avoid it without
> user option.
>
>
>
>
>> On Wed, Sep 12, 2018 at 10:27 AM Eduardo Soldera <
>> eduardo.sold...@arquivei.com.br> wrote:
>>
>>> Hi Raghu! The job_id of our dev job is
>>> 2018-09-12_06_11_48-5600553605191377866.
>>>
>>> Thanks!
>>>
>>> Em qua, 12 de set de 2018 às 14:18, Raghu Angadi 
>>> escreveu:
>>>
>>>> Thanks for debugging.
>>>> Can you provide the job_id of your dev job? The stacktrace shows that
>>>> there is no thread running 'consumerPollLoop()' which can explain stuck
>>>> reader. You will likely find a logs at line 594 & 587 [1].  Dataflow caches
>>>> its readers and DirectRunner may not. That can explain DirectRunner resume
>>>> reads. The expectation in KafkaIO is that Kafka client library takes care
>>>> of retrying in case of connection problems (as documented). It is possible
>>>> that in some cases poll() throws and we need to restart the client in
>>>> KafkaIO.
>>>>
>>>> [1]:
>>>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L594
>>>>
>>>> On Wed, Sep 12, 2018 at 9:59 AM Eduardo Soldera <
>>>> eduardo.sold...@arquivei.com.br> wrote:
>>>>
>>>>> Hi Raghu, thanks for your help.
>>>>> Just answering your previous question, the following logs were the
>>>>> same as before the error, as if the pipeline were still getting the
>>>>> messages, for example:
>>>>>
>>>>> (...)
>>>>> Resetting offset for partition com.arquivei.dataeng.andre-0 to offset
>>>>> 10.
>>>>> Resetting 

Re: Problem with KafkaIO

2018-09-12 Thread Raghu Angadi
On Wed, Sep 12, 2018 at 12:11 PM Raghu Angadi  wrote:

> Thanks for the job id, I looked at the worker logs (following usual
> support oncall access protocol that provides temporary access to things
> like logs in GCP):
>
> Root issue looks like consumerPollLoop() mentioned earlier needs to handle
> unchecked exception. In your case it is clear that poll thread exited with
> a runtime exception. The reader does not check for it and continues to wait
> for poll thread to enqueue messages. A fix should result in an IOException
> for read from the source. The runners will handle that appropriately after
> that.  I will file a jira.
>
> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L678
>

Ignore the link.. was pasted here by mistake.


>
> From the logs (with a comment below each one):
>
>- 2018-09-12 06:13:07.345 PDT Reader-0: reading from kafka_topic-0
>starting at offset 2
>   - Implies the reader is initialized and poll thread is started.
>- 2018-09-12 06:13:07.780 PDT Reader-0: first record offset 2
>   - The reader actually got a message received by the poll thread
>   from Kafka.
>- 2018-09-12 06:16:48.771 PDT Reader-0: exception while fetching
>latest offset for partition kafka_topic-0. will be retried.
>   - This must have happened around the time when network was
>   disrupted. This is from. Actual log is from another periodic task that
>   fetches latest offsets for partitions.
>
> The poll thread must have died around the time network was disrupted.
>
> The following log comes from kafka client itself and is printed every
> second when KafkaIO fetches latest offset. This log seems to be added in
> recent versions. It is probably an unintentional log. I don't think there
> is any better to fetch latest offsets than how KafkaIO does now. This is
> logged inside consumer.position() called at [1].
>
>- 2018-09-12 06:13:11.786 PDT [Consumer clientId=consumer-2,
>groupId=Reader-0_offset_consumer_1735388161_genericPipe] Resetting offset
>for partition com.arquivei.dataeng.andre-0 to offset 3.
>
> [1]:
> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L678
>

This 'Resetting offset' is harmless, but is quite annoying to see in the
worker logs. One way to avoid is to set kafka consumer's log level to
WARNING. Ideally KafkaIO itself should do something to avoid it without
user option.




> On Wed, Sep 12, 2018 at 10:27 AM Eduardo Soldera <
> eduardo.sold...@arquivei.com.br> wrote:
>
>> Hi Raghu! The job_id of our dev job is
>> 2018-09-12_06_11_48-5600553605191377866.
>>
>> Thanks!
>>
>> Em qua, 12 de set de 2018 às 14:18, Raghu Angadi 
>> escreveu:
>>
>>> Thanks for debugging.
>>> Can you provide the job_id of your dev job? The stacktrace shows that
>>> there is no thread running 'consumerPollLoop()' which can explain stuck
>>> reader. You will likely find a logs at line 594 & 587 [1].  Dataflow caches
>>> its readers and DirectRunner may not. That can explain DirectRunner resume
>>> reads. The expectation in KafkaIO is that Kafka client library takes care
>>> of retrying in case of connection problems (as documented). It is possible
>>> that in some cases poll() throws and we need to restart the client in
>>> KafkaIO.
>>>
>>> [1]:
>>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L594
>>>
>>> On Wed, Sep 12, 2018 at 9:59 AM Eduardo Soldera <
>>> eduardo.sold...@arquivei.com.br> wrote:
>>>
>>>> Hi Raghu, thanks for your help.
>>>> Just answering your previous question, the following logs were the same
>>>> as before the error, as if the pipeline were still getting the messages,
>>>> for example:
>>>>
>>>> (...)
>>>> Resetting offset for partition com.arquivei.dataeng.andre-0 to offset
>>>> 10.
>>>> Resetting offset for partition com.arquivei.dataeng.andre-0 to offset
>>>> 15.
>>>> ERROR
>>>> Resetting offset for partition com.arquivei.dataeng.andre-0 to offset
>>>> 22.
>>>> Resetting offset for partition com.arquivei.dataeng.andre-0 to offset
>>>> 30.
>>>> (...)
>>>>
>>>> But when checking the Kafka Consumer Group, the current offset stays at
>>>> 15, the commited offset from the last processed message, before the error.
>>&

Re: Problem with KafkaIO

2018-09-12 Thread Raghu Angadi
Thanks for the job id, I looked at the worker logs (following usual support
oncall access protocol that provides temporary access to things like logs
in GCP):

Root issue looks like consumerPollLoop() mentioned earlier needs to handle
unchecked exception. In your case it is clear that poll thread exited with
a runtime exception. The reader does not check for it and continues to wait
for poll thread to enqueue messages. A fix should result in an IOException
for read from the source. The runners will handle that appropriately after
that.  I will file a jira.
https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L678
>From the logs (with a comment below each one):

   - 2018-09-12 06:13:07.345 PDT Reader-0: reading from kafka_topic-0
   starting at offset 2
  - Implies the reader is initialized and poll thread is started.
   - 2018-09-12 06:13:07.780 PDT Reader-0: first record offset 2
  - The reader actually got a message received by the poll thread from
  Kafka.
   - 2018-09-12 06:16:48.771 PDT Reader-0: exception while fetching latest
   offset for partition kafka_topic-0. will be retried.
  - This must have happened around the time when network was disrupted.
  This is from. Actual log is from another periodic task that
fetches latest
  offsets for partitions.

The poll thread must have died around the time network was disrupted.

The following log comes from kafka client itself and is printed every
second when KafkaIO fetches latest offset. This log seems to be added in
recent versions. It is probably an unintentional log. I don't think there
is any better to fetch latest offsets than how KafkaIO does now. This is
logged inside consumer.position() called at [1].

   - 2018-09-12 06:13:11.786 PDT [Consumer clientId=consumer-2,
   groupId=Reader-0_offset_consumer_1735388161_genericPipe] Resetting offset
   for partition com.arquivei.dataeng.andre-0 to offset 3.

[1]:
https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L678

On Wed, Sep 12, 2018 at 10:27 AM Eduardo Soldera <
eduardo.sold...@arquivei.com.br> wrote:

> Hi Raghu! The job_id of our dev job is
> 2018-09-12_06_11_48-5600553605191377866.
>
> Thanks!
>
> Em qua, 12 de set de 2018 às 14:18, Raghu Angadi 
> escreveu:
>
>> Thanks for debugging.
>> Can you provide the job_id of your dev job? The stacktrace shows that
>> there is no thread running 'consumerPollLoop()' which can explain stuck
>> reader. You will likely find a logs at line 594 & 587 [1].  Dataflow caches
>> its readers and DirectRunner may not. That can explain DirectRunner resume
>> reads. The expectation in KafkaIO is that Kafka client library takes care
>> of retrying in case of connection problems (as documented). It is possible
>> that in some cases poll() throws and we need to restart the client in
>> KafkaIO.
>>
>> [1]:
>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L594
>>
>> On Wed, Sep 12, 2018 at 9:59 AM Eduardo Soldera <
>> eduardo.sold...@arquivei.com.br> wrote:
>>
>>> Hi Raghu, thanks for your help.
>>> Just answering your previous question, the following logs were the same
>>> as before the error, as if the pipeline were still getting the messages,
>>> for example:
>>>
>>> (...)
>>> Resetting offset for partition com.arquivei.dataeng.andre-0 to offset 10.
>>> Resetting offset for partition com.arquivei.dataeng.andre-0 to offset 15.
>>> ERROR
>>> Resetting offset for partition com.arquivei.dataeng.andre-0 to offset 22.
>>> Resetting offset for partition com.arquivei.dataeng.andre-0 to offset 30.
>>> (...)
>>>
>>> But when checking the Kafka Consumer Group, the current offset stays at
>>> 15, the commited offset from the last processed message, before the error.
>>>
>>> We'll file a bug, but we could now reproduce the issue in a Dev scenario.
>>> We started the same pipeline using the direct runner, without Google
>>> Dataflow. We blocked the Kafka Broker network and the same error was
>>> thrown. Then we unblocked the network and the pipeline was able to
>>> successfully process the subsequent messages.
>>> When we started the same pipeline in the Dataflow runner and did the
>>> same test, the same problem from our production scenario happened, Dataflow
>>> couldn't process the new messages. Unfortunately, we've stopped the
>>> dataflow job in production, but the problematic dev job is still running
>>> and the log file of the VM is attache

Re: Problem with KafkaIO

2018-09-12 Thread Raghu Angadi
Thanks for debugging.
Can you provide the job_id of your dev job? The stacktrace shows that there
is no thread running 'consumerPollLoop()' which can explain stuck reader.
You will likely find a logs at line 594 & 587 [1].  Dataflow caches its
readers and DirectRunner may not. That can explain DirectRunner resume
reads. The expectation in KafkaIO is that Kafka client library takes care
of retrying in case of connection problems (as documented). It is possible
that in some cases poll() throws and we need to restart the client in
KafkaIO.

[1]:
https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L594

On Wed, Sep 12, 2018 at 9:59 AM Eduardo Soldera <
eduardo.sold...@arquivei.com.br> wrote:

> Hi Raghu, thanks for your help.
> Just answering your previous question, the following logs were the same as
> before the error, as if the pipeline were still getting the messages, for
> example:
>
> (...)
> Resetting offset for partition com.arquivei.dataeng.andre-0 to offset 10.
> Resetting offset for partition com.arquivei.dataeng.andre-0 to offset 15.
> ERROR
> Resetting offset for partition com.arquivei.dataeng.andre-0 to offset 22.
> Resetting offset for partition com.arquivei.dataeng.andre-0 to offset 30.
> (...)
>
> But when checking the Kafka Consumer Group, the current offset stays at
> 15, the commited offset from the last processed message, before the error.
>
> We'll file a bug, but we could now reproduce the issue in a Dev scenario.
> We started the same pipeline using the direct runner, without Google
> Dataflow. We blocked the Kafka Broker network and the same error was
> thrown. Then we unblocked the network and the pipeline was able to
> successfully process the subsequent messages.
> When we started the same pipeline in the Dataflow runner and did the same
> test, the same problem from our production scenario happened, Dataflow
> couldn't process the new messages. Unfortunately, we've stopped the
> dataflow job in production, but the problematic dev job is still running
> and the log file of the VM is attached. Thank you very much.
> Best regards
>
> Em ter, 11 de set de 2018 às 18:28, Raghu Angadi 
> escreveu:
>
>> Specifically, I am interested if you have any thread running
>> 'consumerPollLoop()' [1]. There should always be one (if a worker is
>> assigned one of the partitions). It is possible that KafkaClient itself is
>> hasn't recovered from the group coordinator error (though unlikely).
>>
>>
>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L570
>>
>> On Tue, Sep 11, 2018 at 12:31 PM Raghu Angadi  wrote:
>>
>>> Hi Eduardo,
>>>
>>> In case of any error, the pipeline should keep on trying to fetch. I
>>> don't know about this particular error. Do you see any others afterwards in
>>> the log?
>>> Couple of things you could try if the logs are not useful :
>>>  - login to one of the VMs and get stacktrace of java worker (look for a
>>> container called java-streaming)
>>>  - file a support bug or stackoverflow question with jobid so that
>>> Dataflow oncall can take a look.
>>>
>>> Raghu.
>>>
>>>
>>> On Tue, Sep 11, 2018 at 12:10 PM Eduardo Soldera <
>>> eduardo.sold...@arquivei.com.br> wrote:
>>>
>>>> Hi,
>>>> We have a Apache Beam pipeline running in Google Dataflow using
>>>> KafkaIO. Suddenly the pipeline stop fetching Kafka messages at all, as our
>>>> other workers from other pipelines continued to get Kafka messages.
>>>>
>>>> At the moment it stopped we got these messages:
>>>>
>>>> I  [Consumer clientId=consumer-1, groupId=genericPipe] Error sending fetch 
>>>> request (sessionId=1396189203, epoch=2431598) to node 3: 
>>>> org.apache.kafka.common.errors.DisconnectException.
>>>> I  [Consumer clientId=consumer-1, groupId=genericPipe] Group coordinator 
>>>> 10.0.52.70:9093 (id: 2147483646 rack: null) is unavailable or invalid, 
>>>> will attempt rediscovery
>>>> I  [Consumer clientId=consumer-1, groupId=genericPipe] Discovered group 
>>>> coordinator 10.0.52.70:9093 (id: 2147483646 rack: null)
>>>>
>>>> And then the pipeline stopped reading the messages.
>>>>
>>>> This is the KafkaIO setup  we have:
>>>>
>>>> KafkaIO.read[String,String]()
>>>>   .withBootstrapServers(server)
>>>>   .withTopic(topic)
>>>>   .withKeyDeserializer(cla

Re: Problem with KafkaIO

2018-09-11 Thread Raghu Angadi
Specifically, I am interested if you have any thread running
'consumerPollLoop()' [1]. There should always be one (if a worker is
assigned one of the partitions). It is possible that KafkaClient itself is
hasn't recovered from the group coordinator error (though unlikely).

https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java#L570

On Tue, Sep 11, 2018 at 12:31 PM Raghu Angadi  wrote:

> Hi Eduardo,
>
> In case of any error, the pipeline should keep on trying to fetch. I don't
> know about this particular error. Do you see any others afterwards in the
> log?
> Couple of things you could try if the logs are not useful :
>  - login to one of the VMs and get stacktrace of java worker (look for a
> container called java-streaming)
>  - file a support bug or stackoverflow question with jobid so that
> Dataflow oncall can take a look.
>
> Raghu.
>
>
> On Tue, Sep 11, 2018 at 12:10 PM Eduardo Soldera <
> eduardo.sold...@arquivei.com.br> wrote:
>
>> Hi,
>> We have a Apache Beam pipeline running in Google Dataflow using KafkaIO.
>> Suddenly the pipeline stop fetching Kafka messages at all, as our other
>> workers from other pipelines continued to get Kafka messages.
>>
>> At the moment it stopped we got these messages:
>>
>> I  [Consumer clientId=consumer-1, groupId=genericPipe] Error sending fetch 
>> request (sessionId=1396189203, epoch=2431598) to node 3: 
>> org.apache.kafka.common.errors.DisconnectException.
>> I  [Consumer clientId=consumer-1, groupId=genericPipe] Group coordinator 
>> 10.0.52.70:9093 (id: 2147483646 rack: null) is unavailable or invalid, will 
>> attempt rediscovery
>> I  [Consumer clientId=consumer-1, groupId=genericPipe] Discovered group 
>> coordinator 10.0.52.70:9093 (id: 2147483646 rack: null)
>>
>> And then the pipeline stopped reading the messages.
>>
>> This is the KafkaIO setup  we have:
>>
>> KafkaIO.read[String,String]()
>>   .withBootstrapServers(server)
>>   .withTopic(topic)
>>   .withKeyDeserializer(classOf[StringDeserializer])
>>   .withValueDeserializer(classOf[StringDeserializer])
>>   .updateConsumerProperties(properties)
>>   .commitOffsetsInFinalize()
>>   .withoutMetadata()
>>
>>  Any help will be much appreciated.
>>
>> Best regards,
>> --
>> Eduardo Soldera Garcia
>> Data Engineer
>> (16) 3509- | www.arquivei.com.br
>> <https://arquivei.com.br/?utm_campaign=assinatura-email_content=assinatura>
>> [image: Arquivei.com.br – Inteligência em Notas Fiscais]
>> <https://arquivei.com.br/?utm_campaign=assinatura-email_content=assinatura>
>> [image: Google seleciona Arquivei para imersão e mentoria no Vale do
>> Silício]
>> <https://arquivei.com.br/blog/google-seleciona-arquivei/?utm_campaign=assinatura-email-launchpad_content=assinatura-launchpad>
>> <https://www.facebook.com/arquivei>
>> <https://www.linkedin.com/company/arquivei>
>> <https://www.youtube.com/watch?v=sSUUKxbXnxk>
>>
>


Re: Problem with KafkaIO

2018-09-11 Thread Raghu Angadi
Hi Eduardo,

In case of any error, the pipeline should keep on trying to fetch. I don't
know about this particular error. Do you see any others afterwards in the
log?
Couple of things you could try if the logs are not useful :
 - login to one of the VMs and get stacktrace of java worker (look for a
container called java-streaming)
 - file a support bug or stackoverflow question with jobid so that Dataflow
oncall can take a look.

Raghu.


On Tue, Sep 11, 2018 at 12:10 PM Eduardo Soldera <
eduardo.sold...@arquivei.com.br> wrote:

> Hi,
> We have a Apache Beam pipeline running in Google Dataflow using KafkaIO.
> Suddenly the pipeline stop fetching Kafka messages at all, as our other
> workers from other pipelines continued to get Kafka messages.
>
> At the moment it stopped we got these messages:
>
> I  [Consumer clientId=consumer-1, groupId=genericPipe] Error sending fetch 
> request (sessionId=1396189203, epoch=2431598) to node 3: 
> org.apache.kafka.common.errors.DisconnectException.
> I  [Consumer clientId=consumer-1, groupId=genericPipe] Group coordinator 
> 10.0.52.70:9093 (id: 2147483646 rack: null) is unavailable or invalid, will 
> attempt rediscovery
> I  [Consumer clientId=consumer-1, groupId=genericPipe] Discovered group 
> coordinator 10.0.52.70:9093 (id: 2147483646 rack: null)
>
> And then the pipeline stopped reading the messages.
>
> This is the KafkaIO setup  we have:
>
> KafkaIO.read[String,String]()
>   .withBootstrapServers(server)
>   .withTopic(topic)
>   .withKeyDeserializer(classOf[StringDeserializer])
>   .withValueDeserializer(classOf[StringDeserializer])
>   .updateConsumerProperties(properties)
>   .commitOffsetsInFinalize()
>   .withoutMetadata()
>
>  Any help will be much appreciated.
>
> Best regards,
> --
> Eduardo Soldera Garcia
> Data Engineer
> (16) 3509- | www.arquivei.com.br
> 
> [image: Arquivei.com.br – Inteligência em Notas Fiscais]
> 
> [image: Google seleciona Arquivei para imersão e mentoria no Vale do
> Silício]
> 
> 
> 
> 
>


Re: Lateness for Spark

2018-09-07 Thread Raghu Angadi
I see. Hopefully someone with more familiarity with Spark runner will chime
in.

On Fri, Sep 7, 2018 at 1:41 PM Vishwas Bm  wrote:

> Hi,
>
> In our use case the watermark is the processing time.
>
> As per beam capability matrix (
> https://beam.apache.org/documentation/runners/capability-matrix/)
> lateness is not supported by spark runner.  But as per the output in our
> use case we are able to see late data getting emitted.
>
> So we wanted to know whether spark runner supports allowed lateness or
> not.
>
>
> Regards,
> Vishwas
>
>
> On Fri, Sep 7, 2018, 10:09 PM Raghu Angadi  wrote:
>
>> Lateness depends on watermark. How did you configure your KafkaIO reader?
>> Did you set custom timestamp function? By default watermark in KafkaIO is
>> set to same as processing time, in which case, your watermark could be
>> close to 13-38-37 (processing time).  Note that this is in general true
>> across all the runners, though I am not aware of any subtle differences in
>> Spark runner.
>>
>> On Fri, Sep 7, 2018 at 7:03 AM rahul patwari 
>> wrote:
>>
>>> Hi,
>>>
>>> We are running a Beam program on Spark. We are using 2.5.0 Beam and
>>> SparkRunner versions. We are seeing Late data in the output emitted by
>>> Spark. As per the capability Matrix, Lateness is not supported in Spark. Is
>>> it supported now? or Are we missing something?
>>>
>>> Steps:
>>>  Read from Kafka, Apply a Fixed Window of 1 Min with Lateness as 2 Min
>>> with Late firings when an element is found with Accumulating Fired Panes,
>>> GroupByKey, ParDo to display the result.
>>>
>>> Below is the output of the ParDo in which we are printing the GroupByKey
>>> result:
>>> Pane Timing: LATE
>>> Processing Time : 2018-09-0713-38-37-7290+
>>> Element Time  : 2018-09-0713-36-59-9990+
>>> Window Start Time  : 2018-09-0713-36-00-+
>>> Window End Time   : 2018-09-0713-37-00-+
>>> Pane Index  : 1
>>> Pane NonSpeculativeIndex : 1
>>>
>>> Regards,
>>> Rahul
>>>
>>


Re: Lateness for Spark

2018-09-07 Thread Raghu Angadi
Lateness depends on watermark. How did you configure your KafkaIO reader?
Did you set custom timestamp function? By default watermark in KafkaIO is
set to same as processing time, in which case, your watermark could be
close to 13-38-37 (processing time).  Note that this is in general true
across all the runners, though I am not aware of any subtle differences in
Spark runner.

On Fri, Sep 7, 2018 at 7:03 AM rahul patwari 
wrote:

> Hi,
>
> We are running a Beam program on Spark. We are using 2.5.0 Beam and
> SparkRunner versions. We are seeing Late data in the output emitted by
> Spark. As per the capability Matrix, Lateness is not supported in Spark. Is
> it supported now? or Are we missing something?
>
> Steps:
>  Read from Kafka, Apply a Fixed Window of 1 Min with Lateness as 2 Min
> with Late firings when an element is found with Accumulating Fired Panes,
> GroupByKey, ParDo to display the result.
>
> Below is the output of the ParDo in which we are printing the GroupByKey
> result:
> Pane Timing: LATE
> Processing Time : 2018-09-0713-38-37-7290+
> Element Time  : 2018-09-0713-36-59-9990+
> Window Start Time  : 2018-09-0713-36-00-+
> Window End Time   : 2018-09-0713-37-00-+
> Pane Index  : 1
> Pane NonSpeculativeIndex : 1
>
> Regards,
> Rahul
>


Re: Streaming mode Dataflow - how to make Autoscaling kick in for FileIO.match operations

2018-08-30 Thread Raghu Angadi
May be I misunderstood the set up. Can you post pseudo code for part of you
pipeline? (From pubsub read till your custom reader). You might need to
Reshuffle before file read.

On Wed, Aug 29, 2018 at 6:18 PM Aniruddh Sharma 
wrote:

> Thanks Raghu
>
> Our objective is to parallelize Read operation itself. We will read a
> single message from Pub Sub and we need to do a lookup of all files present
> in that path which may be lets say 10,000 and then we need to run read of
> 10,000 files in parallel. Right now by default , its not happening by doing
> a FileIO.match and then doing a Custom Reader after that step.
>
> Thanks
> Aniruddh
>
> On Wed, Aug 29, 2018 at 1:07 PM Raghu Angadi  wrote:
>
>>
>> Unbounded sources likes KafkaIO, KinesisIO etc are also supported in
>> addition to PubSub. Unbounded source has API to report backlog. (We will
>> update the documentation).
>> FileIO.matches() is a SplittableDoFn (this is new interface for IO),
>> there is a way to report backlog, but not clear how well it defined or
>> tested.
>>
>> That aside, Dataflow scales based on messages buffered between the stages
>> in the pipeline, not just on the backlog reported by sources. This is a
>> less reliable signal, but might help in your case. It is likely that most
>> of your processing is right after reading from the files. Adding a shuffle
>> between those to could help. i.e.
>> From : read() --> Process() -->   [ Here, read() and Process() might
>> be fused and run in the same stage inline ]
>> To  : Read() --> Reshuffle --> Process() ...
>>
>> This is strictly a work around might be good enough in practice.
>> Reshuffle can be used to shuffle randomly or to a fixed number of shards.
>> It might be better some times to limit it to say 100 shards.
>>
>> On Wed, Aug 29, 2018 at 8:37 AM asharma...@gmail.com <
>> asharma...@gmail.com> wrote:
>>
>>> Excerpt for Autoscaling on Streaming mode
>>> "Currently, PubsubIO is the only source that supports autoscaling on
>>> streaming pipelines. All SDK-provided sinks are supported. In this Beta
>>> release, Autoscaling works smoothest when reading from Cloud Pub/Sub
>>> subscriptions tied to topics published with small batches and when writing
>>> to sinks with low latency. In extreme cases (i.e. Cloud Pub/Sub
>>> subscriptions with large publishing batches or sinks with very high
>>> latency), autoscaling is known to become coarse-grained. This will be
>>> improved in future releases."
>>>
>>
>>
>>
>>>
>>> For our use case we cannot write messages in PubSub but we are writing
>>> file paths in PubSub and then after reading the FilePath we want to do
>>> FileIO.match and go from there. But for these use cases Autoscaling is not
>>> kicking in Streaming mode.
>>>
>>> Is there a way to override the 'Backlog' (hint for autcoscaling to kick
>>> in) that in this case one message in pub sub is indicator of millions of
>>> message so that Autoscaling treats as if one message in PubSub is equal to
>>> amount of 1 million pending messages and if if it sees there are 100
>>> messages in PubSub then it knows it has to process 100 million records
>>> corresponding to these 100 messages and start kicking in Autoscaling.
>>>
>>> Or may be I am thinking on completely wrong line. Basically any way
>>> where we can trigger Autoscaling in streaming mode after FileIO.match
>>> operation
>>>
>>> Thanks
>>> Aniruddh
>>>
>>


Re: Controlling Kafka Checkpoint Persistence

2018-08-21 Thread Raghu Angadi
On Tue, Aug 21, 2018 at 2:49 PM Micah Whitacre  wrote:

> > Is there a reason you can't trust the runner to be durable storage for
> inprocess work?
>
> That's a fair question.  Are there any good resources documenting the
> durability/stability of the different runners?  I assume there are some
> stability requirements regarding its handling of "bundles" but it would be
> nice to have that info available.  One of the reasons we are targeting the
> Direct runner is to let us work with the project and let us temporarily
> delay picking a runner.  Durability seems like another important aspect to
> evaluate.
>

Could you describe your durability requirements a bit more?
All the major runners comparable durability guarantees on processing within
a running pipeline (these are required for Beam model).  The differences
arise at the boundaries: what happens when you stop the pipeline, can the
pipeline be updated with new code with the old state, etc.

An often confusing area is about side effects (like committing Kafka
offsets in your case).. the users always have to assume that processing
might be retried (even if it rarely occurs).


>
> On Tue, Aug 21, 2018 at 4:24 PM, Raghu Angadi  wrote:
>
>> On Tue, Aug 21, 2018 at 2:04 PM Lukasz Cwik  wrote:
>>
>>> Is there a reason you can't trust the runner to be durable storage for
>>> inprocess work?
>>>
>>> I can understand that the DirectRunner only stores things in memory but
>>> other runners have stronger durability guarantees.
>>>
>>
>> I think the requirement is about producing a side effect (committing
>> offsets to Kafka) after some processing completes in the pipeline. Wait()
>> transform helps with that. The the user still has to commit the offsets
>> explicitly and can't get similar functionality in KafkaIO.
>>
>>
>>> On Tue, Aug 21, 2018 at 9:58 AM Raghu Angadi  wrote:
>>>
>>>> I think by 'KafkaUnboundedSource checkpointing' you mean enabling
>>>> 'commitOffsetsInFinalize()' on KafkaIO source.
>>>> It is better option than enable.auto.commit, but does not exactly do
>>>> what you want in this moment. It is invoked after the first stage ('Simple
>>>> Transformation' in your case). This is certainly true for Dataflow and I
>>>> think is also the case for DirectRunner.
>>>>
>>>> I don't see way to leverage built-in checkpoint for consistency
>>>> externally. You would have to manually commit offsets.
>>>>
>>>> On Tue, Aug 21, 2018 at 8:55 AM Micah Whitacre 
>>>> wrote:
>>>>
>>>>> I'm starting with a very simple pipeline that will read from Kafka ->
>>>>> Simple Transformation -> GroupByKey -> Persist the data.  We are also
>>>>> applying some simple windowing/triggering that will persist the data after
>>>>> every 100 elements or every 60 seconds to balance slow trickles of data as
>>>>> well as not storing too much in memory.  For now I'm just running with the
>>>>> DirectRunner since this is just a small processing problem.
>>>>>
>>>>> With the potential for failure during the persisting of the data, we
>>>>> want to ensure that the Kafka offsets are not updated until we have
>>>>> successfully persisted the data.  Looking at KafkaIO it seems like our two
>>>>> options for persisting offsets are:
>>>>> * Kafka's enable.auto.commit
>>>>> * KafkaUnboundedSource checkpointing.
>>>>>
>>>>> The first option would commit prematurely before we could guarantee
>>>>> the data was persisted.  I can't unfortunately find many details about the
>>>>> checkpointing so I was wondering if there was a way to configure it or 
>>>>> tune
>>>>> it more appropriately.
>>>>>
>>>>> Specifically I'm hoping to understand the flow so I can rely on the
>>>>> built in KafkaIO functionality without having to write our own offset
>>>>> management.  Or is it more common to write your own?
>>>>>
>>>>> Thanks,
>>>>> Micah
>>>>>
>>>>
>


Re: Controlling Kafka Checkpoint Persistence

2018-08-21 Thread Raghu Angadi
On Tue, Aug 21, 2018 at 2:04 PM Lukasz Cwik  wrote:

> Is there a reason you can't trust the runner to be durable storage for
> inprocess work?
>
> I can understand that the DirectRunner only stores things in memory but
> other runners have stronger durability guarantees.
>

I think the requirement is about producing a side effect (committing
offsets to Kafka) after some processing completes in the pipeline. Wait()
transform helps with that. The the user still has to commit the offsets
explicitly and can't get similar functionality in KafkaIO.


> On Tue, Aug 21, 2018 at 9:58 AM Raghu Angadi  wrote:
>
>> I think by 'KafkaUnboundedSource checkpointing' you mean enabling
>> 'commitOffsetsInFinalize()' on KafkaIO source.
>> It is better option than enable.auto.commit, but does not exactly do what
>> you want in this moment. It is invoked after the first stage ('Simple
>> Transformation' in your case). This is certainly true for Dataflow and I
>> think is also the case for DirectRunner.
>>
>> I don't see way to leverage built-in checkpoint for consistency
>> externally. You would have to manually commit offsets.
>>
>> On Tue, Aug 21, 2018 at 8:55 AM Micah Whitacre 
>> wrote:
>>
>>> I'm starting with a very simple pipeline that will read from Kafka ->
>>> Simple Transformation -> GroupByKey -> Persist the data.  We are also
>>> applying some simple windowing/triggering that will persist the data after
>>> every 100 elements or every 60 seconds to balance slow trickles of data as
>>> well as not storing too much in memory.  For now I'm just running with the
>>> DirectRunner since this is just a small processing problem.
>>>
>>> With the potential for failure during the persisting of the data, we
>>> want to ensure that the Kafka offsets are not updated until we have
>>> successfully persisted the data.  Looking at KafkaIO it seems like our two
>>> options for persisting offsets are:
>>> * Kafka's enable.auto.commit
>>> * KafkaUnboundedSource checkpointing.
>>>
>>> The first option would commit prematurely before we could guarantee the
>>> data was persisted.  I can't unfortunately find many details about the
>>> checkpointing so I was wondering if there was a way to configure it or tune
>>> it more appropriately.
>>>
>>> Specifically I'm hoping to understand the flow so I can rely on the
>>> built in KafkaIO functionality without having to write our own offset
>>> management.  Or is it more common to write your own?
>>>
>>> Thanks,
>>> Micah
>>>
>>


Re: Controlling Kafka Checkpoint Persistence

2018-08-21 Thread Raghu Angadi
I think by 'KafkaUnboundedSource checkpointing' you mean enabling
'commitOffsetsInFinalize()' on KafkaIO source.
It is better option than enable.auto.commit, but does not exactly do what
you want in this moment. It is invoked after the first stage ('Simple
Transformation' in your case). This is certainly true for Dataflow and I
think is also the case for DirectRunner.

I don't see way to leverage built-in checkpoint for consistency externally.
You would have to manually commit offsets.

On Tue, Aug 21, 2018 at 8:55 AM Micah Whitacre  wrote:

> I'm starting with a very simple pipeline that will read from Kafka ->
> Simple Transformation -> GroupByKey -> Persist the data.  We are also
> applying some simple windowing/triggering that will persist the data after
> every 100 elements or every 60 seconds to balance slow trickles of data as
> well as not storing too much in memory.  For now I'm just running with the
> DirectRunner since this is just a small processing problem.
>
> With the potential for failure during the persisting of the data, we want
> to ensure that the Kafka offsets are not updated until we have successfully
> persisted the data.  Looking at KafkaIO it seems like our two options for
> persisting offsets are:
> * Kafka's enable.auto.commit
> * KafkaUnboundedSource checkpointing.
>
> The first option would commit prematurely before we could guarantee the
> data was persisted.  I can't unfortunately find many details about the
> checkpointing so I was wondering if there was a way to configure it or tune
> it more appropriately.
>
> Specifically I'm hoping to understand the flow so I can rely on the built
> in KafkaIO functionality without having to write our own offset
> management.  Or is it more common to write your own?
>
> Thanks,
> Micah
>


Re: Dynamic topologies / replay

2018-08-17 Thread Raghu Angadi
On Thu, Aug 16, 2018 at 2:20 PM Thomas Browne  wrote:

> I just watched the excellent presentation by Markku Leppisto in Singapore.
> I consult for a financial broker in London.
> Our use case is streaming financial data on which various analytics are
> performed to find relative value trading opportunities in the fixed income
> markets.
>
> I have two questions about Apache Beam:
>
> a) Does it, or will it, support dynamic topologies? We have many analysts
> all of whom want slight variations of the topologies, and they may want to
> change the topologies, or the function ("node") parameters, at runtime. Is
> this possible, and if not is "second prize" of simply growing the DAG with
> non-recombinant node branches at runtime, possible?
>

Topologies are fixed. Some form of "second prize" is probably doable, but I
think the pipeline author needs to handle it at runtime.

b) Does, or will Apache beam support "replays"? We find quite often that
> historical inputs change, a long time later (for example, a bond price from
> a few weeks ago is seen to have been erroneous, and is changed). We then
> have to re-run from that point forward as all downstream calcs are
> dependent on all prices. I believe Apache Flink supports this type of
> functionality? Is this possilble with Beam?
>

A streaming pipeline can be restarted from earlier resume point as long as
the sources support it. Other than that, I don't think there is much more
explicit support (e.g. in-built support to combine updated historical
aggregates to future updates etc). Flink's save-points should help with
Beam on Flink as well, it might make it simpler to save multiple snapshots.


> I am evaluating the various dataflow programming environments that are
> springing up and will advise based partly on the above.
>
> Many thanks,
>
> Thomas
>
>


Re: Retry on failures when using KafkaIO and Google Dataflow

2018-08-16 Thread Raghu Angadi
On Thu, Aug 16, 2018 at 4:54 PM Lukasz Cwik  wrote:

> Raghu, yes, I was thinking that advance would continuously return false.
> This would force a runner to checkpoint and then resume at which point we
> would have a stable reading point.
>

When the actual checkpoint occurs is still transparent to the reader. Only
way it knows if there was one for sure is when a new reader is created with
non-null CheckpointMark. In the case of Dataflow, readers are cached and
might be created only once during the life of a job.


> On Thu, Aug 16, 2018 at 3:17 PM Raghu Angadi  wrote:
>
>> The only way reshuffle could help is if you are able to separate
>> processing that is prone to errors from processing that produces side
>> effects i.e.
>> IO --> DoFn_B_prone_to_exceptions --> Reshuffle --> (B)
>> DoFn_B_producing_side_effects
>>
>> This way, it might look like (B) processes each record exactly once, but
>> it is not guaranteed (e.g. a worker could crash suddenly).
>>
>> Raghu.
>>
>> On Thu, Aug 16, 2018 at 11:12 AM Raghu Angadi  wrote:
>>
>>> On Thu, Aug 16, 2018 at 10:39 AM André Missaglia <
>>> andre.missag...@arquivei.com.br> wrote:
>>>
>>>> Hello everyone.
>>>>
>>>> Thanks for your answers. We've managed to solve our problem using the
>>>> solutions proposed here.
>>>>
>>>> First, there was no need for using autocommit. But switching to
>>>> "commitOffsetsInFinalize()" also didn't help. In our example, if a failure
>>>> occured when processing message #25, the runner would retry the whole batch
>>>> again, moving from "at-most-once processing" to "at-least-once processing".
>>>>
>>>
>>> It is always been at-least-once for DoFn side effects and external
>>> interactions. That is not specific to KafkaIO. Using reshuffle does not
>>> change that.
>>>
>>> Raghu.
>>>
>>>
>>>>
>>>> Of course, if the transformation is idempotent, it doesn't really
>>>> matter. But it is not always the case for us.
>>>>
>>>> Then, we tried using Reshuffle.viaRandomKey(). While the order of the
>>>> messages was lost, all messages were successfully processed exactly once.
>>>>
>>>> But one thing that still bugs me is that Reshuffle is deprecated[1].
>>>> I've found some discussion[2] on the internet about this, but i'm still not
>>>> sure if it is safe to use that transformation in this use case. Any
>>>> thoughts on this?
>>>>
>>>> [1]
>>>> https://beam.apache.org/documentation/sdks/javadoc/2.6.0/org/apache/beam/sdk/transforms/Reshuffle.html#viaRandomKey--
>>>> [2] https://www.mail-archive.com/dev@beam.apache.org/msg07247.html
>>>>
>>>> Em qui, 16 de ago de 2018 às 13:36, Raghu Angadi 
>>>> escreveu:
>>>>
>>>>> I am not sure if it helps much. External checkpoints like Kafka
>>>>> 'autocommit' outside Beam's own checkpoint domain will always have quirks
>>>>> like this. I wanted to ask what their use case for using autocommit was
>>>>> (over commitOffsetsInFinalize())
>>>>>
>>>>> If we wanted to, it is not clear to how an unbounded source can do
>>>>> that. A reader (by design I think) does not have visibility into when a
>>>>> checkpoint happens, bundle boundaries etc. E.g. it can try returning
>>>>> 'false' for start() and advance() initially, but does not know when to
>>>>> actually return records (in Dataflow, it could wait a few seconds).
>>>>>
>>>>> On Thu, Aug 16, 2018 at 8:21 AM Lukasz Cwik  wrote:
>>>>>
>>>>>> Raghu, based upon your description, do you think it would be a good
>>>>>> for KafkaIO to checkpoint on the first read without producing any actual
>>>>>> records?
>>>>>>
>>>>>> On Wed, Aug 15, 2018 at 11:49 AM Raghu Angadi 
>>>>>> wrote:
>>>>>>
>>>>>>>
>>>>>>> It is due to "enable.autocommit=true".  Auto commit is an option to
>>>>>>> Kafka client and how and when it commits is totally out of control of 
>>>>>>> Beam
>>>>>>> & KafkaIO.
>>>>>>> Could you try setting commitOffsetsInFinalize()[1] in KafkaIO rather
>>>>>>> than 'enable.autocommit'? That would ensure exac

Re: Retry on failures when using KafkaIO and Google Dataflow

2018-08-16 Thread Raghu Angadi
On Thu, Aug 16, 2018 at 10:39 AM André Missaglia <
andre.missag...@arquivei.com.br> wrote:

> Hello everyone.
>
> Thanks for your answers. We've managed to solve our problem using the
> solutions proposed here.
>
> First, there was no need for using autocommit. But switching to
> "commitOffsetsInFinalize()" also didn't help. In our example, if a failure
> occured when processing message #25, the runner would retry the whole batch
> again, moving from "at-most-once processing" to "at-least-once processing".
>

It is always been at-least-once for DoFn side effects and external
interactions. That is not specific to KafkaIO. Using reshuffle does not
change that.

Raghu.


>
> Of course, if the transformation is idempotent, it doesn't really matter.
> But it is not always the case for us.
>
> Then, we tried using Reshuffle.viaRandomKey(). While the order of the
> messages was lost, all messages were successfully processed exactly once.
>
> But one thing that still bugs me is that Reshuffle is deprecated[1]. I've
> found some discussion[2] on the internet about this, but i'm still not sure
> if it is safe to use that transformation in this use case. Any thoughts on
> this?
>
> [1]
> https://beam.apache.org/documentation/sdks/javadoc/2.6.0/org/apache/beam/sdk/transforms/Reshuffle.html#viaRandomKey--
> [2] https://www.mail-archive.com/dev@beam.apache.org/msg07247.html
>
> Em qui, 16 de ago de 2018 às 13:36, Raghu Angadi 
> escreveu:
>
>> I am not sure if it helps much. External checkpoints like Kafka
>> 'autocommit' outside Beam's own checkpoint domain will always have quirks
>> like this. I wanted to ask what their use case for using autocommit was
>> (over commitOffsetsInFinalize())
>>
>> If we wanted to, it is not clear to how an unbounded source can do that.
>> A reader (by design I think) does not have visibility into when a
>> checkpoint happens, bundle boundaries etc. E.g. it can try returning
>> 'false' for start() and advance() initially, but does not know when to
>> actually return records (in Dataflow, it could wait a few seconds).
>>
>> On Thu, Aug 16, 2018 at 8:21 AM Lukasz Cwik  wrote:
>>
>>> Raghu, based upon your description, do you think it would be a good for
>>> KafkaIO to checkpoint on the first read without producing any actual
>>> records?
>>>
>>> On Wed, Aug 15, 2018 at 11:49 AM Raghu Angadi 
>>> wrote:
>>>
>>>>
>>>> It is due to "enable.autocommit=true".  Auto commit is an option to
>>>> Kafka client and how and when it commits is totally out of control of Beam
>>>> & KafkaIO.
>>>> Could you try setting commitOffsetsInFinalize()[1] in KafkaIO rather
>>>> than 'enable.autocommit'? That would ensure exactly once processing.
>>>>
>>>> That said, you might be interested in understanding why your example
>>>> failed:
>>>> enable.autocommit is not such a bad option by itself, but there is a
>>>> corner case where it can cause issues like this.
>>>> When a reader is initialized, it's start offset is determined in this
>>>> (specifically in Dataflow, but roughly accurate on other runners too):
>>>>
>>>>- (a) If there is a  checkpoint for the reader split  (true for all
>>>>reads except for very first read bundle read by the split from Kafka), 
>>>> the
>>>>offset comes from checkpoint. This is how exactly once is ensures. Here 
>>>> the
>>>>offset commit by Kafka client with 'autocommit' does not matter.
>>>>- (b) If there is no checkpoint, (i.e. for the first bundle of
>>>>records) KafkaIO does not set any offset explicitly and lets Kafka 
>>>> client
>>>>decide. That implies it depend on your ConsumrConfig. So ConsumerConfig
>>>>decides the offset when a pipeline first starts.
>>>>
>>>> In your example, when there was an exception for Message 25, it was
>>>> still processing the first bundle of records and there was no Dataflow
>>>> checkpoint. It kept hitting (b). Kafka's 'autocommit' is out of bounds, and
>>>> it might have committed offset 60 in one of the reties. The next retry
>>>> incorrectly reads from 60.
>>>>
>>>> I hope this helps. Enabling autocommit in only useful when you want to
>>>> restart your pipeline from scratch (rather than 'updating' you Dataflow
>>>> pipeline) and still want to *roughly* resume from where the previous
>>>> pipeline left off. Eve

Re: Retry on failures when using KafkaIO and Google Dataflow

2018-08-16 Thread Raghu Angadi
I am not sure if it helps much. External checkpoints like Kafka
'autocommit' outside Beam's own checkpoint domain will always have quirks
like this. I wanted to ask what their use case for using autocommit was
(over commitOffsetsInFinalize())

If we wanted to, it is not clear to how an unbounded source can do that. A
reader (by design I think) does not have visibility into when a checkpoint
happens, bundle boundaries etc. E.g. it can try returning 'false' for
start() and advance() initially, but does not know when to actually return
records (in Dataflow, it could wait a few seconds).

On Thu, Aug 16, 2018 at 8:21 AM Lukasz Cwik  wrote:

> Raghu, based upon your description, do you think it would be a good for
> KafkaIO to checkpoint on the first read without producing any actual
> records?
>
> On Wed, Aug 15, 2018 at 11:49 AM Raghu Angadi  wrote:
>
>>
>> It is due to "enable.autocommit=true".  Auto commit is an option to Kafka
>> client and how and when it commits is totally out of control of Beam &
>> KafkaIO.
>> Could you try setting commitOffsetsInFinalize()[1] in KafkaIO rather than
>> 'enable.autocommit'? That would ensure exactly once processing.
>>
>> That said, you might be interested in understanding why your example
>> failed:
>> enable.autocommit is not such a bad option by itself, but there is a
>> corner case where it can cause issues like this.
>> When a reader is initialized, it's start offset is determined in this
>> (specifically in Dataflow, but roughly accurate on other runners too):
>>
>>- (a) If there is a  checkpoint for the reader split  (true for all
>>reads except for very first read bundle read by the split from Kafka), the
>>offset comes from checkpoint. This is how exactly once is ensures. Here 
>> the
>>offset commit by Kafka client with 'autocommit' does not matter.
>>- (b) If there is no checkpoint, (i.e. for the first bundle of
>>records) KafkaIO does not set any offset explicitly and lets Kafka client
>>decide. That implies it depend on your ConsumrConfig. So ConsumerConfig
>>decides the offset when a pipeline first starts.
>>
>> In your example, when there was an exception for Message 25, it was still
>> processing the first bundle of records and there was no Dataflow
>> checkpoint. It kept hitting (b). Kafka's 'autocommit' is out of bounds, and
>> it might have committed offset 60 in one of the reties. The next retry
>> incorrectly reads from 60.
>>
>> I hope this helps. Enabling autocommit in only useful when you want to
>> restart your pipeline from scratch (rather than 'updating' you Dataflow
>> pipeline) and still want to *roughly* resume from where the previous
>> pipeline left off. Even there, commitOffsetsInFinalize() is better. In
>> either case, exactly once processing is not guaranteed when a pipeline
>> restart, only way currently to achieve that is to 'update' the pipeline.
>>
>> [1]:
>> https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java#L623
>>
>> Raghu.
>>
>> On Wed, Aug 15, 2018 at 10:14 AM Leonardo Miguel <
>> leonardo.mig...@arquivei.com.br> wrote:
>>
>>> Hello,
>>>
>>> I'm using the KafkaIO source in my Beam pipeline, testing the scenario
>>> where intermitent errors may happen (e.g. DB connection failure) with the
>>> Dataflow runner.
>>> So I produced a sequence of 200 messages containing sequential numbers
>>> (1-200) to a topic, and then executed the following pipeline:
>>>
>>> p.apply("Read from kafka", KafkaIO.read()
>>> .withBootstrapServers(server)
>>> .withTopic(topic)
>>> .withKeyDeserializer(StringDeserializer.class)
>>> .withValueDeserializer(StringDeserializer.class)
>>> .updateConsumerProperties(properties)
>>> .withoutMetadata())
>>> .apply(Values.create())
>>> .apply(ParDo.of(new StepTest()));
>>>
>>> Where StepTest is defined as follows:
>>>
>>> public class StepTest extends DoFn {
>>> @ProcessElement
>>> public void processElement(ProcessContext pc) {
>>> String element = pc.element();
>>>
>>> if (randomErrorOccurs()) {
>>> throw new RuntimeException("Failed ... " + element);
>>> } else {
>>> LOG.info(element);
>>> }
>>> }
>>> }
>>>
>>> The consumer configuration has "enable.auto.commit=true".
>>> I wo

Re: Retry on failures when using KafkaIO and Google Dataflow

2018-08-15 Thread Raghu Angadi
It is due to "enable.autocommit=true".  Auto commit is an option to Kafka
client and how and when it commits is totally out of control of Beam &
KafkaIO.
Could you try setting commitOffsetsInFinalize()[1] in KafkaIO rather than
'enable.autocommit'? That would ensure exactly once processing.

That said, you might be interested in understanding why your example
failed:
enable.autocommit is not such a bad option by itself, but there is a corner
case where it can cause issues like this.
When a reader is initialized, it's start offset is determined in this
(specifically in Dataflow, but roughly accurate on other runners too):

   - (a) If there is a  checkpoint for the reader split  (true for all
   reads except for very first read bundle read by the split from Kafka), the
   offset comes from checkpoint. This is how exactly once is ensures. Here the
   offset commit by Kafka client with 'autocommit' does not matter.
   - (b) If there is no checkpoint, (i.e. for the first bundle of records)
   KafkaIO does not set any offset explicitly and lets Kafka client decide.
   That implies it depend on your ConsumrConfig. So ConsumerConfig decides the
   offset when a pipeline first starts.

In your example, when there was an exception for Message 25, it was still
processing the first bundle of records and there was no Dataflow
checkpoint. It kept hitting (b). Kafka's 'autocommit' is out of bounds, and
it might have committed offset 60 in one of the reties. The next retry
incorrectly reads from 60.

I hope this helps. Enabling autocommit in only useful when you want to
restart your pipeline from scratch (rather than 'updating' you Dataflow
pipeline) and still want to *roughly* resume from where the previous
pipeline left off. Even there, commitOffsetsInFinalize() is better. In
either case, exactly once processing is not guaranteed when a pipeline
restart, only way currently to achieve that is to 'update' the pipeline.

[1]:
https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java#L623

Raghu.

On Wed, Aug 15, 2018 at 10:14 AM Leonardo Miguel <
leonardo.mig...@arquivei.com.br> wrote:

> Hello,
>
> I'm using the KafkaIO source in my Beam pipeline, testing the scenario
> where intermitent errors may happen (e.g. DB connection failure) with the
> Dataflow runner.
> So I produced a sequence of 200 messages containing sequential numbers
> (1-200) to a topic, and then executed the following pipeline:
>
> p.apply("Read from kafka", KafkaIO.read()
> .withBootstrapServers(server)
> .withTopic(topic)
> .withKeyDeserializer(StringDeserializer.class)
> .withValueDeserializer(StringDeserializer.class)
> .updateConsumerProperties(properties)
> .withoutMetadata())
> .apply(Values.create())
> .apply(ParDo.of(new StepTest()));
>
> Where StepTest is defined as follows:
>
> public class StepTest extends DoFn {
> @ProcessElement
> public void processElement(ProcessContext pc) {
> String element = pc.element();
>
> if (randomErrorOccurs()) {
> throw new RuntimeException("Failed ... " + element);
> } else {
> LOG.info(element);
> }
> }
> }
>
> The consumer configuration has "enable.auto.commit=true".
> I would expect that all the numbers get printed, and if an exception is
> thrown, Dataflow's runner would retry processing that failed message until
> it eventually works.
> However, what happened in my pipeline was different: when errors start
> happening due to my code, it caused some messages to be never processed,
> and some were actually lost forever.
>
> I would expect something like:
>
> {...}
> 22
> 23
> 24
> Failed ... 25
> {A new reader starts}
> Reader-0: first record offset 60
> 61
> 62
> {...}
> {Dataflow retries 25}
> Failed ... 25
> {...}
> and so on... (exception would never cease to happen in this case and
> Dataflow would retry forever)
>
> My output was something like:
>
> {...}
> 22
> 23
> 24
> Failed ... 25
> {A new reader starts}
> Reader-0: first record offset 60
> 61
> 62
> {...}
>
> Message #25 never gets reprocessed, and all the messages up to 60 are
> lost, probably the ones in the same processing bundle as 25. Even more
> curious is that this behaviour doesn't happen when using the PubSubIO
> source, which produces the first mentioned output.
>
> My questions are:
> What is a good way of handling errors with Kafka source if I want all
> messages to be processed exactly once?
> Is there any Kafka or Dataflow configuration that I may be missing?
> Please let me know of your thoughts.
>
> Andre (cc) is part of our team and will be together in this discussion.
>
> --
> []s
>
> Leonardo Alves Miguel
> Data Engineer
> (16) 3509- | www.arquivei.com.br
> 
> [image: Arquivei.com.br – Inteligência em Notas Fiscais]
> 
> 

Re: Write bulks files from streaming app

2018-07-20 Thread Raghu Angadi
On Fri, Jul 20, 2018 at 2:58 AM Jozef Vilcek  wrote:

> Hm, that is interesting idea to make the write composite and merge files
> later. Do not know Beam well yet.
> I will look into it and learn about Wait.on() transform (wonder how it
> will work with late fires). Thanks!
>
> But keeps me thinking...
> Does it make sense to have support from SDK?
> Is my use case that uncommon? Not fit for Beam? How does others out there
> does similar thing?
>

SDK does allow it. Looks like you are running into scaling and memory
limits with amount of state stored in large windows. This is something that
will improve. I am not familiar enough with Flink runner to  comment on
specifics. I was mainly thinking of a work around.

Raghu.


>
> On Thu, Jul 19, 2018 at 11:21 PM Raghu Angadi  wrote:
>
>> One option (but requires more code): Write to smaller files with frequent
>> triggers to directory_X and once the window properly closes, copy all the
>> files to a single file in your own DoFn. This is certainly more code on
>> your part, but might be worth it. You can use Wait.on() transoform to run
>> your finalizer DoFn right after the window that writes smaller files closes.
>>
>>
>> On Thu, Jul 19, 2018 at 2:43 AM Jozef Vilcek 
>> wrote:
>>
>>> Hey,
>>>
>>> I am looking for the advice.
>>>
>>> I am trying to do a stream processing with Beam on Flink runtime.
>>> Reading data from Kafka, doing some processing with it which is not
>>> important here and in the same time want to store consumed data to history
>>> storage for archive and reprocessing, which is HDFS.
>>>
>>> Now, the part of writing batches to HDFS is giving me hard time.
>>> Logically, I want to do:
>>>
>>> fileIO = FileIO.writeDynamic()
>>> .by(destinationFn)
>>> .via(AvroIO.sink(avroClass))
>>> .to(path)
>>> .withNaming(namingFn)
>>> .withTempDirectory(tmp)
>>> .withNumShards(shards)
>>>
>>> data
>>>.withFixedWindow(1H, afterWatermarkTrigger, discardFiredPanes)
>>>.saveTo(fileIO)
>>>
>>>
>>> This write generates in Flink execution graph 3 operators, which I do
>>> not full understand yet.
>>>
>>> Now, the problem is, that I am not able to run this at scale.
>>>
>>> If I want to write big enough files to not to have lots of files on
>>> HDFS, I keep running into the OOM. With Flink, I use rocksdb state backend
>>> and I was warned about this JIRA which is probably related to my OOM
>>> https://issues.apache.org/jira/browse/FLINK-8297
>>> Therefore, I need to trigger more often and small batches which leads to
>>> too many files on HDFS.
>>>
>>> Question here is, if there is some path I do not see how to make this
>>> work ( write bulks of data to HDFS of my choosing without running to memory
>>> troubles ). Also, keeping whole window data which is designated for write
>>> to output to filesystem in state involves more IO.
>>>
>>> Thanks for any thoughts and guidelines,
>>> Jozef
>>>
>>>


Re: Write bulks files from streaming app

2018-07-19 Thread Raghu Angadi
One option (but requires more code): Write to smaller files with frequent
triggers to directory_X and once the window properly closes, copy all the
files to a single file in your own DoFn. This is certainly more code on
your part, but might be worth it. You can use Wait.on() transoform to run
your finalizer DoFn right after the window that writes smaller files closes.


On Thu, Jul 19, 2018 at 2:43 AM Jozef Vilcek  wrote:

> Hey,
>
> I am looking for the advice.
>
> I am trying to do a stream processing with Beam on Flink runtime. Reading
> data from Kafka, doing some processing with it which is not important here
> and in the same time want to store consumed data to history storage for
> archive and reprocessing, which is HDFS.
>
> Now, the part of writing batches to HDFS is giving me hard time.
> Logically, I want to do:
>
> fileIO = FileIO.writeDynamic()
> .by(destinationFn)
> .via(AvroIO.sink(avroClass))
> .to(path)
> .withNaming(namingFn)
> .withTempDirectory(tmp)
> .withNumShards(shards)
>
> data
>.withFixedWindow(1H, afterWatermarkTrigger, discardFiredPanes)
>.saveTo(fileIO)
>
>
> This write generates in Flink execution graph 3 operators, which I do not
> full understand yet.
>
> Now, the problem is, that I am not able to run this at scale.
>
> If I want to write big enough files to not to have lots of files on HDFS,
> I keep running into the OOM. With Flink, I use rocksdb state backend and I
> was warned about this JIRA which is probably related to my OOM
> https://issues.apache.org/jira/browse/FLINK-8297
> Therefore, I need to trigger more often and small batches which leads to
> too many files on HDFS.
>
> Question here is, if there is some path I do not see how to make this work
> ( write bulks of data to HDFS of my choosing without running to memory
> troubles ). Also, keeping whole window data which is designated for write
> to output to filesystem in state involves more IO.
>
> Thanks for any thoughts and guidelines,
> Jozef
>
>


Re: Routing events by key

2018-07-09 Thread Raghu Angadi
JB, reshuffle deprecation is discussed mutliple times, most recent one is
from May :
https://lists.apache.org/thread.html/820064a81c86a6d44f21f0d6c68ea3f46cec151e5e1a0b52eeed3fbf@%3Cdev.beam.apache.org%3E
I filed https://issues.apache.org/jira/browse/BEAM-4372. Summary: It can be
used, and perhaps needs a rename. Just mentioned the thread here as a
reference, further discussion could be directed to the above dev thread or
the jira.

Thanks Robert for more detailed description.

Raghu.

On Fri, Jul 6, 2018 at 11:11 PM Jean-Baptiste Onofré 
wrote:

> Hi Raghu,
>
> AFAIR, Reshuffle is considered as deprecated. Maybe it would be better
> to avoid to use it no ?
>
> Regards
> JB
>
> On 06/07/2018 18:28, Raghu Angadi wrote:
> > I would use Reshuffle()[1] with entity id as the key. It internally does
> > a GroupByKey and sets up windowing such that it does not buffer anything.
> >
> > [1]
> > :
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java#L51
>
> >
> > On Fri, Jul 6, 2018 at 8:01 AM Niels Basjes  > <mailto:ni...@basjes.nl>> wrote:
> >
> > Hi,
> >
> > I have an unbounded stream of change events each of which has the id
> > of the entity that is changed.
> > To avoid the need for locking in the persistence layer that is
> > needed in part of my processing I want to route all events based on
> > this entity id.
> > That way I know for sure that all events around a single entity go
> > through the same instance of my processing sequentially, hence no
> > need for locking or other synchronization regarding this persistence.
> >
> > At this point my best guess is that I need to use the GroupByKey but
> > that seems to need a Window.
> > I think I don't want a window because the stream is unbounded and I
> > want the lowest possible latency (i.e. a Window of 1 second would be
> > ok for this usecase).
> > Also I want to be 100% sure that all events for a specific id go to
> > only a single instance because I do not want any race conditions.
> >
> > My simple question is: What does that look like in the Beam Java API?
> >
> > --
> > Best regards / Met vriendelijke groeten,
> >
> > Niels Basjes
> >
>
> --
> Jean-Baptiste Onofré
> jbono...@apache.org
> http://blog.nanthrax.net
> Talend - http://www.talend.com
>


Re: Routing events by key

2018-07-06 Thread Raghu Angadi
I would use Reshuffle()[1] with entity id as the key. It internally does a
GroupByKey and sets up windowing such that it does not buffer anything.

[1] :
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java#L51


On Fri, Jul 6, 2018 at 8:01 AM Niels Basjes  wrote:

> Hi,
>
> I have an unbounded stream of change events each of which has the id of
> the entity that is changed.
> To avoid the need for locking in the persistence layer that is needed in
> part of my processing I want to route all events based on this entity id.
> That way I know for sure that all events around a single entity go through
> the same instance of my processing sequentially, hence no need for locking
> or other synchronization regarding this persistence.
>
> At this point my best guess is that I need to use the GroupByKey but that
> seems to need a Window.
> I think I don't want a window because the stream is unbounded and I want
> the lowest possible latency (i.e. a Window of 1 second would be ok for this
> usecase).
> Also I want to be 100% sure that all events for a specific id go to only a
> single instance because I do not want any race conditions.
>
> My simple question is: What does that look like in the Beam Java API?
>
> --
> Best regards / Met vriendelijke groeten,
>
> Niels Basjes
>


Re: PubSubIO throughput/parallelism options?

2018-06-22 Thread Raghu Angadi
On Fri, Jun 22, 2018 at 3:06 PM Lukasz Cwik  wrote:

> Are you sure you have scaled your pipeline up based upon the amount of
> traffic your receiving?
> What do you see your average CPU utilization?
> Do you have any functions which perform blocking calls?
>

Agreed. Need to look into why the job is not processing as much input as it
should.

There is no such setting in PubsubIO. In Dataflow, the messages are pulled
by streaming worker ('shuffler') using multiple threads. These are not
pulled by Java worker running user code.

Raghu.

>
> On Fri, Jun 22, 2018 at 4:26 AM Hrish  wrote:
>
>> I am using PubSubIO to read from Google PubSub as an unbounded source.
>> I've noticed that the rate at which is receives messages is pretty slow
>> compared to the rate at which messages are being pushed.
>>
>> In the Google PubSub APIs, I can tweak a pull subscriber's settings using
>> Subscriber.defaultBuilder().setExecutorProvider, where I can set the core
>> pool size for the thread pool that will pull messages.
>>
>> Is there any equivalent way of tuning the PubSubIO class as well? Or any
>> other way of achieving a higher pull rate?
>>
>


Re: kafkaIO Run with Spark Runner: "streaming-job-executor-0"

2018-06-13 Thread Raghu Angadi
Can you check the logs on the worker?

On Wed, Jun 13, 2018 at 2:26 AM  wrote:

> Dear all,
>
>
>
> I am using the kafkaIO in my project (Beam 2.0.0 with Spark runner).
>
> My running environment is:
>
> OS: Ubuntn 14.04.3 LTS
>
> The different version for these tools:
>
> JAVA: JDK 1.8
>
> Beam 2.0.0 (Spark runner with Standalone mode)
>
> Spark 1.6.0
>
> Standalone mode :One driver node: ubuntu7; One master node: ubuntu8; Two
> worker nodes: ubuntu8 and ubuntu9
>
> Kafka: 2.10-0.10.1.1
>
>
>
> The java code of my project is:
>
>
> ==
>
> SparkPipelineOptions options = PipelineOptionsFactory.*as*
> (SparkPipelineOptions.*class*);
>
> options.setRunner(SparkRunner.*class*);
>
> options.setSparkMaster("spark://ubuntu8:7077");
>
> options.setAppName("App kafkaBeamTest");
>
> options.setJobName("Job kafkaBeamTest");
>
> *options**.setMaxRecordsPerBatch(1000L);*
>
>
>
> Pipeline p = Pipeline.create(options);
>
>
>
> System.out.println("Beamtokafka");
>
> PCollection> readData = p.apply(KafkaIO. String>read()
>
> .withBootstrapServers(ubuntu7:9092)
>
> .withTopic("kafkasink")
>
> .withKeyDeserializer(LongDeserializer.class)
>
> .withValueDeserializer(StringDeserializer.class)
>
>.withoutMetadata()
>
>);
>
>
>
> PCollection> readDivideData = readData.
>
>
> apply(Window.>into(FixedWindows.of(Duration.standardSeconds(1)))
>
>  .triggering(AfterWatermark.pastEndOfWindow()
>
>
> .withLateFirings(AfterProcessingTime.pastFirstElementInPane().plusDelayOf(Duration.ZERO)))
>
>  .withAllowedLateness(Duration.ZERO) .discardingFiredPanes());
>
>
>
> System.out.println("CountData");
>
>
>
> PCollection> countData =
> readDivideData.apply(Count.perKey());
>
>
>
> p.run();
>
>
> ==
>
>
>
> The message of error is:
>
>
> ==
>
> Exception in thread "streaming-job-executor-0" java.lang.Error:
> java.lang.InterruptedException
>
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1155)
>
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
> at java.lang.Thread.run(Thread.java:748)
>
> Caused by: java.lang.InterruptedException
>
> at java.lang.Object.wait(Native Method)
>
> at java.lang.Object.wait(Object.java:502)
>
> at
> org.apache.spark.scheduler.JobWaiter.awaitResult(JobWaiter.scala:73)
>
> at
> org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:612)
>
> at org.apache.spark.SparkContext.runJob(SparkContext.scala:1832)
>
> at org.apache.spark.SparkContext.runJob(SparkContext.scala:1845)
>
> at org.apache.spark.SparkContext.runJob(SparkContext.scala:1858)
>
> at org.apache.spark.SparkContext.runJob(SparkContext.scala:1929)
>
> at org.apache.spark.rdd.RDD$$anonfun$foreach$1.apply(RDD.scala:912)
>
> at org.apache.spark.rdd.RDD$$anonfun$foreach$1.apply(RDD.scala:910)
>
> at
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150)
>
> at
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:111)
>
> at org.apache.spark.rdd.RDD.withScope(RDD.scala:316)
>
> at org.apache.spark.rdd.RDD.foreach(RDD.scala:910)
>
> …
>
> at org.apache.spark.streaming.scheduler.Job.run(Job.scala:39)
>
> at
> org.apache.spark.streaming.scheduler.JobScheduler$JobHandler$$anonfun$run$1.apply$mcV$sp(JobScheduler.scala:224)
>
> at
> org.apache.spark.streaming.scheduler.JobScheduler$JobHandler$$anonfun$run$1.apply(JobScheduler.scala:224)
>
> at
> org.apache.spark.streaming.scheduler.JobScheduler$JobHandler$$anonfun$run$1.apply(JobScheduler.scala:224)
>
> at scala.util.DynamicVariable.withValue(DynamicVariable.scala:57)
>
> at
> org.apache.spark.streaming.scheduler.JobScheduler$JobHandler.run(JobScheduler.scala:223)
>
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>
> ... 2 more
>
>
> ==
>
>
>
> Maven 3.5.0, in which related dependencies are listed in my project’s
> pom.xml:
>
> 
>
> org.apache.beam
>
>   beam-sdks-java-core
>
>   2.0.0
>
> 
>
> 
>
> org.apache.beam
>
>beam-sdks-java-io-kafka
>
>2.0.0
>
> 
>
> 
>
> org.apache.spark
>
>   spark-core_2.10
>
>   1.6.0
>
> 
>
> 
>
> org.apache.spark
>
>   spark-streaming_2.10
>
>   1.6.0
>
> 
>
>
>
> 
>
> org.apache.kafka
>
>   kafka-clients
>
>   0.10.1.1
>
> 
>
> 
>
> org.apache.kafka
>
>   kafka_2.10
>
>   0.10.1.1
>
> 
>
>
>
>
>
> When I use the above code in Spark Runner (Local [4]), this project worked
> well (2000~4000 data/s). However, if I run it on Standalone mode, it failed
> along with the above error.
>
>
>
> If you have any idea about the error ("streaming-job-executor-0"), I 

Re: [FYI] New Apache Beam Swag Store!

2018-06-08 Thread Raghu Angadi
Woo-hoo! This is terrific.

If we are increasing color choices I would like black or charcoal... Beam
logo would really pop on a dark background.

On Fri, Jun 8, 2018 at 3:32 PM Griselda Cuevas  wrote:

> Hi Beam Community,
>
> I just want to share with you the exciting news about our brand new Apache
> Beam Swag Store!
>
> You can find it here: https://store-beam.myshopify.com/
>
> *How does it work?*
>
>- You can just select the items you want and check-out. Our Vendor
>ships to anywhere in the world and normally can have swag to be delivered
>within 1 week. Each company or user will need to pay for their own swag.
>- If you are hosting an event or representing Beam at one, reach out
>to me or the beam-events-meetups slack channel, I'll be happy to review
>your event and see if we can sponsor the swag. We'll have codes for this
>occasions thanks to Google, who has sponsored an initial inventory.
>
> If you have feedback, ideas on new swag, questions or suggestions, reach
> out to me and/or Matthias Baetens.
>
> Happy Friday!
> G
>
>
>


Re: GlobalWindows, Watermarks and triggers

2018-06-06 Thread Raghu Angadi
> * the PCollection has a notion of "which elements it has emitted for a
given position of the watermark"
This is not correct (to me it reads to me like you are saying something
close to 'PCollection is a stream of (element, watermark) tuples').
Every element in a PCollection has an associated with a event_timestamp..
it is a tuple of (element, timestamp) tuples. Watermark is not associated
with a PCollection, and is completely independent of event timestamp.

IOW, Watermark is by definition monotonic. When a stage in your pipeline
sets its watermark to 'X' what it means is that each of its inputs (sources
like PubSub, or stages) has communicated a watermark timestamp Y saying it
expects all its future elements will have event timestamp >= Y.  X =
min(Y). A custom source that receives events with monotonically increasing
timestamp can just report the timestamp of the last element emitted as it
watermark.

OnElementCountAtLeast(1) has no dependence on watermark progression, since
trigger does not depend on time.


On Mon, Jun 4, 2018 at 12:07 PM Stephan Kotze 
wrote:

> Hi there.
>
> I have a question regarding the completeness of results in a GlobalWindow
> for a pipeline which receives all events in order. (no late/missing data).
>
> The question is based on reasoning about Beam that takes 3 pieces of (our
> current) understanding into account:
>
> 1)Global Window Watermark
> As I understand it a PCollection with a GlobalWindow and ParDo will be
> running a watermark (which is what allows Triggers in stateful DoFns to
> fire for example).
>
> If this is the case,
>  * the PCollection has a notion of "which elements it has emitted for a
> given position of the watermark"
>  * the PCollection will also know which results from upstream
> PTransforms/Pcollections etc. are still in flight
>  * the PCollection will emit results and update its watermark once
> Upstream elements have all provided their results and shifted their
> watermarks.
>
> 2) Triggering on Watermark
>  For Fixed windows for example we have the
> "AfterWatermark".pastEndOfWindow() trigger.
>  In the case of Global windows however, the GlobalWindow never "ends", so
> the watermark cannot progress past this point and we'll never get any
> results for something like
> newGlobalWindow().trigger(AfterWatermark.pastEndOfWindow())
>
> 3) Ordering between upstream PCollections/PTransforms and GlobalWindows
> In the following pipeline:  Source -> fixedWindow(1m) -> GlobalWindow(),
> the 1Min segments can arrive out of order in the global window, even if the
> source was ordered (with no late data)
>
> Thus the questions.
>
> If I have a ParDo on the GlobalWindow that is triggered by
> OnElementCountAtLeast(1)  and events can arrive out of order, how can the
> ParDo have a watermark that moves only forward when it's possible To
> trigger on any amount of elements having arrived (this would appear to
> allow the emission of 2 later events, followed by an earlier event for
> another trigger).
>
> Or?
>
> Does the OnElementCountAtLeast only trigger once ALL upstream elements up
> to and including the watermark have arrived? (Though they may be unordered
> in the DoFn's input for example, it is still a complete list with All
> upstream produced elements between the last watermark and the "new" one
> that will be set once the ParDo has completed).
>
> I stress the point because it has some important repercussions for us (so
> I'm just paraphrasing the question slightly below, to try and make it as
> clear as possible :))
>
> How can a PTransform/PCollection on a Global Window have a monotonic
> watermark if events can trigger calcs with out of order events (when using
> a trigger such as OnElementCountAtLeast(1)), or is there a Guarantee, that
> when the trigger fires, we will receive a complete list of upstream results
> up to the time of the latest event in the Collection we receive to process.
>
> Hopefully I've explained the question concisely enough :)
>
> Stephan
>
>


Re: Understanding GenerateSequence and SideInputs

2018-05-24 Thread Raghu Angadi
On Thu, May 24, 2018 at 1:11 PM Carlos Alonso  wrote:

> Hi everyone!!
>
> I'm building a pipeline to store streaming data into BQ and I'm using the
> pattern: Slowly changing lookup cache described here:
> https://cloud.google.com/blog/big-data/2017/06/guide-to-common-cloud-dataflow-use-case-patterns-part-1
>  to
> hold and refresh the table schemas (as they may change from time to time).
>
> Now I'd like to understand how that is scheduled on a distributed system.
> Who is running that code? One random node? One node but always the same?
> All nodes?
>

GenerateSequence() is uses an unbounded source. Like any unbounded source,
it can has a set of 'splits' ('desiredNumSplits' [1] is set by runtime).
Each of the splits run in parallel.. a typical runtime distributes these
across the workers. Typically they stay on a worker unless there is a
reason to redistribute (autoscaling, workers unresponsive etc). W.r.t. user
application there are no guarantees about affinity.

[1]
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java#L337


>
> Also, what are the GenerateSequence guarantees in terms of precision? I
> have it configured to generate 1 element every 5 minutes and most of the
> time it works exact, but sometimes it doesn't... Is that expected?
>

Each of the splits mentioned above essentially runs 'advance() [2]' in a
loop. It check current walltime to decide if it needs to emit next element.
If the value you see off by a few seconds, it would imply 'advance()' was
not called during that time by the framework. Runtime frameworks usually
don't provide any hard or soft deadlines for scheduling work.

[1]
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java#L337

[2]
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java#L426


> Regards
>


Re: Controlling parallelism of a ParDo Transform while writing to DB

2018-05-17 Thread Raghu Angadi
If you want tight control on parallelism, you can 'reshuffle' the elements
into fixed number of shards. See
https://stackoverflow.com/questions/46116443/dataflow-streaming-job-not-scaleing-past-1-worker

On Tue, May 15, 2018 at 11:21 AM Harshvardhan Agrawal <
harshvardhan.ag...@gmail.com> wrote:

> Hi Guys,
>
> I am currently in the process of developing a pipeline using Apache Beam
> with Flink as an execution engine. As a part of the process I read data
> from Kafka and perform a bunch of transformations that involve joins,
> aggregations as well as lookups to an external DB.
>
> The idea is that we want to have higher parallelism with Flink when we are
> performing the aggregations but eventually coalesce the data and have
> lesser number of processes writing to the DB so that the target DB can
> handle it (for example say I want to have a parallelism of 40 for
> aggregations but only 10 when writing to target DB).
>
> Is there any way we could do that in Beam?
>
> Regards,
>
> Harsh
> --
>
> *Regards,Harshvardhan Agrawal*
> *267.991.6618 | LinkedIn *
>


Re: Create SideInputs for PTranform using lookups for each window of data

2018-05-15 Thread Raghu Angadi
You are applying windowing to 'billingDataPairs' in the example above. Side
input pairs with all the main input windows that exactly match or
completely fall within the side input window. Common use case is a side
input defined in default global window and it matches all the main input
windows.



On Tue, May 15, 2018 at 4:40 PM Harshvardhan Agrawal <
harshvardhan.ag...@gmail.com> wrote:

> Got it.
>
> Since I am not applying any windowing strategy to the side input, does
> beam automatically pickup the windowing strategy for the side inputs from
> the main input? By that I mean the scope of the side input would be a per
> window one and it would be different for every window. Is that correct?
>
> Regards,
> Harsh
>
> On Tue, May 15, 2018 at 17:54 Lukasz Cwik <lc...@google.com> wrote:
>
>> Using deduplicate + side inputs will allow you to have a consistent view
>> of the account information for the entire window which can be nice since it
>> gives consistent processing semantics but using a simple in memory cache to
>> reduce the amount of lookups will likely be much easier to debug and
>> simpler to implement and maintain.
>>
>> On Tue, May 15, 2018 at 2:31 PM Harshvardhan Agrawal <
>> harshvardhan.ag...@gmail.com> wrote:
>>
>>> Thanks Raghu!
>>>
>>> Lukasz,
>>>
>>> Do you think lookups would be a better option than side inputs in my
>>> case?
>>>
>>>
>>> On Tue, May 15, 2018 at 16:33 Raghu Angadi <rang...@google.com> wrote:
>>>
>>>> It should work. I think you need apply Distinct before looking up
>>>> account info :
>>>> billingDataPairs.apply(Keys.create()).apply(Distinct.create()).apply("LookupAccounts",
>>>> ...).
>>>> Note that all of the accounts are stored in single in-memory map. It
>>>> should be small enough for that.
>>>>
>>>> On Tue, May 15, 2018 at 1:15 PM Harshvardhan Agrawal <
>>>> harshvardhan.ag...@gmail.com> wrote:
>>>>
>>>>> Well ideally, I actually made the example a little easy. In the actual
>>>>> example I have multiple reference datasets. Say, I have a tuple of Account
>>>>> and Product as the key. The reason we don’t do the lookup in the DoFn
>>>>> directly is that we don’t want to lookup the data for the same account or
>>>>> same product multiple times across workers in a window.
>>>>>
>>>>> What I was thinking was that it might be better to perform the lookup
>>>>> only once for each account and product in a window and then supply them as
>>>>> side inputs to the main input.
>>>>>
>>>>> On Tue, May 15, 2018 at 16:03 Lukasz Cwik <lc...@google.com> wrote:
>>>>>
>>>>>> Is there a reason you don't want to read the accounting information
>>>>>> within the DoFn directly from the datastore, it seems like that would be
>>>>>> your simplest approach.
>>>>>>
>>>>>> On Tue, May 15, 2018 at 12:43 PM Harshvardhan Agrawal <
>>>>>> harshvardhan.ag...@gmail.com> wrote:
>>>>>>
>>>>>>> Hi,
>>>>>>>
>>>>>>> No we don’t receive any such information from Kafka.
>>>>>>>
>>>>>>> The account information in the external store does change. Every
>>>>>>> time we have a change in the account information we will have to 
>>>>>>> recompute
>>>>>>> all the billing info. Our source systems will make sure that they 
>>>>>>> publish
>>>>>>> messages for those accounts again.
>>>>>>>
>>>>>>>
>>>>>>> On Tue, May 15, 2018 at 15:11 Lukasz Cwik <lc...@google.com> wrote:
>>>>>>>
>>>>>>>> For each BillingModel you receive over Kafka, how "fresh" should
>>>>>>>> the account information be?
>>>>>>>> Does the account information in the external store change?
>>>>>>>>
>>>>>>>> On Tue, May 15, 2018 at 11:22 AM Harshvardhan Agrawal <
>>>>>>>> harshvardhan.ag...@gmail.com> wrote:
>>>>>>>>
>>>>>>>>> Hi,
>>>>>>>>>
>>>>>>>>> We have certain billing data that arrives to us from Kafka. The
>>>>>>>>>

Re: Create SideInputs for PTranform using lookups for each window of data

2018-05-15 Thread Raghu Angadi
It should work. I think you need apply Distinct before looking up account
info :
billingDataPairs.apply(Keys.create()).apply(Distinct.create()).apply("LookupAccounts",
...).
Note that all of the accounts are stored in single in-memory map. It should
be small enough for that.

On Tue, May 15, 2018 at 1:15 PM Harshvardhan Agrawal <
harshvardhan.ag...@gmail.com> wrote:

> Well ideally, I actually made the example a little easy. In the actual
> example I have multiple reference datasets. Say, I have a tuple of Account
> and Product as the key. The reason we don’t do the lookup in the DoFn
> directly is that we don’t want to lookup the data for the same account or
> same product multiple times across workers in a window.
>
> What I was thinking was that it might be better to perform the lookup only
> once for each account and product in a window and then supply them as side
> inputs to the main input.
>
> On Tue, May 15, 2018 at 16:03 Lukasz Cwik  wrote:
>
>> Is there a reason you don't want to read the accounting information
>> within the DoFn directly from the datastore, it seems like that would be
>> your simplest approach.
>>
>> On Tue, May 15, 2018 at 12:43 PM Harshvardhan Agrawal <
>> harshvardhan.ag...@gmail.com> wrote:
>>
>>> Hi,
>>>
>>> No we don’t receive any such information from Kafka.
>>>
>>> The account information in the external store does change. Every time we
>>> have a change in the account information we will have to recompute all the
>>> billing info. Our source systems will make sure that they publish messages
>>> for those accounts again.
>>>
>>>
>>> On Tue, May 15, 2018 at 15:11 Lukasz Cwik  wrote:
>>>
 For each BillingModel you receive over Kafka, how "fresh" should the
 account information be?
 Does the account information in the external store change?

 On Tue, May 15, 2018 at 11:22 AM Harshvardhan Agrawal <
 harshvardhan.ag...@gmail.com> wrote:

> Hi,
>
> We have certain billing data that arrives to us from Kafka. The
> billing data is in json and it contains an account ID. In order for us to
> generate the final report we need to use some account data associated with
> the account id and is stored in an external database.
>
> It is possible that we get multiple billing info messages for the same
> account. We want to be able to lookup the account information for the
> messages in a window and then supply that as a side input to the next
> PTransform.
>
> Is it possible to achieve that in Beam?
>
> Here is my attempt:
>
> PCollection> billingDataPairs =
> p.apply("ReadBillingInfo", KafkaIO.read()
>  .withBootstrapServers(KAFKA_BOOTSTRAP_SERVER)
>  .withTopic(KAFKA_TOPIC)
>  .withKeyDeserializer(StringDeserializer.class)
>  .withValueDeserializer(StringDeserializer.class)
>  )
>  .apply("Window",
> Window.into(FixedWindows.of(Duration.standardSeconds(30
>  .apply("ProcessKafkaMessages",new KafkaProcessor());
>
>  PCollection billingData =
> billingDataPairs.apply(GroupByKey.create());
>
>  PCollectionView> accountData =
> billingDataPairs.apply("LookupAccounts",new
> AccountLookupClient()).apply(View.asMap());
>
> billingDataPairs.apply(ParDo.of(new DoFn BillingModel>>(){
> @ProcessElement
> public void processElement(ProcessContext ctx) {
> Integer accountId = ctx.element().getKey();
> Iterable billingModel = ctx.element().getValue();
> Account account = ctx.sideinput(accountData).get(accountId);
> }
> }));
>
> Regards,
> Harsh
> --
>
> *Regards,Harshvardhan Agrawal*
> *267.991.6618 | LinkedIn
> *
>
 --
>>>
>>> *Regards,Harshvardhan Agrawal*
>>> *267.991.6618 | LinkedIn *
>>>
>> --
>
> *Regards,Harshvardhan Agrawal*
> *267.991.6618 | LinkedIn *
>


Re: Apache Beam delta between windows.

2018-05-04 Thread Raghu Angadi
Hi Stephan,

If you could provide a concrete example with a small input, and the output
you except, I think I would understand requirements better. I am still not
sure if simple SlidingWindows aren't enough for your use case.

On Fri, May 4, 2018 at 9:44 AM Lukasz Cwik  wrote:

> Thanks for updating the description and I feel as though I better
> understand your problem and your idea about publishing to a queue is clever
> and if you feel like it works for you you should go with it.
>
> Here are some more details about if you were going to use the overlapping
> sliding windows or a StatefulDoFn.
>
> With overlapping sliding windows, you'll have a fixed set of the 1 minute
> aggregations which means that you can only compute an aggregation
> approximation for T. This can be fine if your aggregation function uses
> something like an exponential decay as the further you go back, the less
> relevant the information is.
>
> If you use a StatefulDoFn, it is partitioned per key and window, which
> means that the only way you can see all the data for a single key is to use
> the global window. Every time a new record comes in, you look at all the 5m
> aggregations that you have stored in state and recompute and update state
> and also output each 5m aggregation that would have changed. Note that you
> should evict aggregations from state for records that are old (knowing when
> to evict is dependent on what is the trigger defined on the 1min
> aggregation function).
>
> Depending on which one is appropriate for your aggregation function, I
> believe using the overlapping sliding windows will be best as all garbage
> collection will be handled automatically by the system so you don't have to
> worry about leaks but if your aggregation function doesn't work well as an
> approximation, then use the StatefulDoFn or your idea with the queue.
>
>
> On Fri, May 4, 2018 at 9:39 AM Stephan Kotze 
> wrote:
>
>> More experimenting:.
>>
>> I can get the concept working using a CoGroupBy IF I break the circular
>> dependency by introducing a Message Queue for example: (Same should apply
>> for using the Queue as a side input)
>>
>> ...
>>
>> PCollection> per1minAggs = ...;
>>
>> PCollection> previous5minAggs = p
>> .apply(*READ_FROM_PREVIOUS_5MIN_AGGS_QUEUE()*)
>> .apply(*SHIFT_INTO_NEW_WINDOW_VIA_CUSTOM_WINDOWFN()*);
>>
>> PCollection> coGroup = KeyedPCollectionTuple
>> .of(per1minAggsTag, *PER_1MIN_TAG*)
>> .and(previousPer5minAggsTag, *PREVIOUS_PER_5MIN_TAG*)
>> .apply(CoGroupByKey.create());
>> ...
>>
>> apply(ParDo.of(new DoFn, KV> OrderAggregateTS>>() {
>> @ProcessElement
>> public void processElement(ProcessContext c, BoundedWindow w) {
>> 
>> Iterable current1MinResults =
>> getByTupleTag(*PER_1MIN_TAG*)
>> OrderAggregateTS previous5MinResult =
>> getByTupleTag(*PREVIOUS_PER_5MIN_TAG*)
>> result = woh(previous5MinResult,current1MinResults)!
>> c.output(result);
>> .
>> }
>> }))
>> .apply(WRITE_TO_PREVIOUS_5MIN_AGGS_QUEUE())
>>
>> Because I can create the DoFn without a direct "confluent api" reference,
>> I can declare the DoFn with a Stream of data that will be there when it
>> runs, but doesn't need to be available at "compile time"  so to speak.
>>
>> It feels like a hoop too many to jump through, which makes me think it's
>> not the correct way to approach the issue of trying to carry values between
>> invocations of the same DoFn.
>>
>> Am I missing something obvious?
>>
>> Stephan
>>
>>
>>
>> On Fri, May 4, 2018 at 12:40 PM, Stephan Kotze > > wrote:
>>
>>> Thanks Lukasz.
>>>
>>> So I've been playing with the problem a bit more (and updated the
>>> question on Stack Overflow to reflect).
>>>
>>> The custom windowFn, love it! Will learn to embrace it.
>>>
>>> The problem however, is now the recursive nature of the declaration of
>>> the DoFn/Stream/SideInput (which probably shows I don't understand the
>>> fundamental interplay properly).
>>>
>>> I think a bit of pseudo code might illustrate best:
>>>
>>> /*My 5 minute aggregator  (over 1 minute intervals) that also requires
>>> some if it's previous running results*/
>>>
>>>
>>> PCollection> per1MinAggs =
>>> PCollection> per5MinAggs =
>>> per1MinAggs
>>> .apply(Window.into(5m))
>>> .apply(GroupByKey.create())
>>> .apply(
>>> ParDo.of(
>>> new CalculateAggregateFromAggregateTS()
>>> )
>>> .withSideInputs(previous5MinAggsResultView)   //*1
>>>   );
>>>
>>>
>>> //My view projecting backwards
>>> PCollectionView> previous5MinAggsResultView =
>>> per5MinAggs //*2
>>> 

Re: Apache Beam delta between windows.

2018-05-03 Thread Raghu Angadi
>
> Window of Windows:
>   1: KV --> groupByKey() --> fixedWindow(1hour)
> --> count() --> KV
>   2: KV --> groupByKey()
> fixedWindow(X*hour) --> ParDo(Iterable>)
> This allows me get my hands on the collection of
> KV that I can iterate over and emit the delta
> between them.
> However, the larger fixed window seems arbitrary and unnecessary as I
> want to carry the delta indefinitely, not start from 0 every few hours.


You can use 'slidingWindow(2*hours)' in 2, right?


On Thu, May 3, 2018 at 11:27 AM Stephan Kotze 
wrote:

> Hi everyone
>
> I've already posted this question on stackoverflow @
> https://stackoverflow.com/questions/50161186/apache-beam-delta-between-windows
>
> But I thought it made sense to post it here as well.
>
> So here we go:
>
> I am trying to determine the delta between values calculated in different
> fixed windows and emit them.
>
>T-2 T-1
>T
>
> |-|
> |-|
>  userID |   clickEventCount = 3   |   clickEventCount = 1
>  |   clickEventCount = 6   |
>
> | |
>|
> ˅ ˅
>˅
> userID   clickEventCountDelta = 3   clickEventCountDelta =
> -2 clickEventCountDelta = 5
>
>
> Something like this would be grand:
> 1. KV --> groupByKey() --> fixedWindow(1hour)
> --> count() --> KV
> 2. KV --> ?(deltaFn) -->
> KV
>
>
> I'm struggling a bit to find a solution that is both elegant and scalable,
> so any help would be greatly appreciated.
>
> Some things I've considered/tried:
> Window of Windows:
>   1: KV --> groupByKey() --> fixedWindow(1hour)
> --> count() --> KV
>   2: KV --> groupByKey()
> fixedWindow(X*hour) --> ParDo(Iterable>)
>
> This allows me get my hands on the collection of
> KV that I can iterate over and emit the delta
> between them.
> However, the larger fixed window seems arbitrary and unnecessary as I
> want to carry the delta indefinitely, not start from 0 every few hours.
>
> Shifting the previous count's timestamp
> 1: Essentially attempting to take the KV
> PTransform and emitting an additional KV with an
> adjusted timestamp.
> 2: Then grabbing this time shifted timestamp as a side input to a
> PTransform/doFn for calculating the delta with the previous period.
>
> This seemed like a cute idea, but it very quickly became a mess and
> doesn't "feel" to be the right approach.
>
> Using an external cache:
> 1: writing the KV + timestamp+window to a
> distributed cache.
> 2: Grabbing the previous value from the cache and computing the
> delta.
>
> Doesn’t seem totally unreasonable to be able to reach back in time to
> the count of the previous window via a cache. However it "feels" wrong and
> highly inefficient given that I have the data in a PCollection somewhere
> nearby.
>
> Stateful DoFns
> Seems reasonable, but is it overkill (having to initialise them with
> cache lookbacks anyways as they are reset on window closes)
>
> Sliding Window over 2 X Windows:
> If I create a sliding window 2x the duration of the underlying count
> windows, I could potentially indefinitely emit the delta between every two
> events.
> This also feels odd, but could be the most elegant solution here?
>
>
> None of the above really feel like the right way to approach deltas, and
> it does appear that the Compute model's state Doesn't cater for the "feed
> forward" of data in time.
>
> So :)
> I've done quite a bit of searching and I can't seem to find anything
> leading me clearly into a particular direction.
> I'm probably missing something big here, so any help would be much
> appreciated.
>
>


Re: Advice on parallelizing network calls in DoFn

2018-03-15 Thread Raghu Angadi
In streaming, a simple way is to add a reshuffle to increase parallelism.
When you are external-call bound, extra cost of reshuffle is negligible.
e.g.
https://stackoverflow.com/questions/46116443/dataflow-streaming-job-not-scaleing-past-1-worker

Note that by default Dataflow workers use a couple of hundred threads as
required. This can be increased with a pipeline option if you prefer. I am
not sure of other runners.

On Thu, Mar 15, 2018 at 8:25 AM Falcon Taylor-Carter <
fal...@bounceexchange.com> wrote:

> Hello Pablo,
>
> Thanks for checking up (I'm working with Josh on this problem). It seems
> there isn't a built-in process for this kind of use case currently, and
> that the best process right now is to handle our own bundling and threading
> in the DoFn. If you had any other suggestions, or anything to keep in mind
> in doing this, let us know!
>
> Falcon
>
> On Tue, Mar 13, 2018 at 4:52 PM, Pablo Estrada  wrote:
>
>> I'd just like to close the loop. Josh, did you get an answer/guidance on
>> how to proceed with your pipeline?
>> Or maybe we'll need a new thread to figure that out : )
>> Best
>> -P.
>>
>>
>> On Fri, Mar 9, 2018 at 1:39 PM Josh Ferge 
>> wrote:
>>
>>> Hello all:
>>>
>>> Our team has a pipeline that make external network calls. These
>>> pipelines are currently super slow, and the hypothesis is that they are
>>> slow because we are not threading for our network calls. The github issue
>>> below provides some discussion around this:
>>>
>>> https://github.com/apache/beam/pull/957
>>>
>>> In beam 1.0, there was IntraBundleParallelization, which helped with
>>> this. However, this was removed because it didn't comply with a few BEAM
>>> paradigms.
>>>
>>> Questions going forward:
>>>
>>> What is advised for jobs that make blocking network calls? It seems
>>> bundling the elements into groups of size X prior to passing to the DoFn,
>>> and managing the threading within the function might work. thoughts?
>>> Are these types of jobs even suitable for beam?
>>> Are there any plans to develop features that help with this?
>>>
>>> Thanks
>>>
>> --
>> Got feedback? go/pabloem-feedback
>> 
>>
>
>


Re: The problem of kafkaIO sdk for data latency

2018-03-06 Thread Raghu Angadi
This message was sent with Gmail's confidential mode. You can open it by clicking this  link for user@beam.apache.org.


Re: The problem of kafkaIO sdk for data latency

2018-03-02 Thread Raghu Angadi
I recently noticed that DirectRunner was leaking readers eventually
crashing my pipeline. It is fixed in master (PR 4658
, version 2.4.0-SNAPSHOT). Can
you try that? In my case the pipeline ran out of file descriptors.

Note that DirectRunner is not particularly optimized for runtime
performance. It is often used for initial testing. Since the performance is
alright for you initially, trying it out with master might help.

Note that TimestampedValue<> does not actually change the timestamp of the
event. KafkaIO uses processing time for event time by default. Please see
JavaDoc for KafkaIO for more options.


On Wed, Feb 28, 2018 at 6:59 PM  wrote:

> Dear all,
>
>
>
> I am using the kafkaIO sdk in my project (Beam 2.0.0 with Direct runner).
>
>
>
> With using this sdk, there are a situation about *data* *latency*, and
> the description of situation is in the following.
>
>
>
> The data come from kafak with a fixed speed: 100 data size/ 1 sec.
>
>
>
> I create a fixed window within 1 sec without delay. I found that the data
> size is 70, 80, 104, or greater than or equal to 104.
>
>
>
> After one day, the data latency happens in my running time, and the data
> size will be only 10 in each window.
>
>
>
> *In order to clearly explain it, I also provide my code in the following. *
>
> " PipelineOptions readOptions = PipelineOptionsFactory.*create*();
>
> *final* Pipeline p = Pipeline.*create*(readOptions);
>
>
>
> PCollection>> readData =
>
>   p.apply(KafkaIO.*read*()
>
>  .withBootstrapServers("127.0.0.1:9092")
>
>  .withTopic("kafkasink")
>
>  .withKeyDeserializer(StringDeserializer.*class*)
>
>  .withValueDeserializer(StringDeserializer.*class*)
>
>  .withoutMetadata())
>
>  .apply(ParDo.*of*(*new* *DoFn,
> TimestampedValue>>()* {
>
> @ProcessElement
>
> *public* *void* test(ProcessContext c) *throws* ParseException {
>
> String element = c.element().getValue();
>
> *try* {
>
>   JsonNode arrNode = *new* ObjectMapper().readTree(element);
>
>   String t = arrNode.path("v").findValue("Timestamp"
> ).textValue();
>
>   DateTimeFormatter formatter = 
> DateTimeFormatter.*ofPattern*("MM/dd/
> HH:mm:ss.");
>
>  LocalDateTime dateTime = LocalDateTime.*parse*(t, formatter);
>
>  java.time.Instant java_instant = dateTime.atZone(ZoneId.
> *systemDefault*()).toInstant();
>
>  Instant timestamp  = *new* Instant(java_instant.toEpochMilli());
>
>
>   c.output(TimestampedValue.*of*(c.element(), timestamp));
>
> } *catch* (JsonGenerationException e) {
>
> e.printStackTrace();
>
> } *catch* (JsonMappingException e) {
>
> e.printStackTrace();
>
>   } *catch* (IOException e) {
>
> e.printStackTrace();
>
>   }
>
> }}));
>
>
>
> PCollection>> readDivideData =
> readData.apply(
>
>   Window.>> *into*(FixedWindows.
> *of*(Duration.*standardSeconds*(1))
>
>   .withOffset(Duration.*ZERO*))
>
>   .triggering(AfterWatermark.*pastEndOfWindow*()
>
>  .withLateFirings(AfterProcessingTime.*pastFirstElementInPane*
> ()
>
>.plusDelayOf(Duration.*ZERO*)))
>
>   .withAllowedLateness(Duration.*ZERO*)
>
>   .discardingFiredPanes());"
>
>
>
> *In addition, the running result is as shown in the following.*
>
> "data-size=104
>
> coming-data-time=2018-02-27 02:00:49.117
>
> window-time=2018-02-27 02:00:49.999
>
>
>
> data-size=70
>
> coming-data-time=2018-02-27 02:00:50.318
>
> window-time=2018-02-27 02:00:50.999
>
>
>
> data-size=104
>
> coming-data-time=2018-02-27 02:00:51.102
>
> window-time=2018-02-27 02:00:51.999
>
>
>
> After one day:
>
> data-size=10
>
> coming-data-time=2018-02-28 02:05:48.217
>
> window-time=2018-03-01 10:35:16.999 "
>
>
>
> *For repeating my situation, my running environment is:*
>
> OS: Ubuntn 14.04.3 LTS
>
>
>
> JAVA: JDK 1.7
>
>
>
> *Beam 2.0.0 (with Direct runner)*
>
>
>
> *Kafka 2.10-0.10.1.1*
>
>
>
> Maven 3.5.0, in which dependencies are listed in pom.xml:
>
> 
>
>   org.apache.beam
>
>   beam-*sdks*-java-core
>
>   2.0.0
>
> 
>
> 
>
>org.apache.beam
>
>   beam-runners-direct-java
>
>   2.0.0
>
>   runtime
>
> 
>
>
>
> 
>
> org.apache.beam
>
>beam-*sdks*-java-*io*-*kafka*
>
>2.0.0
>
> 
>
>
>
>
>
> 
>
> 
>
>org.apache.kafka
>
>*kafka*-clients
>
>0.10.0.1
>
> 
>
>
>
> If you have any idea about the problem (data latency), I am looking
> forward to hearing from you.
>
>
>
> Thanks
>
>
>
> Rick
>
>
> --
> 本信件可能包含工研院機密資訊,非指定之收件者,請勿使用或揭露本信件內容,並請銷毀此信件。 This email may contain
> confidential information. Please do not use or disclose it in any way 

Re: Missing watermark?

2018-02-20 Thread Raghu Angadi
Both the sources have to provide watermark.

- KafkaIO : It can have its watermark stuck if you don't have any records
in one of the partitions. Thats is a bug. The management of timestamps and
watermarks in KafkaIO are being updated in
https://github.com/apache/beam/pull/4680.
- TextIO.watchForNewFiles() - I am not sure how the watermark is handled by
TextIO. Didn't notice any mentions of in implementation.

On Tue, Feb 20, 2018 at 10:13 AM, Vilhelm von Ehrenheim <
vonehrenh...@gmail.com> wrote:

> Hi all!
> I have a somewhat complicated stateful DoFn that i would like to add an
> event time timer on. My goal with the timer is to not output anything until
> sufficient amount of state has been built up in a Global window.
>
> In doing this I realize that the watermark doesn’t seem to progress at all
> (regardless of the timer) and in Google Dataflow the displayed watermark is
> just “-“ when clicking on the ParDo(DoFn) node.
>
> The DoFn is reading flattened input from a TextIO.watchForNewFiles and
> KafkaIO. The Flatten element had a watermark set.
>
> I have written tests for my DoFn that all pass using TestStream but since
> I there explicitly set the watermark progression all is fine.
>
> What can I do to look into why there is no watermark progression for a
> specific  PTransform?
>
> Regards,
> Vilhelm von Ehrenheim
>


Re: Lateness droppings debugging

2018-02-08 Thread Raghu Angadi
What are the advantages of holding back watermark over allowed lateness
(duration of both being approximately same)?

On Thu, Feb 8, 2018 at 1:38 PM, Robert Bradshaw <rober...@google.com> wrote:

> You can set the timestamp attribute of your pubsub messages which will
> hold back the watermark, see
>
> https://beam.apache.org/documentation/sdks/javadoc/2.
> 2.0/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.Read.html#
> withTimestampAttribute-java.lang.String-
>
> However, if you're mixing historical and live data, it may make more
> sense to read these as two separate sources (e.g. the static data from
> a set of files, the live data from pubsub) and then flatten them for
> further processing.
>
> On Thu, Feb 8, 2018 at 1:23 PM, Carlos Alonso <car...@mrcalonso.com>
> wrote:
> > Yes, the data is finite (although it comes through PubSub, so I guess is
> > considered unbounded).
> > How could I hold the watermark and prevent it from moving?
> >
> > Thanks!
> >
> > On Thu, Feb 8, 2018 at 10:06 PM Robert Bradshaw <rober...@google.com>
> wrote:
> >>
> >> Where is the watermark for this old data coming from? Rather than
> >> messing with allowed lateness, would it be possible to hold the
> >> watermark back appropriately during the time you're injecting old data
> >> (assuming there's only a finite amount of it)?
> >>
> >> On Thu, Feb 8, 2018 at 12:56 PM, Carlos Alonso <car...@mrcalonso.com>
> >> wrote:
> >> > Thanks for your responses!!
> >> >
> >> > I have a scenario where I have to reprocess very disordered data for 4
> >> > or 5
> >> > years and I don't want to lose any data. I'm thinking of setting a
> very
> >> > big
> >> > allowed lateness (5 years), but before doing that I'd like to
> understand
> >> > the
> >> > consequences that may have. I guess memory wise will be very consuming
> >> > as no
> >> > window will ever expire, but I guess I could overcome that with brute
> >> > force
> >> > (many machines with many RAM) but, are there more concerns I should be
> >> > aware
> >> > of? This should be a one-off thing.
> >> >
> >> > Thanks!
> >> >
> >> > On Thu, Feb 8, 2018 at 6:59 PM Raghu Angadi <rang...@google.com>
> wrote:
> >> >>
> >> >> On Wed, Feb 7, 2018 at 10:33 PM, Pawel Bartoszek
> >> >> <pawelbartosze...@gmail.com> wrote:
> >> >>>
> >> >>> Hi Raghu,
> >> >>> Can you provide more details about increasing allowed lateness? Even
> >> >>> if I
> >> >>> do that I still need to compare event time of record with processing
> >> >>> time(system current time) in my ParDo?
> >> >>
> >> >>
> >> >> I see. PaneInfo() associated with each element has 'Timing' enum, so
> we
> >> >> can tell if the element is late, but it does not tell how late.
> >> >> How about this : We can have a periodic timer firing every minute and
> >> >> store the scheduled time of the timer in state as the watermark time.
> >> >> We
> >> >> could compare element time to this stored time for good approximation
> >> >> (may
> >> >> require parallel stage with global window, dropping any events that
> >> >> 'clearly
> >> >> within limits' based on current time). There are probably other ways
> to
> >> >> do
> >> >> this with timers within existing stage.
> >> >>
> >> >>>
> >> >>> Pawel
> >> >>>
> >> >>> On 8 February 2018 at 05:40, Raghu Angadi <rang...@google.com>
> wrote:
> >> >>>>
> >> >>>> The watermark is not directly available, you essentially infer from
> >> >>>> fired triggers (e.g. fixed windows). I would consider some of these
> >> >>>> options
> >> >>>> :
> >> >>>>   - Adhoc debugging : if the pipeline is close to realtime, you can
> >> >>>> just
> >> >>>> guess if a element will be dropped based on its timestamp and
> current
> >> >>>> time
> >> >>>> in the first stage (before first aggregation)
> >> >>>>   - Increase allowed lateness (say to 3 days) and drop the elements
> >> >>>> yourself you notice are later than 1 day.
> >> >>>>   - Place the elements into another window with larger allowed
> >> >>>> lateness
> >> >>>> and log very late elements in another parallel aggregation (see
> >> >>>> TriggerExample.java in Beam repo).
> >> >>>>
> >> >>>> On Wed, Feb 7, 2018 at 2:55 PM, Carlos Alonso <
> car...@mrcalonso.com>
> >> >>>> wrote:
> >> >>>>>
> >> >>>>> Hi everyone!!
> >> >>>>>
> >> >>>>> I have a streaming job running with fixed windows of one hour and
> >> >>>>> allowed lateness of two days and the number of dropped due to
> >> >>>>> lateness
> >> >>>>> elements is slowly, but continuously growing and I'd like to
> >> >>>>> understand
> >> >>>>> which elements are those.
> >> >>>>>
> >> >>>>> I'd like to get the watermark from inside the job to compare it
> >> >>>>> against
> >> >>>>> each element and write log messages with the ones that will be
> >> >>>>> potentially
> >> >>>>> discarded Does that approach make any sense? I which case...
> How
> >> >>>>> can I
> >> >>>>> get the watermark from inside the job? Any other ideas?
> >> >>>>>
> >> >>>>> Thanks in advance!!
> >> >>>>
> >> >>>>
> >> >>>
> >> >
>


Re: Lateness droppings debugging

2018-02-08 Thread Raghu Angadi
On Wed, Feb 7, 2018 at 10:33 PM, Pawel Bartoszek <pawelbartosze...@gmail.com
> wrote:

> Hi Raghu,
> Can you provide more details about increasing allowed lateness? Even if I
> do that I still need to compare event time of record with processing
> time(system current time) in my ParDo?
>

I see. PaneInfo() associated with each element has 'Timing' enum, so we can
tell if the element is late, but it does not tell how late.
How about this : We can have a periodic timer firing every minute and store
the scheduled time of the timer in state as the watermark time. We could
compare element time to this stored time for good approximation (may
require parallel stage with global window, dropping any events that
'clearly within limits' based on current time). There are probably other
ways to do this with timers within existing stage.


> Pawel
>
> On 8 February 2018 at 05:40, Raghu Angadi <rang...@google.com> wrote:
>
>> The watermark is not directly available, you essentially infer from fired
>> triggers (e.g. fixed windows). I would consider some of these options :
>>   - Adhoc debugging : if the pipeline is close to realtime, you can just
>> guess if a element will be dropped based on its timestamp and current time
>> in the first stage (before first aggregation)
>>   - Increase allowed lateness (say to 3 days) and drop the elements
>> yourself you notice are later than 1 day.
>>   - Place the elements into another window with larger allowed lateness
>> and log very late elements in another parallel aggregation (see
>> TriggerExample.java in Beam repo).
>>
>> On Wed, Feb 7, 2018 at 2:55 PM, Carlos Alonso <car...@mrcalonso.com>
>> wrote:
>>
>>> Hi everyone!!
>>>
>>> I have a streaming job running with fixed windows of one hour and
>>> allowed lateness of two days and the number of dropped due to lateness
>>> elements is slowly, but continuously growing and I'd like to understand
>>> which elements are those.
>>>
>>> I'd like to get the watermark from inside the job to compare it against
>>> each element and write log messages with the ones that will be potentially
>>> discarded Does that approach make any sense? I which case... How can I
>>> get the watermark from inside the job? Any other ideas?
>>>
>>> Thanks in advance!!
>>>
>>
>>
>


Re: Lateness droppings debugging

2018-02-07 Thread Raghu Angadi
The watermark is not directly available, you essentially infer from fired
triggers (e.g. fixed windows). I would consider some of these options :
  - Adhoc debugging : if the pipeline is close to realtime, you can just
guess if a element will be dropped based on its timestamp and current time
in the first stage (before first aggregation)
  - Increase allowed lateness (say to 3 days) and drop the elements
yourself you notice are later than 1 day.
  - Place the elements into another window with larger allowed lateness and
log very late elements in another parallel aggregation (see
TriggerExample.java in Beam repo).

On Wed, Feb 7, 2018 at 2:55 PM, Carlos Alonso  wrote:

> Hi everyone!!
>
> I have a streaming job running with fixed windows of one hour and allowed
> lateness of two days and the number of dropped due to lateness elements is
> slowly, but continuously growing and I'd like to understand which elements
> are those.
>
> I'd like to get the watermark from inside the job to compare it against
> each element and write log messages with the ones that will be potentially
> discarded Does that approach make any sense? I which case... How can I
> get the watermark from inside the job? Any other ideas?
>
> Thanks in advance!!
>


Re: Deprecate and remove support for Kafka 0.9.x and older version

2018-02-06 Thread Raghu Angadi
Agreed. We will ask users again before we will remove runtime support in a
future version. No timeline yet.
PR 4603 removes compile time support (i.e. can't run unit tests). Please
let me know that looks good.

Raghu.

On Tue, Feb 6, 2018 at 7:40 AM, Ismaël Mejía <ieme...@gmail.com> wrote:

> Agree with JB, showing deprecation is ok, but I think it is worth to
> support Kafka 0.9.x for some extra time. Users tend to stay in old
> data stores because migrating these clusters isn't always so easy.
>
>
> On Tue, Feb 6, 2018 at 3:56 PM, Jean-Baptiste Onofré <j...@nanthrax.net>
> wrote:
> > +1 to flag as deprecated, but I would wait a bit before simply removing
> it.
> >
> > Regards
> > JB
> >
> > On 02/03/2018 01:12 AM, Raghu Angadi wrote:
> >> Is anyone using Apache Beam with Kafka 0.9.x and older?
> >>
> >> I am thinking of deprecating 0.9.x and 0.10.0 in Beam 2.4 and remove
> support in
> >> 2.5 or later. 0.10.1 and up will be supported. 0.10.1+ includes much
> better
> >> timestamp support.
> >>
> >> By deprecation I mean KafkaIO would continue to work with an older
> version at
> >> runtime, but would not build with it (e.g. `mvn
> -Dkafka.clients.version=0.9.1`
> >> fails).  We can print a deprecation warning at runtime.
> >>
> >> [1]: https://github.com/apache/kafka/commit/
> 23c69d62a0cabf06c4db8b338f0ca824dc6d81a7
> >
> > --
> > Jean-Baptiste Onofré
> > jbono...@apache.org
> > http://blog.nanthrax.net
> > Talend - http://www.talend.com
>


Re: KafkaIO reading from latest offset when pipeline fails on FlinkRunner

2018-02-05 Thread Raghu Angadi
Hi Sushil,

That is expected behavior. If you don't have any saved checkpoint, the
pipeline would start from scratch. It does not have any connection to
previous run.

On Thu, Feb 1, 2018 at 1:29 AM, Sushil Ks <sushil...@gmail.com> wrote:

> Hi,
>Apologies for delay in my reply,
>
> @Raghu Angadi
> This checkpoints 20 mins, as you mentioned before any
> checkpoint is created and if the pipeline restarts, it's reading from the
> latest offset.
>
> @Mingmin
> Thanks a lot for sharing your learnings, However in case of any
> *UserCodeException* while processing the element as part of ParDo after
> materializing the window, the pipeline drops the unprocessed elements and
> restarts. Is this expected from Beam?
>
>
> On Wed, Jan 17, 2018 at 2:13 AM, Kenneth Knowles <k...@google.com> wrote:
>
>> Is there a JIRA filed for this? I think this discussion should live in a
>> ticket.
>>
>> Kenn
>>
>> On Wed, Jan 10, 2018 at 11:00 AM, Mingmin Xu <mingm...@gmail.com> wrote:
>>
>>> @Sushil, I have several jobs running on KafkaIO+FlinkRunner, hope my
>>> experience can help you a bit.
>>>
>>> For short, `ENABLE_AUTO_COMMIT_CONFIG` doesn't meet your requirement,
>>> you need to leverage exactly-once checkpoint/savepoint in Flink. The reason
>>> is,  with `ENABLE_AUTO_COMMIT_CONFIG` KafkaIO commits offset after data is
>>> read, and once job is restarted KafkaIO reads from last_committed_offset.
>>>
>>> In my jobs, I enable external(external should be optional I think?)
>>> checkpoint on exactly-once mode in Flink cluster. When the job auto-restart
>>> on failures it doesn't lost data. In case of manually redeploy the job, I
>>> use savepoint to cancel and launch the job.
>>>
>>> Mingmin
>>>
>>> On Wed, Jan 10, 2018 at 10:34 AM, Raghu Angadi <rang...@google.com>
>>> wrote:
>>>
>>>> How often does your pipeline checkpoint/snapshot? If the failure
>>>> happens before the first checkpoint, the pipeline could restart without any
>>>> state, in which case KafkaIO would read from latest offset. There is
>>>> probably some way to verify if pipeline is restarting from a checkpoint.
>>>>
>>>> On Sun, Jan 7, 2018 at 10:57 PM, Sushil Ks <sushil...@gmail.com> wrote:
>>>>
>>>>> HI Aljoscha,
>>>>>The issue is let's say I consumed 100 elements in 5
>>>>> mins Fixed Window with *GroupByKey* and later I applied *ParDO* for
>>>>> all those elements. If there is an issue while processing element 70 in
>>>>> *ParDo *and the pipeline restarts with *UserCodeException *it's
>>>>> skipping the rest 30 elements. Wanted to know if this is expected? In case
>>>>> if you still having doubt let me know will share a code snippet.
>>>>>
>>>>> Regards,
>>>>> Sushil Ks
>>>>>
>>>>
>>>>
>>>
>>>
>>> --
>>> 
>>> Mingmin
>>>
>>
>>
>


Deprecate and remove support for Kafka 0.9.x and older version

2018-02-02 Thread Raghu Angadi
Is anyone using Apache Beam with Kafka 0.9.x and older?

I am thinking of deprecating 0.9.x and 0.10.0 in Beam 2.4 and remove
support in 2.5 or later. 0.10.1 and up will be supported. 0.10.1+ includes
much better timestamp support.

By deprecation I mean KafkaIO would continue to work with an older version
at runtime, but would not build with it (e.g. `mvn
-Dkafka.clients.version=0.9.1` fails).  We can print a deprecation warning
at runtime.

[1]:
https://github.com/apache/kafka/commit/23c69d62a0cabf06c4db8b338f0ca824dc6d81a7


Re: KafkaIO reading from latest offset when pipeline fails on FlinkRunner

2018-01-10 Thread Raghu Angadi
How often does your pipeline checkpoint/snapshot? If the failure happens
before the first checkpoint, the pipeline could restart without any state,
in which case KafkaIO would read from latest offset. There is probably some
way to verify if pipeline is restarting from a checkpoint.

On Sun, Jan 7, 2018 at 10:57 PM, Sushil Ks  wrote:

> HI Aljoscha,
>The issue is let's say I consumed 100 elements in 5
> mins Fixed Window with *GroupByKey* and later I applied *ParDO* for all
> those elements. If there is an issue while processing element 70 in
> *ParDo *and the pipeline restarts with *UserCodeException *it's skipping
> the rest 30 elements. Wanted to know if this is expected? In case if you
> still having doubt let me know will share a code snippet.
>
> Regards,
> Sushil Ks
>


Re: Beam Runner hooks for Throughput-based autoscaling

2017-11-20 Thread Raghu Angadi
Thanks for the report Peter. Generally speaking autoscaling in Dataflow
streaming application is based on two factors : backlog (e.g. PubSub in
this case) and CPU used for current throughput. Multiple stages makes it a
bit more complex, but over it is essentially trying to scale CPU resource
so that extrapolated throughput handles current backlog, backlog growth.

As you noted, cases where small number of messages trigger large amount of
work in the pipeline makes backlog estimation tricky. That can cause delays
in upscaling (depending on how the fan out happens in the application), but
I would not expect the pipeline to downscale too early when it still has a
lot of work pending. One possibility is that processing involves blocking
work which might keep CPU utilization lower (say 40-50%).

I am certainly interested in closer look at your specific job. Would you
mind asking it on Stackoverflow (
https://stackoverflow.com/questions/tagged/google-cloud-dataflow)? It is
also open to public. Please provide job_id to look at.

IFAIK, Dataflow is in the only Beam runner that currently supports
autoscaling based on changes in load. Others might need user to trigger
recaling (e.g. Flink).

Raghu.

On Fri, Nov 17, 2017 at 8:02 AM, Peter Mueller  wrote:

> Hi,
>
> I'm curious if anyone can point me towards greater visibility into how
> various Beam Runners manage autoscaling.  We seem to be experiencing
> hiccups during both the 'spin up' and 'spin down' phases, and we're left
> wondering what to do about it.  Here's the background of our particular
> flow:
>
> 1- Binary files arrive on gs://, and object notification duly notifies a
> PubSub topic.
> 2- Each file requires about 1Min of parsing on a standard VM to emit about
> 30K records to downstream areas of the Beam DAG.
> 3- 'Downstream' components include things like inserts to BigQuery,
> storage in GS:, and various sundry other tasks.
> 4- The files in step 1 arrive intermittently, usually in batches of
> 200-300 every hour, making this - we think - an ideal use case for
> autoscaling.
>
> What we're seeing, however, has us a little perplexed:
>
> 1- It looks like when 'workers=1', Beam bites off a little more than it
> can chew, eventually causing some out-of-RAM errors, presumably as the
> first worker tries to process a few of the PubSub messages which, again,
> take about 60 seconds/message to complete because the 'message' in this
> case is that a binary file needs to be deserialized in gs.
> 2- At some point, the runner (in this case, Dataflow), gets the message
> additional workers are needed and real work can now get done.  During this
> phase, errors decrease and throughput rises.   Not only are there more
> deserializers for step1, but the step3/downstream tasks are evenly spread
> out.
> 3-Alas, the previous step gets cut short when Dataflow senses (I'm
> guessing) that enough of the PubSub messages are 'in flight' to begin
> cooling down a little. That seems to come a little too soon, and workers
> are getting pulled as they chew through the PubSub messages themselves -
> even before the messages are 'ACK'd'.
>
> We're still thrilled with Beam, but I'm guessing the less-than-optimal
> spin-up/spin-down phases are resulting in 50% more VM usage than what is
> needed.  What do the runners look for beside PubSub consumption?  Do they
> look at RAM/CPU/etc??? Is there anything a developer can do, beside ACK a
> PubSub message to provide feedback to the runner that more/less resources
> are required?
>
> Incidentally, in case anyone doubted Google's commitment to open-source, I
> spoke about this very topic with an employee there yesterday, and she
> expressed interest in hearing about my use case, especially if it ran on a
> non-Dataflow runner!  We hadn't yet tried our Beam work on Spark (or
> elsewhere), but would obviously be interested in hearing if one runner has
> superior abilities to accept feedback from the workers for THROUGHPUT_BASED
> work.
>
> Thanks in advance,
>
> Peter Mueller
> CTO,
> ATS, Inc.
>
>
>


Re: KafkaIO Autocommit

2017-11-06 Thread Raghu Angadi
Can you post full log of one the workers/executors that reads at least one
record? You can remove application logs.

On Mon, Nov 6, 2017 at 12:45 PM, NerdyNick <nerdyn...@gmail.com> wrote:

> It's all partitions and it's random on how far behind they show. The topic
> in question has a low events/sec but over sized partition count. So not
> every partition has data to be read the full length of time. So given this,
> it's almost like the partition is drained before the batch end read time.
> Which seems to result in the consumer not posting an update to the offset
> within that batch. Because the offsets will get posted just not with the
> rate of the batches. So it'll look like it falls behind then somewhat
> catches up.
>
> On Mon, Nov 6, 2017 at 12:53 PM, Raghu Angadi <rang...@google.com> wrote:
>
>> On Mon, Nov 6, 2017 at 10:39 AM, NerdyNick <nerdyn...@gmail.com> wrote:
>>
>>> Hey Raghu,
>>>
>>> Runner is Spark on Yarn cluster. Executor/Thread would be the
>>> Thread(Core) within a Spark Executor. I have nothing I can find in the logs
>>> saying that they aren't being closed cleanly outside of forced shutdown.
>>> Interval I have now is 30ms with a min read time of 2secs. Watching the
>>> finished tasks for the read job shows each thread running for the 2secs,
>>> some die early with no log as to why. However watching the offset within
>>> Kafka Manager shows the lag growing, as if the commit isn't happening.
>>> However the event throughput shows it matching that of the kafka topic msgs
>>> inbound. I can't seem to find a working option for getting the offsets
>>> KafkaIO believes it has out of either to compare numbers. I tried the Spark
>>> Metrics sinks provided by the runner, as KafkaIO appears to provide it via
>>> those. But they don't appear to work and actually caused issues, no longer
>>> appeared, with the Streaming stats Spark maintains.
>>>
>>
>> May be one or two Kafka partitions aren't being consumed (due to some
>> error). You could check on Kafka if all the partitions are behind or just
>> some. Otherwise what you describe sounds like bugs/issues in the pipeline
>> (spark or user). If feasible, you can try running with direct-runner to see
>> if auto_commit does not advance.
>>
>> I would also encourage you to file bugs about the missing metrics in
>> spark (and/or report here on another thread).
>>
>> Raghu.
>>
>>
>>> Consumer group name is consistent as a bash script is being used to
>>> launch jobs to guarantee uniformity between launches.
>>>
>>> From what I've been able to dissect from the KafkaIO class. I believe
>>> the watermark management layer via the PartitionState might be the best
>>> tiein for doing manual kafka offset management via the
>>> Consumer.commitSync() or Consumer.commitAsync() methods. This would allow
>>> better uniform and consistent reporting of offset in sync with the values
>>> being maintained in the watermark.
>>>
>>> On Mon, Nov 6, 2017 at 11:11 AM, Raghu Angadi <rang...@google.com>
>>> wrote:
>>>
>>>> What is the runner?
>>>> Can you elaborate a bit more what you mean by 'executor/thread
>>>> shutting down'? If the KafkaIO reader shutdown cleanly, it would call close
>>>> the consumer cleanly, triggering auto commit. But if you shutdown a
>>>> pipeline, it might not cleanly close the consumers. What is the auto_commit
>>>> interval you have? Please note that there is no way to coordinate
>>>> consistency between Beam pipeline and externally maintained auto commit
>>>> offsets since it is outside Beam. 'Drain' feature Dataflow can help (it
>>>> lets a clean shutdown of the pipeline), also note that many runners provide
>>>> clean ways to update a pipeline that keeps all the state from previous run
>>>> (in this case Kafka offsets), which is the only way for Beam to provide its
>>>> processing guarantees across runs.
>>>>
>>>> KafkaIO leaves auto_commit handling  entirely to KafkaConsumer. If you
>>>> are seeing consumer is not honoring the auto_committed offset, please check
>>>> the log from KafkConsumer on the worker. Only user error I could think of
>>>> is some typo in consumer group name upon restart.
>>>>
>>>> Currently KafkaIO does not actively participate in auto_commit
>>>> management. It lets user directly set KafkaConsumer configuration. May be
>>>> there is a case for some more active sup

Re: KafkaIO Autocommit

2017-11-06 Thread Raghu Angadi
On Mon, Nov 6, 2017 at 10:39 AM, NerdyNick <nerdyn...@gmail.com> wrote:

> Hey Raghu,
>
> Runner is Spark on Yarn cluster. Executor/Thread would be the Thread(Core)
> within a Spark Executor. I have nothing I can find in the logs saying that
> they aren't being closed cleanly outside of forced shutdown. Interval I
> have now is 30ms with a min read time of 2secs. Watching the finished tasks
> for the read job shows each thread running for the 2secs, some die early
> with no log as to why. However watching the offset within Kafka Manager
> shows the lag growing, as if the commit isn't happening. However the event
> throughput shows it matching that of the kafka topic msgs inbound. I can't
> seem to find a working option for getting the offsets KafkaIO believes it
> has out of either to compare numbers. I tried the Spark Metrics sinks
> provided by the runner, as KafkaIO appears to provide it via those. But
> they don't appear to work and actually caused issues, no longer appeared,
> with the Streaming stats Spark maintains.
>

May be one or two Kafka partitions aren't being consumed (due to some
error). You could check on Kafka if all the partitions are behind or just
some. Otherwise what you describe sounds like bugs/issues in the pipeline
(spark or user). If feasible, you can try running with direct-runner to see
if auto_commit does not advance.

I would also encourage you to file bugs about the missing metrics in spark
(and/or report here on another thread).

Raghu.


> Consumer group name is consistent as a bash script is being used to launch
> jobs to guarantee uniformity between launches.
>
> From what I've been able to dissect from the KafkaIO class. I believe the
> watermark management layer via the PartitionState might be the best tiein
> for doing manual kafka offset management via the Consumer.commitSync() or
> Consumer.commitAsync() methods. This would allow better uniform and
> consistent reporting of offset in sync with the values being maintained in
> the watermark.
>
> On Mon, Nov 6, 2017 at 11:11 AM, Raghu Angadi <rang...@google.com> wrote:
>
>> What is the runner?
>> Can you elaborate a bit more what you mean by 'executor/thread shutting
>> down'? If the KafkaIO reader shutdown cleanly, it would call close the
>> consumer cleanly, triggering auto commit. But if you shutdown a pipeline,
>> it might not cleanly close the consumers. What is the auto_commit interval
>> you have? Please note that there is no way to coordinate consistency
>> between Beam pipeline and externally maintained auto commit offsets since
>> it is outside Beam. 'Drain' feature Dataflow can help (it lets a clean
>> shutdown of the pipeline), also note that many runners provide clean ways
>> to update a pipeline that keeps all the state from previous run (in this
>> case Kafka offsets), which is the only way for Beam to provide its
>> processing guarantees across runs.
>>
>> KafkaIO leaves auto_commit handling  entirely to KafkaConsumer. If you
>> are seeing consumer is not honoring the auto_committed offset, please check
>> the log from KafkConsumer on the worker. Only user error I could think of
>> is some typo in consumer group name upon restart.
>>
>> Currently KafkaIO does not actively participate in auto_commit
>> management. It lets user directly set KafkaConsumer configuration. May be
>> there is a case for some more active support for auto_commit management.
>> Please provide more details in your case so that we can discuss actual
>> specifics and potential improvements it provides.
>>
>>
>> On Mon, Nov 6, 2017 at 8:15 AM, NerdyNick <nerdyn...@gmail.com> wrote:
>>
>>> There seems to be a lot of oddities with the auto offset committer and
>>> the watermark management as well as kafka offsets in general.
>>>
>>> One issue I keep having is the auto committer will just not commit any
>>> offsets. So the topic will look like it's backing up. From what I've been
>>> able to trace on it it appears to be in relation to the executor/thread
>>> shutting down before the auto commit has a chance to run. Even though the
>>> min read times are set. It still prematurely shuts down. Turning auto
>>> commit interval down seems to help but doesn't resolve the issue. Just
>>> seems to allow it to correct itself much quicker.
>>>
>>> Another I just had happen is after restarting a pipeline the auto
>>> committed offsets reset to the earliest record and the pipeline appears to
>>> be working on those records. Which is odd in contrary to a lot of things.
>>> When I shut the pipeline down it was only a few thousand records behind.
>&

Re: KafkaIO Autocommit

2017-11-06 Thread Raghu Angadi
What is the runner?
Can you elaborate a bit more what you mean by 'executor/thread shutting
down'? If the KafkaIO reader shutdown cleanly, it would call close the
consumer cleanly, triggering auto commit. But if you shutdown a pipeline,
it might not cleanly close the consumers. What is the auto_commit interval
you have? Please note that there is no way to coordinate consistency
between Beam pipeline and externally maintained auto commit offsets since
it is outside Beam. 'Drain' feature Dataflow can help (it lets a clean
shutdown of the pipeline), also note that many runners provide clean ways
to update a pipeline that keeps all the state from previous run (in this
case Kafka offsets), which is the only way for Beam to provide its
processing guarantees across runs.

KafkaIO leaves auto_commit handling  entirely to KafkaConsumer. If you are
seeing consumer is not honoring the auto_committed offset, please check the
log from KafkConsumer on the worker. Only user error I could think of is
some typo in consumer group name upon restart.

Currently KafkaIO does not actively participate in auto_commit management.
It lets user directly set KafkaConsumer configuration. May be there is a
case for some more active support for auto_commit management. Please
provide more details in your case so that we can discuss actual specifics
and potential improvements it provides.


On Mon, Nov 6, 2017 at 8:15 AM, NerdyNick  wrote:

> There seems to be a lot of oddities with the auto offset committer and the
> watermark management as well as kafka offsets in general.
>
> One issue I keep having is the auto committer will just not commit any
> offsets. So the topic will look like it's backing up. From what I've been
> able to trace on it it appears to be in relation to the executor/thread
> shutting down before the auto commit has a chance to run. Even though the
> min read times are set. It still prematurely shuts down. Turning auto
> commit interval down seems to help but doesn't resolve the issue. Just
> seems to allow it to correct itself much quicker.
>
> Another I just had happen is after restarting a pipeline the auto
> committed offsets reset to the earliest record and the pipeline appears to
> be working on those records. Which is odd in contrary to a lot of things.
> When I shut the pipeline down it was only a few thousand records behind.
> The consumer is configured to start at the latest offset not the earliest.
> Give that It would appear the recorded watermarks had an odd corruption or
> something where they believed they where in the past.
>
> --
> Nick Verbeck - NerdyNick
> 
> NerdyNick.com
> Coloco.ubuntu-rocks.org
>


Re: KafkaIO and Avro

2017-10-19 Thread Raghu Angadi
Ah, nice. It works.

On Thu, Oct 19, 2017 at 1:44 PM, Eugene Kirpichov <kirpic...@google.com>
wrote:

> The following compiles fine:
>
>
> p.apply(KafkaIO.<String, Envelope>read()
> .withBootstrapServers("kafka:9092")
> .withTopic("dbserver1.inventory.customers")
> .withKeyDeserializer(StringDeserializer.class)
> .withValueDeserializerAndCoder(
> (Class)KafkaAvroDeserializer.class, AvroCoder.of(Envelope.class))
>
>
> On Thu, Oct 19, 2017 at 12:21 PM Raghu Angadi <rang...@google.com> wrote:
>
>> Same for me. It does not look like there is an annotation to suppress the
>> error.
>>
>>
>> On Thu, Oct 19, 2017 at 12:18 PM, Tim Robertson <
>> timrobertson...@gmail.com> wrote:
>>
>>> Hi Eugene,
>>>
>>> I understood that was where Andrew started and reported this.  I tried
>>> and saw the same as him.
>>>
>>> incompatible types: 
>>> java.lang.Class
>>> cannot be converted to org.apache.kafka.common.
>>> serialization.Deserializer
>>>
>>> similarly with
>>> (Class>) KafkaAvroDeserializer.class
>>>
>>>
>>>
>>> On Thu, Oct 19, 2017 at 9:00 PM, Eugene Kirpichov <kirpic...@google.com>
>>> wrote:
>>>
>>>> I don't think extending the class is necessary. Not sure I understand
>>>> why a simple type casting for withDeserializerAndCoder doesn't work? Have
>>>> you tried this?
>>>>
>>>> p.apply(KafkaIO.<String, Envelope>read()
>>>>   .withValueDeserializerAndCoder((Deserializer)
>>>> KafkaAvroDeserializer.class,
>>>>   AvroCoder.of(Envelope.class))
>>>>
>>>> On Thu, Oct 19, 2017 at 11:45 AM Tim Robertson <
>>>> timrobertson...@gmail.com> wrote:
>>>>
>>>>> Hi Raghu
>>>>>
>>>>> I tried that but with KafkaAvroDeserializer already implementing
>>>>> Deserializer I couldn't get it to work... I didn't spend too
>>>>> much time though and agree something like that would be cleaner.
>>>>>
>>>>> Cheers,
>>>>> Tim
>>>>>
>>>>> On Thu, Oct 19, 2017 at 7:54 PM, Raghu Angadi <rang...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Thanks Tim.
>>>>>>
>>>>>> How about extending KafkaAvroDeserializer rather than
>>>>>> AbstractKafkaAvroDeserializer?
>>>>>>
>>>>>> TypedKafkaAvroDeserializer class below is useful, but not directly
>>>>>> usable by the yet. It needs to store the actual type in Kafka consumer
>>>>>> config to retrieve at run time.
>>>>>> Even without storing the class, it is still useful. It simplifies
>>>>>> user code:
>>>>>>
>>>>>> public class EnvelopeKafkaAvroDeserializer extends
>>>>>> TypedKafkaAvroDeserializer {}
>>>>>>
>>>>>> This should be part of same package as KafkaAvroDeserializer
>>>>>> (surprised it is not there yet).
>>>>>>
>>>>>> On Thu, Oct 19, 2017 at 3:07 AM, Tim Robertson <
>>>>>> timrobertson...@gmail.com> wrote:
>>>>>>
>>>>>>> Happy to hear
>>>>>>>
>>>>>>> I wonder if we could do something like this (totally untested):
>>>>>>>
>>>>>>> public class TypedKafkaAvroDeserializer extends
>>>>>>> AbstractKafkaAvroDeserializer implements Deserializer {
>>>>>>>@Override
>>>>>>> public T deserialize(String s, byte[] bytes) {
>>>>>>> return (T) this.deserialize(bytes);
>>>>>>> }
>>>>>>> }
>>>>>>>
>>>>>>> On Thu, Oct 19, 2017 at 12:03 PM, Andrew Jones <
>>>>>>> andrew+b...@andrew-jones.com> wrote:
>>>>>>>
>>>>>>>> Thanks Tim, that works!
>>>>>>>>
>>>>>>>> Full code is:
>>>>>>>>
>>>>>>>> public class EnvelopeKafkaAvroDeserializer extends
>>>>>>>> AbstractKafkaAvroDeserializer implements Deserializer {
>>>>&

Re: KafkaIO and Avro

2017-10-19 Thread Raghu Angadi
Same for me. It does not look like there is an annotation to suppress the
error.

On Thu, Oct 19, 2017 at 12:18 PM, Tim Robertson <timrobertson...@gmail.com>
wrote:

> Hi Eugene,
>
> I understood that was where Andrew started and reported this.  I tried and
> saw the same as him.
>
> incompatible types: 
> java.lang.Class
> cannot be converted to org.apache.kafka.common.serialization.Deserializer<
> org.gbif.pipelines.io.avro.Envelope>
>
> similarly with
> (Class>) KafkaAvroDeserializer.class
>
>
>
> On Thu, Oct 19, 2017 at 9:00 PM, Eugene Kirpichov <kirpic...@google.com>
> wrote:
>
>> I don't think extending the class is necessary. Not sure I understand why
>> a simple type casting for withDeserializerAndCoder doesn't work? Have you
>> tried this?
>>
>> p.apply(KafkaIO.<String, Envelope>read()
>>   .withValueDeserializerAndCoder((Deserializer)Kafka
>> AvroDeserializer.class,
>>   AvroCoder.of(Envelope.class))
>>
>> On Thu, Oct 19, 2017 at 11:45 AM Tim Robertson <timrobertson...@gmail.com>
>> wrote:
>>
>>> Hi Raghu
>>>
>>> I tried that but with KafkaAvroDeserializer already implementing
>>> Deserializer I couldn't get it to work... I didn't spend too
>>> much time though and agree something like that would be cleaner.
>>>
>>> Cheers,
>>> Tim
>>>
>>> On Thu, Oct 19, 2017 at 7:54 PM, Raghu Angadi <rang...@google.com>
>>> wrote:
>>>
>>>> Thanks Tim.
>>>>
>>>> How about extending KafkaAvroDeserializer rather
>>>> than AbstractKafkaAvroDeserializer?
>>>>
>>>> TypedKafkaAvroDeserializer class below is useful, but not directly
>>>> usable by the yet. It needs to store the actual type in Kafka consumer
>>>> config to retrieve at run time.
>>>> Even without storing the class, it is still useful. It simplifies user
>>>> code:
>>>>
>>>> public class EnvelopeKafkaAvroDeserializer extends
>>>> TypedKafkaAvroDeserializer {}
>>>>
>>>> This should be part of same package as KafkaAvroDeserializer (surprised
>>>> it is not there yet).
>>>>
>>>> On Thu, Oct 19, 2017 at 3:07 AM, Tim Robertson <
>>>> timrobertson...@gmail.com> wrote:
>>>>
>>>>> Happy to hear
>>>>>
>>>>> I wonder if we could do something like this (totally untested):
>>>>>
>>>>> public class TypedKafkaAvroDeserializer extends
>>>>> AbstractKafkaAvroDeserializer implements Deserializer {
>>>>>@Override
>>>>> public T deserialize(String s, byte[] bytes) {
>>>>> return (T) this.deserialize(bytes);
>>>>> }
>>>>> }
>>>>>
>>>>> On Thu, Oct 19, 2017 at 12:03 PM, Andrew Jones <
>>>>> andrew+b...@andrew-jones.com> wrote:
>>>>>
>>>>>> Thanks Tim, that works!
>>>>>>
>>>>>> Full code is:
>>>>>>
>>>>>> public class EnvelopeKafkaAvroDeserializer extends
>>>>>> AbstractKafkaAvroDeserializer implements Deserializer {
>>>>>> @Override
>>>>>> public void configure(Map<String, ?> configs, boolean isKey) {
>>>>>> configure(new KafkaAvroDeserializerConfig(configs));
>>>>>> }
>>>>>>
>>>>>> @Override
>>>>>> public Envelope deserialize(String s, byte[] bytes) {
>>>>>> return (Envelope) this.deserialize(bytes);
>>>>>> }
>>>>>>
>>>>>> @Override
>>>>>> public void close() {}
>>>>>> }
>>>>>>
>>>>>> Nicer than my solution so think that is the one I'm going to go with
>>>>>> for now.
>>>>>>
>>>>>> Thanks,
>>>>>> Andrew
>>>>>>
>>>>>>
>>>>>> On Thu, Oct 19, 2017, at 10:20 AM, Tim Robertson wrote:
>>>>>>
>>>>>> Hi Andrew,
>>>>>>
>>>>>> I also saw the same behaviour.
>>>>>>
>>>>>> It's not pretty but perhaps try this? It was my last idea I ran out
>>>>>> of time to try...
>>>>>>
>>>>>>
>>>>>> *// Basically a cop

Re: KafkaIO and Avro

2017-10-19 Thread Raghu Angadi
Thanks Tim.

How about extending KafkaAvroDeserializer rather
than AbstractKafkaAvroDeserializer?

TypedKafkaAvroDeserializer class below is useful, but not directly usable
by the yet. It needs to store the actual type in Kafka consumer config to
retrieve at run time.
Even without storing the class, it is still useful. It simplifies user code:

public class EnvelopeKafkaAvroDeserializer extends
TypedKafkaAvroDeserializer {}

This should be part of same package as KafkaAvroDeserializer (surprised it
is not there yet).

On Thu, Oct 19, 2017 at 3:07 AM, Tim Robertson <timrobertson...@gmail.com>
wrote:

> Happy to hear
>
> I wonder if we could do something like this (totally untested):
>
> public class TypedKafkaAvroDeserializer extends
> AbstractKafkaAvroDeserializer implements Deserializer {
>@Override
> public T deserialize(String s, byte[] bytes) {
> return (T) this.deserialize(bytes);
> }
> }
>
> On Thu, Oct 19, 2017 at 12:03 PM, Andrew Jones <
> andrew+b...@andrew-jones.com> wrote:
>
>> Thanks Tim, that works!
>>
>> Full code is:
>>
>> public class EnvelopeKafkaAvroDeserializer extends
>> AbstractKafkaAvroDeserializer implements Deserializer {
>> @Override
>> public void configure(Map<String, ?> configs, boolean isKey) {
>> configure(new KafkaAvroDeserializerConfig(configs));
>> }
>>
>> @Override
>> public Envelope deserialize(String s, byte[] bytes) {
>> return (Envelope) this.deserialize(bytes);
>> }
>>
>> @Override
>> public void close() {}
>> }
>>
>> Nicer than my solution so think that is the one I'm going to go with for
>> now.
>>
>> Thanks,
>> Andrew
>>
>>
>> On Thu, Oct 19, 2017, at 10:20 AM, Tim Robertson wrote:
>>
>> Hi Andrew,
>>
>> I also saw the same behaviour.
>>
>> It's not pretty but perhaps try this? It was my last idea I ran out of
>> time to try...
>>
>>
>> *// Basically a copy KafkaAvroDeserializer with the casts in 
>> deserialize**public class *EnvelopeAvroDeserializer *extends 
>> *AbstractKafkaAvroDeserializer *implements *Deserializer {
>>
>>   ...
>>
>>   *public *Envelope deserialize(String s, *byte*[] bytes) {
>>
>> *return *(Envelope) *this*.deserialize(bytes);
>>
>>   }
>>
>>
>>
>>   *public *Envelope deserialize(String s, *byte*[] bytes, Schema 
>> readerSchema) {
>>
>> *return *(Envelope) *this*.deserialize(bytes, readerSchema);
>>
>>   }
>>
>>
>>
>>   ...
>>
>> }
>>
>>  Tim
>>
>>
>> On Thu, Oct 19, 2017 at 10:52 AM, Andrew Jones <
>> andrew+b...@andrew-jones.com> wrote:
>>
>>
>> Using Object doesn't work unfortunately. I get an 'Unable to
>> automatically infer a Coder' error at runtime.
>>
>> This is the code:
>>
>> p.apply(KafkaIO.<String, Object>read()
>> .withValueDeserializer(KafkaAvroDeserializer.class)
>>
>> It compiles, but at runtime:
>>
>> Caused by: java.lang.RuntimeException: Unable to automatically infer a
>> Coder for the Kafka Deserializer class 
>> io.confluent.kafka.serializers.KafkaAvroDeserializer:
>> no coder registered for type class java.lang.Object
>> at org.apache.beam.sdk.io.kafka.KafkaIO.inferCoder(KafkaIO.java:1696)
>>
>> So far the only thing I've got working is this, where I use the
>> ByteArrayDeserializer and then parse Avro myself:
>>
>> private static KafkaAvroDecoder avroDecoder;
>> static {
>> final Properties props = new Properties();
>> props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka:9092");
>> props.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG,
>> "http://registry:8081;);
>> props.put(KafkaAvroDeserializerConfig.SPECIFIC_AVRO_READER_CONFIG,
>> true);
>> VerifiableProperties vProps = new VerifiableProperties(props);
>> avroDecoder = new KafkaAvroDecoder(vProps);
>> }
>>
>> public static void main(String[] args) {
>>
>> PipelineOptions options = PipelineOptionsFactory.create();
>> Pipeline p = Pipeline.create(options);
>>
>> p.apply(KafkaIO.<byte[], byte[]>read()
>> .withBootstrapServers("kafka:9092")
>> .withTopic("dbserver1.inventory.customers")
>> .withKeyDeserializer(ByteArrayDeserializer.class)
>>  

Re: Limit the number of DoFn instances per worker?

2017-10-18 Thread Raghu Angadi
One way you can limit DoFn parallelism is to reshuffle input into fixed
number of shards. If you want to limit it to 32 across 8 workers, you can
reshuffle into 32 shards. In Dataflow, this roughly evenly distribute among
the workers. You can take a look at the this stackoverflow question

where
the user wanted to increase parallelism.
In Dataflow DoFn parallelism depends not just on cores on the workers.. it
can have much higher parallism if it is after a GroupByKey (depending on
key cardinality)

Regd Autoscaling: if you have small number of records with a lot processing
(order of minutes), it might be hard to trigger upscaling if this skew is
very high. All the pending records might fit in internal queues and the
runner might see zero backlog. Limiting parallelism with reshuffle should
help with this.


On Tue, Oct 17, 2017 at 5:33 PM, Derek Hao Hu 
wrote:

> Thanks Rafal and Lukasz! These are great suggestions! One quick question
> about using semaphore though, would it be possible for multiple elements to
> pile up in a particular worker instance, waiting to acquire the semaphore
> but can't? I'll definitely test it though.
>
> Lukasz, let me try to explain why I feel this autoscaling might not be the
> ideal solution first. I'll definitely contact dataflow-feedb...@google.com
> as well but I'll try to give some of my [probably incorrect] thoughts.
>
> So basically based on my understanding if Beam tries to allocate multiple
> elements to a single machine, let's assume an ideal computational model
> where each single core takes T time to finish processing an element but if
> all 32 cores can be used to process this element then it takes T/32 time.
>
> Therefore, if we have 32 incoming elements, if Beam allocates 32 threads
> on a worker instance for this DoFn, each element using a single core will
> be finished in T time and therefore there would be no back log during this
> time since all the elements are being processed. But if we can tune the
> parameter to say Beam should allocate fewer elements per worker instance,
> then this creates a backlog and autoscaling might trigger earlier, so
> technically the overall system lag might actually be better?
>
> I haven't tested this hypothesis yet but basically the above is my
> reasoning.
>
> Thanks,
>
> Derek
>
> On Tue, Oct 17, 2017 at 8:49 AM, Lukasz Cwik  wrote:
>
>> The `numberOfWorkerHarnessThreads` is worker wide and not per DoFn.
>> Setting this value to constrain how many threads are executing will impact
>> all parts of your pipeline. One idea is to use a Semaphore as a static
>> object within your DoFn with a fixed number of allowed actors that can
>> enter and execute your Tensorflow.
>>
>> class TfDoFn {
>>   private static final int MAX_TF_ACTORS = 4;
>>   private static final Semaphore semaphore = new Semaphore(MAX_TF_ACTORS,
>> true);
>>
>>   @ProcessElement
>>   public void processElement(X x) {
>> try {
>>   semaphore.acquire();
>>   // Do TF work
>> } finally {
>>   semaphore.release();
>> }
>>   }
>> }
>>
>> This will ensure that your processing each TF item in a more timely
>> manner but it will still mean that there could be many other TF items which
>> are still sitting around waiting for the semaphore to be acquired.
>>
>> As an alternative, I would recommend contacting
>> dataflow-feedb...@google.com specifically referencing how you believe
>> autoscaling is not working well for your usecase/pipeline. Also provide a
>> description of your pipeline and some job ids (if possible).
>>
>>
>> On Mon, Oct 16, 2017 at 6:26 PM, Rafal Wojdyla  wrote:
>>
>>> Hi.
>>> To answer your question: if we limit ourselves to DataflowRunner, you
>>> could use `numberOfWorkerHarnessThreads`. See more here
>>> .
>>> That said, I'm not gonna comment whether that is a good remedy for your
>>> actual problem.
>>> - rav
>>>
>>> On Mon, Oct 16, 2017 at 8:48 PM, Derek Hao Hu 
>>> wrote:
>>>
 Hi,

 ​Is there an easy way to limit the number of DoFn instances per worker?

 The use case is like this: we are calling TensorFlow in our DoFn and
 each TensorFlow call would automatically try to allocate the available CPU
 resources. So in a streaming pipeline, what I'm seeing is the inference
 time will become longer over time if autoscaling didn't catch up. My
 hypothesis is that Beam is trying to allocate a specific number of elements
 (maybe the number of cores?) on each worker for a particular DoFn and then
 these TensorFlow threads contend for CPU cycles. Therefore, I would like to
 know whether it's possible to limit the number of threads a pipeline runner
 can allocate for a DoFn per worker. 

Re: [VOTE] [DISCUSSION] Remove support for Java 7

2017-10-17 Thread Raghu Angadi
+1.

On Tue, Oct 17, 2017 at 2:11 PM, David McNeill  wrote:

> The final version of Beam that supports Java 7 should be clearly stated in
> the docs, so those stuck on old production infrastructure for other java
> app dependencies know where to stop upgrading.
>
> David McNeill
> 021 721 015
>
>
>
> On 18 October 2017 at 05:16, Ismaël Mejía  wrote:
>
>> We have discussed recently in the developer mailing list about the
>> idea of removing support for Java 7 on Beam. There are multiple
>> reasons for this:
>>
>> - Java 7 has not received public updates for almost two years and most
>> companies are moving / have already moved to Java 8.
>> - A good amount of the systems Beam users rely on have decided to drop
>> Java 7 support, e.g. Spark, Flink, Elasticsearch, even Hadoop plans to
>> do it on version 3.
>> - Most Big data distributions and Cloud managed Spark/Hadoop services
>> have already moved to Java 8.
>> - Recent versions of core libraries Beam uses are moving to be Java 8
>> only (or mostly), e.g. Guava, Google Auto, etc.
>> - Java 8 has some nice features that can make Beam code nicer e.g.
>> lambdas, streams.
>>
>> Considering that Beam is a ‘recent’ project we expect users to be
>> already using Java 8. However we wanted first to ask the opinion of
>> the Beam users on this subject. It could be the case that some of the
>> users are still dealing with some old cluster running on Java 7 or
>> have another argument to keep the Java 7 compatibility.
>>
>> So, please vote:
>> +1 Yes, go ahead and move Beam support to Java 8.
>>  0 Do whatever you want. I don’t have a preference.
>> -1 Please keep Java 7 compatibility (if possible add your argument to
>> keep supporting for Java 7).
>>
>
>


Re: Problem with autoscaling

2017-09-06 Thread Raghu Angadi
KafkaIO also supports autoscaling on Dataflow. In fact any unbounded source
is supported as long as it implements 'getSplitBacklogBytes()' or
'getTotalBacklogBytes()' interface. E.g. PR 3551
 adds backlog support for
KinesisIO.

On Tue, Sep 5, 2017 at 10:19 PM, Lukasz Cwik  wrote:

> That is correct, autoscaling for streaming is only supported in Pubsub.
> What sources were you interested in?
>
> On Mon, Sep 4, 2017 at 12:54 AM, Derek Hao Hu 
> wrote:
>
>> I've used PubSubIO for autoscaling on a streaming pipeline and it seems
>> to be working fine so far.
>>
>> I don't know if it is the **only** source that supports autoscaling on
>> streaming pipelines though.
>>
>> On Mon, Sep 4, 2017 at 12:52 AM, Vilhelm von Ehrenheim <
>> vonehrenh...@gmail.com> wrote:
>>
>>> Hi!
>>> I cannot make my streaming job autoscale on Dataflow. To find an answer
>>> to why I was looking around in the Dataflow docs and found this:
>>>
>>> > Currently, PubSubIO is the only source that supports autoscaling on
>>> streaming pipelines.
>>>
>>> on the following page: https://cloud.google.com
>>> /dataflow/service/dataflow-service-desc#autoscaling
>>>
>>> Does anyone know if this is true? I know this is not the forum for
>>> Dataflow questions in general but I though someone else here might have
>>> experience that support or contradict this.
>>>
>>> Thanks,
>>> Vilhelm von Ehrenheim
>>>
>>
>>
>>
>> --
>> Derek Hao Hu
>>
>> Software Engineer | Snapchat
>> Snap Inc.
>>
>
>


Re: Kafka offset management

2017-06-30 Thread Raghu Angadi
Gwilym,

I think your understanding is correct, with one caveat as noted below. As
Jingsong suggested committing offsets in '
KafkaCheckpointMark.finalizeCheckpoint()
<https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java#L48>'
is required. I had left a comment there. It is fairly straight forward to
add this as an option.

Note that finalizing would give at least once semantics only if you drain a
pipeline before restarting it. If a pipeline is killed or crashes, you can
still miss some records. Finalize checkpoint is called in Dataflow once the
messages are checkpointed for the current stage. The downstream stages
might not have processed them. Draining a pipeline ensures that all the
input is processed through the pipeline.

> The consumer factory is used because of some runtime SSL key/truststore
setup:

btw, KafkaIO includes api to set
<https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java#L207>
consumer
configs. Wasn't it enough?
Did you get trust store config working with Dataflow? Last I remember
"ssl.truststore.location" had to be a local file on the worker and it was
not easy to make that accessible.

On Thu, Jun 29, 2017 at 10:47 PM, JingsongLee <lzljs3620...@aliyun.com>
wrote:

> Sorry, forget user mail group. + @user
>
> Yeah, although KafkaIO (exposed offsets interface) is different from
> PubSubIO, committing the offsets to Kafka in finalizeCheckpoint is also
> a way.
> Welcome to contribute and maybe @Raghu Angadi can show more messages.
>
> Best, Jingsong Lee
>
> --
> From:Gwilym Evans <gwilym.ev...@bigcommerce.com>
> Time:2017 Jun 30 (Fri) 13:27
> To:JingsongLee <lzljs3620...@aliyun.com>
> Subject:Re: Kafka offset management
>
> Thanks for the info. I'll have a look and see if I can do anything similar.
>
> I am very new to Beam internals, but I've been having a look at the
> KafkaIO code and comparing it to the PubSubIO code.
>
> I think that the finalizeCheckpoint implementation for KafkaIO should
> definitely be committing the offsets to Kafka, if possible. But perhaps
> only when a Kafka group.id is configured, as committing offsets for
> random or blank group IDs is kind of pointless.
>
> I think I'll take a shot at making and contributing this, even if it's
> optional. Unless you can think of a reason to specifically not do this?
>
> Though, looking a the KafkaIO source for this, there is even a comment
> there alluding to the fact that this should maybe be done to provide better
> restart options.
>
> -Gwilym
>
>
> On 30 June 2017 at 05:08, JingsongLee <lzljs3620...@aliyun.com> wrote:
> Oh. I know what you mean.
>
> In our production, if we need to re-run (lose checkpoint and state when
> a job crashes, is canceled, or is drained), we will set the KafkaIO
> startTime to start a new job, because we generally know the last consumer
> timestamp of previous job. (Do not be too precise, back
> to a safe point can be)
> This feature is finished in 2.1.0 version.
> Jira: https://issues.apache.org/jira/browse/BEAM-2248
>
> A more accurate way is re-run by kafka offsets(not support yet), but you
> should konw the last snapshot of job.
>
> Best, Jingsong Lee
>
> --
> From:Gwilym Evans <gwilym.ev...@bigcommerce.com>
> Time:2017 Jun 30 (Fri) 12:20
> To:user <user@beam.apache.org>; JingsongLee <lzljs3620...@aliyun.com>
> Subject:Re: Kafka offset management
>
> Hi JingsongLee,
>
> Thanks for the reply.
>
> What I'm trying to avoid are lost / skipped messages due to two situations:
>
> 1. Lost offsets, or
> 2. Premature offset commits
>
> I've researched snapshot checkpoints, and from what I understand these are
> only maintained in Dataflow when a job is updated. If a job crashes, is
> cancelled, or is drained, then the checkpoints are lost. This is situation
> (1) above.
>
> From what I understand about auto-commit offsets, it's where the Kafka
> client periodically commits offsets it has polled automatically. In the
> case of Beam and Dataflow, this would be even if the offsets it is
> committing has not yet been fully processed by the pipeline. This is
> situation (2) above.
>
> So far I'm not seeing a way to avoid data loss besides resetting to the
> earliest offset when a job starts. But, given we retain data in our Kafka
> topics for up to 7 days, that is not feasible from a performance point of
> view.
>
> Can anyone confirm / deny my understanding here?
>
> Cheers,
> Gwilym
>
> On 30 June 2017 a

Re: How to decrease latency when using PubsubIO.Read?

2017-05-24 Thread Raghu Angadi
Thanks. Looked a some job system level metrics. I see minimal latency
within Dataflow pipeline itself, you might not see much improvement from
Reshuffle() (may be ~0.25 seconds).

Do you have control on publisher? Publishers might be batching hundreds of
messages, which adds latency. You can try to reduce that. Even then PubSub
itself does some batching internally which might limit overall latency.
Removing publisher batching is worth a try.

On Wed, May 24, 2017 at 11:46 AM, Josh <jof...@gmail.com> wrote:

> Hi Raghu,
>
> My job ID is 2017-05-24_02_46_42-11524480684503077480 - thanks for taking
> a look!
>
> Yes I'm using BigtableIO for the sink and I am measuring the end-to-end
> latency. It seems to take 3-6 seconds typically, I would like to get it
> down to ~1s.
>
> Thanks,
> Josh
>
> On Wed, May 24, 2017 at 6:50 PM, Raghu Angadi <rang...@google.com> wrote:
>
>> Josh,
>>
>> Can you share your job_id? I could take look. Are you measuring latency
>> end-to-end (publisher to when it appears on BT?). Are you using BigtableIO
>> for sink?
>>
>> There is no easy way to use more workers when auto-scaling is enabled. It
>> thinks your backlog and CPU are low enough and does not need to scale.
>> Raghu.
>>
>> On Wed, May 24, 2017 at 10:14 AM, Josh <jof...@gmail.com> wrote:
>>
>>> Thanks Ankur, that's super helpful! I will give these optimisations a go.
>>>
>>> About the "No operations completed" message - there are a few of these
>>> in the logs (but very few, like 1 an hour or something) - so probably no
>>> need to scale up Bigtable.
>>> I did however see a lot of INFO messages "Wrote 0 records" in the logs. 
>>> Probably
>>> about 50% of the "Wrote n records" messages are zero. While the other 50%
>>> are quite high (e.g. "Wrote 80 records"). Not sure if that could indicate a
>>> bad setting?
>>>
>>> Josh
>>>
>>>
>>>
>>> On Wed, May 24, 2017 at 5:22 PM, Ankur Chauhan <an...@malloc64.com>
>>> wrote:
>>>
>>>> There are two main things to see here:
>>>>
>>>> * In the logs, are there any messages like "No operations completed
>>>> within the last 61 seconds. There are still 1 simple operations and 1
>>>> complex operations in progress.” This means you are underscaled on the
>>>> bigtable side and would benefit from  increasing the node count.
>>>> * We also saw some improvement in performance (workload dependent) by
>>>> going to a bigger worker machine type.
>>>> * Another optimization that worked for our use case:
>>>>
>>>> // streaming dataflow has larger machines with smaller bundles, so we can 
>>>> queue up a lot more without blowing up
>>>> private static BigtableOptions 
>>>> createStreamingBTOptions(AnalyticsPipelineOptions opts) {
>>>> return new BigtableOptions.Builder()
>>>> .setProjectId(opts.getProject())
>>>> .setInstanceId(opts.getBigtableInstanceId())
>>>> .setUseCachedDataPool(true)
>>>> .setDataChannelCount(32)
>>>> .setBulkOptions(new BulkOptions.Builder()
>>>> .setUseBulkApi(true)
>>>> .setBulkMaxRowKeyCount(2048)
>>>> .setBulkMaxRequestSize(8_388_608L)
>>>> .setAsyncMutatorWorkerCount(32)
>>>> .build())
>>>> .build();
>>>> }
>>>>
>>>>
>>>> There is a lot of trial and error involved in getting the end-to-end
>>>> latency down so I would suggest enabling the profiling using the
>>>> —saveProfilesToGcs option and get a sense of what is exactly happening.
>>>>
>>>> — Ankur Chauhan
>>>>
>>>> On May 24, 2017, at 9:09 AM, Josh <jof...@gmail.com> wrote:
>>>>
>>>> Ah ok - I am using the Dataflow runner. I didn't realise about the
>>>> custom implementation being provided at runtime...
>>>>
>>>> Any ideas of how to tweak my job to either lower the latency consuming
>>>> from PubSub or to lower the latency in writing to Bigtable?
>>>>
>>>>
>>>> On Wed, May 24, 2017 at 4:14 PM, Lukasz Cwik <lc...@google.com> wrote:
>>>>
>>>>> What runner are you using (Flink, Spark, Google Cloud Dataflow, Apex,
>>>>> ...)

Re: How to decrease latency when using PubsubIO.Read?

2017-05-24 Thread Raghu Angadi
On Wed, May 24, 2017 at 10:14 AM, Josh  wrote:

> Thanks Ankur, that's super helpful! I will give these optimisations a go.
>
> About the "No operations completed" message - there are a few of these in
> the logs (but very few, like 1 an hour or something) - so probably no need
> to scale up Bigtable.
> I did however see a lot of INFO messages "Wrote 0 records" in the logs. 
> Probably
> about 50% of the "Wrote n records" messages are zero. While the other 50%
> are quite high (e.g. "Wrote 80 records"). Not sure if that could indicate a
> bad setting?
>

If this is a single stage pipeline (no GroupByKey()), high number could
indicate bundle size in Dataflow. A larger bundle could increase latency.
You can try a work around by adding a reshuffle
.
Something like :
  .apply(...)
  .apply(message -> (random.nextLong(), message)) // add a random key.
  .apply(Reshuffle.of())
  .apply((i, message) -> message) // strip key
  .apply(sink)

This would bring 'n' down to 1 or so. I don't know why there are messages
that say '0' records.

Raghu.


> Josh
>
>
>
> On Wed, May 24, 2017 at 5:22 PM, Ankur Chauhan  wrote:
>
>> There are two main things to see here:
>>
>> * In the logs, are there any messages like "No operations completed
>> within the last 61 seconds. There are still 1 simple operations and 1
>> complex operations in progress.” This means you are underscaled on the
>> bigtable side and would benefit from  increasing the node count.
>> * We also saw some improvement in performance (workload dependent) by
>> going to a bigger worker machine type.
>> * Another optimization that worked for our use case:
>>
>> // streaming dataflow has larger machines with smaller bundles, so we can 
>> queue up a lot more without blowing up
>> private static BigtableOptions 
>> createStreamingBTOptions(AnalyticsPipelineOptions opts) {
>> return new BigtableOptions.Builder()
>> .setProjectId(opts.getProject())
>> .setInstanceId(opts.getBigtableInstanceId())
>> .setUseCachedDataPool(true)
>> .setDataChannelCount(32)
>> .setBulkOptions(new BulkOptions.Builder()
>> .setUseBulkApi(true)
>> .setBulkMaxRowKeyCount(2048)
>> .setBulkMaxRequestSize(8_388_608L)
>> .setAsyncMutatorWorkerCount(32)
>> .build())
>> .build();
>> }
>>
>>
>> There is a lot of trial and error involved in getting the end-to-end
>> latency down so I would suggest enabling the profiling using the
>> —saveProfilesToGcs option and get a sense of what is exactly happening.
>>
>> — Ankur Chauhan
>>
>> On May 24, 2017, at 9:09 AM, Josh  wrote:
>>
>> Ah ok - I am using the Dataflow runner. I didn't realise about the custom
>> implementation being provided at runtime...
>>
>> Any ideas of how to tweak my job to either lower the latency consuming
>> from PubSub or to lower the latency in writing to Bigtable?
>>
>>
>> On Wed, May 24, 2017 at 4:14 PM, Lukasz Cwik  wrote:
>>
>>> What runner are you using (Flink, Spark, Google Cloud Dataflow, Apex,
>>> ...)?
>>>
>>> On Wed, May 24, 2017 at 8:09 AM, Ankur Chauhan 
>>> wrote:
>>>
 Sorry that was an autocorrect error. I meant to ask - what dataflow
 runner are you using? If you are using google cloud dataflow then the
 PubsubIO class is not the one doing the reading from the pubsub topic. They
 provide a custom implementation at run time.

 Ankur Chauhan
 Sent from my iPhone

 On May 24, 2017, at 07:52, Josh  wrote:

 Hi Ankur,

 What do you mean by runner address?
 Would you be able to link me to the comment you're referring to?

 I am using the PubsubIO.Read class from Beam 2.0.0 as found here:
 https://github.com/apache/beam/blob/release-2.0.0/sdks/java/
 io/google-cloud-platform/src/main/java/org/apache/beam/sdk/i
 o/gcp/pubsub/PubsubIO.java

 Thanks,
 Josh

 On Wed, May 24, 2017 at 3:36 PM, Ankur Chauhan 
 wrote:

> What runner address you using. Google cloud dataflow uses a closed
> source version of the pubsub reader as noted in a comment on Read class.
>
> Ankur Chauhan
> Sent from my iPhone
>
> On May 24, 2017, at 04:05, Josh  wrote:
>
> Hi all,
>
> I'm using PubsubIO.Read to consume a Pubsub stream, and my job then
> writes the data out to Bigtable. I'm currently seeing a latency of 3-5
> seconds between the messages being published and being written to 
> Bigtable.
>
> I want to try and decrease the latency to <1s if possible - does
> anyone have any tips for doing this?
>
> I noticed that there is a 

Re: KafkaIO nothing received?

2017-05-09 Thread Raghu Angadi
Hi Conrad,

Kafka consumer in Beam is 0.9 or above. Almost certainly you are running a
0.9 or newer servers. I don't think 0.9 new client can talk to old brokers
(but 0.9 brokers can talk to older clients). How did you confirm the server
version? You can check the server log. But I might be mistaken about this
incompatibility.

Can you post 'jstack' of the application when it is stuck (assuming you are
running using DirectRunner)?

> Kafka 0.8 requires a zookeeper connect property to be set, but I can’t
set this using updateConsumerProperties as the value gets discarded.

KafkaIO does not place any restrictions on ConsumerConfig except for key
and value deserializers. The message about discarding these would be from
Kafka consumer itself. I think it ignores configuration settings that it
does not know about and logs them
<https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L721>
.

Raghu.

On Mon, May 8, 2017 at 1:57 AM, Conrad Crampton <conrad.cramp...@secdata.com
> wrote:

> Hi Raghu,
>
> Yeah, the job just waits and does nothing. It reports the correct offset
> (this changes when I use ‘earliest’ or ‘latest’), but nothing is received.
> There are definitely messages in the queue. I am using Beam 0.6.
>
> With my other application using Flink, I am using the FlinkKafkaConsumer08
> libraries (and not the FlinkKafkaConsumer09)  as I am sure I had a similar
> problem then i.e. no errors reported and appears to work fine, but nothing
> actually received in the streaming job.
>
> Kafka 0.8 requires a zookeeper connect property to be set, but I can’t set
> this using updateConsumerProperties as the value gets discarded.
>
> Thanks
>
> Conrad
>
>
>
> *From: *Raghu Angadi <rang...@google.com>
> *Reply-To: *"user@beam.apache.org" <user@beam.apache.org>
> *Date: *Thursday, 4 May 2017 at 18:27
>
> *To: *"user@beam.apache.org" <user@beam.apache.org>
> *Subject: *Re: KafkaIO nothing received?
>
>
>
> Conrad,
>
>
>
> It does not look like there is a version incompatibility. You would see
> errors during during initialization otherwise. Log line "INFO: Reader-0:
> reading from test-http-logs-json-0 starting at offset 0" says it Kafka
> consumer was able to connect to servers.
>
>
>
> Does the the job just wait inside p.run()? What is the version of Beam you
> are using? If it is just waiting for records, please ensure the topic has
> messages (using kafka-console-consumer.sh etc). Alternately you can try
> reading from another topic you mentioned that worked fine.
>
>
>
> Raghu.
>
>
>
> On Thu, May 4, 2017 at 10:07 AM, Conrad Crampton <
> conrad.cramp...@secdata.com> wrote:
>
> Hi,
>
> Ok, good to know I’m not going totally mad.
>
> I think I may have been running around in circles unnecessarily 
>
> I am using kafka as part of an HDP installation (with Ambari). The Ambari
> interface is reporting my kafka version as 0.9.0.2.3 and indeed the output
> given previously give
>
> INFO: Kafka version : 0.9.0.1 (which doesn’t make particular sense). So I
> have ssh’d onto the server and looked at the libs for kafka and they are
> kafka_2.10-0.8.2.2.3.0.0-2557.jar so I’m guessing something is not quite
> right. This is incredibly frustrating as it looks like I am trying to
> connect to  v0.9 kafka but it’s actually v0.8 which clearly is very
> different wrt/ zookeeper etc. This is also backup up by trying the
> kafka-console-consumer.sh (and all the other tools) ask for mandatory
> zookeeper options which shouldn’t be necessary as far as I understand it
> for v0.9.
>
>
>
> I am currently looking at https://github.com/mxm/incubator-beam/blob/
> 63bce07d8c6cc5e610ad24e915e2585fef582567/runners/flink/
> examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/
> KafkaIOExamples.java to see if I can use this code somehow to use Beam
> with Kafka v0.8. I am really hoping to as I have no option to upgrade
> currently and I really like the abstraction of Beam.
>
>
>
> Thanks
>
> Conrad
>
>
>
>
>
> *From: *Mingmin Xu <mingm...@gmail.com>
> *Reply-To: *"user@beam.apache.org" <user@beam.apache.org>
> *Date: *Thursday, 4 May 2017 at 17:59
> *To: *"user@beam.apache.org" <user@beam.apache.org>
> *Subject: *Re: KafkaIO nothing received?
>
>
>
> @Conrad,
>
> Your code should be good to go, I can run it in my local env. There're two
> points you may have a check:
>
> 1). does the topic have data there, you can confirm with kafka cli '
> *bin/kafka-console-consumer.sh*';
>
> 2). is the port in bootstrapServers right? By de

Re: Kafka Offset handling for Restart/failure scenarios.

2017-03-21 Thread Raghu Angadi
Expanding a bit more on what Dan wrote:

   - In Dataflow, there are two modes of restarting a job : regular stop
   and then start & an *update*. The checkpoint is carried over only in the
   case of update.
   - Update is the only to keep 'exactly-once' semantics.
   - If the requirements are not very strict, you can enable offset commits
   in Kafka itself. KafkaIO lets you configure this. Here the pipeline would
   start reading from approximately where it left off in the previous run.
  - When a offset commits are enabled, KafkaIO could this by
  implementing 'finalize()' API on KafkaCheckpointMark [1].
  - This is runner independent.
  - The compromise is that this might skip a few records or read a few
  old records when the pipeline is restarted.
  - This does not override 'resume from checkpoint' support when runner
  provides KafkaCheckpointMark. Externally committed offsets are used only
  when KafkaIO's own CheckpointMark is not available.

[1]:
https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java#L50

On Tue, Mar 21, 2017 at 5:28 PM, Dan Halperin  wrote:

> [We should keep user list involved if that's where the discussion
> originally was :)]
>
> Jins George's original question was a good one. The right way to resume
> from the previous offset here is what we're already doing – use the
> KafkaCheckpointMark. In Beam, the runner maintains the state and not the
> external system. Beam runners are responsible for maintaining the
> checkpoint marks, and for redoing all uncommitted (uncheckpointed) work. If
> a user disables checkpointing, then they are explicitly opting into "redo
> all work" on restart.
>
> --> If checkpointing is enabled but the KafkaCheckpointMark is not being
> provided, then I'm inclined to agree with Amit that there may simply be a
> bug in the FlinkRunner. (+aljoscha)
>
> For what Mingmin Xu asked about: presumably if the Kafka source is
> initially configured to "read from latest offset", when it restarts with no
> checkpoint this will automatically go find the latest offset. That would
> mimic at-most-once semantics in a buggy runner that did not provide
> checkpointing.
>
> Dan
>
> On Tue, Mar 21, 2017 at 2:59 PM, Mingmin Xu  wrote:
>
>> In SparkRunner, the default checkpoint storage is TmpCheckpointDirFactory.
>> Can it restore during job restart? --Not test the runner in streaming for
>> some time.
>>
>> Regarding to data-completeness, I would use at-most-once when few data
>> missing(mostly tasknode failure) is tolerated, compared to the performance
>> cost introduced by 'state'/'checkpoint'.
>>
>> On Tue, Mar 21, 2017 at 1:36 PM, Amit Sela  wrote:
>>
>> > On Tue, Mar 21, 2017 at 7:26 PM Mingmin Xu  wrote:
>> >
>> > > Move discuss to dev-list
>> > >
>> > > Savepoint in Flink, also checkpoint in Spark, should be good enough to
>> > > handle this case.
>> > >
>> > > When people don't enable these features, for example only need
>> > at-most-once
>> > >
>> > The Spark runner forces checkpointing on any streaming (Beam)
>> application,
>> > mostly because it uses mapWithState for reading from UnboundedSource and
>> > updateStateByKey form GroupByKey - so by design, Spark runner is
>> > at-least-once. Generally, I always thought that applications that
>> require
>> > at-most-once are more focused on processing time only, as they only care
>> > about whatever get's ingested into the pipeline at a specific time and
>> > don't care (up to the point of losing data) about correctness.
>> > I would be happy to hear more about your use case.
>> >
>> > > semantic, each unbounded IO should try its best to restore from last
>> > > offset, although CheckpointMark is null. Any ideas?
>> > >
>> > > Mingmin
>> > >
>> > > On Tue, Mar 21, 2017 at 9:39 AM, Dan Halperin 
>> > wrote:
>> > >
>> > > > hey,
>> > > >
>> > > > The native Beam UnboundedSource API supports resuming from
>> checkpoint
>> > --
>> > > > that specifically happens here
>> > > > <
>> > > https://github.com/apache/beam/blob/master/sdks/java/io/kafk
>> > a/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java#L674>
>> > > when
>> > > > the KafkaCheckpointMark is non-null.
>> > > >
>> > > > The FlinkRunner should be providing the KafkaCheckpointMark from the
>> > most
>> > > > recent savepoint upon restore.
>> > > >
>> > > > There shouldn't be any "special" Flink runner support needed, nor is
>> > the
>> > > > State API involved.
>> > > >
>> > > > Dan
>> > > >
>> > > > On Tue, Mar 21, 2017 at 9:01 AM, Jean-Baptiste Onofré <
>> j...@nanthrax.net
>> > >
>> > > > wrote:
>> > > >
>> > > >> Would not it be Flink runner specific ?
>> > > >>
>> > > >> Maybe the State API could do the same in a runner agnostic way
>> (just
>> > > >> thinking loud) ?
>> > > >>
>> > > >> Regards
>> > > >> JB
>> > > >>
>> > > >> On 

  1   2   >