Skip to content

Commit b3a9ea1

Browse files
sjwiesmantzulitai
authored andcommitted
[FLINK-16557][docs] Document YAML-ized Kafka egresses / ingresses in Stateful Functions documentation
This closes #63.
1 parent 7fcdae2 commit b3a9ea1

File tree

2 files changed

+250
-34
lines changed

2 files changed

+250
-34
lines changed

statefun-docs/docs/io_module/index.rst

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ Based on the concept of Ingress (input) and Egress (output) points, and built on
3232
.. _ingress:
3333

3434
Ingress
35-
^^^^^^^^
35+
=======
3636

3737
An Ingress is an input point where data is consumed from an external system and forwarded to zero or more functions.
3838
It is defined via an ``IngressIdentifier`` and an ``IngressSpec``.
@@ -71,7 +71,7 @@ The spec defines the details of how to connect to the external system, which is
7171
spec: # ingress specific configurations
7272
7373
Router
74-
""""""
74+
^^^^^^
7575

7676
A router is a stateless operator that takes each record from an ingress and routes it to zero or more functions.
7777
Routers are bound to the system via a stateful function module, and unlike other components, an ingress may have any number of routers.
@@ -101,7 +101,7 @@ Routers are bound to the system via a stateful function module, and unlike other
101101
.. _egress:
102102

103103
Egress
104-
^^^^^^
104+
======
105105

106106
Egress is the opposite of ingress; it is a point that takes messages and writes them to external systems.
107107
Each egress is defined using two components, an ``EgressIdentifier`` and an ``EgressSpec``.

statefun-docs/docs/io_module/kafka.rst

Lines changed: 247 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -19,13 +19,14 @@ Apache Kafka
1919

2020
Stateful Functions offers an Apache Kafka I/O Module for reading from and writing to Kafka topics.
2121
It is based on Apache Flink's universal `Kafka connector <https://ci.apache.org/projects/flink/flink-docs-stable/dev/connectors/kafka.html>`_ and provides exactly-once processing semantics.
22+
The Kafka I/O Module is configurable in Yaml or Java.
2223

2324
.. contents:: :local:
2425

2526
Dependency
26-
===========
27+
==========
2728

28-
To use the Kafka I/O Module, please include the following dependency in your pom.
29+
To use the Kafka I/O Module in Java, please include the following dependency in your pom.
2930

3031
.. code-block:: xml
3132
@@ -36,44 +37,178 @@ To use the Kafka I/O Module, please include the following dependency in your pom
3637
<scope>provided</scope>
3738
</dependency>
3839
39-
Kafka Ingress Builder
40-
=====================
40+
Kafka Ingress Spec
41+
==================
4142

42-
A ``KafkaIngressBuilder`` declares an ingress spec for consuming from Kafka cluster.
43+
A ``KafkaIngressSpec`` declares an ingress spec for consuming from Kafka cluster.
4344

4445
It accepts the following arguments:
4546

4647
1) The ingress identifier associated with this ingress
4748
2) The topic name / list of topic names
4849
3) The address of the bootstrap servers
4950
4) The consumer group id to use
50-
5) A ``KafkaIngressDeserializer`` for deserializing data from Kafka
51+
5) A ``KafkaIngressDeserializer`` for deserializing data from Kafka (Java only)
5152
6) The position to start consuming from
5253

53-
.. literalinclude:: ../../src/main/java/org/apache/flink/statefun/docs/io/kafka/IngressSpecs.java
54-
:language: java
55-
:lines: 18-
54+
.. tabs::
5655

57-
The ingress allows configuring the startup position to be one of the following:
56+
.. group-tab:: Java
5857

59-
* ``KafkaIngressStartupPosition#fromGroupOffsets()`` (default): starts from offsets that were committed to Kafka for the specified consumer group.
60-
* ``KafkaIngressStartupPosition#fromEarliest()``: starts from the earliest offset.
61-
* ``KafkaIngressStartupPosition#fromLatest()``: starts from the latest offset.
62-
* ``KafkaIngressStartupPosition#fromSpecificOffsets(Map)``: starts from specific offsets, defined as a map of partitions to their target starting offset.
63-
* ``KafkaIngressStartupPosition#fromDate(Date)``: starts from offsets that have an ingestion time larger than or equal to a specified date.
58+
.. literalinclude:: ../../src/main/java/org/apache/flink/statefun/docs/io/kafka/IngressSpecs.java
59+
:language: java
60+
:lines: 18-
6461

