deltacat 1.1.6__py3-none-any.whl → 1.1.7__py3-none-any.whl

This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
Files changed (28) hide show
  1. deltacat/__init__.py +1 -1
  2. deltacat/aws/constants.py +21 -4
  3. deltacat/aws/s3u.py +48 -21
  4. deltacat/compute/compactor/model/round_completion_info.py +4 -0
  5. deltacat/compute/compactor_v2/compaction_session.py +51 -25
  6. deltacat/compute/compactor_v2/constants.py +12 -0
  7. deltacat/compute/compactor_v2/model/compaction_session.py +21 -0
  8. deltacat/compute/compactor_v2/steps/hash_bucket.py +6 -0
  9. deltacat/compute/compactor_v2/steps/merge.py +6 -0
  10. deltacat/compute/compactor_v2/utils/task_options.py +4 -1
  11. deltacat/storage/interface.py +10 -3
  12. deltacat/tests/aws/test_s3u.py +84 -3
  13. deltacat/tests/catalog/test_default_catalog_impl.py +2 -0
  14. deltacat/tests/compute/compact_partition_test_cases.py +61 -0
  15. deltacat/tests/compute/compactor_v2/test_compaction_session.py +2 -0
  16. deltacat/tests/compute/test_compact_partition_incremental.py +89 -32
  17. deltacat/tests/compute/test_compact_partition_rebase_then_incremental.py +21 -26
  18. deltacat/tests/compute/test_util_create_table_deltas_repo.py +45 -2
  19. deltacat/tests/local_deltacat_storage/__init__.py +38 -19
  20. deltacat/tests/utils/ray_utils/__init__.py +0 -0
  21. deltacat/tests/utils/ray_utils/test_concurrency.py +50 -0
  22. deltacat/tests/utils/test_resources.py +28 -0
  23. deltacat/utils/resources.py +45 -0
  24. {deltacat-1.1.6.dist-info → deltacat-1.1.7.dist-info}/METADATA +1 -1
  25. {deltacat-1.1.6.dist-info → deltacat-1.1.7.dist-info}/RECORD +28 -25
  26. {deltacat-1.1.6.dist-info → deltacat-1.1.7.dist-info}/LICENSE +0 -0
  27. {deltacat-1.1.6.dist-info → deltacat-1.1.7.dist-info}/WHEEL +0 -0
  28. {deltacat-1.1.6.dist-info → deltacat-1.1.7.dist-info}/top_level.txt +0 -0
deltacat/__init__.py CHANGED
@@ -44,7 +44,7 @@ from deltacat.types.tables import TableWriteMode
44
44
 
45
45
  deltacat.logs.configure_deltacat_logger(logging.getLogger(__name__))
46
46
 
47
- __version__ = "1.1.6"
47
+ __version__ = "1.1.7"
48
48
 
49
49
 
