RussellSpitzer commented on a change in pull request #1264: URL: https://github.com/apache/iceberg/pull/1264#discussion_r467119880
########## File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java ########## @@ -0,0 +1,780 @@ +/* + * 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. + */ + +package org.apache.iceberg.actions; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.spark.source.ThreeColumnRecord; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import static org.apache.iceberg.types.Types.NestedField.optional; + +public abstract class TestExpireSnapshotsAction extends SparkTestBase { + + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + private static final Schema SCHEMA = new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get()) + ); + + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + + private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA")); + + static final DataFile FILE_A = DataFiles.builder(SPEC) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=0") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_B = DataFiles.builder(SPEC) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=1") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_C = DataFiles.builder(SPEC) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=2") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_D = DataFiles.builder(SPEC) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=3") // easy way to set partition data for now + .withRecordCount(1) + .build(); + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + private File tableDir; + private String tableLocation; + private Table table; + + @Before + public void setupTableLocation() throws Exception { + this.tableDir = temp.newFolder(); + this.tableLocation = tableDir.toURI().toString(); + this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); + } + + private Dataset<Row> buildDF(List<ThreeColumnRecord> records) { + return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + } + + private void writeDF(Dataset<Row> df, String mode) { + df.select("c1", "c2", "c3") + .write() + .format("iceberg") + .mode(mode) + .save(tableLocation); + } + + private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted, + Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) { + + Assert.assertEquals("Incorrect number of manifest files deleted", + expectedManifestsDeleted, results.getManifestFilesDeleted()); + Assert.assertEquals("Incorrect number of datafiles deleted", + expectedDatafiles, results.getDataFilesDeleted()); + Assert.assertEquals("Incorrect number of manifest lists deleted", + expectedManifestListsDeleted, results.getManifestListsDeleted()); + } + + @Test + public void testFilesCleaned() throws Exception { + Dataset<Row> df = buildDF(RECORDS); + + writeDF(df, "append"); + + List<Path> expiredDataFiles = Files + .list(tableDir.toPath().resolve("data").resolve("c1=1")) + .collect(Collectors.toList()); + + Assert.assertEquals("There should be a data file to delete but there was none.", + 2, expiredDataFiles.size()); + + writeDF(df, "overwrite"); + writeDF(df, "append"); + + long end = System.currentTimeMillis(); + while (end <= table.currentSnapshot().timestampMillis()) { + end = System.currentTimeMillis(); + } + + ExpireSnapshotsActionResult results = + Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute(); + + table.refresh(); + + Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots())); + + for (Path p : expiredDataFiles) { + Assert.assertFalse(String.format("File %s still exists but should have been deleted", p), + Files.exists(p)); + } + + checkExpirationResults(1L, 2L, 2L, results); + } + + @Test + public void dataFilesCleanupWithParallelTasks() throws IOException { + + table.newFastAppend() + .appendFile(FILE_A) + .commit(); + + table.newFastAppend() + .appendFile(FILE_B) + .commit(); + + table.newRewrite() + .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D)) + .commit(); + long thirdSnapshotId = table.currentSnapshot().snapshotId(); + + table.newRewrite() + .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C)) + .commit(); + long fourthSnapshotId = table.currentSnapshot().snapshotId(); + + long t4 = System.currentTimeMillis(); + while (t4 <= table.currentSnapshot().timestampMillis()) { + t4 = System.currentTimeMillis(); + } + + Set<String> deletedFiles = Sets.newHashSet(); + Set<String> deleteThreads = ConcurrentHashMap.newKeySet(); + AtomicInteger deleteThreadsIndex = new AtomicInteger(0); + + Actions.forTable(table).expireSnapshots() + .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> { + Thread thread = new Thread(runnable); + thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement()); + thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits + return thread; + })) + .expireOlderThan(t4) + .deleteWith(s -> { + deleteThreads.add(Thread.currentThread().getName()); + deletedFiles.add(s); + }) + .execute(); + + // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory + Assert.assertEquals(deleteThreads, + Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3")); + + Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); + Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString())); + } + + @Test + public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception { + Dataset<Row> df = buildDF(RECORDS); + + writeDF(df, "append"); + + ExpireSnapshotsActionResult results = + Actions.forTable(table).expireSnapshots().execute(); + + checkExpirationResults(0L, 0L, 0L, results); + } + + @Test + public void testCleanupRepeatedOverwrites() throws Exception { Review comment: New test - Not Ported ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
