You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Jonathan Hsieh (JIRA)" <ji...@apache.org> on 2013/09/14 01:42:52 UTC

[jira] [Comment Edited] (HBASE-9529) Audit of hbase-client @InterfaceAudience.Public apis

    [ https://issues.apache.org/jira/browse/HBASE-9529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13767180#comment-13767180 ] 

Jonathan Hsieh edited comment on HBASE-9529 at 9/13/13 11:41 PM:
-----------------------------------------------------------------

tl;dr
* Make coproc api private (eventually limited private)
* All Exceptions Public
* Action, Caller* , Callable* Private
* Expose RegionLoad/RegionState because ClusterStats/ServerLoad are public
* All filters public except for FilterWrapper (private), and maybe FilterBase (eventually LimitedPrivate)
* All security private except for exceptions (they are all managed via configuration, never client code)
* Only expose HQuorumPeer (standalone zk) in zookeeper package, privatize all others (internals)
* BloomType, Compression, DataBlockEncoding  (in hbase-common) need to be public for HColumnDescriptor

Legend:

\* make Private 
! make Limited Private, but make Private in the mean time.
+ unmarked or Private and make Public 
If missing assume will add private.

= hbase-client

@Public org.apache.hadoop.hbase

Coprocessor ! 
CoprocessorEnvironment !

ClusterStatus // needs RegionState public
HColumnDescriptor // need to open up Compression enum in hbase-common.
HRegionInfo
HRegionLocation 
HTableDescriptor 
RegionLoad  + // exposed by ServerLoad
RegionState + // exposed by ClusterStatus
ServerLoad
ServerName

ClockOutOfSyncException
DoNotRetryIOException
DroppedSnapshotException
InvalidFamilyOperationException
MasterNotRunningException
NotAllMetaRegionsOnlineException
NotServingRegionException
PleaseHoldException
RegionException
RegionTooBusyException
TableExistsException
TableInfoMissingException + // should be public
TableNotDisabledException
TableNotEnabledException
TableNotFoundException
UnknownRegionException
UnknownScannerException
YouAreDeadException
ZooKeeperConnectionException

@Public o.a.h.h.catalog // all private

@Public o.a.h.h.client

Attributes
HConnection
HTableInterface // mark coprocessorService method Private
HTableInterfaceFactory
ResultScanner
RetryingCallable *
Row 

AbstractClientScanner * 
Action * // ?? I think this is internal
Append
ClientScanner
ConnectionUtils * 
Delete
Get
HBaseAdmin
HConnectable * (currently no mark)
HConnectionManager  // hide  #execute which takes an Hconnectable
HTable
HTableFactory
HTableMultiplexer
HTablePool * // marked deprecated
HTableUtil // is this user code or internals? mentione in docs, keep public 
Increment 
MultiAction * // I think this is internal to 
MultiResponse
Mutation
Operation
OperationWithAttributes
Put
RegionServerCallable *
Result
RowMutations
RpcRetryingCaller * 
Scan // exposes IsolationLevel, TimeRange (time range is assymetric -- set(int,int), but returns TimeRange.  Add setTimeRange(TimeRange)
ScannerCallable * 
UnmodifyableHTableDescriptor 

Durability // exposed by Mutation
IsolationLevel

NoServerForRegionException
RegionOfflineException
RetriesExhaustedException
RetriesExhaustedWithDetailsException
ScannerTimeoutException
WrongRowIOException *

@Public o.a.h.h.client.coprocessor   // hiding this since is is kind of a mess.
Batch.Call *
Batch.Callback *
AggregationClient * // TODO all this seems to expose raw protobuf messages, which is weird.  not sure what the righ way to use this is.
Batch *
BigDecimalColumnInterpreter * 
LongColumnInterpreter * 
SecureBulkLoadClient * // need help here -- do clients directly use this? if no answer make private

@Public o.a.h.h.client.metrics
ScanMetrics // where is this exposed?

@Public o.a.h.h.client.replication
ReplicationAdmin +  /// TODO double check with jd

@Public o.a.h.h.coprocessor
ColumnInterpreter // javadoc has soms strange strike through.  why public?
CoprocessorException 

@Public o.a.h.h.executor // all private

@Public o.a.h.h.filter // all public, 
BinaryComparator
BinaryPrefixComparator
BitComparator
ByteArrayComparable
ColumnCountGetFilter
ColumnPaginationFilter
ColumnPrefixFilter
ColumnRangeFilter
CompareFilter
DependentColumnFilter
FamilyFilter
Filter
FilterBase // maybe make private, eventually limited private
FilterList 
FilterWrapper // private
FirstKeyOnlyFilter
FirstKeyValueMatchingQualifiersFilter
FuzzyRowFilter
InclusiveStopFilter
KeyOnlyFilter
MultipleColumnPrefixFilter
NullComparator
PageFilter
ParseConstants
ParseFilter
PrefixFilter
QualifierFilter
RandomRowFilter
RegexStringComparator
RowFilter
SingleColumnValueExcludeFilter
SingleColumnValueFilter
SkipFilter
SubstringComparator
TimestampsFilter
ValueFilter
WhileMatchFilter

BitComparator.BitwiseOp + // these static inners should be exposed, this is part of BitComparator
CompareFilter.CompareOp + 
Filter.ReturnCode + // exposed as part of Filter api.
FilterList.Operator + 

IncompatibleFilterException
InvalidRowFilterException

@Public o.a.h.h.ipc
BlockingRpcCallback * 

BadAuthException +
CallerDisconnectedException +
FatalConnectionException +
RemoteWithExtrasException +
RpcClient.CallTimeoutException +
RpcClient.FailedServerException +
ServerNotRunningYetException +
StoppedRpcClientException +
UnsupportedCellCodecException +
UnsupportedCompressionCodecException +
WrongVersionException +

@Public o.a.h.h.master
RegionState + 

@Public o.a.h.h.protobuf // all private

@Public o.a.h.h.regionserver 
BloomType // expose for HColumnDescription

LeaseException +
NoSuchColumnFamilyException + 
RegionAlreadyInTransitionException
RegionServerRunningException + 
RegionServerStoppedException +
WrongRegionException 

@Public o.a.h.h.regionserver.wal
FailedLogCloseException +

@Public o.a.h.h.replication // all private, looks good 


@Public o.a.h.h.security // make all Private  except for exception (talked to jimmy)
AccessDeniedException +
@Public o.a.h.h.security.access // make all Private 
@Public o.a.h.h.security.token // make all Private

@Public o.a.h.h.snapshot
CorruptedSnapshotException
ExportSnapshotException
HBaseSnapshotException
RestoreSnapshotException +
SnapshotCreationException + 
SnapshotDoesNotExistException 
SnapshotExistsException
TablePartiallyOpenException
UnknownSnapshotException

@Public o.a.h.h.util
PoolMap.Pool *
PoolMap * 
Sleeper * 
PoolMap.PoolType *
FileSystemVersionException

@Public o.a.h.h.zookeeper
HQuorumPeer

RecoverableZooKeeper *
ZKAssign *
ZKConfig *
ZKLeaderManager * 
ZKTable *
ZKTableReadOnly *
ZKUtil *
ZKUtil.NodeAndData *
ZKUtil.ZKUtilOp *
ZKUtil.ZKUtilOp.CreateAndFailSilent *
ZKUtil.ZKUtilOp.DeleteNodeFailSilent *
ZKUtil.ZKUtilOp.SetData *
ZooKeeperListener *
ZooKeeperNodeTracker *
ZooKeeperWatcher *




                
      was (Author: jmhsieh):
    tl;dr
* Make coproc api private (eventually limited private)
* All Exceptions Public
* Action, Caller* , Callable* Private
* Expose RegionLoad/RegionState because ClusterStats/ServerLoad are public
* All filters public except for FilterWrapper (private), and maybe FilterBase (eventually LimitedPrivate)
* All security private except for exceptions (they are all managed via configuration, never client code)
* Only expose HQuorumPeer (standalone zk) in zookeeper package, privatize all others (internals)
* BloomType, Compression, DataBlockEncoding  (in hbase-common) need to be public for HColumnDescriptor

Legend:

* make Private 
! make Limited Private, but make Private in the mean time.
+ unmarked or Private and make Public 
If missing assume will add private.

= hbase-client

@Public org.apache.hadoop.hbase

Coprocessor ! 
CoprocessorEnvironment !

ClusterStatus // needs RegionState public
HColumnDescriptor // need to open up Compression enum in hbase-common.
HRegionInfo
HRegionLocation 
HTableDescriptor 
RegionLoad  + // exposed by ServerLoad
RegionState + // exposed by ClusterStatus
ServerLoad
ServerName

ClockOutOfSyncException
DoNotRetryIOException
DroppedSnapshotException
InvalidFamilyOperationException
MasterNotRunningException
NotAllMetaRegionsOnlineException
NotServingRegionException
PleaseHoldException
RegionException
RegionTooBusyException
TableExistsException
TableInfoMissingException + // should be public
TableNotDisabledException
TableNotEnabledException
TableNotFoundException
UnknownRegionException
UnknownScannerException
YouAreDeadException
ZooKeeperConnectionException

@Public o.a.h.h.catalog // all private

@Public o.a.h.h.client

Attributes
HConnection
HTableInterface // mark coprocessorService method Private
HTableInterfaceFactory
ResultScanner
RetryingCallable *
Row 

AbstractClientScanner * 
Action * // ?? I think this is internal
Append
ClientScanner
ConnectionUtils * 
Delete
Get
HBaseAdmin
HConnectable * (currently no mark)
HConnectionManager  // hide  #execute which takes an Hconnectable
HTable
HTableFactory
HTableMultiplexer
HTablePool * // marked deprecated
HTableUtil // is this user code or internals? mentione in docs, keep public 
Increment 
MultiAction * // I think this is internal to 
MultiResponse
Mutation
Operation
OperationWithAttributes
Put
RegionServerCallable *
Result
RowMutations
RpcRetryingCaller * 
Scan // exposes IsolationLevel, TimeRange (time range is assymetric -- set(int,int), but returns TimeRange.  Add setTimeRange(TimeRange)
ScannerCallable * 
UnmodifyableHTableDescriptor 

Durability // exposed by Mutation
IsolationLevel

NoServerForRegionException
RegionOfflineException
RetriesExhaustedException
RetriesExhaustedWithDetailsException
ScannerTimeoutException
WrongRowIOException *

@Public o.a.h.h.client.coprocessor   // hiding this since is is kind of a mess.
Batch.Call *
Batch.Callback *
AggregationClient * // TODO all this seems to expose raw protobuf messages, which is weird.  not sure what the righ way to use this is.
Batch *
BigDecimalColumnInterpreter * 
LongColumnInterpreter * 
SecureBulkLoadClient * // need help here -- do clients directly use this? if no answer make private

@Public o.a.h.h.client.metrics
ScanMetrics // where is this exposed?

@Public o.a.h.h.client.replication
ReplicationAdmin +  /// TODO double check with jd

@Public o.a.h.h.coprocessor
ColumnInterpreter // javadoc has soms strange strike through.  why public?
CoprocessorException 

@Public o.a.h.h.executor // all private

@Public o.a.h.h.filter // all public, 
BinaryComparator
BinaryPrefixComparator
BitComparator
ByteArrayComparable
ColumnCountGetFilter
ColumnPaginationFilter
ColumnPrefixFilter
ColumnRangeFilter
CompareFilter
DependentColumnFilter
FamilyFilter
Filter
FilterBase // maybe make private, eventually limited private
FilterList 
FilterWrapper // private
FirstKeyOnlyFilter
FirstKeyValueMatchingQualifiersFilter
FuzzyRowFilter
InclusiveStopFilter
KeyOnlyFilter
MultipleColumnPrefixFilter
NullComparator
PageFilter
ParseConstants
ParseFilter
PrefixFilter
QualifierFilter
RandomRowFilter
RegexStringComparator
RowFilter
SingleColumnValueExcludeFilter
SingleColumnValueFilter
SkipFilter
SubstringComparator
TimestampsFilter
ValueFilter
WhileMatchFilter

BitComparator.BitwiseOp + // these static inners should be exposed, this is part of BitComparator
CompareFilter.CompareOp + 
Filter.ReturnCode + // exposed as part of Filter api.
FilterList.Operator + 

IncompatibleFilterException
InvalidRowFilterException

@Public o.a.h.h.ipc
BlockingRpcCallback * 

BadAuthException +
CallerDisconnectedException +
FatalConnectionException +
RemoteWithExtrasException +
RpcClient.CallTimeoutException +
RpcClient.FailedServerException +
ServerNotRunningYetException +
StoppedRpcClientException +
UnsupportedCellCodecException +
UnsupportedCompressionCodecException +
WrongVersionException +

@Public o.a.h.h.master
RegionState + 

@Public o.a.h.h.protobuf // all private

@Public o.a.h.h.regionserver 
BloomType // expose for HColumnDescription

LeaseException +
NoSuchColumnFamilyException + 
RegionAlreadyInTransitionException
RegionServerRunningException + 
RegionServerStoppedException +
WrongRegionException 

@Public o.a.h.h.regionserver.wal
FailedLogCloseException +

@Public o.a.h.h.replication // all private, looks good 


@Public o.a.h.h.security // make all Private  except for exception (talked to jimmy)
AccessDeniedException +
@Public o.a.h.h.security.access // make all Private 
@Public o.a.h.h.security.token // make all Private

@Public o.a.h.h.snapshot
CorruptedSnapshotException
ExportSnapshotException
HBaseSnapshotException
RestoreSnapshotException +
SnapshotCreationException + 
SnapshotDoesNotExistException 
SnapshotExistsException
TablePartiallyOpenException
UnknownSnapshotException

@Public o.a.h.h.util
PoolMap.Pool *
PoolMap * 
Sleeper * 
PoolMap.PoolType *
FileSystemVersionException

@Public o.a.h.h.zookeeper
HQuorumPeer

RecoverableZooKeeper *
ZKAssign *
ZKConfig *
ZKLeaderManager * 
ZKTable *
ZKTableReadOnly *
ZKUtil *
ZKUtil.NodeAndData *
ZKUtil.ZKUtilOp *
ZKUtil.ZKUtilOp.CreateAndFailSilent *
ZKUtil.ZKUtilOp.DeleteNodeFailSilent *
ZKUtil.ZKUtilOp.SetData *
ZooKeeperListener *
ZooKeeperNodeTracker *
ZooKeeperWatcher *




                  
> Audit of hbase-client @InterfaceAudience.Public apis
> ----------------------------------------------------
>
>                 Key: HBASE-9529
>                 URL: https://issues.apache.org/jira/browse/HBASE-9529
>             Project: HBase
>          Issue Type: Sub-task
>            Reporter: Jonathan Hsieh
>             Fix For: 0.98.0, 0.96.0
>
>
> Similar to HBASE-9523, let's do an audit of the hbase-client public api.  This is easier to do now that the we can publish only the public api javadoc 
> http://hbase.apache.org/apidocs/  (notice it only has Public apis now!)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira