HBase
  1. HBase
  2. HBASE-7290 Online snapshots
  3. HBASE-7651

RegionServerSnapshotManager fails with CancellationException if previous snapshot fails in per region task

    Details

    • Type: Sub-task Sub-task
    • Status: Closed
    • Priority: Blocker Blocker
    • Resolution: Fixed
    • Affects Version/s: hbase-7290
    • Fix Version/s: 0.95.0
    • Component/s: snapshots
    • Labels:
      None
    • Hadoop Flags:
      Reviewed

      Description

      I've reproduced this problem consistently on a 20 node cluster.

      The first run fails on a node (jon-snaphots-2 in this case) to take snapshot due to a NotServingRegionException (this is acceptable)

      2013-01-23 13:32:48,631 DEBUG org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher:  accepting received exception
      org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable via jon-snapshots-2.ent.cloudera.com,22101,1358976524369:org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable: org.apache.hadoop.hbase.NotServingRegionException: TestTable,0002493652,1358976652443.b858147ad87a7812ac9a73dd8fef36ad. is closing
              at org.apache.hadoop.hbase.errorhandling.ForeignException.deserialize(ForeignException.java:184)
              at org.apache.hadoop.hbase.procedure.ZKProcedureCoordinatorRpcs.abort(ZKProcedureCoordinatorRpcs.java:240)
              at org.apache.hadoop.hbase.procedure.ZKProcedureCoordinatorRpcs$1.nodeCreated(ZKProcedureCoordinatorRpcs.java:182)
              at org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeeperWatcher.java:294)
              at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:519)
              at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:495)
      Caused by: org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable: org.apache.hadoop.hbase.NotServingRegionException: TestTable,0002493652,1358976652443.b858147ad87a7812ac9a73dd8fef36ad. is closing
              at org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager$SnapshotSubprocedurePool.waitForOutstandingTasks(RegionServerSnapshotManager.java:343)
              at org.apache.hadoop.hbase.regionserver.snapshot.FlushSnapshotSubprocedure.flushSnapshot(FlushSnapshotSubprocedure.java:107)
              at org.apache.hadoop.hbase.regionserver.snapshot.FlushSnapshotSubprocedure.insideBarrier(FlushSnapshotSubprocedure.java:123)
              at org.apache.hadoop.hbase.procedure.Subprocedure.call(Subprocedure.java:181)
              at org.apache.hadoop.hbase.procedure.Subprocedure.call(Subprocedure.java:52)
              at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
              at java.util.concurrent.FutureTask.run(FutureTask.java:138)
              at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
              at java.lang.Thread.run(Thread.java:662)
      2013-01-23 13:32:48,631 DEBUG org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher:  Recieved error, notifying listeners...
      2013-01-23 13:32:48,730 ERROR org.apache.hadoop.hbase.procedure.Procedure: Procedure 'pe-6' execution failed!
      org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable via jon-snapshots-2.ent.cloudera.com,22101,1358976524369:org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable: org.apache.hadoop.hbase.NotServingRegionException: TestTable,0002493652,1358976652443.b858147ad87a7812ac9a73dd8fef36ad. is closing
              at org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher.rethrowException(ForeignExceptionDispatcher.java:84)
              at org.apache.hadoop.hbase.procedure.Procedure.waitForLatch(Procedure.java:357)
              at org.apache.hadoop.hbase.procedure.Procedure.call(Procedure.java:203)
              at org.apache.hadoop.hbase.procedure.Procedure.call(Procedure.java:68)
              at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
              at java.util.concurrent.FutureTask.run(FutureTask.java:138)
              at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
              at java.lang.Thread.run(Thread.java:662)
      Caused by: org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable: org.apache.hadoop.hbase.NotServingRegionException: TestTable,0002493652,1358976652443.b858147ad87a7812ac9a73dd8fef36ad. is closing
              at org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager$SnapshotSubprocedurePool.waitForOutstandingTasks(RegionServerSnapshotManager.java:343)
              at org.apache.hadoop.hbase.regionserver.snapshot.FlushSnapshotSubprocedure.flushSnapshot(FlushSnapshotSubprocedure.java:107)
              at org.apache.hadoop.hbase.regionserver.snapshot.FlushSnapshotSubprocedure.insideBarrier(FlushSnapshotSubprocedure.java:123)
              at org.apache.hadoop.hbase.procedure.Subprocedure.call(Subprocedure.java:181)
              at org.apache.hadoop.hbase.procedure.Subprocedure.call(Subprocedure.java:52)
              ... 5 more
      

      Subsequent snapshot attempts that require jon-snapshot-2 to participate fail like this:

      org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable via jon-snapshots-2.ent.cloudera.com,22101,1358976524369:org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable: java.util.concurrent.CancellationException
              at org.apache.hadoop.hbase.errorhandling.ForeignException.deserialize(ForeignException.java:184)
              at org.apache.hadoop.hbase.procedure.ZKProcedureCoordinatorRpcs.abort(ZKProcedureCoordinatorRpcs.java:240)
              at org.apache.hadoop.hbase.procedure.ZKProcedureCoordinatorRpcs$1.nodeCreated(ZKProcedureCoordinatorRpcs.java:182)
              at org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeeperWatcher.java:294)
              at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:519)
              at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:495)
      Caused by: org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable: java.util.concurrent.CancellationException
              at org.apache.hadoop.hbase.procedure.Subprocedure.cancel(Subprocedure.java:270)
              at org.apache.hadoop.hbase.procedure.Subprocedure.call(Subprocedure.java:202)
              at org.apache.hadoop.hbase.procedure.Subprocedure.call(Subprocedure.java:52)
              at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
              at java.util.concurrent.FutureTask.run(FutureTask.java:138)
              at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
              at java.lang.Thread.run(Thread.java:662)
      2013-01-23 13:32:59,557 DEBUG org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher:  Recieved error, notifying listeners...
      2013-01-23 13:32:59,810 ERROR org.apache.hadoop.hbase.procedure.Procedure: Procedure 'pe-7' execution failed!
      org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable via jon-snapshots-2.ent.cloudera.com,22101,1358976524369:org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable: java.util.concurrent.CancellationException
              at org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher.rethrowException(ForeignExceptionDispatcher.java:84)
              at org.apache.hadoop.hbase.procedure.Procedure.waitForLatch(Procedure.java:357)
              at org.apache.hadoop.hbase.procedure.Procedure.call(Procedure.java:203)
              at org.apache.hadoop.hbase.procedure.Procedure.call(Procedure.java:68)
              at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
              at java.util.concurrent.FutureTask.run(FutureTask.java:138)
              at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
              at java.lang.Thread.run(Thread.java:662)
      Caused by: org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable: java.util.concurrent.CancellationException
              at org.apache.hadoop.hbase.procedure.Subprocedure.cancel(Subprocedure.java:270)
              at org.apache.hadoop.hbase.procedure.Subprocedure.call(Subprocedure.java:202)
              at org.apache.hadoop.hbase.procedure.Subprocedure.call(Subprocedure.java:52)
              ... 5 more
      
      1. hbase-7651.patch
        20 kB
        Jonathan Hsieh
      2. hbase-7651.v2.patch
        13 kB
        Jonathan Hsieh

        Activity

        Hide
        Jonathan Hsieh added a comment -

        NSRE's are possible with this snapshotting implementation (master gets a list of regions/regionservers to care about, regions move, and then the snapshotting request is sent to the rs's.)

        Restarting the particular node (jon-snapshot-2 from the example) fixes the problem but when the next NSRE pops up elsewhere we get stuck again.

        Show
        Jonathan Hsieh added a comment - NSRE's are possible with this snapshotting implementation (master gets a list of regions/regionservers to care about, regions move, and then the snapshotting request is sent to the rs's.) Restarting the particular node (jon-snapshot-2 from the example) fixes the problem but when the next NSRE pops up elsewhere we get stuck again.
        Hide
        Ted Yu added a comment -

        Line 343 in RegionServerSnapshotManager#waitForOutstandingTasks():

                    LOG.warn("cancelling region task");
                    f.cancel(true);
        

        Shall we pass false to cancel() ?

        Show
        Ted Yu added a comment - Line 343 in RegionServerSnapshotManager#waitForOutstandingTasks(): LOG.warn( "cancelling region task" ); f.cancel( true ); Shall we pass false to cancel() ?
        Hide
        Jonathan Hsieh added a comment -

        I took a bit of digging but I have a fix that I'm currently testing.

        Root cause: We have a shared SnapshotSubprocedurePool object per region server for all snapshots instead of one per snapshot per regionserver. Internally it uses a java.util.concurrent.ExecutorCompletionService that has its own java.util.concurrent.ThreadPoolExecutor. We were actually getting cancelled snapshot region tasks from the previous snapshots request!

        The SnasphotSubprocedurePool needs to be drained of all Futures – when cancelling we still need to pull them out of the completion queue. Previously we we only pulled the successful ones – any others would remain and the next snapshot request would enqueue more snapshot region tasks but fail right a way pulling out the old cancelled tasks. Since the completion queue is filled as tasks are completed/cancelled, we never drain the thing completely.

        I have a rough draft of the patch running on a 20 node cluster currently and results are promising so far. I'll polish and post after it goes for a few hours.

        Show
        Jonathan Hsieh added a comment - I took a bit of digging but I have a fix that I'm currently testing. Root cause: We have a shared SnapshotSubprocedurePool object per region server for all snapshots instead of one per snapshot per regionserver. Internally it uses a java.util.concurrent.ExecutorCompletionService that has its own java.util.concurrent.ThreadPoolExecutor. We were actually getting cancelled snapshot region tasks from the previous snapshots request! The SnasphotSubprocedurePool needs to be drained of all Futures – when cancelling we still need to pull them out of the completion queue. Previously we we only pulled the successful ones – any others would remain and the next snapshot request would enqueue more snapshot region tasks but fail right a way pulling out the old cancelled tasks. Since the completion queue is filled as tasks are completed/cancelled, we never drain the thing completely. I have a rough draft of the patch running on a 20 node cluster currently and results are promising so far. I'll polish and post after it goes for a few hours.
        Hide
        Jonathan Hsieh added a comment -

        Here's a test program I wrote to understand the sematnics better.

        public class TestTaskCancels {
          private static final Log LOG = LogFactory.getLog(TestTaskCancels.class);
        
          @Test
          public void testCompletionServiceCancellationSemantics() throws InterruptedException { 
            
            ThreadPoolExecutor executor = new ThreadPoolExecutor(1, 3, 60000, TimeUnit.MILLISECONDS,
                new LinkedBlockingQueue<Runnable>(), new DaemonThreadFactory("pool"));
            ExecutorCompletionService<Void> taskPool = new ExecutorCompletionService<Void>(executor);
        
            Callable<Void> failFast = new Callable<Void>() {
              @Override
              public Void call() throws Exception {
                Thread.sleep(500);
                LOG.info("failing first");
                throw new RuntimeException("failing first");
              }
            };
            
            int count = 5;
            Callable<Void>[] succeedSlows = new Callable[count];
            for (int i = 0; i < count ; i++) {
              final int j = i;
              succeedSlows[i] = new Callable<Void>() {
                @Override
                public Void call() throws Exception {
                  LOG.info(j + ": started right a way but");
                  Thread.sleep(1000);
                  LOG.info(j + ": succeed after 1s");
                  return null;
                }
              };
            }
        
            Collection<Future<Void>> tasks = new ArrayList<Future<Void>>();
            tasks.add(taskPool.submit(failFast));
            for (int i = 0; i < count; i++) {
              tasks.add(taskPool.submit(succeedSlows[i]));
            }
            
            try {
              for (int i=0; i< count +1; i++) {
                Future<Void> f = taskPool.take();
                f.get();
              }
            } catch (InterruptedException e) {
              LOG.warn("Got InterruptedException in SnapshotSubprocedurePool", e);
            } catch (ExecutionException e) {
              LOG.warn("Got ExecutionException in SnapshotSubprocedurePool", e.getCause());
            } finally {
              for (Future<Void> f: tasks) {
                if (!f.isDone()){
                  LOG.warn("cancelling region task");
                  f.cancel(false);
                }
              }
            }
            
            // kind of a hack -- we can't get the number of remaining elements in the taskPool queue.
            Thread.sleep(100);
            
            int polled = 0;
            while(taskPool.poll() != null) {
              LOG.info("polled out a task");
              polled ++ ;
            }
            LOG.info("State of task pool: " + taskPool);
            assertEquals(5, polled);
          }
        }
        
        Show
        Jonathan Hsieh added a comment - Here's a test program I wrote to understand the sematnics better. public class TestTaskCancels { private static final Log LOG = LogFactory.getLog(TestTaskCancels.class); @Test public void testCompletionServiceCancellationSemantics() throws InterruptedException { ThreadPoolExecutor executor = new ThreadPoolExecutor(1, 3, 60000, TimeUnit.MILLISECONDS, new LinkedBlockingQueue< Runnable >(), new DaemonThreadFactory( "pool" )); ExecutorCompletionService< Void > taskPool = new ExecutorCompletionService< Void >(executor); Callable< Void > failFast = new Callable< Void >() { @Override public Void call() throws Exception { Thread .sleep(500); LOG.info( "failing first" ); throw new RuntimeException( "failing first" ); } }; int count = 5; Callable< Void >[] succeedSlows = new Callable[count]; for ( int i = 0; i < count ; i++) { final int j = i; succeedSlows[i] = new Callable< Void >() { @Override public Void call() throws Exception { LOG.info(j + ": started right a way but" ); Thread .sleep(1000); LOG.info(j + ": succeed after 1s" ); return null ; } }; } Collection<Future< Void >> tasks = new ArrayList<Future< Void >>(); tasks.add(taskPool.submit(failFast)); for ( int i = 0; i < count; i++) { tasks.add(taskPool.submit(succeedSlows[i])); } try { for ( int i=0; i< count +1; i++) { Future< Void > f = taskPool.take(); f.get(); } } catch (InterruptedException e) { LOG.warn( "Got InterruptedException in SnapshotSubprocedurePool" , e); } catch (ExecutionException e) { LOG.warn( "Got ExecutionException in SnapshotSubprocedurePool" , e.getCause()); } finally { for (Future< Void > f: tasks) { if (!f.isDone()){ LOG.warn( "cancelling region task" ); f.cancel( false ); } } } // kind of a hack -- we can't get the number of remaining elements in the taskPool queue. Thread .sleep(100); int polled = 0; while (taskPool.poll() != null ) { LOG.info( "polled out a task" ); polled ++ ; } LOG.info( "State of task pool: " + taskPool); assertEquals(5, polled); } }
        Hide
        Jonathan Hsieh added a comment -

        First cut of patch. Need to add unit test. Something very similar to this has been testing on 20 node cluster for several hours. Not all snapshot succeed but the snapshotting mechanism no longer gets stuck.

        Show
        Jonathan Hsieh added a comment - First cut of patch. Need to add unit test. Something very similar to this has been testing on 20 node cluster for several hours. Not all snapshot succeed but the snapshotting mechanism no longer gets stuck.
        Hide
        Matteo Bertozzi added a comment -

        +1, the core of this patch is having one SubprocedurePool per Subprocedure

             switch (snapshot.getType()) {
             case FLUSH:
        +      SnapshotSubprocedurePool taskManager =
        +        new SnapshotSubprocedurePool(rss.getServerName().toString(), conf);
               return new FlushSnapshotSubprocedure(member, exnDispatcher, wakeMillis,
                   timeoutMillis, involvedRegions, snapshot, taskManager);
        
        Show
        Matteo Bertozzi added a comment - +1, the core of this patch is having one SubprocedurePool per Subprocedure switch (snapshot.getType()) { case FLUSH: + SnapshotSubprocedurePool taskManager = + new SnapshotSubprocedurePool(rss.getServerName().toString(), conf); return new FlushSnapshotSubprocedure(member, exnDispatcher, wakeMillis, timeoutMillis, involvedRegions, snapshot, taskManager);
        Hide
        Ted Yu added a comment -

        I think the above change is fine - considering we're going to support concurrent snapshots in the future.

        In cleanup():

        +    } catch (InterruptedException e1) {
        +      // TODO currently eating this exception, think about this.
        

        Do you want to perform some action above (such as restoring interrupt status) ?

        There're a lot of new log statements added in this patch. Can you remove some ?

        Thanks for fixing this bug.

        Show
        Ted Yu added a comment - I think the above change is fine - considering we're going to support concurrent snapshots in the future. In cleanup(): + } catch (InterruptedException e1) { + // TODO currently eating this exception, think about this . Do you want to perform some action above (such as restoring interrupt status) ? There're a lot of new log statements added in this patch. Can you remove some ? Thanks for fixing this bug.
        Hide
        Jonathan Hsieh added a comment -

        thanks for the reminder on the TODO – missed it on the clean up pass. I'll restore the interrupt flag. I'll pull the unrelated logs/spelling fixes to this into another patch.

        If you all don't mind, I'm going to punt on writing a unit test for this – I've spent an hour or two on this and it is non-trivial. Instead, here's the an excerpt of the system level test I've bene using that triggered the problem (it happened usually within 10 snapshot attempts on a 20 node cluster).

        SSCOUNT=30
        
        # start loading data
        hbase org.apache.hadoop.hbase.PerformanceEvaluation --rows=$ROWS randomWrite 1 2> report.perf-eval.err &
        
        # sleep a little to let the table be created
        sleep 15
        
        # fail error if Perf eval table is missing.
        echo "Kernel.exit 127 if list('TestTable').size == 0" | hbase shell
        
        echo "==== take snapshots"
        # take snapshots
        # pe-snapshots.sh
        echo "(1..$SSCOUNT).map {|x| snapshot 'TestTable', 'pe-'+x.to_s}" | hbase shell
        # fail error if no snapshot are taken, some may fail (this is ok)
        echo "Kernel.exit 127 if list_snapshots('pe-.*').size == 0" | hbase shell
        
        Show
        Jonathan Hsieh added a comment - thanks for the reminder on the TODO – missed it on the clean up pass. I'll restore the interrupt flag. I'll pull the unrelated logs/spelling fixes to this into another patch. If you all don't mind, I'm going to punt on writing a unit test for this – I've spent an hour or two on this and it is non-trivial. Instead, here's the an excerpt of the system level test I've bene using that triggered the problem (it happened usually within 10 snapshot attempts on a 20 node cluster). SSCOUNT=30 # start loading data hbase org.apache.hadoop.hbase.PerformanceEvaluation --rows=$ROWS randomWrite 1 2> report.perf-eval.err & # sleep a little to let the table be created sleep 15 # fail error if Perf eval table is missing. echo "Kernel.exit 127 if list('TestTable').size == 0" | hbase shell echo "==== take snapshots" # take snapshots # pe-snapshots.sh echo "(1..$SSCOUNT).map {|x| snapshot 'TestTable', 'pe-'+x.to_s}" | hbase shell # fail error if no snapshot are taken, some may fail ( this is ok) echo "Kernel.exit 127 if list_snapshots('pe-.*').size == 0" | hbase shell
        Hide
        Ted Yu added a comment -

        What's the content of pe-snapshots.sh ?

        What are (typical) values of list_snapshots('pe-.*').size during the test run ?

        Thanks

        Show
        Ted Yu added a comment - What's the content of pe-snapshots.sh ? What are (typical) values of list_snapshots('pe-.*').size during the test run ? Thanks
        Hide
        Jonathan Hsieh added a comment - - edited

        pe-snapshots.sh was the line right below it:

        echo "(1..$SSCOUNT).map {|x| snapshot 'TestTable', 'pe-'+x.to_s}" | hbase shell
        

        list_snapshots lists the names of the snapshots that matche the regex – so we try to take SSCOUNT=30 snapshots call pe-1, pe-2...pe-30, and then list all that match pe-*. This fails only if there are not snapshots. In the logs we'd see the first log from the description and then all others would fail with essentially the second log in the description.

        Show
        Jonathan Hsieh added a comment - - edited pe-snapshots.sh was the line right below it: echo "(1..$SSCOUNT).map {|x| snapshot 'TestTable', 'pe-'+x.to_s}" | hbase shell list_snapshots lists the names of the snapshots that matche the regex – so we try to take SSCOUNT=30 snapshots call pe-1, pe-2...pe-30, and then list all that match pe-*. This fails only if there are not snapshots. In the logs we'd see the first log from the description and then all others would fail with essentially the second log in the description.
        Hide
        Jonathan Hsieh added a comment -

        (I used to do this manually, but now this is part of a automatic rig.)

        Show
        Jonathan Hsieh added a comment - (I used to do this manually, but now this is part of a automatic rig.)
        Hide
        Ted Yu added a comment -

        then all others would fail with essentially the second log in the description.

        With the patch, the second log wouldn't appear, right ?

        +1 on patch after cleaning up log statements.

        Show
        Ted Yu added a comment - then all others would fail with essentially the second log in the description. With the patch, the second log wouldn't appear, right ? +1 on patch after cleaning up log statements.
        Hide
        Jonathan Hsieh added a comment -

        v2 just contains the core changes and log fixes from the files where core changes occured.

        Show
        Jonathan Hsieh added a comment - v2 just contains the core changes and log fixes from the files where core changes occured.
        Hide
        Jonathan Hsieh added a comment -

        correct – the second log does not show up anymore – if there is a problem in a RS's region task, it exits in a clean enough way to that future snapshots requests can succeed. (before the patch, all subsequent snapshotting requests would get stuck).

        Show
        Jonathan Hsieh added a comment - correct – the second log does not show up anymore – if there is a problem in a RS's region task, it exits in a clean enough way to that future snapshots requests can succeed. (before the patch, all subsequent snapshotting requests would get stuck).
        Hide
        Jonathan Hsieh added a comment -

        I've committed to the snapshots branch.

        Show
        Jonathan Hsieh added a comment - I've committed to the snapshots branch.
        Hide
        Ted Yu added a comment -

        In class SnapshotSubprocedurePool:

              executor = new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.MILLISECONDS,
                  new LinkedBlockingQueue<Runnable>(), new DaemonThreadFactory("rs("
                      + name + ")-snapshot-pool"));
        

        To support concurrent snapshots, we can consider giving the executor number of threads proportional to number of regions involved in the snapshot.

        Show
        Ted Yu added a comment - In class SnapshotSubprocedurePool: executor = new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.MILLISECONDS, new LinkedBlockingQueue< Runnable >(), new DaemonThreadFactory( "rs(" + name + ")-snapshot-pool" )); To support concurrent snapshots, we can consider giving the executor number of threads proportional to number of regions involved in the snapshot.
        Hide
        Jonathan Hsieh added a comment -

        If we want to change it, let's file a follow on jira to change it.. I believe it could be helpful.

        Show
        Jonathan Hsieh added a comment - If we want to change it, let's file a follow on jira to change it.. I believe it could be helpful.
        Hide
        Ted Yu added a comment -

        In your test runs, how much time did taking / restoring snapshot operation last ?

        Show
        Ted Yu added a comment - In your test runs, how much time did taking / restoring snapshot operation last ?
        Hide
        Jonathan Hsieh added a comment -

        On a loaded 5 node cluster taking an online flush snapshot takes around 2-10 seconds. Cloning is much more variable 5-10 seconds with spikes the order of a minute. I believe much of this time because enabling is slow (often triggers compactions).

        On the larger cluster it is closer to 5-30s to take, and 10-25s with a few that take minutes to clone. It is not as robust on the larger case – I'm actually getting quite a few timeouts and am investigating them further.

        Show
        Jonathan Hsieh added a comment - On a loaded 5 node cluster taking an online flush snapshot takes around 2-10 seconds. Cloning is much more variable 5-10 seconds with spikes the order of a minute. I believe much of this time because enabling is slow (often triggers compactions). On the larger cluster it is closer to 5-30s to take, and 10-25s with a few that take minutes to clone. It is not as robust on the larger case – I'm actually getting quite a few timeouts and am investigating them further.
        Hide
        stack added a comment -

        Marking closed.

        Show
        stack added a comment - Marking closed.

          People

          • Assignee:
            Jonathan Hsieh
            Reporter:
            Jonathan Hsieh
          • Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development