Стратегия Incremental
Bases: HWMStrategy
Incremental strategy for :ref:db-reader/:ref:file-downloader.
Used for fetching only new rows/files from a source
by filtering items not covered by the previous :ref:HWM value.
For :ref:db-reader:
First incremental run is just the same as :obj:SnapshotStrategy <onetl.strategy.snapshot_strategy.SnapshotStrategy>:
.. code:: sql
SELECT id, data FROM mydata;
Then the max value of ``id`` column (e.g. ``1000``) will be saved as ``HWM`` to :ref:`HWM Store <hwm>`.
Next incremental run will read only new data from the source:
.. code:: sql
SELECT id, data FROM mydata WHERE id > 1000; -- hwm value
Pay attention to resulting dataframe **does not include** row with ``id=1000`` because it has been read before.
.. warning::
If code inside the context manager raised an exception, like:
.. code:: python
with IncrementalStrategy():
df = reader.run() # something went wrong here
writer.run(df) # or here
# or here...
When DBReader will **NOT** update HWM in HWM Store.
This allows to resume reading process from the *last successful run*.
For :ref:file-downloader:
Behavior depends on hwm type.
.. tabs::
.. tab:: FileListHWM
First incremental run is just the same as :obj:`SnapshotStrategy <onetl.strategy.snapshot_strategy.SnapshotStrategy>` -
all files are downloaded:
.. code:: bash
$ hdfs dfs -ls /path
/path/my/file1
/path/my/file2
.. code:: python
DownloadResult(
...,
successful={
LocalFile("/downloaded/file1"),
LocalFile("/downloaded/file2"),
},
)
Then the list of original file paths is saved as ``FileListHWM`` object into :ref:`HWM Store <hwm>`:
.. code:: python
FileListHWM(
...,
entity="/path",
value=[
"/path/my/file1",
"/path/my/file2",
],
)
Next incremental run will download only new files which were added to the source since previous run:
.. code:: bash
$ hdfs dfs -ls /path
/path/my/file1
/path/my/file2
/path/my/file3
.. code:: python
# only files which are not covered by FileListHWM
DownloadResult(
...,
successful={
LocalFile("/downloaded/file3"),
},
)
Value of ``FileListHWM`` will be updated and saved to :ref:`HWM Store <hwm>`:
.. code:: python
FileListHWM(
...,
directory="/path",
value=[
"/path/my/file1",
"/path/my/file2",
"/path/my/file3",
],
)
.. tab:: FileModifiedTimeHWM
First incremental run is just the same as :obj:`SnapshotStrategy <onetl.strategy.snapshot_strategy.SnapshotStrategy>` -
all files are downloaded:
.. code:: bash
$ hdfs dfs -ls /path
/path/my/file1
/path/my/file2
.. code:: python
DownloadResult(
...,
successful={
LocalFile("/downloaded/file1"),
LocalFile("/downloaded/file2"),
},
)
Then the maximum modified time of original files is saved as ``FileModifiedTimeHWM`` object into :ref:`HWM Store <hwm>`:
.. code:: python
FileModifiedTimeHWM(
...,
directory="/path",
value=datetime.datetime(2025, 1, 1, 11, 22, 33, 456789, tzinfo=timezone.utc),
)
Next incremental run will download only files from the source which were modified or created since previous run:
.. code:: bash
$ hdfs dfs -ls /path
/path/my/file1
/path/my/file2
/path/my/file3
.. code:: python
# only files which are not covered by FileModifiedTimeHWM
DownloadResult(
...,
successful={
LocalFile("/downloaded/file3"),
},
)
Value of ``FileModifiedTimeHWM`` will be updated and and saved to :ref:`HWM Store <hwm>`:
.. code:: python
FileModifiedTimeHWM(
...,
directory="/path",
value=datetime.datetime(2025, 1, 1, 22, 33, 44, 567890, tzinfo=timezone.utc),
)
.. warning::
FileDownload updates HWM in HWM Store at the end of ``.run()`` call,
**NOT** while exiting strategy context. This is because:
* FileDownloader does not raise exceptions if some file cannot be downloaded.
* FileDownloader creates files on local filesystem, and file content may differ for different :obj:`modes <onetl.file.file_downloader.file_downloader.FileDownloader.Options.mode>`.
* It can remove files from the source if :obj:`delete_source <onetl.file.file_downloader.file_downloader.FileDownloader.Options.delete_source>` is set to ``True``.
.. versionadded:: 0.1.0
Parameters
offset : Any, default: None
If passed, the offset value will be used to read rows which appeared in the source after the previous read.
For example, previous incremental run returned rows:
.. code::
898
899
900
1000
Current HWM value is 1000.
But since then few more rows appeared in the source:
.. code::
898
899
900
901 # new
902 # new
...
999 # new
1000
and you need to read them too.
So you can set ``offset=100``, so a next incremental run will generate SQL query like:
.. code:: sql
SELECT id, data FROM public.mydata WHERE id > 900;
-- 900 = 1000 - 100 = hwm - offset
and return rows since 901 (**not** 900), **including** 1000 which was already captured by HWM.
.. warning::
This can lead to reading duplicated values from the table.
You probably need additional deduplication step to handle them
.. warning::
Cannot be used with :ref:`file-downloader`
.. note::
``offset`` value will be subtracted from the HWM, so it should have a proper type.
For example, for ``TIMESTAMP`` column ``offset`` type should be :obj:`datetime.timedelta`, not :obj:`int`
Examples
.. tabs::
.. tab:: Incremental run with :ref:`db-reader`
.. code:: python
from onetl.db import DBReader, DBWriter
from onetl.strategy import IncrementalStrategy
reader = DBReader(
connection=postgres,
source="public.mydata",
columns=["id", "data"],
hwm=DBReader.AutoDetectHWM(name="some_hwm_name", expression="id"),
)
writer = DBWriter(connection=hive, target="db.newtable")
with IncrementalStrategy():
df = reader.run()
writer.run(df)
.. code:: sql
-- previous HWM value was 1000
-- DBReader will generate query like:
SELECT id, data
FROM public.mydata
WHERE id > 1000; --- from HWM (EXCLUDING first row)
.. tab:: Incremental run with :ref:`db-reader` and ``IncrementalStrategy(offset=...)``
.. code:: python
from onetl.db import DBReader, DBWriter
from onetl.strategy import IncrementalStrategy
reader = DBReader(
connection=postgres,
source="public.mydata",
columns=["id", "data"],
hwm=DBReader.AutoDetectHWM(name="some_hwm_name", expression="id"),
)
writer = DBWriter(connection=hive, target="db.newtable")
with IncrementalStrategy(offset=100):
df = reader.run()
writer.run(df)
.. code:: sql
-- previous HWM value was 1000
-- DBReader will generate query like:
SELECT id, data
FROM public.mydata
WHERE id > 900; -- from HWM-offset (EXCLUDING first row)
``offset`` and ``hwm.expression`` can be a date or datetime, not only integer:
.. code:: python
from onetl.db import DBReader, DBWriter
from datetime import timedelta
reader = DBReader(
connection=postgres,
source="public.mydata",
columns=["business_dt", "data"],
hwm=DBReader.AutoDetectHWM(name="some_hwm_name", expression="business_dt"),
)
writer = DBWriter(connection=hive, target="db.newtable")
with IncrementalStrategy(offset=timedelta(days=1)):
df = reader.run()
writer.run(df)
.. code:: sql
-- previous HWM value was '2021-01-10'
-- DBReader will generate query like:
SELECT business_dt, data
FROM public.mydata
WHERE business_dt > CAST('2021-01-09' AS DATE); -- from HWM-offset (EXCLUDING first row)
.. code-tab:: py Incremental run with :ref:`db-reader` and :ref:`kafka`
from onetl.db import DBReader, DBWriter
from onetl.strategy import IncrementalStrategy
reader = DBReader(
connection=kafka,
source="topic_name",
hwm=DBReader.AutoDetectHWM(name="some_hwm_name", expression="offset"),
)
writer = DBWriter(connection=hive, target="db.newtable")
with IncrementalStrategy():
df = reader.run()
# current run will fetch only messages which were added since previous run
.. code-tab:: py Incremental run with :ref:`file-downloader` and ``hwm=FileListHWM(...)``
from onetl.file import FileDownloader
from onetl.strategy import SnapshotStrategy
from etl_entities.hwm import FileListHWM
downloader = FileDownloader(
connection=sftp,
source_path="/remote",
local_path="/local",
hwm=FileListHWM( # mandatory for IncrementalStrategy
name="my_unique_hwm_name",
),
)
with IncrementalStrategy():
df = downloader.run()
# current run will download only files which were added since previous run
.. code-tab:: py Incremental run with :ref:`file-downloader` and ``hwm=FileModifiedTimeHWM(...)``
from onetl.file import FileDownloader
from onetl.strategy import SnapshotStrategy
from etl_entities.hwm import FileModifiedTimeHWM
downloader = FileDownloader(
connection=sftp,
source_path="/remote",
local_path="/local",
hwm=FileModifiedTimeHWM( # mandatory for IncrementalStrategy
name="my_unique_hwm_name",
),
)
with IncrementalStrategy():
df = downloader.run()
# current run will download only files which were modified/created since previous run
Source code in onetl/strategy/incremental_strategy.py
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 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 | |