3737
3838from __future__ import absolute_import
3939
40- from apache_beam import ExternalTransform
41- from apache_beam import pvalue
42- from apache_beam .coders import BytesCoder
43- from apache_beam .coders import IterableCoder
44- from apache_beam .coders import TupleCoder
45- from apache_beam .coders .coders import LengthPrefixCoder
46- from apache_beam .portability .api .external_transforms_pb2 import ConfigValue
47- from apache_beam .portability .api .external_transforms_pb2 import ExternalConfigurationPayload
48- from apache_beam .transforms import ptransform
49-
50-
51- class ReadFromKafka (ptransform .PTransform ):
40+ import typing
41+
42+ from past .builtins import unicode
43+
44+ from apache_beam .transforms .external import ExternalTransform
45+ from apache_beam .transforms .external import NamedTupleBasedPayloadBuilder
46+
47+ ReadFromKafkaSchema = typing .NamedTuple (
48+ 'ReadFromKafkaSchema' ,
49+ [
50+ ('consumer_config' , typing .List [typing .Tuple [unicode , unicode ]]),
51+ ('topics' , typing .List [unicode ]),
52+ ('key_deserializer' , unicode ),
53+ ('value_deserializer' , unicode ),
54+ ]
55+ )
56+
57+
58+ class ReadFromKafka (ExternalTransform ):
5259 """
5360 An external PTransform which reads from Kafka and returns a KV pair for
5461 each item in the specified Kafka topics. If no Kafka Deserializer for
@@ -64,11 +71,13 @@ class ReadFromKafka(ptransform.PTransform):
6471 byte_array_deserializer = 'org.apache.kafka.common.serialization.' \
6572 'ByteArrayDeserializer'
6673
74+ URN = 'beam:external:java:kafka:read:v1'
75+
6776 def __init__ (self , consumer_config ,
6877 topics ,
6978 key_deserializer = byte_array_deserializer ,
7079 value_deserializer = byte_array_deserializer ,
71- expansion_service = 'localhost:8097' ):
80+ expansion_service = None ):
7281 """
7382 Initializes a read operation from Kafka.
7483
@@ -88,38 +97,32 @@ def __init__(self, consumer_config,
8897 serialization.ByteArrayDeserializer'.
8998 :param expansion_service: The address (host:port) of the ExpansionService.
9099 """
91- super (ReadFromKafka , self ).__init__ ()
92- self ._urn = 'beam:external:java:kafka:read:v1'
93- self .consumer_config = consumer_config
94- self .topics = topics
95- self .key_deserializer = key_deserializer
96- self .value_deserializer = value_deserializer
97- self .expansion_service = expansion_service
98-
99- def expand (self , pbegin ):
100- if not isinstance (pbegin , pvalue .PBegin ):
101- raise Exception ("ReadFromKafka must be a root transform" )
102-
103- args = {
104- 'consumer_config' :
105- _encode_map (self .consumer_config ),
106- 'topics' :
107- _encode_list (self .topics ),
108- 'key_deserializer' :
109- _encode_str (self .key_deserializer ),
110- 'value_deserializer' :
111- _encode_str (self .value_deserializer ),
112- }
113-
114- payload = ExternalConfigurationPayload (configuration = args )
115- return pbegin .apply (
116- ExternalTransform (
117- self ._urn ,
118- payload .SerializeToString (),
119- self .expansion_service ))
120-
121-
122- class WriteToKafka (ptransform .PTransform ):
100+ super (ReadFromKafka , self ).__init__ (
101+ self .URN ,
102+ NamedTupleBasedPayloadBuilder (
103+ ReadFromKafkaSchema (
104+ consumer_config = list (consumer_config .items ()),
105+ topics = topics ,
106+ key_deserializer = key_deserializer ,
107+ value_deserializer = value_deserializer ,
108+ )
109+ ),
110+ expansion_service
111+ )
112+
113+
114+ WriteToKafkaSchema = typing .NamedTuple (
115+ 'WriteToKafkaSchema' ,
116+ [
117+ ('producer_config' , typing .List [typing .Tuple [unicode , unicode ]]),
118+ ('topic' , unicode ),
119+ ('key_serializer' , unicode ),
120+ ('value_serializer' , unicode ),
121+ ]
122+ )
123+
124+
125+ class WriteToKafka (ExternalTransform ):
123126 """
124127 An external PTransform which writes KV data to a specified Kafka topic.
125128 If no Kafka Serializer for key/value is provided, then key/value are
@@ -132,11 +135,13 @@ class WriteToKafka(ptransform.PTransform):
132135 byte_array_serializer = 'org.apache.kafka.common.serialization.' \
133136 'ByteArraySerializer'
134137
138+ URN = 'beam:external:java:kafka:write:v1'
139+
135140 def __init__ (self , producer_config ,
136141 topic ,
137142 key_serializer = byte_array_serializer ,
138143 value_serializer = byte_array_serializer ,
139- expansion_service = 'localhost:8097' ):
144+ expansion_service = None ):
140145 """
141146 Initializes a write operation to Kafka.
142147
@@ -156,62 +161,15 @@ def __init__(self, producer_config,
156161 serialization.ByteArraySerializer'.
157162 :param expansion_service: The address (host:port) of the ExpansionService.
158163 """
159- super (WriteToKafka , self ).__init__ ()
160- self ._urn = 'beam:external:java:kafka:write:v1'
161- self .producer_config = producer_config
162- self .topic = topic
163- self .key_serializer = key_serializer
164- self .value_serializer = value_serializer
165- self .expansion_service = expansion_service
166-
167- def expand (self , pvalue ):
168- args = {
169- 'producer_config' :
170- _encode_map (self .producer_config ),
171- 'topic' :
172- _encode_str (self .topic ),
173- 'key_serializer' :
174- _encode_str (self .key_serializer ),
175- 'value_serializer' :
176- _encode_str (self .value_serializer ),
177- }
178-
179- payload = ExternalConfigurationPayload (configuration = args )
180- return pvalue .apply (
181- ExternalTransform (
182- self ._urn ,
183- payload .SerializeToString (),
184- self .expansion_service ))
185-
186-
187- def _encode_map (dict_obj ):
188- kv_list = [(key .encode ('utf-8' ), val .encode ('utf-8' ))
189- for key , val in dict_obj .items ()]
190- coder = IterableCoder (TupleCoder (
191- [LengthPrefixCoder (BytesCoder ()), LengthPrefixCoder (BytesCoder ())]))
192- coder_urns = ['beam:coder:iterable:v1' ,
193- 'beam:coder:kv:v1' ,
194- 'beam:coder:bytes:v1' ,
195- 'beam:coder:bytes:v1' ]
196- return ConfigValue (
197- coder_urn = coder_urns ,
198- payload = coder .encode (kv_list ))
199-
200-
201- def _encode_list (list_obj ):
202- encoded_list = [val .encode ('utf-8' ) for val in list_obj ]
203- coder = IterableCoder (LengthPrefixCoder (BytesCoder ()))
204- coder_urns = ['beam:coder:iterable:v1' ,
205- 'beam:coder:bytes:v1' ]
206- return ConfigValue (
207- coder_urn = coder_urns ,
208- payload = coder .encode (encoded_list ))
209-
210-
211- def _encode_str (str_obj ):
212- encoded_str = str_obj .encode ('utf-8' )
213- coder = LengthPrefixCoder (BytesCoder ())
214- coder_urns = ['beam:coder:bytes:v1' ]
215- return ConfigValue (
216- coder_urn = coder_urns ,
217- payload = coder .encode (encoded_str ))
164+ super (WriteToKafka , self ).__init__ (
165+ self .URN ,
166+ NamedTupleBasedPayloadBuilder (
167+ WriteToKafkaSchema (
168+ producer_config = list (producer_config .items ()),
169+ topic = topic ,
170+ key_serializer = key_serializer ,
171+ value_serializer = value_serializer ,
172+ )
173+ ),
174+ expansion_service
175+ )
0 commit comments