Add support for CopyObject across regions and multiple Minio IPs

This PR adds CopyObject support for objects residing in buckets
in different Minio instances (where Minio instances are part of
a federated setup).

Also, added support for multiple Minio domain IPs. This is required
for distributed deployments, where one deployment may have multiple
nodes, each with a different public IP.
This commit is contained in:
Nitish Tiwari 2018-05-12 00:32:30 +05:30 committed by kannappanr
parent f30c95a301
commit 6ce7265c8c
12 changed files with 208 additions and 82 deletions

View file

@ -31,12 +31,17 @@ import (
"sync"
"github.com/coreos/etcd/client"
"github.com/gorilla/mux"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/dns"
"github.com/minio/minio/pkg/event"
"github.com/minio/minio/pkg/hash"
"github.com/minio/minio/pkg/policy"
"github.com/minio/minio/pkg/sync/errgroup"
"github.com/minio/minio-go/pkg/set"
)
// Check if there are buckets on server without corresponding entry in etcd backend and
@ -47,7 +52,6 @@ import (
// -- If yes, check if the IP of entry matches local IP. This means entry is for this instance.
// -- If IP of the entry doesn't match, this means entry is for another instance. Log an error to console.
func initFederatorBackend(objLayer ObjectLayer) {
// List all buckets
b, err := objLayer.ListBuckets(context.Background())
if err != nil {
logger.LogIf(context.Background(), err)
@ -60,15 +64,14 @@ func initFederatorBackend(objLayer ObjectLayer) {
g.Go(func() error {
r, gerr := globalDNSConfig.Get(b[index].Name)
if gerr != nil {
if client.IsKeyNotFound(gerr) {
// Make a new entry
if client.IsKeyNotFound(gerr) || gerr == dns.ErrNoEntriesFound {
return globalDNSConfig.Put(b[index].Name)
}
return gerr
}
if r.Host != globalDomainIP {
// Log error that entry already present for different host
return fmt.Errorf("Unable to add bucket DNS entry for bucket %s, an entry exists for the same bucket. Use %s to access the bucket, or rename it to a unique value", b[index].Name, globalDomainIP)
if globalDomainIPs.Intersection(set.CreateStringSet(getHostsSlice(r)...)).IsEmpty() {
// There is already an entry for this bucket, with all IP addresses different. This indicates a bucket name collision. Log an error and continue.
return fmt.Errorf("Unable to add bucket DNS entry for bucket %s, an entry exists for the same bucket. Use one of these IP addresses %v to access the bucket", b[index].Name, globalDomainIPs.ToSlice())
}
return nil
}, index)
@ -418,7 +421,7 @@ func (api objectAPIHandlers) PutBucketHandler(w http.ResponseWriter, r *http.Req
if globalDNSConfig != nil {
if _, err := globalDNSConfig.Get(bucket); err != nil {
if client.IsKeyNotFound(err) {
if client.IsKeyNotFound(err) || err == dns.ErrNoEntriesFound {
// Proceed to creating a bucket.
if err = objectAPI.MakeBucketWithLocation(ctx, bucket, location); err != nil {
writeErrorResponse(w, toAPIErrorCode(err), r.URL)

View file

@ -28,10 +28,13 @@ import (
"time"
etcd "github.com/coreos/etcd/client"
"github.com/minio/cli"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/auth"
"github.com/minio/minio/pkg/dns"
"github.com/minio/minio-go/pkg/set"
)
// Check for updates and print a notification message
@ -59,7 +62,7 @@ func initConfig() {
} else {
if etcd.IsKeyNotFound(err) {
logger.FatalIf(newConfig(), "Unable to initialize minio config for the first time.")
logger.Info("Created minio configuration file successfully at", globalEtcdClient.Endpoints())
logger.Info("Created minio configuration file successfully at %v", globalEtcdClient.Endpoints())
} else {
logger.FatalIf(err, "Unable to load config version: '%s'.", serverConfigVersion)
}
@ -168,10 +171,20 @@ func handleCommonEnvVars() {
logger.FatalIf(err, "Unable to initialize etcd with %s", etcdEndpoints)
}
globalDomainIP = os.Getenv("MINIO_PUBLIC_IP")
if globalDomainName != "" && globalDomainIP != "" && globalEtcdClient != nil {
minioEndpointsEnv, ok := os.LookupEnv("MINIO_PUBLIC_IPS")
if ok {
minioEndpoints := strings.Split(minioEndpointsEnv, ",")
globalDomainIPs = set.NewStringSet()
for i, ip := range minioEndpoints {
if net.ParseIP(ip) == nil {
logger.FatalIf(errInvalidArgument, "Unable to initialize Minio server with invalid MINIO_PUBLIC_IPS[%d]: %s", i, ip)
}
globalDomainIPs.Add(ip)
}
}
if globalDomainName != "" && !globalDomainIPs.IsEmpty() && globalEtcdClient != nil {
var err error
globalDNSConfig, err = dns.NewCoreDNS(globalDomainName, globalDomainIP, globalMinioPort, globalEtcdClient)
globalDNSConfig, err = dns.NewCoreDNS(globalDomainName, globalDomainIPs, globalMinioPort, globalEtcdClient)
logger.FatalIf(err, "Unable to initialize DNS config for %s.", globalDomainName)
}

View file

@ -26,9 +26,12 @@ import (
"strings"
"time"
"github.com/minio/minio-go/pkg/set"
"github.com/coreos/etcd/client"
humanize "github.com/dustin/go-humanize"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/dns"
"github.com/minio/minio/pkg/handlers"
"github.com/minio/minio/pkg/sys"
"github.com/rs/cors"
@ -629,29 +632,37 @@ func (f bucketForwardingHandler) ServeHTTP(w http.ResponseWriter, r *http.Reques
return
}
bucket, object := urlPath2BucketObjectName(r.URL.Path)
// MakeBucket request
// MakeBucket requests should be handled at current endpoint
if r.Method == http.MethodPut && bucket != "" && object == "" {
f.handler.ServeHTTP(w, r)
return
}
// ListBucket request
// ListBucket requests should be handled at current endpoint as
// all buckets data can be fetched from here.
if r.Method == http.MethodGet && bucket == "" && object == "" {
f.handler.ServeHTTP(w, r)
return
}
// CopyObject requests should be handled at current endpoint as path style
// requests have target bucket and object in URI and source details are in
// header fields
if r.Method == http.MethodPut && r.Header.Get("X-Amz-Copy-Source") != "" {
f.handler.ServeHTTP(w, r)
return
}
sr, err := globalDNSConfig.Get(bucket)
if err != nil {
if client.IsKeyNotFound(err) {
if client.IsKeyNotFound(err) || err == dns.ErrNoEntriesFound {
writeErrorResponse(w, ErrNoSuchBucket, r.URL)
} else {
writeErrorResponse(w, toAPIErrorCode(err), r.URL)
}
return
}
if sr.Host != globalDomainIP {
backendURL := fmt.Sprintf("http://%s:%d", sr.Host, sr.Port)
if globalDomainIPs.Intersection(set.CreateStringSet(getHostsSlice(sr)...)).IsEmpty() {
backendURL := fmt.Sprintf("http://%s:%d", sr[0].Host, sr[0].Port)
if globalIsSSL {
backendURL = fmt.Sprintf("https://%s:%d", sr.Host, sr.Port)
backendURL = fmt.Sprintf("https://%s:%d", sr[0].Host, sr[0].Port)
}
r.URL, err = url.Parse(backendURL)
if err != nil {

View file

@ -22,6 +22,8 @@ import (
"runtime"
"time"
"github.com/minio/minio-go/pkg/set"
etcd "github.com/coreos/etcd/client"
humanize "github.com/dustin/go-humanize"
"github.com/fatih/color"
@ -161,8 +163,8 @@ var (
globalPublicCerts []*x509.Certificate
globalIsEnvDomainName bool
globalDomainName string // Root domain for virtual host style requests
globalDomainIP string // Root domain IP address
globalDomainName string // Root domain for virtual host style requests
globalDomainIPs set.StringSet // Root domain IP address(s) for a distributed Minio deployment
globalListingTimeout = newDynamicTimeout( /*30*/ 600*time.Second /*5*/, 600*time.Second) // timeout for listing related ops
globalObjectTimeout = newDynamicTimeout( /*1*/ 10*time.Minute /*10*/, 600*time.Second) // timeout for Object API related ops

View file

@ -20,12 +20,16 @@ import (
"context"
"encoding/hex"
"fmt"
"net"
"path"
"runtime"
"strconv"
"strings"
"unicode/utf8"
miniogo "github.com/minio/minio-go"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/dns"
"github.com/skyrings/skyring-common/tools/uuid"
)
@ -277,6 +281,40 @@ func isMinioReservedBucket(bucketName string) bool {
return bucketName == minioReservedBucket
}
// Returns a minio-go Client configured to access remote host described by destDNSRecord
// Applicable only in a federated deployment
func getRemoteInstanceClient(destDNSRecord dns.SrvRecord) (*miniogo.Core, error) {
// In a federated deployment, all the instances share config files and hence expected to have same
// credentials. So, access current instances creds and use it to create client for remote instance
client, err := miniogo.NewCore(net.JoinHostPort(destDNSRecord.Host, strconv.Itoa(destDNSRecord.Port)), globalServerConfig.Credential.AccessKey, globalServerConfig.Credential.SecretKey, globalIsSSL)
if err != nil {
return nil, err
}
return client, nil
}
// Checks if a remote putobject call is needed for CopyObject operation
// 1. If source and destination bucket names are same, it means no call needed to etcd to get destination info
// 2. If destination bucket doesn't exist locally, only then a etcd call is needed
func isRemoteCallRequired(ctx context.Context, src, dst string, objAPI ObjectLayer) bool {
if src == dst {
return false
}
if _, err := objAPI.GetBucketInfo(ctx, dst); err == toObjectErr(errVolumeNotFound, dst) {
return true
}
return false
}
// returns a slice of hosts by reading a slice of DNS records
func getHostsSlice(records []dns.SrvRecord) []string {
var hosts []string
for _, r := range records {
hosts = append(hosts, r.Host)
}
return hosts
}
// byBucketName is a collection satisfying sort.Interface.
type byBucketName []BucketInfo

View file

@ -536,13 +536,51 @@ func (api objectAPIHandlers) CopyObjectHandler(w http.ResponseWriter, r *http.Re
return
}
// Copy source object to destination, if source and destination
// object is same then only metadata is updated.
objInfo, err := objectAPI.CopyObject(ctx, srcBucket, srcObject, dstBucket, dstObject, srcInfo)
if err != nil {
pipeWriter.CloseWithError(err)
writeErrorResponse(w, toAPIErrorCode(err), r.URL)
return
var objInfo ObjectInfo
// _, err = objectAPI.GetBucketInfo(ctx, dstBucket)
// if err == toObjectErr(errVolumeNotFound, dstBucket) && !cpSrcDstSame
if isRemoteCallRequired(ctx, srcBucket, dstBucket, objectAPI) {
if globalDNSConfig != nil {
if dstRecord, errEtcd := globalDNSConfig.Get(dstBucket); errEtcd == nil {
go func() {
if gerr := objectAPI.GetObject(ctx, srcBucket, srcObject, 0, srcInfo.Size, srcInfo.Writer, srcInfo.ETag); gerr != nil {
pipeWriter.CloseWithError(gerr)
writeErrorResponse(w, ErrInternalError, r.URL)
return
}
// Close writer explicitly to indicate data has been written
defer srcInfo.Writer.Close()
}()
// Send PutObject request to appropriate instance (in federated deployment)
client, rerr := getRemoteInstanceClient(dstRecord[0])
if rerr != nil {
pipeWriter.CloseWithError(rerr)
writeErrorResponse(w, ErrInternalError, r.URL)
return
}
remoteObjInfo, rerr := client.PutObject(dstBucket, dstObject, srcInfo.Reader, srcInfo.Size, "", "", srcInfo.UserDefined)
if rerr != nil {
pipeWriter.CloseWithError(rerr)
writeErrorResponse(w, ErrInternalError, r.URL)
return
}
objInfo.ETag = remoteObjInfo.ETag
objInfo.ModTime = remoteObjInfo.LastModified
}
} else {
writeErrorResponse(w, ErrNoSuchBucket, r.URL)
return
}
} else {
// Copy source object to destination, if source and destination
// object is same then only metadata is updated.
objInfo, err = objectAPI.CopyObject(ctx, srcBucket, srcObject, dstBucket, dstObject, srcInfo)
if err != nil {
pipeWriter.CloseWithError(err)
writeErrorResponse(w, toAPIErrorCode(err), r.URL)
return
}
}
pipeReader.Close()

View file

@ -91,7 +91,7 @@ ENVIRONMENT VARIABLES:
BUCKET-DNS:
MINIO_DOMAIN: To enable bucket DNS requests, set this value to Minio host domain name.
MINIO_PUBLIC_IP: To enable bucket DNS requests, set this value to Minio host public IP.
MINIO_PUBLIC_IPS: To enable bucket DNS requests, set this value to list of Minio host public IP(s) delimited by ",".
MINIO_ETCD_ENDPOINTS: To enable bucket DNS requests, set this value to list of etcd endpoints delimited by ",".
EXAMPLES:

View file

@ -40,6 +40,7 @@ import (
"github.com/minio/minio/browser"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/auth"
"github.com/minio/minio/pkg/dns"
"github.com/minio/minio/pkg/event"
"github.com/minio/minio/pkg/hash"
"github.com/minio/minio/pkg/policy"
@ -138,7 +139,7 @@ func (web *webAPIHandlers) MakeBucket(r *http.Request, args *MakeBucketArgs, rep
if globalDNSConfig != nil {
if _, err := globalDNSConfig.Get(args.BucketName); err != nil {
if etcd.IsKeyNotFound(err) {
if etcd.IsKeyNotFound(err) || err == dns.ErrNoEntriesFound {
// Proceed to creating a bucket.
if err = objectAPI.MakeBucketWithLocation(context.Background(), args.BucketName, globalServerConfig.GetRegion()); err != nil {
return toJSONError(err)

View file

@ -21,21 +21,21 @@ Bucket lookup federation requires two dependencies
```
export MINIO_ETCD_ENDPOINTS="http://remote-etcd1:2379,http://remote-etcd2:4001"
export MINIO_DOMAIN=domain.com
export MINIO_PUBLIC_IP=44.35.2.1
export MINIO_PUBLIC_IPS=44.35.2.1,44.35.2.2,44.35.2.3,44.35.2.4
minio server http://rack{1...4}.host{1...4}.domain.com/mnt/export{1...32}
```
> cluster2
```
export MINIO_ETCD_ENDPOINTS="http://remote-etcd1:2379,http://remote-etcd2:4001"
export MINIO_DOMAIN=domain.com
export MINIO_PUBLIC_IP=44.35.2.2
export MINIO_PUBLIC_IPS=44.35.1.1,44.35.1.2,44.35.1.3,44.35.1.4
minio server http://rack{5...8}.host{5...8}.domain.com/mnt/export{1...32}
```
In this configuration you can see `MINIO_ETCD_ENDPOINTS` points to the etcd backend which manages Minio's
`config.json` and bucket DNS SRV records. `MINIO_DOMAIN` indicates the domain suffix for the bucket which
will be used to resolve bucket through DNS. For example if you have a bucket such as `mybucket`, the
client can use now `mybucket.domain.com` to directly resolve itself to the right cluster. `MINIO_PUBLIC_IP`
client can use now `mybucket.domain.com` to directly resolve itself to the right cluster. `MINIO_PUBLIC_IPS`
points to the public IP address where each cluster might be accessible, this is unique for each cluster.
NOTE: `mybucket` only exists on one cluster either `cluster1` or `cluster2` this is random and

View file

@ -21,18 +21,22 @@ import (
"encoding/json"
"errors"
"fmt"
"net"
"sort"
"strconv"
"strings"
"time"
"github.com/minio/minio-go/pkg/set"
"github.com/coredns/coredns/plugin/etcd/msg"
etcd "github.com/coreos/etcd/client"
)
// ErrNoEntriesFound - Indicates no entries were found for the given key (directory)
var ErrNoEntriesFound = errors.New("No entries found for this key")
// create a new coredns service record for the bucket.
func newCoreDNSMsg(bucket string, ip string, port int, ttl uint32) ([]byte, error) {
func newCoreDNSMsg(bucket, ip string, port int, ttl uint32) ([]byte, error) {
return json.Marshal(&SrvRecord{
Host: ip,
Port: port,
@ -41,12 +45,24 @@ func newCoreDNSMsg(bucket string, ip string, port int, ttl uint32) ([]byte, erro
})
}
// Retrieves list of DNS entries for a bucket.
// Retrieves list of DNS entries for the domain.
func (c *coreDNS) List() ([]SrvRecord, error) {
kapi := etcd.NewKeysAPI(c.etcdClient)
key := msg.Path(fmt.Sprintf("%s.", c.domainName), defaultPrefixPath)
return c.list(key)
}
// Retrieves DNS records for a bucket.
func (c *coreDNS) Get(bucket string) ([]SrvRecord, error) {
key := msg.Path(fmt.Sprintf("%s.%s.", bucket, c.domainName), defaultPrefixPath)
return c.list(key)
}
// Retrieves list of entries under the key passed.
// Note that this method fetches entries upto only two levels deep.
func (c *coreDNS) list(key string) ([]SrvRecord, error) {
kapi := etcd.NewKeysAPI(c.etcdClient)
ctx, cancel := context.WithTimeout(context.Background(), defaultContextTimeout)
r, err := kapi.Get(ctx, key, nil)
r, err := kapi.Get(ctx, key, &etcd.GetOptions{Recursive: true})
cancel()
if err != nil {
return nil, err
@ -54,48 +70,52 @@ func (c *coreDNS) List() ([]SrvRecord, error) {
var srvRecords []SrvRecord
for _, n := range r.Node.Nodes {
var srvRecord SrvRecord
if err = json.Unmarshal([]byte(n.Value), &srvRecord); err != nil {
return nil, err
if !n.Dir {
var srvRecord SrvRecord
if err = json.Unmarshal([]byte(n.Value), &srvRecord); err != nil {
return nil, err
}
srvRecord.Key = strings.TrimPrefix(n.Key, key)
srvRecords = append(srvRecords, srvRecord)
} else {
// As this is a directory, loop through all the nodes inside
for _, n1 := range n.Nodes {
var srvRecord SrvRecord
if err = json.Unmarshal([]byte(n1.Value), &srvRecord); err != nil {
return nil, err
}
srvRecord.Key = strings.TrimPrefix(n1.Key, key)
srvRecord.Key = strings.TrimSuffix(srvRecord.Key, srvRecord.Host)
srvRecords = append(srvRecords, srvRecord)
}
}
srvRecord.Key = strings.TrimPrefix(n.Key, key)
srvRecords = append(srvRecords, srvRecord)
}
sort.Slice(srvRecords, func(i int, j int) bool {
return srvRecords[i].Key < srvRecords[j].Key
})
if srvRecords != nil {
sort.Slice(srvRecords, func(i int, j int) bool {
return srvRecords[i].Key < srvRecords[j].Key
})
} else {
return nil, ErrNoEntriesFound
}
return srvRecords, nil
}
// Retrieves DNS record for a bucket.
func (c *coreDNS) Get(bucket string) (SrvRecord, error) {
kapi := etcd.NewKeysAPI(c.etcdClient)
key := msg.Path(fmt.Sprintf("%s.%s.", bucket, c.domainName), defaultPrefixPath)
ctx, cancel := context.WithTimeout(context.Background(), defaultContextTimeout)
r, err := kapi.Get(ctx, key, nil)
cancel()
if err != nil {
return SrvRecord{}, err
}
var sr SrvRecord
if err = json.Unmarshal([]byte(r.Node.Value), &sr); err != nil {
return SrvRecord{}, err
}
sr.Key = strings.TrimPrefix(r.Node.Key, key)
return sr, nil
}
// Adds DNS entries into etcd endpoint in CoreDNS etcd messae format.
// Adds DNS entries into etcd endpoint in CoreDNS etcd message format.
func (c *coreDNS) Put(bucket string) error {
bucketMsg, err := newCoreDNSMsg(bucket, c.domainIP, c.domainPort, defaultTTL)
if err != nil {
return err
var err error
for ip := range c.domainIPs {
var bucketMsg []byte
bucketMsg, err = newCoreDNSMsg(bucket, ip, c.domainPort, defaultTTL)
if err != nil {
return err
}
kapi := etcd.NewKeysAPI(c.etcdClient)
key := msg.Path(fmt.Sprintf("%s.%s", bucket, c.domainName), defaultPrefixPath)
key = key + "/" + ip
ctx, cancel := context.WithTimeout(context.Background(), defaultContextTimeout)
_, err = kapi.Set(ctx, key, string(bucketMsg), nil)
cancel()
}
kapi := etcd.NewKeysAPI(c.etcdClient)
key := msg.Path(fmt.Sprintf("%s.%s.", bucket, c.domainName), defaultPrefixPath)
ctx, cancel := context.WithTimeout(context.Background(), defaultContextTimeout)
_, err = kapi.Set(ctx, key, string(bucketMsg), nil)
cancel()
return err
}
@ -111,18 +131,15 @@ func (c *coreDNS) Delete(bucket string) error {
// CoreDNS - represents dns config for coredns server.
type coreDNS struct {
domainName, domainIP string
domainPort int
etcdClient etcd.Client
domainName string
domainIPs set.StringSet
domainPort int
etcdClient etcd.Client
}
// NewCoreDNS - initialize a new coreDNS set/unset values.
func NewCoreDNS(domainName, domainIP, domainPort string, etcdClient etcd.Client) (Config, error) {
if domainName == "" || domainIP == "" || etcdClient == nil {
return nil, errors.New("invalid argument")
}
if net.ParseIP(domainIP) == nil {
func NewCoreDNS(domainName string, domainIPs set.StringSet, domainPort string, etcdClient etcd.Client) (Config, error) {
if domainName == "" || domainIPs.IsEmpty() || etcdClient == nil {
return nil, errors.New("invalid argument")
}
@ -133,7 +150,7 @@ func NewCoreDNS(domainName, domainIP, domainPort string, etcdClient etcd.Client)
return &coreDNS{
domainName: domainName,
domainIP: domainIP,
domainIPs: domainIPs,
domainPort: port,
etcdClient: etcdClient,
}, nil

View file

@ -60,6 +60,6 @@ type SrvRecord struct {
type Config interface {
Put(key string) error
List() ([]SrvRecord, error)
Get(key string) (SrvRecord, error)
Get(key string) ([]SrvRecord, error)
Delete(key string) error
}

View file

@ -140,6 +140,9 @@ func saveFileConfigEtcd(filename string, clnt etcd.Client, v interface{}) error
kapi := etcd.NewKeysAPI(clnt)
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute)
_, err = kapi.Update(ctx, filename, string(dataBytes))
if etcd.IsKeyNotFound(err) {
_, err = kapi.Create(ctx, filename, string(dataBytes))
}
cancel()
return err
}