github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/transforms/sql.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 """Package for SqlTransform and related classes.""" 19 20 # pytype: skip-file 21 22 import typing 23 24 from apache_beam.transforms.external import BeamJarExpansionService 25 from apache_beam.transforms.external import ExternalTransform 26 from apache_beam.transforms.external import NamedTupleBasedPayloadBuilder 27 28 __all__ = ['SqlTransform'] 29 30 SqlTransformSchema = typing.NamedTuple( 31 'SqlTransformSchema', [('query', str), ('dialect', typing.Optional[str])]) 32 33 34 class SqlTransform(ExternalTransform): 35 """A transform that can translate a SQL query into PTransforms. 36 37 Input PCollections must have a schema. Currently, there are two ways to define 38 a schema for a PCollection: 39 40 1) Register a `typing.NamedTuple` type to use RowCoder, and specify it as the 41 output type. For example:: 42 43 Purchase = typing.NamedTuple('Purchase', 44 [('item_name', unicode), ('price', float)]) 45 coders.registry.register_coder(Purchase, coders.RowCoder) 46 with Pipeline() as p: 47 purchases = (p | beam.io... 48 | beam.Map(..).with_output_types(Purchase)) 49 50 2) Produce `beam.Row` instances. Note this option will fail if Beam is unable 51 to infer data types for any of the fields. For example:: 52 53 with Pipeline() as p: 54 purchases = (p | beam.io... 55 | beam.Map(lambda x: beam.Row(item_name=unicode(..), 56 price=float(..)))) 57 58 Similarly, the output of SqlTransform is a PCollection with a schema. 59 The columns produced by the query can be accessed as attributes. For example:: 60 61 purchases | SqlTransform(\"\"\" 62 SELECT item_name, COUNT(*) AS `count` 63 FROM PCOLLECTION GROUP BY item_name\"\"\") 64 | beam.Map(lambda row: "We've sold %d %ss!" % (row.count, 65 row.item_name)) 66 67 Additional examples can be found in 68 `apache_beam.examples.wordcount_xlang_sql`, `apache_beam.examples.sql_taxi`, 69 and `apache_beam.transforms.sql_test`. 70 71 For more details about Beam SQL in general see the `Java transform 72 <https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/extensions/sql/SqlTransform.html>`_, 73 and the `documentation 74 <https://beam.apache.org/documentation/dsls/sql/overview/>`_. 75 """ 76 URN = 'beam:external:java:sql:v1' 77 78 def __init__(self, query, dialect=None, expansion_service=None): 79 """ 80 Creates a SqlTransform which will be expanded to Java's SqlTransform. 81 (See class docs). 82 :param query: The SQL query. 83 :param dialect: (optional) The dialect, e.g. use 'zetasql' for ZetaSQL. 84 :param expansion_service: (optional) The URL of the expansion service to use 85 """ 86 expansion_service = expansion_service or BeamJarExpansionService( 87 ':sdks:java:extensions:sql:expansion-service:shadowJar') 88 super().__init__( 89 self.URN, 90 NamedTupleBasedPayloadBuilder( 91 SqlTransformSchema(query=query, dialect=dialect)), 92 expansion_service=expansion_service)