|
24 | 24 | Helper functions for Delta Lake |
25 | 25 | """ |
26 | 26 | import re |
27 | | -from typing import Iterator |
| 27 | +from typing import Iterator, Optional |
| 28 | + |
28 | 29 |
|
29 | 30 | from delta import DeltaTable |
30 | 31 | from pyspark.sql import SparkSession |
@@ -157,3 +158,66 @@ def get_table_info( |
157 | 158 | return [table_col for table_col in cols if table_col.name not in parts_names], parts, table_path |
158 | 159 |
|
159 | 160 | return cols, [], table_path |
| 161 | + |
| 162 | + |
| 163 | +def delta_compact( |
| 164 | + spark_session: SparkSession, |
| 165 | + path: str, |
| 166 | + retain_hours: float = 48, |
| 167 | + compact_from_predicate: Optional[str] = None, |
| 168 | + target_file_size_bytes: Optional[int] = None, |
| 169 | + vacuum_only: bool = True, |
| 170 | + refresh_cache=False, |
| 171 | +) -> None: |
| 172 | + """ |
| 173 | + Runs bin-packing optimization to reduce number of files/increase average file size in the table physical storage. |
| 174 | + Refreshes delta cache after opt/vacuum have been finished. |
| 175 | + https://docs.delta.io/latest/optimizations-oss.html#optimizations |
| 176 | +
|
| 177 | + :param spark_session: Spark session that will perform the operation. |
| 178 | + :param path: Path to delta table, filesystem or hive. |
| 179 | + :param retain_hours: Age of data to retain, defaults to 48 hours. |
| 180 | + :param compact_from_predicate: Optional predicate to select a subset of data to compact (sql string). |
| 181 | + :param target_file_size_bytes: Optional target file size in bytes. Defaults to system default (1gb for Delta 2.1) if not provided. |
| 182 | + :param vacuum_only: If set to True, will perform a vacuum operation w/o compaction. |
| 183 | + :param refresh_cache: Refreshes table cache for this spark session. |
| 184 | + :return: |
| 185 | + """ |
| 186 | + spark_session.conf.set("spark.databricks.delta.optimize.repartition.enabled", "true") |
| 187 | + |
| 188 | + table_to_compact = ( |
| 189 | + DeltaTable.forPath(sparkSession=spark_session, path=path) |
| 190 | + if "://" in path |
| 191 | + else DeltaTable.forName(sparkSession=spark_session, tableOrViewName=path) |
| 192 | + ) |
| 193 | + |
| 194 | + if not vacuum_only: |
| 195 | + if target_file_size_bytes: |
| 196 | + spark_session.conf.set("spark.databricks.delta.optimize.minFileSize", str(target_file_size_bytes)) |
| 197 | + spark_session.conf.set("spark.databricks.delta.optimize.maxFileSize", str(target_file_size_bytes)) |
| 198 | + |
| 199 | + if compact_from_predicate: |
| 200 | + table_to_compact.optimize().where(compact_from_predicate).executeCompaction() |
| 201 | + else: |
| 202 | + table_to_compact.optimize().executeCompaction() |
| 203 | + |
| 204 | + table_path = f"delta.`{path}`" if "://" in path else path |
| 205 | + current_interval = int( |
| 206 | + re.search( |
| 207 | + r"\b\d+\b", |
| 208 | + table_to_compact.detail().head().properties.get("delta.logRetentionDuration", "interval 168 hours"), |
| 209 | + ).group() |
| 210 | + ) |
| 211 | + |
| 212 | + if current_interval != round(retain_hours): |
| 213 | + spark_session.sql( |
| 214 | + f"ALTER table {table_path} SET TBLPROPERTIES ('delta.logRetentionDuration'='interval {round(retain_hours)} hours')" |
| 215 | + ) |
| 216 | + |
| 217 | + table_to_compact.vacuum(retentionHours=retain_hours) |
| 218 | + |
| 219 | + if refresh_cache: |
| 220 | + if "://" in path: |
| 221 | + spark_session.sql(f"refresh {path}") |
| 222 | + else: |
| 223 | + spark_session.sql(f"refresh table {path}") |
0 commit comments