HBASE-5869 Move SplitLogManager splitlog taskstate and AssignmentManager RegionTransitionData znode datas to pb
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1333099 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d369701569
commit
062660e3f4
|
@ -0,0 +1,43 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Failed deserialization.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@SuppressWarnings("serial")
|
||||
public class DeserializationException extends HBaseException {
|
||||
public DeserializationException() {
|
||||
super();
|
||||
}
|
||||
|
||||
public DeserializationException(final String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public DeserializationException(final String message, final Throwable t) {
|
||||
super(message, t);
|
||||
}
|
||||
|
||||
public DeserializationException(final Throwable t) {
|
||||
super(t);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,44 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Base checked exception in HBase.
|
||||
* @see https://issues.apache.org/jira/browse/HBASE-5796
|
||||
*/
|
||||
@SuppressWarnings("serial")
|
||||
@InterfaceAudience.Private
|
||||
public class HBaseException extends Exception {
|
||||
public HBaseException() {
|
||||
super();
|
||||
}
|
||||
|
||||
public HBaseException(final String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public HBaseException(final String message, final Throwable t) {
|
||||
super(message, t);
|
||||
}
|
||||
|
||||
public HBaseException(final Throwable t) {
|
||||
super(t);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,141 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
* Current state of a region in transition. Holds state of a region as it moves through the
|
||||
* steps that take it from offline to open, etc. Used by regionserver, master, and zk packages.
|
||||
* Encapsulates protobuf serialization/deserialization so we don't leak generated pb outside this
|
||||
* class. Create an instance using {@link #createRegionTransition(EventType, byte[], ServerName)}.
|
||||
* <p>Immutable
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class RegionTransition {
|
||||
private final ZooKeeperProtos.RegionTransition rt;
|
||||
|
||||
/**
|
||||
* Shutdown constructor
|
||||
*/
|
||||
private RegionTransition() {
|
||||
this(null);
|
||||
}
|
||||
|
||||
private RegionTransition(final ZooKeeperProtos.RegionTransition rt) {
|
||||
this.rt = rt;
|
||||
}
|
||||
|
||||
public EventHandler.EventType getEventType() {
|
||||
return EventHandler.EventType.get(this.rt.getEventTypeCode());
|
||||
}
|
||||
|
||||
public ServerName getServerName() {
|
||||
return ProtobufUtil.toServerName(this.rt.getOriginServerName());
|
||||
}
|
||||
|
||||
public long getCreateTime() {
|
||||
return this.rt.getCreateTime();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Full region name
|
||||
*/
|
||||
public byte [] getRegionName() {
|
||||
return this.rt.getRegionName().toByteArray();
|
||||
}
|
||||
|
||||
public byte [] getPayload() {
|
||||
return this.rt.getPayload().toByteArray();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
byte [] payload = getPayload();
|
||||
return "region=" + Bytes.toStringBinary(getRegionName()) + ", state=" + getEventType() +
|
||||
", servername=" + getServerName() + ", createTime=" + this.getCreateTime() +
|
||||
", payload.length=" + (payload == null? 0: payload.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param type
|
||||
* @param regionName
|
||||
* @param sn
|
||||
* @return a serialized pb {@link RegionTransition}
|
||||
* @see #parseRegionTransition(byte[])
|
||||
*/
|
||||
public static RegionTransition createRegionTransition(final EventType type,
|
||||
final byte [] regionName, final ServerName sn) {
|
||||
return createRegionTransition(type, regionName, sn, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param type
|
||||
* @param regionName
|
||||
* @param sn
|
||||
* @param payload May be null
|
||||
* @return a serialized pb {@link RegionTransition}
|
||||
* @see #parseRegionTransition(byte[])
|
||||
*/
|
||||
public static RegionTransition createRegionTransition(final EventType type,
|
||||
final byte [] regionName, final ServerName sn, final byte [] payload) {
|
||||
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName pbsn =
|
||||
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder().
|
||||
setHostName(sn.getHostname()).setPort(sn.getPort()).setStartCode(sn.getStartcode()).build();
|
||||
ZooKeeperProtos.RegionTransition.Builder builder = ZooKeeperProtos.RegionTransition.newBuilder().
|
||||
setEventTypeCode(type.getCode()).setRegionName(ByteString.copyFrom(regionName)).
|
||||
setOriginServerName(pbsn);
|
||||
builder.setCreateTime(System.currentTimeMillis());
|
||||
if (payload != null) builder.setPayload(ByteString.copyFrom(payload));
|
||||
return new RegionTransition(builder.build());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param data Serialized date to parse.
|
||||
* @return A RegionTransition instance made of the passed <code>data</code>
|
||||
* @throws DeserializationException
|
||||
* @see #toByteArray()
|
||||
*/
|
||||
public static RegionTransition parseFrom(final byte [] data) throws DeserializationException {
|
||||
ProtobufUtil.expectPBMagicPrefix(data);
|
||||
try {
|
||||
int prefixLen = ProtobufUtil.lengthOfPBMagic();
|
||||
ZooKeeperProtos.RegionTransition rt = ZooKeeperProtos.RegionTransition.newBuilder().
|
||||
mergeFrom(data, prefixLen, data.length - prefixLen).build();
|
||||
return new RegionTransition(rt);
|
||||
} catch (InvalidProtocolBufferException e) {
|
||||
throw new DeserializationException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return This instance serialized into a byte array
|
||||
* @see #parseFrom(byte[])
|
||||
*/
|
||||
public byte [] toByteArray() {
|
||||
return ProtobufUtil.prependPBMagic(this.rt.toByteArray());
|
||||
}
|
||||
}
|
|
@ -24,9 +24,13 @@ import java.util.regex.Pattern;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RootRegionServer;
|
||||
import org.apache.hadoop.hbase.util.Addressing;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
* Instance of an HBase ServerName.
|
||||
* A server name is used uniquely identifying a server instance and is made
|
||||
|
@ -296,4 +300,47 @@ public class ServerName implements Comparable<ServerName> {
|
|||
return SERVERNAME_PATTERN.matcher(str).matches()? new ServerName(str):
|
||||
new ServerName(str, NON_STARTCODE);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a ServerName from the passed in data bytes.
|
||||
* @param data Data with a serialize server name in it; can handle the old style
|
||||
* servername where servername was host and port. Works too with data that
|
||||
* begins w/ the pb 'PBUF' magic and that is then followed by a protobuf that
|
||||
* has a serialized {@link ServerName} in it.
|
||||
* @return Returns null if <code>data</code> is null else converts passed data
|
||||
* to a ServerName instance.
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
public static ServerName parseFrom(final byte [] data) throws DeserializationException {
|
||||
if (data == null || data.length <= 0) return null;
|
||||
if (ProtobufUtil.isPBMagicPrefix(data)) {
|
||||
int prefixLen = ProtobufUtil.lengthOfPBMagic();
|
||||
try {
|
||||
RootRegionServer rss =
|
||||
RootRegionServer.newBuilder().mergeFrom(data, prefixLen, data.length - prefixLen).build();
|
||||
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName sn = rss.getServer();
|
||||
return new ServerName(sn.getHostName(), sn.getPort(), sn.getStartCode());
|
||||
} catch (InvalidProtocolBufferException e) {
|
||||
// A failed parse of the znode is pretty catastrophic. Rather than loop
|
||||
// retrying hoping the bad bytes will changes, and rather than change
|
||||
// the signature on this method to add an IOE which will send ripples all
|
||||
// over the code base, throw a RuntimeException. This should "never" happen.
|
||||
// Fail fast if it does.
|
||||
throw new DeserializationException(e);
|
||||
}
|
||||
}
|
||||
// The str returned could be old style -- pre hbase-1502 -- which was
|
||||
// hostname and port seperated by a colon rather than hostname, port and
|
||||
// startcode delimited by a ','.
|
||||
String str = Bytes.toString(data);
|
||||
int index = str.indexOf(ServerName.SERVERNAME_SEPARATOR);
|
||||
if (index != -1) {
|
||||
// Presume its ServerName serialized with versioned bytes.
|
||||
return ServerName.parseVersionedServerName(data);
|
||||
}
|
||||
// Presume it a hostname:port format.
|
||||
String hostname = Addressing.parseHostname(str);
|
||||
int port = Addressing.parsePort(str);
|
||||
return new ServerName(hostname, port, -1L);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,102 @@
|
|||
package org.apache.hadoop.hbase;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Counters kept by the distributed WAL split log process.
|
||||
* Used by master and regionserver packages.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class SplitLogCounters {
|
||||
//SplitLogManager counters
|
||||
public static AtomicLong tot_mgr_log_split_batch_start = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_log_split_batch_success =
|
||||
new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_log_split_batch_err = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_new_unexpected_hlogs = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_log_split_start = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_log_split_success = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_log_split_err = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_create_queued = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_create_result = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_already_exists = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_create_err = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_create_retry = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_get_data_queued = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_get_data_result = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_get_data_nonode = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_get_data_err = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_get_data_retry = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_delete_queued = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_delete_result = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_delete_err = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_resubmit = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_resubmit_failed = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_null_data = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_orphan_task_acquired = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_wait_for_zk_delete = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_unacquired_orphan_done = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_resubmit_threshold_reached =
|
||||
new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_missing_state_in_delete =
|
||||
new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_heartbeat = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_rescan = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_rescan_deleted = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_task_deleted = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_resubmit_unassigned = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_relist_logdir = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_resubmit_dead_server_task =
|
||||
new AtomicLong(0);
|
||||
|
||||
// SplitLogWorker counters
|
||||
public static AtomicLong tot_wkr_failed_to_grab_task_no_data =
|
||||
new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_failed_to_grab_task_exception =
|
||||
new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_failed_to_grab_task_owned =
|
||||
new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_failed_to_grab_task_lost_race =
|
||||
new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_task_acquired = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_task_resigned = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_task_done = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_task_err = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_task_heartbeat = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_task_acquired_rescan = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_get_data_queued = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_get_data_result = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_get_data_retry = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_preempt_task = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_task_heartbeat_failed = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_final_transistion_failed =
|
||||
new AtomicLong(0);
|
||||
|
||||
public static void resetCounters() throws Exception {
|
||||
Class<?> cl = (new SplitLogCounters()).getClass();
|
||||
Field[] flds = cl.getDeclaredFields();
|
||||
for (Field fld : flds) {
|
||||
((AtomicLong)fld.get(null)).set(0);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,172 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
* State of a WAL log split during distributed splitting. State is kept up in zookeeper.
|
||||
* Encapsulates protobuf serialization/deserialization so we don't leak generated pb outside of
|
||||
* this class. Used by regionserver and master packages.
|
||||
* <p>Immutable
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class SplitLogTask {
|
||||
private final ServerName originServer;
|
||||
private final ZooKeeperProtos.SplitLogTask.State state;
|
||||
|
||||
public static class Unassigned extends SplitLogTask {
|
||||
public Unassigned(final ServerName originServer) {
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.UNASSIGNED);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Owned extends SplitLogTask {
|
||||
public Owned(final ServerName originServer) {
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.OWNED);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Resigned extends SplitLogTask {
|
||||
public Resigned(final ServerName originServer) {
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.RESIGNED);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Done extends SplitLogTask {
|
||||
public Done(final ServerName originServer) {
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.DONE);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Err extends SplitLogTask {
|
||||
public Err(final ServerName originServer) {
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.ERR);
|
||||
}
|
||||
}
|
||||
|
||||
SplitLogTask(final ZooKeeperProtos.SplitLogTask slt) {
|
||||
this(ProtobufUtil.toServerName(slt.getServerName()), slt.getState());
|
||||
}
|
||||
|
||||
SplitLogTask(final ServerName originServer, final ZooKeeperProtos.SplitLogTask.State state) {
|
||||
this.originServer = originServer;
|
||||
this.state = state;
|
||||
}
|
||||
|
||||
public ServerName getServerName() {
|
||||
return this.originServer;
|
||||
}
|
||||
|
||||
public boolean isUnassigned(final ServerName sn) {
|
||||
return this.originServer.equals(sn) && isUnassigned();
|
||||
}
|
||||
|
||||
public boolean isUnassigned() {
|
||||
return this.state == ZooKeeperProtos.SplitLogTask.State.UNASSIGNED;
|
||||
}
|
||||
|
||||
public boolean isOwned(final ServerName sn) {
|
||||
return this.originServer.equals(sn) && isOwned();
|
||||
}
|
||||
|
||||
public boolean isOwned() {
|
||||
return this.state == ZooKeeperProtos.SplitLogTask.State.OWNED;
|
||||
}
|
||||
|
||||
public boolean isResigned(final ServerName sn) {
|
||||
return this.originServer.equals(sn) && isResigned();
|
||||
}
|
||||
|
||||
public boolean isResigned() {
|
||||
return this.state == ZooKeeperProtos.SplitLogTask.State.RESIGNED;
|
||||
}
|
||||
|
||||
public boolean isDone(final ServerName sn) {
|
||||
return this.originServer.equals(sn) && isDone();
|
||||
}
|
||||
|
||||
public boolean isDone() {
|
||||
return this.state == ZooKeeperProtos.SplitLogTask.State.DONE;
|
||||
}
|
||||
|
||||
public boolean isErr(final ServerName sn) {
|
||||
return this.originServer.equals(sn) && isErr();
|
||||
}
|
||||
|
||||
public boolean isErr() {
|
||||
return this.state == ZooKeeperProtos.SplitLogTask.State.ERR;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return this.state.toString() + " " + this.originServer.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (!(obj instanceof SplitLogTask)) return false;
|
||||
SplitLogTask other = (SplitLogTask)obj;
|
||||
return other.state.equals(this.state) && other.originServer.equals(this.originServer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int hash = 7;
|
||||
hash = 31 * hash + this.state.hashCode();
|
||||
return 31 * hash + this.originServer.hashCode();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param data Serialized date to parse.
|
||||
* @return An SplitLogTaskState instance made of the passed <code>data</code>
|
||||
* @throws DeserializationException
|
||||
* @see #toByteArray()
|
||||
*/
|
||||
public static SplitLogTask parseFrom(final byte [] data) throws DeserializationException {
|
||||
ProtobufUtil.expectPBMagicPrefix(data);
|
||||
try {
|
||||
int prefixLen = ProtobufUtil.lengthOfPBMagic();
|
||||
ZooKeeperProtos.SplitLogTask slt = ZooKeeperProtos.SplitLogTask.newBuilder().
|
||||
mergeFrom(data, prefixLen, data.length - prefixLen).build();
|
||||
return new SplitLogTask(slt);
|
||||
} catch (InvalidProtocolBufferException e) {
|
||||
throw new DeserializationException(Bytes.toStringBinary(data, 0, 64), e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return This instance serialized into a byte array
|
||||
* @see #parseFrom(byte[])
|
||||
*/
|
||||
public byte [] toByteArray() {
|
||||
// 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.
|
||||
HBaseProtos.ServerName snpb = ProtobufUtil.toServerName(this.originServer);
|
||||
ZooKeeperProtos.SplitLogTask slts =
|
||||
ZooKeeperProtos.SplitLogTask.newBuilder().setServerName(snpb).setState(this.state).build();
|
||||
return ProtobufUtil.prependPBMagic(slts.toByteArray());
|
||||
}
|
||||
}
|
|
@ -105,7 +105,7 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
|
|||
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<Runnable> {
|
|||
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) ||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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.
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>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 <code>HRegionInfo#getRegionName()</code>
|
||||
*/
|
||||
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.
|
||||
*
|
||||
* <p>Used when the server name is known (a regionserver is setting it).
|
||||
*
|
||||
* <p>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 <code>HRegionInfo#getRegionName()</code>
|
||||
* @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.
|
||||
*
|
||||
* <p>Used when the server name is known (a regionserver is setting it).
|
||||
*
|
||||
* <p>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 <code>HRegionInfo#getRegionName()</code>
|
||||
* @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.
|
||||
*
|
||||
* <p>One of:
|
||||
* <ul>
|
||||
* <li>{@link EventType#M_ZK_REGION_OFFLINE}
|
||||
* <li>{@link EventType#M_ZK_REGION_CLOSING}
|
||||
* <li>{@link EventType#RS_ZK_REGION_CLOSED}
|
||||
* <li>{@link EventType#RS_ZK_REGION_OPENING}
|
||||
* <li>{@link EventType#RS_ZK_REGION_OPENED}
|
||||
* <li>{@link EventType#RS_ZK_REGION_SPLITTING}
|
||||
* <li>{@link EventType#RS_ZK_REGION_SPLIT}
|
||||
* </ul>
|
||||
* @return type of region transition event
|
||||
*/
|
||||
public EventType getEventType() {
|
||||
return eventType;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the name of the region being transitioned.
|
||||
*
|
||||
* <p>Region name is required so this never returns null.
|
||||
* @return region name, the result of a call to <code>HRegionInfo#getRegionName()</code>
|
||||
*/
|
||||
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;
|
||||
}
|
||||
}
|
|
@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
|
@ -138,7 +139,8 @@ class ActiveMasterManager extends ZooKeeperListener {
|
|||
// Try to become the active master, watch if there is another master.
|
||||
// Write out our ServerName as versioned bytes.
|
||||
try {
|
||||
String backupZNode = ZKUtil.joinZNode(this.watcher.backupMasterAddressesZNode, this.sn.toString());
|
||||
String backupZNode =
|
||||
ZKUtil.joinZNode(this.watcher.backupMasterAddressesZNode, this.sn.toString());
|
||||
if (MasterAddressTracker.setMasterAddress(this.watcher, this.watcher.getMasterAddressZNode(), this.sn)) {
|
||||
// If we were a backup master before, delete our ZNode from the backup
|
||||
// master directory since we are the active now
|
||||
|
@ -174,7 +176,14 @@ class ActiveMasterManager extends ZooKeeperListener {
|
|||
msg = ("A master was detected, but went down before its address " +
|
||||
"could be read. Attempting to become the next active master");
|
||||
} else {
|
||||
ServerName currentMaster = ZKUtil.znodeContentToServerName(bytes);
|
||||
ServerName currentMaster;
|
||||
try {
|
||||
currentMaster = ServerName.parseFrom(bytes);
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Failed parse", e);
|
||||
// Hopefully next time around we won't fail the parse. Dangerous.
|
||||
continue;
|
||||
}
|
||||
if (ServerName.isSameHostnameAndPort(currentMaster, this.sn)) {
|
||||
msg = ("Current master has this master's address, " +
|
||||
currentMaster + "; master was restarted? Deleting node.");
|
||||
|
|
|
@ -48,10 +48,12 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Chore;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HServerLoad;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.RegionTransition;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
|
@ -62,7 +64,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;
|
||||
|
@ -489,29 +490,34 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @throws IOException
|
||||
*/
|
||||
boolean processRegionInTransition(final String encodedRegionName,
|
||||
final HRegionInfo regionInfo,
|
||||
final Map<ServerName,List<Pair<HRegionInfo,Result>>> deadServers)
|
||||
final HRegionInfo regionInfo, final Map<ServerName,List<Pair<HRegionInfo,Result>>> deadServers)
|
||||
throws KeeperException, IOException {
|
||||
Stat stat = new Stat();
|
||||
RegionTransitionData data = ZKAssign.getDataAndWatch(watcher,
|
||||
encodedRegionName, stat);
|
||||
byte [] data = ZKAssign.getDataAndWatch(watcher, encodedRegionName, stat);
|
||||
if (data == null) return false;
|
||||
RegionTransition rt;
|
||||
try {
|
||||
rt = RegionTransition.parseFrom(data);
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Failed parse znode data", e);
|
||||
return false;
|
||||
}
|
||||
HRegionInfo hri = regionInfo;
|
||||
if (hri == null) {
|
||||
if ((hri = getHRegionInfo(data)) == null) return false;
|
||||
if ((hri = getHRegionInfo(rt.getRegionName())) == null) return false;
|
||||
}
|
||||
processRegionsInTransition(data, hri, deadServers, stat.getVersion());
|
||||
processRegionsInTransition(rt, hri, deadServers, stat.getVersion());
|
||||
return true;
|
||||
}
|
||||
|
||||
void processRegionsInTransition(final RegionTransitionData data,
|
||||
final HRegionInfo regionInfo,
|
||||
final Map<ServerName, List<Pair<HRegionInfo, Result>>> deadServers,
|
||||
int expectedVersion)
|
||||
void processRegionsInTransition(final RegionTransition rt, final HRegionInfo regionInfo,
|
||||
final Map<ServerName, List<Pair<HRegionInfo, Result>>> deadServers, int expectedVersion)
|
||||
throws KeeperException {
|
||||
EventType et = rt.getEventType();
|
||||
// Get ServerName. Could be null.
|
||||
ServerName sn = rt.getServerName();
|
||||
String encodedRegionName = regionInfo.getEncodedName();
|
||||
LOG.info("Processing region " + regionInfo.getRegionNameAsString() +
|
||||
" in state " + data.getEventType());
|
||||
LOG.info("Processing region " + regionInfo.getRegionNameAsString() + " in state " + et);
|
||||
synchronized (regionsInTransition) {
|
||||
RegionState regionState = regionsInTransition.get(encodedRegionName);
|
||||
if (regionState != null ||
|
||||
|
@ -519,21 +525,19 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// Just return
|
||||
return;
|
||||
}
|
||||
switch (data.getEventType()) {
|
||||
switch (et) {
|
||||
case M_ZK_REGION_CLOSING:
|
||||
// If zk node of the region was updated by a live server skip this
|
||||
// region and just add it into RIT.
|
||||
if (isOnDeadServer(regionInfo, deadServers) &&
|
||||
(data.getOrigin() == null || !serverManager.isServerOnline(data.getOrigin()))) {
|
||||
if (isOnDeadServer(regionInfo, deadServers) && (sn == null || !isServerOnline(sn))) {
|
||||
// If was on dead server, its closed now. Force to OFFLINE and this
|
||||
// will get it reassigned if appropriate
|
||||
forceOffline(regionInfo, data);
|
||||
forceOffline(regionInfo, rt);
|
||||
} else {
|
||||
// Just insert region into RIT.
|
||||
// If this never updates the timeout will trigger new assignment
|
||||
regionsInTransition.put(encodedRegionName, new RegionState(
|
||||
regionInfo, RegionState.State.CLOSING,
|
||||
data.getStamp(), data.getOrigin()));
|
||||
regionsInTransition.put(encodedRegionName,
|
||||
getRegionState(regionInfo, RegionState.State.CLOSING, rt));
|
||||
}
|
||||
failoverProcessedRegions.put(encodedRegionName, regionInfo);
|
||||
break;
|
||||
|
@ -541,27 +545,23 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
case RS_ZK_REGION_CLOSED:
|
||||
case RS_ZK_REGION_FAILED_OPEN:
|
||||
// Region is closed, insert into RIT and handle it
|
||||
addToRITandCallClose(regionInfo, RegionState.State.CLOSED, data);
|
||||
addToRITandCallClose(regionInfo, RegionState.State.CLOSED, rt);
|
||||
failoverProcessedRegions.put(encodedRegionName, regionInfo);
|
||||
break;
|
||||
|
||||
case M_ZK_REGION_OFFLINE:
|
||||
// If zk node of the region was updated by a live server skip this
|
||||
// region and just add it into RIT.
|
||||
if (isOnDeadServer(regionInfo, deadServers) &&
|
||||
(data.getOrigin() == null ||
|
||||
!serverManager.isServerOnline(data.getOrigin()))) {
|
||||
if (isOnDeadServer(regionInfo, deadServers) && (sn == null || !isServerOnline(sn))) {
|
||||
// Region is offline, insert into RIT and handle it like a closed
|
||||
addToRITandCallClose(regionInfo, RegionState.State.OFFLINE, data);
|
||||
} else if (data.getOrigin() != null &&
|
||||
!serverManager.isServerOnline(data.getOrigin())) {
|
||||
addToRITandCallClose(regionInfo, RegionState.State.OFFLINE, rt);
|
||||
} else if (sn != null && !isServerOnline(sn)) {
|
||||
// to handle cases where offline node is created but sendRegionOpen
|
||||
// RPC is not yet sent
|
||||
addToRITandCallClose(regionInfo, RegionState.State.OFFLINE, data);
|
||||
addToRITandCallClose(regionInfo, RegionState.State.OFFLINE, rt);
|
||||
} else {
|
||||
regionsInTransition.put(encodedRegionName, new RegionState(
|
||||
regionInfo, RegionState.State.PENDING_OPEN, data.getStamp(), data
|
||||
.getOrigin()));
|
||||
regionsInTransition.put(encodedRegionName,
|
||||
getRegionState(regionInfo, RegionState.State.PENDING_OPEN, rt));
|
||||
}
|
||||
failoverProcessedRegions.put(encodedRegionName, regionInfo);
|
||||
break;
|
||||
|
@ -573,9 +573,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// Just insert region into RIT
|
||||
// If this never updates the timeout will trigger new assignment
|
||||
if (regionInfo.isMetaTable()) {
|
||||
regionsInTransition.put(encodedRegionName, new RegionState(
|
||||
regionInfo, RegionState.State.OPENING, data.getStamp(), data
|
||||
.getOrigin()));
|
||||
regionsInTransition.put(encodedRegionName,
|
||||
getRegionState(regionInfo, RegionState.State.OPENING, rt));
|
||||
// If ROOT or .META. table is waiting for timeout monitor to assign
|
||||
// it may take lot of time when the assignment.timeout.period is
|
||||
// the default value which may be very long. We will not be able
|
||||
|
@ -584,17 +583,15 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
processOpeningState(regionInfo);
|
||||
break;
|
||||
}
|
||||
regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
|
||||
RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
|
||||
regionsInTransition.put(encodedRegionName,
|
||||
getRegionState(regionInfo, RegionState.State.OPENING, rt));
|
||||
failoverProcessedRegions.put(encodedRegionName, regionInfo);
|
||||
break;
|
||||
|
||||
case RS_ZK_REGION_OPENED:
|
||||
// Region is opened, insert into RIT and handle it
|
||||
regionsInTransition.put(encodedRegionName, new RegionState(
|
||||
regionInfo, RegionState.State.OPEN,
|
||||
data.getStamp(), data.getOrigin()));
|
||||
ServerName sn = data.getOrigin() == null? null: data.getOrigin();
|
||||
regionsInTransition.put(encodedRegionName,
|
||||
getRegionState(regionInfo, RegionState.State.OPEN, rt));
|
||||
// sn could be null if this server is no longer online. If
|
||||
// that is the case, just let this RIT timeout; it'll be assigned
|
||||
// to new server then.
|
||||
|
@ -605,10 +602,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
} else if (!serverManager.isServerOnline(sn)
|
||||
&& (isOnDeadServer(regionInfo, deadServers)
|
||||
|| regionInfo.isMetaRegion() || regionInfo.isRootRegion())) {
|
||||
forceOffline(regionInfo, data);
|
||||
forceOffline(regionInfo, rt);
|
||||
} else {
|
||||
new OpenedRegionHandler(master, this, regionInfo, sn, expectedVersion)
|
||||
.process();
|
||||
new OpenedRegionHandler(master, this, regionInfo, sn, expectedVersion).process();
|
||||
}
|
||||
failoverProcessedRegions.put(encodedRegionName, regionInfo);
|
||||
break;
|
||||
|
@ -623,19 +619,18 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
/**
|
||||
* Put the region <code>hri</code> into an offline state up in zk.
|
||||
* @param hri
|
||||
* @param oldData
|
||||
* @param oldRt
|
||||
* @throws KeeperException
|
||||
*/
|
||||
private void forceOffline(final HRegionInfo hri,
|
||||
final RegionTransitionData oldData)
|
||||
private void forceOffline(final HRegionInfo hri, final RegionTransition oldRt)
|
||||
throws KeeperException {
|
||||
// If was on dead server, its closed now. Force to OFFLINE and then
|
||||
// handle it like a close; this will get it reassigned if appropriate
|
||||
LOG.debug("RIT " + hri.getEncodedName() + " in state=" +
|
||||
oldData.getEventType() + " was on deadserver; forcing offline");
|
||||
LOG.debug("RIT " + hri.getEncodedName() + " in state=" + oldRt.getEventType() +
|
||||
" was on deadserver; forcing offline");
|
||||
ZKAssign.createOrForceNodeOffline(this.watcher, hri,
|
||||
this.master.getServerName());
|
||||
addToRITandCallClose(hri, RegionState.State.OFFLINE, oldData);
|
||||
addToRITandCallClose(hri, RegionState.State.OFFLINE, oldRt);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -646,12 +641,22 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @param oldData
|
||||
*/
|
||||
private void addToRITandCallClose(final HRegionInfo hri,
|
||||
final RegionState.State state, final RegionTransitionData oldData) {
|
||||
this.regionsInTransition.put(hri.getEncodedName(),
|
||||
new RegionState(hri, state, oldData.getStamp(), oldData.getOrigin()));
|
||||
final RegionState.State state, final RegionTransition oldData) {
|
||||
this.regionsInTransition.put(hri.getEncodedName(), getRegionState(hri, state, oldData));
|
||||
new ClosedRegionHandler(this.master, this, hri).process();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param hri
|
||||
* @param state
|
||||
* @param rt
|
||||
* @return A new {@link RegionState} instance made of the passed arguments
|
||||
*/
|
||||
RegionState getRegionState(final HRegionInfo hri, final RegionState.State state,
|
||||
final RegionTransition rt) {
|
||||
return new RegionState(hri, state, rt.getCreateTime(), rt.getServerName());
|
||||
}
|
||||
|
||||
/**
|
||||
* When a region is closed, it should be removed from the regionsToReopen
|
||||
* @param hri HRegionInfo of the region which was closed
|
||||
|
@ -689,41 +694,46 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* <p>
|
||||
* This deals with skipped transitions (we got a CLOSED but didn't see CLOSING
|
||||
* yet).
|
||||
* @param data
|
||||
* @param rt
|
||||
* @param expectedVersion
|
||||
*/
|
||||
private void handleRegion(final RegionTransitionData data, int expectedVersion) {
|
||||
private void handleRegion(final RegionTransition rt, int expectedVersion) {
|
||||
synchronized(regionsInTransition) {
|
||||
HRegionInfo hri = null;
|
||||
if (data == null || data.getOrigin() == null) {
|
||||
LOG.warn("Unexpected NULL input " + data);
|
||||
if (rt == null) {
|
||||
LOG.warn("Unexpected NULL input " + rt);
|
||||
return;
|
||||
}
|
||||
final ServerName sn = rt.getServerName();
|
||||
if (sn == null) {
|
||||
LOG.warn("Null servername: " + rt);
|
||||
return;
|
||||
}
|
||||
ServerName sn = data.getOrigin();
|
||||
// Check if this is a special HBCK transition
|
||||
if (sn.equals(HConstants.HBCK_CODE_SERVERNAME)) {
|
||||
handleHBCK(data);
|
||||
handleHBCK(rt);
|
||||
return;
|
||||
}
|
||||
String encodedName = HRegionInfo.encodeRegionName(data.getRegionName());
|
||||
final long createTime = rt.getCreateTime();
|
||||
final byte [] regionName = rt.getRegionName();
|
||||
String encodedName = HRegionInfo.encodeRegionName(regionName);
|
||||
String prettyPrintedRegionName = HRegionInfo.prettyPrint(encodedName);
|
||||
// Verify this is a known server
|
||||
if (!serverManager.isServerOnline(sn) &&
|
||||
!this.master.getServerName().equals(sn)
|
||||
&& !ignoreStatesRSOffline.contains(data.getEventType())) {
|
||||
&& !ignoreStatesRSOffline.contains(rt.getEventType())) {
|
||||
LOG.warn("Attempted to handle region transition for server but " +
|
||||
"server is not online: " + prettyPrintedRegionName);
|
||||
return;
|
||||
}
|
||||
// Printing if the event was created a long time ago helps debugging
|
||||
boolean lateEvent = data.getStamp() <
|
||||
(System.currentTimeMillis() - 15000);
|
||||
LOG.debug("Handling transition=" + data.getEventType() +
|
||||
", server=" + data.getOrigin() + ", region=" +
|
||||
boolean lateEvent = createTime < (System.currentTimeMillis() - 15000);
|
||||
LOG.debug("Handling transition=" + rt.getEventType() +
|
||||
", server=" + sn + ", region=" +
|
||||
(prettyPrintedRegionName == null? "null": prettyPrintedRegionName) +
|
||||
(lateEvent? ", which is more than 15 seconds late" : ""));
|
||||
RegionState regionState = regionsInTransition.get(encodedName);
|
||||
switch (data.getEventType()) {
|
||||
switch (rt.getEventType()) {
|
||||
case M_ZK_REGION_OFFLINE:
|
||||
// Nothing to do.
|
||||
break;
|
||||
|
@ -751,7 +761,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
" but region was not first in SPLITTING state; continuing");
|
||||
}
|
||||
// Check it has daughters.
|
||||
byte [] payload = data.getPayload();
|
||||
byte [] payload = rt.getPayload();
|
||||
List<HRegionInfo> daughters = null;
|
||||
try {
|
||||
daughters = Writables.getHRegionInfos(payload, 0, payload.length);
|
||||
|
@ -772,10 +782,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
break;
|
||||
|
||||
case M_ZK_REGION_CLOSING:
|
||||
hri = checkIfInFailover(regionState, encodedName, data);
|
||||
hri = checkIfInFailover(regionState, encodedName, regionName);
|
||||
if (hri != null) {
|
||||
regionState = new RegionState(hri, RegionState.State.CLOSING, data
|
||||
.getStamp(), data.getOrigin());
|
||||
regionState = new RegionState(hri, RegionState.State.CLOSING, createTime, sn);
|
||||
regionsInTransition.put(encodedName, regionState);
|
||||
failoverProcessedRegions.put(encodedName, hri);
|
||||
break;
|
||||
|
@ -785,21 +794,19 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
if (regionState == null ||
|
||||
(!regionState.isPendingClose() && !regionState.isClosing())) {
|
||||
LOG.warn("Received CLOSING for region " + prettyPrintedRegionName +
|
||||
" from server " + data.getOrigin() + " but region was in " +
|
||||
" from server " + sn + " but region was in " +
|
||||
" the state " + regionState + " and not " +
|
||||
"in expected PENDING_CLOSE or CLOSING states");
|
||||
return;
|
||||
}
|
||||
// Transition to CLOSING (or update stamp if already CLOSING)
|
||||
regionState.update(RegionState.State.CLOSING,
|
||||
data.getStamp(), data.getOrigin());
|
||||
regionState.update(RegionState.State.CLOSING, createTime, sn);
|
||||
break;
|
||||
|
||||
case RS_ZK_REGION_CLOSED:
|
||||
hri = checkIfInFailover(regionState, encodedName, data);
|
||||
hri = checkIfInFailover(regionState, encodedName, regionName);
|
||||
if (hri != null) {
|
||||
regionState = new RegionState(hri, RegionState.State.CLOSED, data
|
||||
.getStamp(), data.getOrigin());
|
||||
regionState = new RegionState(hri, RegionState.State.CLOSED, createTime, sn);
|
||||
regionsInTransition.put(encodedName, regionState);
|
||||
removeClosedRegion(regionState.getRegion());
|
||||
new ClosedRegionHandler(master, this, regionState.getRegion())
|
||||
|
@ -811,7 +818,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
if (regionState == null ||
|
||||
(!regionState.isPendingClose() && !regionState.isClosing())) {
|
||||
LOG.warn("Received CLOSED for region " + prettyPrintedRegionName +
|
||||
" from server " + data.getOrigin() + " but region was in " +
|
||||
" from server " + sn + " but region was in " +
|
||||
" the state " + regionState + " and not " +
|
||||
"in expected PENDING_CLOSE or CLOSING states");
|
||||
return;
|
||||
|
@ -819,18 +826,16 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// Handle CLOSED by assigning elsewhere or stopping if a disable
|
||||
// If we got here all is good. Need to update RegionState -- else
|
||||
// what follows will fail because not in expected state.
|
||||
regionState.update(RegionState.State.CLOSED,
|
||||
data.getStamp(), data.getOrigin());
|
||||
regionState.update(RegionState.State.CLOSED, createTime, sn);
|
||||
removeClosedRegion(regionState.getRegion());
|
||||
this.executorService.submit(new ClosedRegionHandler(master,
|
||||
this, regionState.getRegion()));
|
||||
break;
|
||||
|
||||
case RS_ZK_REGION_FAILED_OPEN:
|
||||
hri = checkIfInFailover(regionState, encodedName, data);
|
||||
hri = checkIfInFailover(regionState, encodedName, regionName);
|
||||
if (hri != null) {
|
||||
regionState = new RegionState(hri, RegionState.State.CLOSED, data
|
||||
.getStamp(), data.getOrigin());
|
||||
regionState = new RegionState(hri, RegionState.State.CLOSED, createTime, sn);
|
||||
regionsInTransition.put(encodedName, regionState);
|
||||
new ClosedRegionHandler(master, this, regionState.getRegion())
|
||||
.process();
|
||||
|
@ -840,22 +845,20 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
if (regionState == null ||
|
||||
(!regionState.isPendingOpen() && !regionState.isOpening())) {
|
||||
LOG.warn("Received FAILED_OPEN for region " + prettyPrintedRegionName +
|
||||
" from server " + data.getOrigin() + " but region was in " +
|
||||
" from server " + sn + " but region was in " +
|
||||
" the state " + regionState + " and not in PENDING_OPEN or OPENING");
|
||||
return;
|
||||
}
|
||||
// Handle this the same as if it were opened and then closed.
|
||||
regionState.update(RegionState.State.CLOSED,
|
||||
data.getStamp(), data.getOrigin());
|
||||
regionState.update(RegionState.State.CLOSED, createTime, sn);
|
||||
this.executorService.submit(new ClosedRegionHandler(master,
|
||||
this, regionState.getRegion()));
|
||||
break;
|
||||
|
||||
case RS_ZK_REGION_OPENING:
|
||||
hri = checkIfInFailover(regionState, encodedName, data);
|
||||
hri = checkIfInFailover(regionState, encodedName, regionName);
|
||||
if (hri != null) {
|
||||
regionState = new RegionState(hri, RegionState.State.OPENING, data
|
||||
.getStamp(), data.getOrigin());
|
||||
regionState = new RegionState(hri, RegionState.State.OPENING, createTime, sn);
|
||||
regionsInTransition.put(encodedName, regionState);
|
||||
failoverProcessedRegions.put(encodedName, hri);
|
||||
break;
|
||||
|
@ -866,24 +869,21 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
(!regionState.isPendingOpen() && !regionState.isOpening())) {
|
||||
LOG.warn("Received OPENING for region " +
|
||||
prettyPrintedRegionName +
|
||||
" from server " + data.getOrigin() + " but region was in " +
|
||||
" from server " + sn + " but region was in " +
|
||||
" the state " + regionState + " and not " +
|
||||
"in expected PENDING_OPEN or OPENING states");
|
||||
return;
|
||||
}
|
||||
// Transition to OPENING (or update stamp if already OPENING)
|
||||
regionState.update(RegionState.State.OPENING,
|
||||
data.getStamp(), data.getOrigin());
|
||||
regionState.update(RegionState.State.OPENING, createTime, sn);
|
||||
break;
|
||||
|
||||
case RS_ZK_REGION_OPENED:
|
||||
hri = checkIfInFailover(regionState, encodedName, data);
|
||||
hri = checkIfInFailover(regionState, encodedName, regionName);
|
||||
if (hri != null) {
|
||||
regionState = new RegionState(hri, RegionState.State.OPEN, data
|
||||
.getStamp(), data.getOrigin());
|
||||
regionState = new RegionState(hri, RegionState.State.OPEN, createTime, sn);
|
||||
regionsInTransition.put(encodedName, regionState);
|
||||
new OpenedRegionHandler(master, this, regionState.getRegion(), data
|
||||
.getOrigin(), expectedVersion).process();
|
||||
new OpenedRegionHandler(master, this, regionState.getRegion(), sn, expectedVersion).process();
|
||||
failoverProcessedRegions.put(encodedName, hri);
|
||||
break;
|
||||
}
|
||||
|
@ -892,17 +892,15 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
(!regionState.isPendingOpen() && !regionState.isOpening())) {
|
||||
LOG.warn("Received OPENED for region " +
|
||||
prettyPrintedRegionName +
|
||||
" from server " + data.getOrigin() + " but region was in " +
|
||||
" from server " + sn + " but region was in " +
|
||||
" the state " + regionState + " and not " +
|
||||
"in expected PENDING_OPEN or OPENING states");
|
||||
return;
|
||||
}
|
||||
// Handle OPENED by removing from transition and deleted zk node
|
||||
regionState.update(RegionState.State.OPEN,
|
||||
data.getStamp(), data.getOrigin());
|
||||
regionState.update(RegionState.State.OPEN, createTime, sn);
|
||||
this.executorService.submit(
|
||||
new OpenedRegionHandler(master, this, regionState.getRegion(),
|
||||
data.getOrigin(), expectedVersion));
|
||||
new OpenedRegionHandler(master, this, regionState.getRegion(), sn, expectedVersion));
|
||||
break;
|
||||
|
||||
default:
|
||||
|
@ -920,12 +918,12 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @return hri
|
||||
*/
|
||||
private HRegionInfo checkIfInFailover(RegionState regionState,
|
||||
String encodedName, RegionTransitionData data) {
|
||||
String encodedName, final byte [] regionName) {
|
||||
if (regionState == null && this.failover &&
|
||||
(failoverProcessedRegions.containsKey(encodedName) == false ||
|
||||
failoverProcessedRegions.get(encodedName) == null)) {
|
||||
HRegionInfo hri = this.failoverProcessedRegions.get(encodedName);
|
||||
if (hri == null) hri = getHRegionInfo(data);
|
||||
if (hri == null) hri = getHRegionInfo(regionName);
|
||||
return hri;
|
||||
}
|
||||
return null;
|
||||
|
@ -933,18 +931,18 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
|
||||
/**
|
||||
* Gets the HRegionInfo from the META table
|
||||
* @param data
|
||||
* @param regionName
|
||||
* @return HRegionInfo hri for the region
|
||||
*/
|
||||
private HRegionInfo getHRegionInfo(RegionTransitionData data) {
|
||||
private HRegionInfo getHRegionInfo(final byte [] regionName) {
|
||||
Pair<HRegionInfo, ServerName> p = null;
|
||||
try {
|
||||
p = MetaReader.getRegion(catalogTracker, data.getRegionName());
|
||||
p = MetaReader.getRegion(catalogTracker, regionName);
|
||||
if (p == null) return null;
|
||||
return p.getFirst();
|
||||
} catch (IOException e) {
|
||||
master.abort("Aborting because error occoured while reading "
|
||||
+ Bytes.toStringBinary(data.getRegionName()) + " from .META.", e);
|
||||
master.abort("Aborting because error occoured while reading " +
|
||||
Bytes.toStringBinary(regionName) + " from .META.", e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
@ -1042,22 +1040,22 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* Handle a ZK unassigned node transition triggered by HBCK repair tool.
|
||||
* <p>
|
||||
* This is handled in a separate code path because it breaks the normal rules.
|
||||
* @param data
|
||||
* @param rt
|
||||
*/
|
||||
private void handleHBCK(RegionTransitionData data) {
|
||||
String encodedName = HRegionInfo.encodeRegionName(data.getRegionName());
|
||||
LOG.info("Handling HBCK triggered transition=" + data.getEventType() +
|
||||
", server=" + data.getOrigin() + ", region=" +
|
||||
private void handleHBCK(RegionTransition rt) {
|
||||
String encodedName = HRegionInfo.encodeRegionName(rt.getRegionName());
|
||||
LOG.info("Handling HBCK triggered transition=" + rt.getEventType() +
|
||||
", server=" + rt.getServerName() + ", region=" +
|
||||
HRegionInfo.prettyPrint(encodedName));
|
||||
RegionState regionState = regionsInTransition.get(encodedName);
|
||||
switch (data.getEventType()) {
|
||||
switch (rt.getEventType()) {
|
||||
case M_ZK_REGION_OFFLINE:
|
||||
HRegionInfo regionInfo = null;
|
||||
if (regionState != null) {
|
||||
regionInfo = regionState.getRegion();
|
||||
} else {
|
||||
try {
|
||||
byte[] name = data.getRegionName();
|
||||
byte [] name = rt.getRegionName();
|
||||
Pair<HRegionInfo, ServerName> p = MetaReader.getRegion(catalogTracker, name);
|
||||
regionInfo = p.getFirst();
|
||||
} catch (IOException e) {
|
||||
|
@ -1072,8 +1070,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
break;
|
||||
|
||||
default:
|
||||
LOG.warn("Received unexpected region state from HBCK (" +
|
||||
data.getEventType() + ")");
|
||||
LOG.warn("Received unexpected region state from HBCK: " + rt.toString());
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -1094,18 +1091,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
*/
|
||||
@Override
|
||||
public void nodeCreated(String path) {
|
||||
if(path.startsWith(watcher.assignmentZNode)) {
|
||||
try {
|
||||
Stat stat = new Stat();
|
||||
RegionTransitionData data = ZKAssign.getDataAndWatch(watcher, path, stat);
|
||||
if (data == null) {
|
||||
return;
|
||||
}
|
||||
handleRegion(data, stat.getVersion());
|
||||
} catch (KeeperException e) {
|
||||
master.abort("Unexpected ZK exception reading unassigned node data", e);
|
||||
}
|
||||
}
|
||||
handleAssignmentEvent(path);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1122,17 +1108,21 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
*/
|
||||
@Override
|
||||
public void nodeDataChanged(String path) {
|
||||
if(path.startsWith(watcher.assignmentZNode)) {
|
||||
try {
|
||||
Stat stat = new Stat();
|
||||
RegionTransitionData data = ZKAssign.getDataAndWatch(watcher, path, stat);
|
||||
if (data == null) {
|
||||
return;
|
||||
}
|
||||
handleRegion(data, stat.getVersion());
|
||||
} catch (KeeperException e) {
|
||||
master.abort("Unexpected ZK exception reading unassigned node data", e);
|
||||
}
|
||||
handleAssignmentEvent(path);
|
||||
}
|
||||
|
||||
private void handleAssignmentEvent(final String path) {
|
||||
if (!path.startsWith(watcher.assignmentZNode)) return;
|
||||
try {
|
||||
Stat stat = new Stat();
|
||||
byte [] data = ZKAssign.getDataAndWatch(watcher, path, stat);
|
||||
if (data == null) return;
|
||||
RegionTransition rt = RegionTransition.parseFrom(data);
|
||||
handleRegion(rt, stat.getVersion());
|
||||
} catch (KeeperException e) {
|
||||
master.abort("Unexpected ZK exception reading unassigned node data", e);
|
||||
} catch (DeserializationException e) {
|
||||
master.abort("Unexpected exception deserializing node data", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1983,7 +1973,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
+ "can't be created.");
|
||||
return;
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
} catch (KeeperException ee) {
|
||||
Exception e = ee;
|
||||
if (e instanceof NodeExistsException) {
|
||||
// Handle race between master initiated close and regionserver
|
||||
// orchestrated splitting. See if existing node is in a
|
||||
|
@ -2004,6 +1995,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
return;
|
||||
} catch (KeeperException ke) {
|
||||
LOG.error("Unexpected zk state", ke);
|
||||
} catch (DeserializationException de) {
|
||||
LOG.error("Failed parse", de);
|
||||
}
|
||||
}
|
||||
// If we get here, don't understand whats going on -- abort.
|
||||
|
@ -2125,14 +2118,17 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @param path
|
||||
* @return True if znode is in SPLIT or SPLITTING state.
|
||||
* @throws KeeperException Can happen if the znode went away in meantime.
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
private boolean isSplitOrSplitting(final String path) throws KeeperException {
|
||||
private boolean isSplitOrSplitting(final String path)
|
||||
throws KeeperException, DeserializationException {
|
||||
boolean result = false;
|
||||
// This may fail if the SPLIT or SPLITTING znode gets cleaned up before we
|
||||
// can get data from it.
|
||||
RegionTransitionData data = ZKAssign.getData(master.getZooKeeper(), path);
|
||||
EventType evt = data.getEventType();
|
||||
switch (evt) {
|
||||
byte [] data = ZKAssign.getData(master.getZooKeeper(), path);
|
||||
if (data == null) return false;
|
||||
RegionTransition rt = RegionTransition.parseFrom(data);
|
||||
switch (rt.getEventType()) {
|
||||
case RS_ZK_REGION_SPLIT:
|
||||
case RS_ZK_REGION_SPLITTING:
|
||||
result = true;
|
||||
|
@ -2659,48 +2655,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @throws KeeperException
|
||||
*/
|
||||
private void processDeadServersAndRecoverLostRegions(
|
||||
Map<ServerName, List<Pair<HRegionInfo, Result>>> deadServers,
|
||||
List<String> nodes) throws IOException, KeeperException {
|
||||
if (null != deadServers) {
|
||||
for (Map.Entry<ServerName, List<Pair<HRegionInfo, Result>>> deadServer :
|
||||
deadServers.entrySet()) {
|
||||
List<Pair<HRegionInfo, Result>> regions = deadServer.getValue();
|
||||
for (Pair<HRegionInfo, Result> region : regions) {
|
||||
HRegionInfo regionInfo = region.getFirst();
|
||||
Result result = region.getSecond();
|
||||
// If region was in transition (was in zk) force it offline for
|
||||
// reassign
|
||||
try {
|
||||
RegionTransitionData data = ZKAssign.getData(watcher,
|
||||
regionInfo.getEncodedName());
|
||||
|
||||
// If zk node of this region has been updated by a live server,
|
||||
// we consider that this region is being handled.
|
||||
// So we should skip it and process it in
|
||||
// processRegionsInTransition.
|
||||
if (data != null && data.getOrigin() != null &&
|
||||
serverManager.isServerOnline(data.getOrigin())) {
|
||||
LOG.info("The region " + regionInfo.getEncodedName()
|
||||
+ "is being handled on " + data.getOrigin());
|
||||
continue;
|
||||
}
|
||||
// Process with existing RS shutdown code
|
||||
boolean assign = ServerShutdownHandler.processDeadRegion(
|
||||
regionInfo, result, this, this.catalogTracker);
|
||||
if (assign) {
|
||||
ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
|
||||
master.getServerName());
|
||||
if (!nodes.contains(regionInfo.getEncodedName())) {
|
||||
nodes.add(regionInfo.getEncodedName());
|
||||
}
|
||||
}
|
||||
} catch (KeeperException.NoNodeException nne) {
|
||||
// This is fine
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Map<ServerName, List<Pair<HRegionInfo, Result>>> deadServers, List<String> nodes)
|
||||
throws IOException, KeeperException {
|
||||
processDeadServers(deadServers, nodes);
|
||||
if (!nodes.isEmpty()) {
|
||||
for (String encodedRegionName : nodes) {
|
||||
processRegionInTransition(encodedRegionName, null, deadServers);
|
||||
|
@ -2708,6 +2665,59 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
}
|
||||
|
||||
private void processDeadServers(Map<ServerName, List<Pair<HRegionInfo, Result>>> deadServers,
|
||||
final List<String> nodes)
|
||||
throws KeeperException, IOException {
|
||||
if (deadServers == null) return;
|
||||
for (Map.Entry<ServerName, List<Pair<HRegionInfo, Result>>> deadServer: deadServers.entrySet()) {
|
||||
List<Pair<HRegionInfo, Result>> regions = deadServer.getValue();
|
||||
for (Pair<HRegionInfo, Result> region : regions) {
|
||||
HRegionInfo regionInfo = region.getFirst();
|
||||
Result result = region.getSecond();
|
||||
try {
|
||||
// If region was in transition (was in zk) force it offline for reassign. Check if node
|
||||
// up in zk at all first.
|
||||
if (ZKUtil.checkExists(this.watcher,
|
||||
ZKAssign.getPath(this.watcher, regionInfo.getEncodedName())) != -1) {
|
||||
byte [] data = ZKAssign.getData(watcher, regionInfo.getEncodedName());
|
||||
if (data == null) {
|
||||
LOG.warn("No data in znode for " + regionInfo.getEncodedName());
|
||||
continue;
|
||||
}
|
||||
RegionTransition rt;
|
||||
try {
|
||||
rt = RegionTransition.parseFrom(data);
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Failed parse of znode data for " + regionInfo.getEncodedName(), e);
|
||||
continue;
|
||||
}
|
||||
|
||||
// If zk node of this region has been updated by a live server,
|
||||
// we consider that this region is being handled.
|
||||
// So we should skip it and process it in processRegionsInTransition.
|
||||
ServerName sn = rt.getServerName();
|
||||
if (isServerOnline(sn)) {
|
||||
LOG.info("The region " + regionInfo.getEncodedName() + "is being handled on " + sn);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
// Process with existing RS shutdown code
|
||||
boolean assign = ServerShutdownHandler.processDeadRegion(
|
||||
regionInfo, result, this, this.catalogTracker);
|
||||
if (assign) {
|
||||
ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
|
||||
master.getServerName());
|
||||
if (!nodes.contains(regionInfo.getEncodedName())) {
|
||||
nodes.add(regionInfo.getEncodedName());
|
||||
}
|
||||
}
|
||||
} catch (KeeperException.NoNodeException nne) {
|
||||
// This is fine
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Presumes caller has taken care of necessary locking modifying servers Map.
|
||||
* @param hsi
|
||||
|
@ -2967,27 +2977,28 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
try {
|
||||
String node = ZKAssign.getNodeName(watcher, regionInfo.getEncodedName());
|
||||
Stat stat = new Stat();
|
||||
RegionTransitionData dataInZNode = ZKAssign.getDataNoWatch(watcher, node,
|
||||
stat);
|
||||
if (dataInZNode == null) {
|
||||
byte [] data = ZKAssign.getDataNoWatch(watcher, node, stat);
|
||||
if (data == null) {
|
||||
LOG.warn("Data is null, node " + node + " no longer exists");
|
||||
return;
|
||||
}
|
||||
if (dataInZNode.getEventType() == EventType.RS_ZK_REGION_OPENED) {
|
||||
RegionTransition rt = RegionTransition.parseFrom(data);
|
||||
EventType et = rt.getEventType();
|
||||
if (et == EventType.RS_ZK_REGION_OPENED) {
|
||||
LOG.debug("Region has transitioned to OPENED, allowing "
|
||||
+ "watched event handlers to process");
|
||||
return;
|
||||
} else if (dataInZNode.getEventType() != EventType.RS_ZK_REGION_OPENING &&
|
||||
dataInZNode.getEventType() != EventType.RS_ZK_REGION_FAILED_OPEN ) {
|
||||
LOG.warn("While timing out a region in state OPENING, "
|
||||
+ "found ZK node in unexpected state: "
|
||||
+ dataInZNode.getEventType());
|
||||
} else if (et != EventType.RS_ZK_REGION_OPENING && et != EventType.RS_ZK_REGION_FAILED_OPEN ) {
|
||||
LOG.warn("While timing out a region, found ZK node in unexpected state: " + et);
|
||||
return;
|
||||
}
|
||||
invokeAssign(regionInfo);
|
||||
} catch (KeeperException ke) {
|
||||
LOG.error("Unexpected ZK exception timing out CLOSING region", ke);
|
||||
return;
|
||||
} catch (DeserializationException e) {
|
||||
LOG.error("Unexpected exception parsing CLOSING region", e);
|
||||
return;
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
@ -3019,16 +3030,18 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @return whether the serverName currently hosts the region
|
||||
*/
|
||||
public boolean isCarryingRegion(ServerName serverName, HRegionInfo hri) {
|
||||
RegionTransitionData data = null;
|
||||
RegionTransition rt = null;
|
||||
try {
|
||||
data = ZKAssign.getData(master.getZooKeeper(), hri.getEncodedName());
|
||||
byte [] data = ZKAssign.getData(master.getZooKeeper(), hri.getEncodedName());
|
||||
// This call can legitimately come by null
|
||||
rt = data == null? null: RegionTransition.parseFrom(data);
|
||||
} catch (KeeperException e) {
|
||||
master.abort("Unexpected ZK exception reading unassigned node for region="
|
||||
+ hri.getEncodedName(), e);
|
||||
master.abort("Exception reading unassigned node for region=" + hri.getEncodedName(), e);
|
||||
} catch (DeserializationException e) {
|
||||
master.abort("Exception parsing unassigned node for region=" + hri.getEncodedName(), e);
|
||||
}
|
||||
|
||||
ServerName addressFromZK = (data != null && data.getOrigin() != null) ?
|
||||
data.getOrigin() : null;
|
||||
ServerName addressFromZK = rt != null? rt.getServerName(): null;
|
||||
if (addressFromZK != null) {
|
||||
// if we get something from ZK, we will use the data
|
||||
boolean matchZK = (addressFromZK != null &&
|
||||
|
@ -3048,6 +3061,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
|
||||
return matchAM;
|
||||
}
|
||||
|
||||
/**
|
||||
* Process shutdown server removing any assignments.
|
||||
* @param sn Server that went down.
|
||||
|
@ -3391,7 +3405,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @return True if online.
|
||||
*/
|
||||
public boolean isServerOnline(ServerName serverName) {
|
||||
return this.serverManager.isServerOnline(serverName);
|
||||
return serverName != null && this.serverManager.isServerOnline(serverName);
|
||||
}
|
||||
/**
|
||||
* Shutdown the threadpool executor service
|
||||
|
|
|
@ -49,6 +49,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.Chore;
|
||||
import org.apache.hadoop.hbase.ClusterStatus;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
|
@ -1427,7 +1428,7 @@ Server {
|
|||
List<String> backupMasterStrings;
|
||||
try {
|
||||
backupMasterStrings = ZKUtil.listChildrenNoWatch(this.zooKeeper,
|
||||
this.zooKeeper.backupMasterAddressesZNode);
|
||||
this.zooKeeper.backupMasterAddressesZNode);
|
||||
} catch (KeeperException e) {
|
||||
LOG.warn(this.zooKeeper.prefix("Unable to list backup servers"), e);
|
||||
backupMasterStrings = new ArrayList<String>(0);
|
||||
|
@ -1436,9 +1437,17 @@ Server {
|
|||
backupMasterStrings.size());
|
||||
for (String s: backupMasterStrings) {
|
||||
try {
|
||||
byte[] bytes = ZKUtil.getData(this.zooKeeper, ZKUtil.joinZNode(this.zooKeeper.backupMasterAddressesZNode, s));
|
||||
byte [] bytes =
|
||||
ZKUtil.getData(this.zooKeeper, ZKUtil.joinZNode(this.zooKeeper.backupMasterAddressesZNode, s));
|
||||
if (bytes != null) {
|
||||
backupMasters.add(ZKUtil.znodeContentToServerName(bytes));
|
||||
ServerName sn;
|
||||
try {
|
||||
sn = ServerName.parseFrom(bytes);
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Failed parse, skipping registering backup server", e);
|
||||
continue;
|
||||
}
|
||||
backupMasters.add(sn);
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
LOG.warn(this.zooKeeper.prefix("Unable to get information about " +
|
||||
|
|
|
@ -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;
|
||||
|
@ -250,11 +250,10 @@ public class MasterFileSystem {
|
|||
public void splitLog(final List<ServerName> serverNames) throws IOException {
|
||||
long splitTime = 0, splitLogSize = 0;
|
||||
List<Path> logDirs = new ArrayList<Path>();
|
||||
for(ServerName serverName: serverNames){
|
||||
Path logDir = new Path(this.rootdir,
|
||||
HLog.getHLogDirectoryName(serverName.toString()));
|
||||
for (ServerName serverName: serverNames) {
|
||||
Path logDir = new Path(this.rootdir, HLog.getHLogDirectoryName(serverName.toString()));
|
||||
Path splitDir = logDir.suffix(HLog.SPLITTING_EXT);
|
||||
// rename the directory so a rogue RS doesn't create more HLogs
|
||||
// Rename the directory so a rogue RS doesn't create more HLogs
|
||||
if (fs.exists(logDir)) {
|
||||
if (!this.fs.rename(logDir, splitDir)) {
|
||||
throw new IOException("Failed fs.rename for log split: " + logDir);
|
||||
|
@ -272,7 +271,7 @@ public class MasterFileSystem {
|
|||
LOG.info("No logs to split");
|
||||
return;
|
||||
}
|
||||
|
||||
|
||||
if (distributedLogSplitting) {
|
||||
splitLogManager.handleDeadWorkers(serverNames);
|
||||
splitTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
|
|
|
@ -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,19 @@
|
|||
*/
|
||||
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 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;
|
||||
|
||||
|
@ -38,7 +41,10 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Chore;
|
||||
import org.apache.hadoop.hbase.SplitLogCounters;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SplitLogTask;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.master.SplitLogManager.TaskFinisher.Status;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
|
@ -59,36 +65,31 @@ 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.*;
|
||||
|
||||
/**
|
||||
* Distributes the task of log splitting to the available region servers.
|
||||
* Coordination happens via zookeeper. For every log file that has to be split a
|
||||
* znode is created under /hbase/splitlog. SplitLogWorkers race to grab a task.
|
||||
* znode is created under <code>/hbase/splitlog</code>. SplitLogWorkers race to grab a task.
|
||||
*
|
||||
* SplitLogManager monitors the task znodes that it creates using the
|
||||
* <p>SplitLogManager monitors the task znodes that it creates using the
|
||||
* timeoutMonitor thread. If a task's progress is slow then
|
||||
* resubmit(String, boolean) will take away the task from the owner
|
||||
* {@link SplitLogWorker} and the task will be
|
||||
* upforgrabs again. When the task is done then the task's znode is deleted by
|
||||
* SplitLogManager.
|
||||
* {@link #resubmit(String, Task, ResubmitDirective)} will take away the task from the owner
|
||||
* {@link SplitLogWorker} and the task will be up for grabs again. When the task is done then the
|
||||
* task's znode is deleted by SplitLogManager.
|
||||
*
|
||||
* Clients call {@link #splitLogDistributed(Path)} to split a region server's
|
||||
* <p>Clients call {@link #splitLogDistributed(Path)} to split a region server's
|
||||
* log files. The caller thread waits in this method until all the log files
|
||||
* have been split.
|
||||
*
|
||||
* All the zookeeper calls made by this class are asynchronous. This is mainly
|
||||
* <p>All the zookeeper calls made by this class are asynchronous. This is mainly
|
||||
* to help reduce response time seen by the callers.
|
||||
*
|
||||
* There is race in this design between the SplitLogManager and the
|
||||
* <p>There is race in this design between the SplitLogManager and the
|
||||
* SplitLogWorker. SplitLogManager might re-queue a task that has in reality
|
||||
* already been completed by a SplitLogWorker. We rely on the idempotency of
|
||||
* the log splitting task for correctness.
|
||||
*
|
||||
* It is also assumed that every log splitting task is unique and once
|
||||
* <p>It is also assumed that every log splitting task is unique and once
|
||||
* completed (either with success or with error) it will be not be submitted
|
||||
* again. If a task is resubmitted then there is a risk that old "delete task"
|
||||
* can delete the re-submission.
|
||||
|
@ -97,8 +98,13 @@ import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.*
|
|||
public class SplitLogManager extends ZooKeeperListener {
|
||||
private static final Log LOG = LogFactory.getLog(SplitLogManager.class);
|
||||
|
||||
public static final int DEFAULT_TIMEOUT = 25000; // 25 sec
|
||||
public static final int DEFAULT_ZK_RETRIES = 3;
|
||||
public static final int DEFAULT_MAX_RESUBMIT = 3;
|
||||
public static final int DEFAULT_UNASSIGNED_TIMEOUT = (3 * 60 * 1000); //3 min
|
||||
|
||||
private final Stoppable stopper;
|
||||
private final String serverName;
|
||||
private final ServerName serverName;
|
||||
private final TaskFinisher taskFinisher;
|
||||
private FileSystem fs;
|
||||
private Configuration conf;
|
||||
|
@ -110,11 +116,10 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
private long lastNodeCreateTime = Long.MAX_VALUE;
|
||||
public boolean ignoreZKDeleteForTesting = false;
|
||||
|
||||
private ConcurrentMap<String, Task> tasks =
|
||||
new ConcurrentHashMap<String, Task>();
|
||||
private ConcurrentMap<String, Task> tasks = new ConcurrentHashMap<String, Task>();
|
||||
private TimeoutMonitor timeoutMonitor;
|
||||
|
||||
private Set<String> deadWorkers = null;
|
||||
private volatile Set<ServerName> deadWorkers = null;
|
||||
private Object deadWorkersLock = new Object();
|
||||
|
||||
/**
|
||||
|
@ -130,12 +135,12 @@ 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) {
|
||||
public Status finish(ServerName workerName, String logfile) {
|
||||
String tmpname =
|
||||
ZKSplitLog.getSplitLogDirTmpComponent(workerName, logfile);
|
||||
ZKSplitLog.getSplitLogDirTmpComponent(workerName.toString(), logfile);
|
||||
try {
|
||||
HLogSplitter.moveRecoveredEditsFromTemp(tmpname, logfile, conf);
|
||||
} catch (IOException e) {
|
||||
|
@ -159,28 +164,22 @@ 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;
|
||||
this.stopper = stopper;
|
||||
this.zkretries = conf.getLong("hbase.splitlog.zk.retries",
|
||||
ZKSplitLog.DEFAULT_ZK_RETRIES);
|
||||
this.resubmit_threshold = conf.getLong("hbase.splitlog.max.resubmit",
|
||||
ZKSplitLog.DEFAULT_MAX_RESUBMIT);
|
||||
this.timeout = conf.getInt("hbase.splitlog.manager.timeout",
|
||||
ZKSplitLog.DEFAULT_TIMEOUT);
|
||||
this.zkretries = conf.getLong("hbase.splitlog.zk.retries", DEFAULT_ZK_RETRIES);
|
||||
this.resubmit_threshold = conf.getLong("hbase.splitlog.max.resubmit", DEFAULT_MAX_RESUBMIT);
|
||||
this.timeout = conf.getInt("hbase.splitlog.manager.timeout", DEFAULT_TIMEOUT);
|
||||
this.unassignedTimeout =
|
||||
conf.getInt("hbase.splitlog.manager.unassigned.timeout",
|
||||
ZKSplitLog.DEFAULT_UNASSIGNED_TIMEOUT);
|
||||
conf.getInt("hbase.splitlog.manager.unassigned.timeout", DEFAULT_UNASSIGNED_TIMEOUT);
|
||||
LOG.debug("timeout = " + timeout);
|
||||
LOG.debug("unassigned timeout = " + unassignedTimeout);
|
||||
|
||||
this.serverName = serverName;
|
||||
this.timeoutMonitor = new TimeoutMonitor(
|
||||
conf.getInt("hbase.splitlog.manager.timeoutmonitor.period",
|
||||
1000),
|
||||
stopper);
|
||||
this.timeoutMonitor =
|
||||
new TimeoutMonitor(conf.getInt("hbase.splitlog.manager.timeoutmonitor.period", 1000), stopper);
|
||||
}
|
||||
|
||||
public void finishInitialization() {
|
||||
|
@ -233,9 +232,8 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
* available worker region server. This method must only be called after the
|
||||
* region servers have been brought online.
|
||||
*
|
||||
* @param logDirs
|
||||
* @throws IOException
|
||||
* if there was an error while splitting any log file
|
||||
* @param logDirs List of log dirs to split
|
||||
* @throws IOException If there was an error while splitting any log file
|
||||
* @return cumulative size of the logfiles split
|
||||
*/
|
||||
public long splitLogDistributed(final List<Path> logDirs) throws IOException {
|
||||
|
@ -244,7 +242,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
FileStatus[] logfiles = getFileList(logDirs);
|
||||
status.setStatus("Checking directory contents...");
|
||||
LOG.debug("Scheduling batch of logs to split");
|
||||
tot_mgr_log_split_batch_start.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_log_split_batch_start.incrementAndGet();
|
||||
LOG.info("started splitting logs in " + logDirs);
|
||||
long t = EnvironmentEdgeManager.currentTimeMillis();
|
||||
long totalSize = 0;
|
||||
|
@ -257,14 +255,13 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
// metrics that it drives will also be under-reported.
|
||||
totalSize += lf.getLen();
|
||||
if (enqueueSplitTask(lf.getPath().toString(), batch) == false) {
|
||||
throw new IOException("duplicate log split scheduled for "
|
||||
+ lf.getPath());
|
||||
throw new IOException("duplicate log split scheduled for " + lf.getPath());
|
||||
}
|
||||
}
|
||||
waitForSplittingCompletion(batch, status);
|
||||
if (batch.done != batch.installed) {
|
||||
batch.isDead = true;
|
||||
tot_mgr_log_split_batch_err.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_log_split_batch_err.incrementAndGet();
|
||||
LOG.warn("error while splitting logs in " + logDirs +
|
||||
" installed = " + batch.installed + " but only " + batch.done + " done");
|
||||
throw new IOException("error or interrupt while splitting logs in "
|
||||
|
@ -285,7 +282,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
LOG.warn("Unable to delete log src dir. Ignoring. " + logDir, ioe);
|
||||
}
|
||||
}
|
||||
tot_mgr_log_split_batch_success.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_log_split_batch_success.incrementAndGet();
|
||||
}
|
||||
String msg = "finished splitting (more than or equal to) " + totalSize +
|
||||
" bytes in " + batch.installed + " log files in " + logDirs + " in " +
|
||||
|
@ -303,7 +300,9 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
* @return true if a new entry is created, false if it is already there.
|
||||
*/
|
||||
boolean enqueueSplitTask(String taskname, TaskBatch batch) {
|
||||
tot_mgr_log_split_start.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_log_split_start.incrementAndGet();
|
||||
// This is a znode path under the splitlog dir with the rest of the path made up of an
|
||||
// url encoding of the passed in log to split.
|
||||
String path = ZKSplitLog.getEncodedNodeName(watcher, taskname);
|
||||
Task oldtask = createTaskIfAbsent(path, batch);
|
||||
if (oldtask == null) {
|
||||
|
@ -340,17 +339,17 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
Task task = tasks.get(path);
|
||||
if (task == null) {
|
||||
if (!ZKSplitLog.isRescanNode(watcher, path)) {
|
||||
tot_mgr_unacquired_orphan_done.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_unacquired_orphan_done.incrementAndGet();
|
||||
LOG.debug("unacquired orphan task is done " + path);
|
||||
}
|
||||
} else {
|
||||
synchronized (task) {
|
||||
if (task.status == IN_PROGRESS) {
|
||||
if (status == SUCCESS) {
|
||||
tot_mgr_log_split_success.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_log_split_success.incrementAndGet();
|
||||
LOG.info("Done splitting " + path);
|
||||
} else {
|
||||
tot_mgr_log_split_err.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_log_split_err.incrementAndGet();
|
||||
LOG.warn("Error splitting " + path);
|
||||
}
|
||||
task.status = status;
|
||||
|
@ -376,10 +375,9 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
private void createNode(String path, Long retry_count) {
|
||||
ZKUtil.asyncCreate(this.watcher, path,
|
||||
TaskState.TASK_UNASSIGNED.get(serverName), new CreateAsyncCallback(),
|
||||
retry_count);
|
||||
tot_mgr_node_create_queued.incrementAndGet();
|
||||
SplitLogTask slt = new SplitLogTask.Unassigned(serverName);
|
||||
ZKUtil.asyncCreate(this.watcher, path, slt.toByteArray(), new CreateAsyncCallback(), retry_count);
|
||||
SplitLogCounters.tot_mgr_node_create_queued.incrementAndGet();
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -400,7 +398,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
this.watcher.getRecoverableZooKeeper().getZooKeeper().
|
||||
getData(path, this.watcher,
|
||||
new GetDataAsyncCallback(), retry_count);
|
||||
tot_mgr_get_data_queued.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_get_data_queued.incrementAndGet();
|
||||
}
|
||||
|
||||
private void tryGetDataSetWatch(String path) {
|
||||
|
@ -408,37 +406,36 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
this.watcher.getRecoverableZooKeeper().getZooKeeper().
|
||||
getData(path, this.watcher,
|
||||
new GetDataAsyncCallback(), Long.valueOf(-1) /* retry count */);
|
||||
tot_mgr_get_data_queued.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_get_data_queued.incrementAndGet();
|
||||
}
|
||||
|
||||
private void getDataSetWatchSuccess(String path, byte[] data, int version) {
|
||||
private void getDataSetWatchSuccess(String path, byte[] data, int version)
|
||||
throws DeserializationException {
|
||||
if (data == null) {
|
||||
if (version == Integer.MIN_VALUE) {
|
||||
// assume all done. The task znode suddenly disappeared.
|
||||
setDone(path, SUCCESS);
|
||||
return;
|
||||
}
|
||||
tot_mgr_null_data.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_null_data.incrementAndGet();
|
||||
LOG.fatal("logic error - got null data " + path);
|
||||
setDone(path, FAILURE);
|
||||
return;
|
||||
}
|
||||
data = this.watcher.getRecoverableZooKeeper().removeMetaData(data);
|
||||
// LOG.debug("set watch on " + path + " got data " + new String(data));
|
||||
if (TaskState.TASK_UNASSIGNED.equals(data)) {
|
||||
SplitLogTask slt = SplitLogTask.parseFrom(data);
|
||||
if (slt.isUnassigned()) {
|
||||
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)) {
|
||||
} else if (slt.isOwned()) {
|
||||
heartbeat(path, version, slt.getServerName());
|
||||
} else if (slt.isResigned()) {
|
||||
LOG.info("task " + path + " entered state " + new String(data));
|
||||
resubmitOrFail(path, FORCE);
|
||||
} else if (TaskState.TASK_DONE.equals(data)) {
|
||||
} else if (slt.isDone()) {
|
||||
LOG.info("task " + path + " entered state " + new String(data));
|
||||
if (taskFinisher != null && !ZKSplitLog.isRescanNode(watcher, path)) {
|
||||
if (taskFinisher.finish(TaskState.TASK_DONE.getWriterName(data),
|
||||
ZKSplitLog.getFileName(path)) == Status.DONE) {
|
||||
if (taskFinisher.finish(slt.getServerName(), ZKSplitLog.getFileName(path)) == Status.DONE) {
|
||||
setDone(path, SUCCESS);
|
||||
} else {
|
||||
resubmitOrFail(path, CHECK);
|
||||
|
@ -446,12 +443,11 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
} else {
|
||||
setDone(path, SUCCESS);
|
||||
}
|
||||
} else if (TaskState.TASK_ERR.equals(data)) {
|
||||
} else if (slt.isErr()) {
|
||||
LOG.info("task " + path + " entered state " + new String(data));
|
||||
resubmitOrFail(path, CHECK);
|
||||
} else {
|
||||
LOG.fatal("logic error - unexpected zk state for path = " + path
|
||||
+ " data = " + new String(data));
|
||||
LOG.fatal("logic error - unexpected zk state for path = " + path + " data = " + new String(data));
|
||||
setDone(path, FAILURE);
|
||||
}
|
||||
}
|
||||
|
@ -483,16 +479,14 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
}
|
||||
|
||||
private void heartbeat(String path, int new_version,
|
||||
String workerName) {
|
||||
private void heartbeat(String path, int new_version, ServerName workerName) {
|
||||
Task task = findOrCreateOrphanTask(path);
|
||||
if (new_version != task.last_version) {
|
||||
if (task.isUnassigned()) {
|
||||
LOG.info("task " + path + " acquired by " + workerName);
|
||||
}
|
||||
task.heartbeat(EnvironmentEdgeManager.currentTimeMillis(),
|
||||
new_version, workerName);
|
||||
tot_mgr_heartbeat.incrementAndGet();
|
||||
task.heartbeat(EnvironmentEdgeManager.currentTimeMillis(), new_version, workerName);
|
||||
SplitLogCounters.tot_mgr_heartbeat.incrementAndGet();
|
||||
} else {
|
||||
// duplicate heartbeats - heartbeats w/o zk node version
|
||||
// changing - are possible. The timeout thread does
|
||||
|
@ -502,10 +496,8 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
return;
|
||||
}
|
||||
|
||||
private boolean resubmit(String path, Task task,
|
||||
ResubmitDirective directive) {
|
||||
// its ok if this thread misses the update to task.deleted. It will
|
||||
// fail later
|
||||
private boolean resubmit(String path, Task task, ResubmitDirective directive) {
|
||||
// its ok if this thread misses the update to task.deleted. It will fail later
|
||||
if (task.status != IN_PROGRESS) {
|
||||
return false;
|
||||
}
|
||||
|
@ -518,7 +510,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
if (task.unforcedResubmits >= resubmit_threshold) {
|
||||
if (!task.resubmitThresholdReached) {
|
||||
task.resubmitThresholdReached = true;
|
||||
tot_mgr_resubmit_threshold_reached.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_resubmit_threshold_reached.incrementAndGet();
|
||||
LOG.info("Skipping resubmissions of task " + path +
|
||||
" because threshold " + resubmit_threshold + " reached");
|
||||
}
|
||||
|
@ -533,9 +525,8 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
task.incarnation++;
|
||||
try {
|
||||
// blocking zk call but this is done from the timeout thread
|
||||
if (ZKUtil.setData(this.watcher, path,
|
||||
TaskState.TASK_UNASSIGNED.get(serverName),
|
||||
version) == false) {
|
||||
SplitLogTask slt = new SplitLogTask.Unassigned(this.serverName);
|
||||
if (ZKUtil.setData(this.watcher, path, slt.toByteArray(), version) == false) {
|
||||
LOG.debug("failed to resubmit task " + path +
|
||||
" version changed");
|
||||
task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
|
||||
|
@ -544,15 +535,20 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
} catch (NoNodeException e) {
|
||||
LOG.warn("failed to resubmit because znode doesn't exist " + path +
|
||||
" task done (or forced done by removing the znode)");
|
||||
getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
|
||||
try {
|
||||
getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
|
||||
} catch (DeserializationException e1) {
|
||||
LOG.debug("failed to re-resubmit task " + path + " because of deserialization issue");
|
||||
task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
|
||||
return false;
|
||||
}
|
||||
return false;
|
||||
} catch (KeeperException.BadVersionException e) {
|
||||
LOG.debug("failed to resubmit task " + path +
|
||||
" version changed");
|
||||
LOG.debug("failed to resubmit task " + path + " version changed");
|
||||
task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
|
||||
return false;
|
||||
} catch (KeeperException e) {
|
||||
tot_mgr_resubmit_failed.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_resubmit_failed.incrementAndGet();
|
||||
LOG.warn("failed to resubmit " + path, e);
|
||||
return false;
|
||||
}
|
||||
|
@ -562,7 +558,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
task.setUnassigned();
|
||||
createRescanNode(Long.MAX_VALUE);
|
||||
tot_mgr_resubmit.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_resubmit.incrementAndGet();
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -573,7 +569,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
private void deleteNode(String path, Long retries) {
|
||||
tot_mgr_node_delete_queued.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_node_delete_queued.incrementAndGet();
|
||||
// Once a task znode is ready for delete, that is it is in the TASK_DONE
|
||||
// state, then no one should be writing to it anymore. That is no one
|
||||
// will be updating the znode version any more.
|
||||
|
@ -590,9 +586,9 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
task = tasks.remove(path);
|
||||
if (task == null) {
|
||||
if (ZKSplitLog.isRescanNode(watcher, path)) {
|
||||
tot_mgr_rescan_deleted.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_rescan_deleted.incrementAndGet();
|
||||
}
|
||||
tot_mgr_missing_state_in_delete.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_missing_state_in_delete.incrementAndGet();
|
||||
LOG.debug("deleted task without in memory state " + path);
|
||||
return;
|
||||
}
|
||||
|
@ -600,7 +596,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
task.status = DELETED;
|
||||
task.notify();
|
||||
}
|
||||
tot_mgr_task_deleted.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_task_deleted.incrementAndGet();
|
||||
}
|
||||
|
||||
private void deleteNodeFailure(String path) {
|
||||
|
@ -622,16 +618,16 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
// might miss the watch-trigger that creation of RESCAN node provides.
|
||||
// Since the TimeoutMonitor will keep resubmitting UNASSIGNED tasks
|
||||
// therefore this behavior is safe.
|
||||
SplitLogTask slt = new SplitLogTask.Done(this.serverName);
|
||||
this.watcher.getRecoverableZooKeeper().getZooKeeper().
|
||||
create(ZKSplitLog.getRescanNode(watcher),
|
||||
TaskState.TASK_DONE.get(serverName), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.EPHEMERAL_SEQUENTIAL,
|
||||
create(ZKSplitLog.getRescanNode(watcher), slt.toByteArray(),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL,
|
||||
new CreateRescanAsyncCallback(), Long.valueOf(retries));
|
||||
}
|
||||
|
||||
private void createRescanSuccess(String path) {
|
||||
lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
tot_mgr_rescan.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_rescan.incrementAndGet();
|
||||
getDataSetWatch(path, zkretries);
|
||||
}
|
||||
|
||||
|
@ -668,14 +664,12 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
if (oldtask.status == IN_PROGRESS) {
|
||||
oldtask.batch = batch;
|
||||
batch.installed++;
|
||||
LOG.debug("Previously orphan task " + path +
|
||||
" is now being waited upon");
|
||||
LOG.debug("Previously orphan task " + path + " is now being waited upon");
|
||||
return null;
|
||||
}
|
||||
while (oldtask.status == FAILURE) {
|
||||
LOG.debug("wait for status of task " + path +
|
||||
" to change to DELETED");
|
||||
tot_mgr_wait_for_zk_delete.incrementAndGet();
|
||||
LOG.debug("wait for status of task " + path + " to change to DELETED");
|
||||
SplitLogCounters.tot_mgr_wait_for_zk_delete.incrementAndGet();
|
||||
try {
|
||||
oldtask.wait();
|
||||
} catch (InterruptedException e) {
|
||||
|
@ -701,8 +695,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
assert false : "Deleted task still present in tasks map";
|
||||
return t;
|
||||
}
|
||||
LOG.warn("Failure because two threads can't wait for the same task. " +
|
||||
" path=" + path);
|
||||
LOG.warn("Failure because two threads can't wait for the same task; path=" + path);
|
||||
return oldtask;
|
||||
}
|
||||
}
|
||||
|
@ -713,7 +706,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
task = tasks.putIfAbsent(path, orphanTask);
|
||||
if (task == null) {
|
||||
LOG.info("creating orphan task " + path);
|
||||
tot_mgr_orphan_task_acquired.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_orphan_task_acquired.incrementAndGet();
|
||||
task = orphanTask;
|
||||
}
|
||||
return task;
|
||||
|
@ -781,8 +774,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return ("installed = " + installed + " done = " + done + " error = "
|
||||
+ error);
|
||||
return ("installed = " + installed + " done = " + done + " error = " + error);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -792,7 +784,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
static class Task {
|
||||
volatile long last_update;
|
||||
volatile int last_version;
|
||||
volatile String cur_worker_name;
|
||||
volatile ServerName cur_worker_name;
|
||||
TaskBatch batch;
|
||||
volatile TerminationStatus status;
|
||||
volatile int incarnation;
|
||||
|
@ -829,7 +821,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
last_update = time;
|
||||
}
|
||||
|
||||
public void heartbeat(long time, int version, String worker) {
|
||||
public void heartbeat(long time, int version, ServerName worker) {
|
||||
last_version = version;
|
||||
last_update = time;
|
||||
cur_worker_name = worker;
|
||||
|
@ -841,12 +833,12 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
}
|
||||
|
||||
void handleDeadWorker(String workerName) {
|
||||
void handleDeadWorker(ServerName workerName) {
|
||||
// resubmit the tasks on the TimeoutMonitor thread. Makes it easier
|
||||
// to reason about concurrency. Makes it easier to retry.
|
||||
synchronized (deadWorkersLock) {
|
||||
if (deadWorkers == null) {
|
||||
deadWorkers = new HashSet<String>(100);
|
||||
deadWorkers = new HashSet<ServerName>(100);
|
||||
}
|
||||
deadWorkers.add(workerName);
|
||||
}
|
||||
|
@ -854,13 +846,13 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
void handleDeadWorkers(List<ServerName> serverNames) {
|
||||
List<String> workerNames = new ArrayList<String>(serverNames.size());
|
||||
List<ServerName> workerNames = new ArrayList<ServerName>(serverNames.size());
|
||||
for (ServerName serverName : serverNames) {
|
||||
workerNames.add(serverName.toString());
|
||||
workerNames.add(serverName);
|
||||
}
|
||||
synchronized (deadWorkersLock) {
|
||||
if (deadWorkers == null) {
|
||||
deadWorkers = new HashSet<String>(100);
|
||||
deadWorkers = new HashSet<ServerName>(100);
|
||||
}
|
||||
deadWorkers.addAll(workerNames);
|
||||
}
|
||||
|
@ -882,7 +874,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
int unassigned = 0;
|
||||
int tot = 0;
|
||||
boolean found_assigned_task = false;
|
||||
Set<String> localDeadWorkers;
|
||||
Set<ServerName> localDeadWorkers;
|
||||
|
||||
synchronized (deadWorkersLock) {
|
||||
localDeadWorkers = deadWorkers;
|
||||
|
@ -892,7 +884,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
for (Map.Entry<String, Task> e : tasks.entrySet()) {
|
||||
String path = e.getKey();
|
||||
Task task = e.getValue();
|
||||
String cur_worker = task.cur_worker_name;
|
||||
ServerName cur_worker = task.cur_worker_name;
|
||||
tot++;
|
||||
// don't easily resubmit a task which hasn't been picked up yet. It
|
||||
// might be a long while before a SplitLogWorker is free to pick up a
|
||||
|
@ -905,7 +897,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
found_assigned_task = true;
|
||||
if (localDeadWorkers != null && localDeadWorkers.contains(cur_worker)) {
|
||||
tot_mgr_resubmit_dead_server_task.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_resubmit_dead_server_task.incrementAndGet();
|
||||
if (resubmit(path, task, FORCE)) {
|
||||
resubmitted++;
|
||||
} else {
|
||||
|
@ -948,7 +940,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
}
|
||||
createRescanNode(Long.MAX_VALUE);
|
||||
tot_mgr_resubmit_unassigned.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_resubmit_unassigned.incrementAndGet();
|
||||
LOG.debug("resubmitting unassigned task(s) after timeout");
|
||||
}
|
||||
}
|
||||
|
@ -963,7 +955,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
|
||||
@Override
|
||||
public void processResult(int rc, String path, Object ctx, String name) {
|
||||
tot_mgr_node_create_result.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_node_create_result.incrementAndGet();
|
||||
if (rc != 0) {
|
||||
if (rc == KeeperException.Code.NODEEXISTS.intValue()) {
|
||||
// What if there is a delete pending against this pre-existing
|
||||
|
@ -973,16 +965,16 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
// And all code pieces correctly handle the case of suddenly
|
||||
// disappearing task-znode.
|
||||
LOG.debug("found pre-existing znode " + path);
|
||||
tot_mgr_node_already_exists.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_node_already_exists.incrementAndGet();
|
||||
} else {
|
||||
Long retry_count = (Long)ctx;
|
||||
LOG.warn("create rc =" + KeeperException.Code.get(rc) + " for " +
|
||||
path + " remaining retries=" + retry_count);
|
||||
if (retry_count == 0) {
|
||||
tot_mgr_node_create_err.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_node_create_err.incrementAndGet();
|
||||
createNodeFailure(path);
|
||||
} else {
|
||||
tot_mgr_node_create_retry.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_node_create_retry.incrementAndGet();
|
||||
createNode(path, retry_count - 1);
|
||||
}
|
||||
return;
|
||||
|
@ -1002,19 +994,23 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
@Override
|
||||
public void processResult(int rc, String path, Object ctx, byte[] data,
|
||||
Stat stat) {
|
||||
tot_mgr_get_data_result.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_get_data_result.incrementAndGet();
|
||||
if (rc != 0) {
|
||||
if (rc == KeeperException.Code.SESSIONEXPIRED.intValue()) {
|
||||
LOG.error("ZK session expired. Master is expected to shut down. Abandoning retries.");
|
||||
return;
|
||||
}
|
||||
if (rc == KeeperException.Code.NONODE.intValue()) {
|
||||
tot_mgr_get_data_nonode.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_get_data_nonode.incrementAndGet();
|
||||
// The task znode has been deleted. Must be some pending delete
|
||||
// that deleted the task. Assume success because a task-znode is
|
||||
// is only deleted after TaskFinisher is successful.
|
||||
LOG.warn("task znode " + path + " vanished.");
|
||||
getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
|
||||
try {
|
||||
getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Deserialization problem", e);
|
||||
}
|
||||
return;
|
||||
}
|
||||
Long retry_count = (Long) ctx;
|
||||
|
@ -1027,15 +1023,19 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " +
|
||||
path + " remaining retries=" + retry_count);
|
||||
if (retry_count == 0) {
|
||||
tot_mgr_get_data_err.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_get_data_err.incrementAndGet();
|
||||
getDataSetWatchFailure(path);
|
||||
} else {
|
||||
tot_mgr_get_data_retry.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_get_data_retry.incrementAndGet();
|
||||
getDataSetWatch(path, retry_count - 1);
|
||||
}
|
||||
return;
|
||||
}
|
||||
getDataSetWatchSuccess(path, data, stat.getVersion());
|
||||
try {
|
||||
getDataSetWatchSuccess(path, data, stat.getVersion());
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Deserialization problem", e);
|
||||
}
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
@ -1049,10 +1049,10 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
|
||||
@Override
|
||||
public void processResult(int rc, String path, Object ctx) {
|
||||
tot_mgr_node_delete_result.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_node_delete_result.incrementAndGet();
|
||||
if (rc != 0) {
|
||||
if (rc != KeeperException.Code.NONODE.intValue()) {
|
||||
tot_mgr_node_delete_err.incrementAndGet();
|
||||
SplitLogCounters.tot_mgr_node_delete_err.incrementAndGet();
|
||||
Long retry_count = (Long) ctx;
|
||||
LOG.warn("delete rc=" + KeeperException.Code.get(rc) + " for " +
|
||||
path + " remaining retries=" + retry_count);
|
||||
|
@ -1137,12 +1137,14 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
* @param taskname
|
||||
* @return DONE if task completed successfully, ERR otherwise
|
||||
*/
|
||||
public Status finish(String workerName, String taskname);
|
||||
public Status finish(ServerName workerName, String taskname);
|
||||
}
|
||||
|
||||
enum ResubmitDirective {
|
||||
CHECK(),
|
||||
FORCE();
|
||||
}
|
||||
|
||||
enum TerminationStatus {
|
||||
IN_PROGRESS("in_progress"),
|
||||
SUCCESS("success"),
|
||||
|
|
|
@ -37,6 +37,7 @@ import java.util.NavigableSet;
|
|||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -142,6 +143,7 @@ public final class ProtobufUtil {
|
|||
* for preamble.
|
||||
*/
|
||||
static final byte [] PB_MAGIC = new byte [] {'P', 'B', 'U', 'F'};
|
||||
private static final String PB_MAGIC_STR = Bytes.toString(PB_MAGIC);
|
||||
|
||||
/**
|
||||
* Prepend the passed bytes with four bytes of magic, {@link #PB_MAGIC}, to flag what
|
||||
|
@ -163,6 +165,16 @@ public final class ProtobufUtil {
|
|||
return Bytes.compareTo(PB_MAGIC, 0, PB_MAGIC.length, bytes, 0, PB_MAGIC.length) == 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param bytes
|
||||
* @throws DeserializationException if we are missing the pb magic prefix
|
||||
*/
|
||||
public static void expectPBMagicPrefix(final byte [] bytes) throws DeserializationException {
|
||||
if (!isPBMagicPrefix(bytes)) {
|
||||
throw new DeserializationException("Missing pb magic " + PB_MAGIC_STR + " prefix");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Length of {@link #PB_MAGIC}
|
||||
*/
|
||||
|
@ -237,6 +249,7 @@ public final class ProtobufUtil {
|
|||
*
|
||||
* @param serverName the ServerName to convert
|
||||
* @return the converted protocol buffer ServerName
|
||||
* @see #toServerName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName)
|
||||
*/
|
||||
public static HBaseProtos.ServerName
|
||||
toServerName(final ServerName serverName) {
|
||||
|
@ -259,8 +272,7 @@ public final class ProtobufUtil {
|
|||
* @param proto the protocol buffer ServerName to convert
|
||||
* @return the converted ServerName
|
||||
*/
|
||||
public static ServerName toServerName(
|
||||
final HBaseProtos.ServerName proto) {
|
||||
public static ServerName toServerName(final HBaseProtos.ServerName proto) {
|
||||
if (proto == null) return null;
|
||||
String hostName = proto.getHostName();
|
||||
long startCode = -1;
|
||||
|
@ -280,8 +292,7 @@ public final class ProtobufUtil {
|
|||
* @param proto the RegionInfo to convert
|
||||
* @return the converted HRegionInfo
|
||||
*/
|
||||
public static HRegionInfo
|
||||
toRegionInfo(final RegionInfo proto) {
|
||||
public static HRegionInfo toRegionInfo(final RegionInfo proto) {
|
||||
if (proto == null) return null;
|
||||
byte[] tableName = proto.getTableName().toByteArray();
|
||||
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
|
||||
|
@ -316,8 +327,7 @@ public final class ProtobufUtil {
|
|||
* @param info the HRegionInfo to convert
|
||||
* @return the converted RegionInfo
|
||||
*/
|
||||
public static RegionInfo
|
||||
toRegionInfo(final HRegionInfo info) {
|
||||
public static RegionInfo toRegionInfo(final HRegionInfo info) {
|
||||
if (info == null) return null;
|
||||
RegionInfo.Builder builder = RegionInfo.newBuilder();
|
||||
builder.setTableName(ByteString.copyFrom(info.getTableName()));
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -1347,7 +1347,7 @@ public class HRegionServer extends RegionServer
|
|||
|
||||
// Create the log splitting worker and start it
|
||||
this.splitLogWorker = new SplitLogWorker(this.zooKeeper,
|
||||
this.getConfiguration(), this.getServerName().toString());
|
||||
this.getConfiguration(), this.getServerName());
|
||||
splitLogWorker.start();
|
||||
}
|
||||
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.List;
|
||||
|
@ -32,12 +30,14 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.master.SplitLogManager;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SplitLogCounters;
|
||||
import org.apache.hadoop.hbase.SplitLogTask;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog.TaskState;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
|
@ -71,9 +71,8 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
private static final Log LOG = LogFactory.getLog(SplitLogWorker.class);
|
||||
|
||||
Thread worker;
|
||||
private final String serverName;
|
||||
private final ServerName serverName;
|
||||
private final TaskExecutor splitTaskExecutor;
|
||||
private long zkretries;
|
||||
|
||||
private Object taskReadyLock = new Object();
|
||||
volatile int taskReadySeq = 0;
|
||||
|
@ -85,15 +84,14 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
|
||||
|
||||
public SplitLogWorker(ZooKeeperWatcher watcher, Configuration conf,
|
||||
String serverName, TaskExecutor splitTaskExecutor) {
|
||||
ServerName serverName, TaskExecutor splitTaskExecutor) {
|
||||
super(watcher);
|
||||
this.serverName = serverName;
|
||||
this.splitTaskExecutor = splitTaskExecutor;
|
||||
this.zkretries = conf.getLong("hbase.splitlog.zk.retries", 3);
|
||||
}
|
||||
|
||||
public SplitLogWorker(ZooKeeperWatcher watcher, final Configuration conf,
|
||||
final String serverName) {
|
||||
final ServerName serverName) {
|
||||
this(watcher, conf, serverName, new TaskExecutor () {
|
||||
@Override
|
||||
public Status exec(String filename, CancelableProgressable p) {
|
||||
|
@ -111,24 +109,21 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
// encountered a bad non-retry-able persistent error.
|
||||
try {
|
||||
String tmpname =
|
||||
ZKSplitLog.getSplitLogDirTmpComponent(serverName, filename);
|
||||
ZKSplitLog.getSplitLogDirTmpComponent(serverName.toString(), filename);
|
||||
if (HLogSplitter.splitLogFileToTemp(rootdir, tmpname,
|
||||
fs.getFileStatus(new Path(filename)), fs, conf, p) == false) {
|
||||
return Status.PREEMPTED;
|
||||
}
|
||||
} catch (InterruptedIOException iioe) {
|
||||
LOG.warn("log splitting of " + filename + " interrupted, resigning",
|
||||
iioe);
|
||||
LOG.warn("log splitting of " + filename + " interrupted, resigning", iioe);
|
||||
return Status.RESIGNED;
|
||||
} catch (IOException e) {
|
||||
Throwable cause = e.getCause();
|
||||
if (cause instanceof InterruptedException) {
|
||||
LOG.warn("log splitting of " + filename + " interrupted, resigning",
|
||||
e);
|
||||
LOG.warn("log splitting of " + filename + " interrupted, resigning", e);
|
||||
return Status.RESIGNED;
|
||||
}
|
||||
LOG.warn("log splitting of " + filename + " failed, returning error",
|
||||
e);
|
||||
LOG.warn("log splitting of " + filename + " failed, returning error", e);
|
||||
return Status.ERR;
|
||||
}
|
||||
return Status.DONE;
|
||||
|
@ -149,13 +144,11 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
res = ZKUtil.checkExists(watcher, watcher.splitLogZNode);
|
||||
} catch (KeeperException e) {
|
||||
// ignore
|
||||
LOG.warn("Exception when checking for " + watcher.splitLogZNode +
|
||||
" ... retrying", e);
|
||||
LOG.warn("Exception when checking for " + watcher.splitLogZNode + " ... retrying", e);
|
||||
}
|
||||
if (res == -1) {
|
||||
try {
|
||||
LOG.info(watcher.splitLogZNode + " znode does not exist," +
|
||||
" waiting for master to create one");
|
||||
LOG.info(watcher.splitLogZNode + " znode does not exist, waiting for master to create");
|
||||
Thread.sleep(1000);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.debug("Interrupted while waiting for " + watcher.splitLogZNode
|
||||
|
@ -241,31 +234,40 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
try {
|
||||
try {
|
||||
if ((data = ZKUtil.getDataNoWatch(this.watcher, path, stat)) == null) {
|
||||
tot_wkr_failed_to_grab_task_no_data.incrementAndGet();
|
||||
SplitLogCounters.tot_wkr_failed_to_grab_task_no_data.incrementAndGet();
|
||||
return;
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
LOG.warn("Failed to get data for znode " + path, e);
|
||||
tot_wkr_failed_to_grab_task_exception.incrementAndGet();
|
||||
SplitLogCounters.tot_wkr_failed_to_grab_task_exception.incrementAndGet();
|
||||
return;
|
||||
}
|
||||
if (TaskState.TASK_UNASSIGNED.equals(data) == false) {
|
||||
tot_wkr_failed_to_grab_task_owned.incrementAndGet();
|
||||
SplitLogTask slt;
|
||||
try {
|
||||
slt = SplitLogTask.parseFrom(data);
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Failed parse data for znode " + path, e);
|
||||
SplitLogCounters.tot_wkr_failed_to_grab_task_exception.incrementAndGet();
|
||||
return;
|
||||
}
|
||||
if (slt.isUnassigned() == false) {
|
||||
SplitLogCounters.tot_wkr_failed_to_grab_task_owned.incrementAndGet();
|
||||
return;
|
||||
}
|
||||
|
||||
currentVersion = stat.getVersion();
|
||||
if (attemptToOwnTask(true) == false) {
|
||||
tot_wkr_failed_to_grab_task_lost_race.incrementAndGet();
|
||||
SplitLogCounters.tot_wkr_failed_to_grab_task_lost_race.incrementAndGet();
|
||||
return;
|
||||
}
|
||||
|
||||
if (ZKSplitLog.isRescanNode(watcher, currentTask)) {
|
||||
endTask(TaskState.TASK_DONE, tot_wkr_task_acquired_rescan);
|
||||
endTask(new SplitLogTask.Done(this.serverName),
|
||||
SplitLogCounters.tot_wkr_task_acquired_rescan);
|
||||
return;
|
||||
}
|
||||
LOG.info("worker " + serverName + " acquired task " + path);
|
||||
tot_wkr_task_acquired.incrementAndGet();
|
||||
SplitLogCounters.tot_wkr_task_acquired.incrementAndGet();
|
||||
getDataSetWatchAsync();
|
||||
|
||||
t = System.currentTimeMillis();
|
||||
|
@ -285,15 +287,15 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
});
|
||||
switch (status) {
|
||||
case DONE:
|
||||
endTask(TaskState.TASK_DONE, tot_wkr_task_done);
|
||||
endTask(new SplitLogTask.Done(this.serverName), SplitLogCounters.tot_wkr_task_done);
|
||||
break;
|
||||
case PREEMPTED:
|
||||
tot_wkr_preempt_task.incrementAndGet();
|
||||
SplitLogCounters.tot_wkr_preempt_task.incrementAndGet();
|
||||
LOG.warn("task execution prempted " + path);
|
||||
break;
|
||||
case ERR:
|
||||
if (!exitWorker) {
|
||||
endTask(TaskState.TASK_ERR, tot_wkr_task_err);
|
||||
endTask(new SplitLogTask.Err(this.serverName), SplitLogCounters.tot_wkr_task_err);
|
||||
break;
|
||||
}
|
||||
// if the RS is exiting then there is probably a tons of stuff
|
||||
|
@ -301,13 +303,12 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
//$FALL-THROUGH$
|
||||
case RESIGNED:
|
||||
if (exitWorker) {
|
||||
LOG.info("task execution interrupted because worker is exiting " +
|
||||
path);
|
||||
endTask(TaskState.TASK_RESIGNED, tot_wkr_task_resigned);
|
||||
LOG.info("task execution interrupted because worker is exiting " + path);
|
||||
endTask(new SplitLogTask.Resigned(this.serverName),
|
||||
SplitLogCounters.tot_wkr_task_resigned);
|
||||
} else {
|
||||
tot_wkr_preempt_task.incrementAndGet();
|
||||
LOG.info("task execution interrupted via zk by manager " +
|
||||
path);
|
||||
SplitLogCounters.tot_wkr_preempt_task.incrementAndGet();
|
||||
LOG.info("task execution interrupted via zk by manager " + path);
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
@ -337,15 +338,16 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
*/
|
||||
private boolean attemptToOwnTask(boolean isFirstTime) {
|
||||
try {
|
||||
Stat stat = this.watcher.getRecoverableZooKeeper().setData(currentTask,
|
||||
TaskState.TASK_OWNED.get(serverName), currentVersion);
|
||||
SplitLogTask slt = new SplitLogTask.Owned(this.serverName);
|
||||
Stat stat =
|
||||
this.watcher.getRecoverableZooKeeper().setData(currentTask, slt.toByteArray(), currentVersion);
|
||||
if (stat == null) {
|
||||
LOG.warn("zk.setData() returned null for path " + currentTask);
|
||||
tot_wkr_task_heartbeat_failed.incrementAndGet();
|
||||
SplitLogCounters.tot_wkr_task_heartbeat_failed.incrementAndGet();
|
||||
return (false);
|
||||
}
|
||||
currentVersion = stat.getVersion();
|
||||
tot_wkr_task_heartbeat.incrementAndGet();
|
||||
SplitLogCounters.tot_wkr_task_heartbeat.incrementAndGet();
|
||||
return (true);
|
||||
} catch (KeeperException e) {
|
||||
if (!isFirstTime) {
|
||||
|
@ -363,7 +365,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
currentTask + " " + StringUtils.stringifyException(e1));
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
tot_wkr_task_heartbeat_failed.incrementAndGet();
|
||||
SplitLogCounters.tot_wkr_task_heartbeat_failed.incrementAndGet();
|
||||
return (false);
|
||||
}
|
||||
|
||||
|
@ -373,29 +375,28 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
* @param ts
|
||||
* @param ctr
|
||||
*/
|
||||
private void endTask(ZKSplitLog.TaskState ts, AtomicLong ctr) {
|
||||
private void endTask(SplitLogTask slt, AtomicLong ctr) {
|
||||
String path = currentTask;
|
||||
currentTask = null;
|
||||
try {
|
||||
if (ZKUtil.setData(this.watcher, path, ts.get(serverName),
|
||||
if (ZKUtil.setData(this.watcher, path, slt.toByteArray(),
|
||||
currentVersion)) {
|
||||
LOG.info("successfully transitioned task " + path +
|
||||
" to final state " + ts);
|
||||
LOG.info("successfully transitioned task " + path + " to final state " + slt);
|
||||
ctr.incrementAndGet();
|
||||
return;
|
||||
}
|
||||
LOG.warn("failed to transistion task " + path + " to end state " + ts +
|
||||
LOG.warn("failed to transistion task " + path + " to end state " + slt +
|
||||
" because of version mismatch ");
|
||||
} catch (KeeperException.BadVersionException bve) {
|
||||
LOG.warn("transisition task " + path + " to " + ts +
|
||||
LOG.warn("transisition task " + path + " to " + slt +
|
||||
" failed because of version mismatch", bve);
|
||||
} catch (KeeperException.NoNodeException e) {
|
||||
LOG.fatal("logic error - end task " + path + " " + ts +
|
||||
LOG.fatal("logic error - end task " + path + " " + slt +
|
||||
" failed because task doesn't exist", e);
|
||||
} catch (KeeperException e) {
|
||||
LOG.warn("failed to end task, " + path + " " + ts, e);
|
||||
LOG.warn("failed to end task, " + path + " " + slt, e);
|
||||
}
|
||||
tot_wkr_final_transistion_failed.incrementAndGet();
|
||||
SplitLogCounters.tot_wkr_final_transistion_failed.incrementAndGet();
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -403,10 +404,17 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
this.watcher.getRecoverableZooKeeper().getZooKeeper().
|
||||
getData(currentTask, this.watcher,
|
||||
new GetDataAsyncCallback(), null);
|
||||
tot_wkr_get_data_queued.incrementAndGet();
|
||||
SplitLogCounters.tot_wkr_get_data_queued.incrementAndGet();
|
||||
}
|
||||
|
||||
void getDataSetWatchSuccess(String path, byte[] data) {
|
||||
SplitLogTask slt;
|
||||
try {
|
||||
slt = SplitLogTask.parseFrom(data);
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Failed parse", e);
|
||||
return;
|
||||
}
|
||||
synchronized (grabTaskLock) {
|
||||
if (workerInGrabTask) {
|
||||
// currentTask can change but that's ok
|
||||
|
@ -418,13 +426,12 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
// UNASSIGNED because by the time this worker sets the data watch
|
||||
// the node might have made two transitions - from owned by this
|
||||
// worker to unassigned to owned by another worker
|
||||
if (! TaskState.TASK_OWNED.equals(data, serverName) &&
|
||||
! TaskState.TASK_DONE.equals(data, serverName) &&
|
||||
! TaskState.TASK_ERR.equals(data, serverName) &&
|
||||
! TaskState.TASK_RESIGNED.equals(data, serverName)) {
|
||||
if (! slt.isOwned(this.serverName) &&
|
||||
! slt.isDone(this.serverName) &&
|
||||
! slt.isErr(this.serverName) &&
|
||||
! slt.isResigned(this.serverName)) {
|
||||
LOG.info("task " + taskpath + " preempted from " +
|
||||
serverName + ", current task state and owner=" +
|
||||
new String(data));
|
||||
serverName + ", current task state and owner=" + slt.toString());
|
||||
stopTask();
|
||||
}
|
||||
}
|
||||
|
@ -439,7 +446,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
String taskpath = currentTask;
|
||||
if (taskpath != null && taskpath.equals(path)) {
|
||||
LOG.info("retrying data watch on " + path);
|
||||
tot_wkr_get_data_retry.incrementAndGet();
|
||||
SplitLogCounters.tot_wkr_get_data_retry.incrementAndGet();
|
||||
getDataSetWatchAsync();
|
||||
} else {
|
||||
// no point setting a watch on the task which this worker is not
|
||||
|
@ -543,9 +550,8 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
private final Log LOG = LogFactory.getLog(GetDataAsyncCallback.class);
|
||||
|
||||
@Override
|
||||
public void processResult(int rc, String path, Object ctx, byte[] data,
|
||||
Stat stat) {
|
||||
tot_wkr_get_data_result.incrementAndGet();
|
||||
public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
|
||||
SplitLogCounters.tot_wkr_get_data_result.incrementAndGet();
|
||||
if (rc != 0) {
|
||||
LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " + path);
|
||||
getDataSetWatchFailure(path);
|
||||
|
@ -573,4 +579,4 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
}
|
||||
public Status exec(String name, CancelableProgressable p);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -39,11 +39,11 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.RegionTransition;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.catalog.MetaEditor;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.executor.RegionTransitionData;
|
||||
import org.apache.hadoop.hbase.io.Reference.Range;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
|
@ -858,12 +858,10 @@ public class SplitTransaction {
|
|||
throws KeeperException, IOException {
|
||||
LOG.debug(zkw.prefix("Creating ephemeral node for " +
|
||||
region.getEncodedName() + " in SPLITTING state"));
|
||||
RegionTransitionData data =
|
||||
new RegionTransitionData(EventType.RS_ZK_REGION_SPLITTING,
|
||||
RegionTransition rt = RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_SPLITTING,
|
||||
region.getRegionName(), serverName);
|
||||
|
||||
String node = ZKAssign.getNodeName(zkw, region.getEncodedName());
|
||||
if (!ZKUtil.createEphemeralNodeAndWatch(zkw, node, data.getBytes())) {
|
||||
if (!ZKUtil.createEphemeralNodeAndWatch(zkw, node, rt.toByteArray())) {
|
||||
throw new IOException("Failed create of ephemeral " + node);
|
||||
}
|
||||
// Transition node from SPLITTING to SPLITTING and pick up version so we
|
||||
|
|
|
@ -1715,7 +1715,8 @@ public class HLog implements Syncable {
|
|||
* Construct the HLog directory name
|
||||
*
|
||||
* @param serverName Server name formatted as described in {@link ServerName}
|
||||
* @return the HLog directory name
|
||||
* @return the relative HLog directory name, e.g. <code>.logs/1.example.org,6030,12345</code>
|
||||
* if <code>serverName</code> passed is <code>1.example.org,60030,12345</code>
|
||||
*/
|
||||
public static String getHLogDirectoryName(final String serverName) {
|
||||
StringBuilder dirName = new StringBuilder(HConstants.HREGION_LOGDIR_NAME);
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.RemoteExceptionHandler;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
import org.apache.hadoop.hbase.master.SplitLogManager;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
|
@ -349,17 +350,16 @@ public class HLogSplitter {
|
|||
static public boolean splitLogFileToTemp(Path rootDir, String tmpname,
|
||||
FileStatus logfile, FileSystem fs,
|
||||
Configuration conf, CancelableProgressable reporter) throws IOException {
|
||||
HLogSplitter s = new HLogSplitter(conf, rootDir, null, null /* oldLogDir */,
|
||||
fs);
|
||||
HLogSplitter s = new HLogSplitter(conf, rootDir, null, null /* oldLogDir */, fs);
|
||||
return s.splitLogFileToTemp(logfile, tmpname, reporter);
|
||||
}
|
||||
|
||||
public boolean splitLogFileToTemp(FileStatus logfile, String tmpname,
|
||||
CancelableProgressable reporter) throws IOException {
|
||||
CancelableProgressable reporter)
|
||||
throws IOException {
|
||||
final Map<byte[], Object> logWriters = Collections.
|
||||
synchronizedMap(new TreeMap<byte[], Object>(Bytes.BYTES_COMPARATOR));
|
||||
boolean isCorrupted = false;
|
||||
|
||||
Preconditions.checkState(status == null);
|
||||
status = TaskMonitor.get().createStatus(
|
||||
"Splitting log file " + logfile.getPath() +
|
||||
|
@ -375,7 +375,7 @@ public class HLogSplitter {
|
|||
// How often to send a progress report (default 1/2 the zookeeper session
|
||||
// timeout of if that not set, the split log DEFAULT_TIMEOUT)
|
||||
int period = conf.getInt("hbase.splitlog.report.period",
|
||||
conf.getInt("hbase.splitlog.manager.timeout", ZKSplitLog.DEFAULT_TIMEOUT) / 2);
|
||||
conf.getInt("hbase.splitlog.manager.timeout", SplitLogManager.DEFAULT_TIMEOUT) / 2);
|
||||
int numOpenedFilesBeforeReporting =
|
||||
conf.getInt("hbase.splitlog.report.openedfiles", 3);
|
||||
Path logPath = logfile.getPath();
|
||||
|
@ -474,8 +474,8 @@ public class HLogSplitter {
|
|||
WriterAndPath wap = (WriterAndPath)o;
|
||||
wap.w.close();
|
||||
LOG.debug("Closed " + wap.p);
|
||||
Path dst = getCompletedRecoveredEditsFilePath(wap.p, outputSink
|
||||
.getRegionMaximumEditLogSeqNum(logWritersEntry.getKey()));
|
||||
Path dst = getCompletedRecoveredEditsFilePath(wap.p,
|
||||
outputSink.getRegionMaximumEditLogSeqNum(logWritersEntry.getKey()));
|
||||
if (!dst.equals(wap.p) && fs.exists(dst)) {
|
||||
LOG.warn("Found existing old edits file. It could be the "
|
||||
+ "result of a previous failed split attempt. Deleting " + dst
|
||||
|
@ -658,10 +658,9 @@ public class HLogSplitter {
|
|||
static Path getRegionSplitEditsPath(final FileSystem fs,
|
||||
final Entry logEntry, final Path rootDir, boolean isCreate)
|
||||
throws IOException {
|
||||
Path tableDir = HTableDescriptor.getTableDir(rootDir, logEntry.getKey()
|
||||
.getTablename());
|
||||
Path tableDir = HTableDescriptor.getTableDir(rootDir, logEntry.getKey().getTablename());
|
||||
Path regiondir = HRegion.getRegionDir(tableDir,
|
||||
Bytes.toString(logEntry.getKey().getEncodedRegionName()));
|
||||
Bytes.toString(logEntry.getKey().getEncodedRegionName()));
|
||||
Path dir = HLog.getRegionDirRecoveredEditsDir(regiondir);
|
||||
|
||||
if (!fs.exists(regiondir)) {
|
||||
|
@ -675,8 +674,7 @@ public class HLogSplitter {
|
|||
}
|
||||
// Append file name ends with RECOVERED_LOG_TMPFILE_SUFFIX to ensure
|
||||
// region's replayRecoveredEdits will not delete it
|
||||
String fileName = formatRecoveredEditsFileName(logEntry.getKey()
|
||||
.getLogSeqNum());
|
||||
String fileName = formatRecoveredEditsFileName(logEntry.getKey().getLogSeqNum());
|
||||
fileName = getTmpRecoveredEditsFileName(fileName);
|
||||
return new Path(dir, fileName);
|
||||
}
|
||||
|
@ -1080,8 +1078,7 @@ public class HLogSplitter {
|
|||
private WriterAndPath createWAP(byte[] region, Entry entry,
|
||||
Path rootdir, String tmpname, FileSystem fs, Configuration conf)
|
||||
throws IOException {
|
||||
Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir,
|
||||
tmpname==null);
|
||||
Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir, tmpname==null);
|
||||
if (regionedits == null) {
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -1,6 +1,4 @@
|
|||
/**
|
||||
* Copyright 2009 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
|
||||
|
@ -17,7 +15,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
package org.apache.hadoop.hbase.zookeeper;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
|
@ -28,8 +26,9 @@ import org.apache.zookeeper.WatchedEvent;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class EmptyWatcher implements Watcher {
|
||||
// Used in this package but also by tests so needs to be public
|
||||
public static EmptyWatcher instance = new EmptyWatcher();
|
||||
private EmptyWatcher() {}
|
||||
|
||||
public void process(WatchedEvent event) {}
|
||||
}
|
||||
}
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
|
@ -80,7 +81,12 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker {
|
|||
* @return Server name or null if timed out.
|
||||
*/
|
||||
public ServerName getMasterAddress(final boolean refresh) {
|
||||
return ZKUtil.znodeContentToServerName(super.getData(refresh));
|
||||
try {
|
||||
return ServerName.parseFrom(super.getData(refresh));
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Failed parse", e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -99,7 +105,13 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker {
|
|||
if (data == null){
|
||||
throw new IOException("Can't get master address from ZooKeeper; znode data == null");
|
||||
}
|
||||
return ZKUtil.znodeContentToServerName(data);
|
||||
try {
|
||||
return ServerName.parseFrom(data);
|
||||
} catch (DeserializationException e) {
|
||||
KeeperException ke = new KeeperException.DataInconsistencyException();
|
||||
ke.initCause(e);
|
||||
throw ke;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -116,7 +128,7 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker {
|
|||
public static boolean setMasterAddress(final ZooKeeperWatcher zkw,
|
||||
final String znode, final ServerName master)
|
||||
throws KeeperException {
|
||||
return ZKUtil.createEphemeralNodeAndWatch(zkw, znode, getZNodeData(master));
|
||||
return ZKUtil.createEphemeralNodeAndWatch(zkw, znode, toByteArray(master));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -132,7 +144,7 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker {
|
|||
* @return Content of the master znode as a serialized pb with the pb
|
||||
* magic as prefix.
|
||||
*/
|
||||
static byte [] getZNodeData(final ServerName sn) {
|
||||
static byte [] toByteArray(final ServerName sn) {
|
||||
ZooKeeperProtos.Master.Builder mbuilder = ZooKeeperProtos.Master.newBuilder();
|
||||
HBaseProtos.ServerName.Builder snbuilder = HBaseProtos.ServerName.newBuilder();
|
||||
snbuilder.setHostName(sn.getHostname());
|
||||
|
|
|
@ -19,17 +19,13 @@ package org.apache.hadoop.hbase.zookeeper;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RootRegionServer;
|
||||
import org.apache.hadoop.hbase.util.Addressing;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
* Tracks the root region server location node in zookeeper.
|
||||
* Root region location is set by {@link RootLocationEditor} usually called
|
||||
|
@ -64,7 +60,12 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
|
|||
* @throws InterruptedException
|
||||
*/
|
||||
public ServerName getRootRegionLocation() throws InterruptedException {
|
||||
return ZKUtil.znodeContentToServerName(super.getData(true));
|
||||
try {
|
||||
return ServerName.parseFrom(super.getData(true));
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Failed parse", e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -76,7 +77,11 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
|
|||
*/
|
||||
public static ServerName getRootRegionLocation(final ZooKeeperWatcher zkw)
|
||||
throws KeeperException {
|
||||
return ZKUtil.znodeContentToServerName(ZKUtil.getData(zkw, zkw.rootServerZNode));
|
||||
try {
|
||||
return ServerName.parseFrom(ZKUtil.getData(zkw, zkw.rootServerZNode));
|
||||
} catch (DeserializationException e) {
|
||||
throw ZKUtil.convert(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -97,7 +102,12 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
|
|||
LOG.error(errorMsg);
|
||||
throw new IllegalArgumentException(errorMsg);
|
||||
}
|
||||
return ZKUtil.znodeContentToServerName(super.blockUntilAvailable(timeout, true));
|
||||
try {
|
||||
return ServerName.parseFrom(super.blockUntilAvailable(timeout, true));
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Failed parse", e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -113,7 +123,7 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
|
|||
LOG.info("Setting ROOT region location in ZooKeeper as " + location);
|
||||
// Make the RootRegionServer pb and then get its bytes and save this as
|
||||
// the znode content.
|
||||
byte [] data = getZNodeData(location);
|
||||
byte [] data = toByteArray(location);
|
||||
try {
|
||||
ZKUtil.createAndWatch(zookeeper, zookeeper.rootServerZNode, data);
|
||||
} catch(KeeperException.NodeExistsException nee) {
|
||||
|
@ -127,7 +137,7 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
|
|||
* @param sn What to put into the znode.
|
||||
* @return The content of the root-region-server znode
|
||||
*/
|
||||
static byte [] getZNodeData(final ServerName sn) {
|
||||
static byte [] toByteArray(final ServerName sn) {
|
||||
// ZNode content is a pb message preceeded by some pb magic.
|
||||
HBaseProtos.ServerName pbsn =
|
||||
HBaseProtos.ServerName.newBuilder().setHostName(sn.getHostname()).
|
||||
|
@ -164,6 +174,12 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
|
|||
final long timeout)
|
||||
throws InterruptedException {
|
||||
byte [] data = ZKUtil.blockUntilAvailable(zkw, zkw.rootServerZNode, timeout);
|
||||
return ZKUtil.znodeContentToServerName(data);
|
||||
if (data == null) return null;
|
||||
try {
|
||||
return ServerName.parseFrom(data);
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Failed parse", e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,9 +25,13 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.RegionTransition;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.executor.RegionTransitionData;
|
||||
// We should not be importing this Type here, nor a RegionTransition, etc. This class should be
|
||||
// about zk and bytes only.
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.zookeeper.AsyncCallback;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
@ -145,10 +149,10 @@ 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.createRegionTransition(event, region.getRegionName(), serverName);
|
||||
String node = getNodeName(zkw, region.getEncodedName());
|
||||
ZKUtil.createAndWatch(zkw, node, data.getBytes());
|
||||
ZKUtil.createAndWatch(zkw, node, rt.toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -172,10 +176,10 @@ public class ZKAssign {
|
|||
throws KeeperException {
|
||||
LOG.debug(zkw.prefix("Async create of unassigned node for " +
|
||||
region.getEncodedName() + " with OFFLINE state"));
|
||||
RegionTransitionData data = new RegionTransitionData(
|
||||
EventType.M_ZK_REGION_OFFLINE, region.getRegionName(), serverName);
|
||||
RegionTransition rt =
|
||||
RegionTransition.createRegionTransition(EventType.M_ZK_REGION_OFFLINE, region.getRegionName(), serverName);
|
||||
String node = getNodeName(zkw, region.getEncodedName());
|
||||
ZKUtil.asyncCreate(zkw, node, data.getBytes(), cb, ctx);
|
||||
ZKUtil.asyncCreate(zkw, node, rt.toByteArray(), cb, ctx);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -201,10 +205,10 @@ public class ZKAssign {
|
|||
throws KeeperException, KeeperException.NoNodeException {
|
||||
LOG.debug(zkw.prefix("Forcing existing unassigned node for " +
|
||||
region.getEncodedName() + " to OFFLINE state"));
|
||||
RegionTransitionData data = new RegionTransitionData(
|
||||
EventType.M_ZK_REGION_OFFLINE, region.getRegionName(), serverName);
|
||||
RegionTransition rt =
|
||||
RegionTransition.createRegionTransition(EventType.M_ZK_REGION_OFFLINE, region.getRegionName(), serverName);
|
||||
String node = getNodeName(zkw, region.getEncodedName());
|
||||
ZKUtil.setData(zkw, node, data.getBytes());
|
||||
ZKUtil.setData(zkw, node, rt.toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -267,8 +271,9 @@ public class ZKAssign {
|
|||
throws KeeperException {
|
||||
LOG.debug(zkw.prefix("Creating (or updating) unassigned node for " +
|
||||
region.getEncodedName() + " with OFFLINE state"));
|
||||
RegionTransitionData data = new RegionTransitionData(
|
||||
EventType.M_ZK_REGION_OFFLINE, region.getRegionName(), serverName);
|
||||
RegionTransition rt = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_OFFLINE,
|
||||
region.getRegionName(), serverName, HConstants.EMPTY_BYTE_ARRAY);
|
||||
byte [] data = rt.toByteArray();
|
||||
String node = getNodeName(zkw, region.getEncodedName());
|
||||
Stat stat = new Stat();
|
||||
zkw.sync(node);
|
||||
|
@ -282,15 +287,15 @@ public class ZKAssign {
|
|||
if (hijack && !allowCreation) {
|
||||
return -1;
|
||||
}
|
||||
return ZKUtil.createAndWatch(zkw, node, data.getBytes());
|
||||
return ZKUtil.createAndWatch(zkw, node, data);
|
||||
} else {
|
||||
RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
|
||||
.getEncodedName(), stat);
|
||||
byte [] curDataInZNode = ZKAssign.getDataNoWatch(zkw, region.getEncodedName(), stat);
|
||||
RegionTransition curRt = getRegionTransition(curDataInZNode);
|
||||
// Do not move the node to OFFLINE if znode is in any of the following
|
||||
// state.
|
||||
// Because these are already executed states.
|
||||
if (hijack && null != curDataInZNode) {
|
||||
EventType eventType = curDataInZNode.getEventType();
|
||||
if (hijack && curRt != null) {
|
||||
EventType eventType = curRt.getEventType();
|
||||
if (eventType.equals(EventType.M_ZK_REGION_CLOSING)
|
||||
|| eventType.equals(EventType.RS_ZK_REGION_CLOSED)
|
||||
|| eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
|
||||
|
@ -300,7 +305,7 @@ public class ZKAssign {
|
|||
|
||||
boolean setData = false;
|
||||
try {
|
||||
setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
|
||||
setData = ZKUtil.setData(zkw, node, data, version);
|
||||
// Setdata throws KeeperException which aborts the Master. So we are
|
||||
// catching it here.
|
||||
// If just before setting the znode to OFFLINE if the RS has made any
|
||||
|
@ -315,9 +320,9 @@ public class ZKAssign {
|
|||
} else {
|
||||
// We successfully forced to OFFLINE, reset watch and handle if
|
||||
// the state changed in between our set and the watch
|
||||
RegionTransitionData curData =
|
||||
ZKAssign.getData(zkw, region.getEncodedName());
|
||||
if (curData.getEventType() != data.getEventType()) {
|
||||
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
rt = getRegionTransition(bytes);
|
||||
if (rt.getEventType() != EventType.M_ZK_REGION_OFFLINE) {
|
||||
// state changed, need to process
|
||||
return -1;
|
||||
}
|
||||
|
@ -494,11 +499,11 @@ public class ZKAssign {
|
|||
// If it came back null, node does not exist.
|
||||
throw KeeperException.create(Code.NONODE);
|
||||
}
|
||||
RegionTransitionData data = RegionTransitionData.fromBytes(bytes);
|
||||
if (!data.getEventType().equals(expectedState)) {
|
||||
LOG.warn(zkw.prefix("Attempting to delete unassigned " +
|
||||
"node " + regionName + " in " + expectedState +
|
||||
" state but node is in " + data.getEventType() + " state"));
|
||||
RegionTransition rt = getRegionTransition(bytes);
|
||||
EventType et = rt.getEventType();
|
||||
if (!et.equals(expectedState)) {
|
||||
LOG.warn(zkw.prefix("Attempting to delete unassigned node " + regionName + " in " +
|
||||
expectedState + " state but node is in " + et + " state"));
|
||||
return false;
|
||||
}
|
||||
if (expectedVersion != -1
|
||||
|
@ -564,12 +569,10 @@ public class ZKAssign {
|
|||
throws KeeperException, KeeperException.NodeExistsException {
|
||||
LOG.debug(zkw.prefix("Creating unassigned node for " +
|
||||
region.getEncodedName() + " in a CLOSING state"));
|
||||
|
||||
RegionTransitionData data = new RegionTransitionData(
|
||||
EventType.M_ZK_REGION_CLOSING, region.getRegionName(), serverName);
|
||||
|
||||
RegionTransition rt = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_CLOSING,
|
||||
region.getRegionName(), serverName, HConstants.EMPTY_BYTE_ARRAY);
|
||||
String node = getNodeName(zkw, region.getEncodedName());
|
||||
return ZKUtil.createAndWatch(zkw, node, data.getBytes());
|
||||
return ZKUtil.createAndWatch(zkw, node, rt.toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -748,8 +751,7 @@ public class ZKAssign {
|
|||
ServerName serverName, EventType beginState, EventType endState,
|
||||
int expectedVersion)
|
||||
throws KeeperException {
|
||||
return transitionNode(zkw, region, serverName, beginState, endState,
|
||||
expectedVersion, null);
|
||||
return transitionNode(zkw, region, serverName, beginState, endState, expectedVersion, null);
|
||||
}
|
||||
|
||||
public static int transitionNode(ZooKeeperWatcher zkw, HRegionInfo region,
|
||||
|
@ -773,11 +775,10 @@ public class ZKAssign {
|
|||
// Node no longer exists. Return -1. It means unsuccessful transition.
|
||||
return -1;
|
||||
}
|
||||
RegionTransitionData existingData =
|
||||
RegionTransitionData.fromBytes(existingBytes);
|
||||
RegionTransition rt = getRegionTransition(existingBytes);
|
||||
|
||||
// Verify it is the expected version
|
||||
if(expectedVersion != -1 && stat.getVersion() != expectedVersion) {
|
||||
if (expectedVersion != -1 && stat.getVersion() != expectedVersion) {
|
||||
LOG.warn(zkw.prefix("Attempt to transition the " +
|
||||
"unassigned node for " + encoded +
|
||||
" from " + beginState + " to " + endState + " failed, " +
|
||||
|
@ -799,20 +800,19 @@ public class ZKAssign {
|
|||
}
|
||||
|
||||
// Verify it is in expected state
|
||||
if(!existingData.getEventType().equals(beginState)) {
|
||||
EventType et = rt.getEventType();
|
||||
if (!et.equals(beginState)) {
|
||||
LOG.warn(zkw.prefix("Attempt to transition the " +
|
||||
"unassigned node for " + encoded +
|
||||
" from " + beginState + " to " + endState + " failed, " +
|
||||
"the node existed but was in the state " + existingData.getEventType() +
|
||||
" set by the server " + serverName));
|
||||
"the node existed but was in the state " + et + " set by the server " + serverName));
|
||||
return -1;
|
||||
}
|
||||
|
||||
// Write new data, ensuring data has not changed since we last read it
|
||||
try {
|
||||
RegionTransitionData data = new RegionTransitionData(endState,
|
||||
region.getRegionName(), serverName, payload);
|
||||
if(!ZKUtil.setData(zkw, node, data.getBytes(), stat.getVersion())) {
|
||||
rt = RegionTransition.createRegionTransition(endState, region.getRegionName(), serverName, payload);
|
||||
if(!ZKUtil.setData(zkw, node, rt.toByteArray(), stat.getVersion())) {
|
||||
LOG.warn(zkw.prefix("Attempt to transition the " +
|
||||
"unassigned node for " + encoded +
|
||||
" from " + beginState + " to " + endState + " failed, " +
|
||||
|
@ -835,6 +835,15 @@ public class ZKAssign {
|
|||
}
|
||||
}
|
||||
|
||||
private static RegionTransition getRegionTransition(final byte [] bytes) throws KeeperException {
|
||||
try {
|
||||
return RegionTransition.parseFrom(bytes);
|
||||
} catch (DeserializationException e) {
|
||||
// Convert to a zk exception for now. Otherwise have to change API
|
||||
throw ZKUtil.convert(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the current data in the unassigned node for the specified region name
|
||||
* or fully-qualified path.
|
||||
|
@ -845,19 +854,14 @@ public class ZKAssign {
|
|||
*
|
||||
* @param zkw zk reference
|
||||
* @param pathOrRegionName fully-specified path or region name
|
||||
* @return data for the unassigned node
|
||||
* @return znode content
|
||||
* @throws KeeperException if unexpected zookeeper exception
|
||||
*/
|
||||
public static RegionTransitionData getData(ZooKeeperWatcher zkw,
|
||||
public static byte [] getData(ZooKeeperWatcher zkw,
|
||||
String pathOrRegionName)
|
||||
throws KeeperException {
|
||||
String node = pathOrRegionName.startsWith("/") ?
|
||||
pathOrRegionName : getNodeName(zkw, pathOrRegionName);
|
||||
byte [] data = ZKUtil.getDataAndWatch(zkw, node);
|
||||
if(data == null) {
|
||||
return null;
|
||||
}
|
||||
return RegionTransitionData.fromBytes(data);
|
||||
String node = getPath(zkw, pathOrRegionName);
|
||||
return ZKUtil.getDataAndWatch(zkw, node);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -871,19 +875,14 @@ 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 znode content
|
||||
* @throws KeeperException if unexpected zookeeper exception
|
||||
*/
|
||||
public static RegionTransitionData getDataAndWatch(ZooKeeperWatcher zkw,
|
||||
public static byte [] getDataAndWatch(ZooKeeperWatcher zkw,
|
||||
String pathOrRegionName, Stat stat)
|
||||
throws KeeperException {
|
||||
String node = pathOrRegionName.startsWith("/") ?
|
||||
pathOrRegionName : getNodeName(zkw, pathOrRegionName);
|
||||
byte [] data = ZKUtil.getDataAndWatch(zkw, node, stat);
|
||||
if(data == null) {
|
||||
return null;
|
||||
}
|
||||
return RegionTransitionData.fromBytes(data);
|
||||
String node = getPath(zkw, pathOrRegionName);
|
||||
return ZKUtil.getDataAndWatch(zkw, node, stat);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -897,19 +896,23 @@ 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 znode content
|
||||
* @throws KeeperException if unexpected zookeeper exception
|
||||
*/
|
||||
public static RegionTransitionData getDataNoWatch(ZooKeeperWatcher zkw,
|
||||
public static byte [] getDataNoWatch(ZooKeeperWatcher zkw,
|
||||
String pathOrRegionName, Stat stat)
|
||||
throws KeeperException {
|
||||
String node = pathOrRegionName.startsWith("/") ?
|
||||
pathOrRegionName : getNodeName(zkw, pathOrRegionName);
|
||||
byte [] data = ZKUtil.getDataNoWatch(zkw, node, stat);
|
||||
if (data == null) {
|
||||
return null;
|
||||
}
|
||||
return RegionTransitionData.fromBytes(data);
|
||||
String node = getPath(zkw, pathOrRegionName);
|
||||
return ZKUtil.getDataNoWatch(zkw, node, stat);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param zkw
|
||||
* @param pathOrRegionName
|
||||
* @return Path to znode
|
||||
*/
|
||||
public static String getPath(final ZooKeeperWatcher zkw, final String pathOrRegionName) {
|
||||
return pathOrRegionName.startsWith("/")? pathOrRegionName : getNodeName(zkw, pathOrRegionName);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -983,42 +986,18 @@ public class ZKAssign {
|
|||
}
|
||||
|
||||
/**
|
||||
* Verifies that the specified region is in the specified state in ZooKeeper.
|
||||
* <p>
|
||||
* Returns true if region is in transition and in the specified state in
|
||||
* ZooKeeper. Returns false if the region does not exist in ZK or is in
|
||||
* a different state.
|
||||
* <p>
|
||||
* Method synchronizes() with ZK so will yield an up-to-date result but is
|
||||
* a slow read.
|
||||
* @param zkw
|
||||
* @param region
|
||||
* @param expectedState
|
||||
* @return true if region exists and is in expected state
|
||||
* Presume bytes are serialized unassigned data structure
|
||||
* @param znodeBytes
|
||||
* @return String of the deserialized znode bytes.
|
||||
*/
|
||||
public static boolean verifyRegionState(ZooKeeperWatcher zkw,
|
||||
HRegionInfo region, EventType expectedState)
|
||||
throws KeeperException {
|
||||
String encoded = region.getEncodedName();
|
||||
|
||||
String node = getNodeName(zkw, encoded);
|
||||
zkw.sync(node);
|
||||
|
||||
// Read existing data of the node
|
||||
byte [] existingBytes = null;
|
||||
static String toString(final byte[] znodeBytes) {
|
||||
// This method should not exist. Used by ZKUtil stringifying RegionTransition. Have the
|
||||
// method in here so RegionTransition does not leak into ZKUtil.
|
||||
try {
|
||||
existingBytes = ZKUtil.getDataAndWatch(zkw, node);
|
||||
} catch (KeeperException.NoNodeException nne) {
|
||||
return false;
|
||||
} catch (KeeperException e) {
|
||||
throw e;
|
||||
RegionTransition rt = RegionTransition.parseFrom(znodeBytes);
|
||||
return rt.toString();
|
||||
} catch (DeserializationException e) {
|
||||
return "";
|
||||
}
|
||||
if (existingBytes == null) return false;
|
||||
RegionTransitionData existingData =
|
||||
RegionTransitionData.fromBytes(existingBytes);
|
||||
if (existingData.getEventType() == expectedState){
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
@ -21,13 +19,10 @@ package org.apache.hadoop.hbase.zookeeper;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
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;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -37,30 +32,23 @@ 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
|
||||
* {@link SplitLogWorker}
|
||||
* Common methods and attributes used by {@link SplitLogManager} and {@link SplitLogWorker}
|
||||
* running distributed splitting of WAL logs.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ZKSplitLog {
|
||||
private static final Log LOG = LogFactory.getLog(ZKSplitLog.class);
|
||||
|
||||
public static final int DEFAULT_TIMEOUT = 25000; // 25 sec
|
||||
public static final int DEFAULT_ZK_RETRIES = 3;
|
||||
public static final int DEFAULT_MAX_RESUBMIT = 3;
|
||||
public static final int DEFAULT_UNASSIGNED_TIMEOUT = (3 * 60 * 1000); //3 min
|
||||
|
||||
/**
|
||||
* Gets the full path node name for the log file being split.
|
||||
* This method will url encode the filename.
|
||||
* @param zkw zk reference
|
||||
* @param filename log file name (only the basename)
|
||||
*/
|
||||
public static String getEncodedNodeName(ZooKeeperWatcher zkw,
|
||||
String filename) {
|
||||
return ZKUtil.joinZNode(zkw.splitLogZNode, encode(filename));
|
||||
public static String getEncodedNodeName(ZooKeeperWatcher zkw, String filename) {
|
||||
return ZKUtil.joinZNode(zkw.splitLogZNode, encode(filename));
|
||||
}
|
||||
|
||||
public static String getFileName(String node) {
|
||||
|
@ -68,8 +56,7 @@ public class ZKSplitLog {
|
|||
return decode(basename);
|
||||
}
|
||||
|
||||
|
||||
public static String encode(String s) {
|
||||
static String encode(String s) {
|
||||
try {
|
||||
return URLEncoder.encode(s, "UTF-8");
|
||||
} catch (UnsupportedEncodingException e) {
|
||||
|
@ -77,7 +64,7 @@ public class ZKSplitLog {
|
|||
}
|
||||
}
|
||||
|
||||
public static String decode(String s) {
|
||||
static String decode(String s) {
|
||||
try {
|
||||
return URLDecoder.decode(s, "UTF-8");
|
||||
} catch (UnsupportedEncodingException e) {
|
||||
|
@ -107,53 +94,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);
|
||||
}
|
||||
|
@ -172,8 +112,8 @@ public class ZKSplitLog {
|
|||
return ret;
|
||||
}
|
||||
|
||||
public static String getSplitLogDirTmpComponent(String worker, String file) {
|
||||
return (worker + "_" + ZKSplitLog.encode(file));
|
||||
public static String getSplitLogDirTmpComponent(final String worker, String file) {
|
||||
return worker + "_" + ZKSplitLog.encode(file);
|
||||
}
|
||||
|
||||
public static void markCorrupted(Path rootdir, String tmpname,
|
||||
|
@ -198,81 +138,4 @@ public class ZKSplitLog {
|
|||
public static boolean isCorruptFlagFile(Path file) {
|
||||
return file.getName().equals("corrupt");
|
||||
}
|
||||
|
||||
|
||||
public static class Counters {
|
||||
//SplitLogManager counters
|
||||
public static AtomicLong tot_mgr_log_split_batch_start = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_log_split_batch_success =
|
||||
new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_log_split_batch_err = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_new_unexpected_hlogs = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_log_split_start = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_log_split_success = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_log_split_err = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_create_queued = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_create_result = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_already_exists = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_create_err = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_create_retry = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_get_data_queued = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_get_data_result = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_get_data_nonode = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_get_data_err = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_get_data_retry = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_delete_queued = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_delete_result = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_node_delete_err = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_resubmit = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_resubmit_failed = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_null_data = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_orphan_task_acquired = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_wait_for_zk_delete = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_unacquired_orphan_done = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_resubmit_threshold_reached =
|
||||
new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_missing_state_in_delete =
|
||||
new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_heartbeat = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_rescan = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_rescan_deleted = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_task_deleted = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_resubmit_unassigned = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_relist_logdir = new AtomicLong(0);
|
||||
public static AtomicLong tot_mgr_resubmit_dead_server_task =
|
||||
new AtomicLong(0);
|
||||
|
||||
|
||||
|
||||
// SplitLogWorker counters
|
||||
public static AtomicLong tot_wkr_failed_to_grab_task_no_data =
|
||||
new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_failed_to_grab_task_exception =
|
||||
new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_failed_to_grab_task_owned =
|
||||
new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_failed_to_grab_task_lost_race =
|
||||
new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_task_acquired = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_task_resigned = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_task_done = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_task_err = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_task_heartbeat = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_task_acquired_rescan = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_get_data_queued = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_get_data_result = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_get_data_retry = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_preempt_task = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_task_heartbeat_failed = new AtomicLong(0);
|
||||
public static AtomicLong tot_wkr_final_transistion_failed =
|
||||
new AtomicLong(0);
|
||||
|
||||
public static void resetCounters() throws Exception {
|
||||
Class<?> cl = (new Counters()).getClass();
|
||||
Field[] flds = cl.getDeclaredFields();
|
||||
for (Field fld : flds) {
|
||||
((AtomicLong)fld.get(null)).set(0);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
||||
|
@ -36,28 +35,21 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.EmptyWatcher;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
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;
|
||||
import org.apache.hadoop.hbase.util.Addressing;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.zookeeper.AsyncCallback;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.apache.zookeeper.KeeperException.NoNodeException;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.zookeeper.ZooDefs.Ids;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.apache.zookeeper.data.ACL;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
* Internal HBase utility class for ZooKeeper.
|
||||
*
|
||||
|
@ -359,8 +351,7 @@ public class ZKUtil {
|
|||
* null if parent does not exist
|
||||
* @throws KeeperException if unexpected zookeeper exception
|
||||
*/
|
||||
public static List<String> listChildrenNoWatch(
|
||||
ZooKeeperWatcher zkw, String znode)
|
||||
public static List<String> listChildrenNoWatch(ZooKeeperWatcher zkw, String znode)
|
||||
throws KeeperException {
|
||||
List<String> children = null;
|
||||
try {
|
||||
|
@ -376,7 +367,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 +385,7 @@ public class ZKUtil {
|
|||
}
|
||||
@Override
|
||||
public String toString() {
|
||||
return node + " (" + RegionTransitionData.fromBytes(data) + ")";
|
||||
return node;
|
||||
}
|
||||
public boolean isEmpty() {
|
||||
return (data.length == 0);
|
||||
|
@ -600,6 +593,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<NodeAndData> getChildDataAndWatchForNewChildren(
|
||||
ZooKeeperWatcher zkw, String baseNode) throws KeeperException {
|
||||
|
@ -630,6 +624,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 +1139,21 @@ 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)?
|
||||
ZKAssign.toString(data): // We should not be doing this reaching into another class
|
||||
znode.startsWith(zkw.rootServerZNode)?
|
||||
getServerNameOrEmptyString(data):
|
||||
znode.startsWith(zkw.backupMasterAddressesZNode)?
|
||||
getServerNameOrEmptyString(data):
|
||||
StringUtils.abbreviate(Bytes.toStringBinary(data), 32)))));
|
||||
}
|
||||
|
||||
private static String getServerNameOrEmptyString(final byte [] data) {
|
||||
try {
|
||||
return ServerName.parseFrom(data).toString();
|
||||
} catch (DeserializationException e) {
|
||||
return "";
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1222,44 +1229,14 @@ public class ZKUtil {
|
|||
|
||||
|
||||
/**
|
||||
* Get a ServerName from the passed in znode data bytes.
|
||||
* @param data ZNode data with a server name in it; can handle the old style
|
||||
* servername where servername was host and port. Works too with data that
|
||||
* begins w/ the pb 'PBUF' magic and that its then followed by a protobuf that
|
||||
* has a serialized {@link ServerName} in it.
|
||||
* @return Returns null if <code>data</code> is null else converts passed data
|
||||
* to a ServerName instance.
|
||||
* Convert a {@link DeserializationException} to a more palatable {@link KeeperException}.
|
||||
* Used when can't let a {@link DeserializationException} out w/o changing public API.
|
||||
* @param e Exception to convert
|
||||
* @return Converted exception
|
||||
*/
|
||||
public static ServerName znodeContentToServerName(final byte [] data) {
|
||||
if (data == null || data.length <= 0) return null;
|
||||
if (ProtobufUtil.isPBMagicPrefix(data)) {
|
||||
int prefixLen = ProtobufUtil.lengthOfPBMagic();
|
||||
try {
|
||||
RootRegionServer rss =
|
||||
RootRegionServer.newBuilder().mergeFrom(data, prefixLen, data.length - prefixLen).build();
|
||||
HBaseProtos.ServerName sn = rss.getServer();
|
||||
return new ServerName(sn.getHostName(), sn.getPort(), sn.getStartCode());
|
||||
} catch (InvalidProtocolBufferException e) {
|
||||
// A failed parse of the znode is pretty catastrophic. Rather than loop
|
||||
// retrying hoping the bad bytes will changes, and rather than change
|
||||
// the signature on this method to add an IOE which will send ripples all
|
||||
// over the code base, throw a RuntimeException. This should "never" happen.
|
||||
// Fail fast if it does.
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
// The str returned could be old style -- pre hbase-1502 -- which was
|
||||
// hostname and port seperated by a colon rather than hostname, port and
|
||||
// startcode delimited by a ','.
|
||||
String str = Bytes.toString(data);
|
||||
int index = str.indexOf(ServerName.SERVERNAME_SEPARATOR);
|
||||
if (index != -1) {
|
||||
// Presume its ServerName serialized with versioned bytes.
|
||||
return ServerName.parseVersionedServerName(data);
|
||||
}
|
||||
// Presume it a hostname:port format.
|
||||
String hostname = Addressing.parseHostname(str);
|
||||
int port = Addressing.parsePort(str);
|
||||
return new ServerName(hostname, port, -1L);
|
||||
public static KeeperException convert(final DeserializationException e) {
|
||||
KeeperException ke = new KeeperException.DataInconsistencyException();
|
||||
ke.initCause(e);
|
||||
return ke;
|
||||
}
|
||||
}
|
|
@ -60,3 +60,33 @@ message ClusterUp {
|
|||
// the data is cluster startDate.
|
||||
required string startDate = 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* What we write under unassigned up in zookeeper as a region moves through
|
||||
* open/close, etc., regions. Details a region in transition.
|
||||
*/
|
||||
message RegionTransition {
|
||||
// Code for EventType gotten by doing o.a.h.h.EventHandler.EventType.getCode()
|
||||
required uint32 eventTypeCode = 1;
|
||||
// Full regionname in bytes
|
||||
required bytes regionName = 2;
|
||||
required uint64 createTime = 3;
|
||||
optional ServerName originServerName = 4;
|
||||
optional bytes payload = 5;
|
||||
}
|
||||
|
||||
/**
|
||||
* WAL SplitLog directory znodes have this for content. Used doing distributed
|
||||
* WAL splitting. Holds current state and name of server that originated split.
|
||||
*/
|
||||
message SplitLogTask {
|
||||
enum State {
|
||||
UNASSIGNED = 0;
|
||||
OWNED = 1;
|
||||
RESIGNED = 2;
|
||||
DONE = 3;
|
||||
ERR = 4;
|
||||
}
|
||||
required State state = 1;
|
||||
required ServerName serverName = 2;
|
||||
}
|
||||
|
|
|
@ -81,6 +81,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
|||
import org.apache.hadoop.hbase.util.RegionSplitter;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.util.Writables;
|
||||
import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
|
||||
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
|
||||
|
|
|
@ -58,6 +58,14 @@ import org.junit.experimental.categories.Category;
|
|||
*/
|
||||
@Category(SmallTests.class)
|
||||
public class TestSerialization {
|
||||
@Test
|
||||
public void testSplitLogTask() throws DeserializationException {
|
||||
SplitLogTask slt = new SplitLogTask.Unassigned(new ServerName("mgr,1,1"));
|
||||
byte [] bytes = slt.toByteArray();
|
||||
SplitLogTask sltDeserialized = SplitLogTask.parseFrom(bytes);
|
||||
assertTrue(slt.equals(sltDeserialized));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHServerLoadVersioning() throws IOException {
|
||||
Set<String> cps = new HashSet<String>(0);
|
||||
|
@ -66,7 +74,7 @@ public class TestSerialization {
|
|||
new HServerLoad092.RegionLoad(HConstants.META_TABLE_NAME, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, cps));
|
||||
HServerLoad092 hsl092 = new HServerLoad092(0, 0, 0, 0, regions, cps);
|
||||
byte [] hsl092bytes = Writables.getBytes(hsl092);
|
||||
HServerLoad hsl = (HServerLoad)Writables.getWritable(hsl092bytes, new HServerLoad());
|
||||
Writables.getWritable(hsl092bytes, new HServerLoad());
|
||||
// TO BE CONTINUED
|
||||
}
|
||||
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.HTable;
|
|||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
|
@ -154,8 +155,8 @@ public class TestZooKeeper {
|
|||
|
||||
// It's surprising but sometimes we can still be in connected state.
|
||||
// As it's known (even if not understood) we don't make the the test fail
|
||||
// for this reason.
|
||||
Assert.assertTrue(state == States.CLOSED);
|
||||
// for this reason.)
|
||||
// Assert.assertTrue("state=" + state, state == States.CLOSED);
|
||||
|
||||
// Check that the client recovered
|
||||
ZooKeeperWatcher newConnectionZK = getZooKeeperWatcher(connection);
|
||||
|
@ -179,7 +180,7 @@ public class TestZooKeeper {
|
|||
|
||||
connection.close();
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testRegionServerSessionExpired() throws Exception {
|
||||
LOG.info("Starting testRegionServerSessionExpired");
|
||||
|
|
|
@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
@ -51,7 +50,7 @@ public class TestMetaReaderEditor {
|
|||
private static final Log LOG = LogFactory.getLog(TestMetaReaderEditor.class);
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
private static ZooKeeperWatcher zkw;
|
||||
private static CatalogTracker ct;
|
||||
private static CatalogTracker CT;
|
||||
private final static Abortable ABORTABLE = new Abortable() {
|
||||
private final AtomicBoolean abort = new AtomicBoolean(false);
|
||||
|
||||
|
@ -77,13 +76,13 @@ public class TestMetaReaderEditor {
|
|||
c.setLong("hbase.client.pause", 1000);
|
||||
c.setInt("hbase.client.retries.number", 10);
|
||||
zkw = new ZooKeeperWatcher(c, "TestMetaReaderEditor", ABORTABLE);
|
||||
ct = new CatalogTracker(zkw, c, ABORTABLE);
|
||||
ct.start();
|
||||
CT = new CatalogTracker(zkw, c, ABORTABLE);
|
||||
CT.start();
|
||||
}
|
||||
|
||||
@AfterClass public static void afterClass() throws Exception {
|
||||
ABORTABLE.abort("test ending", null);
|
||||
ct.stop();
|
||||
CT.stop();
|
||||
UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
|
@ -103,15 +102,15 @@ public class TestMetaReaderEditor {
|
|||
int regionCount = UTIL.createMultiRegions(t, HConstants.CATALOG_FAMILY);
|
||||
// Test it works getting a region from just made user table.
|
||||
final List<HRegionInfo> regions =
|
||||
testGettingTableRegions(this.ct, nameBytes, regionCount);
|
||||
MetaTask reader = new MetaTask(this.ct, "reader") {
|
||||
testGettingTableRegions(CT, nameBytes, regionCount);
|
||||
MetaTask reader = new MetaTask(CT, "reader") {
|
||||
@Override
|
||||
void metaTask() throws Throwable {
|
||||
testGetRegion(this.ct, regions.get(0));
|
||||
LOG.info("Read " + regions.get(0).getEncodedName());
|
||||
}
|
||||
};
|
||||
MetaTask writer = new MetaTask(this.ct, "writer") {
|
||||
MetaTask writer = new MetaTask(CT, "writer") {
|
||||
@Override
|
||||
void metaTask() throws Throwable {
|
||||
MetaEditor.addRegionToMeta(this.ct, regions.get(0));
|
||||
|
@ -140,7 +139,7 @@ public class TestMetaReaderEditor {
|
|||
int index = -1;
|
||||
do {
|
||||
index = UTIL.getMiniHBaseCluster().getServerWithMeta();
|
||||
}while (index == -1 &&
|
||||
} while (index == -1 &&
|
||||
startTime + timeOut < System.currentTimeMillis());
|
||||
|
||||
if (index != -1){
|
||||
|
@ -149,8 +148,8 @@ public class TestMetaReaderEditor {
|
|||
}
|
||||
}
|
||||
|
||||
assertTrue("reader: "+reader.toString(), reader.isProgressing());
|
||||
assertTrue("writer: "+writer.toString(), writer.isProgressing());
|
||||
assertTrue("reader: " + reader.toString(), reader.isProgressing());
|
||||
assertTrue("writer: " + writer.toString(), writer.isProgressing());
|
||||
} catch (IOException e) {
|
||||
throw e;
|
||||
} finally {
|
||||
|
@ -216,27 +215,27 @@ public class TestMetaReaderEditor {
|
|||
@Test public void testGetRegionsCatalogTables()
|
||||
throws IOException, InterruptedException {
|
||||
List<HRegionInfo> regions =
|
||||
MetaReader.getTableRegions(ct, HConstants.META_TABLE_NAME);
|
||||
MetaReader.getTableRegions(CT, HConstants.META_TABLE_NAME);
|
||||
assertTrue(regions.size() >= 1);
|
||||
assertTrue(MetaReader.getTableRegionsAndLocations(ct,
|
||||
assertTrue(MetaReader.getTableRegionsAndLocations(CT,
|
||||
Bytes.toString(HConstants.META_TABLE_NAME)).size() >= 1);
|
||||
assertTrue(MetaReader.getTableRegionsAndLocations(ct,
|
||||
assertTrue(MetaReader.getTableRegionsAndLocations(CT,
|
||||
Bytes.toString(HConstants.ROOT_TABLE_NAME)).size() == 1);
|
||||
}
|
||||
|
||||
@Test public void testTableExists() throws IOException {
|
||||
final String name = "testTableExists";
|
||||
final byte [] nameBytes = Bytes.toBytes(name);
|
||||
assertFalse(MetaReader.tableExists(ct, name));
|
||||
assertFalse(MetaReader.tableExists(CT, name));
|
||||
UTIL.createTable(nameBytes, HConstants.CATALOG_FAMILY);
|
||||
assertTrue(MetaReader.tableExists(ct, name));
|
||||
assertTrue(MetaReader.tableExists(CT, name));
|
||||
HBaseAdmin admin = UTIL.getHBaseAdmin();
|
||||
admin.disableTable(name);
|
||||
admin.deleteTable(name);
|
||||
assertFalse(MetaReader.tableExists(ct, name));
|
||||
assertTrue(MetaReader.tableExists(ct,
|
||||
assertFalse(MetaReader.tableExists(CT, name));
|
||||
assertTrue(MetaReader.tableExists(CT,
|
||||
Bytes.toString(HConstants.META_TABLE_NAME)));
|
||||
assertTrue(MetaReader.tableExists(ct,
|
||||
assertTrue(MetaReader.tableExists(CT,
|
||||
Bytes.toString(HConstants.ROOT_TABLE_NAME)));
|
||||
}
|
||||
|
||||
|
@ -245,11 +244,11 @@ public class TestMetaReaderEditor {
|
|||
LOG.info("Started " + name);
|
||||
// Test get on non-existent region.
|
||||
Pair<HRegionInfo, ServerName> pair =
|
||||
MetaReader.getRegion(ct, Bytes.toBytes("nonexistent-region"));
|
||||
MetaReader.getRegion(CT, Bytes.toBytes("nonexistent-region"));
|
||||
assertNull(pair);
|
||||
// Test it works getting a region from meta/root.
|
||||
pair =
|
||||
MetaReader.getRegion(ct, HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
|
||||
MetaReader.getRegion(CT, HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
|
||||
assertEquals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName(),
|
||||
pair.getFirst().getEncodedName());
|
||||
LOG.info("Finished " + name);
|
||||
|
@ -273,8 +272,8 @@ public class TestMetaReaderEditor {
|
|||
|
||||
// Now make sure we only get the regions from 1 of the tables at a time
|
||||
|
||||
assertEquals(1, MetaReader.getTableRegions(ct, Bytes.toBytes(name)).size());
|
||||
assertEquals(1, MetaReader.getTableRegions(ct, greaterName).size());
|
||||
assertEquals(1, MetaReader.getTableRegions(CT, Bytes.toBytes(name)).size());
|
||||
assertEquals(1, MetaReader.getTableRegions(CT, greaterName).size());
|
||||
}
|
||||
|
||||
private static List<HRegionInfo> testGettingTableRegions(final CatalogTracker ct,
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.security.User;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdge;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
|
@ -79,7 +78,6 @@ public class TestWALObserver {
|
|||
private Configuration conf;
|
||||
private FileSystem fs;
|
||||
private Path dir;
|
||||
private MiniDFSCluster cluster;
|
||||
private Path hbaseRootDir;
|
||||
private Path oldLogDir;
|
||||
private Path logDir;
|
||||
|
@ -136,8 +134,6 @@ public class TestWALObserver {
|
|||
|
||||
HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(TEST_TABLE));
|
||||
final HTableDescriptor htd = createBasic3FamilyHTD(Bytes.toString(TEST_TABLE));
|
||||
HRegion region2 = HRegion.createHRegion(hri,
|
||||
hbaseRootDir, this.conf, htd);
|
||||
|
||||
Path basedir = new Path(this.hbaseRootDir, Bytes.toString(TEST_TABLE));
|
||||
deleteDir(basedir);
|
||||
|
@ -236,10 +232,6 @@ public class TestWALObserver {
|
|||
|
||||
final Configuration newConf = HBaseConfiguration.create(this.conf);
|
||||
|
||||
HRegion region2 = HRegion.createHRegion(hri,
|
||||
hbaseRootDir, newConf,htd);
|
||||
|
||||
|
||||
//HLog wal = new HLog(this.fs, this.dir, this.oldLogDir, this.conf);
|
||||
HLog wal = createWAL(this.conf);
|
||||
//Put p = creatPutWith2Families(TEST_ROW);
|
||||
|
|
|
@ -23,9 +23,11 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.RegionTransition;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
|
@ -33,6 +35,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.util.Writables;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
|
@ -71,12 +74,13 @@ public class Mocking {
|
|||
* @param sn Name of the regionserver doing the 'opening'
|
||||
* @param hri Region we're 'opening'.
|
||||
* @throws KeeperException
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
static void fakeRegionServerRegionOpenInZK(final ZooKeeperWatcher w,
|
||||
final ServerName sn, final HRegionInfo hri)
|
||||
throws KeeperException {
|
||||
throws KeeperException, DeserializationException {
|
||||
// Wait till we see the OFFLINE zk node before we proceed.
|
||||
while (!ZKAssign.verifyRegionState(w, hri, EventType.M_ZK_REGION_OFFLINE)) {
|
||||
while (!verifyRegionState(w, hri, EventType.M_ZK_REGION_OFFLINE)) {
|
||||
Threads.sleep(1);
|
||||
}
|
||||
// Get current versionid else will fail on transition from OFFLINE to OPENING below
|
||||
|
@ -94,4 +98,40 @@ public class Mocking {
|
|||
// We should be done now. The master open handler will notice the
|
||||
// transition and remove this regions znode.
|
||||
}
|
||||
|
||||
/**
|
||||
* Verifies that the specified region is in the specified state in ZooKeeper.
|
||||
* <p>
|
||||
* Returns true if region is in transition and in the specified state in
|
||||
* ZooKeeper. Returns false if the region does not exist in ZK or is in
|
||||
* a different state.
|
||||
* <p>
|
||||
* Method synchronizes() with ZK so will yield an up-to-date result but is
|
||||
* a slow read.
|
||||
* @param zkw
|
||||
* @param region
|
||||
* @param expectedState
|
||||
* @return true if region exists and is in expected state
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
static boolean verifyRegionState(ZooKeeperWatcher zkw, HRegionInfo region, EventType expectedState)
|
||||
throws KeeperException, DeserializationException {
|
||||
String encoded = region.getEncodedName();
|
||||
|
||||
String node = ZKAssign.getNodeName(zkw, encoded);
|
||||
zkw.sync(node);
|
||||
|
||||
// Read existing data of the node
|
||||
byte [] existingBytes = null;
|
||||
try {
|
||||
existingBytes = ZKUtil.getDataAndWatch(zkw, node);
|
||||
} catch (KeeperException.NoNodeException nne) {
|
||||
return false;
|
||||
} catch (KeeperException e) {
|
||||
throw e;
|
||||
}
|
||||
if (existingBytes == null) return false;
|
||||
RegionTransition rt = RegionTransition.parseFrom(existingBytes);
|
||||
return rt.getEventType().equals(expectedState);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,23 +29,24 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HServerLoad;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.RegionTransition;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.client.ClientProtocol;
|
||||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.executor.RegionTransitionData;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
|
||||
import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
|
@ -61,10 +62,9 @@ import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
|
|||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.zookeeper.KeeperException.NodeExistsException;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
|
@ -115,6 +115,7 @@ public class TestAssignmentManager {
|
|||
// If abort is called, be sure to fail the test (don't just swallow it
|
||||
// silently as is mockito default).
|
||||
this.server = Mockito.mock(Server.class);
|
||||
Mockito.when(server.getServerName()).thenReturn(new ServerName("master,1,1"));
|
||||
Mockito.when(server.getConfiguration()).thenReturn(HTU.getConfiguration());
|
||||
this.watcher =
|
||||
new ZooKeeperWatcher(HTU.getConfiguration(), "mockedServer", this.server, true);
|
||||
|
@ -159,10 +160,11 @@ public class TestAssignmentManager {
|
|||
* @throws IOException
|
||||
* @throws KeeperException
|
||||
* @throws InterruptedException
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
@Test(timeout = 5000)
|
||||
public void testBalanceOnMasterFailoverScenarioWithOpenedNode()
|
||||
throws IOException, KeeperException, InterruptedException, ServiceException {
|
||||
throws IOException, KeeperException, InterruptedException, ServiceException, DeserializationException {
|
||||
AssignmentManagerWithExtrasForTesting am =
|
||||
setUpMockedAssignmentManager(this.server, this.serverManager);
|
||||
try {
|
||||
|
@ -177,7 +179,7 @@ public class TestAssignmentManager {
|
|||
int versionid =
|
||||
ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
|
||||
assertNotSame(versionid, -1);
|
||||
while (!ZKAssign.verifyRegionState(this.watcher, REGIONINFO,
|
||||
while (!Mocking.verifyRegionState(this.watcher, REGIONINFO,
|
||||
EventType.M_ZK_REGION_OFFLINE)) {
|
||||
Threads.sleep(1);
|
||||
}
|
||||
|
@ -205,7 +207,7 @@ public class TestAssignmentManager {
|
|||
|
||||
@Test(timeout = 5000)
|
||||
public void testBalanceOnMasterFailoverScenarioWithClosedNode()
|
||||
throws IOException, KeeperException, InterruptedException, ServiceException {
|
||||
throws IOException, KeeperException, InterruptedException, ServiceException, DeserializationException {
|
||||
AssignmentManagerWithExtrasForTesting am =
|
||||
setUpMockedAssignmentManager(this.server, this.serverManager);
|
||||
try {
|
||||
|
@ -221,7 +223,7 @@ public class TestAssignmentManager {
|
|||
ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
|
||||
assertNotSame(versionid, -1);
|
||||
am.gate.set(false);
|
||||
while (!ZKAssign.verifyRegionState(this.watcher, REGIONINFO,
|
||||
while (!Mocking.verifyRegionState(this.watcher, REGIONINFO,
|
||||
EventType.M_ZK_REGION_OFFLINE)) {
|
||||
Threads.sleep(1);
|
||||
}
|
||||
|
@ -249,7 +251,7 @@ public class TestAssignmentManager {
|
|||
|
||||
@Test(timeout = 5000)
|
||||
public void testBalanceOnMasterFailoverScenarioWithOfflineNode()
|
||||
throws IOException, KeeperException, InterruptedException, ServiceException {
|
||||
throws IOException, KeeperException, InterruptedException, ServiceException, DeserializationException {
|
||||
AssignmentManagerWithExtrasForTesting am =
|
||||
setUpMockedAssignmentManager(this.server, this.serverManager);
|
||||
try {
|
||||
|
@ -264,7 +266,7 @@ public class TestAssignmentManager {
|
|||
int versionid =
|
||||
ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
|
||||
assertNotSame(versionid, -1);
|
||||
while (!ZKAssign.verifyRegionState(this.watcher, REGIONINFO,
|
||||
while (!Mocking.verifyRegionState(this.watcher, REGIONINFO,
|
||||
EventType.M_ZK_REGION_OFFLINE)) {
|
||||
Threads.sleep(1);
|
||||
}
|
||||
|
@ -306,10 +308,11 @@ public class TestAssignmentManager {
|
|||
* from one server to another mocking regionserver responding over zk.
|
||||
* @throws IOException
|
||||
* @throws KeeperException
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
@Test
|
||||
public void testBalance()
|
||||
throws IOException, KeeperException {
|
||||
throws IOException, KeeperException, DeserializationException {
|
||||
// Create and startup an executor. This is used by AssignmentManager
|
||||
// handling zk callbacks.
|
||||
ExecutorService executor = startupMasterExecutor("testBalanceExecutor");
|
||||
|
@ -345,7 +348,7 @@ public class TestAssignmentManager {
|
|||
// balancer. The zk node will be OFFLINE waiting for regionserver to
|
||||
// transition it through OPENING, OPENED. Wait till we see the OFFLINE
|
||||
// zk node before we proceed.
|
||||
while (!ZKAssign.verifyRegionState(this.watcher, REGIONINFO, EventType.M_ZK_REGION_OFFLINE)) {
|
||||
while (!Mocking.verifyRegionState(this.watcher, REGIONINFO, EventType.M_ZK_REGION_OFFLINE)) {
|
||||
Threads.sleep(1);
|
||||
}
|
||||
// Get current versionid else will fail on transition from OFFLINE to OPENING below
|
||||
|
@ -541,12 +544,12 @@ public class TestAssignmentManager {
|
|||
private static int createNodeSplitting(final ZooKeeperWatcher zkw,
|
||||
final HRegionInfo region, final ServerName serverName)
|
||||
throws KeeperException, IOException {
|
||||
RegionTransitionData data =
|
||||
new RegionTransitionData(EventType.RS_ZK_REGION_SPLITTING,
|
||||
RegionTransition rt =
|
||||
RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_SPLITTING,
|
||||
region.getRegionName(), serverName);
|
||||
|
||||
String node = ZKAssign.getNodeName(zkw, region.getEncodedName());
|
||||
if (!ZKUtil.createEphemeralNodeAndWatch(zkw, node, data.getBytes())) {
|
||||
if (!ZKUtil.createEphemeralNodeAndWatch(zkw, node, rt.toByteArray())) {
|
||||
throw new IOException("Failed create of ephemeral " + node);
|
||||
}
|
||||
// Transition node from SPLITTING to SPLITTING and pick up version so we
|
||||
|
@ -650,12 +653,12 @@ public class TestAssignmentManager {
|
|||
deadServers);
|
||||
}
|
||||
@Override
|
||||
void processRegionsInTransition(final RegionTransitionData data,
|
||||
void processRegionsInTransition(final RegionTransition rt,
|
||||
final HRegionInfo regionInfo,
|
||||
final Map<ServerName, List<Pair<HRegionInfo, Result>>> deadServers,
|
||||
final int expectedVersion) throws KeeperException {
|
||||
while (this.gate.get()) Threads.sleep(1);
|
||||
super.processRegionsInTransition(data, regionInfo, deadServers, expectedVersion);
|
||||
super.processRegionsInTransition(rt, regionInfo, deadServers, expectedVersion);
|
||||
}
|
||||
|
||||
/** reset the watcher */
|
||||
|
|
|
@ -19,7 +19,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.*;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.*;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
@ -57,7 +57,6 @@ import org.apache.hadoop.hbase.util.FSUtils;
|
|||
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.log4j.Level;
|
||||
import org.apache.log4j.Logger;
|
||||
|
@ -73,7 +72,7 @@ public class TestDistributedLogSplitting {
|
|||
Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
|
||||
}
|
||||
|
||||
// Start a cluster with 2 masters and 3 regionservers
|
||||
// Start a cluster with 2 masters and 6 regionservers
|
||||
final int NUM_MASTERS = 2;
|
||||
final int NUM_RS = 6;
|
||||
|
||||
|
@ -83,7 +82,7 @@ public class TestDistributedLogSplitting {
|
|||
HBaseTestingUtility TEST_UTIL;
|
||||
|
||||
private void startCluster(int num_rs) throws Exception{
|
||||
ZKSplitLog.Counters.resetCounters();
|
||||
SplitLogCounters.resetCounters();
|
||||
LOG.info("Starting cluster");
|
||||
conf = HBaseConfiguration.create();
|
||||
conf.getLong("hbase.splitlog.max.resubmit", 0);
|
||||
|
@ -108,48 +107,6 @@ public class TestDistributedLogSplitting {
|
|||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
@Test (timeout=300000)
|
||||
public void testThreeRSAbort() throws Exception {
|
||||
LOG.info("testThreeRSAbort");
|
||||
final int NUM_REGIONS_TO_CREATE = 40;
|
||||
final int NUM_ROWS_PER_REGION = 100;
|
||||
|
||||
startCluster(NUM_RS); // NUM_RS=6.
|
||||
|
||||
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
|
||||
"distributed log splitting test", null);
|
||||
|
||||
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
|
||||
populateDataInTable(NUM_ROWS_PER_REGION, "family");
|
||||
|
||||
|
||||
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
|
||||
assertEquals(NUM_RS, rsts.size());
|
||||
rsts.get(0).getRegionServer().abort("testing");
|
||||
rsts.get(1).getRegionServer().abort("testing");
|
||||
rsts.get(2).getRegionServer().abort("testing");
|
||||
|
||||
long start = EnvironmentEdgeManager.currentTimeMillis();
|
||||
while (cluster.getLiveRegionServerThreads().size() > (NUM_RS - 3)) {
|
||||
if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
|
||||
assertTrue(false);
|
||||
}
|
||||
Thread.sleep(200);
|
||||
}
|
||||
|
||||
start = EnvironmentEdgeManager.currentTimeMillis();
|
||||
while (getAllOnlineRegions(cluster).size() < (NUM_REGIONS_TO_CREATE + 2)) {
|
||||
if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
|
||||
assertTrue(false);
|
||||
}
|
||||
Thread.sleep(200);
|
||||
}
|
||||
|
||||
assertEquals(NUM_REGIONS_TO_CREATE * NUM_ROWS_PER_REGION,
|
||||
TEST_UTIL.countRows(ht));
|
||||
ht.close();
|
||||
}
|
||||
|
||||
@Test (timeout=300000)
|
||||
public void testRecoveredEdits() throws Exception {
|
||||
LOG.info("testRecoveredEdits");
|
||||
|
@ -196,8 +153,7 @@ public class TestDistributedLogSplitting {
|
|||
|
||||
Path tdir = HTableDescriptor.getTableDir(rootdir, table);
|
||||
Path editsdir =
|
||||
HLog.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir,
|
||||
hri.getEncodedName()));
|
||||
HLog.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName()));
|
||||
LOG.debug("checking edits dir " + editsdir);
|
||||
FileStatus[] files = fs.listStatus(editsdir);
|
||||
assertEquals(1, files.length);
|
||||
|
@ -272,6 +228,50 @@ public class TestDistributedLogSplitting {
|
|||
"tot_wkr_preempt_task");
|
||||
}
|
||||
|
||||
@Test (timeout=300000)
|
||||
public void testThreeRSAbort() throws Exception {
|
||||
LOG.info("testThreeRSAbort");
|
||||
final int NUM_REGIONS_TO_CREATE = 40;
|
||||
final int NUM_ROWS_PER_REGION = 100;
|
||||
|
||||
startCluster(NUM_RS); // NUM_RS=6.
|
||||
|
||||
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
|
||||
"distributed log splitting test", null);
|
||||
|
||||
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
|
||||
populateDataInTable(NUM_ROWS_PER_REGION, "family");
|
||||
|
||||
|
||||
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
|
||||
assertEquals(NUM_RS, rsts.size());
|
||||
rsts.get(0).getRegionServer().abort("testing");
|
||||
rsts.get(1).getRegionServer().abort("testing");
|
||||
rsts.get(2).getRegionServer().abort("testing");
|
||||
|
||||
long start = EnvironmentEdgeManager.currentTimeMillis();
|
||||
while (cluster.getLiveRegionServerThreads().size() > (NUM_RS - 3)) {
|
||||
if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
|
||||
assertTrue(false);
|
||||
}
|
||||
Thread.sleep(200);
|
||||
}
|
||||
|
||||
start = EnvironmentEdgeManager.currentTimeMillis();
|
||||
while (getAllOnlineRegions(cluster).size() < (NUM_REGIONS_TO_CREATE + 2)) {
|
||||
if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
|
||||
assertTrue("Timedout", false);
|
||||
}
|
||||
Thread.sleep(200);
|
||||
}
|
||||
|
||||
assertEquals(NUM_REGIONS_TO_CREATE * NUM_ROWS_PER_REGION,
|
||||
TEST_UTIL.countRows(ht));
|
||||
ht.close();
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Test(timeout=25000)
|
||||
public void testDelayedDeleteOnFailure() throws Exception {
|
||||
LOG.info("testDelayedDeleteOnFailure");
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.executor.RegionTransitionData;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
|
@ -57,167 +56,6 @@ import org.junit.experimental.categories.Category;
|
|||
public class TestMasterFailover {
|
||||
private static final Log LOG = LogFactory.getLog(TestMasterFailover.class);
|
||||
|
||||
@Test (timeout=180000)
|
||||
public void testShouldCheckMasterFailOverWhenMETAIsInOpenedState()
|
||||
throws Exception {
|
||||
LOG.info("Starting testShouldCheckMasterFailOverWhenMETAIsInOpenedState");
|
||||
final int NUM_MASTERS = 1;
|
||||
final int NUM_RS = 2;
|
||||
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
conf.setInt("hbase.master.assignment.timeoutmonitor.period", 2000);
|
||||
conf.setInt("hbase.master.assignment.timeoutmonitor.timeout", 8000);
|
||||
// Start the cluster
|
||||
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
|
||||
|
||||
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
|
||||
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
|
||||
|
||||
// Find regionserver carrying meta.
|
||||
List<RegionServerThread> regionServerThreads =
|
||||
cluster.getRegionServerThreads();
|
||||
int count = -1;
|
||||
HRegion metaRegion = null;
|
||||
for (RegionServerThread regionServerThread : regionServerThreads) {
|
||||
HRegionServer regionServer = regionServerThread.getRegionServer();
|
||||
metaRegion = regionServer.getOnlineRegion(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
|
||||
count++;
|
||||
regionServer.abort("");
|
||||
if (null != metaRegion) break;
|
||||
}
|
||||
HRegionServer regionServer = cluster.getRegionServer(count);
|
||||
|
||||
TEST_UTIL.shutdownMiniHBaseCluster();
|
||||
|
||||
// Create a ZKW to use in the test
|
||||
ZooKeeperWatcher zkw =
|
||||
HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
|
||||
metaRegion, regionServer.getServerName());
|
||||
|
||||
LOG.info("Staring cluster for second time");
|
||||
TEST_UTIL.startMiniHBaseCluster(NUM_MASTERS, NUM_RS);
|
||||
|
||||
// Failover should be completed, now wait for no RIT
|
||||
log("Waiting for no more RIT");
|
||||
ZKAssign.blockUntilNoRIT(zkw);
|
||||
|
||||
zkw.close();
|
||||
// Stop the cluster
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple test of master failover.
|
||||
* <p>
|
||||
* Starts with three masters. Kills a backup master. Then kills the active
|
||||
* master. Ensures the final master becomes active and we can still contact
|
||||
* the cluster.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test (timeout=240000)
|
||||
public void testSimpleMasterFailover() throws Exception {
|
||||
|
||||
final int NUM_MASTERS = 3;
|
||||
final int NUM_RS = 3;
|
||||
|
||||
// Create config to use for this cluster
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
|
||||
// Start the cluster
|
||||
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
|
||||
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
|
||||
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
|
||||
|
||||
// get all the master threads
|
||||
List<MasterThread> masterThreads = cluster.getMasterThreads();
|
||||
|
||||
// wait for each to come online
|
||||
for (MasterThread mt : masterThreads) {
|
||||
assertTrue(mt.isAlive());
|
||||
}
|
||||
|
||||
// verify only one is the active master and we have right number
|
||||
int numActive = 0;
|
||||
int activeIndex = -1;
|
||||
ServerName activeName = null;
|
||||
HMaster active = null;
|
||||
for (int i = 0; i < masterThreads.size(); i++) {
|
||||
if (masterThreads.get(i).getMaster().isActiveMaster()) {
|
||||
numActive++;
|
||||
activeIndex = i;
|
||||
active = masterThreads.get(activeIndex).getMaster();
|
||||
activeName = active.getServerName();
|
||||
}
|
||||
}
|
||||
assertEquals(1, numActive);
|
||||
assertEquals(NUM_MASTERS, masterThreads.size());
|
||||
LOG.info("Active master " + activeName);
|
||||
|
||||
// Check that ClusterStatus reports the correct active and backup masters
|
||||
assertNotNull(active);
|
||||
ClusterStatus status = active.getClusterStatus();
|
||||
assertTrue(status.getMaster().equals(activeName));
|
||||
assertEquals(2, status.getBackupMastersSize());
|
||||
assertEquals(2, status.getBackupMasters().size());
|
||||
|
||||
// attempt to stop one of the inactive masters
|
||||
int backupIndex = (activeIndex == 0 ? 1 : activeIndex - 1);
|
||||
HMaster master = cluster.getMaster(backupIndex);
|
||||
LOG.debug("\n\nStopping a backup master: " + master.getServerName() + "\n");
|
||||
cluster.stopMaster(backupIndex, false);
|
||||
cluster.waitOnMaster(backupIndex);
|
||||
|
||||
// Verify still one active master and it's the same
|
||||
for (int i = 0; i < masterThreads.size(); i++) {
|
||||
if (masterThreads.get(i).getMaster().isActiveMaster()) {
|
||||
assertTrue(activeName.equals(masterThreads.get(i).getMaster().getServerName()));
|
||||
activeIndex = i;
|
||||
active = masterThreads.get(activeIndex).getMaster();
|
||||
}
|
||||
}
|
||||
assertEquals(1, numActive);
|
||||
assertEquals(2, masterThreads.size());
|
||||
int rsCount = masterThreads.get(activeIndex).getMaster().getClusterStatus().getServersSize();
|
||||
LOG.info("Active master " + active.getServerName() + " managing " + rsCount + " regions servers");
|
||||
assertEquals(3, rsCount);
|
||||
|
||||
// Check that ClusterStatus reports the correct active and backup masters
|
||||
assertNotNull(active);
|
||||
status = active.getClusterStatus();
|
||||
assertTrue(status.getMaster().equals(activeName));
|
||||
assertEquals(1, status.getBackupMastersSize());
|
||||
assertEquals(1, status.getBackupMasters().size());
|
||||
|
||||
// kill the active master
|
||||
LOG.debug("\n\nStopping the active master " + active.getServerName() + "\n");
|
||||
cluster.stopMaster(activeIndex, false);
|
||||
cluster.waitOnMaster(activeIndex);
|
||||
|
||||
// wait for an active master to show up and be ready
|
||||
assertTrue(cluster.waitForActiveAndReadyMaster());
|
||||
|
||||
LOG.debug("\n\nVerifying backup master is now active\n");
|
||||
// should only have one master now
|
||||
assertEquals(1, masterThreads.size());
|
||||
|
||||
// and he should be active
|
||||
active = masterThreads.get(0).getMaster();
|
||||
assertNotNull(active);
|
||||
status = active.getClusterStatus();
|
||||
ServerName mastername = status.getMaster();
|
||||
assertTrue(mastername.equals(active.getServerName()));
|
||||
assertTrue(active.isActiveMaster());
|
||||
assertEquals(0, status.getBackupMastersSize());
|
||||
assertEquals(0, status.getBackupMasters().size());
|
||||
int rss = status.getServersSize();
|
||||
LOG.info("Active master " + mastername.getServerName() + " managing " +
|
||||
rss + " region servers");
|
||||
assertEquals(3, rss);
|
||||
|
||||
// Stop the cluster
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Complex test of master failover that tests as many permutations of the
|
||||
* different possible states that regions in transition could be in within ZK.
|
||||
|
@ -475,8 +313,9 @@ public class TestMasterFailover {
|
|||
ZKAssign.createNodeOffline(zkw, region, serverName);
|
||||
hrs.openRegion(region);
|
||||
while (true) {
|
||||
RegionTransitionData rtd = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
if (rtd != null && rtd.getEventType() == EventType.RS_ZK_REGION_OPENED) {
|
||||
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
RegionTransition rt = RegionTransition.parseFrom(bytes);
|
||||
if (rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_OPENED)) {
|
||||
break;
|
||||
}
|
||||
Thread.sleep(100);
|
||||
|
@ -488,8 +327,9 @@ public class TestMasterFailover {
|
|||
ZKAssign.createNodeOffline(zkw, region, serverName);
|
||||
hrs.openRegion(region);
|
||||
while (true) {
|
||||
RegionTransitionData rtd = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
if (rtd != null && rtd.getEventType() == EventType.RS_ZK_REGION_OPENED) {
|
||||
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
RegionTransition rt = RegionTransition.parseFrom(bytes);
|
||||
if (rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_OPENED)) {
|
||||
break;
|
||||
}
|
||||
Thread.sleep(100);
|
||||
|
@ -603,16 +443,14 @@ public class TestMasterFailover {
|
|||
final int NUM_MASTERS = 1;
|
||||
final int NUM_RS = 2;
|
||||
|
||||
// Create config to use for this cluster
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
// Create and start the cluster
|
||||
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
// Need to drop the timeout much lower
|
||||
conf.setInt("hbase.master.assignment.timeoutmonitor.period", 2000);
|
||||
conf.setInt("hbase.master.assignment.timeoutmonitor.timeout", 4000);
|
||||
conf.setInt("hbase.master.wait.on.regionservers.mintostart", 1);
|
||||
conf.setInt("hbase.master.wait.on.regionservers.maxtostart", 2);
|
||||
|
||||
// Create and start the cluster
|
||||
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
|
||||
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
|
||||
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
|
||||
log("Cluster started");
|
||||
|
@ -835,8 +673,9 @@ public class TestMasterFailover {
|
|||
ZKAssign.createNodeOffline(zkw, region, deadServerName);
|
||||
hrsDead.openRegion(region);
|
||||
while (true) {
|
||||
RegionTransitionData rtd = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
if (rtd != null && rtd.getEventType() == EventType.RS_ZK_REGION_OPENED) {
|
||||
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
RegionTransition rt = RegionTransition.parseFrom(bytes);
|
||||
if (rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_OPENED)) {
|
||||
break;
|
||||
}
|
||||
Thread.sleep(100);
|
||||
|
@ -850,8 +689,9 @@ public class TestMasterFailover {
|
|||
ZKAssign.createNodeOffline(zkw, region, deadServerName);
|
||||
hrsDead.openRegion(region);
|
||||
while (true) {
|
||||
RegionTransitionData rtd = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
if (rtd != null && rtd.getEventType() == EventType.RS_ZK_REGION_OPENED) {
|
||||
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
RegionTransition rt = RegionTransition.parseFrom(bytes);
|
||||
if (rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_OPENED)) {
|
||||
break;
|
||||
}
|
||||
Thread.sleep(100);
|
||||
|
@ -869,9 +709,11 @@ public class TestMasterFailover {
|
|||
ZKAssign.createNodeOffline(zkw, region, deadServerName);
|
||||
hrsDead.openRegion(region);
|
||||
while (true) {
|
||||
RegionTransitionData rtd = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
if (rtd != null && rtd.getEventType() == EventType.RS_ZK_REGION_OPENED) {
|
||||
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
RegionTransition rt = RegionTransition.parseFrom(bytes);
|
||||
if (rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_OPENED)) {
|
||||
ZKAssign.deleteOpenedNode(zkw, region.getEncodedName());
|
||||
LOG.debug("DELETED " + rt);
|
||||
break;
|
||||
}
|
||||
Thread.sleep(100);
|
||||
|
@ -885,8 +727,9 @@ public class TestMasterFailover {
|
|||
ZKAssign.createNodeOffline(zkw, region, deadServerName);
|
||||
hrsDead.openRegion(region);
|
||||
while (true) {
|
||||
RegionTransitionData rtd = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
if (rtd != null && rtd.getEventType() == EventType.RS_ZK_REGION_OPENED) {
|
||||
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
RegionTransition rt = RegionTransition.parseFrom(bytes);
|
||||
if (rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_OPENED)) {
|
||||
ZKAssign.deleteOpenedNode(zkw, region.getEncodedName());
|
||||
break;
|
||||
}
|
||||
|
@ -984,7 +827,8 @@ public class TestMasterFailover {
|
|||
|
||||
// Now, everything that should be online should be online
|
||||
for (HRegionInfo hri : regionsThatShouldBeOnline) {
|
||||
assertTrue("region=" + hri.getRegionNameAsString(), onlineRegions.contains(hri));
|
||||
assertTrue("region=" + hri.getRegionNameAsString() + ", " + onlineRegions.toString(),
|
||||
onlineRegions.contains(hri));
|
||||
}
|
||||
|
||||
// Everything that should be offline should not be online
|
||||
|
@ -1018,6 +862,167 @@ public class TestMasterFailover {
|
|||
LOG.info("\n\n" + string + " \n\n");
|
||||
}
|
||||
|
||||
@Test (timeout=180000)
|
||||
public void testShouldCheckMasterFailOverWhenMETAIsInOpenedState()
|
||||
throws Exception {
|
||||
LOG.info("Starting testShouldCheckMasterFailOverWhenMETAIsInOpenedState");
|
||||
final int NUM_MASTERS = 1;
|
||||
final int NUM_RS = 2;
|
||||
|
||||
// Start the cluster
|
||||
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
conf.setInt("hbase.master.assignment.timeoutmonitor.period", 2000);
|
||||
conf.setInt("hbase.master.assignment.timeoutmonitor.timeout", 8000);
|
||||
conf.setInt("hbase.master.info.port", -1);
|
||||
|
||||
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
|
||||
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
|
||||
|
||||
// Find regionserver carrying meta.
|
||||
List<RegionServerThread> regionServerThreads =
|
||||
cluster.getRegionServerThreads();
|
||||
int count = -1;
|
||||
HRegion metaRegion = null;
|
||||
for (RegionServerThread regionServerThread : regionServerThreads) {
|
||||
HRegionServer regionServer = regionServerThread.getRegionServer();
|
||||
metaRegion = regionServer.getOnlineRegion(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
|
||||
count++;
|
||||
regionServer.abort("");
|
||||
if (null != metaRegion) break;
|
||||
}
|
||||
HRegionServer regionServer = cluster.getRegionServer(count);
|
||||
|
||||
TEST_UTIL.shutdownMiniHBaseCluster();
|
||||
|
||||
// Create a ZKW to use in the test
|
||||
ZooKeeperWatcher zkw =
|
||||
HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
|
||||
metaRegion, regionServer.getServerName());
|
||||
|
||||
LOG.info("Staring cluster for second time");
|
||||
TEST_UTIL.startMiniHBaseCluster(NUM_MASTERS, NUM_RS);
|
||||
|
||||
// Failover should be completed, now wait for no RIT
|
||||
log("Waiting for no more RIT");
|
||||
ZKAssign.blockUntilNoRIT(zkw);
|
||||
|
||||
zkw.close();
|
||||
// Stop the cluster
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple test of master failover.
|
||||
* <p>
|
||||
* Starts with three masters. Kills a backup master. Then kills the active
|
||||
* master. Ensures the final master becomes active and we can still contact
|
||||
* the cluster.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test (timeout=240000)
|
||||
public void testSimpleMasterFailover() throws Exception {
|
||||
|
||||
final int NUM_MASTERS = 3;
|
||||
final int NUM_RS = 3;
|
||||
|
||||
// Start the cluster
|
||||
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
|
||||
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
|
||||
|
||||
// get all the master threads
|
||||
List<MasterThread> masterThreads = cluster.getMasterThreads();
|
||||
|
||||
// wait for each to come online
|
||||
for (MasterThread mt : masterThreads) {
|
||||
assertTrue(mt.isAlive());
|
||||
}
|
||||
|
||||
// verify only one is the active master and we have right number
|
||||
int numActive = 0;
|
||||
int activeIndex = -1;
|
||||
ServerName activeName = null;
|
||||
HMaster active = null;
|
||||
for (int i = 0; i < masterThreads.size(); i++) {
|
||||
if (masterThreads.get(i).getMaster().isActiveMaster()) {
|
||||
numActive++;
|
||||
activeIndex = i;
|
||||
active = masterThreads.get(activeIndex).getMaster();
|
||||
activeName = active.getServerName();
|
||||
}
|
||||
}
|
||||
assertEquals(1, numActive);
|
||||
assertEquals(NUM_MASTERS, masterThreads.size());
|
||||
LOG.info("Active master " + activeName);
|
||||
|
||||
// Check that ClusterStatus reports the correct active and backup masters
|
||||
assertNotNull(active);
|
||||
ClusterStatus status = active.getClusterStatus();
|
||||
assertTrue(status.getMaster().equals(activeName));
|
||||
assertEquals(2, status.getBackupMastersSize());
|
||||
assertEquals(2, status.getBackupMasters().size());
|
||||
|
||||
// attempt to stop one of the inactive masters
|
||||
int backupIndex = (activeIndex == 0 ? 1 : activeIndex - 1);
|
||||
HMaster master = cluster.getMaster(backupIndex);
|
||||
LOG.debug("\n\nStopping a backup master: " + master.getServerName() + "\n");
|
||||
cluster.stopMaster(backupIndex, false);
|
||||
cluster.waitOnMaster(backupIndex);
|
||||
|
||||
// Verify still one active master and it's the same
|
||||
for (int i = 0; i < masterThreads.size(); i++) {
|
||||
if (masterThreads.get(i).getMaster().isActiveMaster()) {
|
||||
assertTrue(activeName.equals(masterThreads.get(i).getMaster().getServerName()));
|
||||
activeIndex = i;
|
||||
active = masterThreads.get(activeIndex).getMaster();
|
||||
}
|
||||
}
|
||||
assertEquals(1, numActive);
|
||||
assertEquals(2, masterThreads.size());
|
||||
int rsCount = masterThreads.get(activeIndex).getMaster().getClusterStatus().getServersSize();
|
||||
LOG.info("Active master " + active.getServerName() + " managing " + rsCount + " regions servers");
|
||||
assertEquals(3, rsCount);
|
||||
|
||||
// Check that ClusterStatus reports the correct active and backup masters
|
||||
assertNotNull(active);
|
||||
status = active.getClusterStatus();
|
||||
assertTrue(status.getMaster().equals(activeName));
|
||||
assertEquals(1, status.getBackupMastersSize());
|
||||
assertEquals(1, status.getBackupMasters().size());
|
||||
|
||||
// kill the active master
|
||||
LOG.debug("\n\nStopping the active master " + active.getServerName() + "\n");
|
||||
cluster.stopMaster(activeIndex, false);
|
||||
cluster.waitOnMaster(activeIndex);
|
||||
|
||||
// wait for an active master to show up and be ready
|
||||
assertTrue(cluster.waitForActiveAndReadyMaster());
|
||||
|
||||
LOG.debug("\n\nVerifying backup master is now active\n");
|
||||
// should only have one master now
|
||||
assertEquals(1, masterThreads.size());
|
||||
|
||||
// and he should be active
|
||||
active = masterThreads.get(0).getMaster();
|
||||
assertNotNull(active);
|
||||
status = active.getClusterStatus();
|
||||
ServerName mastername = status.getMaster();
|
||||
assertTrue(mastername.equals(active.getServerName()));
|
||||
assertTrue(active.isActiveMaster());
|
||||
assertEquals(0, status.getBackupMastersSize());
|
||||
assertEquals(0, status.getBackupMasters().size());
|
||||
int rss = status.getServersSize();
|
||||
LOG.info("Active master " + mastername.getServerName() + " managing " +
|
||||
rss + " region servers");
|
||||
assertEquals(3, rss);
|
||||
|
||||
// Stop the cluster
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
|
||||
@org.junit.Rule
|
||||
public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
|
||||
new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
|
||||
|
|
|
@ -27,6 +27,7 @@ import java.net.UnknownHostException;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
|
@ -227,10 +228,11 @@ public class TestMasterNoCluster {
|
|||
* @throws IOException
|
||||
* @throws KeeperException
|
||||
* @throws InterruptedException
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
@Test
|
||||
public void testCatalogDeploys()
|
||||
throws IOException, KeeperException, InterruptedException {
|
||||
throws IOException, KeeperException, InterruptedException, DeserializationException {
|
||||
final Configuration conf = TESTUTIL.getConfiguration();
|
||||
final long now = System.currentTimeMillis();
|
||||
// Name for our single mocked up regionserver.
|
||||
|
|
|
@ -19,27 +19,43 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.*;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.resetCounters;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_get_data_nonode;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_heartbeat;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_log_split_batch_success;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_node_create_queued;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_node_create_result;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_orphan_task_acquired;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_rescan;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_rescan_deleted;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_dead_server_task;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_failed;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_threshold_reached;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_unassigned;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_task_deleted;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SplitLogTask;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.master.SplitLogManager.Task;
|
||||
import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
|
||||
import org.apache.hadoop.hbase.regionserver.TestMasterAddressManager.NodeCreationListener;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog.TaskState;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.log4j.Level;
|
||||
|
@ -57,6 +73,7 @@ import org.junit.experimental.categories.Category;
|
|||
@Category(MediumTests.class)
|
||||
public class TestSplitLogManager {
|
||||
private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
|
||||
private final ServerName DUMMY_MASTER = new ServerName("dummy-master,1,1");
|
||||
static {
|
||||
Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
|
||||
}
|
||||
|
@ -125,8 +142,7 @@ public class TestSplitLogManager {
|
|||
public long eval();
|
||||
}
|
||||
|
||||
private void waitForCounter(final AtomicLong ctr, long oldval, long newval,
|
||||
long timems) {
|
||||
private void waitForCounter(final AtomicLong ctr, long oldval, long newval, long timems) {
|
||||
Expr e = new Expr() {
|
||||
public long eval() {
|
||||
return ctr.get();
|
||||
|
@ -180,16 +196,16 @@ public class TestSplitLogManager {
|
|||
@Test
|
||||
public void testTaskCreation() throws Exception {
|
||||
LOG.info("TestTaskCreation - test the creation of a task in zk");
|
||||
|
||||
slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
|
||||
slm.finishInitialization();
|
||||
TaskBatch batch = new TaskBatch();
|
||||
|
||||
String tasknode = submitTaskAndWait(batch, "foo/1");
|
||||
|
||||
byte[] data = ZKUtil.getData(zkw, tasknode);
|
||||
LOG.info("Task node created " + new String(data));
|
||||
assertTrue(TaskState.TASK_UNASSIGNED.equals(data, "dummy-master"));
|
||||
SplitLogTask slt = SplitLogTask.parseFrom(data);
|
||||
LOG.info("Task node created " + slt.toString());
|
||||
assertTrue(slt.isUnassigned(DUMMY_MASTER));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -197,8 +213,8 @@ public class TestSplitLogManager {
|
|||
LOG.info("TestOrphanTaskAcquisition");
|
||||
|
||||
String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
|
||||
zkw.getRecoverableZooKeeper().create(tasknode,
|
||||
TaskState.TASK_OWNED.get("dummy-worker"), Ids.OPEN_ACL_UNSAFE,
|
||||
SplitLogTask slt = new SplitLogTask.Owned(DUMMY_MASTER);
|
||||
zkw.getRecoverableZooKeeper().create(tasknode, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
|
||||
int to = 1000;
|
||||
|
@ -207,7 +223,7 @@ public class TestSplitLogManager {
|
|||
to = to + 2 * 100;
|
||||
|
||||
|
||||
slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
|
||||
slm.finishInitialization();
|
||||
waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, 100);
|
||||
Task task = slm.findOrCreateOrphanTask(tasknode);
|
||||
|
@ -229,12 +245,12 @@ public class TestSplitLogManager {
|
|||
" startup");
|
||||
String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
|
||||
//create an unassigned orphan task
|
||||
zkw.getRecoverableZooKeeper().create(tasknode,
|
||||
TaskState.TASK_UNASSIGNED.get("dummy-worker"), Ids.OPEN_ACL_UNSAFE,
|
||||
SplitLogTask slt = new SplitLogTask.Unassigned(DUMMY_MASTER);
|
||||
zkw.getRecoverableZooKeeper().create(tasknode, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
int version = ZKUtil.checkExists(zkw, tasknode);
|
||||
|
||||
slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
|
||||
slm.finishInitialization();
|
||||
waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, 100);
|
||||
Task task = slm.findOrCreateOrphanTask(tasknode);
|
||||
|
@ -263,24 +279,29 @@ public class TestSplitLogManager {
|
|||
to = to + 2 * 100;
|
||||
|
||||
conf.setInt("hbase.splitlog.max.resubmit", 2);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
|
||||
slm.finishInitialization();
|
||||
TaskBatch batch = new TaskBatch();
|
||||
|
||||
String tasknode = submitTaskAndWait(batch, "foo/1");
|
||||
int version = ZKUtil.checkExists(zkw, tasknode);
|
||||
|
||||
ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
|
||||
final ServerName worker1 = new ServerName("worker1,1,1");
|
||||
final ServerName worker2 = new ServerName("worker2,1,1");
|
||||
final ServerName worker3 = new ServerName("worker3,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Owned(worker1);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
waitForCounter(tot_mgr_heartbeat, 0, 1, 1000);
|
||||
waitForCounter(tot_mgr_resubmit, 0, 1, to + EXTRA_TOLERANCE_MS);
|
||||
int version1 = ZKUtil.checkExists(zkw, tasknode);
|
||||
assertTrue(version1 > version);
|
||||
ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker2"));
|
||||
slt = new SplitLogTask.Owned(worker2);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
waitForCounter(tot_mgr_heartbeat, 1, 2, 1000);
|
||||
waitForCounter(tot_mgr_resubmit, 1, 2, to + 100);
|
||||
int version2 = ZKUtil.checkExists(zkw, tasknode);
|
||||
assertTrue(version2 > version1);
|
||||
ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker3"));
|
||||
slt = new SplitLogTask.Owned(worker3);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
waitForCounter(tot_mgr_heartbeat, 1, 2, 1000);
|
||||
waitForCounter(tot_mgr_resubmit_threshold_reached, 0, 1, to + EXTRA_TOLERANCE_MS);
|
||||
Thread.sleep(to + EXTRA_TOLERANCE_MS);
|
||||
|
@ -293,14 +314,15 @@ public class TestSplitLogManager {
|
|||
|
||||
conf.setInt("hbase.splitlog.manager.timeout", 1000);
|
||||
conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
|
||||
slm.finishInitialization();
|
||||
TaskBatch batch = new TaskBatch();
|
||||
|
||||
String tasknode = submitTaskAndWait(batch, "foo/1");
|
||||
int version = ZKUtil.checkExists(zkw, tasknode);
|
||||
|
||||
ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
|
||||
final ServerName worker1 = new ServerName("worker1,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Owned(worker1);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
waitForCounter(tot_mgr_heartbeat, 0, 1, 1000);
|
||||
waitForCounter(new Expr() {
|
||||
@Override
|
||||
|
@ -312,8 +334,8 @@ public class TestSplitLogManager {
|
|||
int version1 = ZKUtil.checkExists(zkw, tasknode);
|
||||
assertTrue(version1 > version);
|
||||
byte[] taskstate = ZKUtil.getData(zkw, tasknode);
|
||||
assertTrue(Arrays.equals(TaskState.TASK_UNASSIGNED.get("dummy-master"),
|
||||
taskstate));
|
||||
slt = SplitLogTask.parseFrom(taskstate);
|
||||
assertTrue(slt.isUnassigned(DUMMY_MASTER));
|
||||
|
||||
waitForCounter(tot_mgr_rescan_deleted, 0, 1, 1000);
|
||||
} else {
|
||||
|
@ -327,11 +349,13 @@ public class TestSplitLogManager {
|
|||
public void testTaskDone() throws Exception {
|
||||
LOG.info("TestTaskDone - cleanup task node once in DONE state");
|
||||
|
||||
slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
|
||||
slm.finishInitialization();
|
||||
TaskBatch batch = new TaskBatch();
|
||||
String tasknode = submitTaskAndWait(batch, "foo/1");
|
||||
ZKUtil.setData(zkw, tasknode, TaskState.TASK_DONE.get("worker"));
|
||||
final ServerName worker1 = new ServerName("worker1,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Done(worker1);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
synchronized (batch) {
|
||||
while (batch.installed != batch.done) {
|
||||
batch.wait();
|
||||
|
@ -346,12 +370,14 @@ public class TestSplitLogManager {
|
|||
LOG.info("TestTaskErr - cleanup task node once in ERR state");
|
||||
|
||||
conf.setInt("hbase.splitlog.max.resubmit", 0);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
|
||||
slm.finishInitialization();
|
||||
TaskBatch batch = new TaskBatch();
|
||||
|
||||
String tasknode = submitTaskAndWait(batch, "foo/1");
|
||||
ZKUtil.setData(zkw, tasknode, TaskState.TASK_ERR.get("worker"));
|
||||
final ServerName worker1 = new ServerName("worker1,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Err(worker1);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
synchronized (batch) {
|
||||
while (batch.installed != batch.error) {
|
||||
batch.wait();
|
||||
|
@ -359,27 +385,34 @@ public class TestSplitLogManager {
|
|||
}
|
||||
waitForCounter(tot_mgr_task_deleted, 0, 1, 1000);
|
||||
assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
|
||||
conf.setInt("hbase.splitlog.max.resubmit", ZKSplitLog.DEFAULT_MAX_RESUBMIT);
|
||||
conf.setInt("hbase.splitlog.max.resubmit", SplitLogManager.DEFAULT_MAX_RESUBMIT);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskResigned() throws Exception {
|
||||
LOG.info("TestTaskResigned - resubmit task node once in RESIGNED state");
|
||||
|
||||
slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
|
||||
assertEquals(tot_mgr_resubmit.get(), 0);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
|
||||
slm.finishInitialization();
|
||||
assertEquals(tot_mgr_resubmit.get(), 0);
|
||||
TaskBatch batch = new TaskBatch();
|
||||
String tasknode = submitTaskAndWait(batch, "foo/1");
|
||||
ZKUtil.setData(zkw, tasknode, TaskState.TASK_RESIGNED.get("worker"));
|
||||
assertEquals(tot_mgr_resubmit.get(), 0);
|
||||
final ServerName worker1 = new ServerName("worker1,1,1");
|
||||
assertEquals(tot_mgr_resubmit.get(), 0);
|
||||
SplitLogTask slt = new SplitLogTask.Resigned(worker1);
|
||||
assertEquals(tot_mgr_resubmit.get(), 0);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
int version = ZKUtil.checkExists(zkw, tasknode);
|
||||
|
||||
waitForCounter(tot_mgr_resubmit, 0, 1, 1000);
|
||||
// Could be small race here.
|
||||
if (tot_mgr_resubmit.get() == 0) waitForCounter(tot_mgr_resubmit, 0, 1, 1000);
|
||||
assertEquals(tot_mgr_resubmit.get(), 1);
|
||||
int version1 = ZKUtil.checkExists(zkw, tasknode);
|
||||
assertTrue(version1 > version);
|
||||
assertTrue("version1=" + version1 + ", version=" + version, version1 > version);
|
||||
|
||||
byte[] taskstate = ZKUtil.getData(zkw, tasknode);
|
||||
assertTrue(Arrays.equals(taskstate,
|
||||
TaskState.TASK_UNASSIGNED.get("dummy-master")));
|
||||
slt = SplitLogTask.parseFrom(taskstate);
|
||||
assertTrue(slt.isUnassigned(DUMMY_MASTER));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -389,8 +422,9 @@ public class TestSplitLogManager {
|
|||
|
||||
// create an orphan task in OWNED state
|
||||
String tasknode1 = ZKSplitLog.getEncodedNodeName(zkw, "orphan/1");
|
||||
zkw.getRecoverableZooKeeper().create(tasknode1,
|
||||
TaskState.TASK_OWNED.get("dummy-worker"), Ids.OPEN_ACL_UNSAFE,
|
||||
final ServerName worker1 = new ServerName("worker1,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Owned(worker1);
|
||||
zkw.getRecoverableZooKeeper().create(tasknode1, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
|
||||
int to = 1000;
|
||||
|
@ -399,7 +433,7 @@ public class TestSplitLogManager {
|
|||
conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
|
||||
|
||||
|
||||
slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
|
||||
slm.finishInitialization();
|
||||
waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, 100);
|
||||
|
||||
|
@ -411,8 +445,9 @@ public class TestSplitLogManager {
|
|||
// keep updating the orphan owned node every to/2 seconds
|
||||
for (int i = 0; i < (3 * to)/100; i++) {
|
||||
Thread.sleep(100);
|
||||
ZKUtil.setData(zkw, tasknode1,
|
||||
TaskState.TASK_OWNED.get("dummy-worker"));
|
||||
final ServerName worker2 = new ServerName("worker1,1,1");
|
||||
slt = new SplitLogTask.Owned(worker2);
|
||||
ZKUtil.setData(zkw, tasknode1, slt.toByteArray());
|
||||
}
|
||||
|
||||
// since we have stopped heartbeating the owned node therefore it should
|
||||
|
@ -429,31 +464,34 @@ public class TestSplitLogManager {
|
|||
LOG.info("testDeadWorker");
|
||||
|
||||
conf.setLong("hbase.splitlog.max.resubmit", 0);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
|
||||
slm.finishInitialization();
|
||||
TaskBatch batch = new TaskBatch();
|
||||
|
||||
String tasknode = submitTaskAndWait(batch, "foo/1");
|
||||
int version = ZKUtil.checkExists(zkw, tasknode);
|
||||
|
||||
ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
|
||||
waitForCounter(tot_mgr_heartbeat, 0, 1, 1000);
|
||||
slm.handleDeadWorker("worker1");
|
||||
waitForCounter(tot_mgr_resubmit, 0, 1, 1000);
|
||||
waitForCounter(tot_mgr_resubmit_dead_server_task, 0, 1, 1000);
|
||||
final ServerName worker1 = new ServerName("worker1,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Owned(worker1);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
if (tot_mgr_heartbeat.get() == 0) waitForCounter(tot_mgr_heartbeat, 0, 1, 1000);
|
||||
slm.handleDeadWorker(worker1);
|
||||
if (tot_mgr_resubmit.get() == 0) waitForCounter(tot_mgr_resubmit, 0, 1, 3000);
|
||||
if (tot_mgr_resubmit_dead_server_task.get() == 0) {
|
||||
waitForCounter(tot_mgr_resubmit_dead_server_task, 0, 1, 3000);
|
||||
}
|
||||
|
||||
int version1 = ZKUtil.checkExists(zkw, tasknode);
|
||||
assertTrue(version1 > version);
|
||||
byte[] taskstate = ZKUtil.getData(zkw, tasknode);
|
||||
assertTrue(Arrays.equals(TaskState.TASK_UNASSIGNED.get("dummy-master"),
|
||||
taskstate));
|
||||
slt = SplitLogTask.parseFrom(taskstate);
|
||||
assertTrue(slt.isUnassigned(DUMMY_MASTER));
|
||||
return;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptyLogDir() throws Exception {
|
||||
LOG.info("testEmptyLogDir");
|
||||
slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
|
||||
slm.finishInitialization();
|
||||
FileSystem fs = TEST_UTIL.getTestFileSystem();
|
||||
Path emptyLogDirPath = new Path(fs.getWorkingDirectory(),
|
||||
|
@ -467,7 +505,7 @@ public class TestSplitLogManager {
|
|||
public void testVanishingTaskZNode() throws Exception {
|
||||
LOG.info("testVanishingTaskZNode");
|
||||
conf.setInt("hbase.splitlog.manager.unassigned.timeout", 0);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
|
||||
slm.finishInitialization();
|
||||
FileSystem fs = TEST_UTIL.getTestFileSystem();
|
||||
final Path logDir = new Path(fs.getWorkingDirectory(),
|
||||
|
|
|
@ -19,12 +19,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.resetCounters;
|
||||
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_failed_to_grab_task_lost_race;
|
||||
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_failed_to_grab_task_owned;
|
||||
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_preempt_task;
|
||||
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_acquired;
|
||||
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_acquired_rescan;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
|
@ -35,10 +29,11 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SplitLogCounters;
|
||||
import org.apache.hadoop.hbase.SplitLogTask;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog.TaskState;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.log4j.Level;
|
||||
|
@ -53,6 +48,7 @@ import org.junit.experimental.categories.Category;
|
|||
@Category(MediumTests.class)
|
||||
public class TestSplitLogWorker {
|
||||
private static final Log LOG = LogFactory.getLog(TestSplitLogWorker.class);
|
||||
private final ServerName MANAGER = new ServerName("manager,1,1");
|
||||
static {
|
||||
Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
|
||||
}
|
||||
|
@ -98,7 +94,7 @@ public class TestSplitLogWorker {
|
|||
ZKUtil.createAndFailSilent(zkw, zkw.splitLogZNode);
|
||||
assertTrue(ZKUtil.checkExists(zkw, zkw.splitLogZNode) != -1);
|
||||
LOG.debug(zkw.splitLogZNode + " created");
|
||||
resetCounters();
|
||||
SplitLogCounters.resetCounters();
|
||||
|
||||
}
|
||||
|
||||
|
@ -129,19 +125,20 @@ public class TestSplitLogWorker {
|
|||
@Test
|
||||
public void testAcquireTaskAtStartup() throws Exception {
|
||||
LOG.info("testAcquireTaskAtStartup");
|
||||
ZKSplitLog.Counters.resetCounters();
|
||||
|
||||
zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "tatas"),
|
||||
TaskState.TASK_UNASSIGNED.get("mgr"), Ids.OPEN_ACL_UNSAFE,
|
||||
SplitLogCounters.resetCounters();
|
||||
final String TATAS = "tatas";
|
||||
final ServerName RS = new ServerName("rs,1,1");
|
||||
zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, TATAS),
|
||||
new SplitLogTask.Unassigned(new ServerName("mgr,1,1")).toByteArray(), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
|
||||
SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
|
||||
"rs", neverEndingTask);
|
||||
SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(), RS, neverEndingTask);
|
||||
slw.start();
|
||||
try {
|
||||
waitForCounter(tot_wkr_task_acquired, 0, 1, 100);
|
||||
assertTrue(TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
|
||||
ZKSplitLog.getEncodedNodeName(zkw, "tatas")), "rs"));
|
||||
waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 100);
|
||||
byte [] bytes = ZKUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TATAS));
|
||||
SplitLogTask slt = SplitLogTask.parseFrom(bytes);
|
||||
assertTrue(slt.isOwned(RS));
|
||||
} finally {
|
||||
stopSplitLogWorker(slw);
|
||||
}
|
||||
|
@ -161,28 +158,27 @@ public class TestSplitLogWorker {
|
|||
@Test
|
||||
public void testRaceForTask() throws Exception {
|
||||
LOG.info("testRaceForTask");
|
||||
ZKSplitLog.Counters.resetCounters();
|
||||
|
||||
zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "trft"),
|
||||
TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
|
||||
SplitLogCounters.resetCounters();
|
||||
final String TRFT = "trft";
|
||||
final ServerName SVR1 = new ServerName("svr1,1,1");
|
||||
final ServerName SVR2 = new ServerName("svr2,1,1");
|
||||
zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, TRFT),
|
||||
new SplitLogTask.Unassigned(MANAGER).toByteArray(), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
|
||||
SplitLogWorker slw1 = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
|
||||
"svr1", neverEndingTask);
|
||||
SplitLogWorker slw2 = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
|
||||
"svr2", neverEndingTask);
|
||||
SplitLogWorker slw1 = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(), SVR1, neverEndingTask);
|
||||
SplitLogWorker slw2 = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(), SVR2, neverEndingTask);
|
||||
slw1.start();
|
||||
slw2.start();
|
||||
try {
|
||||
waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
|
||||
waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1000);
|
||||
// Assert that either the tot_wkr_failed_to_grab_task_owned count was set of if
|
||||
// not it, that we fell through to the next counter in line and it was set.
|
||||
assertTrue(waitForCounterBoolean(tot_wkr_failed_to_grab_task_owned, 0, 1, 1000) ||
|
||||
tot_wkr_failed_to_grab_task_lost_race.get() == 1);
|
||||
assertTrue(TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
|
||||
ZKSplitLog.getEncodedNodeName(zkw, "trft")), "svr1") ||
|
||||
TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
|
||||
ZKSplitLog.getEncodedNodeName(zkw, "trft")), "svr2"));
|
||||
assertTrue(waitForCounterBoolean(SplitLogCounters.tot_wkr_failed_to_grab_task_owned, 0, 1, 1000) ||
|
||||
SplitLogCounters.tot_wkr_failed_to_grab_task_lost_race.get() == 1);
|
||||
byte [] bytes = ZKUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TRFT));
|
||||
SplitLogTask slt = SplitLogTask.parseFrom(bytes);
|
||||
assertTrue(slt.isOwned(SVR1) || slt.isOwned(SVR2));
|
||||
} finally {
|
||||
stopSplitLogWorker(slw1);
|
||||
stopSplitLogWorker(slw2);
|
||||
|
@ -192,28 +188,28 @@ public class TestSplitLogWorker {
|
|||
@Test
|
||||
public void testPreemptTask() throws Exception {
|
||||
LOG.info("testPreemptTask");
|
||||
ZKSplitLog.Counters.resetCounters();
|
||||
|
||||
SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
|
||||
"tpt_svr", neverEndingTask);
|
||||
SplitLogCounters.resetCounters();
|
||||
final ServerName SRV = new ServerName("tpt_svr,1,1");
|
||||
final String PATH = ZKSplitLog.getEncodedNodeName(zkw, "tpt_task");
|
||||
SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(), SRV, neverEndingTask);
|
||||
slw.start();
|
||||
try {
|
||||
Thread.yield(); // let the worker start
|
||||
Thread.sleep(100);
|
||||
|
||||
// this time create a task node after starting the splitLogWorker
|
||||
zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "tpt_task"),
|
||||
TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
|
||||
zkw.getRecoverableZooKeeper().create(PATH,
|
||||
new SplitLogTask.Unassigned(MANAGER).toByteArray(), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
|
||||
waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
|
||||
waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1000);
|
||||
assertEquals(1, slw.taskReadySeq);
|
||||
assertTrue(TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
|
||||
ZKSplitLog.getEncodedNodeName(zkw, "tpt_task")), "tpt_svr"));
|
||||
|
||||
ZKUtil.setData(zkw, ZKSplitLog.getEncodedNodeName(zkw, "tpt_task"),
|
||||
TaskState.TASK_UNASSIGNED.get("manager"));
|
||||
waitForCounter(tot_wkr_preempt_task, 0, 1, 1000);
|
||||
byte [] bytes = ZKUtil.getData(zkw, PATH);
|
||||
SplitLogTask slt = SplitLogTask.parseFrom(bytes);
|
||||
assertTrue(slt.isOwned(SRV));
|
||||
slt = new SplitLogTask.Unassigned(MANAGER);
|
||||
ZKUtil.setData(zkw, PATH, slt.toByteArray());
|
||||
waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, 1000);
|
||||
} finally {
|
||||
stopSplitLogWorker(slw);
|
||||
}
|
||||
|
@ -222,35 +218,37 @@ public class TestSplitLogWorker {
|
|||
@Test
|
||||
public void testMultipleTasks() throws Exception {
|
||||
LOG.info("testMultipleTasks");
|
||||
ZKSplitLog.Counters.resetCounters();
|
||||
SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
|
||||
"tmt_svr", neverEndingTask);
|
||||
SplitLogCounters.resetCounters();
|
||||
final ServerName SRV = new ServerName("tmt_svr,1,1");
|
||||
final String PATH1 = ZKSplitLog.getEncodedNodeName(zkw, "tmt_task");
|
||||
SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(), SRV, neverEndingTask);
|
||||
slw.start();
|
||||
try {
|
||||
Thread.yield(); // let the worker start
|
||||
Thread.sleep(100);
|
||||
SplitLogTask unassignedManager = new SplitLogTask.Unassigned(MANAGER);
|
||||
zkw.getRecoverableZooKeeper().create(PATH1, unassignedManager.toByteArray(),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
|
||||
zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "tmt_task"),
|
||||
TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
|
||||
waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
|
||||
waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1000);
|
||||
// now the worker is busy doing the above task
|
||||
|
||||
// create another task
|
||||
zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "tmt_task_2"),
|
||||
TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
final String PATH2 = ZKSplitLog.getEncodedNodeName(zkw, "tmt_task_2");
|
||||
zkw.getRecoverableZooKeeper().create(PATH2, unassignedManager.toByteArray(),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
|
||||
// preempt the first task, have it owned by another worker
|
||||
ZKUtil.setData(zkw, ZKSplitLog.getEncodedNodeName(zkw, "tmt_task"),
|
||||
TaskState.TASK_OWNED.get("another-worker"));
|
||||
waitForCounter(tot_wkr_preempt_task, 0, 1, 1000);
|
||||
final ServerName anotherWorker = new ServerName("another-worker,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Owned(anotherWorker);
|
||||
ZKUtil.setData(zkw, PATH1, slt.toByteArray());
|
||||
waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, 1000);
|
||||
|
||||
waitForCounter(tot_wkr_task_acquired, 1, 2, 1000);
|
||||
waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 1, 2, 1000);
|
||||
assertEquals(2, slw.taskReadySeq);
|
||||
assertTrue(TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
|
||||
ZKSplitLog.getEncodedNodeName(zkw, "tmt_task_2")), "tmt_svr"));
|
||||
byte [] bytes = ZKUtil.getData(zkw, PATH2);
|
||||
slt = SplitLogTask.parseFrom(bytes);
|
||||
assertTrue(slt.isOwned(SRV));
|
||||
} finally {
|
||||
stopSplitLogWorker(slw);
|
||||
}
|
||||
|
@ -259,38 +257,37 @@ public class TestSplitLogWorker {
|
|||
@Test
|
||||
public void testRescan() throws Exception {
|
||||
LOG.info("testRescan");
|
||||
ZKSplitLog.Counters.resetCounters();
|
||||
slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
|
||||
"svr", neverEndingTask);
|
||||
SplitLogCounters.resetCounters();
|
||||
final ServerName SRV = new ServerName("svr,1,1");
|
||||
slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(), SRV, neverEndingTask);
|
||||
slw.start();
|
||||
Thread.yield(); // let the worker start
|
||||
Thread.sleep(100);
|
||||
|
||||
String task = ZKSplitLog.getEncodedNodeName(zkw, "task");
|
||||
zkw.getRecoverableZooKeeper().create(task,
|
||||
TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
|
||||
SplitLogTask slt = new SplitLogTask.Unassigned(MANAGER);
|
||||
zkw.getRecoverableZooKeeper().create(task,slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
|
||||
waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
|
||||
waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1000);
|
||||
// now the worker is busy doing the above task
|
||||
|
||||
// preempt the task, have it owned by another worker
|
||||
ZKUtil.setData(zkw, task, TaskState.TASK_UNASSIGNED.get("manager"));
|
||||
waitForCounter(tot_wkr_preempt_task, 0, 1, 1000);
|
||||
ZKUtil.setData(zkw, task, slt.toByteArray());
|
||||
waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, 1000);
|
||||
|
||||
// create a RESCAN node
|
||||
String rescan = ZKSplitLog.getEncodedNodeName(zkw, "RESCAN");
|
||||
rescan = zkw.getRecoverableZooKeeper().create(rescan,
|
||||
TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
|
||||
rescan = zkw.getRecoverableZooKeeper().create(rescan, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT_SEQUENTIAL);
|
||||
|
||||
waitForCounter(tot_wkr_task_acquired, 1, 2, 1000);
|
||||
waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 1, 2, 1000);
|
||||
// RESCAN node might not have been processed if the worker became busy
|
||||
// with the above task. preempt the task again so that now the RESCAN
|
||||
// node is processed
|
||||
ZKUtil.setData(zkw, task, TaskState.TASK_UNASSIGNED.get("manager"));
|
||||
waitForCounter(tot_wkr_preempt_task, 1, 2, 1000);
|
||||
waitForCounter(tot_wkr_task_acquired_rescan, 0, 1, 1000);
|
||||
ZKUtil.setData(zkw, task, slt.toByteArray());
|
||||
waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 1, 2, 1000);
|
||||
waitForCounter(SplitLogCounters.tot_wkr_task_acquired_rescan, 0, 1, 1000);
|
||||
|
||||
List<String> nodes = ZKUtil.listChildrenNoWatch(zkw, zkw.splitLogZNode);
|
||||
LOG.debug(nodes);
|
||||
|
@ -301,8 +298,8 @@ public class TestSplitLogWorker {
|
|||
String name = ZKSplitLog.getEncodedNodeName(zkw, node);
|
||||
String fn = ZKSplitLog.getFileName(name);
|
||||
byte [] data = ZKUtil.getData(zkw, ZKUtil.joinZNode(zkw.splitLogZNode, fn));
|
||||
String datastr = Bytes.toString(data);
|
||||
assertTrue("data=" + datastr, TaskState.TASK_DONE.equals(data, "svr"));
|
||||
slt = SplitLogTask.parseFrom(data);
|
||||
assertTrue(slt.toString(), slt.isDone(SRV));
|
||||
}
|
||||
}
|
||||
assertEquals(2, num);
|
||||
|
@ -311,5 +308,4 @@ public class TestSplitLogWorker {
|
|||
@org.junit.Rule
|
||||
public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
|
||||
new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
|
||||
}
|
||||
|
||||
}
|
|
@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.client.Delete;
|
|||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.executor.RegionTransitionData;
|
||||
import org.apache.hadoop.hbase.master.handler.SplitRegionHandler;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
||||
|
@ -98,9 +97,10 @@ public class TestSplitTransactionOnCluster {
|
|||
* @throws InterruptedException
|
||||
* @throws NodeExistsException
|
||||
* @throws KeeperException
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
@Test (timeout = 300000) public void testRSSplitEphemeralsDisappearButDaughtersAreOnlinedAfterShutdownHandling()
|
||||
throws IOException, InterruptedException, NodeExistsException, KeeperException {
|
||||
throws IOException, InterruptedException, NodeExistsException, KeeperException, DeserializationException {
|
||||
final byte [] tableName =
|
||||
Bytes.toBytes("ephemeral");
|
||||
|
||||
|
@ -137,12 +137,12 @@ public class TestSplitTransactionOnCluster {
|
|||
Stat stats =
|
||||
TESTING_UTIL.getZooKeeperWatcher().getRecoverableZooKeeper().exists(path, false);
|
||||
LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats=" + stats);
|
||||
RegionTransitionData rtd =
|
||||
ZKAssign.getData(TESTING_UTIL.getZooKeeperWatcher(),
|
||||
hri.getEncodedName());
|
||||
RegionTransition rt =
|
||||
RegionTransition.parseFrom(ZKAssign.getData(TESTING_UTIL.getZooKeeperWatcher(),
|
||||
hri.getEncodedName()));
|
||||
// State could be SPLIT or SPLITTING.
|
||||
assertTrue(rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLIT) ||
|
||||
rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLITTING));
|
||||
assertTrue(rt.getEventType().equals(EventType.RS_ZK_REGION_SPLIT) ||
|
||||
rt.getEventType().equals(EventType.RS_ZK_REGION_SPLITTING));
|
||||
// Now crash the server
|
||||
cluster.abortRegionServer(tableRegionIndex);
|
||||
waitUntilRegionServerDead();
|
||||
|
|
|
@ -25,14 +25,15 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.RegionTransition;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.executor.RegionTransitionData;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -133,9 +134,10 @@ public class TestCloseRegionHandler {
|
|||
* @throws IOException
|
||||
* @throws NodeExistsException
|
||||
* @throws KeeperException
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
@Test public void testZKClosingNodeVersionMismatch()
|
||||
throws IOException, NodeExistsException, KeeperException {
|
||||
throws IOException, NodeExistsException, KeeperException, DeserializationException {
|
||||
final Server server = new MockServer(HTU);
|
||||
final MockRegionServerServices rss = new MockRegionServerServices();
|
||||
rss.setFileSystem(HTU.getTestFileSystem());
|
||||
|
@ -160,9 +162,9 @@ public class TestCloseRegionHandler {
|
|||
handler.process();
|
||||
|
||||
// Handler should remain in M_ZK_REGION_CLOSING
|
||||
RegionTransitionData data =
|
||||
ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName());
|
||||
assertTrue(EventType.M_ZK_REGION_CLOSING == data.getEventType());
|
||||
RegionTransition rt =
|
||||
RegionTransition.parseFrom(ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName()));
|
||||
assertTrue(rt.getEventType().equals(EventType.M_ZK_REGION_CLOSING ));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -170,9 +172,10 @@ public class TestCloseRegionHandler {
|
|||
* @throws IOException
|
||||
* @throws NodeExistsException
|
||||
* @throws KeeperException
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
@Test public void testCloseRegion()
|
||||
throws IOException, NodeExistsException, KeeperException {
|
||||
throws IOException, NodeExistsException, KeeperException, DeserializationException {
|
||||
final Server server = new MockServer(HTU);
|
||||
final MockRegionServerServices rss = new MockRegionServerServices();
|
||||
rss.setFileSystem(HTU.getTestFileSystem());
|
||||
|
@ -196,26 +199,23 @@ public class TestCloseRegionHandler {
|
|||
versionOfClosingNode);
|
||||
handler.process();
|
||||
// Handler should have transitioned it to RS_ZK_REGION_CLOSED
|
||||
RegionTransitionData data =
|
||||
ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName());
|
||||
assertTrue(EventType.RS_ZK_REGION_CLOSED == data.getEventType());
|
||||
RegionTransition rt = RegionTransition.parseFrom(
|
||||
ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName()));
|
||||
assertTrue(rt.getEventType().equals(EventType.RS_ZK_REGION_CLOSED));
|
||||
}
|
||||
|
||||
private void OpenRegion(Server server, RegionServerServices rss,
|
||||
HTableDescriptor htd, HRegionInfo hri)
|
||||
throws IOException, NodeExistsException, KeeperException {
|
||||
// Create it OFFLINE node, which is what Master set before sending OPEN RPC
|
||||
ZKAssign.createNodeOffline(server.getZooKeeper(), hri,
|
||||
server.getServerName());
|
||||
OpenRegionHandler openHandler = new OpenRegionHandler(server, rss, hri,
|
||||
htd);
|
||||
openHandler.process();
|
||||
RegionTransitionData data =
|
||||
ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName());
|
||||
|
||||
// delete the node, which is what Master do after the region is opened
|
||||
ZKAssign.deleteNode(server.getZooKeeper(), hri.getEncodedName(),
|
||||
EventType.RS_ZK_REGION_OPENED);
|
||||
}
|
||||
HTableDescriptor htd, HRegionInfo hri)
|
||||
throws IOException, NodeExistsException, KeeperException, DeserializationException {
|
||||
// Create it OFFLINE node, which is what Master set before sending OPEN RPC
|
||||
ZKAssign.createNodeOffline(server.getZooKeeper(), hri, server.getServerName());
|
||||
OpenRegionHandler openHandler = new OpenRegionHandler(server, rss, hri, htd);
|
||||
openHandler.process();
|
||||
// This parse is not used?
|
||||
RegionTransition.parseFrom(ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName()));
|
||||
// delete the node, which is what Master do after the region is opened
|
||||
ZKAssign.deleteNode(server.getZooKeeper(), hri.getEncodedName(), EventType.RS_ZK_REGION_OPENED);
|
||||
}
|
||||
|
||||
@org.junit.Rule
|
||||
public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
|
||||
|
|
|
@ -26,7 +26,6 @@ import java.io.IOException;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.executor.RegionTransitionData;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
|
@ -148,9 +147,9 @@ public class TestOpenRegionHandler {
|
|||
handler.process();
|
||||
|
||||
// Handler should have transitioned it to FAILED_OPEN
|
||||
RegionTransitionData data =
|
||||
ZKAssign.getData(server.getZooKeeper(), TEST_HRI.getEncodedName());
|
||||
assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, data.getEventType());
|
||||
RegionTransition rt = RegionTransition.parseFrom(
|
||||
ZKAssign.getData(server.getZooKeeper(), TEST_HRI.getEncodedName()));
|
||||
assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, rt.getEventType());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -173,9 +172,9 @@ public class TestOpenRegionHandler {
|
|||
handler.process();
|
||||
|
||||
// Handler should have transitioned it to FAILED_OPEN
|
||||
RegionTransitionData data =
|
||||
ZKAssign.getData(server.getZooKeeper(), TEST_HRI.getEncodedName());
|
||||
assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, data.getEventType());
|
||||
RegionTransition rt = RegionTransition.parseFrom(
|
||||
ZKAssign.getData(server.getZooKeeper(), TEST_HRI.getEncodedName()));
|
||||
assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, rt.getEventType());
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
|||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.RegionTransition;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.client.AdminProtocol;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
|
@ -59,7 +60,6 @@ import org.apache.hadoop.hbase.client.Result;
|
|||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.executor.RegionTransitionData;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
|
@ -920,9 +920,9 @@ public class TestHBaseFsck {
|
|||
|
||||
int iTimes = 0;
|
||||
while (true) {
|
||||
RegionTransitionData rtd = ZKAssign.getData(zkw,
|
||||
region.getEncodedName());
|
||||
if (rtd != null && rtd.getEventType() == EventType.RS_ZK_REGION_OPENED) {
|
||||
RegionTransition rt = RegionTransition.parseFrom(ZKAssign.getData(zkw,
|
||||
region.getEncodedName()));
|
||||
if (rt != null && rt.getEventType() == EventType.RS_ZK_REGION_OPENED) {
|
||||
break;
|
||||
}
|
||||
Thread.sleep(100);
|
||||
|
|
Loading…
Reference in New Issue