func (c *Client) findRegionForRPC(rpc hrpc.Call) (proto.Message, error) { // The region was not in the cache, it // must be looked up in the meta table backoff := backoffStart ctx := rpc.GetContext() for { // Look up the region in the meta table reg, host, port, err := c.locateRegion(ctx, rpc.Table(), rpc.Key()) if err != nil { if err == TableNotFound { return nil, err } // There was an error with the meta table. Let's sleep for some // backoff amount and retry. backoff, err = sleepAndIncreaseBackoff(ctx, backoff) if err != nil { return nil, err } continue } // Check that the region wasn't added to // the cache while we were looking it up. c.regionsLock.Lock() if existing := c.getRegionFromCache(rpc.Table(), rpc.Key()); existing != nil { // The region was added to the cache while we were looking it // up. Send the RPC to the region that was in the cache. c.regionsLock.Unlock() return c.sendRPCToRegion(rpc, existing) } // The region wasn't added to the cache while we were looking it // up. Mark this one as unavailable and add it to the cache. reg.MarkUnavailable() c.regions.put(reg.RegionName, reg) c.regionsLock.Unlock() // Start a goroutine to connect to the region go c.establishRegion(reg, host, port) // Wait for the new region to become // available, and then send the RPC return c.waitOnRegion(rpc, reg) } }
func (c *Client) waitOnRegion(rpc hrpc.Call, reg *regioninfo.Info) (proto.Message, error) { ch := reg.GetAvailabilityChan() if ch == nil { // WTF, this region is available? Maybe it was marked as such // since waitOnRegion was called. return c.sendRPC(rpc) } // The region is unavailable. Wait for it to become available, // or for the deadline to be exceeded. select { case <-ch: return c.sendRPC(rpc) case <-rpc.GetContext().Done(): return nil, ErrDeadline } }
// Queues an RPC targeted at a particular region for handling by the appropriate // region client. Results will be written to the rpc's result and error // channels. func (c *Client) queueRPC(rpc hrpc.Call) error { table := rpc.Table() key := rpc.Key() reg := c.getRegion(table, key) // The first time an RPC is sent to the meta region, the meta client will // have not yet been intialized. Check if this is the case, try to mark // the meta region info as unavailable, and if it hadn't been marked as // unavailable yet start a goroutine to connect to it. if reg == c.metaRegionInfo && c.metaClient == nil { marked := c.metaRegionInfo.MarkUnavailable() if marked { go c.reestablishRegion(c.metaRegionInfo) } } var client *region.Client if reg != nil { ch := reg.GetAvailabilityChan() if ch != nil { select { case <-ch: return c.queueRPC(rpc) case <-rpc.GetContext().Done(): return ErrDeadline } } client = c.clientFor(reg) } else { var err error client, reg, err = c.locateRegion(rpc.GetContext(), table, key) if err != nil { return err } } rpc.SetRegion(reg) return client.QueueRPC(rpc) }
func (c *Client) sendRPCToRegion(rpc hrpc.Call, reg *regioninfo.Info) (proto.Message, error) { // On the first sendRPC to the meta or admin regions, a goroutine must be // manually kicked off for the meta or admin region client if c.adminClient == nil && reg == c.adminRegionInfo && !c.adminRegionInfo.IsUnavailable() || c.metaClient == nil && reg == c.metaRegionInfo && !c.metaRegionInfo.IsUnavailable() { c.regionsLock.Lock() if reg == c.metaRegionInfo && !c.metaRegionInfo.IsUnavailable() || reg == c.adminRegionInfo && !c.adminRegionInfo.IsUnavailable() { reg.MarkUnavailable() go c.reestablishRegion(reg) } c.regionsLock.Unlock() } // The region was in the cache, check // if the region is marked as available if !reg.IsUnavailable() { // The region is available rpc.SetRegion(reg) // Queue the RPC to be sent to the region client := c.clientFor(reg) var err error if client == nil { err = errors.New("no client for this region") } else { err = client.QueueRPC(rpc) } if err != nil { // There was an error queueing the RPC. // Mark the region as unavailable. first := reg.MarkUnavailable() // If this was the first goroutine to mark the region as // unavailable, start a goroutine to reestablish a connection if first { go c.reestablishRegion(reg) } // Block until the region becomes available. return c.waitOnRegion(rpc, reg) } // Wait for the response var res hrpc.RPCResult select { case res = <-rpc.GetResultChan(): case <-rpc.GetContext().Done(): return nil, ErrDeadline } // Check for errors if _, ok := res.Error.(region.RetryableError); ok { // There's an error specific to this region, but // our region client is fine. Mark this region as // unavailable (as opposed to all regions sharing // the client), and start a goroutine to reestablish // it. first := reg.MarkUnavailable() if first { go c.reestablishRegion(reg) } if reg != c.metaRegionInfo && reg != c.adminRegionInfo { // The client won't be in the cache if this is the // meta or admin region c.clients.del(reg) } return c.waitOnRegion(rpc, reg) } else if _, ok := res.Error.(region.UnrecoverableError); ok { // If it was an unrecoverable error, the region client is // considered dead. if reg == c.metaRegionInfo || reg == c.adminRegionInfo { // If this is the admin client or the meta table, mark the // region as unavailable and start up a goroutine to // reconnect if it wasn't already marked as such. first := reg.MarkUnavailable() if first { go c.reestablishRegion(reg) } } else { // Else this is a normal region. Mark all the regions // sharing this region's client as unavailable, and start // a goroutine to reconnect for each of them. downregions := c.clients.clientDown(reg) for _, downreg := range downregions { go c.reestablishRegion(downreg) } } // Fall through to the case of the region being unavailable, // which will result in blocking until it's available again. return c.waitOnRegion(rpc, reg) } else { // RPC was successfully sent, or an unknown type of error // occurred. In either case, return the results. return res.Msg, res.Error } } return c.waitOnRegion(rpc, reg) }
// sendRPC takes an RPC call, and will send it to the correct region server. If // the correct region server is offline or otherwise unavailable, sendRPC will // continually retry until the deadline set on the RPC's context is exceeded. func (c *Client) sendRPC(rpc hrpc.Call) (proto.Message, error) { log.WithFields(log.Fields{ "Type": rpc.GetName(), "Table": string(rpc.Table()), "Key": string(rpc.Key()), }).Debug("Sending RPC") err := c.queueRPC(rpc) if err == ErrDeadline { return nil, err } else if err != nil { log.WithFields(log.Fields{ "Type": rpc.GetName(), "Table": string(rpc.Table()), "Key": string(rpc.Key()), }).Debug("We hit an error queuing the RPC. Resending.") // There was an error locating the region for the RPC, or the client // for the region encountered an error and has shut down. return c.sendRPC(rpc) } if err == nil { var res hrpc.RPCResult resch := rpc.GetResultChan() select { case res = <-resch: case <-rpc.GetContext().Done(): return nil, ErrDeadline } err := res.Error log.WithFields(log.Fields{ "Type": rpc.GetName(), "Table": string(rpc.Table()), "Key": string(rpc.Key()), "Result": res.Msg, "Error": err, }).Debug("Successfully sent RPC. Returning.") if _, ok := err.(region.RetryableError); ok { return c.sendRPC(rpc) } else if _, ok := err.(region.UnrecoverableError); ok { // Prevents dropping into the else block below, // error handling happens a few lines down } else { return res.Msg, res.Error } } // There was an issue related to the network, so we're going to mark the // region as unavailable, and generate the channel used for announcing // when it's available again region := rpc.GetRegion() log.WithFields(log.Fields{ "Type": rpc.GetName(), "Table": string(rpc.Table()), "Key": string(rpc.Key()), }).Debug("Encountered a network error. Region unavailable?") if region != nil { succ := region.MarkUnavailable() if succ { go c.reestablishRegion(region) } } log.WithFields(log.Fields{ "Type": rpc.GetName(), "Table": string(rpc.Table()), "Key": string(rpc.Key()), }).Debug("Retrying sendRPC") return c.sendRPC(rpc) }