|
25 | 25 | import java.util.Map;
|
26 | 26 | import org.apache.hadoop.conf.Configuration;
|
27 | 27 | import org.apache.hadoop.fs.FileStatus;
|
| 28 | +import org.apache.hadoop.fs.Path; |
28 | 29 | import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
29 | 30 | import org.apache.hadoop.hbase.TableName;
|
30 | 31 | import org.apache.hadoop.hbase.backup.BackupInfo;
|
|
36 | 37 | import org.apache.hadoop.hbase.master.HMaster;
|
37 | 38 | import org.apache.hadoop.hbase.master.MasterServices;
|
38 | 39 | import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
|
| 40 | +import org.apache.hadoop.hbase.master.region.MasterRegionFactory; |
39 | 41 | import org.apache.hadoop.hbase.net.Address;
|
40 | 42 | import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
|
41 | 43 | import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
|
@@ -123,27 +125,8 @@ public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
|
123 | 125 | return Collections.emptyList();
|
124 | 126 | }
|
125 | 127 | for (FileStatus file : files) {
|
126 |
| - String fn = file.getPath().getName(); |
127 |
| - if (fn.startsWith(WALProcedureStore.LOG_PREFIX)) { |
| 128 | + if (canDeleteFile(addressToLastBackupMap, file.getPath())) { |
128 | 129 | filteredFiles.add(file);
|
129 |
| - continue; |
130 |
| - } |
131 |
| - |
132 |
| - try { |
133 |
| - Address walServerAddress = |
134 |
| - Address.fromString(BackupUtils.parseHostNameFromLogFile(file.getPath())); |
135 |
| - long walTimestamp = AbstractFSWALProvider.getTimestamp(file.getPath().getName()); |
136 |
| - |
137 |
| - if ( |
138 |
| - !addressToLastBackupMap.containsKey(walServerAddress) |
139 |
| - || addressToLastBackupMap.get(walServerAddress) >= walTimestamp |
140 |
| - ) { |
141 |
| - filteredFiles.add(file); |
142 |
| - } |
143 |
| - } catch (Exception ex) { |
144 |
| - LOG.warn( |
145 |
| - "Error occurred while filtering file: {} with error: {}. Ignoring cleanup of this log", |
146 |
| - file.getPath(), ex.getMessage()); |
147 | 130 | }
|
148 | 131 | }
|
149 | 132 |
|
@@ -176,4 +159,39 @@ public void stop(String why) {
|
176 | 159 | public boolean isStopped() {
|
177 | 160 | return this.stopped;
|
178 | 161 | }
|
| 162 | + |
| 163 | + protected static boolean canDeleteFile(Map<Address, Long> addressToLastBackupMap, Path path) { |
| 164 | + if (isHMasterWAL(path)) { |
| 165 | + return true; |
| 166 | + } |
| 167 | + |
| 168 | + try { |
| 169 | + String hostname = BackupUtils.parseHostNameFromLogFile(path); |
| 170 | + if (hostname == null) { |
| 171 | + LOG.warn( |
| 172 | + "Cannot parse hostname from RegionServer WAL file: {}. Ignoring cleanup of this log", |
| 173 | + path); |
| 174 | + return false; |
| 175 | + } |
| 176 | + Address walServerAddress = Address.fromString(hostname); |
| 177 | + long walTimestamp = AbstractFSWALProvider.getTimestamp(path.getName()); |
| 178 | + |
| 179 | + if ( |
| 180 | + !addressToLastBackupMap.containsKey(walServerAddress) |
| 181 | + || addressToLastBackupMap.get(walServerAddress) >= walTimestamp |
| 182 | + ) { |
| 183 | + return true; |
| 184 | + } |
| 185 | + } catch (Exception ex) { |
| 186 | + LOG.warn("Error occurred while filtering file: {}. Ignoring cleanup of this log", path, ex); |
| 187 | + return false; |
| 188 | + } |
| 189 | + return false; |
| 190 | + } |
| 191 | + |
| 192 | + private static boolean isHMasterWAL(Path path) { |
| 193 | + String fn = path.getName(); |
| 194 | + return fn.startsWith(WALProcedureStore.LOG_PREFIX) |
| 195 | + || fn.endsWith(MasterRegionFactory.ARCHIVED_WAL_SUFFIX); |
| 196 | + } |
179 | 197 | }
|
0 commit comments