/** * The DistributedLogManager may archive/purge any logs for transactionId * less than or equal to minImageTxId. * This is to be used only when the client explicitly manages deletion. If * the cleanup policy is based on sliding time window, then this method need * not be called. * * @param minTxIdToKeep the earliest txid that must be retained * @throws IOException if purging fails */ @Override public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException { Preconditions.checkArgument(minTxIdToKeep > 0, "Invalid transaction id " + minTxIdToKeep); checkClosedOrInError("purgeLogSegmentsOlderThan"); BKLogWriteHandler ledgerHandler = createWriteHandler(true); try { LOG.info("Purging logs for {} older than {}", ledgerHandler.getFullyQualifiedName(), minTxIdToKeep); FutureUtils.result(ledgerHandler.purgeLogSegmentsOlderThanTxnId(minTxIdToKeep)); } finally { Utils.closeQuietly(ledgerHandler); } }
public void deleteLog() throws IOException { lock.checkOwnershipAndReacquire(); FutureUtils.result(purgeLogSegmentsOlderThanTxnId(-1)); try { Utils.closeQuietly(lock); zooKeeperClient.get().exists(logMetadata.getLogSegmentsPath(), false); zooKeeperClient.get().exists(logMetadata.getMaxTxIdPath(), false); if (logMetadata.getLogRootPath().toLowerCase().contains("distributedlog")) { ZKUtil.deleteRecursive(zooKeeperClient.get(), logMetadata.getLogRootPath()); } else { LOG.warn("Skip deletion of unrecognized ZK Path {}", logMetadata.getLogRootPath()); } } catch (InterruptedException ie) { LOG.error("Interrupted while deleting log znodes", ie); throw new DLInterruptedException("Interrupted while deleting " + logMetadata.getLogRootPath(), ie); } catch (KeeperException ke) { LOG.error("Error deleting" + logMetadata.getLogRootPath() + " in zookeeper", ke); } }
@Test public void testExplicitTruncation() throws Exception { String name = "distrlog-truncation-explicit"; DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(conf); confLocal.setExplicitTruncationByApplication(true); Map<Long, DLSN> txid2DLSN = new HashMap<Long, DLSN>(); Pair<DistributedLogManager, AsyncLogWriter> pair = populateData(txid2DLSN, confLocal, name, 4, 10, true); Thread.sleep(1000); for (int i = 1; i <= 4; i++) { int txn = (i-1) * 10 + i; DLSN dlsn = txid2DLSN.get((long)txn); assertTrue(Await.result(pair.getRight().truncate(dlsn))); verifyEntries(name, 1, (i - 1) * 10 + 1, (5 - i + 1) * 10); } // Delete higher dlsn int txn = 43; DLSN dlsn = txid2DLSN.get((long) txn); assertTrue(Await.result(pair.getRight().truncate(dlsn))); verifyEntries(name, 1, 31, 20); Utils.close(pair.getRight()); pair.getLeft().close(); // Try force truncation BKDistributedLogManager dlm = (BKDistributedLogManager)createNewDLM(confLocal, name); BKLogWriteHandler handler = dlm.createWriteHandler(true); FutureUtils.result(handler.purgeLogSegmentsOlderThanTxnId(Integer.MAX_VALUE)); verifyEntries(name, 1, 31, 20); }