From a32642e6f61fc4308a47e8eee356796d1285344d Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Thu, 31 Aug 2017 19:51:26 +0800 Subject: [PATCH] HBASE-18721 Cleanup unused configs and private declaration --- .../hbase/client/BufferedMutatorImpl.java | 6 --- .../regionserver/RegionOpeningState.java | 31 ------------- .../shaded/protobuf/ResponseConverter.java | 31 ------------- .../apache/hadoop/hbase/CellComparator.java | 10 ----- .../org/apache/hadoop/hbase/CellUtil.java | 19 -------- .../org/apache/hadoop/hbase/HConstants.java | 21 +++++---- .../hadoop/hbase/util/ByteBufferUtils.java | 36 --------------- .../org/apache/hadoop/hbase/util/Bytes.java | 45 ------------------- .../procedure/DisableTableProcedure.java | 6 --- .../hbase/namespace/NamespaceAuditor.java | 2 - .../hbase/quotas/QuotaSnapshotStore.java | 9 ---- .../hadoop/hbase/regionserver/HRegion.java | 4 -- .../hbase/regionserver/HRegionServer.java | 3 -- .../hbase/regionserver/StoreScanner.java | 7 --- .../querymatcher/DeleteTracker.java | 17 ------- .../hbase/zookeeper/MiniZooKeeperCluster.java | 2 +- .../assignment/TestAssignmentManager.java | 28 ------------ .../wal/AbstractTestWALReplay.java | 3 +- .../HbaseObjectWritableFor96Migration.java | 7 +-- 19 files changed, 17 insertions(+), 270 deletions(-) delete mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOpeningState.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java index b7d310451e7..912c8f040db 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java @@ -58,12 +58,6 @@ import org.apache.hadoop.hbase.ipc.RpcControllerFactory; @InterfaceAudience.Private @InterfaceStability.Evolving public class BufferedMutatorImpl implements BufferedMutator { - /** - * Key to use setting non-default BufferedMutator implementation - * classname via Configuration. - */ - public static final String HBASE_BUFFEREDMUTATOR_CLASSNAME_KEY = - "hbase.client.bufferedmutator.classname"; private static final Log LOG = LogFactory.getLog(BufferedMutatorImpl.class); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOpeningState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOpeningState.java deleted file mode 100644 index 7e380b05c9d..00000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOpeningState.java +++ /dev/null @@ -1,31 +0,0 @@ -/** - * - * 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.regionserver; - -import org.apache.hadoop.hbase.classification.InterfaceAudience; - -@InterfaceAudience.Private -public enum RegionOpeningState { - - OPENED, - - ALREADY_OPENED, - - FAILED_OPENING; -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java index 98e6f697090..69411c02730 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java @@ -55,7 +55,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCata import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse; -import org.apache.hadoop.hbase.regionserver.RegionOpeningState; import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString; import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController; import org.apache.hadoop.util.StringUtils; @@ -243,36 +242,6 @@ public final class ResponseConverter { return ProtobufUtil.getRegionInfos(proto); } - /** - * Get the region opening state from a OpenRegionResponse - * - * @param proto the OpenRegionResponse - * @return the region opening state - */ - public static RegionOpeningState getRegionOpeningState - (final OpenRegionResponse proto) { - if (proto == null || proto.getOpeningStateCount() != 1) return null; - return RegionOpeningState.valueOf( - proto.getOpeningState(0).name()); - } - - /** - * Get a list of region opening state from a OpenRegionResponse - * - * @param proto the OpenRegionResponse - * @return the list of region opening state - */ - public static List getRegionOpeningStateList( - final OpenRegionResponse proto) { - if (proto == null) return null; - List regionOpeningStates = new ArrayList<>(proto.getOpeningStateCount()); - for (int i = 0; i < proto.getOpeningStateCount(); i++) { - regionOpeningStates.add(RegionOpeningState.valueOf( - proto.getOpeningState(i).name())); - } - return regionOpeningStates; - } - /** * Check if the region is closed from a CloseRegionResponse * diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java index 8fa7e854f50..8187529a1d0 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java @@ -572,16 +572,6 @@ public class CellComparator implements Comparator, Serializable { return comparator.compareTo(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); } - /** - * Comparator that compares row component only of a Cell - */ - public static class RowComparator extends CellComparator { - @Override - public int compare(Cell a, Cell b) { - return compareRows(a, b); - } - } - /** * A {@link CellComparator} for hbase:meta catalog table * {@link KeyValue}s. diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java index 4a5023de31f..d1a72b79a03 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java @@ -1551,25 +1551,6 @@ public final class CellUtil { }; } - private static final Iterator EMPTY_TAGS_ITR = new Iterator() { - @Override - public boolean hasNext() { - return false; - } - - @Override - @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IT_NO_SUCH_ELEMENT", - justification="Intentional") - public Tag next() { - return null; - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }; - /** * Util method to iterate through the tags in the given cell. * diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 54e0eb8c235..ecccca6db9a 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -154,6 +154,7 @@ public final class HConstants { public static final boolean CLUSTER_IS_LOCAL = false; /** Cluster is fully-distributed */ + @Deprecated // unused. see HBASE-13636. remove this in 3.0 public static final boolean CLUSTER_IS_DISTRIBUTED = true; /** Default value for cluster distributed mode */ @@ -208,10 +209,12 @@ public final class HConstants { /** * Parameter name for the wait time for the recoverable zookeeper */ + @Deprecated // unused. see HBASE-3065. remove this in 3.0 public static final String ZOOKEEPER_RECOVERABLE_WAITTIME = "hbase.zookeeper.recoverable.waittime"; /** Default wait time for the recoverable zookeeper */ + @Deprecated // unused. see HBASE-3065. remove this in 3.0 public static final long DEFAULT_ZOOKEPER_RECOVERABLE_WAITIME = 10000; /** Parameter name for the root dir in ZK for this cluster */ @@ -267,6 +270,7 @@ public final class HConstants { public static final String MASTER_IMPL= "hbase.master.impl"; /** Parameter name for what hbase client implementation to use. */ + @Deprecated // unused. see HBASE-7460. remove this in 3.0 public static final String HBASECLIENT_IMPL= "hbase.hbaseclient.impl"; /** Parameter name for how often threads should wake up */ @@ -339,6 +343,7 @@ public final class HConstants { * dynamically by the region servers. This value can be overridden by the * hbase.dynamic.jars.dir config. */ + @Deprecated // unused. see HBASE-12054. remove this in 3.0 public static final String LIB_DIR = "lib"; /** Used to construct the name of the compaction directory during compaction */ @@ -623,6 +628,7 @@ public final class HConstants { /** * Seconds in a week */ + @Deprecated // unused. see HBASE-2692. remove this in 3.0 public static final int WEEK_IN_SECONDS = 7 * 24 * 3600; /** @@ -654,17 +660,6 @@ public final class HConstants { public static final String REGION_IMPL = "hbase.hregion.impl"; - /** modifyTable op for replacing the table descriptor */ - @InterfaceAudience.Private - public static enum Modify { - CLOSE_REGION, - TABLE_COMPACT, - TABLE_FLUSH, - TABLE_MAJOR_COMPACT, - TABLE_SET_HTD, - TABLE_SPLIT - } - /** * Scope tag for locally scoped data. * This data will not be replicated. @@ -954,6 +949,7 @@ public final class HConstants { public static final String BULKLOAD_MAX_RETRIES_NUMBER = "hbase.bulkload.retries.number"; /** HBCK special code name used as server name when manipulating ZK nodes */ + @Deprecated // unused. see HBASE-3789. remove this in 3.0 public static final String HBCK_CODE_NAME = "HBCKServerName"; public static final String KEY_FOR_HOSTNAME_SEEN_BY_MASTER = @@ -1055,10 +1051,13 @@ public final class HConstants { "hbase.regionserver.replication.handler.count"; public static final int DEFAULT_REGION_SERVER_REPLICATION_HANDLER_COUNT = 3; + @Deprecated // unused. see HBASE-10569. remove this in 3.0 public static final String MASTER_HANDLER_COUNT = "hbase.master.handler.count"; + @Deprecated // unused. see HBASE-10569. remove this in 3.0 public static final int DEFAULT_MASTER_HANLDER_COUNT = 25; /** Conf key that specifies timeout value to wait for a region ready */ + @Deprecated // unused. see HBASE-13616. remove this in 3.0 public static final String LOG_REPLAY_WAIT_REGION_TIMEOUT = "hbase.master.log.replay.wait.region.timeout"; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java index 6dde988d97e..ed6ee09fc89 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java @@ -743,42 +743,6 @@ public final class ByteBufferUtils { return l1 - l2; } - /* - * Both values are passed as is read by Unsafe. When platform is Little Endian, have to convert - * to corresponding Big Endian value and then do compare. We do all writes in Big Endian format. - */ - private static boolean lessThanUnsignedLong(long x1, long x2) { - if (UnsafeAccess.littleEndian) { - x1 = Long.reverseBytes(x1); - x2 = Long.reverseBytes(x2); - } - return (x1 + Long.MIN_VALUE) < (x2 + Long.MIN_VALUE); - } - - /* - * Both values are passed as is read by Unsafe. When platform is Little Endian, have to convert - * to corresponding Big Endian value and then do compare. We do all writes in Big Endian format. - */ - private static boolean lessThanUnsignedInt(int x1, int x2) { - if (UnsafeAccess.littleEndian) { - x1 = Integer.reverseBytes(x1); - x2 = Integer.reverseBytes(x2); - } - return (x1 & 0xffffffffL) < (x2 & 0xffffffffL); - } - - /* - * Both values are passed as is read by Unsafe. When platform is Little Endian, have to convert - * to corresponding Big Endian value and then do compare. We do all writes in Big Endian format. - */ - private static boolean lessThanUnsignedShort(short x1, short x2) { - if (UnsafeAccess.littleEndian) { - x1 = Short.reverseBytes(x1); - x2 = Short.reverseBytes(x2); - } - return (x1 & 0xffff) < (x2 & 0xffff); - } - /** * Reads a short value at the given buffer's offset. * @param buffer diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java index 5e6500bcee9..56cd3af6a35 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java @@ -1509,51 +1509,6 @@ public class Bytes implements Comparable { } } - /** - * Returns true if x1 is less than x2, when both values are treated as - * unsigned long. - * Both values are passed as is read by Unsafe. When platform is Little Endian, have to - * convert to corresponding Big Endian value and then do compare. We do all writes in - * Big Endian format. - */ - static boolean lessThanUnsignedLong(long x1, long x2) { - if (UnsafeAccess.littleEndian) { - x1 = Long.reverseBytes(x1); - x2 = Long.reverseBytes(x2); - } - return (x1 + Long.MIN_VALUE) < (x2 + Long.MIN_VALUE); - } - - /** - * Returns true if x1 is less than x2, when both values are treated as - * unsigned int. - * Both values are passed as is read by Unsafe. When platform is Little Endian, have to - * convert to corresponding Big Endian value and then do compare. We do all writes in - * Big Endian format. - */ - static boolean lessThanUnsignedInt(int x1, int x2) { - if (UnsafeAccess.littleEndian) { - x1 = Integer.reverseBytes(x1); - x2 = Integer.reverseBytes(x2); - } - return (x1 & 0xffffffffL) < (x2 & 0xffffffffL); - } - - /** - * Returns true if x1 is less than x2, when both values are treated as - * unsigned short. - * Both values are passed as is read by Unsafe. When platform is Little Endian, have to - * convert to corresponding Big Endian value and then do compare. We do all writes in - * Big Endian format. - */ - static boolean lessThanUnsignedShort(short x1, short x2) { - if (UnsafeAccess.littleEndian) { - x1 = Short.reverseBytes(x1); - x2 = Short.reverseBytes(x2); - } - return (x1 & 0xffff) < (x2 & 0xffff); - } - /** * Lexicographically compare two arrays. * diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java index 58c4bd03d6c..0872a0151b9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java @@ -47,12 +47,6 @@ public class DisableTableProcedure private Boolean traceEnabled = null; - enum MarkRegionOfflineOpResult { - MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL, - BULK_ASSIGN_REGIONS_FAILED, - MARK_ALL_REGIONS_OFFLINE_INTERRUPTED, - } - public DisableTableProcedure() { super(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceAuditor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceAuditor.java index 232309b344f..b4e913f621f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceAuditor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceAuditor.java @@ -39,8 +39,6 @@ import org.apache.hadoop.hbase.quotas.QuotaExceededException; @InterfaceAudience.Private public class NamespaceAuditor { private static final Log LOG = LogFactory.getLog(NamespaceAuditor.class); - static final String NS_AUDITOR_INIT_TIMEOUT = "hbase.namespace.auditor.init.timeout"; - static final int DEFAULT_NS_AUDITOR_INIT_TIMEOUT = 120000; private NamespaceStateManager stateManager; private MasterServices masterServices; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java index 2b5ba59ab66..a5cfbbce008 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java @@ -33,15 +33,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota; @InterfaceAudience.Private public interface QuotaSnapshotStore { - /** - * The current state of a table with respect to the policy set forth by a quota. - */ - @InterfaceAudience.Private - public enum ViolationState { - IN_VIOLATION, - IN_OBSERVANCE, - } - /** * Singleton to represent a table without a quota defined. It is never in violation. */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 50a6cd872da..823ef39236b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -204,10 +204,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY = "hbase.hregion.scan.loadColumnFamiliesOnDemand"; - /** Config key for using mvcc pre-assign feature for put */ - public static final String HREGION_MVCC_PRE_ASSIGN = "hbase.hregion.mvcc.preassign"; - public static final boolean DEFAULT_HREGION_MVCC_PRE_ASSIGN = true; - public static final String HREGION_UNASSIGN_FOR_FNFE = "hbase.hregion.unassign.for.fnfe"; public static final boolean DEFAULT_HREGION_UNASSIGN_FOR_FNFE = true; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 96050a62e00..1fed43e1cce 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -228,9 +228,6 @@ public class HRegionServer extends HasThread implements // Time to pause if master says 'please hold'. Make configurable if needed. private static final int INIT_PAUSE_TIME_MS = 1000; - public static final String REGION_LOCK_AWAIT_TIME_SEC = - "hbase.regionserver.region.lock.await.time.sec"; - public static final int DEFAULT_REGION_LOCK_AWAIT_TIME_SEC = 300; // 5 min private static final Log LOG = LogFactory.getLog(HRegionServer.class); /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java index 9a096f61072..94bee883f46 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java @@ -159,13 +159,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner protected final long readPt; private boolean topChanged = false; - // used by the injection framework to test race between StoreScanner construction and compaction - enum StoreScannerCompactionRace { - BEFORE_SEEK, - AFTER_SEEK, - COMPACT_COMPLETE - } - /** An internal constructor. */ private StoreScanner(Optional store, Scan scan, ScanInfo scanInfo, int numColumns, long readPt, boolean cacheBlocks, ScanType scanType) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DeleteTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DeleteTracker.java index 7c63b1272ec..2fb0224228f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DeleteTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DeleteTracker.java @@ -70,23 +70,6 @@ public interface DeleteTracker extends ShipperListener { */ void reset(); - /** - * Return codes for comparison of two Deletes. - *

- * The codes tell the merging function what to do. - *

- * INCLUDE means add the specified Delete to the merged list. NEXT means move to the next element - * in the specified list(s). - */ - enum DeleteCompare { - INCLUDE_OLD_NEXT_OLD, - INCLUDE_OLD_NEXT_BOTH, - INCLUDE_NEW_NEXT_NEW, - INCLUDE_NEW_NEXT_BOTH, - NEXT_OLD, - NEXT_NEW - } - /** * Returns codes for delete result. The codes tell the ScanQueryMatcher whether the kv is deleted * and why. Based on the delete result, the ScanQueryMatcher will decide the next operation diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java index 2cc89f0765c..fa307955065 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java @@ -234,7 +234,7 @@ public class MiniZooKeeperCluster { standaloneServerFactory = new NIOServerCnxnFactory(); standaloneServerFactory.configure( new InetSocketAddress(currentClientPort), - configuration.getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS, 1000)); + configuration.getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS, HConstants.DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS)); } catch (BindException e) { LOG.debug("Failed binding ZK Server to client port: " + currentClientPort, e); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java index 4d2a8948558..69f8667b7f4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java @@ -789,35 +789,7 @@ public class TestAssignmentManager { } private class GoodSplitExecutor extends NoopRsExecutor { - - /* - @Override - protected RegionOpeningState execOpenRegion(ServerName server, RegionOpenInfo openReq) - throws IOException { - sendTransitionReport(server, openReq.getRegion(), TransitionCode.OPENED); - // Concurrency? - // Now update the state of our cluster in regionsToRegionServers. - SortedSet regions = regionsToRegionServers.get(server); - if (regions == null) { - regions = new ConcurrentSkipListSet(Bytes.BYTES_COMPARATOR); - regionsToRegionServers.put(server, regions); - } - HRegionInfo hri = HRegionInfo.convert(openReq.getRegion()); - if (regions.contains(hri.getRegionName())) { - throw new UnsupportedOperationException(hri.getRegionNameAsString()); - } - regions.add(hri.getRegionName()); - return RegionOpeningState.OPENED; - } - @Override - protected CloseRegionResponse execCloseRegion(ServerName server, byte[] regionName) - throws IOException { - HRegionInfo hri = am.getRegionInfo(regionName); - sendTransitionReport(server, HRegionInfo.convert(hri), TransitionCode.CLOSED); - return CloseRegionResponse.newBuilder().setClosed(true).build(); - }*/ - } private void collectAssignmentManagerMetrics() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java index be725fe6696..d66ad263419 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java @@ -153,7 +153,8 @@ public abstract class AbstractTestWALReplay { if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseRootDir)) { TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true); } - this.mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ? + this.mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, + HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG) ? RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING); this.wals = new WALFactory(conf, null, currentTest.getMethodName()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/HbaseObjectWritableFor96Migration.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/HbaseObjectWritableFor96Migration.java index adf09d42f50..c5ee512d727 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/HbaseObjectWritableFor96Migration.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/HbaseObjectWritableFor96Migration.java @@ -86,7 +86,6 @@ import org.apache.hadoop.hbase.filter.SkipFilter; import org.apache.hadoop.hbase.filter.ValueFilter; import org.apache.hadoop.hbase.filter.WhileMatchFilter; import org.apache.hadoop.hbase.io.WritableWithSize; -import org.apache.hadoop.hbase.regionserver.RegionOpeningState; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.util.Bytes; @@ -172,7 +171,8 @@ class HbaseObjectWritableFor96Migration implements Writable, WritableWithSize, C // Hbase types addToMap(HColumnDescriptor.class, code++); - addToMap(HConstants.Modify.class, code++); + // HConstants.Modify no longer exists; increase code so other classes stay the same. + code++; // We used to have a class named HMsg but its been removed. Rather than // just axe it, use following random Integer class -- we just chose any @@ -262,7 +262,8 @@ class HbaseObjectWritableFor96Migration implements Writable, WritableWithSize, C code++; //addToMap(HServerLoad.class, code++); - addToMap(RegionOpeningState.class, code++); + // RegionOpeningState no longer exists; increase code so other classes stay the same. + code++; addToMap(HTableDescriptor[].class, code++);