github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/io/debezium.py (about) 1 # 2 # Licensed to the Apache Software Foundation (ASF) under one or more 3 # contributor license agreements. See the NOTICE file distributed with 4 # this work for additional information regarding copyright ownership. 5 # The ASF licenses this file to You under the Apache License, Version 2.0 6 # (the "License"); you may not use this file except in compliance with 7 # the License. You may obtain a copy of the License at 8 # 9 # http://www.apache.org/licenses/LICENSE-2.0 10 # 11 # Unless required by applicable law or agreed to in writing, software 12 # distributed under the License is distributed on an "AS IS" BASIS, 13 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 # See the License for the specific language governing permissions and 15 # limitations under the License. 16 # 17 18 """ Unbounded source transform for 19 `Debezium <href="https://debezium.io/"/>`_. 20 21 This transform is currently supported by Beam portable 22 Flink, Spark, and Dataflow v2 runners. 23 24 **Setup** 25 26 Transform provided in this module is cross-language transform 27 implemented in the Beam Java SDK. During the pipeline construction, Python SDK 28 will connect to a Java expansion service to expand this transform. 29 To facilitate this, a small amount of setup is needed before using this 30 transform in a Beam Python pipeline. 31 32 There are several ways to setup cross-language Debezium transform. 33 34 * Option 1: use the default expansion service 35 * Option 2: specify a custom expansion service 36 37 See below for details regarding each of these options. 38 39 *Option 1: Use the default expansion service* 40 41 This is the recommended and easiest setup option for using Python Debezium 42 transform. This option requires following pre-requisites 43 before running the Beam pipeline. 44 45 * Install Java runtime in the computer from where the pipeline is constructed 46 and make sure that 'java' command is available. 47 48 In this option, Python SDK will either download (for released Beam version) or 49 build (when running from a Beam Git clone) a expansion service jar and use 50 that to expand transforms. Currently Debezium transform use the 51 'beam-sdks-java-io-debezium-expansion-service' jar for this purpose. 52 53 *Option 2: specify a custom expansion service* 54 55 In this option, you startup your own expansion service and provide that as 56 a parameter when using the transform provided in this module. 57 58 This option requires following pre-requisites before running the Beam 59 pipeline. 60 61 * Startup your own expansion service. 62 * Update your pipeline to provide the expansion service address when 63 initiating Debezium transform provided in this module. 64 65 Flink Users can use the built-in Expansion Service of the Flink Runner's 66 Job Server. If you start Flink's Job Server, the expansion service will be 67 started on port 8097. For a different address, please set the 68 expansion_service parameter. 69 70 **More information** 71 72 For more information regarding cross-language transforms see: 73 - https://beam.apache.org/roadmap/portability/ 74 75 For more information specific to Flink runner see: 76 - https://beam.apache.org/documentation/runners/flink/ 77 """ 78 79 # pytype: skip-file 80 81 import json 82 from enum import Enum 83 from typing import List 84 from typing import NamedTuple 85 from typing import Optional 86 87 from apache_beam.transforms import DoFn 88 from apache_beam.transforms import ParDo 89 from apache_beam.transforms import PTransform 90 from apache_beam.transforms.external import BeamJarExpansionService 91 from apache_beam.transforms.external import ExternalTransform 92 from apache_beam.transforms.external import NamedTupleBasedPayloadBuilder 93 94 __all__ = ['ReadFromDebezium', 'DriverClassName'] 95 96 97 def default_io_expansion_service(): 98 return BeamJarExpansionService( 99 'sdks:java:io:debezium:expansion-service:shadowJar') 100 101 102 class DriverClassName(Enum): 103 MYSQL = 'MySQL' 104 POSTGRESQL = 'PostgreSQL' 105 ORACLE = 'Oracle' 106 DB2 = 'Db2' 107 108 109 ReadFromDebeziumSchema = NamedTuple( 110 'ReadFromDebeziumSchema', 111 [('connector_class', str), ('username', str), ('password', str), 112 ('host', str), ('port', str), ('max_number_of_records', Optional[int]), 113 ('connection_properties', List[str])]) 114 115 116 class _JsonStringToDictionaries(DoFn): 117 """ A DoFn that consumes a JSON string and yields a python dictionary """ 118 def process(self, json_string): 119 obj = json.loads(json_string) 120 yield obj 121 122 123 class ReadFromDebezium(PTransform): 124 """ 125 An external PTransform which reads from Debezium and returns 126 a Dictionary for each item in the specified database 127 connection. 128 129 Experimental; no backwards compatibility guarantees. 130 """ 131 URN = 'beam:transform:org.apache.beam:debezium_read:v1' 132 133 def __init__( 134 self, 135 connector_class, 136 username, 137 password, 138 host, 139 port, 140 max_number_of_records=None, 141 connection_properties=None, 142 expansion_service=None): 143 """ 144 Initializes a read operation from Debezium. 145 146 :param connector_class: name of the jdbc driver class 147 :param username: database username 148 :param password: database password 149 :param host: database host 150 :param port: database port 151 :param max_number_of_records: maximum number of records 152 to be fetched before stop. 153 :param connection_properties: properties of the debezium 154 connection passed as string 155 with format 156 [propertyName=property;]* 157 :param expansion_service: The address (host:port) 158 of the ExpansionService. 159 """ 160 self.params = ReadFromDebeziumSchema( 161 connector_class=connector_class.value, 162 username=username, 163 password=password, 164 host=host, 165 port=port, 166 max_number_of_records=max_number_of_records, 167 connection_properties=connection_properties) 168 self.expansion_service = expansion_service or default_io_expansion_service() 169 170 def expand(self, pbegin): 171 return ( 172 pbegin | ExternalTransform( 173 self.URN, 174 NamedTupleBasedPayloadBuilder(self.params), 175 self.expansion_service, 176 ) | ParDo(_JsonStringToDictionaries()))