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  package org.apache.hadoop.hbase;
19  
20  import java.io.IOException;
21  
22  import org.apache.hadoop.hbase.classification.InterfaceAudience;
23  import org.apache.hadoop.hbase.exceptions.DeserializationException;
24  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
25  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
26  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
27  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
28  import org.apache.hadoop.hbase.util.Bytes;
29  
30  /**
31   * State of a WAL log split during distributed splitting.  State is kept up in zookeeper.
32   * Encapsulates protobuf serialization/deserialization so we don't leak generated pb outside of
33   * this class.  Used by regionserver and master packages.
34   * <p>Immutable
35   */
36  @InterfaceAudience.Private
37  public class SplitLogTask {
38    private final ServerName originServer;
39    private final ZooKeeperProtos.SplitLogTask.State state;
40    private final ZooKeeperProtos.SplitLogTask.RecoveryMode mode;
41  
42    public static class Unassigned extends SplitLogTask {
43      public Unassigned(final ServerName originServer, final RecoveryMode mode) {
44        super(originServer, ZooKeeperProtos.SplitLogTask.State.UNASSIGNED, mode);
45      }
46    }
47  
48    public static class Owned extends SplitLogTask {
49      public Owned(final ServerName originServer, final RecoveryMode mode) {
50        super(originServer, ZooKeeperProtos.SplitLogTask.State.OWNED, mode);
51      }
52    }
53  
54    public static class Resigned extends SplitLogTask {
55      public Resigned(final ServerName originServer, final RecoveryMode mode) {
56        super(originServer, ZooKeeperProtos.SplitLogTask.State.RESIGNED, mode);
57      }
58    }
59  
60    public static class Done extends SplitLogTask {
61      public Done(final ServerName originServer, final RecoveryMode mode) {
62        super(originServer, ZooKeeperProtos.SplitLogTask.State.DONE, mode);
63      }
64    }
65  
66    public static class Err extends SplitLogTask {
67      public Err(final ServerName originServer, final RecoveryMode mode) {
68        super(originServer, ZooKeeperProtos.SplitLogTask.State.ERR, mode);
69      }
70    }
71  
72    SplitLogTask(final ZooKeeperProtos.SplitLogTask slt) {
73      this.originServer = ProtobufUtil.toServerName(slt.getServerName());
74      this.state = slt.getState();
75      this.mode = (slt.hasMode()) ? slt.getMode() : 
76        ZooKeeperProtos.SplitLogTask.RecoveryMode.UNKNOWN;
77    }
78  
79    SplitLogTask(final ServerName originServer, final ZooKeeperProtos.SplitLogTask.State state,
80        final ZooKeeperProtos.SplitLogTask.RecoveryMode mode) {
81      this.originServer = originServer;
82      this.state = state;
83      this.mode = mode;
84    }
85  
86    public ServerName getServerName() {
87      return this.originServer;
88    }
89    
90    public ZooKeeperProtos.SplitLogTask.RecoveryMode getMode() {
91      return this.mode;
92    }
93  
94    public boolean isUnassigned(final ServerName sn) {
95      return this.originServer.equals(sn) && isUnassigned();
96    }
97  
98    public boolean isUnassigned() {
99      return this.state == ZooKeeperProtos.SplitLogTask.State.UNASSIGNED;
100   }
101 
102   public boolean isOwned(final ServerName sn) {
103     return this.originServer.equals(sn) && isOwned();
104   }
105 
106   public boolean isOwned() {
107     return this.state == ZooKeeperProtos.SplitLogTask.State.OWNED;
108   }
109 
110   public boolean isResigned(final ServerName sn) {
111     return this.originServer.equals(sn) && isResigned();
112   }
113 
114   public boolean isResigned() {
115     return this.state == ZooKeeperProtos.SplitLogTask.State.RESIGNED;
116   }
117 
118   public boolean isDone(final ServerName sn) {
119     return this.originServer.equals(sn) && isDone();
120   }
121 
122   public boolean isDone() {
123     return this.state == ZooKeeperProtos.SplitLogTask.State.DONE;
124   }
125 
126   public boolean isErr(final ServerName sn) {
127     return this.originServer.equals(sn) && isErr();
128   }
129 
130   public boolean isErr() {
131     return this.state == ZooKeeperProtos.SplitLogTask.State.ERR;
132   }
133 
134   @Override
135   public String toString() {
136     return this.state.toString() + " " + this.originServer.toString();
137   }
138 
139   @Override
140   public boolean equals(Object obj) {
141     if (!(obj instanceof SplitLogTask)) return false;
142     SplitLogTask other = (SplitLogTask)obj;
143     return other.state.equals(this.state) && other.originServer.equals(this.originServer);
144   }
145 
146   @Override
147   public int hashCode() {
148     int hash = 7;
149     hash = 31 * hash + this.state.hashCode();
150     return 31 * hash + this.originServer.hashCode();
151   }
152 
153   /**
154    * @param data Serialized date to parse.
155    * @return An SplitLogTaskState instance made of the passed <code>data</code>
156    * @throws DeserializationException 
157    * @see #toByteArray()
158    */
159   public static SplitLogTask parseFrom(final byte [] data) throws DeserializationException {
160     ProtobufUtil.expectPBMagicPrefix(data);
161     try {
162       int prefixLen = ProtobufUtil.lengthOfPBMagic();
163       ZooKeeperProtos.SplitLogTask.Builder builder = ZooKeeperProtos.SplitLogTask.newBuilder();
164       ProtobufUtil.mergeFrom(builder, data, prefixLen, data.length - prefixLen);
165       return new SplitLogTask(builder.build());
166     } catch (IOException e) {
167       throw new DeserializationException(Bytes.toStringBinary(data, 0, 64), e);
168     }
169   }
170 
171   /**
172    * @return This instance serialized into a byte array
173    * @see #parseFrom(byte[])
174    */
175   public byte [] toByteArray() {
176     // First create a pb ServerName.  Then create a ByteString w/ the TaskState
177     // bytes in it.  Finally create a SplitLogTaskState passing in the two
178     // pbs just created.
179     HBaseProtos.ServerName snpb = ProtobufUtil.toServerName(this.originServer);
180     ZooKeeperProtos.SplitLogTask slts =
181       ZooKeeperProtos.SplitLogTask.newBuilder().setServerName(snpb).setState(this.state).
182       setMode(this.mode).build();
183     return ProtobufUtil.prependPBMagic(slts.toByteArray());
184   }
185 }