Skip to content

Commit

Permalink
[Enhancement] Support fe starting with only image no bdb log
Browse files Browse the repository at this point in the history
Signed-off-by: xiangguangyxg <[email protected]>
  • Loading branch information
xiangguangyxg committed Dec 30, 2024
1 parent fbc725c commit 0ddd635
Show file tree
Hide file tree
Showing 5 changed files with 26 additions and 31 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -134,10 +134,12 @@ public static BDBEnvironment initBDBEnvironment(String nodeName) throws JournalE

// constructor
String selfNodeHostPort = NetUtils.getHostPortInAccessibleFormat(selfNode.first, selfNode.second);

boolean isFirstTimeStartUp = false;

File dbEnv = new File(getBdbDir());
if (!dbEnv.exists()) {
dbEnv.mkdirs();
isFirstTimeStartUp = true;
}

Pair<String, Integer> helperNode = GlobalStateMgr.getCurrentState().getNodeMgr().getHelperNode();
Expand All @@ -147,7 +149,7 @@ public static BDBEnvironment initBDBEnvironment(String nodeName) throws JournalE
helperHostPort, GlobalStateMgr.getCurrentState().isElectable());

// setup
bdbEnvironment.setup();
bdbEnvironment.setup(isFirstTimeStartUp);
return bdbEnvironment;
}

Expand All @@ -165,25 +167,27 @@ protected BDBEnvironment(File envHome, String selfNodeName, String selfNodeHostP
}

// The setup() method opens the environment and database
protected void setup() throws JournalException, InterruptedException {
protected void setup(boolean isFirstTimeStartUp) throws JournalException, InterruptedException {
this.closing = false;
ensureHelperInLocal();
initConfigs(isElectable);
initConfigs(isFirstTimeStartUp);
setupEnvironment();
}

protected void initConfigs(boolean isElectable) throws JournalException {
protected void initConfigs(boolean isFirstTimeStartUp) throws JournalException {
// Almost never used, just in case the master can not restart
if (Config.bdbje_reset_election_group) {
if (!isElectable) {
String errMsg = "Current node is not in the electable_nodes list. will exit";
LOG.error(errMsg);
throw new JournalException(errMsg);
}
DbResetRepGroup resetUtility = new DbResetRepGroup(envHome, STARROCKS_JOURNAL_GROUP, selfNodeName,
selfNodeHostPort);
resetUtility.reset();
LOG.info("group has been reset.");
if (!isFirstTimeStartUp) {
DbResetRepGroup resetUtility = new DbResetRepGroup(envHome, STARROCKS_JOURNAL_GROUP, selfNodeName,
selfNodeHostPort);
resetUtility.reset();
LOG.info("group has been reset.");
}
}

// set replication config
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1265,16 +1265,7 @@ private void transferToLeader() {
dominationStartTimeMs = System.currentTimeMillis();

try {
// Log the first frontend
if (nodeMgr.isFirstTimeStartUp()) {
// if isFirstTimeStartUp is true, frontends must contain this Node.
Frontend self = nodeMgr.getMySelf();
Preconditions.checkNotNull(self);
// OP_ADD_FIRST_FRONTEND is emitted, so it can write to BDBJE even if canWrite is false
editLog.logAddFirstFrontend(self);
}

if (Config.bdbje_reset_election_group) {
if (Config.bdbje_reset_election_group || nodeMgr.isFirstTimeStartUp()) {
nodeMgr.resetFrontends();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ public void testSetupStandalone() throws Exception {
selfNodeHostPort,
selfNodeHostPort,
true);
environment.setup();
environment.setup(true);

CloseSafeDatabase db = environment.openDatabase("testdb");
DatabaseEntry key = randomEntry();
Expand Down Expand Up @@ -117,7 +117,7 @@ public void testSetupStandaloneMultitimes() throws Exception {
selfNodeHostPort,
selfNodeHostPort,
true);
environment.setup();
environment.setup(true);
}
Assert.fail();
} finally {
Expand Down Expand Up @@ -173,7 +173,7 @@ private void initClusterMasterFollowerNoRetry() throws Exception {
leaderNodeHostPort,
leaderNodeHostPort,
true);
leaderEnvironment.setup();
leaderEnvironment.setup(true);
Assert.assertEquals(0, leaderEnvironment.getDatabaseNames().size());

// set up 2 followers
Expand All @@ -188,7 +188,7 @@ private void initClusterMasterFollowerNoRetry() throws Exception {
leaderNodeHostPort,
true);
followerEnvironments[i] = followerEnvironment;
followerEnvironment.setup();
followerEnvironment.setup(true);
Assert.assertEquals(0, followerEnvironment.getDatabaseNames().size());
}
BDBEnvironment.RETRY_TIME = 3;
Expand Down Expand Up @@ -243,7 +243,7 @@ public void checkForNullParam(final Object param, final String name) {
true);
Assert.assertTrue(true);
try {
maserEnvironment.setup();
maserEnvironment.setup(true);
} catch (JournalException e) {
LOG.warn("got Rollback Exception, as expect, ", e);
}
Expand All @@ -268,7 +268,7 @@ private void leaderFailOver() throws Exception {
if (followerEnvironments[i].getReplicatedEnvironment().getState() == ReplicatedEnvironment.State.MASTER) {
newMasterEnvironment = followerEnvironments[i];
LOG.warn("=========> new leader is {}", newMasterEnvironment.getReplicatedEnvironment().getNodeName());
newMasterEnvironment.setup();
newMasterEnvironment.setup(true);
newMasterFollowerIndex = i;
break;
}
Expand All @@ -282,7 +282,7 @@ private void leaderFailOver() throws Exception {
leaderNodeHostPort,
leaderNodeHostPort,
true);
oldMasterEnvironment.setup();
oldMasterEnvironment.setup(true);
LOG.warn("============> old leader is setup as follower");
Thread.sleep(1000);

Expand Down Expand Up @@ -335,7 +335,7 @@ protected void testAddBadFollowerBase(boolean failover) throws Exception {
true);
LOG.warn("=========> start new follower for the first time");
// should set up successfully as a standalone leader
newfollowerEnvironment.setup();
newfollowerEnvironment.setup(true);
newfollowerEnvironment.close();

// 2. bad new follower start for the second time
Expand All @@ -348,7 +348,7 @@ protected void testAddBadFollowerBase(boolean failover) throws Exception {
true);
LOG.warn("==========> start new follower for the second time");
try {
newfollowerEnvironment.setup();
newfollowerEnvironment.setup(true);
} catch (Exception e) {
LOG.warn("===========> failed for the second time, as expect, ", e);
}
Expand All @@ -370,7 +370,7 @@ public void testGetDatabase() throws Exception {
selfNodeHostPort,
selfNodeHostPort,
true);
environment.setup();
environment.setup(true);

new MockUp<ReplicatedEnvironment>() {
@Mock
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ private BDBEnvironment initBDBEnv(String name) throws Exception {
selfNodeHostPort,
selfNodeHostPort,
true);
environment.setup();
environment.setup(true);
return environment;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,7 @@ private BDBEnvironment initBDBEnv() throws Exception {
selfNodeHostPort,
selfNodeHostPort,
true);
environment.setup();
environment.setup(true);
return environment;
}

Expand Down

0 comments on commit 0ddd635

Please sign in to comment.