1
0
Fork 0
mirror of https://github.com/chrislusf/seaweedfs synced 2024-06-26 20:38:16 +02:00

better IP v6 support

This commit is contained in:
Chris Lu 2021-09-07 19:29:42 -07:00
parent 0128239c0f
commit 574485ec69
11 changed files with 46 additions and 207 deletions

View file

@ -4,7 +4,6 @@ import (
"fmt"
"net/http"
"os"
"strconv"
"strings"
"time"
@ -207,7 +206,7 @@ func (fo *FilerOptions) startFiler() {
}
if *fo.publicPort != 0 {
publicListeningAddress := *fo.bindIp + ":" + strconv.Itoa(*fo.publicPort)
publicListeningAddress := util.JoinHostPort(*fo.bindIp, *fo.publicPort)
glog.V(0).Infoln("Start Seaweed filer server", util.Version(), "public at", publicListeningAddress)
publicListener, e := util.NewListener(publicListeningAddress, 0)
if e != nil {
@ -222,7 +221,7 @@ func (fo *FilerOptions) startFiler() {
glog.V(0).Infof("Start Seaweed Filer %s at %s:%d", util.Version(), *fo.ip, *fo.port)
filerListener, e := util.NewListener(
*fo.bindIp+":"+strconv.Itoa(*fo.port),
util.JoinHostPort(*fo.bindIp, *fo.port),
time.Duration(10)*time.Second,
)
if e != nil {
@ -231,7 +230,7 @@ func (fo *FilerOptions) startFiler() {
// starting grpc server
grpcPort := *fo.port + 10000
grpcL, err := util.NewListener(*fo.bindIp+":"+strconv.Itoa(grpcPort), 0)
grpcL, err := util.NewListener(util.JoinHostPort(*fo.bindIp, grpcPort), 0)
if err != nil {
glog.Fatalf("failed to listen on grpc port %d: %v", grpcPort, err)
}

View file

@ -7,7 +7,6 @@ import (
"net/http"
"os"
"sort"
"strconv"
"strings"
"time"
@ -116,7 +115,7 @@ func startMaster(masterOption MasterOptions, masterWhiteList []string) {
r := mux.NewRouter()
ms := weed_server.NewMasterServer(r, masterOption.toMasterOption(masterWhiteList), peers)
listeningAddress := *masterOption.ipBind + ":" + strconv.Itoa(*masterOption.port)
listeningAddress := util.JoinHostPort(*masterOption.ipBind, *masterOption.port)
glog.V(0).Infof("Start Seaweed Master %s at %s", util.Version(), listeningAddress)
masterListener, e := util.NewListener(listeningAddress, 0)
if e != nil {
@ -132,7 +131,7 @@ func startMaster(masterOption MasterOptions, masterWhiteList []string) {
r.HandleFunc("/cluster/status", raftServer.StatusHandler).Methods("GET")
// starting grpc server
grpcPort := *masterOption.port + 10000
grpcL, err := util.NewListener(*masterOption.ipBind+":"+strconv.Itoa(grpcPort), 0)
grpcL, err := util.NewListener(util.JoinHostPort(*masterOption.ipBind, grpcPort), 0)
if err != nil {
glog.Fatalf("master failed to listen on grpc port %d: %v", grpcPort, err)
}
@ -163,7 +162,7 @@ func startMaster(masterOption MasterOptions, masterWhiteList []string) {
func checkPeers(masterIp string, masterPort int, peers string) (masterAddress string, cleanedPeers []string) {
glog.V(0).Infof("current: %s:%d peers:%s", masterIp, masterPort, peers)
masterAddress = masterIp + ":" + strconv.Itoa(masterPort)
masterAddress = util.JoinHostPort(masterIp, masterPort)
if peers != "" {
cleanedPeers = strings.Split(peers, ",")
}

View file

@ -13,7 +13,6 @@ import (
"github.com/gorilla/mux"
"google.golang.org/grpc/reflection"
"net/http"
"strconv"
"strings"
"time"
)
@ -114,7 +113,7 @@ func startMasterFollower(masterOptions MasterOptions) {
r := mux.NewRouter()
ms := weed_server.NewMasterServer(r, option, masters)
listeningAddress := *masterOptions.ipBind + ":" + strconv.Itoa(*masterOptions.port)
listeningAddress := util.JoinHostPort(*masterOptions.ipBind, *masterOptions.port)
glog.V(0).Infof("Start Seaweed Master %s at %s", util.Version(), listeningAddress)
masterListener, e := util.NewListener(listeningAddress, 0)
if e != nil {
@ -123,7 +122,7 @@ func startMasterFollower(masterOptions MasterOptions) {
// starting grpc server
grpcPort := *masterOptions.port + 10000
grpcL, err := util.NewListener(*masterOptions.ipBind+":"+strconv.Itoa(grpcPort), 0)
grpcL, err := util.NewListener(util.JoinHostPort(*masterOptions.ipBind, grpcPort), 0)
if err != nil {
glog.Fatalf("master failed to listen on grpc port %d: %v", grpcPort, err)
}

View file

@ -3,7 +3,6 @@ package command
import (
"context"
"fmt"
"strconv"
"time"
"google.golang.org/grpc/reflection"
@ -100,7 +99,7 @@ func (msgBrokerOpt *MessageBrokerOptions) startQueueServer() bool {
}, grpcDialOption)
// start grpc listener
grpcL, err := util.NewListener(":"+strconv.Itoa(*msgBrokerOpt.port), 0)
grpcL, err := util.NewListener(util.JoinHostPort("", *msgBrokerOpt.port), 0)
if err != nil {
glog.Fatalf("failed to listen on grpc port %d: %v", *msgBrokerOpt.port, err)
}

View file

@ -194,7 +194,7 @@ func (v VolumeServerOptions) startVolumeServer(volumeFolders, maxVolumeCounts, v
*v.publicPort = *v.port
}
if *v.publicUrl == "" {
*v.publicUrl = *v.ip + ":" + strconv.Itoa(*v.publicPort)
*v.publicUrl = util.JoinHostPort(*v.ip, *v.publicPort)
}
volumeMux := http.NewServeMux()
@ -308,7 +308,7 @@ func (v VolumeServerOptions) isSeparatedPublicPort() bool {
func (v VolumeServerOptions) startGrpcService(vs volume_server_pb.VolumeServerServer) *grpc.Server {
grpcPort := *v.port + 10000
grpcL, err := util.NewListener(*v.bindIp+":"+strconv.Itoa(grpcPort), 0)
grpcL, err := util.NewListener(util.JoinHostPort(*v.bindIp, grpcPort), 0)
if err != nil {
glog.Fatalf("failed to listen on grpc port %d: %v", grpcPort, err)
}
@ -324,7 +324,7 @@ func (v VolumeServerOptions) startGrpcService(vs volume_server_pb.VolumeServerSe
}
func (v VolumeServerOptions) startPublicHttpService(handler http.Handler) httpdown.Server {
publicListeningAddress := *v.bindIp + ":" + strconv.Itoa(*v.publicPort)
publicListeningAddress := util.JoinHostPort(*v.bindIp, *v.publicPort)
glog.V(0).Infoln("Start Seaweed volume server", util.Version(), "public at", publicListeningAddress)
publicListener, e := util.NewListener(publicListeningAddress, time.Duration(*v.idleConnectionTimeout)*time.Second)
if e != nil {
@ -351,7 +351,7 @@ func (v VolumeServerOptions) startClusterHttpService(handler http.Handler) httpd
keyFile = viper.GetString("https.volume.key")
}
listeningAddress := *v.bindIp + ":" + strconv.Itoa(*v.port)
listeningAddress := util.JoinHostPort(*v.bindIp, *v.port)
glog.V(0).Infof("Start Seaweed volume server %s at %s", util.Version(), listeningAddress)
listener, e := util.NewListener(listeningAddress, time.Duration(*v.idleConnectionTimeout)*time.Second)
if e != nil {
@ -373,7 +373,7 @@ func (v VolumeServerOptions) startClusterHttpService(handler http.Handler) httpd
}
func (v VolumeServerOptions) startTcpService(volumeServer *weed_server.VolumeServer) {
listeningAddress := *v.bindIp + ":" + strconv.Itoa(*v.port+20000)
listeningAddress := util.JoinHostPort(*v.bindIp,*v.port+20000)
glog.V(0).Infoln("Start Seaweed volume server", util.Version(), "tcp at", listeningAddress)
listener, e := util.NewListener(listeningAddress, 0)
if e != nil {

View file

@ -6,6 +6,8 @@ import (
"github.com/chrislusf/seaweedfs/weed/wdclient"
"io"
"math"
"net/url"
"strings"
"time"
"github.com/golang/protobuf/proto"
@ -108,6 +110,9 @@ func retriedFetchChunkData(urlStrings []string, cipherKey []byte, isGzipped bool
for waitTime := time.Second; waitTime < util.RetryWaitTime; waitTime += waitTime / 2 {
for _, urlString := range urlStrings {
receivedData = receivedData[:0]
if strings.Contains(urlString, "%") {
urlString = url.PathEscape(urlString)
}
shouldRetry, err = util.ReadUrlAsStream(urlString+"?readDeleted=true", cipherKey, isGzipped, isFullChunk, offset, size, func(data []byte) {
receivedData = append(receivedData, data...)
})

View file

@ -2,14 +2,12 @@ package topology
import (
"fmt"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/chrislusf/seaweedfs/weed/storage/types"
"github.com/chrislusf/seaweedfs/weed/util"
"strconv"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage"
)
type DataNode struct {
@ -207,7 +205,7 @@ func (dn *DataNode) MatchLocation(ip string, port int) bool {
}
func (dn *DataNode) Url() string {
return dn.Ip + ":" + strconv.Itoa(dn.Port)
return util.JoinHostPort(dn.Ip, dn.Port)
}
func (dn *DataNode) ToMap() interface{} {

View file

@ -3,7 +3,7 @@ package topology
import (
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/storage/types"
"strconv"
"github.com/chrislusf/seaweedfs/weed/util"
"time"
)
@ -38,7 +38,7 @@ func (r *Rack) GetOrCreateDataNode(ip string, port int, publicUrl string, maxVol
return dn
}
}
dn := NewDataNode(ip + ":" + strconv.Itoa(port))
dn := NewDataNode(util.JoinHostPort(ip, port))
dn.Ip = ip
dn.Port = port
dn.PublicUrl = publicUrl

View file

@ -179,7 +179,7 @@ func getWritableRemoteReplications(s *storage.Store, grpcDialOption grpc.DialOpt
// not on local store, or has replications
lookupResult, lookupErr := operation.LookupVolumeId(masterFn, grpcDialOption, volumeId.String())
if lookupErr == nil {
selfUrl := s.Ip + ":" + strconv.Itoa(s.Port)
selfUrl := util.JoinHostPort(s.Ip, s.Port)
for _, location := range lookupResult.Locations {
if location.Url != selfUrl {
remoteLocations = append(remoteLocations, location)

View file

@ -15,6 +15,18 @@ func DetectedHostAddress() string {
return ""
}
if v4Address := selectIpV4(netInterfaces, true); v4Address != ""{
return v4Address
}
if v6Address := selectIpV4(netInterfaces, false); v6Address != ""{
return v6Address
}
return "localhost"
}
func selectIpV4(netInterfaces []net.Interface, isIpV4 bool) string {
for _, netInterface := range netInterfaces {
if (netInterface.Flags & net.FlagUp) == 0 {
continue
@ -26,14 +38,19 @@ func DetectedHostAddress() string {
for _, a := range addrs {
if ipNet, ok := a.(*net.IPNet); ok && !ipNet.IP.IsLoopback() {
if ipNet.IP.To4() != nil {
return ipNet.IP.String()
if isIpV4 {
if ipNet.IP.To4() != nil {
return ipNet.IP.String()
}
} else {
if ipNet.IP.To16() != nil {
return ipNet.IP.String()
}
}
}
}
}
return "localhost"
return ""
}
func JoinHostPort(host string, port int) string {

View file

@ -1,177 +0,0 @@
package net2
import (
"fmt"
"log"
"net"
"os"
"strings"
"sync"
)
var myHostname string
var myHostnameOnce sync.Once
// Like os.Hostname but caches first successful result, making it cheap to call it
// over and over.
// It will also crash whole process if fetching Hostname fails!
func MyHostname() string {
myHostnameOnce.Do(func() {
var err error
myHostname, err = os.Hostname()
if err != nil {
log.Fatal(err)
}
})
return myHostname
}
var myIp4 *net.IPAddr
var myIp4Once sync.Once
// Resolves `MyHostname()` to an Ip4 address. Caches first successful result, making it
// cheap to call it over and over.
// It will also crash whole process if resolving the IP fails!
func MyIp4() *net.IPAddr {
myIp4Once.Do(func() {
var err error
myIp4, err = net.ResolveIPAddr("ip4", MyHostname())
if err != nil {
log.Fatal(err)
}
})
return myIp4
}
var myIp6 *net.IPAddr
var myIp6Once sync.Once
// Resolves `MyHostname()` to an Ip6 address. Caches first successful result, making it
// cheap to call it over and over.
// It will also crash whole process if resolving the IP fails!
func MyIp6() *net.IPAddr {
myIp6Once.Do(func() {
var err error
myIp6, err = net.ResolveIPAddr("ip6", MyHostname())
if err != nil {
log.Fatal(err)
}
})
return myIp6
}
// This returns the list of local ip addresses which other hosts can connect
// to (NOTE: Loopback ip is ignored).
// Also resolves Hostname to an address and adds it to the list too, so
// IPs from /etc/hosts can work too.
func GetLocalIPs() ([]*net.IP, error) {
hostname, err := os.Hostname()
if err != nil {
return nil, fmt.Errorf("Failed to lookup hostname: %v", err)
}
// Resolves IP Address from Hostname, this way overrides in /etc/hosts
// can work too for IP resolution.
ipInfo, err := net.ResolveIPAddr("ip4", hostname)
if err != nil {
return nil, fmt.Errorf("Failed to resolve ip: %v", err)
}
ips := []*net.IP{&ipInfo.IP}
// TODO(zviad): Is rest of the code really necessary?
addrs, err := net.InterfaceAddrs()
if err != nil {
return nil, fmt.Errorf("Failed to get interface addresses: %v", err)
}
for _, addr := range addrs {
ipnet, ok := addr.(*net.IPNet)
if !ok {
continue
}
if ipnet.IP.IsLoopback() {
continue
}
ips = append(ips, &ipnet.IP)
}
return ips, nil
}
var localhostIPNets []*net.IPNet
func init() {
for _, mask := range []string{"127.0.0.1/8", "::1/128"} {
_, ipnet, err := net.ParseCIDR(mask)
if err != nil {
panic(err)
}
localhostIPNets = append(localhostIPNets, ipnet)
}
}
func IsLocalhostIp(ipStr string) bool {
ip := net.ParseIP(ipStr)
if ip == nil {
return false
}
for _, ipnet := range localhostIPNets {
if ipnet.Contains(ip) {
return true
}
}
return false
}
// Given a host string, return true if the host is an ip (v4/v6) localhost.
func IsLocalhost(host string) bool {
return IsLocalhostIp(host) ||
host == "localhost" ||
host == "ip6-localhost" ||
host == "ipv6-localhost"
}
// Resolves hostnames in addresses to actual IP4 addresses. Skips all invalid addresses
// and all addresses that can't be resolved.
// `addrs` are assumed to be of form: ["<hostname>:<port>", ...]
// Returns an error in addition to resolved addresses if not all resolutions succeed.
func ResolveIP4s(addrs []string) ([]string, error) {
resolvedAddrs := make([]string, 0, len(addrs))
var lastErr error
for _, server := range addrs {
hostPort := strings.Split(server, ":")
if len(hostPort) != 2 {
lastErr = fmt.Errorf("Skipping invalid address: %s", server)
continue
}
ip, err := net.ResolveIPAddr("ip4", hostPort[0])
if err != nil {
lastErr = err
continue
}
resolvedAddrs = append(resolvedAddrs, ip.IP.String()+":"+hostPort[1])
}
return resolvedAddrs, lastErr
}
func LookupValidAddrs() (map[string]bool, error) {
hostName, err := os.Hostname()
if err != nil {
return nil, err
}
addrs, err := net.LookupHost(hostName)
if err != nil {
return nil, err
}
validAddrs := make(map[string]bool)
validAddrs[hostName] = true
for _, addr := range addrs {
validAddrs[addr] = true
}
// Special case localhost/127.0.0.1 so that this works on devVMs. It should
// have no affect in production.
validAddrs["127.0.0.1"] = true
validAddrs["localhost"] = true
return validAddrs, nil
}