github.com/tsuna/gohbase@v0.0.0-20250731002811-4ffcadfba63e/client.go (about) 1 // Copyright (C) 2015 The GoHBase Authors. All rights reserved. 2 // This file is part of GoHBase. 3 // Use of this source code is governed by the Apache License 2.0 4 // that can be found in the COPYING file. 5 6 package gohbase 7 8 import ( 9 "context" 10 "encoding/binary" 11 "encoding/json" 12 "fmt" 13 "log/slog" 14 "net" 15 "sync" 16 "time" 17 18 "github.com/tsuna/gohbase/compression" 19 "github.com/tsuna/gohbase/hrpc" 20 "github.com/tsuna/gohbase/pb" 21 "github.com/tsuna/gohbase/region" 22 "github.com/tsuna/gohbase/zk" 23 "google.golang.org/protobuf/proto" 24 "modernc.org/b/v2" 25 ) 26 27 const ( 28 defaultRPCQueueSize = 100 29 defaultFlushInterval = 20 * time.Millisecond 30 defaultZkRoot = "/hbase" 31 defaultZkTimeout = 30 * time.Second 32 defaultEffectiveUser = "root" 33 ) 34 35 // Client a regular HBase client 36 type Client interface { 37 Scan(s *hrpc.Scan) hrpc.Scanner 38 Get(g *hrpc.Get) (*hrpc.Result, error) 39 Put(p *hrpc.Mutate) (*hrpc.Result, error) 40 Delete(d *hrpc.Mutate) (*hrpc.Result, error) 41 Append(a *hrpc.Mutate) (*hrpc.Result, error) 42 Increment(i *hrpc.Mutate) (int64, error) 43 CheckAndPut(p *hrpc.Mutate, family string, qualifier string, 44 expectedValue []byte) (bool, error) 45 SendBatch(ctx context.Context, batch []hrpc.Call) (res []hrpc.RPCResult, allOK bool) 46 CacheRegions(table []byte) error 47 Close() 48 } 49 50 // RPCClient is core client of gohbase. It's exposed for testing. 51 type RPCClient interface { 52 SendRPC(rpc hrpc.Call) (proto.Message, error) 53 } 54 55 // Option is a function used to configure optional config items for a Client. 56 type Option func(*client) 57 58 // A Client provides access to an HBase cluster. 59 type client struct { 60 clientType region.ClientType 61 62 regions keyRegionCache 63 64 // Maps a hrpc.RegionInfo to the *region.Client that we think currently 65 // serves it. 66 clients clientRegionCache 67 68 metaRegionInfo hrpc.RegionInfo 69 70 adminRegionInfo hrpc.RegionInfo 71 72 // The maximum size of the RPC queue in the region client 73 rpcQueueSize int 74 75 // zkClient is zookeeper for retrieving meta and admin information 76 zkClient zk.Client 77 78 // The root zookeeper path for Hbase. By default, this is usually "/hbase". 79 zkRoot string 80 81 // The zookeeper session timeout 82 zkTimeout time.Duration 83 84 // The timeout before flushing the RPC queue in the region client 85 flushInterval time.Duration 86 87 // The user used when accessing regions. 88 effectiveUser string 89 90 // How long to wait for a region lookup (either meta lookup or finding 91 // meta in ZooKeeper). Should be greater than or equal to the ZooKeeper 92 // session timeout. 93 regionLookupTimeout time.Duration 94 95 // regionReadTimeout is the maximum amount of time to wait for regionserver reply 96 regionReadTimeout time.Duration 97 98 done chan struct{} 99 closeOnce sync.Once 100 101 newRegionClientFn func(string, region.ClientType, int, time.Duration, 102 string, time.Duration, compression.Codec, 103 func(ctx context.Context, network, addr string) (net.Conn, error), 104 *slog.Logger) hrpc.RegionClient 105 106 compressionCodec compression.Codec 107 108 // zkDialer is used in the zkClient to connect to the quorum 109 zkDialer func(ctx context.Context, network, addr string) (net.Conn, error) 110 // regionDialer is passed into the region client to connect to hbase in a custom way, 111 // such as SOCKS proxy. 112 regionDialer func(ctx context.Context, network, addr string) (net.Conn, error) 113 // logger that could be defined by user 114 logger *slog.Logger 115 } 116 117 // NewClient creates a new HBase client. 118 func NewClient(zkquorum string, options ...Option) Client { 119 return newClient(zkquorum, options...) 120 } 121 122 func newClient(zkquorum string, options ...Option) *client { 123 c := &client{ 124 clientType: region.RegionClient, 125 rpcQueueSize: defaultRPCQueueSize, 126 flushInterval: defaultFlushInterval, 127 metaRegionInfo: region.NewInfo( 128 0, 129 []byte("hbase"), 130 []byte("meta"), 131 []byte("hbase:meta,,1"), 132 nil, 133 nil), 134 zkRoot: defaultZkRoot, 135 zkTimeout: defaultZkTimeout, 136 effectiveUser: defaultEffectiveUser, 137 regionLookupTimeout: region.DefaultLookupTimeout, 138 regionReadTimeout: region.DefaultReadTimeout, 139 done: make(chan struct{}), 140 newRegionClientFn: region.NewClient, 141 logger: slog.Default(), 142 } 143 for _, option := range options { 144 option(c) 145 } 146 c.logger.Debug("Creating new client.", "Host", slog.StringValue(zkquorum)) 147 148 //Have to create the zkClient after the Options have been set 149 //since the zkTimeout could be changed as an option 150 c.zkClient = zk.NewClient(zkquorum, c.zkTimeout, c.zkDialer, c.logger) 151 c.regions = keyRegionCache{ 152 logger: c.logger, 153 regions: b.TreeNew[[]byte, hrpc.RegionInfo](region.Compare), 154 } 155 c.clients = clientRegionCache{ 156 logger: c.logger, 157 regions: make(map[hrpc.RegionClient]map[hrpc.RegionInfo]struct{}), 158 } 159 160 return c 161 } 162 163 // DebugState information about the clients keyRegionCache, and clientRegionCache 164 func DebugState(c Client) ([]byte, error) { 165 166 debugInfoJson, err := json.Marshal(c) 167 if err != nil { 168 if cclient, ok := c.(*client); ok { 169 cclient.logger.Error("Cannot turn client into JSON bytes array", "error", err) 170 } else { 171 slog.Error("Cannot turn client into JSON bytes array", "error", err) 172 } 173 } 174 return debugInfoJson, err 175 } 176 177 func (c *client) MarshalJSON() ([]byte, error) { 178 179 var done string 180 if c.done != nil { 181 select { 182 case <-c.done: 183 done = "Closed" 184 default: 185 done = "Not Closed" 186 } 187 } else { 188 done = "nil" 189 } 190 191 rcc := &c.clients 192 krc := &c.regions 193 194 // create map for all ClientRegions (clntRegion Ptr -> JSONified Client Region) 195 clientRegionsMap := map[string]hrpc.RegionClient{} 196 // create map for all RegionInfos (Region Ptr -> JSONified RegionInfo) 197 keyRegionInfosMap := map[string]hrpc.RegionInfo{} 198 199 clientRegionCacheValues := rcc.debugInfo(keyRegionInfosMap, clientRegionsMap) 200 keyRegionCacheValues := krc.debugInfo(keyRegionInfosMap) 201 202 state := struct { 203 ClientType region.ClientType 204 ClientRegionMap map[string]hrpc.RegionClient 205 RegionInfoMap map[string]hrpc.RegionInfo 206 KeyRegionCache map[string]string 207 ClientRegionCache map[string][]string 208 MetaRegionInfo hrpc.RegionInfo 209 AdminRegionInfo hrpc.RegionInfo 210 Done_Status string 211 RegionLookupTimeout time.Duration 212 RegionReadTimeout time.Duration 213 }{ 214 ClientType: c.clientType, 215 ClientRegionMap: clientRegionsMap, 216 RegionInfoMap: keyRegionInfosMap, 217 KeyRegionCache: keyRegionCacheValues, 218 ClientRegionCache: clientRegionCacheValues, 219 MetaRegionInfo: c.metaRegionInfo, 220 AdminRegionInfo: c.adminRegionInfo, 221 Done_Status: done, 222 RegionLookupTimeout: c.regionLookupTimeout, 223 RegionReadTimeout: c.regionReadTimeout, 224 } 225 226 jsonVal, err := json.Marshal(state) 227 return jsonVal, err 228 } 229 230 // RpcQueueSize will return an option that will set the size of the RPC queues 231 // used in a given client 232 func RpcQueueSize(size int) Option { 233 return func(c *client) { 234 c.rpcQueueSize = size 235 } 236 } 237 238 // ZookeeperRoot will return an option that will set the zookeeper root path used in a given client. 239 func ZookeeperRoot(root string) Option { 240 return func(c *client) { 241 c.zkRoot = root 242 } 243 } 244 245 // ZookeeperTimeout will return an option that will set the zookeeper session timeout. 246 func ZookeeperTimeout(to time.Duration) Option { 247 return func(c *client) { 248 c.zkTimeout = to 249 } 250 } 251 252 // RegionLookupTimeout will return an option that sets the region lookup timeout 253 func RegionLookupTimeout(to time.Duration) Option { 254 return func(c *client) { 255 c.regionLookupTimeout = to 256 } 257 } 258 259 // RegionReadTimeout will return an option that sets the region read timeout 260 func RegionReadTimeout(to time.Duration) Option { 261 return func(c *client) { 262 c.regionReadTimeout = to 263 } 264 } 265 266 // EffectiveUser will return an option that will set the user used when accessing regions. 267 func EffectiveUser(user string) Option { 268 return func(c *client) { 269 c.effectiveUser = user 270 } 271 } 272 273 // FlushInterval will return an option that will set the timeout for flushing 274 // the RPC queues used in a given client 275 func FlushInterval(interval time.Duration) Option { 276 return func(c *client) { 277 c.flushInterval = interval 278 } 279 } 280 281 // CompressionCodec will return an option to set compression codec between 282 // client and server. The only currently supported codec is "snappy". 283 // Default is no compression. 284 func CompressionCodec(codec string) Option { 285 return func(c *client) { 286 c.compressionCodec = compression.New(codec) 287 } 288 } 289 290 // ZooKeeperDialer will return an option to pass the given dialer function 291 // into the ZooKeeper client Connect() call, which allows for customizing 292 // network connections. 293 func ZooKeeperDialer(dialer func( 294 ctx context.Context, network, addr string) (net.Conn, error)) Option { 295 return func(c *client) { 296 c.zkDialer = dialer 297 } 298 } 299 300 // RegionDialer will return an option that uses the specified Dialer for 301 // connecting to region servers. This allows for connecting through proxies. 302 func RegionDialer(dialer func( 303 ctx context.Context, network, addr string) (net.Conn, error)) Option { 304 return func(c *client) { 305 c.regionDialer = dialer 306 } 307 } 308 309 // Logger will return an option to set *slog.Logger instance 310 func Logger(logger *slog.Logger) Option { 311 return func(c *client) { 312 c.logger = logger 313 } 314 } 315 316 // Close closes connections to hbase master and regionservers 317 func (c *client) Close() { 318 c.closeOnce.Do(func() { 319 close(c.done) 320 if c.clientType == region.MasterClient { 321 if ac := c.adminRegionInfo.Client(); ac != nil { 322 ac.Close() 323 } 324 } 325 c.clients.closeAll() 326 }) 327 } 328 329 func (c *client) Scan(s *hrpc.Scan) hrpc.Scanner { 330 return newScanner(c, s, c.logger) 331 } 332 333 func (c *client) Get(g *hrpc.Get) (*hrpc.Result, error) { 334 pbmsg, err := c.SendRPC(g) 335 if err != nil { 336 return nil, err 337 } 338 339 r, ok := pbmsg.(*pb.GetResponse) 340 if !ok { 341 return nil, fmt.Errorf("sendRPC returned not a GetResponse") 342 } 343 344 return hrpc.ToLocalResult(r.Result), nil 345 } 346 347 func (c *client) Put(p *hrpc.Mutate) (*hrpc.Result, error) { 348 return c.mutate(p) 349 } 350 351 func (c *client) Delete(d *hrpc.Mutate) (*hrpc.Result, error) { 352 return c.mutate(d) 353 } 354 355 func (c *client) Append(a *hrpc.Mutate) (*hrpc.Result, error) { 356 return c.mutate(a) 357 } 358 359 func (c *client) Increment(i *hrpc.Mutate) (int64, error) { 360 r, err := c.mutate(i) 361 if err != nil { 362 return 0, err 363 } 364 365 if len(r.Cells) != 1 { 366 return 0, fmt.Errorf("increment returned %d cells, but we expected exactly one", 367 len(r.Cells)) 368 } 369 370 val := binary.BigEndian.Uint64(r.Cells[0].Value) 371 return int64(val), nil 372 } 373 374 func (c *client) mutate(m *hrpc.Mutate) (*hrpc.Result, error) { 375 pbmsg, err := c.SendRPC(m) 376 if err != nil { 377 return nil, err 378 } 379 380 r, ok := pbmsg.(*pb.MutateResponse) 381 if !ok { 382 return nil, fmt.Errorf("sendRPC returned not a MutateResponse") 383 } 384 385 return hrpc.ToLocalResult(r.Result), nil 386 } 387 388 func (c *client) CheckAndPut(p *hrpc.Mutate, family string, 389 qualifier string, expectedValue []byte) (bool, error) { 390 cas, err := hrpc.NewCheckAndPut(p, family, qualifier, expectedValue) 391 if err != nil { 392 return false, err 393 } 394 395 pbmsg, err := c.SendRPC(cas) 396 if err != nil { 397 return false, err 398 } 399 400 r, ok := pbmsg.(*pb.MutateResponse) 401 if !ok { 402 return false, fmt.Errorf("sendRPC returned a %T instead of MutateResponse", pbmsg) 403 } 404 405 if r.Processed == nil { 406 return false, fmt.Errorf("protobuf in the response didn't contain the field "+ 407 "indicating whether the CheckAndPut was successful or not: %s", r) 408 } 409 410 return r.GetProcessed(), nil 411 } 412 413 // CacheRegions scan the meta region to get all the regions and populate to cache. 414 // This can be used to warm up region cache 415 func (c *client) CacheRegions(table []byte) error { 416 _, err := c.findAllRegions(context.Background(), table) 417 return err 418 }