|
| 1 | +# Architecture of a Delta Lake table |
| 2 | + |
| 3 | +A Delta table consists of Parquet files that contain data and a transaction log that stores metadata about the transactions. |
| 4 | + |
| 5 | + |
| 6 | + |
| 7 | +Let's create a Delta table, perform some operations, and inspect the files that are created. |
| 8 | + |
| 9 | +## Delta Lake transaction examples |
| 10 | + |
| 11 | +Start by creating a pandas DataFrame and writing it out to a Delta table. |
| 12 | + |
| 13 | +```python |
| 14 | +import pandas as pd |
| 15 | +from deltalake import DeltaTable, write_deltalake |
| 16 | + |
| 17 | +df = pd.DataFrame({"num": [1, 2, 3], "letter": ["a", "b", "c"]}) |
| 18 | +write_deltalake("tmp/some-table", df) |
| 19 | +``` |
| 20 | + |
| 21 | +Now inspect the files created in storage: |
| 22 | + |
| 23 | +``` |
| 24 | +tmp/some-table |
| 25 | +├── 0-62dffa23-bbe1-4496-8fb5-bff6724dc677-0.parquet |
| 26 | +└── _delta_log |
| 27 | + └── 00000000000000000000.json |
| 28 | +``` |
| 29 | + |
| 30 | +The Parquet file stores the data that was written. The `_delta_log` directory stores metadata about the transactions. Let's inspect the `_delta_log/00000000000000000000.json` file. |
| 31 | + |
| 32 | +```json |
| 33 | +{ |
| 34 | + "protocol": { |
| 35 | + "minReaderVersion": 1, |
| 36 | + "minWriterVersion": 1 |
| 37 | + } |
| 38 | +} |
| 39 | +{ |
| 40 | + "metaData": { |
| 41 | + "id": "b96ea1a2-1830-4da2-8827-5334cc6104ed", |
| 42 | + "name": null, |
| 43 | + "description": null, |
| 44 | + "format": { |
| 45 | + "provider": "parquet", |
| 46 | + "options": {} |
| 47 | + }, |
| 48 | + "schemaString": "{\"type\":\"struct\",\"fields\":[{\"name\":\"num\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"letter\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}", |
| 49 | + "partitionColumns": [], |
| 50 | + "createdTime": 1701740315599, |
| 51 | + "configuration": {} |
| 52 | + } |
| 53 | +} |
| 54 | +{ |
| 55 | + "add": { |
| 56 | + "path": "0-62dffa23-bbe1-4496-8fb5-bff6724dc677-0.parquet", |
| 57 | + "size": 2208, |
| 58 | + "partitionValues": {}, |
| 59 | + "modificationTime": 1701740315597, |
| 60 | + "dataChange": true, |
| 61 | + "stats": "{\"numRecords\": 3, \"minValues\": {\"num\": 1, \"letter\": \"a\"}, \"maxValues\": {\"num\": 3, \"letter\": \"c\"}, \"nullCount\": {\"num\": 0, \"letter\": 0}}" |
| 62 | + } |
| 63 | +} |
| 64 | +{ |
| 65 | + "commitInfo": { |
| 66 | + "timestamp": 1701740315602, |
| 67 | + "operation": "CREATE TABLE", |
| 68 | + "operationParameters": { |
| 69 | + "location": "file:///Users/matthew.powers/Documents/code/delta/delta-examples/notebooks/python-deltalake/tmp/some-table", |
| 70 | + "metadata": "{\"configuration\":{},\"created_time\":1701740315599,\"description\":null,\"format\":{\"options\":{},\"provider\":\"parquet\"},\"id\":\"b96ea1a2-1830-4da2-8827-5334cc6104ed\",\"name\":null,\"partition_columns\":[],\"schema\":{\"fields\":[{\"metadata\":{},\"name\":\"num\",\"nullable\":true,\"type\":\"long\"},{\"metadata\":{},\"name\":\"letter\",\"nullable\":true,\"type\":\"string\"}],\"type\":\"struct\"}}", |
| 71 | + "protocol": "{\"minReaderVersion\":1,\"minWriterVersion\":1}", |
| 72 | + "mode": "ErrorIfExists" |
| 73 | + }, |
| 74 | + "clientVersion": "delta-rs.0.17.0" |
| 75 | + } |
| 76 | +} |
| 77 | +``` |
| 78 | + |
| 79 | +The tranasction log file contains the following information: |
| 80 | + |
| 81 | +* the files added to the Delta table |
| 82 | +* schema of the files |
| 83 | +* column level metadata including the min/max value for each file |
| 84 | + |
| 85 | +Create another pandas DataFrame and append it to the Delta table to see how this transaction is recorded. |
| 86 | + |
| 87 | +```python |
| 88 | +df = pd.DataFrame({"num": [8, 9], "letter": ["dd", "ee"]}) |
| 89 | +write_deltalake(f"{cwd}/tmp/delta-table", df, mode="append") |
| 90 | +``` |
| 91 | + |
| 92 | +Here are the files in storage: |
| 93 | + |
| 94 | +``` |
| 95 | +tmp/some-table |
| 96 | +├── 0-62dffa23-bbe1-4496-8fb5-bff6724dc677-0.parquet |
| 97 | +├── 1-57abb6fb-2249-43ba-a7be-cf09bcc230de-0.parquet |
| 98 | +└── _delta_log |
| 99 | + ├── 00000000000000000000.json |
| 100 | + └── 00000000000000000001.json |
| 101 | +``` |
| 102 | + |
| 103 | +Here are the contents of the `_delta_log/00000000000000000001.json` file: |
| 104 | + |
| 105 | +```json |
| 106 | +{ |
| 107 | + "add": { |
| 108 | + "path": "1-57abb6fb-2249-43ba-a7be-cf09bcc230de-0.parquet", |
| 109 | + "size": 2204, |
| 110 | + "partitionValues": {}, |
| 111 | + "modificationTime": 1701740386169, |
| 112 | + "dataChange": true, |
| 113 | + "stats": "{\"numRecords\": 2, \"minValues\": {\"num\": 8, \"letter\": \"dd\"}, \"maxValues\": {\"num\": 9, \"letter\": \"ee\"}, \"nullCount\": {\"num\": 0, \"letter\": 0}}" |
| 114 | + } |
| 115 | +} |
| 116 | +{ |
| 117 | + "commitInfo": { |
| 118 | + "timestamp": 1701740386169, |
| 119 | + "operation": "WRITE", |
| 120 | + "operationParameters": { |
| 121 | + "partitionBy": "[]", |
| 122 | + "mode": "Append" |
| 123 | + }, |
| 124 | + "clientVersion": "delta-rs.0.17.0" |
| 125 | + } |
| 126 | +} |
| 127 | +``` |
| 128 | + |
| 129 | +The transaction log records that the second file has been persisted in the Delta table. |
| 130 | + |
| 131 | +Now create a third pandas DataFrame and overwrite the Delta table with the new data. |
| 132 | + |
| 133 | +```python |
| 134 | +df = pd.DataFrame({"num": [11, 22], "letter": ["aa", "bb"]}) |
| 135 | +write_deltalake(f"{cwd}/tmp/delta-table", df, mode="append") |
| 136 | +``` |
| 137 | + |
| 138 | +Here are the files in storage: |
| 139 | + |
| 140 | +``` |
| 141 | +tmp/some-table |
| 142 | +├── 0-62dffa23-bbe1-4496-8fb5-bff6724dc677-0.parquet |
| 143 | +├── 1-57abb6fb-2249-43ba-a7be-cf09bcc230de-0.parquet |
| 144 | +├── 2-95ef2108-480c-4b89-96f0-ff9185dab9ad-0.parquet |
| 145 | +└── _delta_log |
| 146 | + ├── 00000000000000000000.json |
| 147 | + ├── 00000000000000000001.json |
| 148 | + └── 00000000000000000002.json |
| 149 | +``` |
| 150 | + |
| 151 | +Here are the contents of the `_delta_log/0002.json` file: |
| 152 | + |
| 153 | +```json |
| 154 | +{ |
| 155 | + "add": { |
| 156 | + "path": "2-95ef2108-480c-4b89-96f0-ff9185dab9ad-0.parquet", |
| 157 | + "size": 2204, |
| 158 | + "partitionValues": {}, |
| 159 | + "modificationTime": 1701740465102, |
| 160 | + "dataChange": true, |
| 161 | + "stats": "{\"numRecords\": 2, \"minValues\": {\"num\": 11, \"letter\": \"aa\"}, \"maxValues\": {\"num\": 22, \"letter\": \"bb\"}, \"nullCount\": {\"num\": 0, \"letter\": 0}}" |
| 162 | + } |
| 163 | +} |
| 164 | +{ |
| 165 | + "remove": { |
| 166 | + "path": "0-62dffa23-bbe1-4496-8fb5-bff6724dc677-0.parquet", |
| 167 | + "deletionTimestamp": 1701740465102, |
| 168 | + "dataChange": true, |
| 169 | + "extendedFileMetadata": false, |
| 170 | + "partitionValues": {}, |
| 171 | + "size": 2208 |
| 172 | + } |
| 173 | +} |
| 174 | +{ |
| 175 | + "remove": { |
| 176 | + "path": "1-57abb6fb-2249-43ba-a7be-cf09bcc230de-0.parquet", |
| 177 | + "deletionTimestamp": 1701740465102, |
| 178 | + "dataChange": true, |
| 179 | + "extendedFileMetadata": false, |
| 180 | + "partitionValues": {}, |
| 181 | + "size": 2204 |
| 182 | + } |
| 183 | +} |
| 184 | +{ |
| 185 | + "commitInfo": { |
| 186 | + "timestamp": 1701740465102, |
| 187 | + "operation": "WRITE", |
| 188 | + "operationParameters": { |
| 189 | + "mode": "Overwrite", |
| 190 | + "partitionBy": "[]" |
| 191 | + }, |
| 192 | + "clientVersion": "delta-rs.0.17.0" |
| 193 | + } |
| 194 | +} |
| 195 | +``` |
| 196 | + |
| 197 | +This transaction adds a data file and marks the two exising data files for removal. Marking a file for removal in the transaction log is known as "tombstoning the file" or a "logical delete". This is different from a "physical delete" which actually removes the data file from storage. |
| 198 | + |
| 199 | +## How Delta table operations differ from data lakes |
| 200 | + |
| 201 | +Data lakes consist of data files persisted in storage. They don't have a transaction log that retain metadata about the transactions. |
| 202 | + |
| 203 | +Data lakes perform transactions differently than Delta tables. |
| 204 | + |
| 205 | +When you perform an overwrite tranasction with a Delta table, you logically delete the exiting data without physically removing it. |
| 206 | + |
| 207 | +Data lakes don't support logical deletes, so you have to physically delete the data from storage. |
| 208 | + |
| 209 | +Logical data operations are safer because they can be rolled back if they don't complete successfully. Physically removing data from storage can be dangerous, especially if it's before a transaction is complete. |
| 210 | + |
| 211 | +We're now ready to look into Delta Lake ACID transactions in more detail. |
0 commit comments