Skip to content

Commit b941f01

Browse files
author
Tanuj Khurana
committed
Handle waitTime returned by the setHAGroupStatusToSync API
1 parent d0e30d5 commit b941f01

File tree

3 files changed

+77
-43
lines changed

3 files changed

+77
-43
lines changed

phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java

Lines changed: 41 additions & 35 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import org.apache.phoenix.jdbc.HAGroupStateListener;
2828
import org.apache.phoenix.jdbc.HAGroupStoreManager;
2929
import org.apache.phoenix.jdbc.HAGroupStoreRecord;
30+
import org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode;
3031
import org.apache.phoenix.replication.metrics.MetricsReplicationLogDiscovery;
3132
import org.apache.phoenix.replication.metrics.MetricsReplicationLogForwarderSourceFactory;
3233
import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting;
@@ -49,10 +50,12 @@ public class ReplicationLogDiscoveryForwarder extends ReplicationLogDiscovery {
4950
public static final String REPLICATION_LOG_COPY_THROUGHPUT_BYTES_PER_MS_KEY =
5051
"phoenix.replication.log.copy.throughput.bytes.per.ms";
5152
// TODO: come up with a better default after testing
52-
public static final double DEFAULT_LOG_COPY_THROUGHPUT_BYTES_PER_MS = 1;
53+
public static final double DEFAULT_LOG_COPY_THROUGHPUT_BYTES_PER_MS = 1.0;
5354

5455
private final ReplicationLogGroup logGroup;
55-
private double copyThroughputThresholdBytesPerMs;
56+
private final double copyThroughputThresholdBytesPerMs;
57+
// the timestamp (in future) at which we will attempt to set the HAGroup state to SYNC
58+
private long syncUpdateTS;
5659

5760
/**
5861
* Create a tracker for the replication logs in the fallback cluster.
@@ -76,6 +79,8 @@ public ReplicationLogDiscoveryForwarder(ReplicationLogGroup logGroup) {
7679
this.copyThroughputThresholdBytesPerMs =
7780
conf.getDouble(REPLICATION_LOG_COPY_THROUGHPUT_BYTES_PER_MS_KEY,
7881
DEFAULT_LOG_COPY_THROUGHPUT_BYTES_PER_MS);
82+
// initialize to 0
83+
this.syncUpdateTS = 0;
7984
}
8085

8186
@Override
@@ -102,14 +107,7 @@ public void init() throws IOException {
102107
&& HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC.equals(toState)) {
103108
LOG.info("Received ACTIVE_NOT_IN_SYNC event for {}", logGroup);
104109
// If the current mode is SYNC only then switch to SYNC_AND_FORWARD mode
105-
if (logGroup.checkAndSetMode(SYNC, SYNC_AND_FORWARD)) {
106-
// replication mode switched, notify the event handler
107-
try {
108-
logGroup.sync();
109-
} catch (IOException e) {
110-
LOG.info("Failed to send sync event to {}", logGroup);
111-
}
112-
}
110+
checkAndSetModeAndNotify(SYNC, SYNC_AND_FORWARD);
113111
}
114112
};
115113

@@ -126,14 +124,7 @@ public void init() throws IOException {
126124
&& HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC.equals(toState)) {
127125
LOG.info("Received ACTIVE_IN_SYNC event for {}", logGroup);
128126
// Set the current mode to SYNC
129-
if (logGroup.checkAndSetMode(SYNC_AND_FORWARD, SYNC)) {
130-
// replication mode switched, notify the event handler
131-
try {
132-
logGroup.sync();
133-
} catch (IOException e) {
134-
LOG.info("Failed to send sync event to {}", logGroup);
135-
}
136-
}
127+
checkAndSetModeAndNotify(SYNC_AND_FORWARD, SYNC);
137128
}
138129
};
139130

@@ -148,7 +139,7 @@ public void init() throws IOException {
148139
protected void processFile(Path src) throws IOException {
149140
FileSystem srcFS = replicationLogTracker.getFileSystem();
150141
FileStatus srcStat = srcFS.getFileStatus(src);
151-
long ts = EnvironmentEdgeManager.currentTimeMillis();
142+
long ts = replicationLogTracker.getFileTimestamp(srcStat.getPath());
152143
ReplicationShardDirectoryManager remoteShardManager = logGroup.getStandbyShardManager();
153144
Path dst = remoteShardManager.getWriterPath(ts, logGroup.getServerName().getServerName());
154145
long startTime = EnvironmentEdgeManager.currentTimeMillis();
@@ -160,14 +151,7 @@ protected void processFile(Path src) throws IOException {
160151
if (logGroup.getMode() == STORE_AND_FORWARD
161152
&& isLogCopyThroughputAboveThreshold(srcStat.getLen(), copyTime)) {
162153
// start recovery by switching to SYNC_AND_FORWARD
163-
if (logGroup.checkAndSetMode(STORE_AND_FORWARD, SYNC_AND_FORWARD)) {
164-
// replication mode switched, notify the event handler
165-
try {
166-
logGroup.sync();
167-
} catch (IOException e) {
168-
LOG.info("Failed to send sync event to {}", logGroup);
169-
}
170-
}
154+
checkAndSetModeAndNotify(STORE_AND_FORWARD, SYNC_AND_FORWARD);
171155
}
172156
}
173157

@@ -183,16 +167,22 @@ protected void processNoMoreRoundsLeft() throws IOException {
183167
LOG.info("Processed all the replication log files for {}", logGroup);
184168
// if this RS is still in STORE_AND_FORWARD mode like when it didn't process any file
185169
// move this RS to SYNC_AND_FORWARD
186-
if (logGroup.checkAndSetMode(STORE_AND_FORWARD, SYNC_AND_FORWARD)) {
187-
// replication mode switched, notify the event handler
170+
checkAndSetModeAndNotify(STORE_AND_FORWARD, SYNC_AND_FORWARD);
171+
172+
if (syncUpdateTS <= EnvironmentEdgeManager.currentTimeMillis()) {
188173
try {
189-
logGroup.sync();
190-
} catch (IOException e) {
191-
LOG.info("Failed to send sync event to {}", logGroup);
174+
long waitTime = logGroup.setHAGroupStatusToSync();
175+
if (waitTime != 0) {
176+
syncUpdateTS = EnvironmentEdgeManager.currentTimeMillis() + waitTime;
177+
LOG.info("HAGroup {} will try to update HA state to sync at {}",
178+
logGroup, syncUpdateTS);
179+
} else {
180+
LOG.info("HAGroup {} updated HA state to SYNC", logGroup);
181+
}
182+
} catch (Exception e) {
183+
LOG.info("Could not update status to sync for {}", logGroup, e);
192184
}
193185
}
194-
// TODO ensure the mTime on the group store record is older than the wait sync timeout
195-
logGroup.setHAGroupStatusToSync();
196186
}
197187
}
198188

@@ -205,7 +195,7 @@ protected void processNoMoreRoundsLeft() throws IOException {
205195
* @return True if the throughput is good else false
206196
*/
207197
private boolean isLogCopyThroughputAboveThreshold(long fileSize, long copyTime) {
208-
double actualThroughputBytesPerMs = copyTime != 0 ? fileSize/copyTime : 0;
198+
double actualThroughputBytesPerMs = copyTime != 0 ? ((double) fileSize)/copyTime : 0;
209199
return actualThroughputBytesPerMs >= copyThroughputThresholdBytesPerMs;
210200
}
211201

@@ -219,4 +209,20 @@ protected MetricsReplicationLogDiscovery createMetricsSource() {
219209
protected ReplicationLogTracker getReplicationLogTracker() {
220210
return replicationLogTracker;
221211
}
212+
213+
/**
214+
* Helper API to check and set the replication mode and then notify the disruptor
215+
*/
216+
private boolean checkAndSetModeAndNotify(ReplicationMode expectedMode, ReplicationMode newMode) {
217+
boolean ret = logGroup.checkAndSetMode(expectedMode, newMode);
218+
if (ret) {
219+
// replication mode switched, notify the event handler
220+
try {
221+
logGroup.sync();
222+
} catch (IOException e) {
223+
LOG.info("Failed to notify event handler for {}", logGroup, e);
224+
}
225+
}
226+
return ret;
227+
}
222228
}

phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogGroup.java

Lines changed: 4 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -858,15 +858,12 @@ protected void setHAGroupStatusToStoreAndForward() throws Exception {
858858
}
859859
}
860860

861-
protected void setHAGroupStatusToSync() throws IOException {
861+
protected long setHAGroupStatusToSync() throws Exception {
862862
try {
863-
haGroupStoreManager.setHAGroupStatusToSync(haGroupName);
864-
} catch (IOException ex) {
865-
// TODO logging
866-
throw ex;
863+
return haGroupStoreManager.setHAGroupStatusToSync(haGroupName);
867864
} catch (Exception ex) {
868-
// TODO logging
869-
throw new IOException(ex);
865+
LOG.info("HAGroup {} failed to set status to SYNC", this, ex);
866+
throw ex;
870867
}
871868
}
872869

phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarderTest.java

Lines changed: 32 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -82,7 +82,7 @@ public Object answer(InvocationOnMock invocation) {
8282
} catch (IOException e) {
8383
throw new RuntimeException(e);
8484
}
85-
return null;
85+
return 0L;
8686
}
8787
}).when(haGroupStoreManager).setHAGroupStatusToSync(haGroupName);
8888

@@ -141,4 +141,35 @@ public Boolean call() throws Exception {
141141
executor.shutdownNow();
142142
}
143143
}
144+
145+
@Test
146+
public void testSyncModeUpdateWaitTime() throws Exception {
147+
final long[] waitTime = {8L};
148+
int roundDurationSeconds =
149+
logGroup.getFallbackShardManager().getReplicationRoundDurationSeconds();
150+
151+
doAnswer(new Answer<Object>() {
152+
@Override
153+
public Object answer(InvocationOnMock invocation) {
154+
long ret = 0L;
155+
if (waitTime[0] > 0) {
156+
ret = waitTime[0];
157+
// reset to 0
158+
waitTime[0] = 0;
159+
} else {
160+
// explicitly set the replication mode to SYNC
161+
logGroup.setMode(SYNC);
162+
try {
163+
logGroup.sync();
164+
} catch (IOException e) {
165+
throw new RuntimeException(e);
166+
}
167+
}
168+
return ret;
169+
}
170+
}).when(haGroupStoreManager).setHAGroupStatusToSync(haGroupName);
171+
Thread.sleep(roundDurationSeconds * 4 * 1000);
172+
// we should have switched back to the SYNC mode
173+
assertEquals(SYNC, logGroup.getMode());
174+
}
144175
}

0 commit comments

Comments
 (0)