View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.procedure2;
20  
21  import java.io.IOException;
22  import java.io.InputStream;
23  import java.io.OutputStream;
24  
25  import org.apache.hadoop.hbase.classification.InterfaceAudience;
26  import org.apache.hadoop.hbase.classification.InterfaceStability;
27  import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData;
28  
29  /**
30   * A SequentialProcedure describes one step in a procedure chain.
31   *   -> Step 1 -> Step 2 -> Step 3
32   *
33   * The main difference from a base Procedure is that the execute() of a
34   * SequentialProcedure will be called only once, there will be no second
35   * execute() call once the child are finished. which means once the child
36   * of a SequentialProcedure are completed the SequentialProcedure is completed too.
37   */
38  @InterfaceAudience.Private
39  @InterfaceStability.Evolving
40  public abstract class SequentialProcedure<TEnvironment> extends Procedure<TEnvironment> {
41    private boolean executed = false;
42  
43    @Override
44    protected Procedure[] doExecute(final TEnvironment env)
45        throws ProcedureYieldException, InterruptedException {
46      updateTimestamp();
47      try {
48        Procedure[] children = !executed ? execute(env) : null;
49        executed = !executed;
50        return children;
51      } finally {
52        updateTimestamp();
53      }
54    }
55  
56    @Override
57    protected void doRollback(final TEnvironment env)
58        throws IOException, InterruptedException {
59      updateTimestamp();
60      if (executed) {
61        try {
62          rollback(env);
63          executed = !executed;
64        } finally {
65          updateTimestamp();
66        }
67      }
68    }
69  
70    @Override
71    protected void serializeStateData(final OutputStream stream) throws IOException {
72      SequentialProcedureData.Builder data = SequentialProcedureData.newBuilder();
73      data.setExecuted(executed);
74      data.build().writeDelimitedTo(stream);
75    }
76  
77    @Override
78    protected void deserializeStateData(final InputStream stream) throws IOException {
79      SequentialProcedureData data = SequentialProcedureData.parseDelimitedFrom(stream);
80      executed = data.getExecuted();
81    }
82  }