65-
On startup, if the specified startup offset for a partition is out-of-range or does not exist (which may be the case if the ingress is configured to
66-
start from group offsets, specific offsets, or from a date), then the ingress will fallback to using the position configured
67-
using ``KafkaIngressBuilder#withAutoOffsetResetPosition(KafkaIngressAutoResetPosition)``. By default, this is set to be the latest position.
62+
.. group-tab:: Yaml
63+
64+
.. code-block:: yaml
65+
66+
version: "1.0"
67+
68+
module:
69+
meta:
70+
type: remote
71+
spec:
72+
ingresses:
73+
- ingress:
74+
meta:
75+
type: statefun.kafka.io/routable-protobuf-ingress
76+
id: example/user-ingress
77+
spec:
78+
address: kafka-broker:9092
79+
consumerGroupId: routable-kafka-e2e
80+
startupPosition:
81+
type: earliest
82+
topics:
83+
- topic: messages-1
84+
typeUrl: com.googleapis/com.company.MessageWithAddress
85+
targets:
86+
- example-namespace/my-function-1
87+
- example-namespace/my-function-2
6888
6989
The ingress also accepts properties to directly configure the Kafka client, using ``KafkaIngressBuilder#withProperties(Properties)``.
7090
Please refer to the Kafka `consumer configuration <https://docs.confluent.io/current/installation/configuration/consumer-configs.html>`_ documentation for the full list of available properties.
7191
Note that configuration passed using named methods, such as ``KafkaIngressBuilder#withConsumerGroupId(String)``, will have higher precedence and overwrite their respective settings in the provided properties.
7292

93+
Startup Position
94+
^^^^^^^^^^^^^^^^
95+
96+
The ingress allows configuring the startup position to be one of the following:
97+
98+
**From Group Offset (default)**
99+
100+
Starts from offsets that were committed to Kafka for the specified consumer group.
101+
102+
.. tabs::
103+
104+
.. group-tab:: Java
105+
106+
.. code-block:: none
107+
108+
KafkaIngressStartupPosition#fromGroupOffsets();
109+
110+
.. group-tab:: Yaml
111+
112+
.. code-block:: yaml
113+
114+
startupPosition:
115+
type: group-offsets
116+
117+
**Earliest**
118+
119+
Starts from the earliest offset.
120+
121+
.. tabs::
122+
123+
.. group-tab:: Java
124+
125+
.. code-block:: none
126+
127+
KafkaIngressStartupPosition#fromEarliest();
128+
129+
.. group-tab:: Yaml
130+
131+
.. code-block:: yaml
132+
133+
startupPosition:
134+
type: earliest
135+
136+
**Latest**
137+
138+
Starts from the latest offset.
139+
140+
.. tabs::
141+
142+
.. group-tab:: Java
143+
144+
.. code-block:: none
145+
146+
KafkaIngressStartupPosition#fromLatest();
147+
148+
.. group-tab:: Yaml
149+
150+
.. code-block:: yaml
151+
152+
startupPosition:
153+
type: latest
154+
155+
**Specific Offsets**
156+
157+
Starts from specific offsets, defined as a map of partitions to their target starting offset.
158+
159+
.. tabs::
160+
161+
.. group-tab:: Java
162+
163+
.. code-block:: none
164+
165+
Map<TopicPartition, Long> offsets = new HashMap<>();
166+
offsets.add(new TopicPartition("user-topic", 0), 91);
167+
offsets.add(new TopicPartition("user-topic", 11), 11);
168+
offsets.add(new TopicPartition("user-topic", 8), 8);
169+
170+
KafkaIngressStartupPosition#fromSpecificOffsets(offsets);
171+
172+
.. group-tab:: Yaml
173+
174+
.. code-block:: yaml
175+
176+
startupPosition:
177+
type: specific-offsets
178+
offsets:
179+
- user-topic/0: 91
180+
- user-topic/1: 11
181+
- user-topic/2: 8
182+
183+
**Date**
184+
185+
Starts from offsets that have an ingestion time larger than or equal to a specified date.
186+
187+
.. tabs::
188+
189+
.. group-tab:: Java
190+
191+
.. code-block:: none
192+
193+
KafkaIngressStartupPosition#fromDate(ZonedDateTime.now());
194+
195+
.. group-tab:: Yaml
196+
197+
.. code-block:: yaml
198+
199+
startupPosition:
200+
type: date
201+
date: 2020-02-01 04:15:00.00 Z
202+
203+
On startup, if the specified startup offset for a partition is out-of-range or does not exist (which may be the case if the ingress is configured to
204+
start from group offsets, specific offsets, or from a date), then the ingress will fallback to using the position configured
205+
using ``KafkaIngressBuilder#withAutoOffsetResetPosition(KafkaIngressAutoResetPosition)``.
206+
By default, this is set to be the latest position.
207+
73208
Kafka Deserializer
74-
""""""""""""""""""
209+
^^^^^^^^^^^^^^^^^^
75210

