github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/yaml/README.md (about) 1 <!-- 2 Licensed to the Apache Software Foundation (ASF) under one 3 or more contributor license agreements. See the NOTICE file 4 distributed with this work for additional information 5 regarding copyright ownership. The ASF licenses this file 6 to you under the Apache License, Version 2.0 (the 7 "License"); you may not use this file except in compliance 8 with the License. You may obtain a copy of the License at 9 10 http://www.apache.org/licenses/LICENSE-2.0 11 12 Unless required by applicable law or agreed to in writing, 13 software distributed under the License is distributed on an 14 "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 KIND, either express or implied. See the License for the 16 specific language governing permissions and limitations 17 under the License. 18 --> 19 20 # Beam YAML API 21 22 While Beam provides powerful APIs for authoring sophisticated data 23 processing pipelines, it often still has too high a barrier for 24 getting started and authoring simple pipelines. Even setting up the 25 environment, installing the dependencies, and setting up the project 26 can be an overwhelming amount of boilerplate for some (though 27 https://beam.apache.org/blog/beam-starter-projects/ has gone a long 28 way in making this easier). 29 30 Here we provide a simple declarative syntax for describing pipelines 31 that does not require coding experience or learning how to use an 32 SDK—any text editor will do. 33 Some installation may be required to actually *execute* a pipeline, but 34 we envision various services (such as Dataflow) to accept yaml pipelines 35 directly obviating the need for even that in the future. 36 We also anticipate the ability to generate code directly from these 37 higher-level yaml descriptions, should one want to graduate to a full 38 Beam SDK (and possibly the other direction as well as far as possible). 39 40 Though we intend this syntax to be easily authored (and read) directly by 41 humans, this may also prove a useful intermediate representation for 42 tools to use as well, either as output (e.g. a pipeline authoring GUI) 43 or consumption (e.g. a lineage analysis tool) and expect it to be more 44 easily manipulated and semantically meaningful than the Beam protos 45 themselves (which concern themselves more with execution). 46 47 It should be noted that everything here is still EXPERIMENTAL and subject 48 to change. Feedback is welcome at dev@apache.beam.org. 49 50 ## Example pipelines 51 52 Here is a simple pipeline that reads some data from csv files and 53 writes it out in json format. 54 55 ``` 56 pipeline: 57 transforms: 58 - type: ReadFromCsv 59 path: /path/to/input*.csv 60 - type: WriteToJson 61 path: /path/to/output.json 62 input: ReadFromCsv 63 ``` 64 65 We can also add a transformation 66 67 ``` 68 pipeline: 69 transforms: 70 - type: ReadFromCsv 71 path: /path/to/input*.csv 72 - type: PyFilter 73 keep: "lambda x: x.col3 > 100" 74 input: ReadFromCsv 75 - type: WriteToJson 76 path: /path/to/output.json 77 input: PyFilter 78 ``` 79 80 or two. 81 82 ``` 83 pipeline: 84 transforms: 85 - type: ReadFromCsv 86 path: /path/to/input*.csv 87 - type: PyFilter 88 keep: "lambda x: x.col3 > 100" 89 input: ReadFromCsv 90 - type: Sql 91 name: MySqlTransform 92 query: "select col1, count(*) as cnt from PCOLLECTION group by col1" 93 input: PyFilter 94 - type: WriteToJson 95 path: /path/to/output.json 96 input: MySqlTransform 97 ``` 98 99 If the pipeline is linear, we can let the inputs be implicit by designating 100 the pipeline as a `chain` type. 101 102 ``` 103 pipeline: 104 type: chain 105 106 transforms: 107 - type: ReadFromCsv 108 path: /path/to/input*.csv 109 - type: PyFilter 110 keep: "lambda x: x.col3 > 100" 111 - type: Sql 112 name: MySqlTransform 113 query: "select col1, count(*) as cnt from PCOLLECTION group by col1" 114 - type: WriteToJson 115 path: /path/to/output.json 116 ``` 117 118 As syntactic sugar, we can name the first and last transforms in our pipeline 119 as `source` and `sink`. 120 121 ``` 122 pipeline: 123 type: chain 124 125 source: 126 type: ReadFromCsv 127 path: /path/to/input*.csv 128 129 transforms: 130 - type: PyFilter 131 keep: "lambda x: x.col3 > 100" 132 133 - type: Sql 134 name: MySqlTransform 135 query: "select col1, count(*) as cnt from PCOLLECTION group by col1" 136 137 sink: 138 type: WriteToJson 139 path: /path/to/output.json 140 ``` 141 142 Arbitrary non-linear pipelines are supported as well, though in this case 143 inputs must be explicitly named. 144 Here we read two sources, join them, and write two outputs. 145 146 ``` 147 pipeline: 148 - type: ReadFromCsv 149 name: ReadLeft 150 path: /path/to/left*.csv 151 152 - type: ReadFromCsv 153 name: ReadRight 154 path: /path/to/right*.csv 155 156 - type: Sql 157 query: select left.col1, right.col2 from left join right using (col3) 158 input: 159 left: ReadLeft 160 right: ReadRight 161 162 - type: WriteToJson 163 name: WriteAll 164 input: Sql 165 path: /path/to/all.json 166 167 - type: PyFilter 168 name: FilterToBig 169 input: Sql 170 keep: "lambda x: x.col2 > 100" 171 172 - type: WriteToCsv 173 name: WriteBig 174 input: FilterToBig 175 path: /path/to/big.csv 176 ``` 177 178 One can, however, nest `chains` within a non-linear pipeline. 179 For example, here `ExtraProcessingForBigRows` is itself a "chain" transform 180 that has a single input and contains its own sink. 181 182 ``` 183 pipeline: 184 - type: ReadFromCsv 185 name: ReadLeft 186 path: /path/to/left*.csv 187 188 - type: ReadFromCsv 189 name: ReadRight 190 path: /path/to/right*.csv 191 192 - type: Sql 193 query: select left.col1, right.col2 from left join right using (col3) 194 input: 195 left: ReadLeft 196 right: ReadRight 197 198 - type: WriteToJson 199 name: WriteAll 200 input: Sql 201 path: /path/to/all.json 202 203 - type: chain 204 name: ExtraProcessingForBigRows 205 input: Sql 206 transforms: 207 - type: PyFilter 208 keep: "lambda x: x.col2 > 100" 209 - type: PyFilter 210 keep: "lambda x: len(x.col1) > 10" 211 - type: PyFilter 212 keep: "lambda x: x.col1 > 'z'" 213 sink: 214 type: WriteToCsv 215 path: /path/to/big.csv 216 ``` 217 218 ## Windowing 219 220 This API can be used to define both streaming and batch pipelines. 221 In order to meaningfully aggregate elements in a streaming pipeline, 222 some kind of windowing is typically required. Beam's 223 [windowing](https://beam.apache.org/documentation/programming-guide/#windowing) 224 and [triggering](https://beam.apache.org/documentation/programming-guide/#triggers) 225 can be be declared using the same WindowInto transform available in all other 226 SDKs. 227 228 ``` 229 pipeline: 230 type: chain 231 transforms: 232 - type: ReadFromPubSub 233 topic: myPubSubTopic 234 - type: WindowInto 235 windowing: 236 type: fixed 237 size: 60 238 - type: SomeAggregation 239 - type: WriteToPubSub 240 topic: anotherPubSubTopic 241 ``` 242 243 Rather than using an explicit `WindowInto` operation, one may instead tag a 244 transform itself with a specified windowing which will cause its inputs 245 (and hence the transform itself) to be applied with that windowing. 246 247 ``` 248 pipeline: 249 type: chain 250 transforms: 251 - type: ReadFromPubSub 252 topic: myPubSubTopic 253 - type: SomeAggregation 254 windowing: 255 type: sliding 256 size: 60 257 period: 10 258 - type: WriteToPubSub 259 topic: anotherPubSubTopic 260 ``` 261 262 Note that the `Sql` operation itself is often a from of aggregation, and 263 applying a windowing (or consuming an already windowed input) will cause all 264 grouping to be done per window. 265 266 ``` 267 pipeline: 268 type: chain 269 transforms: 270 - type: ReadFromPubSub 271 topic: myPubSubTopic 272 - type: Sql 273 query: "select col1, count(*) as c from PCOLLECTION" 274 windowing: 275 type: sessions 276 gap: 60 277 - type: WriteToPubSub 278 topic: anotherPubSubTopic 279 ``` 280 281 The specified windowing is applied to all inputs, in this case resulting in 282 a join per window. 283 284 ``` 285 pipeline: 286 - type: ReadFromPubSub 287 name: ReadLeft 288 topic: leftTopic 289 290 - type: ReadFromPubSub 291 name: ReadRight 292 topic: rightTopic 293 294 - type: Sql 295 query: select left.col1, right.col2 from left join right using (col3) 296 input: 297 left: ReadLeft 298 right: ReadRight 299 windowing: 300 type: fixed 301 size: 60 302 ``` 303 304 For a transform with no inputs, the specified windowing is instead applied to 305 its output(s). As per the Beam model, the windowing is then inherited by all 306 consuming operations. This is especially useful for root operations like Read. 307 308 ``` 309 pipeline: 310 type: chain 311 transforms: 312 - type: ReadFromPubSub 313 topic: myPubSubTopic 314 windowing: 315 type: fixed 316 size: 60 317 - type: Sql 318 query: "select col1, count(*) as c from PCOLLECTION" 319 - type: WriteToPubSub 320 topic: anotherPubSubTopic 321 ``` 322 323 One can also specify windowing at the top level of a pipeline (or composite), 324 which is a shorthand to simply applying this same windowing to all root 325 operations (that don't otherwise specify their own windowing), 326 and can be an effective way to apply it everywhere. 327 328 ``` 329 pipeline: 330 type: chain 331 transforms: 332 - type: ReadFromPubSub 333 topic: myPubSubTopic 334 - type: Sql 335 query: "select col1, count(*) as c from PCOLLECTION" 336 - type: WriteToPubSub 337 topic: anotherPubSubTopic 338 windowing: 339 type: fixed 340 size: 60 341 ``` 342 343 Note that all these windowing specifications are compatible with the `source` 344 and `sink` syntax as well 345 346 ``` 347 pipeline: 348 type: chain 349 350 source: 351 type: ReadFromPubSub 352 topic: myPubSubTopic 353 windowing: 354 type: fixed 355 size: 10 356 357 transforms: 358 - type: Sql 359 query: "select col1, count(*) as c from PCOLLECTION" 360 361 sink: 362 type: WriteToCsv 363 path: /path/to/output.json 364 windowing: 365 type: fixed 366 size: 300 367 ``` 368 369 370 ## Providers 371 372 Though we aim to offer a large suite of built-in transforms, it is inevitable 373 that people will want to be able to author their own. This is made possible 374 through the notion of Providers which leverage expansion services and 375 schema transforms. 376 377 For example, one could build a jar that vends a 378 [cross language transform](https://beam.apache.org/documentation/sdks/python-multi-language-pipelines/) 379 or [schema transform](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.html) 380 and then use it in a transform as follows 381 382 ``` 383 pipeline: 384 type: chain 385 source: 386 type: ReadFromCsv 387 path: /path/to/input*.csv 388 389 transforms: 390 - type: MyCustomTransform 391 args: 392 arg: whatever 393 394 sink: 395 type: WriteToJson 396 path: /path/to/output.json 397 398 providers: 399 - type: javaJar 400 jar: /path/or/url/to/myExpansionService.jar 401 transforms: 402 MyCustomTransform: "urn:registered:in:expansion:service" 403 ``` 404 405 Arbitrary Python transforms can be provided as well, using the syntax 406 407 ``` 408 providers: 409 - type: pythonPackage 410 packages: 411 - my_pypi_package>=version 412 - /path/to/local/package.zip 413 transforms: 414 MyCustomTransform: "pkg.subpkg.PTransformClassOrCallable" 415 ``` 416 417 ## Running pipelines 418 419 The Beam yaml parser is currently included as part of the Apache Beam Python SDK. 420 This can be installed (e.g. within a virtual environment) as 421 422 ``` 423 pip install apache_beam 424 ``` 425 426 In addition, several of the provided transforms (such as SQL) are implemented 427 in Java and their expansion will require a working Java interpeter. (The 428 requisite artifacts will be automatically downloaded from the apache maven 429 repositories, so no further installs will be required.) 430 Docker is also currently required for local execution of these 431 cross-language-requiring transforms, but not for submission to a non-local 432 runner such as Flink or Dataflow. 433 434 Once the prerequisites are installed, you can execute a pipeline defined 435 in a yaml file as 436 437 ``` 438 python -m apache_beam.yaml.main --pipeline_spec_file=/path/to/pipeline.yaml [other pipeline options such as the runner] 439 ```