github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/runners/dataflow/native_io/iobase.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 """Dataflow native sources and sinks. 19 20 For internal use only; no backwards-compatibility guarantees. 21 """ 22 23 # pytype: skip-file 24 25 import logging 26 from typing import TYPE_CHECKING 27 from typing import Optional 28 29 from apache_beam import pvalue 30 from apache_beam.io import iobase 31 from apache_beam.transforms import ptransform 32 from apache_beam.transforms.display import HasDisplayData 33 34 if TYPE_CHECKING: 35 from apache_beam import coders 36 37 _LOGGER = logging.getLogger(__name__) 38 39 40 def _dict_printable_fields(dict_object, skip_fields): 41 """Returns a list of strings for the interesting fields of a dict.""" 42 return [ 43 '%s=%r' % (name, value) for name, 44 value in dict_object.items() 45 # want to output value 0 but not None nor [] 46 if (value or value == 0) and name not in skip_fields 47 ] 48 49 50 _minor_fields = [ 51 'coder', 52 'key_coder', 53 'value_coder', 54 'config_bytes', 55 'elements', 56 'append_trailing_newlines', 57 'strip_trailing_newlines', 58 'compression_type' 59 ] 60 61 62 class NativeSource(iobase.SourceBase): 63 """A source implemented by Dataflow service. 64 65 This class is to be only inherited by sources natively implemented by Cloud 66 Dataflow service, hence should not be sub-classed by users. 67 68 This class is deprecated and should not be used to define new sources. 69 """ 70 coder = None # type: Optional[coders.Coder] 71 72 def reader(self): 73 """Returns a NativeSourceReader instance associated with this source.""" 74 raise NotImplementedError 75 76 def is_bounded(self): 77 return True 78 79 def __repr__(self): 80 return '<{name} {vals}>'.format( 81 name=self.__class__.__name__, 82 vals=', '.join(_dict_printable_fields(self.__dict__, _minor_fields))) 83 84 85 class NativeSourceReader(object): 86 """A reader for a source implemented by Dataflow service.""" 87 def __enter__(self): 88 """Opens everything necessary for a reader to function properly.""" 89 raise NotImplementedError 90 91 def __exit__(self, exception_type, exception_value, traceback): 92 """Cleans up after a reader executed.""" 93 raise NotImplementedError 94 95 def __iter__(self): 96 """Returns an iterator over all the records of the source.""" 97 raise NotImplementedError 98 99 @property 100 def returns_windowed_values(self): 101 """Returns whether this reader returns windowed values.""" 102 return False 103 104 def get_progress(self): 105 """Returns a representation of how far the reader has read. 106 107 Returns: 108 A SourceReaderProgress object that gives the current progress of the 109 reader. 110 """ 111 112 def request_dynamic_split(self, dynamic_split_request): 113 """Attempts to split the input in two parts. 114 115 The two parts are named the "primary" part and the "residual" part. The 116 current 'NativeSourceReader' keeps processing the primary part, while the 117 residual part will be processed elsewhere (e.g. perhaps on a different 118 worker). 119 120 The primary and residual parts, if concatenated, must represent the 121 same input as the current input of this 'NativeSourceReader' before this 122 call. 123 124 The boundary between the primary part and the residual part is 125 specified in a framework-specific way using 'DynamicSplitRequest' e.g., 126 if the framework supports the notion of positions, it might be a 127 position at which the input is asked to split itself (which is not 128 necessarily the same position at which it *will* split itself); it 129 might be an approximate fraction of input, or something else. 130 131 This function returns a 'DynamicSplitResult', which encodes, in a 132 framework-specific way, the information sufficient to construct a 133 description of the resulting primary and residual inputs. For example, it 134 might, again, be a position demarcating these parts, or it might be a pair 135 of fully-specified input descriptions, or something else. 136 137 After a successful call to 'request_dynamic_split()', subsequent calls 138 should be interpreted relative to the new primary. 139 140 Args: 141 dynamic_split_request: A 'DynamicSplitRequest' describing the split 142 request. 143 144 Returns: 145 'None' if the 'DynamicSplitRequest' cannot be honored (in that 146 case the input represented by this 'NativeSourceReader' stays the same), 147 or a 'DynamicSplitResult' describing how the input was split into a 148 primary and residual part. 149 """ 150 _LOGGER.debug( 151 'SourceReader %r does not support dynamic splitting. Ignoring dynamic ' 152 'split request: %r', 153 self, 154 dynamic_split_request) 155 156 157 class ReaderProgress(object): 158 """A representation of how far a NativeSourceReader has read.""" 159 def __init__( 160 self, 161 position=None, 162 percent_complete=None, 163 remaining_time=None, 164 consumed_split_points=None, 165 remaining_split_points=None): 166 167 self._position = position 168 169 if percent_complete is not None: 170 percent_complete = float(percent_complete) 171 if percent_complete < 0 or percent_complete > 1: 172 raise ValueError( 173 'The percent_complete argument was %f. Must be in range [0, 1].' % 174 percent_complete) 175 self._percent_complete = percent_complete 176 177 self._remaining_time = remaining_time 178 self._consumed_split_points = consumed_split_points 179 self._remaining_split_points = remaining_split_points 180 181 @property 182 def position(self): 183 """Returns progress, represented as a ReaderPosition object.""" 184 return self._position 185 186 @property 187 def percent_complete(self): 188 """Returns progress, represented as a percentage of total work. 189 190 Progress range from 0.0 (beginning, nothing complete) to 1.0 (end of the 191 work range, entire WorkItem complete). 192 193 Returns: 194 Progress represented as a percentage of total work. 195 """ 196 return self._percent_complete 197 198 @property 199 def remaining_time(self): 200 """Returns progress, represented as an estimated time remaining.""" 201 return self._remaining_time 202 203 @property 204 def consumed_split_points(self): 205 return self._consumed_split_points 206 207 @property 208 def remaining_split_points(self): 209 return self._remaining_split_points 210 211 212 class ReaderPosition(object): 213 """A representation of position in an iteration of a 'NativeSourceReader'.""" 214 def __init__( 215 self, 216 end=None, 217 key=None, 218 byte_offset=None, 219 record_index=None, 220 shuffle_position=None, 221 concat_position=None): 222 """Initializes ReaderPosition. 223 224 A ReaderPosition may get instantiated for one of these position types. Only 225 one of these should be specified. 226 227 Args: 228 end: position is past all other positions. For example, this may be used 229 to represent the end position of an unbounded range. 230 key: position is a string key. 231 byte_offset: position is a byte offset. 232 record_index: position is a record index 233 shuffle_position: position is a base64 encoded shuffle position. 234 concat_position: position is a 'ConcatPosition'. 235 """ 236 237 self.end = end 238 self.key = key 239 self.byte_offset = byte_offset 240 self.record_index = record_index 241 self.shuffle_position = shuffle_position 242 243 if concat_position is not None: 244 assert isinstance(concat_position, ConcatPosition) 245 self.concat_position = concat_position 246 247 248 class ConcatPosition(object): 249 """A position that encapsulate an inner position and an index. 250 251 This is used to represent the position of a source that encapsulate several 252 other sources. 253 """ 254 def __init__(self, index, position): 255 """Initializes ConcatPosition. 256 257 Args: 258 index: index of the source currently being read. 259 position: inner position within the source currently being read. 260 """ 261 262 if position is not None: 263 assert isinstance(position, ReaderPosition) 264 self.index = index 265 self.position = position 266 267 268 class DynamicSplitRequest(object): 269 """Specifies how 'NativeSourceReader.request_dynamic_split' should split. 270 """ 271 def __init__(self, progress): 272 assert isinstance(progress, ReaderProgress) 273 self.progress = progress 274 275 276 class DynamicSplitResult(object): 277 pass 278 279 280 class DynamicSplitResultWithPosition(DynamicSplitResult): 281 def __init__(self, stop_position): 282 assert isinstance(stop_position, ReaderPosition) 283 self.stop_position = stop_position 284 285 286 class NativeSink(HasDisplayData): 287 """A sink implemented by Dataflow service. 288 289 This class is to be only inherited by sinks natively implemented by Cloud 290 Dataflow service, hence should not be sub-classed by users. 291 """ 292 def writer(self): 293 """Returns a SinkWriter for this source.""" 294 raise NotImplementedError 295 296 def __repr__(self): 297 return '<{name} {vals}>'.format( 298 name=self.__class__.__name__, 299 vals=_dict_printable_fields(self.__dict__, _minor_fields)) 300 301 302 class NativeSinkWriter(object): 303 """A writer for a sink implemented by Dataflow service.""" 304 def __enter__(self): 305 """Opens everything necessary for a writer to function properly.""" 306 raise NotImplementedError 307 308 def __exit__(self, exception_type, exception_value, traceback): 309 """Cleans up after a writer executed.""" 310 raise NotImplementedError 311 312 @property 313 def takes_windowed_values(self): 314 """Returns whether this writer takes windowed values.""" 315 return False 316 317 def Write(self, o): # pylint: disable=invalid-name 318 """Writes a record to the sink associated with this writer.""" 319 raise NotImplementedError 320 321 322 class _NativeWrite(ptransform.PTransform): 323 """A PTransform for writing to a Dataflow native sink. 324 325 These are sinks that are implemented natively by the Dataflow service 326 and hence should not be updated by users. These sinks are processed 327 using a Dataflow native write transform. 328 329 Applying this transform results in a ``pvalue.PDone``. 330 """ 331 def __init__(self, sink): 332 """Initializes a Write transform. 333 334 Args: 335 sink: Sink to use for the write 336 """ 337 super().__init__() 338 self.sink = sink 339 340 def expand(self, pcoll): 341 self._check_pcollection(pcoll) 342 return pvalue.PDone(pcoll.pipeline)