76-
The Kafka ingress needs to know how to turn the binary data in Kafka into Java objects.
211+
When using the Java api, the Kafka ingress needs to know how to turn the binary data in Kafka into Java objects.
77212
The ``KafkaIngressDeserializer`` allows users to specify such a schema.
78213
The ``T deserialize(ConsumerRecord<byte[], byte[]> record)`` method gets called for each Kafka message, passing the key, value, and metadata from Kafka.
79214

@@ -90,30 +225,111 @@ It accepts the following arguments:
90225

91226
1) The egress identifier associated with this egress
92227
2) The address of the bootstrap servers
93-
3) A ``KafkaEgressSerializer`` for serializing data into Kafka
228+
3) A ``KafkaEgressSerializer`` for serializing data into Kafka (Java only)
94229
4) The fault tolerance semantic
95230
5) Properties for the Kafka producer
96231

97-
.. literalinclude:: ../../src/main/java/org/apache/flink/statefun/docs/io/kafka/EgressSpecs.java
98-
:language: java
99-
:lines: 18-
232+
.. tabs::
233+
234+
.. group-tab:: Java
235+
236+
.. literalinclude:: ../../src/main/java/org/apache/flink/statefun/docs/io/kafka/EgressSpecs.java
237+
:language: java
238+
:lines: 18-
239+
240+
.. group-tab:: Yaml
241+
242+
.. code-block:: yaml
243+
244+
version: "1.0"
245+
246+
module:
247+
meta:
248+
type: remote
249+
spec:
250+
egresses:
251+
- egress:
252+
meta:
253+
type: statefun.kafka.io/generic-egress
254+
id: example/output-messages
255+
spec:
256+
address: kafka-broker:9092
257+
deliverySemantic:
258+
type: exactly-once
259+
transactionTimeoutMillis: 100000
260+
properties:
261+
- foo.config: bar
100262
101263
Please refer to the Kafka `producer configuration <https://docs.confluent.io/current/installation/configuration/producer-configs.html>`_ documentation for the full list of available properties.
102264

103265
Kafka Egress and Fault Tolerance
104-
""""""""""""""""""""""""""""""""
266+
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
105267

106268
With fault tolerance enabled, the Kafka egress can provide exactly-once delivery guarantees.
107-
You can choose three different modes of operating based through the ``KafkaEgressBuilder``.
269+
You can choose three different modes of operation.
270+
271+
**None**
272+
273+
Nothing is guaranteed, produced records can be lost or duplicated.
274+
275+
.. tabs::
276+
277+
.. group-tab:: Java
278+
279+
.. code-block:: none
280+
281+
KafkaEgressBuilder#withNoProducerSemantics();
282+
283+
.. group-tab:: Yaml
284+
285+
.. code-block:: yaml
286+
287+
deliverySemantic:
288+
type: none
289+
290+
**At Least Once**
291+
292+
Stateful Functions will guarantee that no records will be lost but they can be duplicated.
293+
294+
.. tabs::
295+
296+
.. group-tab:: Java
297+
298+
.. code-block:: none
299+
300+
KafkaEgressBuilder#withAtLeastOnceProducerSemantics();
301+
302+
.. group-tab:: Yaml
303+
304+
.. code-block:: yaml
305+
306+
deliverySemantic:
307+
type: at-least-once
308+
309+
**Exactly Once**
310+
311+
Stateful Functions uses Kafka transactions to provide exactly-once semantics.
312+
313+
.. tabs::
314+
315+
.. group-tab:: Java
316+
317+
.. code-block:: none
318+
319+
KafkaEgressBuilder#withExactlyOnceProducerSemantics(Duration.minutes(15));
320+
321+
.. group-tab:: Yaml
322+
323+
.. code-block:: yaml
108324
109-
* ``KafkaEgressBuilder#withNoProducerSemantics``: Nothing is guaranteed. Produced records can be lost or duplicated.
110-
* ``KafkaEgressBuilder#withAtLeastOnceProducerSemantics``: Stateful Functions will guarantee that nor records will be lost but they can be duplicated.
111-
* ``KafkaEgressBuilder#withExactlyOnceProducerSemantics``: Stateful Functions uses Kafka transactions to provide exactly-once semantics.
325+
deliverySemantic:
326+
type: exactly-once
327+
transactionTimeoutMillis: 900000 # 15 min
112328
113329
Kafka Serializer
114-
""""""""""""""""
330+
^^^^^^^^^^^^^^^^
115331

116-
The Kafka egress needs to know how to turn Java objects into binary data.
332+
When using the Java api, the Kafka egress needs to know how to turn Java objects into binary data.
117333
The ``KafkaEgressSerializer`` allows users to specify such a schema.
118334
The ``ProducerRecord<byte[], byte[]> serialize(T out)`` method gets called for each message, allowing users to set a key, value, and other metadata.
119335

0 commit comments

Comments
 (0)