Add sync => go-sync
This commit is contained in:
104
sync/lock/consul/consul.go
Normal file
104
sync/lock/consul/consul.go
Normal file
@@ -0,0 +1,104 @@
|
||||
// Package consul is a consul implemenation of lock
|
||||
package consul
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"net"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/api"
|
||||
lock "github.com/micro/go-micro/sync/lock"
|
||||
)
|
||||
|
||||
type consulLock struct {
|
||||
sync.Mutex
|
||||
|
||||
locks map[string]*api.Lock
|
||||
opts lock.Options
|
||||
c *api.Client
|
||||
}
|
||||
|
||||
func (c *consulLock) Acquire(id string, opts ...lock.AcquireOption) error {
|
||||
var options lock.AcquireOptions
|
||||
for _, o := range opts {
|
||||
o(&options)
|
||||
}
|
||||
|
||||
if options.Wait <= time.Duration(0) {
|
||||
options.Wait = api.DefaultLockWaitTime
|
||||
}
|
||||
|
||||
ttl := fmt.Sprintf("%v", options.TTL)
|
||||
if options.TTL <= time.Duration(0) {
|
||||
ttl = api.DefaultLockSessionTTL
|
||||
}
|
||||
|
||||
l, err := c.c.LockOpts(&api.LockOptions{
|
||||
Key: c.opts.Prefix + id,
|
||||
LockWaitTime: options.Wait,
|
||||
SessionTTL: ttl,
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
_, err = l.Lock(nil)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
c.Lock()
|
||||
c.locks[id] = l
|
||||
c.Unlock()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *consulLock) Release(id string) error {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
l, ok := c.locks[id]
|
||||
if !ok {
|
||||
return errors.New("lock not found")
|
||||
}
|
||||
err := l.Unlock()
|
||||
delete(c.locks, id)
|
||||
return err
|
||||
}
|
||||
|
||||
func (c *consulLock) String() string {
|
||||
return "consul"
|
||||
}
|
||||
|
||||
func NewLock(opts ...lock.Option) lock.Lock {
|
||||
var options lock.Options
|
||||
for _, o := range opts {
|
||||
o(&options)
|
||||
}
|
||||
|
||||
config := api.DefaultConfig()
|
||||
|
||||
// set host
|
||||
// config.Host something
|
||||
// check if there are any addrs
|
||||
if len(options.Nodes) > 0 {
|
||||
addr, port, err := net.SplitHostPort(options.Nodes[0])
|
||||
if ae, ok := err.(*net.AddrError); ok && ae.Err == "missing port in address" {
|
||||
port = "8500"
|
||||
config.Address = fmt.Sprintf("%s:%s", options.Nodes[0], port)
|
||||
} else if err == nil {
|
||||
config.Address = fmt.Sprintf("%s:%s", addr, port)
|
||||
}
|
||||
}
|
||||
|
||||
client, _ := api.NewClient(config)
|
||||
|
||||
return &consulLock{
|
||||
locks: make(map[string]*api.Lock),
|
||||
opts: options,
|
||||
c: client,
|
||||
}
|
||||
}
|
115
sync/lock/etcd/etcd.go
Normal file
115
sync/lock/etcd/etcd.go
Normal file
@@ -0,0 +1,115 @@
|
||||
// Package etcd is an etcd implementation of lock
|
||||
package etcd
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"log"
|
||||
"path"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/micro/go-micro/sync/lock"
|
||||
client "go.etcd.io/etcd/clientv3"
|
||||
cc "go.etcd.io/etcd/clientv3/concurrency"
|
||||
)
|
||||
|
||||
type etcdLock struct {
|
||||
opts lock.Options
|
||||
path string
|
||||
client *client.Client
|
||||
|
||||
sync.Mutex
|
||||
locks map[string]*elock
|
||||
}
|
||||
|
||||
type elock struct {
|
||||
s *cc.Session
|
||||
m *cc.Mutex
|
||||
}
|
||||
|
||||
func (e *etcdLock) Acquire(id string, opts ...lock.AcquireOption) error {
|
||||
var options lock.AcquireOptions
|
||||
for _, o := range opts {
|
||||
o(&options)
|
||||
}
|
||||
|
||||
// make path
|
||||
path := path.Join(e.path, strings.Replace(e.opts.Prefix+id, "/", "-", -1))
|
||||
|
||||
var sopts []cc.SessionOption
|
||||
if options.TTL > 0 {
|
||||
sopts = append(sopts, cc.WithTTL(int(options.TTL.Seconds())))
|
||||
}
|
||||
|
||||
s, err := cc.NewSession(e.client, sopts...)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
m := cc.NewMutex(s, path)
|
||||
|
||||
ctx, _ := context.WithCancel(context.Background())
|
||||
|
||||
if err := m.Lock(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
e.Lock()
|
||||
e.locks[id] = &elock{
|
||||
s: s,
|
||||
m: m,
|
||||
}
|
||||
e.Unlock()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *etcdLock) Release(id string) error {
|
||||
e.Lock()
|
||||
defer e.Unlock()
|
||||
v, ok := e.locks[id]
|
||||
if !ok {
|
||||
return errors.New("lock not found")
|
||||
}
|
||||
err := v.m.Unlock(context.Background())
|
||||
delete(e.locks, id)
|
||||
return err
|
||||
}
|
||||
|
||||
func (e *etcdLock) String() string {
|
||||
return "etcd"
|
||||
}
|
||||
|
||||
func NewLock(opts ...lock.Option) lock.Lock {
|
||||
var options lock.Options
|
||||
for _, o := range opts {
|
||||
o(&options)
|
||||
}
|
||||
|
||||
var endpoints []string
|
||||
|
||||
for _, addr := range options.Nodes {
|
||||
if len(addr) > 0 {
|
||||
endpoints = append(endpoints, addr)
|
||||
}
|
||||
}
|
||||
|
||||
if len(endpoints) == 0 {
|
||||
endpoints = []string{"http://127.0.0.1:2379"}
|
||||
}
|
||||
|
||||
// TODO: parse addresses
|
||||
c, err := client.New(client.Config{
|
||||
Endpoints: endpoints,
|
||||
})
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
return &etcdLock{
|
||||
path: "/micro/lock",
|
||||
client: c,
|
||||
opts: options,
|
||||
locks: make(map[string]*elock),
|
||||
}
|
||||
}
|
27
sync/lock/lock.go
Normal file
27
sync/lock/lock.go
Normal file
@@ -0,0 +1,27 @@
|
||||
// Package lock provides distributed locking
|
||||
package lock
|
||||
|
||||
import (
|
||||
"time"
|
||||
)
|
||||
|
||||
// Lock is a distributed locking interface
|
||||
type Lock interface {
|
||||
// Acquire a lock with given id
|
||||
Acquire(id string, opts ...AcquireOption) error
|
||||
// Release the lock with given id
|
||||
Release(id string) error
|
||||
}
|
||||
|
||||
type Options struct {
|
||||
Nodes []string
|
||||
Prefix string
|
||||
}
|
||||
|
||||
type AcquireOptions struct {
|
||||
TTL time.Duration
|
||||
Wait time.Duration
|
||||
}
|
||||
|
||||
type Option func(o *Options)
|
||||
type AcquireOption func(o *AcquireOptions)
|
33
sync/lock/options.go
Normal file
33
sync/lock/options.go
Normal file
@@ -0,0 +1,33 @@
|
||||
package lock
|
||||
|
||||
import (
|
||||
"time"
|
||||
)
|
||||
|
||||
// Nodes sets the addresses the underlying lock implementation
|
||||
func Nodes(a ...string) Option {
|
||||
return func(o *Options) {
|
||||
o.Nodes = a
|
||||
}
|
||||
}
|
||||
|
||||
// Prefix sets a prefix to any lock ids used
|
||||
func Prefix(p string) Option {
|
||||
return func(o *Options) {
|
||||
o.Prefix = p
|
||||
}
|
||||
}
|
||||
|
||||
// TTL sets the lock ttl
|
||||
func TTL(t time.Duration) AcquireOption {
|
||||
return func(o *AcquireOptions) {
|
||||
o.TTL = t
|
||||
}
|
||||
}
|
||||
|
||||
// Wait sets the wait time
|
||||
func Wait(t time.Duration) AcquireOption {
|
||||
return func(o *AcquireOptions) {
|
||||
o.Wait = t
|
||||
}
|
||||
}
|
29
sync/lock/redis/pool.go
Normal file
29
sync/lock/redis/pool.go
Normal file
@@ -0,0 +1,29 @@
|
||||
package redis
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/gomodule/redigo/redis"
|
||||
)
|
||||
|
||||
type pool struct {
|
||||
sync.Mutex
|
||||
i int
|
||||
addrs []string
|
||||
}
|
||||
|
||||
func (p *pool) Get() redis.Conn {
|
||||
for i := 0; i < 3; i++ {
|
||||
p.Lock()
|
||||
addr := p.addrs[p.i%len(p.addrs)]
|
||||
p.i++
|
||||
p.Unlock()
|
||||
|
||||
c, err := redis.Dial("tcp", addr)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
return c
|
||||
}
|
||||
return nil
|
||||
}
|
94
sync/lock/redis/redis.go
Normal file
94
sync/lock/redis/redis.go
Normal file
@@ -0,0 +1,94 @@
|
||||
// Package redis is a redis implemenation of lock
|
||||
package redis
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/go-redsync/redsync"
|
||||
"github.com/micro/go-micro/sync/lock"
|
||||
)
|
||||
|
||||
type redisLock struct {
|
||||
sync.Mutex
|
||||
|
||||
locks map[string]*redsync.Mutex
|
||||
opts lock.Options
|
||||
c *redsync.Redsync
|
||||
}
|
||||
|
||||
func (r *redisLock) Acquire(id string, opts ...lock.AcquireOption) error {
|
||||
var options lock.AcquireOptions
|
||||
for _, o := range opts {
|
||||
o(&options)
|
||||
}
|
||||
|
||||
var ropts []redsync.Option
|
||||
|
||||
if options.Wait > time.Duration(0) {
|
||||
ropts = append(ropts, redsync.SetRetryDelay(options.Wait))
|
||||
ropts = append(ropts, redsync.SetTries(1))
|
||||
}
|
||||
|
||||
if options.TTL > time.Duration(0) {
|
||||
ropts = append(ropts, redsync.SetExpiry(options.TTL))
|
||||
}
|
||||
|
||||
m := r.c.NewMutex(r.opts.Prefix+id, ropts...)
|
||||
err := m.Lock()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
r.Lock()
|
||||
r.locks[id] = m
|
||||
r.Unlock()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r *redisLock) Release(id string) error {
|
||||
r.Lock()
|
||||
defer r.Unlock()
|
||||
m, ok := r.locks[id]
|
||||
if !ok {
|
||||
return errors.New("lock not found")
|
||||
}
|
||||
|
||||
unlocked := m.Unlock()
|
||||
delete(r.locks, id)
|
||||
|
||||
if !unlocked {
|
||||
return errors.New("lock not unlocked")
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r *redisLock) String() string {
|
||||
return "redis"
|
||||
}
|
||||
|
||||
func NewLock(opts ...lock.Option) lock.Lock {
|
||||
var options lock.Options
|
||||
for _, o := range opts {
|
||||
o(&options)
|
||||
}
|
||||
|
||||
nodes := options.Nodes
|
||||
|
||||
if len(nodes) == 0 {
|
||||
nodes = []string{"127.0.0.1:6379"}
|
||||
}
|
||||
|
||||
rpool := redsync.New([]redsync.Pool{&pool{
|
||||
addrs: nodes,
|
||||
}})
|
||||
|
||||
return &redisLock{
|
||||
locks: make(map[string]*redsync.Mutex),
|
||||
opts: options,
|
||||
c: rpool,
|
||||
}
|
||||
}
|
Reference in New Issue
Block a user