-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathdebezium.py
More file actions
206 lines (164 loc) · 7.81 KB
/
debezium.py
File metadata and controls
206 lines (164 loc) · 7.81 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
""" Unbounded source transform for
`Debezium <href="https://debezium.io/"/>`_.
This transform is currently supported by Beam portable
Flink, Spark, and Dataflow v2 runners.
**Setup**
Transform provided in this module is cross-language transform
implemented in the Beam Java SDK. During the pipeline construction, Python SDK
will connect to a Java expansion service to expand this transform.
To facilitate this, a small amount of setup is needed before using this
transform in a Beam Python pipeline.
There are several ways to setup cross-language Debezium transform.
* Option 1: use the default expansion service
* Option 2: specify a custom expansion service
See below for details regarding each of these options.
*Option 1: Use the default expansion service*
This is the recommended and easiest setup option for using Python Debezium
transform. This option requires following pre-requisites
before running the Beam pipeline.
* Install Java runtime in the computer from where the pipeline is constructed
and make sure that 'java' command is available.
In this option, Python SDK will either download (for released Beam version) or
build (when running from a Beam Git clone) a expansion service jar and use
that to expand transforms. Currently Debezium transform use the
'beam-sdks-java-io-debezium-expansion-service' jar for this purpose.
*Option 2: specify a custom expansion service*
In this option, you startup your own expansion service and provide that as
a parameter when using the transform provided in this module.
This option requires following pre-requisites before running the Beam
pipeline.
* Startup your own expansion service.
* Update your pipeline to provide the expansion service address when
initiating Debezium transform provided in this module.
Flink Users can use the built-in Expansion Service of the Flink Runner's
Job Server. If you start Flink's Job Server, the expansion service will be
started on port 8097. For a different address, please set the
expansion_service parameter.
**More information**
For more information regarding cross-language transforms see:
- https://beam.apache.org/roadmap/portability/
For more information specific to Flink runner see:
- https://beam.apache.org/documentation/runners/flink/
"""
# pytype: skip-file
import json
from enum import Enum
from typing import List
from typing import NamedTuple
from typing import Optional
from apache_beam.transforms import Map
from apache_beam.transforms import PTransform
from apache_beam.transforms.external import BeamJarExpansionService
from apache_beam.transforms.external import ExternalTransform
from apache_beam.transforms.external import NamedTupleBasedPayloadBuilder
__all__ = ['ReadFromDebezium', 'DriverClassName']
def default_io_expansion_service():
return BeamJarExpansionService(
'sdks:java:io:debezium:expansion-service:shadowJar')
class DriverClassName(Enum):
MYSQL = 'MySQL'
POSTGRESQL = 'PostgreSQL'
ORACLE = 'Oracle'
DB2 = 'Db2'
ReadFromDebeziumSchema = NamedTuple(
'ReadFromDebeziumSchema',
[('connector_class', str), ('username', str), ('password', str),
('host', str), ('port', str), ('max_number_of_records', Optional[int]),
('connection_properties', List[str]),
('start_offset', Optional[List[str]]),
('offset_storage_path', Optional[str])])
class ReadFromDebezium(PTransform):
"""
An external PTransform which reads from Debezium and returns
a Dictionary for each item in the specified database
connection.
Experimental; no backwards compatibility guarantees.
"""
URN = 'beam:transform:org.apache.beam:debezium_read:v1'
def __init__(
self,
connector_class,
username,
password,
host,
port,
max_number_of_records=None,
connection_properties=None,
start_offset=None,
offset_storage_path=None,
expansion_service=None):
"""
Initializes a read operation from Debezium.
:param connector_class: name of the jdbc driver class
:param username: database username
:param password: database password
:param host: database host
:param port: database port
:param max_number_of_records: maximum number of records
to be fetched before stop.
:param connection_properties: properties of the debezium
connection passed as string
with format [propertyName=property;]*
:param start_offset: starting offset to resume the connector from
a previously seen position. Provided as a list
of "key=value" strings, where numeric values are
encoded as their decimal string representation.
Example for PostgreSQL::
start_offset=["lsn=28160840"]
Example for MySQL::
start_offset=["file=binlog.000001", "pos=156"]
Obtain the offset from the JSON output of a
previous pipeline run (the "metadata" field
contains connector-specific position info) or
via ``SourceRecord.sourceOffset()`` in a custom
Java SourceRecordMapper.
:param offset_storage_path: path to a file where the connector offset
is automatically saved after each checkpoint
and loaded on pipeline startup, allowing the
pipeline to resume from where it left off.
Supports any filesystem available to the
Beam runner (local, GCS, S3, etc.).
Example::
offset_storage_path=(
"gs://my-bucket/debezium/offset.json"
)
When set, takes precedence over
``start_offset``.
:param expansion_service: The address (host:port)
of the ExpansionService.
"""
self.params = ReadFromDebeziumSchema(
connector_class=connector_class.value,
username=username,
password=password,
host=host,
port=str(port),
max_number_of_records=max_number_of_records,
connection_properties=connection_properties,
start_offset=start_offset,
offset_storage_path=offset_storage_path)
self.expansion_service = expansion_service or default_io_expansion_service()
def expand(self, pbegin):
return (
pbegin | ExternalTransform(
self.URN,
NamedTupleBasedPayloadBuilder(self.params),
self.expansion_service,
).with_output_types(str)
| 'JsonToDict' >> Map(json.loads).with_output_types(dict))