Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[server/kv]fix bug for deleting remote kv dir #245

Open
wants to merge 2 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@
import com.alibaba.fluss.config.ConfigOptions;
import com.alibaba.fluss.config.Configuration;
import com.alibaba.fluss.exception.KvStorageException;
import com.alibaba.fluss.fs.FileSystem;
import com.alibaba.fluss.fs.FsPath;
import com.alibaba.fluss.memory.LazyMemorySegmentPool;
import com.alibaba.fluss.memory.MemorySegmentPool;
import com.alibaba.fluss.metadata.KvFormat;
Expand All @@ -42,6 +44,7 @@
import javax.annotation.concurrent.ThreadSafe;

import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -74,21 +77,29 @@ public final class KvManager extends TabletManagerBase {
/** The memory segment pool to allocate memorySegment. */
private final MemorySegmentPool memorySegmentPool;

private final FsPath remoteKvDir;

private final FileSystem remoteFileSystem;

private KvManager(
File dataDir,
Configuration conf,
ZooKeeperClient zkClient,
int recoveryThreadsPerDataDir,
LogManager logManager) {
LogManager logManager)
throws IOException {
super(TabletType.KV, dataDir, conf, recoveryThreadsPerDataDir);
this.logManager = logManager;
this.arrowBufferAllocator = new RootAllocator(Long.MAX_VALUE);
this.memorySegmentPool = LazyMemorySegmentPool.create(conf);
this.zkClient = zkClient;
this.remoteKvDir = FlussPaths.remoteKvDir(conf);
this.remoteFileSystem = remoteKvDir.getFileSystem();
}

public static KvManager create(
Configuration conf, ZooKeeperClient zkClient, LogManager logManager) {
Configuration conf, ZooKeeperClient zkClient, LogManager logManager)
throws IOException {
String dataDirString = conf.getString(ConfigOptions.DATA_DIR);
File dataDir = new File(dataDirString).getAbsoluteFile();
return new KvManager(
Expand Down Expand Up @@ -270,4 +281,17 @@ public KvTablet loadKv(File tabletDir) throws Exception {
this.currentKvs.put(tableBucket, kvTablet);
return kvTablet;
}

public void dropRemoteKvSnapshot(PhysicalTablePath physicalTablePath, TableBucket tableBucket) {
FsPath remoteKvTabletDir =
FlussPaths.remoteKvTabletDir(remoteKvDir, physicalTablePath, tableBucket);
try {
if (remoteFileSystem.exists(remoteKvTabletDir)) {
remoteFileSystem.delete(remoteKvTabletDir, true);
LOG.info("Delete remote table bucket snapshot of {} success.", tableBucket);
}
} catch (Exception e) {
LOG.error("Failed to delete remote kv tablet path {}.", remoteKvTabletDir, e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -177,34 +177,36 @@ public void triggerSnapshot() {
// of using guardedExecutor
guardedExecutor.execute(
() -> {
LOG.debug("TableBucket {} triggers snapshot.", tableBucket);
long triggerTime = System.currentTimeMillis();

Optional<SnapshotRunnable> snapshotRunnableOptional;
try {
snapshotRunnableOptional = target.initSnapshot();
} catch (Exception e) {
LOG.error("Fail to init snapshot during triggering snapshot.", e);
return;
}
if (snapshotRunnableOptional.isPresent()) {
SnapshotRunnable runnable = snapshotRunnableOptional.get();
asyncOperationsThreadPool.execute(
() ->
asyncSnapshotPhase(
triggerTime,
runnable.getSnapshotId(),
runnable.getCoordinatorEpoch(),
runnable.getBucketLeaderEpoch(),
runnable.getSnapshotLocation(),
runnable.getSnapshotRunnable()));
} else {
scheduleNextSnapshot();
LOG.debug(
"TableBucket {} has no data updates since last snapshot, "
+ "skip this one and schedule the next one in {} seconds",
tableBucket,
periodicSnapshotDelay / 1000);
if (started && !periodicExecutor.isShutdown()) {
LOG.debug("TableBucket {} triggers snapshot.", tableBucket);
long triggerTime = System.currentTimeMillis();

Optional<SnapshotRunnable> snapshotRunnableOptional;
try {
snapshotRunnableOptional = target.initSnapshot();
} catch (Exception e) {
LOG.error("Fail to init snapshot during triggering snapshot.", e);
return;
}
if (snapshotRunnableOptional.isPresent()) {
SnapshotRunnable runnable = snapshotRunnableOptional.get();
asyncOperationsThreadPool.execute(
() ->
asyncSnapshotPhase(
triggerTime,
runnable.getSnapshotId(),
runnable.getCoordinatorEpoch(),
runnable.getBucketLeaderEpoch(),
runnable.getSnapshotLocation(),
runnable.getSnapshotRunnable()));
} else {
scheduleNextSnapshot();
LOG.debug(
"TableBucket {} has no data updates since last snapshot, "
+ "skip this one and schedule the next one in {} seconds",
tableBucket,
periodicSnapshotDelay / 1000);
}
}
});
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1239,6 +1239,10 @@ private StopReplicaResultForBucket stopReplica(
}

remoteLogManager.stopReplica(replicaToDelete, delete && replicaToDelete.isLeader());
if (delete) {
kvManager.dropRemoteKvSnapshot(
replicaToDelete.getPhysicalTablePath(), replicaToDelete.getTableBucket());
}
}

// If we were the leader, we may have some operations still waiting for completion.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@
import com.alibaba.fluss.record.KvRecordBatch;
import com.alibaba.fluss.rpc.gateway.TabletServerGateway;
import com.alibaba.fluss.rpc.messages.PutKvRequest;
import com.alibaba.fluss.server.entity.StopReplicaData;
import com.alibaba.fluss.server.entity.StopReplicaResultForBucket;
import com.alibaba.fluss.server.kv.snapshot.CompletedSnapshot;
import com.alibaba.fluss.server.kv.snapshot.ZooKeeperCompletedSnapshotHandleStore;
import com.alibaba.fluss.server.tablet.TabletServer;
Expand All @@ -35,11 +37,17 @@
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.RegisterExtension;

import java.io.File;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CompletableFuture;

import static com.alibaba.fluss.record.TestData.DATA1_TABLE_INFO_PK;
import static com.alibaba.fluss.server.coordinator.CoordinatorContext.INITIAL_COORDINATOR_EPOCH;
import static com.alibaba.fluss.testutils.DataTestUtils.genKvRecordBatch;
import static com.alibaba.fluss.testutils.DataTestUtils.genKvRecords;
import static com.alibaba.fluss.testutils.DataTestUtils.getKeyValuePairs;
Expand Down Expand Up @@ -85,6 +93,7 @@ void testKvSnapshot() throws Exception {
}
}

Set<File> bucketDirs = new HashSet<>();
for (TableBucket tableBucket : tableBuckets) {
long tableId = tableBucket.getTableId();
int bucket = tableBucket.getBucket();
Expand Down Expand Up @@ -122,6 +131,7 @@ void testKvSnapshot() throws Exception {
Tuple2.of("k1", new Object[] {1, "k1"}),
Tuple2.of("k2", new Object[] {2, "k2"})));
KvTestUtils.checkSnapshot(completedSnapshot, expectedKeyValues, 2);
bucketDirs.add(new File(completedSnapshot.getSnapshotLocation().getParent().getPath()));

// put kv batch again
kvRecordBatch =
Expand Down Expand Up @@ -160,8 +170,24 @@ void testKvSnapshot() throws Exception {
assertThat(replica.getLogTablet().getMinRetainOffset())
.as("Replica %s min retain offset", replica)
.isEqualTo(6));
CompletableFuture<List<StopReplicaResultForBucket>> future =
new CompletableFuture<>();
server.getReplicaManager()
.stopReplicas(
INITIAL_COORDINATOR_EPOCH,
Collections.singletonList(
new StopReplicaData(
tableBucket, true, INITIAL_COORDINATOR_EPOCH, 1)),
future::complete);
}
}
checkBucketDirsDeleted(bucketDirs);
}

private void checkBucketDirsDeleted(Set<File> bucketDirs) {
for (File bucketDir : bucketDirs) {
assertThat(bucketDir.exists()).isFalse();
}
}

private static Configuration initConfig() {
Expand Down
Loading