You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/01/28 22:22:43 UTC

[GitHub] [iceberg] samarthjain commented on a change in pull request #3432: Doc: add a page to explain row-level deletes

samarthjain commented on a change in pull request #3432:
URL: https://github.com/apache/iceberg/pull/3432#discussion_r794927101



##########
File path: site/docs/row-level-deletes.md
##########
@@ -0,0 +1,190 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+# Row-Level Deletes
+
+Iceberg supports metadata-based deletion through the `DeleteFiles` interface.
+It allows you to quickly delete a specific file or any file that might match a given expression without the need to read or write any data in the table.
+
+Row-level deletes target more complicated use cases such as general data protection regulation (GDPR).
+Copy-on-write and merge-on-read are two different approaches to handle row-level delete operations. Here are their definitions in Iceberg:
+
+- **copy-on-write**: a delete directly rewrites all the affected data files.
+- **merge-on-read**: delete information is encoded in the form of _delete files_. The table reader can apply all delete information at read time.
+
+Overall, copy-on-write is more efficient in reading data, whereas merge-on-read is more efficient in writing deletes, but requires more maintenance and tuning to be performant in reading data with deletes.
+Users can choose to use **both** copy-on-write and merge-on-read for the same Iceberg table based on different situations. 
+For example, a time-partitioned table can have newer partitions maintained with the merge-on-read approach through a streaming pipeline,
+and older partitions maintained with the copy-on-write approach to apply less frequent GDPR deletes from batch ETL jobs.
+
+There are use cases that could only be supported by one approach such as change data capture (CDC).
+There are also limitations for different compute engines that lead them to prefer one approach over another.
+Please check out the documentation of the specific compute engines to see the details of their capabilities related to row-level deletes.
+This article will focus on explaining Iceberg's core design of copy-on-write and merge-on-read.
+
+!!!Note
+    Update is modeled as a delete with an insert within the same transaction in Iceberg, so this article only explains delete-related concepts. 
+
+## Copy-on-write
+
+In the copy-on-write approach, given a user's delete requirement, the write process would search for all the affected data files and perform a rewrite operation.
+
+For example, consider an unpartitioned table with schema `(id bigint, category string, data string)` that has the following files:
+
+```
+file A: (1, 'c1', 'data1'), (2, 'c1', 'data2')
+file B: (3, 'c2', 'data1'), (4, 'c2', 'data2')
+file C: (5, 'c3', 'data3'), (6, 'c3', 'data2')
+```
+
+A copy-on-write deletion of `data='data1'` can rewrite files A and B into a new file D. D contains the rows that were not deleted from files A and B. The table after the deletion looks like:
+
+```
+file C: (5, 'c3', 'data3'), (6, 'c3', 'data2')
+file D: (2, 'c1', 'data2'), (4, 'c2', 'data2')
+```
+
+There is no effect on read side in the copy-on-write approach.
+
+## Merge-on-read
+
+### Definitions
+
+Iceberg supports 2 different types of row-level delete files: **position deletes** and **equality deletes**.
+The **sequence number** concept is also needed to describe the relative age of data and delete files.
+If you are unfamiliar with these concepts, please read the [row-level deletes](../spec/#row-level-deletes) and [sequence numbers](../spec/#sequence-numbers) sections in the Iceberg spec for more information before proceeding.
+
+
+Also note that because row-level delete files are valid Iceberg data files, all rows in each delete file must belong to the same partition.
+For a partitioned table, if a delete file belongs to `Unpartitioned` (the partition has no partition field), then the delete file is called a **global delete**. 
+Otherwise, it is called a **partition delete**.
+
+### Writing delete files
+
+From the end user's perspective, it is very rare to directly request deletion of a specific row of a specific file. 
+A delete requirement usually comes as a predicate such as `id = 5` or `date < TIMESTAMP '2000-01-01'`. 
+
+Given a predicate, a compute engine can perform a scan to know the data files and row positions affected by the predicate and then write partition position deletes.
+The scan can be a table scan, or a scan of unflushed files stored in memory, local RocksDB, etc. for use cases like streaming.
+It is in theory possible to write global position deletes for partitioned tables, 
+but it is always preferred to write partition position deletes because the writer already knows the exact partition to use after the scan.
+
+When performing a scan is too expensive or time-consuming, the compute engine can use equality deletes for faster write.
+It can convert an input predicate to global equality deletes, or convert it to equality predicates for each affected partition and write partition equality deletes.
+However, such conversion might not always be possible without scanning data (e.g. delete all data with `price > 2.33`). In those cases, using position deletes is preferred.
+
+For example, in a CDC pipeline, both partition position deletes and partition equality deletes are used.
+Consider an unpartitioned table with schema `(id bigint, category string, data string)` that has the following files with sequence numbers:
+
+```
+seq=0 file A: (1, 'c1', 'data1'), (2, 'c1', 'data2')
+seq=0 file B: (3, 'c2', 'data1'), (4, 'c2', 'data2')
+```
+
+The CDC pipeline writing to the table currently contains unflushed data `(1, 'c10', 'data10')` that will be committed as file C in the table.
+For a new delete predicate `id = 1`, the writer first checks the unflushed data index in memory and performs a position delete of file C at row position 0.
+It then writes an equality delete row `(1, NULL, NULL)` that is applied to all the existing data files A and B in the table.
+After the next commit checkpoint, the new table contains the following files:
+
+```
+seq=0 file A: (1, 'c1', 'data1'), (2, 'c1', 'data2')
+seq=0 file B: (3, 'c2', 'data1'), (4, 'c2', 'data2')
+seq=1 file C: (1, 'c10', 'data10')
+seq=1 position delete D: ('C', 0)
+seq=1 equality delete E: (1, NULL, NULL)
+```
+
+### Reading data with delete files
+
+During Iceberg's scan file planning phase, a delete file index is constructed to filter the delete files associated with each data file using the following rules:
+
+1. equality delete files are applied to data files of strictly lower sequence numbers
+2. position delete files are applied to data files of equal or lower sequence numbers
+3. further pruning is performed by comparing the partition and column statistics information of each pair of delete and data file. Therefore, for a partitioned table, partition deletes are always preferred to global deletes.
+
+In the CDC example in the last section, position delete D is applied to file C, and equality delete E is applied to file A.
+
+After the planning phase, each data file to read is associated with a set of delete files to merge with.
+In general, position deletes are easier to merge, because they are already sorted by file path and row position when writing.
+Applying position deletes to a data file can be viewed as merging two sorted lists, which can be done efficiently.
+In contrast, applying equality deletes to a data files requires loading all rows to memory and checking every row in a data file against every equality predicate.

Review comment:
       By `equality predicate` do you mean `equality delete predicate` ? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org