Skip to content

Commit 8e653ca

Browse files
committed
HBASE-22810 Initialize an separate ThreadPoolExecutor for taking/restoring snapshot
1 parent 8c1edb3 commit 8e653ca

File tree

4 files changed

+38
-7
lines changed

4 files changed

+38
-7
lines changed

hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java

Lines changed: 2 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -206,13 +206,13 @@ public enum EventType {
206206
* C_M_SNAPSHOT_TABLE<br>
207207
* Client asking Master to snapshot an offline table.
208208
*/
209-
C_M_SNAPSHOT_TABLE (48, ExecutorType.MASTER_TABLE_OPERATIONS),
209+
C_M_SNAPSHOT_TABLE (48, ExecutorType.MASTER_SNAPSHOT_OPERATIONS),
210210
/**
211211
* Messages originating from Client to Master.<br>
212212
* C_M_RESTORE_SNAPSHOT<br>
213213
* Client asking Master to restore a snapshot.
214214
*/
215-
C_M_RESTORE_SNAPSHOT (49, ExecutorType.MASTER_TABLE_OPERATIONS),
215+
C_M_RESTORE_SNAPSHOT (49, ExecutorType.MASTER_SNAPSHOT_OPERATIONS),
216216

217217
// Updates from master to ZK. This is done by the master and there is
218218
// nothing to process by either Master or RS
@@ -321,11 +321,6 @@ public static EventType get(final int code) {
321321
throw new IllegalArgumentException("Unknown code " + code);
322322
}
323323

324-
public boolean isOnlineSchemaChangeSupported() {
325-
return this.equals(EventType.C_M_ADD_FAMILY) || this.equals(EventType.C_M_DELETE_FAMILY) ||
326-
this.equals(EventType.C_M_MODIFY_FAMILY) || this.equals(EventType.C_M_MODIFY_TABLE);
327-
}
328-
329324
ExecutorType getExecutorServiceType() {
330325
return this.executor;
331326
}

hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@ public enum ExecutorType {
3434
MASTER_RS_SHUTDOWN (5),
3535
MASTER_META_SERVER_OPERATIONS (6),
3636
M_LOG_REPLAY_OPS (7),
37+
MASTER_SNAPSHOT_OPERATIONS (8),
3738

3839
// RegionServer executor services
3940
RS_OPEN_REGION (20),

hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1431,6 +1431,8 @@ private void startServiceThreads() throws IOException{
14311431
conf.getInt("hbase.master.executor.meta.serverops.threads", 5));
14321432
this.executorService.startExecutorService(ExecutorType.M_LOG_REPLAY_OPS,
14331433
conf.getInt("hbase.master.executor.logreplayops.threads", 10));
1434+
this.executorService.startExecutorService(ExecutorType.MASTER_SNAPSHOT_OPERATIONS,
1435+
conf.getInt("hbase.master.executor.snapshot.threads", 3));
14341436

14351437
// We depend on there being only one instance of this executor running
14361438
// at a time. To do concurrency, would need fencing of enable/disable of

hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java

Lines changed: 33 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@
2828

2929
import java.io.IOException;
3030
import java.io.StringWriter;
31+
import java.util.concurrent.CountDownLatch;
3132
import java.util.concurrent.ThreadPoolExecutor;
3233
import java.util.concurrent.atomic.AtomicBoolean;
3334
import java.util.concurrent.atomic.AtomicInteger;
@@ -41,6 +42,7 @@
4142
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorStatus;
4243
import org.apache.hadoop.hbase.testclassification.MiscTests;
4344
import org.apache.hadoop.hbase.testclassification.SmallTests;
45+
import org.junit.Assert;
4446
import org.junit.ClassRule;
4547
import org.junit.Test;
4648
import org.junit.experimental.categories.Category;
@@ -219,5 +221,36 @@ public boolean evaluate() throws Exception {
219221
executorService.shutdown();
220222
}
221223

224+
@Test
225+
public void testSnapshotHandlers() throws Exception {
226+
final Configuration conf = HBaseConfiguration.create();
227+
final Server server = mock(Server.class);
228+
when(server.getConfiguration()).thenReturn(conf);
229+
230+
ExecutorService executorService = new ExecutorService("testSnapshotHandlers");
231+
executorService.startExecutorService(ExecutorType.MASTER_SNAPSHOT_OPERATIONS, 1);
232+
233+
CountDownLatch latch = new CountDownLatch(1);
234+
executorService.submit(new EventHandler(server, EventType.C_M_SNAPSHOT_TABLE) {
235+
@Override
236+
public void process() throws IOException {
237+
try {
238+
latch.await();
239+
} catch (InterruptedException e) {
240+
Thread.currentThread().interrupt();
241+
}
242+
}
243+
});
244+
245+
int activeCount = executorService.getExecutor(ExecutorType.MASTER_SNAPSHOT_OPERATIONS)
246+
.getThreadPoolExecutor().getActiveCount();
247+
Assert.assertEquals(activeCount, 1);
248+
latch.countDown();
249+
Waiter.waitFor(conf, 3000, () -> {
250+
int count = executorService.getExecutor(ExecutorType.MASTER_SNAPSHOT_OPERATIONS)
251+
.getThreadPoolExecutor().getActiveCount();
252+
return count == 0;
253+
});
254+
}
222255
}
223256

0 commit comments

Comments
 (0)