Known Issues in Apache HBase

This topic describes known issues and workarounds for using HBase in this release of Cloudera Runtime.

HBASE-24885: If an operator uses HBCK2 to invoke multiple `assigns` operations against one Region or happens to invoke HBCK2 `assigns` while HBase is re-assigning a Region, it is possible that the Region will be abnormally assigned. For example, unassigned, stuck in transition, and doubly-assigned.
Obtain a fix for this issue. Operators should definitely not schedule multiple assigns for a single Region at the same time, however there is still a potential race condition.
OpDB Data Hub cluster fails to initialize if you are reusing a cloud storage location that was used by an older OpDB Data Hub cluster
Workaround: Stop HBase using Cloudera Manager before deleting an OpDB Data Hub cluster.
IntegrationTestReplication fails if replication does not finish before the verify phase begins

During IntegrationTestReplication, if the verify phase starts before the replication phase finishes, the test will fail because the target cluster does not contain all of the data. If the HBase services in the target cluster does not have enough memory, long garbage-collection pauses might occur.

Workaround: Use the -t flag to set the timeout value before starting verification.
HDFS encryption with HBase

Cloudera has tested the performance impact of using HDFS encryption with HBase. The overall overhead of HDFS encryption on HBase performance is in the range of 3 to 4% for both read and update workloads. Scan performance has not been thoroughly tested.

Workaround: N/A
AccessController postOperation problems in asynchronous operations
When security and Access Control are enabled, the following problems occur:
  • If a Delete Table fails for a reason other than missing permissions, the access rights are removed but the table may still exist and may be used again.
  • If hbaseAdmin.modifyTable() is used to delete column families, the rights are not removed from the Access Control List (ACL) table. The portOperation is implemented only for postDeleteColumn().
  • If Create Table fails, full rights for that table persist for the user who attempted to create it. If another user later succeeds in creating the table, the user who made the failed attempt still has the full rights.
Workaround: N/A
Apache Issue: HBASE-6992
Bulk load is not supported when the source is the local HDFS
The bulk load feature (the completebulkload command) is not supported when the source is the local HDFS and the target is an object store, such as S3/ABFS.
Workaround: Use distcp to move the HFiles from HDFS to S3 and then run bulk load from S3 to S3.
Apache Issue: N/A
Storing Medium Objects (MOBs) in HBase is currently not supported
Storing MOBs in HBase relies on bulk loading files, and this is not currently supported when HBase is configured to use cloud storage (S3).
Workaround: N/A
Apache Issue: N/A

Known incompatibilities when upgrading from CDH and HDP

This section describes incompatible changes when you upgrade from CDH and HDP. Use this list to understand some of the deprecated items and incompatibilities if you are moving from HDP 2.x or CDH 5.x. Note that Cloudera Runtime uses HBase 2.x.x whereas older CDH and HDP used Apache HBase 1.x.

List of Major Changes

  • HBASE-16189 and HBASE-18945- You cannot open the HBase 2.x HFiles through HBase 1.x version.

  • HBASE-18240 - Changed the ReplicationEndpoint Interface. It also introduces a new hbase-third party 1.0 that packages all the third party utilities, which are expected to run in the hbase cluster.

Co-processor API changes

  • HBASE-16769 - Deprecated PB references from MasterObserver and RegionServerObserver.

  • HBASE-17312 - [JDK8] Use default method for Observer Coprocessors. The interface classes of BaseMasterAndRegionObserver, BaseMasterObserver, BaseRegionObserver, BaseRegionServerObserver and BaseWALObserver uses JDK8’s 'default' keyword to provide empty and no-op implementations.

  • Interface HTableInterface introduces following changes to the methods listed below:

[−] interface CoprocessorEnvironment changes
Change Result
Abstract method getTable ( TableName ) has been removed. A client program may be interrupted by NoSuchMethodError exception.
Abstract method getTable ( TableName, ExecutorService ) has been removed. A client program may be interrupted by NoSuchMethodError exception.
  • Public Audience

The following tables describes the coprocessor changes.

[−] class CoprocessorRpcChannel (1)
Change Result
This class has become interface. A client program may be interrupted by IncompatibleClassChangeError or InstantiationError exception depending on the usage of this class.

Class CoprocessorHost<E> changes

Classes that were Audience Private but were removed.

Change Result
Type of field coprocessors has been changed from java.util.SortedSet<E> to org.apache.hadoop.hbase.util.SortedList<E>. A client program may be interrupted by NoSuchFieldError exception.