50
50
  __all__ = [
deltacat/aws/constants.py CHANGED
@@ -1,10 +1,27 @@
1
- from typing import List, Set
1
+ import botocore
2
+ from typing import Set
2
3
 
3
4
  from deltacat.utils.common import env_integer, env_string
4
5
 
5
6
  DAFT_MAX_S3_CONNECTIONS_PER_FILE = env_integer("DAFT_MAX_S3_CONNECTIONS_PER_FILE", 8)
6
7
  BOTO_MAX_RETRIES = env_integer("BOTO_MAX_RETRIES", 5)
7
- TIMEOUT_ERROR_CODES: List[str] = ["ReadTimeoutError", "ConnectTimeoutError"]
8
- RETRYABLE_PUT_OBJECT_ERROR_CODES: Set[str] = {"Throttling", "SlowDown"}
8
+ BOTO_TIMEOUT_ERROR_CODES: Set[str] = {"ReadTimeoutError", "ConnectTimeoutError"}
9
+ BOTO_THROTTLING_ERROR_CODES: Set[str] = {"Throttling", "SlowDown"}
10
+ RETRYABLE_TRANSIENT_ERRORS = (
11
+ OSError,
12
+ botocore.exceptions.ConnectionError,
13
+ botocore.exceptions.HTTPClientError,
14
+ botocore.exceptions.NoCredentialsError,
15
+ botocore.exceptions.ConnectTimeoutError,
16
+ botocore.exceptions.ReadTimeoutError,
17
+ )
9
18
  AWS_REGION = env_string("AWS_REGION", "us-east-1")
10
- RETRY_STOP_AFTER_DELAY = env_integer("RETRY_STOP_AFTER_DELAY", 10 * 60)
19
+ UPLOAD_DOWNLOAD_RETRY_STOP_AFTER_DELAY = env_integer(
20
+ "UPLOAD_DOWNLOAD_RETRY_STOP_AFTER_DELAY", 10 * 60
21
+ )
22
+ UPLOAD_SLICED_TABLE_RETRY_STOP_AFTER_DELAY = env_integer(
23
+ "UPLOAD_SLICED_TABLE_RETRY_STOP_AFTER_DELAY", 30 * 60
24
+ )
25
+ DOWNLOAD_MANIFEST_ENTRY_RETRY_STOP_AFTER_DELAY = env_integer(
26
+ "DOWNLOAD_MANIFEST_ENTRY_RETRY_STOP_AFTER_DELAY", 30 * 60
27
+ )
deltacat/aws/s3u.py CHANGED
@@ -6,8 +6,12 @@ from uuid import uuid4
6
6
  from botocore.config import Config
7
7
  from deltacat.aws.constants import (
8
8
  BOTO_MAX_RETRIES,
9
- RETRY_STOP_AFTER_DELAY,
10
- RETRYABLE_PUT_OBJECT_ERROR_CODES,
9
+ UPLOAD_DOWNLOAD_RETRY_STOP_AFTER_DELAY,
10
+ BOTO_THROTTLING_ERROR_CODES,
11
+ RETRYABLE_TRANSIENT_ERRORS,
12
+ BOTO_TIMEOUT_ERROR_CODES,
13
+ UPLOAD_SLICED_TABLE_RETRY_STOP_AFTER_DELAY,
14
+ DOWNLOAD_MANIFEST_ENTRY_RETRY_STOP_AFTER_DELAY,
11
15
  )
12
16
 
13
17
  import pyarrow as pa
@@ -15,7 +19,7 @@ import ray
15
19
  import s3fs
16
20
  from boto3.resources.base import ServiceResource
17
21
  from botocore.client import BaseClient
18
- from botocore.exceptions import ClientError, NoCredentialsError
22
+ from botocore.exceptions import ClientError
19
23
  from ray.data.block import Block, BlockAccessor, BlockMetadata
20
24
  from ray.data.datasource import BlockWritePathProvider
21
25
  from ray.types import ObjectRef
@@ -29,9 +33,6 @@ from tenacity import (
29
33
  from deltacat.utils.ray_utils.concurrency import invoke_parallel
30
34
  import deltacat.aws.clients as aws_utils
31
35
  from deltacat import logs
32
- from deltacat.aws.constants import (
33
- TIMEOUT_ERROR_CODES,
34
- )
35
36
  from deltacat.exceptions import NonRetryableError, RetryableError
36
37
  from deltacat.storage import (
37
38
  DistributedDataset,
@@ -257,10 +258,21 @@ def read_file(
257
258
  )
258
259
  return table
259
260
  except ClientError as e:
260
- if e.response["Error"]["Code"] in TIMEOUT_ERROR_CODES:
261
+ if (
262
+ e.response["Error"]["Code"]
263
+ in BOTO_TIMEOUT_ERROR_CODES | BOTO_THROTTLING_ERROR_CODES
264
+ ):
261
265
  # Timeout error not caught by botocore
262
- raise RetryableError(f"Retry table download from: {s3_url}") from e
263
- raise NonRetryableError(f"Failed table download from: {s3_url}") from e
266
+ raise RetryableError(
267
+ f"Retry table download from: {s3_url} after receiving {type(e).__name__}"
268
+ ) from e
269
+ raise NonRetryableError(
270
+ f"Failed table download from: {s3_url} after receiving {type(e).__name__}"
271
+ ) from e
272
+ except RETRYABLE_TRANSIENT_ERRORS as e:
273
+ raise RetryableError(
274
+ f"Retry upload for: {s3_url} after receiving {type(e).__name__}"
275
+ ) from e
264
276
  except BaseException as e:
265
277
  logger.warn(
266
278
  f"Read has failed for {s3_url} and content_type={content_type} "
@@ -285,7 +297,7 @@ def upload_sliced_table(
285
297
  # @retry decorator can't be pickled by Ray, so wrap upload in Retrying
286
298
  retrying = Retrying(
287
299
  wait=wait_random_exponential(multiplier=1, max=60),
288
- stop=stop_after_delay(30 * 60),
300
+ stop=stop_after_delay(UPLOAD_SLICED_TABLE_RETRY_STOP_AFTER_DELAY),
289
301
  retry=retry_if_exception_type(RetryableError),
290
302
  )
291
303
 
@@ -366,8 +378,23 @@ def upload_table(
366
378
  except ClientError as e:
367
379
  if e.response["Error"]["Code"] == "NoSuchKey":
368
380
  # s3fs may swallow S3 errors - we were probably throttled
369
- raise RetryableError(f"Retry table upload to: {s3_url}") from e
370
- raise NonRetryableError(f"Failed table upload to: {s3_url}") from e
381
+ raise RetryableError(
382
+ f"Retry table download from: {s3_url} after receiving {type(e).__name__}"
383
+ ) from e
384
+ if (
385
+ e.response["Error"]["Code"]
386
+ in BOTO_TIMEOUT_ERROR_CODES | BOTO_THROTTLING_ERROR_CODES
387
+ ):
388
+ raise RetryableError(
389
+ f"Retry table download from: {s3_url} after receiving {type(e).__name__}"
390
+ ) from e
391
+ raise NonRetryableError(
392
+ f"Failed table upload to: {s3_url} after receiving {type(e).__name__}"
393
+ ) from e
394
+ except RETRYABLE_TRANSIENT_ERRORS as e:
395
+ raise RetryableError(
396
+ f"Retry upload for: {s3_url} after receiving {type(e).__name__}"
397
+ ) from e
371
398
  except BaseException as e:
372
399
  logger.warn(
373
400
  f"Upload has failed for {s3_url} and content_type={content_type}. Error: {e}",
@@ -415,7 +442,7 @@ def download_manifest_entry(
415
442
  # @retry decorator can't be pickled by Ray, so wrap download in Retrying
416
443
  retrying = Retrying(
417
444
  wait=wait_random_exponential(multiplier=1, max=60),
418
- stop=stop_after_delay(30 * 60),
445
+ stop=stop_after_delay(DOWNLOAD_MANIFEST_ENTRY_RETRY_STOP_AFTER_DELAY),
419
446
  retry=retry_if_not_exception_type(NonRetryableError),
420
447
  )
421
448
  table = retrying(
@@ -511,7 +538,7 @@ def upload(s3_url: str, body, **s3_client_kwargs) -> Dict[str, Any]:
511
538
  s3 = s3_client_cache(None, **s3_client_kwargs)
512
539
  retrying = Retrying(
513
540
  wait=wait_random_exponential(multiplier=1, max=15),
514
- stop=stop_after_delay(RETRY_STOP_AFTER_DELAY),
541
+ stop=stop_after_delay(UPLOAD_DOWNLOAD_RETRY_STOP_AFTER_DELAY),
515
542
  retry=retry_if_exception_type(RetryableError),
516
543
  )
517
544
  return retrying(
@@ -531,18 +558,18 @@ def _put_object(
531
558
  Body=body, Bucket=bucket, Key=key, **s3_put_object_kwargs
532
559
  )
533
560
  except ClientError as e:
534
- if e.response["Error"]["Code"] in RETRYABLE_PUT_OBJECT_ERROR_CODES:
561
+ if e.response["Error"]["Code"] in BOTO_THROTTLING_ERROR_CODES:
535
562
  raise RetryableError(
536
563
  f"Retry upload for: {bucket}/{key} after receiving {e.response['Error']['Code']}"
537
564
  ) from e
538
565
  raise NonRetryableError(f"Failed table upload to: {bucket}/{key}") from e
539
- except NoCredentialsError as e:
566
+ except RETRYABLE_TRANSIENT_ERRORS as e:
540
567
  raise RetryableError(
541
- f"Failed to fetch credentials when putting object into: {bucket}/{key}"
568
+ f"Retry upload for: {bucket}/{key} after receiving {type(e).__name__}"
542
569
  ) from e
543
570
  except BaseException as e:
544
571
  logger.error(
545
- f"Upload has failed for {bucket}/{key}. Error: {e}",
572
+ f"Upload has failed for {bucket}/{key}. Error: {type(e).__name__}",
546
573
  exc_info=True,
547
574
  )
548
575
  raise NonRetryableError(f"Failed table upload to: {bucket}/{key}") from e
@@ -556,7 +583,7 @@ def download(
556
583
  s3 = s3_client_cache(None, **s3_client_kwargs)
557
584
  retrying = Retrying(
558
585
  wait=wait_random_exponential(multiplier=1, max=15),
559
- stop=stop_after_delay(RETRY_STOP_AFTER_DELAY),
586
+ stop=stop_after_delay(UPLOAD_DOWNLOAD_RETRY_STOP_AFTER_DELAY),
560
587
  retry=retry_if_exception_type(RetryableError),
561
588
  )
562
589
  return retrying(
@@ -581,9 +608,9 @@ def _get_object(s3_client, bucket: str, key: str, fail_if_not_found: bool = True
581
608
  f"Failed get object from: {bucket}/{key}"
582
609
  ) from e
583
610
  logger.info(f"file not found: {bucket}/{key}")
584
- except NoCredentialsError as e:
611
+ except RETRYABLE_TRANSIENT_ERRORS as e:
585
612
  raise RetryableError(
586
- f"Failed to fetch credentials when getting object from: {bucket}/{key}"
613
+ f"Retry get object: {bucket}/{key} after receiving {type(e).__name__}"
587
614
  ) from e
588
615
 
589
616
  return None
@@ -128,3 +128,7 @@ class RoundCompletionInfo(dict):
128
128
  @property
129
129
  def input_average_record_size_bytes(self) -> Optional[float]:
130
130
  return self.get("inputAverageRecordSizeBytes")
131
+
132
+ @staticmethod
133
+ def get_audit_bucket_name_and_key(compaction_audit_url: str) -> Tuple[str, str]:
134
+ return compaction_audit_url.replace("s3://", "").split("/", 1)
@@ -24,6 +24,9 @@ from deltacat.compute.compactor import (
24
24
  )
25
25
  from deltacat.compute.compactor_v2.model.merge_result import MergeResult
26
26
  from deltacat.compute.compactor_v2.model.hash_bucket_result import HashBucketResult
27
+ from deltacat.compute.compactor_v2.model.compaction_session import (
28
+ ExecutionCompactionResult,
29
+ )
27
30
  from deltacat.compute.compactor.model.materialize_result import MaterializeResult
28
31
  from deltacat.compute.compactor_v2.utils.merge import (
29
32
  generate_local_merge_input,
@@ -41,8 +44,11 @@ from deltacat.compute.compactor_v2.deletes.utils import prepare_deletes
41
44
  from deltacat.storage import (
42
45
  Delta,
43
46
  DeltaLocator,
47
+ DeltaType,
44
48
  Manifest,
45
49
  Partition,
50
+ Stream,
51
+ StreamLocator,
46
52
  )
47
53
  from deltacat.compute.compactor.model.compact_partition_params import (
48
54
  CompactPartitionParams,
@@ -57,7 +63,7 @@ from deltacat.compute.compactor_v2.utils import io
57
63
  from deltacat.compute.compactor.utils import round_completion_file as rcf
58
64
  from deltacat.utils.metrics import metrics
59
65
 
60
- from typing import List, Optional, Tuple
66
+ from typing import List, Optional
61
67
  from collections import defaultdict
62
68
  from deltacat.compute.compactor.model.compaction_session_audit_info import (
63
69
  CompactionSessionAuditInfo,
@@ -81,35 +87,52 @@ logger = logs.configure_deltacat_logger(logging.getLogger(__name__))
81
87
 
82
88
  @metrics
83
89
  def compact_partition(params: CompactPartitionParams, **kwargs) -> Optional[str]:
84
-
85
90
  assert (
86
91
  params.hash_bucket_count is not None and params.hash_bucket_count >= 1
87
92
  ), "hash_bucket_count is a required arg for compactor v2"
88
93
 
89
94
  with memray.Tracker(
90
- f"compaction_partition.bin"
95
+ "compaction_partition.bin"
91
96
  ) if params.enable_profiler else nullcontext():
92
- (new_partition, new_rci, new_rcf_partition_locator,) = _execute_compaction(
97
+ execute_compaction_result: ExecutionCompactionResult = _execute_compaction(
93
98
  params,
94
99
  **kwargs,
95
100
  )
96
-
101
+ compaction_session_type: str = (
102
+ "INPLACE"
103
+ if execute_compaction_result.is_inplace_compacted
104
+ else "NON-INPLACE"
105
+ )
97
106
  logger.info(
98
107
  f"Partition-{params.source_partition_locator} -> "
99
- f"Compaction session data processing completed"
108
+ f"{compaction_session_type} Compaction session data processing completed"
100
109
  )
101
- round_completion_file_s3_url = None
102
- if new_partition:
103
- logger.info(f"Committing compacted partition to: {new_partition.locator}")
104
- partition: Partition = params.deltacat_storage.commit_partition(
105
- new_partition, **params.deltacat_storage_kwargs
110
+ round_completion_file_s3_url: Optional[str] = None
111
+ if execute_compaction_result.new_compacted_partition:
112
+ previous_partition: Optional[Partition] = None
113
+ if execute_compaction_result.is_inplace_compacted:
114
+ previous_partition: Optional[
115
+ Partition
116
+ ] = params.deltacat_storage.get_partition(
117
+ params.source_partition_locator.stream_locator,
118
+ params.source_partition_locator.partition_values,
119
+ **params.deltacat_storage_kwargs,
120
+ )
121
+ # NOTE: Retrieving the previous partition again as the partition_id may have changed by the time commit_partition is called.
122
+ logger.info(
123
+ f"Committing compacted partition to: {execute_compaction_result.new_compacted_partition.locator} "
124
+ f"using previous partition: {previous_partition.locator if previous_partition else None}"
106
125
  )
107
- logger.info(f"Committed compacted partition: {partition}")
108
-
126
+ commited_partition: Partition = params.deltacat_storage.commit_partition(
127
+ execute_compaction_result.new_compacted_partition,
128
+ previous_partition,
129
+ **params.deltacat_storage_kwargs,
130
+ )
131
+ logger.info(f"Committed compacted partition: {commited_partition}")
109
132
  round_completion_file_s3_url = rcf.write_round_completion_file(
110
133
  params.compaction_artifact_s3_bucket,
111
- new_rcf_partition_locator,
112
- new_rci,
134
+ execute_compaction_result.new_round_completion_file_partition_locator,
135
+ execute_compaction_result.new_round_completion_info,
113
136
  **params.s3_client_kwargs,
114
137
  )
115
138
  else:
@@ -123,7 +146,7 @@ def compact_partition(params: CompactPartitionParams, **kwargs) -> Optional[str]
123
146
 
124
147
  def _execute_compaction(
125
148
  params: CompactPartitionParams, **kwargs
126
- ) -> Tuple[Optional[Partition], Optional[RoundCompletionInfo], Optional[str]]:
149
+ ) -> ExecutionCompactionResult:
127
150
 
128
151
  rcf_source_partition_locator = (
129
152
  params.rebase_source_partition_locator or params.source_partition_locator
@@ -142,7 +165,7 @@ def _execute_compaction(
142
165
 
143
166
  compaction_start = time.monotonic()
144
167
 
145
- task_max_parallelism = params.task_max_parallelism
168
+ task_max_parallelism: int = params.task_max_parallelism
146
169
 
147
170
  if params.pg_config:
148
171
  logger.info(
@@ -205,7 +228,7 @@ def _execute_compaction(
205
228
  )
206
229
  if not input_deltas:
207
230
  logger.info("No input deltas found to compact.")
208
- return None, None, None
231
+ return ExecutionCompactionResult(None, None, None, False)
209
232
 
210
233
  delete_strategy: Optional[DeleteStrategy] = None
211
234
  delete_file_envelopes: Optional[List[DeleteFileEnvelope]] = None
@@ -217,7 +240,7 @@ def _execute_compaction(
217
240
  for delete_file_envelope in delete_file_envelopes:
218
241
  delete_file_size_bytes += delete_file_envelope.table_size_bytes
219
242
  logger.info(
220
- f" Input deltas contain DELETE-type deltas. Total delete file size={delete_file_size_bytes}."
243
+ f" Input deltas contain {DeltaType.DELETE}-type deltas. Total delete file size={delete_file_size_bytes}."
221
244
  f" Total length of delete file envelopes={len(delete_file_envelopes)}"
222
245
  )
223
246
  uniform_deltas: List[DeltaAnnotated] = io.create_uniform_input_deltas(
@@ -247,14 +270,16 @@ def _execute_compaction(
247
270
  )
248
271
 
249
272
  # create a new stream for this round
250
- compacted_stream_locator = params.destination_partition_locator.stream_locator
251
- compacted_stream = params.deltacat_storage.get_stream(
273
+ compacted_stream_locator: Optional[
274
+ StreamLocator
275
+ ] = params.destination_partition_locator.stream_locator
276
+ compacted_stream: Stream = params.deltacat_storage.get_stream(
252
277
  compacted_stream_locator.namespace,
253
278
  compacted_stream_locator.table_name,
254
279
  compacted_stream_locator.table_version,
255
280
  **params.deltacat_storage_kwargs,
256
281
  )
257
- compacted_partition = params.deltacat_storage.stage_partition(
282
+ compacted_partition: Partition = params.deltacat_storage.stage_partition(
258
283
  compacted_stream,
259
284
  params.destination_partition_locator.partition_values,
260
285
  **params.deltacat_storage_kwargs,
@@ -532,7 +557,7 @@ def _execute_compaction(
532
557
 
533
558
  # Note: An appropriate last stream position must be set
534
559
  # to avoid correctness issue.
535
- merged_delta = Delta.merge_deltas(
560
+ merged_delta: Delta = Delta.merge_deltas(
536
561
  deltas,
537
562
  stream_position=params.last_stream_position_to_compact,
538
563
  )
@@ -545,7 +570,7 @@ def _execute_compaction(
545
570
  )
546
571
  logger.info(record_info_msg)
547
572
 
548
- compacted_delta = params.deltacat_storage.commit_delta(
573
+ compacted_delta: Delta = params.deltacat_storage.commit_delta(
549
574
  merged_delta,
550
575
  properties=kwargs.get("properties", {}),
551
576
  **params.deltacat_storage_kwargs,
@@ -653,8 +678,9 @@ def _execute_compaction(
653
678
  f"and rcf source partition_id of {rcf_source_partition_locator.partition_id}."
654
679
  )
655
680
  rcf_source_partition_locator = compacted_partition.locator
656
- return (
681
+ return ExecutionCompactionResult(
657
682
  compacted_partition,
658
683
  new_round_completion_info,
659
684
  rcf_source_partition_locator,
685
+ is_inplace_compacted,
660
686
  )
@@ -1,3 +1,5 @@
1
+ from deltacat.utils.common import env_integer
2
+
1
3
  TOTAL_BYTES_IN_SHA1_HASH = 20
2
4
 
3
5
  PK_DELIMITER = "L6kl7u5f"
@@ -41,6 +43,16 @@ DROP_DUPLICATES = True
41
43
  # size in metadata to pyarrow table size.
42
44
  PARQUET_TO_PYARROW_INFLATION = 4
43
45
 
46
+ # A merge task will fail after this timeout
47
+ # The default is currently double the observed maximum.
48
+ # This timeout depends on total data processed per task.
49
+ MERGE_TASK_TIMEOUT_IN_SECONDS = env_integer("MERGE_TASK_TIMEOUT_IN_SECONDS", 25 * 60)
50
+
51
+ # A hash bucket task will fail after this timeout
52
+ HASH_BUCKET_TASK_TIMEOUT_IN_SECONDS = env_integer(
53
+ "HASH_BUCKET_TASK_TIMEOUT_IN_SECONDS", 25 * 60
54
+ )
55
+
44
56
  # Metric Names
45
57
  # Time taken for a hash bucket task
46
58
  HASH_BUCKET_TIME_IN_SECONDS = "hash_bucket_time"
@@ -0,0 +1,21 @@
1
+ from dataclasses import dataclass, fields
2
+
3
+ from deltacat.storage import (
4
+ Partition,
5
+ PartitionLocator,
6
+ )
7
+ from deltacat.compute.compactor import (
8
+ RoundCompletionInfo,
9
+ )
10
+ from typing import Optional
11
+
12
+
13
+ @dataclass(frozen=True)
14
+ class ExecutionCompactionResult:
15
+ new_compacted_partition: Optional[Partition]
16
+ new_round_completion_info: Optional[RoundCompletionInfo]
17
+ new_round_completion_file_partition_locator: Optional[PartitionLocator]
18
+ is_inplace_compacted: bool
19
+
20
+ def __iter__(self):
21
+ return (getattr(self, field.name) for field in fields(self))
@@ -29,12 +29,14 @@ from deltacat.utils.metrics import emit_timer_metrics, failure_metric, success_m
29
29
  from deltacat.utils.resources import (
30
30
  get_current_process_peak_memory_usage_in_bytes,
31
31
  ProcessUtilizationOverTimeRange,
32
+ timeout,
32
33
  )
33
34
  from deltacat.constants import BYTES_PER_GIBIBYTE
34
35
  from deltacat.compute.compactor_v2.constants import (
35
36
  HASH_BUCKET_TIME_IN_SECONDS,
36
37
  HASH_BUCKET_FAILURE_COUNT,
37
38
  HASH_BUCKET_SUCCESS_COUNT,
39
+ HASH_BUCKET_TASK_TIMEOUT_IN_SECONDS,
38
40
  )
39
41
 
40
42
  if importlib.util.find_spec("memray"):
@@ -96,8 +98,12 @@ def _group_file_records_by_pk_hash_bucket(
96
98
  return hb_to_delta_file_envelopes, total_record_count, total_size_bytes
97
99
 
98
100
 
101
+ # TODO: use timeout parameter in ray.remote
102
+ # https://github.com/ray-project/ray/issues/18916
103
+ # Note: order of decorators is important
99
104
  @success_metric(name=HASH_BUCKET_SUCCESS_COUNT)
100
105
  @failure_metric(name=HASH_BUCKET_FAILURE_COUNT)
106
+ @timeout(HASH_BUCKET_TASK_TIMEOUT_IN_SECONDS)
101
107
  def _timed_hash_bucket(input: HashBucketInput):
102
108
  task_id = get_current_ray_task_id()
103
109
  worker_id = get_current_ray_worker_id()
@@ -28,6 +28,7 @@ from deltacat.utils.metrics import emit_timer_metrics, failure_metric, success_m
28
28
  from deltacat.utils.resources import (
29
29
  get_current_process_peak_memory_usage_in_bytes,
30
30
  ProcessUtilizationOverTimeRange,
31
+ timeout,
31
32
  )
32
33
  from deltacat.compute.compactor_v2.utils.primary_key_index import (
33
34
  generate_pk_hash_column,
@@ -46,6 +47,7 @@ from deltacat.compute.compactor_v2.constants import (
46
47
  MERGE_TIME_IN_SECONDS,
47
48
  MERGE_SUCCESS_COUNT,
48
49
  MERGE_FAILURE_COUNT,
50
+ MERGE_TASK_TIMEOUT_IN_SECONDS,
49
51
  )
50
52
 
51
53
 
@@ -484,8 +486,12 @@ def _copy_manifests_from_hash_bucketing(
484
486
  return materialized_results
485
487
 
486
488
 
489
+ # TODO: use timeout parameter in ray.remote
490
+ # https://github.com/ray-project/ray/issues/18916
491
+ # Note: order of decorators is important
487
492
  @success_metric(name=MERGE_SUCCESS_COUNT)
488
493
  @failure_metric(name=MERGE_FAILURE_COUNT)
494
+ @timeout(MERGE_TASK_TIMEOUT_IN_SECONDS)
489
495
  def _timed_merge(input: MergeInput) -> MergeResult:
490
496
  task_id = get_current_ray_task_id()
491
497
  worker_id = get_current_ray_worker_id()
@@ -1,5 +1,6 @@
1
1
  import botocore
2
2
  import logging
3
+ import tenacity
3
4
  from typing import Dict, Optional, List, Tuple, Any
4
5
  from deltacat import logs
5
6
  from deltacat.compute.compactor_v2.model.merge_file_group import (
@@ -66,7 +67,8 @@ def get_task_options(
66
67
 
67
68
  # NOTE: With DEFAULT scheduling strategy in Ray 2.20.0, autoscaler does
68
69
  # not spin up enough nodes fast and hence we see only approximately
69
- # 20 tasks get scheduled out of 100 tasks in queue.
70
+ # 20 tasks get scheduled out of 100 tasks in queue. Hence, we use SPREAD
71
+ # which is also ideal for merge and hash bucket tasks.
70
72
  # https://docs.ray.io/en/latest/ray-core/scheduling/index.html
71
73
  task_opts = {"num_cpus": cpu, "memory": memory, "scheduling_strategy": "SPREAD"}
72
74
 
@@ -83,6 +85,7 @@ def get_task_options(
83
85
  ConnectionError,
84
86
  TimeoutError,
85
87
  DaftTransientError,
88
+ tenacity.RetryError,
86
89
  ]
87
90
 
88
91
  return task_opts
@@ -414,11 +414,18 @@ def stage_partition(
414
414
  raise NotImplementedError("stage_partition not implemented")
415
415
 
416
416
 
417
- def commit_partition(partition: Partition, *args, **kwargs) -> Partition:
417
+ def commit_partition(
418
+ partition: Partition,
419
+ previous_partition: Optional[Partition] = None,
420
+ *args,
421
+ **kwargs
422
+ ) -> Partition:
418
423
  """
419
424
  Commits the given partition to its associated table version stream,
420
- replacing any previous partition registered for the same stream and
421
- partition values. Returns the registered partition. If the partition's
425
+ replacing any previous partition (i.e., "partition being replaced") registered for the same stream and
426
+ partition values.
427
+ If the previous_partition is passed as an argument, the specified previous_partition will be the partition being replaced, otherwise it will be retrieved.
428
+ Returns the registered partition. If the partition's
422
429
  previous delta stream position is specified, then the commit will
423
430
  be rejected if it does not match the actual previous stream position of
424
431
  the partition being replaced. If the partition's previous partition ID is
@@ -1,4 +1,8 @@
1
1
  import unittest
2
+
3
+ import botocore
4
+
5
+ from deltacat.aws.constants import RETRYABLE_TRANSIENT_ERRORS
2
6
  from deltacat.aws.s3u import UuidBlockWritePathProvider, CapturedBlockWritePaths
3
7
 
4
8
 
@@ -9,7 +13,13 @@ from unittest.mock import patch
9
13
  import boto3
10
14
  import pytest
11
15
  from boto3.resources.base import ServiceResource
12
- from botocore.exceptions import ClientError, NoCredentialsError
16
+ from botocore.exceptions import (
17
+ ClientError,
18
+ NoCredentialsError,
19
+ ReadTimeoutError,
20
+ ConnectTimeoutError,
21
+ HTTPClientError,
22
+ )
13
23
  from deltacat.exceptions import NonRetryableError
14
24
  from moto import mock_s3
15
25
  from tenacity import RetryError
@@ -64,7 +74,7 @@ class TestDownloadUpload(unittest.TestCase):
64
74
  assert downloaded_file["ResponseMetadata"]["HTTPStatusCode"] == 200
65
75
  assert downloaded_body == body
66
76
 
67
- @patch("deltacat.aws.s3u.RETRY_STOP_AFTER_DELAY", 1)
77
+ @patch("deltacat.aws.s3u.UPLOAD_DOWNLOAD_RETRY_STOP_AFTER_DELAY", 1)
68
78
  @patch("deltacat.aws.s3u.s3_client_cache")
69
79
  def test_upload_throttled(self, mock_s3_client_cache):
70
80
  uri = f"s3://{self.TEST_S3_BUCKET_NAME}/{self.TEST_S3_KEY}"
@@ -87,6 +97,51 @@ class TestDownloadUpload(unittest.TestCase):
87
97
 
88
98
  assert mock_s3.put_object.call_count > 3
89
99
 
100
+ @patch("deltacat.aws.s3u.UPLOAD_SLICED_TABLE_RETRY_STOP_AFTER_DELAY", 1)
101
+ @patch("deltacat.aws.s3u.ManifestEntry")
102
+ @patch("deltacat.aws.s3u._get_metadata")
103
+ @patch("deltacat.aws.s3u.CapturedBlockWritePaths")
104
+ def test_upload_sliced_table_retry(
105
+ self,
106
+ mock_captured_block_write_paths,
107
+ mock_get_metadata,
108
+ mock_manifest_entry,
109
+ ):
110
+ mock_manifest_entry.from_s3_obj_url.side_effect = OSError(
111
+ "Please reduce your request rate.."
112
+ )
113
+ mock_get_metadata.return_value = [mock.MagicMock()]
114
+ cbwp = CapturedBlockWritePaths()
115
+ cbwp._write_paths = ["s3_write_path"]
116
+ cbwp._block_refs = [mock.MagicMock()]
117
+ mock_captured_block_write_paths.return_value = cbwp
118
+ with pytest.raises(RetryError):
119
+ s3u.upload_sliced_table(
120
+ mock.MagicMock(),
121
+ "s3-prefix",
122
+ mock.MagicMock(),
123
+ mock.MagicMock(),
124
+ mock.MagicMock(),
125
+ mock.MagicMock(),
126
+ )
127
+
128
+ @patch("deltacat.aws.s3u.UPLOAD_DOWNLOAD_RETRY_STOP_AFTER_DELAY", 1)
129
+ @patch("deltacat.aws.s3u.s3_client_cache")
130
+ def test_upload_transient_error_retry(self, mock_s3_client_cache):
131
+ uri = f"s3://{self.TEST_S3_BUCKET_NAME}/{self.TEST_S3_KEY}"
132
+ body = "test-body"
133
+ transient_errors = [*RETRYABLE_TRANSIENT_ERRORS]
134
+ mock_s3_client_cache.return_value = mock_s3 = mock.MagicMock()
135
+
136
+ while transient_errors:
137
+ err_cls = transient_errors.pop()
138
+ err_obj = self._populate_error_by_type(err_cls)
139
+ mock_s3.put_object.side_effect = err_obj
140
+ with pytest.raises(RetryError):
141
+ s3u.upload(uri, body)
142
+
143
+ assert mock_s3.put_object.call_count > len(RETRYABLE_TRANSIENT_ERRORS)
144
+
90
145
  @patch("deltacat.aws.s3u.s3_client_cache")
91
146
  def test_upload_unexpected_error_code(self, mock_s3_client_cache):
92
147
  uri = f"s3://{self.TEST_S3_BUCKET_NAME}/{self.TEST_S3_KEY}"
@@ -100,7 +155,7 @@ class TestDownloadUpload(unittest.TestCase):
100
155
  assert file is None
101
156
  assert mock_s3.put_object.call_count == 1
102
157
 
103
- @patch("deltacat.aws.s3u.RETRY_STOP_AFTER_DELAY", 1)
158
+ @patch("deltacat.aws.s3u.UPLOAD_DOWNLOAD_RETRY_STOP_AFTER_DELAY", 1)
104
159
  @patch("deltacat.aws.s3u.s3_client_cache")
105
160
  def test_download_throttled(self, mock_s3_client_cache):
106
161
  uri = f"s3://{self.TEST_S3_BUCKET_NAME}/{self.TEST_S3_KEY}"
@@ -122,3 +177,29 @@ class TestDownloadUpload(unittest.TestCase):
122
177
 
123
178
  file = s3u.download(uri, fail_if_not_found=False)
124
179
  assert file is None
180
+
181
+ @patch("deltacat.aws.s3u.UPLOAD_DOWNLOAD_RETRY_STOP_AFTER_DELAY", 1)
182
+ @patch("deltacat.aws.s3u.s3_client_cache")
183
+ def test_download_transient_error_retry(self, mock_s3_client_cache):
184
+ uri = f"s3://{self.TEST_S3_BUCKET_NAME}/{self.TEST_S3_KEY}"
185
+ transient_errors = [*RETRYABLE_TRANSIENT_ERRORS]
186
+ mock_s3_client_cache.return_value = mock_s3 = mock.MagicMock()
187
+
188
+ while transient_errors:
189
+ err_cls = transient_errors.pop()
190
+ err_obj = self._populate_error_by_type(err_cls)
191
+ mock_s3.get_object.side_effect = err_obj
192
+ with pytest.raises(RetryError):
193
+ s3u.download(uri)
194
+
195
+ assert mock_s3.get_object.call_count > len(RETRYABLE_TRANSIENT_ERRORS)
196
+
197
+ @staticmethod
198
+ def _populate_error_by_type(err_cls):
199
+ if err_cls in (ReadTimeoutError, ConnectTimeoutError):
200
+ err_obj = err_cls(endpoint_url="127.0.0.1")
201
+ elif err_cls in (HTTPClientError, botocore.exceptions.ConnectionError):
202
+ err_obj = err_cls(endpoint_url="127.0.0.1", error=Exception)
203
+ else:
204
+ err_obj = err_cls()
205
+ return err_obj
@@ -36,6 +36,8 @@ class TestReadTable(unittest.TestCase):
36
36
  @classmethod
37
37
  def doClassCleanups(cls) -> None:
38
38
  os.remove(cls.DB_FILE_PATH)
39
+ ray.shutdown()
40
+ super().tearDownClass()
39
41
 
40
42
  def test_daft_distributed_read_sanity(self):
41
43
  # setup