diff --git src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java index 4121508..fef0c58 100644 --- src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java +++ src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java @@ -105,7 +105,7 @@ public abstract class EventHandler implements Runnable, Comparable { public enum EventType { // Messages originating from RS (NOTE: there is NO direct communication from // RS to Master). These are a result of RS updates into ZK. - //RS_ZK_REGION_CLOSING (1), // It is replaced by M_ZK_REGION_CLOSING(HBASE-4739) + // RS_ZK_REGION_CLOSING (1), // It is replaced by M_ZK_REGION_CLOSING(HBASE-4739) RS_ZK_REGION_CLOSED (2), // RS has finished closing a region RS_ZK_REGION_OPENING (3), // RS is in process of opening a region RS_ZK_REGION_OPENED (4), // RS has finished opening a region @@ -140,10 +140,27 @@ public abstract class EventHandler implements Runnable, Comparable { M_SERVER_SHUTDOWN (70), // Master is processing shutdown of a RS M_META_SERVER_SHUTDOWN (72); // Master is processing shutdown of RS hosting a meta region (-ROOT- or .META.). + private final int code; + /** * Constructor */ - EventType(int value) {} + EventType(final int code) { + this.code = code; + } + + public int getCode() { + return this.code; + } + + public static EventType get(final int code) { + // Is this going to be slow? Its used rare but still... + for (EventType et: EventType.values()) { + if (et.getCode() == code) return et; + } + throw new IllegalArgumentException("Unknown code " + code); + } + public boolean isOnlineSchemaChangeSupported() { return ( this.equals(EventType.C_M_ADD_FAMILY) || @@ -258,4 +275,10 @@ public abstract class EventHandler implements Runnable, Comparable { public String getInformativeName() { return this.getClass().toString(); } + + public static void main(String[] args) { + System.out.println(EventType.C_M_ADD_FAMILY.ordinal()); + System.out.println(EventType.C_M_ADD_FAMILY.getCode()); + System.out.println(EventType.get(44)); + } } diff --git src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java index 06ca377..d898f38 100644 --- src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java +++ src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java @@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.executor; import java.io.IOException; -import java.io.PrintWriter; import java.io.Writer; import java.lang.management.ThreadInfo; import java.util.List; @@ -30,8 +29,6 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.RejectedExecutionHandler; -import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -40,7 +37,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.executor.EventHandler.EventHandlerListener; -import org.apache.hadoop.hbase.executor.EventHandler.EventType; import org.apache.hadoop.hbase.monitoring.ThreadMonitoring; import com.google.common.collect.Lists; diff --git src/main/java/org/apache/hadoop/hbase/executor/RegionTransitionData.java src/main/java/org/apache/hadoop/hbase/executor/RegionTransitionData.java deleted file mode 100644 index 35d7b70..0000000 --- src/main/java/org/apache/hadoop/hbase/executor/RegionTransitionData.java +++ /dev/null @@ -1,252 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.executor; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.executor.EventHandler.EventType; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.Writable; - -/** - * Data serialized into ZooKeeper for region transitions. - */ -@InterfaceAudience.Private -public class RegionTransitionData implements Writable { - /** - * Type of transition event (offline, opening, opened, closing, closed). - * Required. - */ - private EventType eventType; - - /** Region being transitioned. Required. */ - private byte [] regionName; - - /** Server event originated from. Optional. */ - private ServerName origin; - - /** Time the event was created. Required but automatically set. */ - private long stamp; - - private byte [] payload; - - /** - * Writable constructor. Do not use directly. - */ - public RegionTransitionData() {} - - /** - * Construct data for a new region transition event with the specified event - * type and region name. - * - *

Used when the server name is not known (the master is setting it). This - * happens during cluster startup or during failure scenarios. When - * processing a failed regionserver, the master assigns the regions from that - * server to other servers though the region was never 'closed'. During - * master failover, the new master may have regions stuck in transition - * without a destination so may have to set regions offline and generate a new - * assignment. - * - *

Since only the master uses this constructor, the type should always be - * {@link EventType#M_ZK_REGION_OFFLINE}. - * - * @param eventType type of event - * @param regionName name of region as per HRegionInfo#getRegionName() - */ - public RegionTransitionData(EventType eventType, byte [] regionName) { - this(eventType, regionName, null); - } - - /** - * Construct data for a new region transition event with the specified event - * type, region name, and server name. - * - *

Used when the server name is known (a regionserver is setting it). - * - *

Valid types for this constructor are {@link EventType#M_ZK_REGION_CLOSING}, - * {@link EventType#RS_ZK_REGION_CLOSED}, {@link EventType#RS_ZK_REGION_OPENING}, - * {@link EventType#RS_ZK_REGION_SPLITTING}, - * and {@link EventType#RS_ZK_REGION_OPENED}. - * - * @param eventType type of event - * @param regionName name of region as per HRegionInfo#getRegionName() - * @param origin Originating {@link ServerName} - */ - public RegionTransitionData(EventType eventType, byte [] regionName, - final ServerName origin) { - this(eventType, regionName, origin, null); - } - - /** - * Construct data for a new region transition event with the specified event - * type, region name, and server name. - * - *

Used when the server name is known (a regionserver is setting it). - * - *

Valid types for this constructor are {@link EventType#RS_ZK_REGION_SPLIT} - * since SPLIT is only type that currently carries a payload. - * - * @param eventType type of event - * @param regionName name of region as per HRegionInfo#getRegionName() - * @param serverName Originating {@link ServerName} - * @param payload Payload examples include the daughters involved in a - * {@link EventType#RS_ZK_REGION_SPLIT}. Can be null - */ - public RegionTransitionData(EventType eventType, byte [] regionName, - final ServerName serverName, final byte [] payload) { - this.eventType = eventType; - this.stamp = System.currentTimeMillis(); - this.regionName = regionName; - this.origin = serverName; - this.payload = payload; - } - - /** - * Gets the type of region transition event. - * - *

One of: - *

    - *
  • {@link EventType#M_ZK_REGION_OFFLINE} - *
  • {@link EventType#M_ZK_REGION_CLOSING} - *
  • {@link EventType#RS_ZK_REGION_CLOSED} - *
  • {@link EventType#RS_ZK_REGION_OPENING} - *
  • {@link EventType#RS_ZK_REGION_OPENED} - *
  • {@link EventType#RS_ZK_REGION_SPLITTING} - *
  • {@link EventType#RS_ZK_REGION_SPLIT} - *
- * @return type of region transition event - */ - public EventType getEventType() { - return eventType; - } - - /** - * Gets the name of the region being transitioned. - * - *

Region name is required so this never returns null. - * @return region name, the result of a call to HRegionInfo#getRegionName() - */ - public byte [] getRegionName() { - return regionName; - } - - /** - * Gets the server the event originated from. If null, this event originated - * from the master. - * - * @return server name of originating regionserver, or null if from master - */ - public ServerName getOrigin() { - return origin; - } - - /** - * Gets the timestamp when this event was created. - * - * @return stamp event was created - */ - public long getStamp() { - return stamp; - } - - /** - * @return Payload if any. - */ - public byte [] getPayload() { - return this.payload; - } - - @Override - public void readFields(DataInput in) throws IOException { - // the event type byte - eventType = EventType.values()[in.readShort()]; - // the timestamp - stamp = in.readLong(); - // the encoded name of the region being transitioned - regionName = Bytes.readByteArray(in); - // remaining fields are optional so prefixed with boolean - // the name of the regionserver sending the data - if (in.readBoolean()) { - byte [] versionedBytes = Bytes.readByteArray(in); - this.origin = ServerName.parseVersionedServerName(versionedBytes); - } - if (in.readBoolean()) { - this.payload = Bytes.readByteArray(in); - } - } - - @Override - public void write(DataOutput out) throws IOException { - out.writeShort(eventType.ordinal()); - out.writeLong(System.currentTimeMillis()); - Bytes.writeByteArray(out, regionName); - // remaining fields are optional so prefixed with boolean - out.writeBoolean(this.origin != null); - if (this.origin != null) { - Bytes.writeByteArray(out, this.origin.getVersionedBytes()); - } - out.writeBoolean(this.payload != null); - if (this.payload != null) { - Bytes.writeByteArray(out, this.payload); - } - } - - /** - * Get the bytes for this instance. Throws a {@link RuntimeException} if - * there is an error deserializing this instance because it represents a code - * bug. - * @return binary representation of this instance - */ - public byte [] getBytes() { - try { - return Writables.getBytes(this); - } catch(IOException e) { - throw new RuntimeException(e); - } - } - - /** - * Get an instance from bytes. Throws a {@link RuntimeException} if - * there is an error serializing this instance from bytes because it - * represents a code bug. - * @param bytes binary representation of this instance - * @return instance of this class - */ - public static RegionTransitionData fromBytes(byte [] bytes) { - try { - RegionTransitionData data = new RegionTransitionData(); - Writables.getWritable(bytes, data); - return data; - } catch(IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public String toString() { - return "region=" + Bytes.toStringBinary(regionName) + ", origin=" + this.origin + - ", state=" + eventType; - } -} diff --git src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java index 15b9afe..0586267 100644 --- src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java +++ src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java @@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventHandler.EventType; import org.apache.hadoop.hbase.executor.ExecutorService; -import org.apache.hadoop.hbase.executor.RegionTransitionData; import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; import org.apache.hadoop.hbase.master.AssignmentManager.RegionState.State; import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler; @@ -490,8 +489,7 @@ public class AssignmentManager extends ZooKeeperListener { final Map>> deadServers) throws KeeperException, IOException { Stat stat = new Stat(); - RegionTransitionData data = ZKAssign.getDataAndWatch(watcher, - encodedRegionName, stat); + RegionTransitionData data = ZKAssign.getDataAndWatch(watcher, encodedRegionName, stat); if (data == null) return false; HRegionInfo hri = regionInfo; if (hri == null) { diff --git src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java index 692f194..f203e0d 100644 --- src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java +++ src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java @@ -104,7 +104,7 @@ public class MasterFileSystem { conf.getBoolean(HConstants.DISTRIBUTED_LOG_SPLITTING_KEY, true); if (this.distributedLogSplitting) { this.splitLogManager = new SplitLogManager(master.getZooKeeper(), - master.getConfiguration(), master, master.getServerName().toString()); + master.getConfiguration(), master, master.getServerName()); this.splitLogManager.finishInitialization(); } else { this.splitLogManager = null; diff --git src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java index 919c65f..9e56409 100644 --- src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java +++ src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java @@ -1,7 +1,5 @@ /** - * Copyright 2011 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one + * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file @@ -19,14 +17,52 @@ */ package org.apache.hadoop.hbase.master; -import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.*; +import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.CHECK; +import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.FORCE; +import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.DELETED; +import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.FAILURE; +import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.IN_PROGRESS; +import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.SUCCESS; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_get_data_err; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_get_data_nonode; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_get_data_queued; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_get_data_result; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_get_data_retry; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_heartbeat; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_log_split_batch_err; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_log_split_batch_start; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_log_split_batch_success; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_log_split_err; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_log_split_start; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_log_split_success; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_missing_state_in_delete; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_node_already_exists; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_node_create_err; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_node_create_queued; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_node_create_result; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_node_create_retry; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_node_delete_err; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_node_delete_queued; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_node_delete_result; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_null_data; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_orphan_task_acquired; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_rescan; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_rescan_deleted; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_resubmit; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_resubmit_dead_server_task; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_resubmit_failed; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_resubmit_threshold_reached; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_resubmit_unassigned; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_task_deleted; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_unacquired_orphan_done; +import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_wait_for_zk_delete; import java.io.IOException; import java.util.ArrayList; import java.util.HashSet; -import java.util.Set; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -43,6 +79,7 @@ import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.master.SplitLogManager.TaskFinisher.Status; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; +import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState; import org.apache.hadoop.hbase.regionserver.SplitLogWorker; import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -59,10 +96,8 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.data.Stat; -import org.apache.hadoop.hbase.zookeeper.ZKSplitLog.TaskState; -import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.*; -import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.*; +import com.google.protobuf.InvalidProtocolBufferException; /** * Distributes the task of log splitting to the available region servers. @@ -98,7 +133,7 @@ public class SplitLogManager extends ZooKeeperListener { private static final Log LOG = LogFactory.getLog(SplitLogManager.class); private final Stoppable stopper; - private final String serverName; + private final ServerName serverName; private final TaskFinisher taskFinisher; private FileSystem fs; private Configuration conf; @@ -130,7 +165,7 @@ public class SplitLogManager extends ZooKeeperListener { * @param serverName */ public SplitLogManager(ZooKeeperWatcher zkw, final Configuration conf, - Stoppable stopper, String serverName) { + Stoppable stopper, ServerName serverName) { this(zkw, conf, stopper, serverName, new TaskFinisher() { @Override public Status finish(String workerName, String logfile) { @@ -159,7 +194,7 @@ public class SplitLogManager extends ZooKeeperListener { * @param tf task finisher */ public SplitLogManager(ZooKeeperWatcher zkw, Configuration conf, - Stoppable stopper, String serverName, TaskFinisher tf) { + Stoppable stopper, ServerName serverName, TaskFinisher tf) { super(zkw); this.taskFinisher = tf; this.conf = conf; @@ -375,10 +410,42 @@ public class SplitLogManager extends ZooKeeperListener { return; } + /** + * Generate the bytes to stuff into the splitlog znode. + * @param state Current state + * @param sn Server + * @return A pb data structure serialized into a byte array with + * state and sn in it. + */ + private byte [] getZNodeData(SplitLogTaskState.State state, final ServerName sn) { + // First create a pb ServerName. Then create a ByteString w/ the TaskState + // bytes in it. Finally create a SplitLogTaskState passing in the two + // pbs just created. + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName snpb = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder(). + setHostName(sn.getHostname()).setPort(sn.getPort()).setStartCode(sn.getStartcode()).build(); + return SplitLogTaskState.newBuilder().setServer(snpb).setState(state).build().toByteArray(); + } + + + /** + * The inverse of {@link #getZNodeData(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.State, ServerName)} + * @param data Data to parse. + * @return An SplitLogTaskState instance made of the passed data + */ + private SplitLogTaskState parseZNodeData(final byte [] data) { + try { + return SplitLogTaskState.parseFrom(data); + } catch (InvalidProtocolBufferException e) { + LOG.warn("Failed parse", e); + } + return null; + } + private void createNode(String path, Long retry_count) { ZKUtil.asyncCreate(this.watcher, path, - TaskState.TASK_UNASSIGNED.get(serverName), new CreateAsyncCallback(), - retry_count); + getZNodeData(SplitLogTaskState.State.UNASSIGNED, serverName), + new CreateAsyncCallback(), retry_count); tot_mgr_node_create_queued.incrementAndGet(); return; } @@ -424,20 +491,26 @@ public class SplitLogManager extends ZooKeeperListener { return; } data = this.watcher.getRecoverableZooKeeper().removeMetaData(data); - // LOG.debug("set watch on " + path + " got data " + new String(data)); - if (TaskState.TASK_UNASSIGNED.equals(data)) { + SplitLogTaskState slts = parseZNodeData(data); + switch (slts.getState()) { + case UNASSIGNED: LOG.debug("task not yet acquired " + path + " ver = " + version); handleUnassignedTask(path); - } else if (TaskState.TASK_OWNED.equals(data)) { - heartbeat(path, version, - TaskState.TASK_OWNED.getWriterName(data)); - } else if (TaskState.TASK_RESIGNED.equals(data)) { + break; + + case OWNED: + heartbeat(path, version, toString(SplitLogTaskState.State.OWNED, slts)); + break; + + case RESIGNED: LOG.info("task " + path + " entered state " + new String(data)); resubmitOrFail(path, FORCE); - } else if (TaskState.TASK_DONE.equals(data)) { + break; + + case DONE: LOG.info("task " + path + " entered state " + new String(data)); if (taskFinisher != null && !ZKSplitLog.isRescanNode(watcher, path)) { - if (taskFinisher.finish(TaskState.TASK_DONE.getWriterName(data), + if (taskFinisher.finish(toString(SplitLogTaskState.State.DONE, slts), ZKSplitLog.getFileName(path)) == Status.DONE) { setDone(path, SUCCESS); } else { @@ -446,16 +519,32 @@ public class SplitLogManager extends ZooKeeperListener { } else { setDone(path, SUCCESS); } - } else if (TaskState.TASK_ERR.equals(data)) { + break; + + case ERR: LOG.info("task " + path + " entered state " + new String(data)); resubmitOrFail(path, CHECK); - } else { + break; + + default: LOG.fatal("logic error - unexpected zk state for path = " + path + " data = " + new String(data)); setDone(path, FAILURE); } } + /** + * @param state + * @param splts + * @return A String made of state and sn with a + * space separator. + */ + private String toString(final SplitLogTaskState.State state, final SplitLogTaskState splts) { + return state.toString() + " " + + (splts == null? "": ServerName.getServerName(splts.getServer().getHostName(), + splts.getServer().getPort(), splts.getServer().getStartCode())); + } + private void getDataSetWatchFailure(String path) { LOG.warn("failed to set data watch " + path); setDone(path, FAILURE); @@ -483,8 +572,7 @@ public class SplitLogManager extends ZooKeeperListener { } } - private void heartbeat(String path, int new_version, - String workerName) { + private void heartbeat(String path, int new_version, String workerName) { Task task = findOrCreateOrphanTask(path); if (new_version != task.last_version) { if (task.isUnassigned()) { @@ -534,7 +622,7 @@ public class SplitLogManager extends ZooKeeperListener { try { // blocking zk call but this is done from the timeout thread if (ZKUtil.setData(this.watcher, path, - TaskState.TASK_UNASSIGNED.get(serverName), + getZNodeData(SplitLogTaskState.State.UNASSIGNED, this.serverName), version) == false) { LOG.debug("failed to resubmit task " + path + " version changed"); @@ -624,8 +712,8 @@ public class SplitLogManager extends ZooKeeperListener { // therefore this behavior is safe. this.watcher.getRecoverableZooKeeper().getZooKeeper(). create(ZKSplitLog.getRescanNode(watcher), - TaskState.TASK_DONE.get(serverName), Ids.OPEN_ACL_UNSAFE, - CreateMode.EPHEMERAL_SEQUENTIAL, + getZNodeData(SplitLogTaskState.State.DONE, this.serverName), + Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL, new CreateRescanAsyncCallback(), Long.valueOf(retries)); } diff --git src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java index 8457bdc..fde1066 100644 --- src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java +++ src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java @@ -1786,6 +1786,1494 @@ public final class ZooKeeperProtos { // @@protoc_insertion_point(class_scope:ClusterUp) } + public interface RegionTransitionOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required uint32 eventType = 1; + boolean hasEventType(); + int getEventType(); + + // required .RegionSpecifier regionName = 2; + boolean hasRegionName(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegionName(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionNameOrBuilder(); + + // required uint64 createTime = 3; + boolean hasCreateTime(); + long getCreateTime(); + + // optional .ServerName originServerName = 4; + boolean hasOriginServerName(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getOriginServerName(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getOriginServerNameOrBuilder(); + + // optional bytes payload = 5; + boolean hasPayload(); + com.google.protobuf.ByteString getPayload(); + } + public static final class RegionTransition extends + com.google.protobuf.GeneratedMessage + implements RegionTransitionOrBuilder { + // Use RegionTransition.newBuilder() to construct. + private RegionTransition(Builder builder) { + super(builder); + } + private RegionTransition(boolean noInit) {} + + private static final RegionTransition defaultInstance; + public static RegionTransition getDefaultInstance() { + return defaultInstance; + } + + public RegionTransition getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_RegionTransition_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_RegionTransition_fieldAccessorTable; + } + + private int bitField0_; + // required uint32 eventType = 1; + public static final int EVENTTYPE_FIELD_NUMBER = 1; + private int eventType_; + public boolean hasEventType() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public int getEventType() { + return eventType_; + } + + // required .RegionSpecifier regionName = 2; + public static final int REGIONNAME_FIELD_NUMBER = 2; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier regionName_; + public boolean hasRegionName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegionName() { + return regionName_; + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionNameOrBuilder() { + return regionName_; + } + + // required uint64 createTime = 3; + public static final int CREATETIME_FIELD_NUMBER = 3; + private long createTime_; + public boolean hasCreateTime() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getCreateTime() { + return createTime_; + } + + // optional .ServerName originServerName = 4; + public static final int ORIGINSERVERNAME_FIELD_NUMBER = 4; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName originServerName_; + public boolean hasOriginServerName() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getOriginServerName() { + return originServerName_; + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getOriginServerNameOrBuilder() { + return originServerName_; + } + + // optional bytes payload = 5; + public static final int PAYLOAD_FIELD_NUMBER = 5; + private com.google.protobuf.ByteString payload_; + public boolean hasPayload() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public com.google.protobuf.ByteString getPayload() { + return payload_; + } + + private void initFields() { + eventType_ = 0; + regionName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + createTime_ = 0L; + originServerName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + payload_ = com.google.protobuf.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasEventType()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRegionName()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasCreateTime()) { + memoizedIsInitialized = 0; + return false; + } + if (!getRegionName().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + if (hasOriginServerName()) { + if (!getOriginServerName().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeUInt32(1, eventType_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, regionName_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, createTime_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeMessage(4, originServerName_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, payload_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt32Size(1, eventType_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, regionName_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(3, createTime_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(4, originServerName_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(5, payload_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition) obj; + + boolean result = true; + result = result && (hasEventType() == other.hasEventType()); + if (hasEventType()) { + result = result && (getEventType() + == other.getEventType()); + } + result = result && (hasRegionName() == other.hasRegionName()); + if (hasRegionName()) { + result = result && getRegionName() + .equals(other.getRegionName()); + } + result = result && (hasCreateTime() == other.hasCreateTime()); + if (hasCreateTime()) { + result = result && (getCreateTime() + == other.getCreateTime()); + } + result = result && (hasOriginServerName() == other.hasOriginServerName()); + if (hasOriginServerName()) { + result = result && getOriginServerName() + .equals(other.getOriginServerName()); + } + result = result && (hasPayload() == other.hasPayload()); + if (hasPayload()) { + result = result && getPayload() + .equals(other.getPayload()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasEventType()) { + hash = (37 * hash) + EVENTTYPE_FIELD_NUMBER; + hash = (53 * hash) + getEventType(); + } + if (hasRegionName()) { + hash = (37 * hash) + REGIONNAME_FIELD_NUMBER; + hash = (53 * hash) + getRegionName().hashCode(); + } + if (hasCreateTime()) { + hash = (37 * hash) + CREATETIME_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getCreateTime()); + } + if (hasOriginServerName()) { + hash = (37 * hash) + ORIGINSERVERNAME_FIELD_NUMBER; + hash = (53 * hash) + getOriginServerName().hashCode(); + } + if (hasPayload()) { + hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; + hash = (53 * hash) + getPayload().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransitionOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_RegionTransition_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_RegionTransition_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getRegionNameFieldBuilder(); + getOriginServerNameFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + eventType_ = 0; + bitField0_ = (bitField0_ & ~0x00000001); + if (regionNameBuilder_ == null) { + regionName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + } else { + regionNameBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + createTime_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + if (originServerNameBuilder_ == null) { + originServerName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + } else { + originServerNameBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000008); + payload_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition build() { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.eventType_ = eventType_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (regionNameBuilder_ == null) { + result.regionName_ = regionName_; + } else { + result.regionName_ = regionNameBuilder_.build(); + } + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.createTime_ = createTime_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + if (originServerNameBuilder_ == null) { + result.originServerName_ = originServerName_; + } else { + result.originServerName_ = originServerNameBuilder_.build(); + } + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.payload_ = payload_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition.getDefaultInstance()) return this; + if (other.hasEventType()) { + setEventType(other.getEventType()); + } + if (other.hasRegionName()) { + mergeRegionName(other.getRegionName()); + } + if (other.hasCreateTime()) { + setCreateTime(other.getCreateTime()); + } + if (other.hasOriginServerName()) { + mergeOriginServerName(other.getOriginServerName()); + } + if (other.hasPayload()) { + setPayload(other.getPayload()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasEventType()) { + + return false; + } + if (!hasRegionName()) { + + return false; + } + if (!hasCreateTime()) { + + return false; + } + if (!getRegionName().isInitialized()) { + + return false; + } + if (hasOriginServerName()) { + if (!getOriginServerName().isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + eventType_ = input.readUInt32(); + break; + } + case 18: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.newBuilder(); + if (hasRegionName()) { + subBuilder.mergeFrom(getRegionName()); + } + input.readMessage(subBuilder, extensionRegistry); + setRegionName(subBuilder.buildPartial()); + break; + } + case 24: { + bitField0_ |= 0x00000004; + createTime_ = input.readUInt64(); + break; + } + case 34: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder(); + if (hasOriginServerName()) { + subBuilder.mergeFrom(getOriginServerName()); + } + input.readMessage(subBuilder, extensionRegistry); + setOriginServerName(subBuilder.buildPartial()); + break; + } + case 42: { + bitField0_ |= 0x00000010; + payload_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required uint32 eventType = 1; + private int eventType_ ; + public boolean hasEventType() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public int getEventType() { + return eventType_; + } + public Builder setEventType(int value) { + bitField0_ |= 0x00000001; + eventType_ = value; + onChanged(); + return this; + } + public Builder clearEventType() { + bitField0_ = (bitField0_ & ~0x00000001); + eventType_ = 0; + onChanged(); + return this; + } + + // required .RegionSpecifier regionName = 2; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier regionName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionNameBuilder_; + public boolean hasRegionName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegionName() { + if (regionNameBuilder_ == null) { + return regionName_; + } else { + return regionNameBuilder_.getMessage(); + } + } + public Builder setRegionName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier value) { + if (regionNameBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + regionName_ = value; + onChanged(); + } else { + regionNameBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder setRegionName( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) { + if (regionNameBuilder_ == null) { + regionName_ = builderForValue.build(); + onChanged(); + } else { + regionNameBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder mergeRegionName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier value) { + if (regionNameBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + regionName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance()) { + regionName_ = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.newBuilder(regionName_).mergeFrom(value).buildPartial(); + } else { + regionName_ = value; + } + onChanged(); + } else { + regionNameBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder clearRegionName() { + if (regionNameBuilder_ == null) { + regionName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + onChanged(); + } else { + regionNameBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionNameBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getRegionNameFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionNameOrBuilder() { + if (regionNameBuilder_ != null) { + return regionNameBuilder_.getMessageOrBuilder(); + } else { + return regionName_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> + getRegionNameFieldBuilder() { + if (regionNameBuilder_ == null) { + regionNameBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>( + regionName_, + getParentForChildren(), + isClean()); + regionName_ = null; + } + return regionNameBuilder_; + } + + // required uint64 createTime = 3; + private long createTime_ ; + public boolean hasCreateTime() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getCreateTime() { + return createTime_; + } + public Builder setCreateTime(long value) { + bitField0_ |= 0x00000004; + createTime_ = value; + onChanged(); + return this; + } + public Builder clearCreateTime() { + bitField0_ = (bitField0_ & ~0x00000004); + createTime_ = 0L; + onChanged(); + return this; + } + + // optional .ServerName originServerName = 4; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName originServerName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> originServerNameBuilder_; + public boolean hasOriginServerName() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getOriginServerName() { + if (originServerNameBuilder_ == null) { + return originServerName_; + } else { + return originServerNameBuilder_.getMessage(); + } + } + public Builder setOriginServerName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) { + if (originServerNameBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + originServerName_ = value; + onChanged(); + } else { + originServerNameBuilder_.setMessage(value); + } + bitField0_ |= 0x00000008; + return this; + } + public Builder setOriginServerName( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (originServerNameBuilder_ == null) { + originServerName_ = builderForValue.build(); + onChanged(); + } else { + originServerNameBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000008; + return this; + } + public Builder mergeOriginServerName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) { + if (originServerNameBuilder_ == null) { + if (((bitField0_ & 0x00000008) == 0x00000008) && + originServerName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()) { + originServerName_ = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder(originServerName_).mergeFrom(value).buildPartial(); + } else { + originServerName_ = value; + } + onChanged(); + } else { + originServerNameBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000008; + return this; + } + public Builder clearOriginServerName() { + if (originServerNameBuilder_ == null) { + originServerName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + onChanged(); + } else { + originServerNameBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder getOriginServerNameBuilder() { + bitField0_ |= 0x00000008; + onChanged(); + return getOriginServerNameFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getOriginServerNameOrBuilder() { + if (originServerNameBuilder_ != null) { + return originServerNameBuilder_.getMessageOrBuilder(); + } else { + return originServerName_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> + getOriginServerNameFieldBuilder() { + if (originServerNameBuilder_ == null) { + originServerNameBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder>( + originServerName_, + getParentForChildren(), + isClean()); + originServerName_ = null; + } + return originServerNameBuilder_; + } + + // optional bytes payload = 5; + private com.google.protobuf.ByteString payload_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasPayload() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public com.google.protobuf.ByteString getPayload() { + return payload_; + } + public Builder setPayload(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + payload_ = value; + onChanged(); + return this; + } + public Builder clearPayload() { + bitField0_ = (bitField0_ & ~0x00000010); + payload_ = getDefaultInstance().getPayload(); + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:RegionTransition) + } + + static { + defaultInstance = new RegionTransition(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:RegionTransition) + } + + public interface SplitLogTaskStateOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .SplitLogTaskState.State state = 1; + boolean hasState(); + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.State getState(); + + // required .ServerName server = 2; + boolean hasServer(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServer(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder(); + } + public static final class SplitLogTaskState extends + com.google.protobuf.GeneratedMessage + implements SplitLogTaskStateOrBuilder { + // Use SplitLogTaskState.newBuilder() to construct. + private SplitLogTaskState(Builder builder) { + super(builder); + } + private SplitLogTaskState(boolean noInit) {} + + private static final SplitLogTaskState defaultInstance; + public static SplitLogTaskState getDefaultInstance() { + return defaultInstance; + } + + public SplitLogTaskState getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_SplitLogTaskState_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_SplitLogTaskState_fieldAccessorTable; + } + + public enum State + implements com.google.protobuf.ProtocolMessageEnum { + UNASSIGNED(0, 0), + OWNED(1, 1), + RESIGNED(2, 2), + DONE(3, 3), + ERR(4, 4), + ; + + public static final int UNASSIGNED_VALUE = 0; + public static final int OWNED_VALUE = 1; + public static final int RESIGNED_VALUE = 2; + public static final int DONE_VALUE = 3; + public static final int ERR_VALUE = 4; + + + public final int getNumber() { return value; } + + public static State valueOf(int value) { + switch (value) { + case 0: return UNASSIGNED; + case 1: return OWNED; + case 2: return RESIGNED; + case 3: return DONE; + case 4: return ERR; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static com.google.protobuf.Internal.EnumLiteMap + internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public State findValueByNumber(int number) { + return State.valueOf(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(index); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.getDescriptor().getEnumTypes().get(0); + } + + private static final State[] VALUES = { + UNASSIGNED, OWNED, RESIGNED, DONE, ERR, + }; + + public static State valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int index; + private final int value; + + private State(int index, int value) { + this.index = index; + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:SplitLogTaskState.State) + } + + private int bitField0_; + // required .SplitLogTaskState.State state = 1; + public static final int STATE_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.State state_; + public boolean hasState() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.State getState() { + return state_; + } + + // required .ServerName server = 2; + public static final int SERVER_FIELD_NUMBER = 2; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName server_; + public boolean hasServer() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServer() { + return server_; + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder() { + return server_; + } + + private void initFields() { + state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.State.UNASSIGNED; + server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasState()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasServer()) { + memoizedIsInitialized = 0; + return false; + } + if (!getServer().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeEnum(1, state_.getNumber()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, server_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(1, state_.getNumber()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, server_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState) obj; + + boolean result = true; + result = result && (hasState() == other.hasState()); + if (hasState()) { + result = result && + (getState() == other.getState()); + } + result = result && (hasServer() == other.hasServer()); + if (hasServer()) { + result = result && getServer() + .equals(other.getServer()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasState()) { + hash = (37 * hash) + STATE_FIELD_NUMBER; + hash = (53 * hash) + hashEnum(getState()); + } + if (hasServer()) { + hash = (37 * hash) + SERVER_FIELD_NUMBER; + hash = (53 * hash) + getServer().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskStateOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_SplitLogTaskState_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_SplitLogTaskState_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getServerFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.State.UNASSIGNED; + bitField0_ = (bitField0_ & ~0x00000001); + if (serverBuilder_ == null) { + server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + } else { + serverBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState build() { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.state_ = state_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (serverBuilder_ == null) { + result.server_ = server_; + } else { + result.server_ = serverBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.getDefaultInstance()) return this; + if (other.hasState()) { + setState(other.getState()); + } + if (other.hasServer()) { + mergeServer(other.getServer()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasState()) { + + return false; + } + if (!hasServer()) { + + return false; + } + if (!getServer().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 8: { + int rawValue = input.readEnum(); + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.State value = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.State.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(1, rawValue); + } else { + bitField0_ |= 0x00000001; + state_ = value; + } + break; + } + case 18: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder(); + if (hasServer()) { + subBuilder.mergeFrom(getServer()); + } + input.readMessage(subBuilder, extensionRegistry); + setServer(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // required .SplitLogTaskState.State state = 1; + private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.State state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.State.UNASSIGNED; + public boolean hasState() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.State getState() { + return state_; + } + public Builder setState(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.State value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + state_ = value; + onChanged(); + return this; + } + public Builder clearState() { + bitField0_ = (bitField0_ & ~0x00000001); + state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.State.UNASSIGNED; + onChanged(); + return this; + } + + // required .ServerName server = 2; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverBuilder_; + public boolean hasServer() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServer() { + if (serverBuilder_ == null) { + return server_; + } else { + return serverBuilder_.getMessage(); + } + } + public Builder setServer(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) { + if (serverBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + server_ = value; + onChanged(); + } else { + serverBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder setServer( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serverBuilder_ == null) { + server_ = builderForValue.build(); + onChanged(); + } else { + serverBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder mergeServer(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) { + if (serverBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + server_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()) { + server_ = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder(server_).mergeFrom(value).buildPartial(); + } else { + server_ = value; + } + onChanged(); + } else { + serverBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder clearServer() { + if (serverBuilder_ == null) { + server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + onChanged(); + } else { + serverBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder getServerBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getServerFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder() { + if (serverBuilder_ != null) { + return serverBuilder_.getMessageOrBuilder(); + } else { + return server_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> + getServerFieldBuilder() { + if (serverBuilder_ == null) { + serverBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder>( + server_, + getParentForChildren(), + isClean()); + server_ = null; + } + return serverBuilder_; + } + + // @@protoc_insertion_point(builder_scope:SplitLogTaskState) + } + + static { + defaultInstance = new SplitLogTaskState(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:SplitLogTaskState) + } + private static com.google.protobuf.Descriptors.Descriptor internal_static_RootRegionServer_descriptor; private static @@ -1806,6 +3294,16 @@ public final class ZooKeeperProtos { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_ClusterUp_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_RegionTransition_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_RegionTransition_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_SplitLogTaskState_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_SplitLogTaskState_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -1819,9 +3317,17 @@ public final class ZooKeeperProtos { "gionServer\022\033\n\006server\030\001 \002(\0132\013.ServerName\"" + "%\n\006Master\022\033\n\006master\030\001 \002(\0132\013.ServerName\"\036" + "\n\tClusterId\022\021\n\tclusterId\030\001 \002(\t\"\036\n\tCluste" + - "rUp\022\021\n\tstartDate\030\001 \002(\tBE\n*org.apache.had" + - "oop.hbase.protobuf.generatedB\017ZooKeeperP" + - "rotosH\001\210\001\001\240\001\001" + "rUp\022\021\n\tstartDate\030\001 \002(\t\"\227\001\n\020RegionTransit" + + "ion\022\021\n\teventType\030\001 \002(\r\022$\n\nregionName\030\002 \002" + + "(\0132\020.RegionSpecifier\022\022\n\ncreateTime\030\003 \002(\004" + + "\022%\n\020originServerName\030\004 \001(\0132\013.ServerName\022" + + "\017\n\007payload\030\005 \001(\014\"\236\001\n\021SplitLogTaskState\022\'" + + "\n\005state\030\001 \002(\0162\030.SplitLogTaskState.State\022", + "\033\n\006server\030\002 \002(\0132\013.ServerName\"C\n\005State\022\016\n" + + "\nUNASSIGNED\020\000\022\t\n\005OWNED\020\001\022\014\n\010RESIGNED\020\002\022\010" + + "\n\004DONE\020\003\022\007\n\003ERR\020\004BE\n*org.apache.hadoop.h" + + "base.protobuf.generatedB\017ZooKeeperProtos" + + "H\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -1860,6 +3366,22 @@ public final class ZooKeeperProtos { new java.lang.String[] { "StartDate", }, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp.Builder.class); + internal_static_RegionTransition_descriptor = + getDescriptor().getMessageTypes().get(4); + internal_static_RegionTransition_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_RegionTransition_descriptor, + new java.lang.String[] { "EventType", "RegionName", "CreateTime", "OriginServerName", "Payload", }, + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition.class, + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition.Builder.class); + internal_static_SplitLogTaskState_descriptor = + getDescriptor().getMessageTypes().get(5); + internal_static_SplitLogTaskState_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_SplitLogTaskState_descriptor, + new java.lang.String[] { "State", "Server", }, + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.class, + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTaskState.Builder.class); return null; } }; diff --git src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java index e94b672..a53a03f 100644 --- src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java @@ -27,8 +27,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.executor.RegionTransitionData; import org.apache.hadoop.hbase.executor.EventHandler.EventType; +import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.Code; @@ -145,8 +145,7 @@ public class ZKAssign { throws KeeperException, KeeperException.NodeExistsException { LOG.debug(zkw.prefix("Creating unassigned node for " + region.getEncodedName() + " in OFFLINE state")); - RegionTransitionData data = new RegionTransitionData(event, - region.getRegionName(), serverName); + RegionTransition rt = RegionTransition.newBuilder().setEventType(event.), region.getRegionName(), serverName); String node = getNodeName(zkw, region.getEncodedName()); ZKUtil.createAndWatch(zkw, node, data.getBytes()); } @@ -845,19 +844,15 @@ public class ZKAssign { * * @param zkw zk reference * @param pathOrRegionName fully-specified path or region name - * @return data for the unassigned node + * @return data structure made from data in the unassigned node or null if node does not exist * @throws KeeperException if unexpected zookeeper exception */ - public static RegionTransitionData getData(ZooKeeperWatcher zkw, + public static RegionTransition getData(ZooKeeperWatcher zkw, String pathOrRegionName) throws KeeperException { - String node = pathOrRegionName.startsWith("/") ? - pathOrRegionName : getNodeName(zkw, pathOrRegionName); + String node = getPath(zkw, pathOrRegionName); byte [] data = ZKUtil.getDataAndWatch(zkw, node); - if(data == null) { - return null; - } - return RegionTransitionData.fromBytes(data); + return data == null? null: RegionTransition.parseFrom(data); } /** @@ -871,19 +866,15 @@ public class ZKAssign { * @param zkw zk reference * @param pathOrRegionName fully-specified path or region name * @param stat object to populate the version. - * @return data for the unassigned node + * @return data structure made from data in the unassigned node or null if node does not exist * @throws KeeperException if unexpected zookeeper exception */ - public static RegionTransitionData getDataAndWatch(ZooKeeperWatcher zkw, + public static RegionTransition getDataAndWatch(ZooKeeperWatcher zkw, String pathOrRegionName, Stat stat) throws KeeperException { - String node = pathOrRegionName.startsWith("/") ? - pathOrRegionName : getNodeName(zkw, pathOrRegionName); + String node = getPath(zkw, pathOrRegionName); byte [] data = ZKUtil.getDataAndWatch(zkw, node, stat); - if(data == null) { - return null; - } - return RegionTransitionData.fromBytes(data); + return data == null? null: RegionTransition.parseFrom(data); } /** @@ -897,19 +888,19 @@ public class ZKAssign { * @param zkw zk reference * @param pathOrRegionName fully-specified path or region name * @param stat object to store node info into on getData call - * @return data for the unassigned node or null if node does not exist + * @return data structure made from data in the unassigned node or null if node does not exist * @throws KeeperException if unexpected zookeeper exception */ - public static RegionTransitionData getDataNoWatch(ZooKeeperWatcher zkw, + public static RegionTransition getDataNoWatch(ZooKeeperWatcher zkw, String pathOrRegionName, Stat stat) throws KeeperException { - String node = pathOrRegionName.startsWith("/") ? - pathOrRegionName : getNodeName(zkw, pathOrRegionName); + String node = getPath(zkw, pathOrRegionName); byte [] data = ZKUtil.getDataNoWatch(zkw, node, stat); - if (data == null) { - return null; - } - return RegionTransitionData.fromBytes(data); + return data == null? null: RegionTransition.parseFrom(data); + } + + private static String getPath(final ZooKeeperWatcher zkw, final String pathOrRegionName) { + return pathOrRegionName.startsWith("/")? pathOrRegionName : getNodeName(zkw, pathOrRegionName); } /** diff --git src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java index 30d7fe9..61b4789 100644 --- src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java @@ -25,7 +25,6 @@ import java.lang.reflect.Field; import java.net.URLDecoder; import java.net.URLEncoder; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.concurrent.atomic.AtomicLong; @@ -37,7 +36,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.master.SplitLogManager; import org.apache.hadoop.hbase.regionserver.SplitLogWorker; -import org.apache.hadoop.hbase.util.Bytes; /** * Common methods and attributes used by {@link SplitLogManager} and @@ -107,53 +105,6 @@ public class ZKSplitLog { return dirname.equals(zkw.splitLogZNode); } - public static enum TaskState { - TASK_UNASSIGNED("unassigned"), - TASK_OWNED("owned"), - TASK_RESIGNED("resigned"), - TASK_DONE("done"), - TASK_ERR("err"); - - private final byte[] state; - private TaskState(String s) { - state = s.getBytes(); - } - - public byte[] get(String serverName) { - return (Bytes.add(state, " ".getBytes(), serverName.getBytes())); - } - - public String getWriterName(byte[] data) { - String str = Bytes.toString(data); - return str.substring(str.indexOf(' ') + 1); - } - - - /** - * @param s - * @return True if {@link #state} is a prefix of s. False otherwise. - */ - public boolean equals(byte[] s) { - if (s.length < state.length) { - return (false); - } - for (int i = 0; i < state.length; i++) { - if (state[i] != s[i]) { - return (false); - } - } - return (true); - } - - public boolean equals(byte[] s, String serverName) { - return (Arrays.equals(s, get(serverName))); - } - @Override - public String toString() { - return new String(state); - } - } - public static Path getSplitLogDir(Path rootdir, String tmpname) { return new Path(new Path(rootdir, HConstants.SPLIT_LOGDIR_NAME), tmpname); } @@ -199,7 +150,6 @@ public class ZKSplitLog { return file.getName().equals("corrupt"); } - public static class Counters { //SplitLogManager counters public static AtomicLong tot_mgr_log_split_batch_start = new AtomicLong(0); @@ -275,4 +225,4 @@ public class ZKSplitLog { } } } -} +} \ No newline at end of file diff --git src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java index 46a6fde..4e557a6 100644 --- src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java @@ -26,7 +26,6 @@ import java.io.PrintWriter; import java.net.InetSocketAddress; import java.net.Socket; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Properties; @@ -39,7 +38,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.EmptyWatcher; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.executor.RegionTransitionData; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RootRegionServer; @@ -359,8 +357,7 @@ public class ZKUtil { * null if parent does not exist * @throws KeeperException if unexpected zookeeper exception */ - public static List listChildrenNoWatch( - ZooKeeperWatcher zkw, String znode) + public static List listChildrenNoWatch(ZooKeeperWatcher zkw, String znode) throws KeeperException { List children = null; try { @@ -376,7 +373,9 @@ public class ZKUtil { /** * Simple class to hold a node path and node data. + * @deprecated Unused */ + @Deprecated public static class NodeAndData { private String node; private byte [] data; @@ -392,7 +391,7 @@ public class ZKUtil { } @Override public String toString() { - return node + " (" + RegionTransitionData.fromBytes(data) + ")"; + return node; } public boolean isEmpty() { return (data.length == 0); @@ -600,6 +599,7 @@ public class ZKUtil { * @return list of data of children of the specified node, an empty list if the node * exists but has no children, and null if the node does not exist * @throws KeeperException if unexpected zookeeper exception + * @deprecated Unused */ public static List getChildDataAndWatchForNewChildren( ZooKeeperWatcher zkw, String baseNode) throws KeeperException { @@ -630,6 +630,7 @@ public class ZKUtil { * @param expectedVersion * @throws KeeperException if unexpected zookeeper exception * @throws KeeperException.BadVersionException if version mismatch + * @deprecated Unused */ public static void updateExistingNodeData(ZooKeeperWatcher zkw, String znode, byte [] data, int expectedVersion) @@ -1144,9 +1145,9 @@ public class ZKUtil { " byte(s) of data from znode " + znode + (watcherSet? " and set watcher; ": "; data=") + (data == null? "null": data.length == 0? "empty": ( - znode.startsWith(zkw.assignmentZNode) ? - RegionTransitionData.fromBytes(data).toString() - : StringUtils.abbreviate(Bytes.toStringBinary(data), 32))))); + znode.startsWith(zkw.assignmentZNode)? + RegionTransitionData.fromBytes(data).toString(): + StringUtils.abbreviate(Bytes.toStringBinary(data), 32))))); } /** diff --git src/main/protobuf/ZooKeeper.proto src/main/protobuf/ZooKeeper.proto index 961ab65..1469e3a 100644 --- src/main/protobuf/ZooKeeper.proto +++ src/main/protobuf/ZooKeeper.proto @@ -60,3 +60,30 @@ message ClusterUp { // the data is cluster startDate. required string startDate = 1; } + +/** + * What we write under unassigned. Details a region in transition. + */ +message RegionTransition { + required uint32 eventType = 1; + required RegionSpecifier regionName = 2; + required uint64 createTime = 3; + optional ServerName originServerName = 4; + optional bytes payload = 5; +} + +/** + * Splitlog dir znodes. Used doing distributed splits. + * Holds current state and name of server that originated split. + */ +message SplitLogTaskState { + enum State { + UNASSIGNED = 0; + OWNED = 1; + RESIGNED = 2; + DONE = 3; + ERR = 4; + } + required State state = 1; + required ServerName server = 2; +}