MasterObserver changes

The following changes are introduced to the MasterObserver interface.

[−] interface MasterObserver (14)
Change Result
Abstract method voidpostCloneSnapshot ( ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method voidpostCreateTable ( ObserverContext<MasterCoprocessorEnvironment>, HTableDescriptor, HRegionInfo[ ] ) has been removed from this interface. A client program may be interrupted by NoSuchMethodErrorexception.
Abstract method voidpostDeleteSnapshot (ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription ) has been removed from this interface. A client program may be interrupted by NoSuchMethodErrorexception.
Abstract method voidpostGetTableDescriptors ( ObserverContext<MasterCoprocessorEnvironment>, List<HTableDescriptor> ) has been removed from this interface. A client program may be interrupted by NoSuchMethodErrorexception.
Abstract method voidpostModifyTable ( ObserverContext<MasterCoprocessorEnvironment>, TableName, HTableDescriptor ) has been removed from this interface. A client program may be interrupted by NoSuchMethodErrorexception.
Abstract method voidpostRestoreSnapshot ( ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface. A client program may be interrupted by NoSuchMethodErrorexception.
Abstract method voidpostSnapshot ( ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface. A client program may be interrupted by NoSuchMethodErrorexception.
Abstract method voidpreCloneSnapshot ( ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface. A client program may be interrupted by NoSuchMethodErrorexception.
Abstract method voidpreCreateTable ( ObserverContext<MasterCoprocessorEnvironment>, HTableDescriptor, HRegionInfo[ ] ) has been removed from this interface. A client program may be interrupted by NoSuchMethodErrorexception.
Abstract method voidpreDeleteSnapshot ( ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription ) has been removed from this interface. A client program may be interrupted by NoSuchMethodErrorexception.
Abstract method voidpreGetTableDescriptors ( ObserverContext<MasterCoprocessorEnvironment>, List<TableName>, List<HTableDescriptor> ) has been removed from this interface. A client program may be interrupted by NoSuchMethodErrorexception.
Abstract method voidpreModifyTable ( ObserverContext<MasterCoprocessorEnvironment>, TableName, HTableDescriptor ) has been removed from this interface. A client program may be interrupted by NoSuchMethodErrorexception.
Abstract method voidpreRestoreSnapshot ( ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface. A client program may be interrupted by NoSuchMethodErrorexception.
Abstract method voidpreSnapshot ( ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface. A client program may be interrupted by NoSuchMethodErrorexception.

RegionObserver interface changes

The following changes are introduced to the RegionObserver interface.

[−] interface RegionObserver (13)
Change Result
Abstract method voidpostCloseRegionOperation ( ObserverContext<RegionCoprocessorEnvironment>, HRegion.Operation ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method voidpostCompactSelection ( ObserverContext<RegionCoprocessorEnvironment>, Store, ImmutableList<StoreFile> ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method voidpostCompactSelection ( ObserverContext<RegionCoprocessorEnvironment>, Store, ImmutableList<StoreFile>, CompactionRequest ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method voidpostGetClosestRowBefore ( ObserverContext<RegionCoprocessorEnvironment>, byte[ ], byte[ ], Result ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method DeleteTrackerpostInstantiateDeleteTracker ( ObserverContext<RegionCoprocessorEnvironment>, DeleteTracker ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method voidpostSplit ( ObserverContext<RegionCoprocessorEnvironment>, HRegion, HRegion ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method voidpostStartRegionOperation ( ObserverContext<RegionCoprocessorEnvironment>, HRegion.Operation ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method StoreFile.ReaderpostStoreFileReaderOpen ( ObserverContext<RegionCoprocessorEnvironment>, FileSystem, Path, FSDataInputStreamWrapper, long, CacheConfig, Reference, StoreFile.Reader ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method voidpostWALRestore ( ObserverContext<RegionCoprocessorEnvironment>, HRegionInfo, HLogKey, WALEdit ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method InternalScannerpreFlushScannerOpen ( ObserverContext<RegionCoprocessorEnvironment>, Store, KeyValueScanner, InternalScanner ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method voidpreGetClosestRowBefore ( ObserverContext<RegionCoprocessorEnvironment>, byte[ ], byte[ ], Result ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method StoreFile.ReaderpreStoreFileReaderOpen ( ObserverContext<RegionCoprocessorEnvironment>, FileSystem, Path, FSDataInputStreamWrapper, long, CacheConfig, Reference, StoreFile.Reader ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method voidpreWALRestore ( ObserverContext<RegionCoprocessorEnvironment>, HRegionInfo, HLogKey, WALEdit ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.

WALObserver interface changes

The following changes are introduced to the WALObserver interface.

[−] interface WALObserver
Change Result
Abstract method voidpostWALWrite ( ObserverContext<WALCoprocessorEnvironment>, HRegionInfo, HLogKey, WALEdit ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method booleanpreWALWrite ( ObserverContext<WALCoprocessorEnvironment>, HRegionInfo, HLogKey, WALEdit ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.

Scheduler changes

Following methods are now changed to abstract.

[−]class RpcScheduler (1)

Change Result
Abstract method void dispatch ( CallRunner ) has been removed from this class. A client program may be interrupted by NoSuchMethodError exception.

hbase-server-1.0.0.jar, RpcScheduler.class package org.apache.hadoop.hbase.ipc

[−] RpcScheduler.dispatch ( CallRunner p1 ) [abstract] : void 1

org/apache/hadoop/hbase/ipc/RpcScheduler.dispatch:(Lorg/apache/hadoop/hbase/ipc/CallRunner;)V

Change Result
Return value type has been changed from void to boolean. This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.

The following abstract methods have been removed.

[−]interface PriorityFunction (2)
Change Result
Abstract method longgetDeadline ( RPCProtos.RequestHeader, Message ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method int getPriority ( RPCProtos.RequestHeader, Message ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.

Server API changes

[−] class RpcServer (12)

Change Result
Type of field CurCall has been changed from java.lang.ThreadLocal<RpcServer.Call> to java.lang.ThreadLocal<RpcCall>. A client program may be interrupted by NoSuchFieldError exception.
This class became abstract. A client program may be interrupted by InstantiationError exception.
Abstract method int getNumOpenConnections ( ) has been added to this class. This class became abstract and a client program may be interrupted by InstantiationError exception.
Field callQueueSize of type org.apache.hadoop.hbase.util.Counter has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field connectionList of type java.util.List<RpcServer.Connection> has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field maxIdleTime of type int has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field numConnections of type int has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field port of type int has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field purgeTimeout of type long has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field responder of type RpcServer.Responder has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field socketSendBufferSize of type int has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field thresholdIdleConnections of type int has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.

Following abstract method has been removed:

Change Result
Abstract method Pair<Message,CellScanner>call ( BlockingService, Descriptors.MethodDescriptor, Message, CellScanner, long, MonitoredRPCHandler ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.

Replication and WAL changes

HBASE-18733: WALKey has been purged completely. Following are the changes to the WALKey:

[−] classWALKey (8)
Change Result
Access level of field clusterIds has been changed from protected to private. A client program may be interrupted by IllegalAccessError exception.
Access level of field compressionContext has been changed from protected to private. A client program may be interrupted by IllegalAccessError exception.
Access level of field encodedRegionName has been changed from protected to private. A client program may be interrupted by IllegalAccessError exception.
Access level of field tablename has been changed from protectedto private. A client program may be interrupted by IllegalAccessError exception.
Access level of field writeTime has been changed from protectedto private. A client program may be interrupted by IllegalAccessError exception.

Following fields have been removed:

Change Result
Field LOG of type org.apache.commons.logging.Log has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field VERSION of type WALKey.Version has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field logSeqNum of type long has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.

Following are the changes to the WALEdit.class: hbase-server-1.0.0.jar, WALEdit.class package org.apache.hadoop.hbase.regionserver.wal

Deprecated APIs or coprocessor

HBASE-16769 - Protocol Buffers (PB) references from MasterObserver and RegionServerObserver has been removed.

Admin Interface API changes

You cannot administer a CDP Runtime Data Hub cluster using a client that includes RelicationAdmin, ACC, Thrift and REST usage of Admin ops. Methods returning protobufs have been changed to return POJOs instead. Returns have changed from void to Future for async methods. HBASE-18106 - Admin.listProcedures and Admin.listLocks were renamed to getProcedures and getLocks. MapReduce makes use of Admin doing following admin.getClusterStatus() to calcluate Splits.

  • Thrift usage of Admin API:
    compact(ByteBuffer) createTable(ByteBuffer, List<ColumnDescriptor>) deleteTable(ByteBuffer) disableTable(ByteBuffer) 
    enableTable(ByteBuffer) getTableNames() majorCompact(ByteBuffer)
  • REST usage of Admin API:
    hbase-rest org.apache.hadoop.hbase.rest RootResource getTableList() TableName[] tableNames = servlet.getAdmin().listTableNames(); 
    SchemaResource delete(UriInfo) Admin admin = servlet.getAdmin(); update(TableSchemaModel, boolean, UriInfo) Admin admin = servlet.getAdmin(); 
    StorageClusterStatusResource get(UriInfo) ClusterStatus status = servlet.getAdmin().getClusterStatus(); StorageClusterVersionResource get(UriInfo) model.setVersion(servlet.getAdmin().getClusterStatus().getHBaseVersion()); TableResource exists() return servlet.getAdmin().tableExists(TableName.valueOf(table));

[−] interface Admin (9)

Following are the changes to the Admin interface:

Change Result
Abstract method createTableAsync ( HTableDescriptor, byte[ ][ ] ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method disableTableAsync ( TableName ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method enableTableAsync ( TableName ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method getCompactionState ( TableName ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method getCompactionStateForRegion ( byte[ ] ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method isSnapshotFinished ( HBaseProtos.SnapshotDescription ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method snapshot ( String, TableName, HBaseProtos.SnapshotDescription.Type ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method snapshot ( HBaseProtos.SnapshotDescription ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method takeSnapshotAsync ( HBaseProtos.SnapshotDescription ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.

[−] Admin.createTableAsync ( HTableDescriptor p1, byte[ ][ ] p2 ) [abstract] : void 1

org/apache/hadoop/hbase/client/Admin.createTableAsync:(Lorg/apache/hadoop/hbase/HTableDescriptor;[[B)V

Change Result
Return value type has been changed from void to java.util.concurrent.Future<java.lang.Void>. This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.

[−] Admin.disableTableAsync ( TableName p1 ) [abstract] : void 1

org/apache/hadoop/hbase/client/Admin.disableTableAsync:(Lorg/apache/hadoop/hbase/TableName;)V

Change Result
Return value type has been changed from void to java.util.concurrent.Future<java.lang.Void>. This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.

Admin.enableTableAsync ( TableName p1 ) [abstract] : void 1

org/apache/hadoop/hbase/client/Admin.enableTableAsync:(Lorg/apache/hadoop/hbase/TableName;)V

Change Result
Return value type has been changed from void to java.util.concurrent.Future<java.lang.Void>. This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.

Admin.enableTableAsync ( TableName p1 ) [abstract] : void 1

org/apache/hadoop/hbase/client/Admin.getCompactionState:(Lorg/apache/hadoop/hbase/TableName;)Lorg/apache/hadoop/hbase/protobuf/generated/AdminProtos$GetRegionInfoResponse$CompactionState;

Change Result
Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState to CompactionState. This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.

[−] Admin.getCompactionStateForRegion ( byte[ ] p1 ) [abstract] : AdminProtos.GetRegionInfoResponse.CompactionState 1

org/apache/hadoop/hbase/client/Admin.getCompactionStateForRegion:([B)Lorg/apache/hadoop/hbase/protobuf/generated/AdminProtos$GetRegionInfoResponse$CompactionState;

Change Result
Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState to CompactionState. This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.

HTableDescriptor and HColumnDescriptor changes

HTableDescriptor and HColumnDescriptor has become interfaces and you can create it through Builders. HCD has become CFD. It no longer implements writable interface. package org.apache.hadoop.hbase[−] class HColumnDescriptor (1)
Change Result
Removed super-interface org.apache.hadoop.io.WritableComparable<HColumnDescriptor>. A client program may be interrupted by NoSuchMethodError exception.

HColumnDescriptor in 1.0.0 {code} @InterfaceAudience.Public @InterfaceStability.Evolving public class HColumnDescriptor implements WritableComparable<HColumnDescriptor> { {code}

HColumnDescriptor in 2.0 {code} @InterfaceAudience.Public @Deprecated // remove it in 3.0 public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HColumnDescriptor> { {code}

For META_TABLEDESC, the maker method had been deprecated already in HTD in 1.0.0. OWNER_KEY is still in HTD.

class HTableDescriptor (3)

hbase-client-1.0.0.jar, HTableDescriptor.class package org.apache.hadoop.hbase

Change Result
Removed super-interface org.apache.hadoop.io.WritableComparable<HTableDescriptor>. A client program may be interrupted by NoSuchMethodError exception.
Field META_TABLEDESC of type HTableDescriptor has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.

[−] HTableDescriptor.getColumnFamilies ( ) : HColumnDescriptor[ ] (1)

org/apache/hadoop/hbase/HTableDescriptor.getColumnFamilies:()[Lorg/apache/hadoop/hbase/HColumnDescriptor;[−] class HColumnDescriptor (1)
Change Result
Return value type has been changed from HColumnDescriptor[]to client.ColumnFamilyDescriptor[]. This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
[−] interface Table (4)
Change Result
Abstract method batch ( List<?> ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method batchCallback ( List<?>, Batch.Callback<R> )has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method getWriteBufferSize ( ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method setWriteBufferSize ( long ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.

Deprecated buffer methods

  • HBASE-13298- Clarify if Table.{set|get}WriteBufferSize() is deprecated or not.

  • LockTimeoutException and OperationConflictException classes have been removed.

class OperationConflictException (1)

Change Result
This class has been removed. A client program may be interrupted by NoClassDefFoundErrorexception.

class class LockTimeoutException (1)

Change Result This class has been removed. A client program may be interrupted by NoClassDefFoundErrorexception.

Filter API changes

Following methods have been removed: package org.apache.hadoop.hbase.filter

[−] class Filter (2)
Change Result
Abstract method getNextKeyHint ( KeyValue ) has been removed from this class. A client program may be interrupted by NoSuchMethodError exception.
Abstract method transform ( KeyValue ) has been removed from this class. A client program may be interrupted by NoSuchMethodError exception.
  • HBASE-12296 Filters should work with ByteBufferedCell.

  • HConnection is removed in HBase 2.0.

  • RegionLoad and ServerLoad internally moved to shaded PB.

[−] class RegionLoad (1)
Change Result
Type of field regionLoadPB has been changed from protobuf.generated.ClusterStatusProtos.RegionLoad to shaded.protobuf.generated.ClusterStatusProtos.RegionLoad. A client program may be interrupted by NoSuchFieldError exception.

HBASE-15783:AccessControlConstants#OP_ATTRIBUTE_ACL_STRATEGY_CELL_FIRST is not used any more. package org.apache.hadoop.hbase.security.access

[−] interface AccessControlConstants (3)
Change Result
Field OP_ATTRIBUTE_ACL_STRATEGY of type java.lang.Stringhas been removed from this interface. A client program may be interrupted by NoSuchFieldError exception.
Field OP_ATTRIBUTE_ACL_STRATEGY_CELL_FIRST of type byte[] has been removed from this interface. A client program may be interrupted by NoSuchFieldError exception.
Field OP_ATTRIBUTE_ACL_STRATEGY_DEFAULT of type byte[] has been removed from this interface. A client program may be interrupted by NoSuchFieldError exception.
ServerLoad returns long instead of int 1

hbase-client-1.0.0.jar, ServerLoad.class package org.apache.hadoop.hbase

[−] ServerLoad.getNumberOfRequests ( ) : int 1

org/apache/hadoop/hbase/ServerLoad.getNumberOfRequests:()I

Change Result
Return value type has been changed from int to long. This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.

[−] ServerLoad.getNumberOfRequests ( ) : int 1

org/apache/hadoop/hbase/ServerLoad.getReadRequestsCount:()I

Change Result
Return value type has been changed from int to long. This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.

b[−] ServerLoad.getTotalNumberOfRequests ( ) : int 1

org/apache/hadoop/hbase/ServerLoad.getTotalNumberOfRequests:()I

Change Result
Return value type has been changed from int to long. This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.

[−]ServerLoad.getWriteRequestsCount ( ) : int 1

org/apache/hadoop/hbase/ServerLoad.getWriteRequestsCount:()I

Change Result
Return value type has been changed from int to long. This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
  • HBASE-13636 Remove deprecation for HBASE-4072 (Reading of zoo.cfg)

  • HConstants are removed. HBASE-16040 Remove configuration "hbase.replication"

[−]class HConstants (6)

Change Result
Field DEFAULT_HBASE_CONFIG_READ_ZOOKEEPER_CONFIG of type boolean has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field HBASE_CONFIG_READ_ZOOKEEPER_CONFIG of type java.lang.String has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field REPLICATION_ENABLE_DEFAULT of type boolean has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field REPLICATION_ENABLE_KEY of type java.lang.String has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field ZOOKEEPER_CONFIG_NAME of type java.lang.String has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.
Field ZOOKEEPER_USEMULTI of type java.lang.String has been removed from this class. A client program may be interrupted by NoSuchFieldError exception.

HBASE-18732: [compat 1-2] HBASE-14047 removed Cell methods without deprecation cycle.

[−]interface Cell 5

Change Result
Abstract method getFamily ( ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method getMvccVersion ( ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method getQualifier ( ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method getRow ( ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.
Abstract method getValue ( ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.

HBASE-18795:Expose KeyValue.getBuffer() for tests alone. Allows KV#getBuffer in tests only that was deprecated previously.

Region scanner changes

[−]interface RegionScanner (1)
Change Result
Abstract method boolean nextRaw ( List<Cell>, int ) has been removed from this interface. A client program may be interrupted by NoSuchMethodError exception.

StoreFile changes

[−] class StoreFile (1)
Change Result
This class became interface. A client program may be interrupted by IncompatibleClassChangeError or InstantiationError exception dependent on the usage of this class.

Mapreduce changes

HFile*Format has been removed.

ClusterStatus changes

HBASE-15843: Replace RegionState.getRegionInTransition() Map with a Set hbase-client-1.0.0.jar, ClusterStatus.class package org.apache.hadoop.hbase

[−] ClusterStatus.getRegionsInTransition ( ) : Map<String,RegionState> 1

org/apache/hadoop/hbase/ClusterStatus.getRegionsInTransition:()Ljava/util/Map;

Change Result
Return value type has been changed from java.util.Map<java.lang.String,master.RegionState> to java.util.List<master.RegionState>. This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.

Other changes in ClusterStatus include removal of convert methods that were no longer necessary after purge of PB from API.

Purge of Protocol Buffers from API

Protocol Buffers (PB) has been deprecated in APIs.

[−] HBaseSnapshotException.getSnapshotDescription ( ) : HBaseProtos.SnapshotDescription 1

org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.getSnapshotDescription:()Lorg/apache/hadoop/hbase/protobuf/generated/HBaseProtos$SnapshotDescription;
Change Result
Return value type has been changed from
 org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription to org.apache.hadoop.hbase.client.SnapshotDescription
.
This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.

HBASE-15609: Remove PB references from Result, DoubleColumnInterpreter and any such public facing class for 2.0. hbase-client-1.0.0.jar, Result.class package org.apache.hadoop.hbase.client

[−] Result.getStats ( ) : ClientProtos.RegionLoadStats 1

org/apache/hadoop/hbase/client/Result.getStats:()Lorg/apache/hadoop/hbase/protobuf/generated/ClientProtos$RegionLoadStats;

Change Result
Return value type has been changed from
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStats to RegionLoadStats
.
This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.

PrettyPrinter changes

hbase-server-1.0.0.jar, HFilePrettyPrinter.class package org.apache.hadoop.hbase.io.hfile

Change Result
Return value type has been changed from void to int. This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.

Technical Service Bulletins

TSB 2021-494: Accumulated WAL Files Cannot be Cleaned up When Using Phoenix Secondary Global Indexes
The Write-ahead-log (WAL) files for Phoenix tables that have secondary global indexes defined on them, cannot be automatically cleaned up by HBase, leading to excess storage usage and possible error due to filling up the storage. Accumulated WAL files can lead to lengthy restart times as they must all be played back to ensure no dataloss occurs on restart. This can have follow-on HDFS impact if the number of WAL files overwhelm HDFS Name Node.
Upstream JIRA
Knowledge article
For the latest update on this issue see the corresponding Knowledge article: TSB 2021-494: Accumulated WAL Files Cannot be Cleaned up When Using Phoenix Secondary Global Indexes
TSB 2021-506: Active HBase MOB files can be removed
Actively used MOB files can be deleted by MobFileCleanerChore due to incorrect serialization of reference file names. This is causing data loss on MOB-enabled tables.
Upstream JIRA
Knowledge article
For the latest update on this issue see the corresponding Knowledge article: TSB 2021-506: Active HBase MOB files can be removed
TSB 2022-569: HBase normalizer can cause table inconsistencies by merging non-adjacent regions
The normalizer in HBase is a background job responsible for splitting or merging HBase regions to optimize the number of regions and the distribution of the size of the regions in HBase tables. Due to the bug described in HBASE-24376, the normalizer can cause region inconsistencies (region overlaps/holes) by merging non-adjacent regions.
Upstream JIRA
HBASE-24376
Knowledge article
For the latest update on this issue, see the corresponding Knowledge article: TSB 2022-569: HBase normalizer can cause table inconsistencies by merging non-adjacent regions