1818 */
1919package org .apache .iceberg ;
2020
21+ import java .util .Objects ;
2122import java .util .Set ;
2223import java .util .concurrent .ExecutorService ;
24+ import java .util .concurrent .atomic .AtomicLong ;
2325import java .util .function .Consumer ;
2426import org .apache .iceberg .exceptions .NotFoundException ;
27+ import org .apache .iceberg .exceptions .ValidationException ;
2528import org .apache .iceberg .io .BulkDeletionFailureException ;
2629import org .apache .iceberg .io .CloseableIterable ;
2730import org .apache .iceberg .io .FileIO ;
2831import org .apache .iceberg .io .SupportsBulkOperations ;
32+ import org .apache .iceberg .relocated .com .google .common .base .MoreObjects ;
2933import org .apache .iceberg .relocated .com .google .common .collect .Sets ;
3034import org .apache .iceberg .util .Tasks ;
3135import org .slf4j .Logger ;
@@ -42,6 +46,9 @@ public void accept(String file) {
4246 };
4347
4448 private static final Logger LOG = LoggerFactory .getLogger (FileCleanupStrategy .class );
49+ protected static final String MANIFEST = "manifest" ;
50+ protected static final String MANIFEST_LIST = "manifest list" ;
51+ protected static final String STATISTICS_FILES = "statistics files" ;
4552
4653 protected final FileIO fileIO ;
4754 protected final ExecutorService planExecutorService ;
@@ -72,7 +79,7 @@ protected FileCleanupStrategy(
7279 * @param afterExpiration table metadata after snapshot expiration
7380 * @param cleanupLevel controls which types of files are eligible for deletion
7481 */
75- public abstract void cleanFiles (
82+ public abstract DeleteSummary cleanFiles (
7683 TableMetadata beforeExpiration ,
7784 TableMetadata afterExpiration ,
7885 ExpireSnapshots .CleanupLevel cleanupLevel );
@@ -99,8 +106,9 @@ protected CloseableIterable<ManifestFile> readManifests(Snapshot snapshot) {
99106 }
100107 }
101108
102- protected void deleteFiles (Set <String > pathsToDelete , String fileType ) {
109+ protected void deleteFiles (Set <String > pathsToDelete , String fileType , DeleteSummary summary ) {
103110 if (deleteFunc == null && fileIO instanceof SupportsBulkOperations ) {
111+ int failures = 0 ;
104112 try {
105113 ((SupportsBulkOperations ) fileIO ).deleteFiles (pathsToDelete );
106114 } catch (BulkDeletionFailureException e ) {
@@ -110,9 +118,11 @@ protected void deleteFiles(Set<String> pathsToDelete, String fileType) {
110118 pathsToDelete .size (),
111119 fileType ,
112120 e );
121+ failures = e .numberFailedObjects ();
113122 } catch (RuntimeException e ) {
114123 LOG .warn ("Bulk deletion failed" , e );
115124 }
125+ summary .deletedFiles (fileType , pathsToDelete .size () - failures );
116126 } else {
117127 Consumer <String > deleteFuncToUse = deleteFunc == null ? defaultDeleteFunc : deleteFunc ;
118128
@@ -124,7 +134,92 @@ protected void deleteFiles(Set<String> pathsToDelete, String fileType) {
124134 .suppressFailureWhenFinished ()
125135 .onFailure (
126136 (file , thrown ) -> LOG .warn ("Delete failed for {} file: {}" , fileType , file , thrown ))
127- .run (deleteFuncToUse ::accept );
137+ .run (
138+ file -> {
139+ deleteFuncToUse .accept (file );
140+ summary .deletedFile (fileType );
141+ });
142+ }
143+ }
144+
145+ static class DeleteSummary {
146+ private final AtomicLong dataFilesCount = new AtomicLong (0L );
147+ private final AtomicLong positionDeleteFilesCount = new AtomicLong (0L );
148+ private final AtomicLong equalityDeleteFilesCount = new AtomicLong (0L );
149+ private final AtomicLong manifestsCount = new AtomicLong (0L );
150+ private final AtomicLong manifestListsCount = new AtomicLong (0L );
151+ private final AtomicLong statisticsFilesCount = new AtomicLong (0L );
152+
153+ public void deletedFiles (String type , int numFiles ) {
154+ if (FileContent .DATA .name ().equalsIgnoreCase (type )) {
155+ dataFilesCount .addAndGet (numFiles );
156+
157+ } else if (FileContent .POSITION_DELETES .name ().equalsIgnoreCase (type )) {
158+ positionDeleteFilesCount .addAndGet (numFiles );
159+
160+ } else if (FileContent .EQUALITY_DELETES .name ().equalsIgnoreCase (type )) {
161+ equalityDeleteFilesCount .addAndGet (numFiles );
162+
163+ } else if (MANIFEST .equalsIgnoreCase (type )) {
164+ manifestsCount .addAndGet (numFiles );
165+
166+ } else if (MANIFEST_LIST .equalsIgnoreCase (type )) {
167+ manifestListsCount .addAndGet (numFiles );
168+
169+ } else if (STATISTICS_FILES .equalsIgnoreCase (type )) {
170+ statisticsFilesCount .addAndGet (numFiles );
171+
172+ } else {
173+ throw new ValidationException ("Illegal file type: %s" , type );
174+ }
175+ }
176+
177+ public void deletedFile (String type ) {
178+ if (FileContent .DATA .name ().equalsIgnoreCase (type )) {
179+ dataFilesCount .incrementAndGet ();
180+
181+ } else if (FileContent .POSITION_DELETES .name ().equalsIgnoreCase (type )) {
182+ positionDeleteFilesCount .incrementAndGet ();
183+
184+ } else if (FileContent .EQUALITY_DELETES .name ().equalsIgnoreCase (type )) {
185+ equalityDeleteFilesCount .incrementAndGet ();
186+
187+ } else if (MANIFEST .equalsIgnoreCase (type )) {
188+ manifestsCount .incrementAndGet ();
189+
190+ } else if (MANIFEST_LIST .equalsIgnoreCase (type )) {
191+ manifestListsCount .incrementAndGet ();
192+
193+ } else if (STATISTICS_FILES .equalsIgnoreCase (type )) {
194+ statisticsFilesCount .incrementAndGet ();
195+
196+ } else {
197+ throw new ValidationException ("Illegal file type: %s" , type );
198+ }
199+ }
200+
201+ public long dataFilesCount () {
202+ return dataFilesCount .get ();
203+ }
204+
205+ public long positionDeleteFilesCount () {
206+ return positionDeleteFilesCount .get ();
207+ }
208+
209+ public long equalityDeleteFilesCount () {
210+ return equalityDeleteFilesCount .get ();
211+ }
212+
213+ public long manifestsCount () {
214+ return manifestsCount .get ();
215+ }
216+
217+ public long manifestListsCount () {
218+ return manifestListsCount .get ();
219+ }
220+
221+ public long statisticsFilesCount () {
222+ return statisticsFilesCount .get ();
128223 }
129224 }
130225
@@ -141,6 +236,46 @@ protected Set<String> expiredStatisticsFilesLocations(
141236 return Sets .difference (statsFileLocationsBeforeExpiration , statsFileLocationsAfterExpiration );
142237 }
143238
239+ protected static class FileInfo {
240+ private final FileContent content ;
241+ private final String path ;
242+
243+ public FileInfo (FileContent content , String path ) {
244+ this .content = content ;
245+ this .path = path ;
246+ }
247+
248+ public FileContent getContent () {
249+ return content ;
250+ }
251+
252+ public String getPath () {
253+ return path ;
254+ }
255+
256+ @ Override
257+ public boolean equals (Object other ) {
258+ if (this == other ) {
259+ return true ;
260+ } else if (other == null || getClass () != other .getClass ()) {
261+ return false ;
262+ }
263+
264+ FileInfo fileInfo = (FileInfo ) other ;
265+ return Objects .equals (content , fileInfo .content ) && Objects .equals (path , fileInfo .path );
266+ }
267+
268+ @ Override
269+ public int hashCode () {
270+ return Objects .hash (content , path );
271+ }
272+
273+ @ Override
274+ public String toString () {
275+ return MoreObjects .toStringHelper (this ).add ("content" , content ).add ("path" , path ).toString ();
276+ }
277+ }
278+
144279 private Set <String > statsFileLocations (TableMetadata tableMetadata ) {
145280 Set <String > statsFileLocations = Sets .newHashSet ();
146281
0 commit comments