Skip to content

Commit 62191ee

Browse files
authored
Deletion Vectors read support (#1516)
First see if we can produce some deletion vectors. Resolves #1549
1 parent 278f764 commit 62191ee

File tree

12 files changed

+1165
-860
lines changed

12 files changed

+1165
-860
lines changed

dev/provision.py

Lines changed: 85 additions & 74 deletions
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,7 @@
1414
# KIND, either express or implied. See the License for the
1515
# specific language governing permissions and limitations
1616
# under the License.
17+
import math
1718

1819
from pyspark.sql import SparkSession
1920
from pyspark.sql.functions import current_date, date_add, expr
@@ -113,89 +114,99 @@
113114
"""
114115
)
115116

116-
spark.sql(
117-
f"""
118-
CREATE OR REPLACE TABLE {catalog_name}.default.test_positional_mor_deletes (
119-
dt date,
120-
number integer,
121-
letter string
122-
)
123-
USING iceberg
124-
TBLPROPERTIES (
125-
'write.delete.mode'='merge-on-read',
126-
'write.update.mode'='merge-on-read',
127-
'write.merge.mode'='merge-on-read',
128-
'format-version'='2'
129-
);
130-
"""
131-
)
117+
# Merge on read has been implemented in version ≥2:
118+
# v2: Using positional deletes
119+
# v3: Using deletion vectors
132120

133-
spark.sql(
134-
f"""
135-
INSERT INTO {catalog_name}.default.test_positional_mor_deletes
136-
VALUES
137-
(CAST('2023-03-01' AS date), 1, 'a'),
138-
(CAST('2023-03-02' AS date), 2, 'b'),
139-
(CAST('2023-03-03' AS date), 3, 'c'),
140-
(CAST('2023-03-04' AS date), 4, 'd'),
141-
(CAST('2023-03-05' AS date), 5, 'e'),
142-
(CAST('2023-03-06' AS date), 6, 'f'),
143-
(CAST('2023-03-07' AS date), 7, 'g'),
144-
(CAST('2023-03-08' AS date), 8, 'h'),
145-
(CAST('2023-03-09' AS date), 9, 'i'),
146-
(CAST('2023-03-10' AS date), 10, 'j'),
147-
(CAST('2023-03-11' AS date), 11, 'k'),
148-
(CAST('2023-03-12' AS date), 12, 'l');
149-
"""
150-
)
121+
for format_version in [2, 3]:
122+
identifier = f'{catalog_name}.default.test_positional_mor_deletes_v{format_version}'
123+
spark.sql(
124+
f"""
125+
CREATE OR REPLACE TABLE {identifier} (
126+
dt date,
127+
number integer,
128+
letter string
129+
)
130+
USING iceberg
131+
TBLPROPERTIES (
132+
'write.delete.mode'='merge-on-read',
133+
'write.update.mode'='merge-on-read',
134+
'write.merge.mode'='merge-on-read',
135+
'format-version'='{format_version}'
136+
);
137+
"""
138+
)
139+
140+
spark.sql(
141+
f"""
142+
INSERT INTO {identifier}
143+
VALUES
144+
(CAST('2023-03-01' AS date), 1, 'a'),
145+
(CAST('2023-03-02' AS date), 2, 'b'),
146+
(CAST('2023-03-03' AS date), 3, 'c'),
147+
(CAST('2023-03-04' AS date), 4, 'd'),
148+
(CAST('2023-03-05' AS date), 5, 'e'),
149+
(CAST('2023-03-06' AS date), 6, 'f'),
150+
(CAST('2023-03-07' AS date), 7, 'g'),
151+
(CAST('2023-03-08' AS date), 8, 'h'),
152+
(CAST('2023-03-09' AS date), 9, 'i'),
153+
(CAST('2023-03-10' AS date), 10, 'j'),
154+
(CAST('2023-03-11' AS date), 11, 'k'),
155+
(CAST('2023-03-12' AS date), 12, 'l');
156+
"""
157+
)
151158

152-
spark.sql(f"ALTER TABLE {catalog_name}.default.test_positional_mor_deletes CREATE TAG tag_12")
159+
spark.sql(f"ALTER TABLE {identifier} CREATE TAG tag_12")
153160

154-
spark.sql(f"ALTER TABLE {catalog_name}.default.test_positional_mor_deletes CREATE BRANCH without_5")
161+
spark.sql(f"ALTER TABLE {identifier} CREATE BRANCH without_5")
155162

156-
spark.sql(f"DELETE FROM {catalog_name}.default.test_positional_mor_deletes.branch_without_5 WHERE number = 5")
163+
spark.sql(f"DELETE FROM {identifier}.branch_without_5 WHERE number = 5")
157164

158-
spark.sql(f"DELETE FROM {catalog_name}.default.test_positional_mor_deletes WHERE number = 9")
165+
spark.sql(f"DELETE FROM {identifier} WHERE number = 9")
159166

160-
spark.sql(
161-
f"""
162-
CREATE OR REPLACE TABLE {catalog_name}.default.test_positional_mor_double_deletes (
163-
dt date,
164-
number integer,
165-
letter string
166-
)
167-
USING iceberg
168-
TBLPROPERTIES (
169-
'write.delete.mode'='merge-on-read',
170-
'write.update.mode'='merge-on-read',
171-
'write.merge.mode'='merge-on-read',
172-
'format-version'='2'
173-
);
174-
"""
175-
)
167+
identifier = f'{catalog_name}.default.test_positional_mor_double_deletes_v{format_version}'
176168

177-
spark.sql(
178-
f"""
179-
INSERT INTO {catalog_name}.default.test_positional_mor_double_deletes
180-
VALUES
181-
(CAST('2023-03-01' AS date), 1, 'a'),
182-
(CAST('2023-03-02' AS date), 2, 'b'),
183-
(CAST('2023-03-03' AS date), 3, 'c'),
184-
(CAST('2023-03-04' AS date), 4, 'd'),
185-
(CAST('2023-03-05' AS date), 5, 'e'),
186-
(CAST('2023-03-06' AS date), 6, 'f'),
187-
(CAST('2023-03-07' AS date), 7, 'g'),
188-
(CAST('2023-03-08' AS date), 8, 'h'),
189-
(CAST('2023-03-09' AS date), 9, 'i'),
190-
(CAST('2023-03-10' AS date), 10, 'j'),
191-
(CAST('2023-03-11' AS date), 11, 'k'),
192-
(CAST('2023-03-12' AS date), 12, 'l');
193-
"""
194-
)
169+
spark.sql(
170+
f"""
171+
CREATE OR REPLACE TABLE {identifier} (
172+
dt date,
173+
number integer,
174+
letter string
175+
)
176+
USING iceberg
177+
TBLPROPERTIES (
178+
'write.delete.mode'='merge-on-read',
179+
'write.update.mode'='merge-on-read',
180+
'write.merge.mode'='merge-on-read',
181+
'format-version'='2'
182+
);
183+
"""
184+
)
195185

196-
spark.sql(f"DELETE FROM {catalog_name}.default.test_positional_mor_double_deletes WHERE number = 9")
186+
spark.sql(
187+
f"""
188+
INSERT INTO {identifier}
189+
VALUES
190+
(CAST('2023-03-01' AS date), 1, 'a'),
191+
(CAST('2023-03-02' AS date), 2, 'b'),
192+
(CAST('2023-03-03' AS date), 3, 'c'),
193+
(CAST('2023-03-04' AS date), 4, 'd'),
194+
(CAST('2023-03-05' AS date), 5, 'e'),
195+
(CAST('2023-03-06' AS date), 6, 'f'),
196+
(CAST('2023-03-07' AS date), 7, 'g'),
197+
(CAST('2023-03-08' AS date), 8, 'h'),
198+
(CAST('2023-03-09' AS date), 9, 'i'),
199+
(CAST('2023-03-10' AS date), 10, 'j'),
200+
(CAST('2023-03-11' AS date), 11, 'k'),
201+
(CAST('2023-03-12' AS date), 12, 'l');
202+
"""
203+
)
197204

198-
spark.sql(f"DELETE FROM {catalog_name}.default.test_positional_mor_double_deletes WHERE letter == 'f'")
205+
# Perform two deletes, should produce:
206+
# v2: two positional delete files in v2
207+
# v3: one deletion vector since they are merged
208+
spark.sql(f"DELETE FROM {identifier} WHERE number = 9")
209+
spark.sql(f"DELETE FROM {identifier} WHERE letter == 'f'")
199210

200211
all_types_dataframe = (
201212
spark.range(0, 5, 1, 5)

0 commit comments

Comments
 (0)