mirror of https://github.com/minio/minio.git
Avoid using fastjson parser pool, move back to jsoniter (#8190)
It looks like from implementation point of view fastjson parser pool doesn't behave the same way as expected when dealing many `xl.json` from multiple disks. The fastjson parser pool usage ends up returning incorrect xl.json entries for checksums, with references pointing to older entries. This led to the subtle bug where checksum info is duplicated from a previous xl.json read of a different file from different disk.
This commit is contained in:
parent
428836d4e1
commit
b52a3e523c
|
@ -1854,7 +1854,7 @@ func (a adminAPIHandlers) ConsoleLogHandler(w http.ResponseWriter, r *http.Reque
|
||||||
globalConsoleSys.Subscribe(logCh, doneCh, node, limitLines, nil)
|
globalConsoleSys.Subscribe(logCh, doneCh, node, limitLines, nil)
|
||||||
|
|
||||||
for _, peer := range peers {
|
for _, peer := range peers {
|
||||||
if node == "" || strings.ToLower(peer.host.Name) == strings.ToLower(node) {
|
if node == "" || strings.EqualFold(peer.host.Name, node) {
|
||||||
peer.ConsoleLog(logCh, doneCh)
|
peer.ConsoleLog(logCh, doneCh)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,7 @@ import (
|
||||||
"bytes"
|
"bytes"
|
||||||
"context"
|
"context"
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
|
"fmt"
|
||||||
"path"
|
"path"
|
||||||
"runtime"
|
"runtime"
|
||||||
"strings"
|
"strings"
|
||||||
|
@ -136,7 +137,10 @@ func (sys *ConfigSys) Init(objAPI ObjectLayer) error {
|
||||||
// of the object layer.
|
// of the object layer.
|
||||||
// - Write quorum not met when upgrading configuration
|
// - Write quorum not met when upgrading configuration
|
||||||
// version is needed.
|
// version is needed.
|
||||||
for range newRetryTimerSimple(doneCh) {
|
retryTimerCh := newRetryTimerSimple(doneCh)
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
case <-retryTimerCh:
|
||||||
if err := initConfig(objAPI); err != nil {
|
if err := initConfig(objAPI); err != nil {
|
||||||
if strings.Contains(err.Error(), InsufficientReadQuorum{}.Error()) ||
|
if strings.Contains(err.Error(), InsufficientReadQuorum{}.Error()) ||
|
||||||
strings.Contains(err.Error(), InsufficientWriteQuorum{}.Error()) {
|
strings.Contains(err.Error(), InsufficientWriteQuorum{}.Error()) {
|
||||||
|
@ -145,9 +149,11 @@ func (sys *ConfigSys) Init(objAPI ObjectLayer) error {
|
||||||
}
|
}
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
break
|
|
||||||
}
|
|
||||||
return nil
|
return nil
|
||||||
|
case <-globalOSSignalCh:
|
||||||
|
return fmt.Errorf("Initializing config sub-system gracefully stopped")
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewConfigSys - creates new config system object.
|
// NewConfigSys - creates new config system object.
|
||||||
|
|
|
@ -27,10 +27,10 @@ import (
|
||||||
pathutil "path"
|
pathutil "path"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
jsoniter "github.com/json-iterator/go"
|
||||||
"github.com/minio/minio/cmd/logger"
|
"github.com/minio/minio/cmd/logger"
|
||||||
"github.com/minio/minio/pkg/lock"
|
"github.com/minio/minio/pkg/lock"
|
||||||
"github.com/minio/minio/pkg/mimedb"
|
"github.com/minio/minio/pkg/mimedb"
|
||||||
"github.com/valyala/fastjson"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// FS format, and object metadata.
|
// FS format, and object metadata.
|
||||||
|
@ -203,37 +203,6 @@ func (m *fsMetaV1) WriteTo(lk *lock.LockedFile) (n int64, err error) {
|
||||||
return fi.Size(), nil
|
return fi.Size(), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func parseFSVersion(v *fastjson.Value) string {
|
|
||||||
return string(v.GetStringBytes("version"))
|
|
||||||
}
|
|
||||||
|
|
||||||
func parseFSMetaMap(v *fastjson.Value) map[string]string {
|
|
||||||
metaMap := make(map[string]string)
|
|
||||||
// Get fsMetaV1.Meta map.
|
|
||||||
v.GetObject("meta").Visit(func(k []byte, kv *fastjson.Value) {
|
|
||||||
metaMap[string(k)] = string(kv.GetStringBytes())
|
|
||||||
})
|
|
||||||
return metaMap
|
|
||||||
}
|
|
||||||
|
|
||||||
func parseFSPartsArray(v *fastjson.Value) []ObjectPartInfo {
|
|
||||||
// Get xlMetaV1.Parts array
|
|
||||||
var partsArray []ObjectPartInfo
|
|
||||||
for _, result := range v.GetArray("parts") {
|
|
||||||
partsArray = append(partsArray, ObjectPartInfo{
|
|
||||||
Number: result.GetInt("number"),
|
|
||||||
Name: string(result.GetStringBytes("name")),
|
|
||||||
ETag: string(result.GetStringBytes("etag")),
|
|
||||||
Size: result.GetInt64("size"),
|
|
||||||
ActualSize: result.GetInt64("actualSize"),
|
|
||||||
})
|
|
||||||
}
|
|
||||||
return partsArray
|
|
||||||
}
|
|
||||||
|
|
||||||
// fs.json parser pool
|
|
||||||
var fsParserPool fastjson.ParserPool
|
|
||||||
|
|
||||||
func (m *fsMetaV1) ReadFrom(ctx context.Context, lk *lock.LockedFile) (n int64, err error) {
|
func (m *fsMetaV1) ReadFrom(ctx context.Context, lk *lock.LockedFile) (n int64, err error) {
|
||||||
var fsMetaBuf []byte
|
var fsMetaBuf []byte
|
||||||
fi, err := lk.Stat()
|
fi, err := lk.Stat()
|
||||||
|
@ -253,18 +222,11 @@ func (m *fsMetaV1) ReadFrom(ctx context.Context, lk *lock.LockedFile) (n int64,
|
||||||
return 0, io.EOF
|
return 0, io.EOF
|
||||||
}
|
}
|
||||||
|
|
||||||
parser := fsParserPool.Get()
|
var json = jsoniter.ConfigCompatibleWithStandardLibrary
|
||||||
defer fsParserPool.Put(parser)
|
if err = json.Unmarshal(fsMetaBuf, m); err != nil {
|
||||||
|
|
||||||
var v *fastjson.Value
|
|
||||||
v, err = parser.ParseBytes(fsMetaBuf)
|
|
||||||
if err != nil {
|
|
||||||
return 0, err
|
return 0, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// obtain version.
|
|
||||||
m.Version = parseFSVersion(v)
|
|
||||||
|
|
||||||
// Verify if the format is valid, return corrupted format
|
// Verify if the format is valid, return corrupted format
|
||||||
// for unrecognized formats.
|
// for unrecognized formats.
|
||||||
if !isFSMetaValid(m.Version) {
|
if !isFSMetaValid(m.Version) {
|
||||||
|
@ -273,12 +235,6 @@ func (m *fsMetaV1) ReadFrom(ctx context.Context, lk *lock.LockedFile) (n int64,
|
||||||
return 0, errCorruptedFormat
|
return 0, errCorruptedFormat
|
||||||
}
|
}
|
||||||
|
|
||||||
// obtain parts information
|
|
||||||
m.Parts = parseFSPartsArray(v)
|
|
||||||
|
|
||||||
// obtain metadata.
|
|
||||||
m.Meta = parseFSMetaMap(v)
|
|
||||||
|
|
||||||
// Success.
|
// Success.
|
||||||
return int64(len(fsMetaBuf)), nil
|
return int64(len(fsMetaBuf)), nil
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,9 +28,9 @@ import (
|
||||||
"strings"
|
"strings"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
jsoniter "github.com/json-iterator/go"
|
||||||
"github.com/minio/minio/cmd/logger"
|
"github.com/minio/minio/cmd/logger"
|
||||||
mioutil "github.com/minio/minio/pkg/ioutil"
|
mioutil "github.com/minio/minio/pkg/ioutil"
|
||||||
"github.com/valyala/fastjson"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// Returns EXPORT/.minio.sys/multipart/SHA256/UPLOADID
|
// Returns EXPORT/.minio.sys/multipart/SHA256/UPLOADID
|
||||||
|
@ -472,16 +472,13 @@ func (fs *FSObjects) ListObjectParts(ctx context.Context, bucket, object, upload
|
||||||
return result, err
|
return result, err
|
||||||
}
|
}
|
||||||
|
|
||||||
parser := fsParserPool.Get()
|
var fsMeta fsMetaV1
|
||||||
defer fsParserPool.Put(parser)
|
var json = jsoniter.ConfigCompatibleWithStandardLibrary
|
||||||
|
if err = json.Unmarshal(fsMetaBytes, &fsMeta); err != nil {
|
||||||
var v *fastjson.Value
|
|
||||||
v, err = parser.ParseBytes(fsMetaBytes)
|
|
||||||
if err != nil {
|
|
||||||
return result, err
|
return result, err
|
||||||
}
|
}
|
||||||
|
|
||||||
result.UserDefined = parseFSMetaMap(v)
|
result.UserDefined = fsMeta.Meta
|
||||||
return result, nil
|
return result, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
17
cmd/fs-v1.go
17
cmd/fs-v1.go
|
@ -30,6 +30,7 @@ import (
|
||||||
"sync/atomic"
|
"sync/atomic"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
jsoniter "github.com/json-iterator/go"
|
||||||
"github.com/minio/minio-go/v6/pkg/s3utils"
|
"github.com/minio/minio-go/v6/pkg/s3utils"
|
||||||
"github.com/minio/minio/cmd/logger"
|
"github.com/minio/minio/cmd/logger"
|
||||||
"github.com/minio/minio/pkg/lifecycle"
|
"github.com/minio/minio/pkg/lifecycle"
|
||||||
|
@ -38,7 +39,6 @@ import (
|
||||||
"github.com/minio/minio/pkg/mimedb"
|
"github.com/minio/minio/pkg/mimedb"
|
||||||
"github.com/minio/minio/pkg/mountinfo"
|
"github.com/minio/minio/pkg/mountinfo"
|
||||||
"github.com/minio/minio/pkg/policy"
|
"github.com/minio/minio/pkg/policy"
|
||||||
"github.com/valyala/fastjson"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// Default etag is used for pre-existing objects.
|
// Default etag is used for pre-existing objects.
|
||||||
|
@ -1093,22 +1093,19 @@ func (fs *FSObjects) getObjectETag(ctx context.Context, bucket, entry string, lo
|
||||||
return "", toObjectErr(err, bucket, entry)
|
return "", toObjectErr(err, bucket, entry)
|
||||||
}
|
}
|
||||||
|
|
||||||
parser := fsParserPool.Get()
|
var fsMeta fsMetaV1
|
||||||
defer fsParserPool.Put(parser)
|
var json = jsoniter.ConfigCompatibleWithStandardLibrary
|
||||||
|
if err = json.Unmarshal(fsMetaBuf, &fsMeta); err != nil {
|
||||||
var v *fastjson.Value
|
return "", err
|
||||||
v, err = parser.ParseBytes(fsMetaBuf)
|
|
||||||
if err != nil {
|
|
||||||
return "", toObjectErr(err, bucket, entry)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Check if FS metadata is valid, if not return error.
|
// Check if FS metadata is valid, if not return error.
|
||||||
if !isFSMetaValid(parseFSVersion(v)) {
|
if !isFSMetaValid(fsMeta.Version) {
|
||||||
logger.LogIf(ctx, errCorruptedFormat)
|
logger.LogIf(ctx, errCorruptedFormat)
|
||||||
return "", toObjectErr(errCorruptedFormat, bucket, entry)
|
return "", toObjectErr(errCorruptedFormat, bucket, entry)
|
||||||
}
|
}
|
||||||
|
|
||||||
return extractETag(parseFSMetaMap(v)), nil
|
return extractETag(fsMeta.Meta), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListObjects - list all objects at prefix upto maxKeys., optionally delimited by '/'. Maintains the list pool
|
// ListObjects - list all objects at prefix upto maxKeys., optionally delimited by '/'. Maintains the list pool
|
||||||
|
|
|
@ -24,10 +24,10 @@ import (
|
||||||
"net/http"
|
"net/http"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
jsoniter "github.com/json-iterator/go"
|
||||||
minio "github.com/minio/minio/cmd"
|
minio "github.com/minio/minio/cmd"
|
||||||
"github.com/minio/minio/cmd/logger"
|
"github.com/minio/minio/cmd/logger"
|
||||||
"github.com/minio/minio/pkg/hash"
|
"github.com/minio/minio/pkg/hash"
|
||||||
"github.com/valyala/fastjson"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
|
@ -137,101 +137,11 @@ func (m gwMetaV1) ObjectToPartOffset(ctx context.Context, offset int64) (partInd
|
||||||
return 0, 0, minio.InvalidRange{}
|
return 0, 0, minio.InvalidRange{}
|
||||||
}
|
}
|
||||||
|
|
||||||
// parses gateway metadata stat info from metadata json
|
// Constructs GWMetaV1 using `jsoniter` lib to retrieve each field.
|
||||||
func parseGWStat(v *fastjson.Value) (si minio.StatInfo, err error) {
|
|
||||||
// obtain stat info.
|
|
||||||
st := v.GetObject("stat")
|
|
||||||
var mb []byte
|
|
||||||
mb, err = st.Get("modTime").StringBytes()
|
|
||||||
if err != nil {
|
|
||||||
return si, err
|
|
||||||
}
|
|
||||||
// fetching modTime.
|
|
||||||
si.ModTime, err = time.Parse(time.RFC3339, string(mb))
|
|
||||||
if err != nil {
|
|
||||||
return si, err
|
|
||||||
}
|
|
||||||
// obtain Stat.Size .
|
|
||||||
si.Size, err = st.Get("size").Int64()
|
|
||||||
if err != nil {
|
|
||||||
return si, err
|
|
||||||
}
|
|
||||||
return si, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// parses gateway metadata version from metadata json
|
|
||||||
func parseGWVersion(v *fastjson.Value) string {
|
|
||||||
return string(v.GetStringBytes("version"))
|
|
||||||
}
|
|
||||||
|
|
||||||
// parses gateway ETag from metadata json
|
|
||||||
func parseGWETag(v *fastjson.Value) string {
|
|
||||||
return string(v.GetStringBytes("etag"))
|
|
||||||
}
|
|
||||||
|
|
||||||
// parses gateway metadata format from metadata json
|
|
||||||
func parseGWFormat(v *fastjson.Value) string {
|
|
||||||
return string(v.GetStringBytes("format"))
|
|
||||||
}
|
|
||||||
|
|
||||||
// parses gateway metadata json to get list of ObjectPartInfo
|
|
||||||
func parseGWParts(v *fastjson.Value) []minio.ObjectPartInfo {
|
|
||||||
// Parse the GW Parts.
|
|
||||||
partsResult := v.GetArray("parts")
|
|
||||||
partInfo := make([]minio.ObjectPartInfo, len(partsResult))
|
|
||||||
for i, p := range partsResult {
|
|
||||||
partInfo[i] = minio.ObjectPartInfo{
|
|
||||||
Number: p.GetInt("number"),
|
|
||||||
Name: string(p.GetStringBytes("name")),
|
|
||||||
ETag: string(p.GetStringBytes("etag")),
|
|
||||||
Size: p.GetInt64("size"),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return partInfo
|
|
||||||
}
|
|
||||||
|
|
||||||
// parses gateway metadata json to get the metadata map
|
|
||||||
func parseGWMetaMap(v *fastjson.Value) map[string]string {
|
|
||||||
metaMap := make(map[string]string)
|
|
||||||
// Get gwMetaV1.Meta map.
|
|
||||||
v.GetObject("meta").Visit(func(k []byte, kv *fastjson.Value) {
|
|
||||||
metaMap[string(k)] = string(kv.GetStringBytes())
|
|
||||||
})
|
|
||||||
return metaMap
|
|
||||||
}
|
|
||||||
|
|
||||||
var gwParserPool fastjson.ParserPool
|
|
||||||
|
|
||||||
// Constructs GWMetaV1 using `fastjson` lib to retrieve each field.
|
|
||||||
func gwMetaUnmarshalJSON(ctx context.Context, gwMetaBuf []byte) (gwMeta gwMetaV1, err error) {
|
func gwMetaUnmarshalJSON(ctx context.Context, gwMetaBuf []byte) (gwMeta gwMetaV1, err error) {
|
||||||
parser := gwParserPool.Get()
|
var json = jsoniter.ConfigCompatibleWithStandardLibrary
|
||||||
defer gwParserPool.Put(parser)
|
err = json.Unmarshal(gwMetaBuf, &gwMeta)
|
||||||
|
|
||||||
var v *fastjson.Value
|
|
||||||
v, err = parser.ParseBytes(gwMetaBuf)
|
|
||||||
if err != nil {
|
|
||||||
return gwMeta, err
|
return gwMeta, err
|
||||||
}
|
|
||||||
|
|
||||||
// obtain version.
|
|
||||||
gwMeta.Version = parseGWVersion(v)
|
|
||||||
// obtain format.
|
|
||||||
gwMeta.Format = parseGWFormat(v)
|
|
||||||
// Parse gwMetaV1.Stat .
|
|
||||||
stat, err := parseGWStat(v)
|
|
||||||
if err != nil {
|
|
||||||
logger.LogIf(ctx, err)
|
|
||||||
return gwMeta, err
|
|
||||||
}
|
|
||||||
gwMeta.ETag = parseGWETag(v)
|
|
||||||
gwMeta.Stat = stat
|
|
||||||
|
|
||||||
// Parse the GW Parts.
|
|
||||||
gwMeta.Parts = parseGWParts(v)
|
|
||||||
// parse gwMetaV1.
|
|
||||||
gwMeta.Meta = parseGWMetaMap(v)
|
|
||||||
|
|
||||||
return gwMeta, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// readGWMeta reads `dare.meta` and returns back GW metadata structure.
|
// readGWMeta reads `dare.meta` and returns back GW metadata structure.
|
||||||
|
|
|
@ -20,6 +20,7 @@ import (
|
||||||
"bytes"
|
"bytes"
|
||||||
"context"
|
"context"
|
||||||
"encoding/xml"
|
"encoding/xml"
|
||||||
|
"fmt"
|
||||||
"path"
|
"path"
|
||||||
"strings"
|
"strings"
|
||||||
"sync"
|
"sync"
|
||||||
|
@ -134,7 +135,10 @@ func (sys *LifecycleSys) Init(objAPI ObjectLayer) error {
|
||||||
// the following reasons:
|
// the following reasons:
|
||||||
// - Read quorum is lost just after the initialization
|
// - Read quorum is lost just after the initialization
|
||||||
// of the object layer.
|
// of the object layer.
|
||||||
for range newRetryTimerSimple(doneCh) {
|
retryTimerCh := newRetryTimerSimple(doneCh)
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
case <-retryTimerCh:
|
||||||
// Load LifecycleSys once during boot.
|
// Load LifecycleSys once during boot.
|
||||||
if err := sys.refresh(objAPI); err != nil {
|
if err := sys.refresh(objAPI); err != nil {
|
||||||
if err == errDiskNotFound ||
|
if err == errDiskNotFound ||
|
||||||
|
@ -145,9 +149,11 @@ func (sys *LifecycleSys) Init(objAPI ObjectLayer) error {
|
||||||
}
|
}
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
break
|
|
||||||
}
|
|
||||||
return nil
|
return nil
|
||||||
|
case <-globalOSSignalCh:
|
||||||
|
return fmt.Errorf("Initializing Lifecycle sub-system gracefully stopped")
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Refresh LifecycleSys.
|
// Refresh LifecycleSys.
|
||||||
|
|
|
@ -808,7 +808,10 @@ func (sys *NotificationSys) Init(objAPI ObjectLayer) error {
|
||||||
// the following reasons:
|
// the following reasons:
|
||||||
// - Read quorum is lost just after the initialization
|
// - Read quorum is lost just after the initialization
|
||||||
// of the object layer.
|
// of the object layer.
|
||||||
for range newRetryTimerSimple(doneCh) {
|
retryTimerCh := newRetryTimerSimple(doneCh)
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
case <-retryTimerCh:
|
||||||
if err := sys.refresh(objAPI); err != nil {
|
if err := sys.refresh(objAPI); err != nil {
|
||||||
if err == errDiskNotFound ||
|
if err == errDiskNotFound ||
|
||||||
strings.Contains(err.Error(), InsufficientReadQuorum{}.Error()) ||
|
strings.Contains(err.Error(), InsufficientReadQuorum{}.Error()) ||
|
||||||
|
@ -818,9 +821,11 @@ func (sys *NotificationSys) Init(objAPI ObjectLayer) error {
|
||||||
}
|
}
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
break
|
|
||||||
}
|
|
||||||
return nil
|
return nil
|
||||||
|
case <-globalOSSignalCh:
|
||||||
|
return fmt.Errorf("Initializing Notification sub-system gracefully stopped")
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// AddRulesMap - adds rules map for bucket name.
|
// AddRulesMap - adds rules map for bucket name.
|
||||||
|
|
|
@ -163,7 +163,10 @@ func (sys *PolicySys) Init(objAPI ObjectLayer) error {
|
||||||
// the following reasons:
|
// the following reasons:
|
||||||
// - Read quorum is lost just after the initialization
|
// - Read quorum is lost just after the initialization
|
||||||
// of the object layer.
|
// of the object layer.
|
||||||
for range newRetryTimerSimple(doneCh) {
|
retryTimerCh := newRetryTimerSimple(doneCh)
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
case <-retryTimerCh:
|
||||||
// Load PolicySys once during boot.
|
// Load PolicySys once during boot.
|
||||||
if err := sys.refresh(objAPI); err != nil {
|
if err := sys.refresh(objAPI); err != nil {
|
||||||
if err == errDiskNotFound ||
|
if err == errDiskNotFound ||
|
||||||
|
@ -174,9 +177,11 @@ func (sys *PolicySys) Init(objAPI ObjectLayer) error {
|
||||||
}
|
}
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
break
|
|
||||||
}
|
|
||||||
return nil
|
return nil
|
||||||
|
case <-globalOSSignalCh:
|
||||||
|
return fmt.Errorf("Initializing Policy sub-system gracefully stopped")
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewPolicySys - creates new policy system.
|
// NewPolicySys - creates new policy system.
|
||||||
|
|
|
@ -75,8 +75,6 @@ func (c ChecksumInfo) MarshalJSON() ([]byte, error) {
|
||||||
|
|
||||||
// UnmarshalJSON - should never be called, instead xlMetaV1UnmarshalJSON() should be used.
|
// UnmarshalJSON - should never be called, instead xlMetaV1UnmarshalJSON() should be used.
|
||||||
func (c *ChecksumInfo) UnmarshalJSON(data []byte) error {
|
func (c *ChecksumInfo) UnmarshalJSON(data []byte) error {
|
||||||
logger.LogIf(context.Background(), errUnexpected)
|
|
||||||
|
|
||||||
var info checksumInfoJSON
|
var info checksumInfoJSON
|
||||||
if err := json.Unmarshal(data, &info); err != nil {
|
if err := json.Unmarshal(data, &info); err != nil {
|
||||||
return err
|
return err
|
||||||
|
|
|
@ -18,15 +18,13 @@ package cmd
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"encoding/hex"
|
|
||||||
"errors"
|
"errors"
|
||||||
"hash/crc32"
|
"hash/crc32"
|
||||||
"path"
|
"path"
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
|
||||||
|
|
||||||
|
jsoniter "github.com/json-iterator/go"
|
||||||
"github.com/minio/minio/cmd/logger"
|
"github.com/minio/minio/cmd/logger"
|
||||||
"github.com/valyala/fastjson"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// Returns number of errors that occurred the most (incl. nil) and the
|
// Returns number of errors that occurred the most (incl. nil) and the
|
||||||
|
@ -117,167 +115,11 @@ func hashOrder(key string, cardinality int) []int {
|
||||||
return nums
|
return nums
|
||||||
}
|
}
|
||||||
|
|
||||||
func parseXLStat(v *fastjson.Value) (si statInfo, err error) {
|
// Constructs xlMetaV1 using `jsoniter` lib.
|
||||||
// obtain stat info.
|
|
||||||
st := v.GetObject("stat")
|
|
||||||
var mb []byte
|
|
||||||
mb, err = st.Get("modTime").StringBytes()
|
|
||||||
if err != nil {
|
|
||||||
return si, err
|
|
||||||
}
|
|
||||||
// fetching modTime.
|
|
||||||
si.ModTime, err = time.Parse(time.RFC3339, string(mb))
|
|
||||||
if err != nil {
|
|
||||||
return si, err
|
|
||||||
}
|
|
||||||
// obtain Stat.Size .
|
|
||||||
si.Size, err = st.Get("size").Int64()
|
|
||||||
if err != nil {
|
|
||||||
return si, err
|
|
||||||
}
|
|
||||||
return si, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func parseXLVersion(v *fastjson.Value) string {
|
|
||||||
return string(v.GetStringBytes("version"))
|
|
||||||
}
|
|
||||||
|
|
||||||
func parseXLFormat(v *fastjson.Value) string {
|
|
||||||
return string(v.GetStringBytes("format"))
|
|
||||||
}
|
|
||||||
|
|
||||||
func parseXLRelease(v *fastjson.Value) string {
|
|
||||||
return string(v.GetStringBytes("minio", "release"))
|
|
||||||
}
|
|
||||||
|
|
||||||
func parseXLErasureInfo(ctx context.Context, v *fastjson.Value) (ErasureInfo, error) {
|
|
||||||
erasure := ErasureInfo{}
|
|
||||||
// parse the xlV1Meta.Erasure.Distribution.
|
|
||||||
er := v.GetObject("erasure")
|
|
||||||
disResult := er.Get("distribution").GetArray()
|
|
||||||
distribution := make([]int, len(disResult))
|
|
||||||
var err error
|
|
||||||
for i, dis := range disResult {
|
|
||||||
distribution[i], err = dis.Int()
|
|
||||||
if err != nil {
|
|
||||||
return erasure, err
|
|
||||||
}
|
|
||||||
}
|
|
||||||
erasure.Distribution = distribution
|
|
||||||
|
|
||||||
erasure.Algorithm = string(er.Get("algorithm").GetStringBytes())
|
|
||||||
erasure.DataBlocks = er.Get("data").GetInt()
|
|
||||||
erasure.ParityBlocks = er.Get("parity").GetInt()
|
|
||||||
erasure.BlockSize = er.Get("blockSize").GetInt64()
|
|
||||||
erasure.Index = er.Get("index").GetInt()
|
|
||||||
checkSumsResult := er.Get("checksum").GetArray()
|
|
||||||
|
|
||||||
// Parse xlMetaV1.Erasure.Checksum array.
|
|
||||||
checkSums := make([]ChecksumInfo, len(checkSumsResult))
|
|
||||||
for i, ck := range checkSumsResult {
|
|
||||||
algorithm := BitrotAlgorithmFromString(string(ck.GetStringBytes("algorithm")))
|
|
||||||
if !algorithm.Available() {
|
|
||||||
logger.LogIf(ctx, errBitrotHashAlgoInvalid)
|
|
||||||
return erasure, errBitrotHashAlgoInvalid
|
|
||||||
}
|
|
||||||
srcHash := ck.GetStringBytes("hash")
|
|
||||||
n, err := hex.Decode(srcHash, srcHash)
|
|
||||||
if err != nil {
|
|
||||||
logger.LogIf(ctx, err)
|
|
||||||
return erasure, err
|
|
||||||
}
|
|
||||||
nmb := ck.GetStringBytes("name")
|
|
||||||
if nmb == nil {
|
|
||||||
return erasure, errCorruptedFormat
|
|
||||||
}
|
|
||||||
checkSums[i] = ChecksumInfo{
|
|
||||||
Name: string(nmb),
|
|
||||||
Algorithm: algorithm,
|
|
||||||
Hash: srcHash[:n],
|
|
||||||
}
|
|
||||||
}
|
|
||||||
erasure.Checksums = checkSums
|
|
||||||
return erasure, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func parseXLParts(partsResult []*fastjson.Value) []ObjectPartInfo {
|
|
||||||
// Parse the XL Parts.
|
|
||||||
partInfo := make([]ObjectPartInfo, len(partsResult))
|
|
||||||
for i, p := range partsResult {
|
|
||||||
partInfo[i] = ObjectPartInfo{
|
|
||||||
Number: p.GetInt("number"),
|
|
||||||
Name: string(p.GetStringBytes("name")),
|
|
||||||
ETag: string(p.GetStringBytes("etag")),
|
|
||||||
Size: p.GetInt64("size"),
|
|
||||||
ActualSize: p.GetInt64("actualSize"),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return partInfo
|
|
||||||
}
|
|
||||||
|
|
||||||
func parseXLMetaMap(v *fastjson.Value) map[string]string {
|
|
||||||
metaMap := make(map[string]string)
|
|
||||||
// Get xlMetaV1.Meta map.
|
|
||||||
v.GetObject("meta").Visit(func(k []byte, kv *fastjson.Value) {
|
|
||||||
metaMap[string(k)] = string(kv.GetStringBytes())
|
|
||||||
})
|
|
||||||
return metaMap
|
|
||||||
}
|
|
||||||
|
|
||||||
// xl.json Parser pool
|
|
||||||
var xlParserPool fastjson.ParserPool
|
|
||||||
|
|
||||||
// Constructs XLMetaV1 using `fastjson` lib to retrieve each field.
|
|
||||||
func xlMetaV1UnmarshalJSON(ctx context.Context, xlMetaBuf []byte) (xlMeta xlMetaV1, err error) {
|
func xlMetaV1UnmarshalJSON(ctx context.Context, xlMetaBuf []byte) (xlMeta xlMetaV1, err error) {
|
||||||
parser := xlParserPool.Get()
|
var json = jsoniter.ConfigCompatibleWithStandardLibrary
|
||||||
defer xlParserPool.Put(parser)
|
err = json.Unmarshal(xlMetaBuf, &xlMeta)
|
||||||
|
|
||||||
var v *fastjson.Value
|
|
||||||
v, err = parser.ParseBytes(xlMetaBuf)
|
|
||||||
if err != nil {
|
|
||||||
return xlMeta, err
|
return xlMeta, err
|
||||||
}
|
|
||||||
|
|
||||||
// obtain version.
|
|
||||||
xlMeta.Version = parseXLVersion(v)
|
|
||||||
// obtain format.
|
|
||||||
xlMeta.Format = parseXLFormat(v)
|
|
||||||
|
|
||||||
// Validate if the xl.json we read is sane, return corrupted format.
|
|
||||||
if !isXLMetaFormatValid(xlMeta.Version, xlMeta.Format) {
|
|
||||||
// For version mismatchs and unrecognized format, return corrupted format.
|
|
||||||
logger.LogIf(ctx, errCorruptedFormat)
|
|
||||||
return xlMeta, errCorruptedFormat
|
|
||||||
}
|
|
||||||
|
|
||||||
// Parse xlMetaV1.Stat .
|
|
||||||
stat, err := parseXLStat(v)
|
|
||||||
if err != nil {
|
|
||||||
logger.LogIf(ctx, err)
|
|
||||||
return xlMeta, err
|
|
||||||
}
|
|
||||||
|
|
||||||
xlMeta.Stat = stat
|
|
||||||
// parse the xlV1Meta.Erasure fields.
|
|
||||||
xlMeta.Erasure, err = parseXLErasureInfo(ctx, v)
|
|
||||||
if err != nil {
|
|
||||||
return xlMeta, err
|
|
||||||
}
|
|
||||||
|
|
||||||
// Check for scenario where checksum information missing for some parts.
|
|
||||||
partsResult := v.Get("parts").GetArray()
|
|
||||||
if len(xlMeta.Erasure.Checksums) != len(partsResult) {
|
|
||||||
return xlMeta, errCorruptedFormat
|
|
||||||
}
|
|
||||||
|
|
||||||
// Parse the XL Parts.
|
|
||||||
xlMeta.Parts = parseXLParts(partsResult)
|
|
||||||
// Get the xlMetaV1.Realse field.
|
|
||||||
xlMeta.Minio.Release = parseXLRelease(v)
|
|
||||||
// parse xlMetaV1.
|
|
||||||
xlMeta.Meta = parseXLMetaMap(v)
|
|
||||||
|
|
||||||
return xlMeta, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// read xl.json from the given disk, parse and return xlV1MetaV1.Parts.
|
// read xl.json from the given disk, parse and return xlV1MetaV1.Parts.
|
||||||
|
@ -298,7 +140,7 @@ func readXLMetaParts(ctx context.Context, disk StorageAPI, bucket string, object
|
||||||
return xlMeta.Parts, xlMeta.Meta, nil
|
return xlMeta.Parts, xlMeta.Meta, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// read xl.json from the given disk and parse xlV1Meta.Stat and xlV1Meta.Meta using fastjson.
|
// read xl.json from the given disk and parse xlV1Meta.Stat and xlV1Meta.Meta using jsoniter.
|
||||||
func readXLMetaStat(ctx context.Context, disk StorageAPI, bucket string, object string) (si statInfo,
|
func readXLMetaStat(ctx context.Context, disk StorageAPI, bucket string, object string) (si statInfo,
|
||||||
mp map[string]string, e error) {
|
mp map[string]string, e error) {
|
||||||
// Reads entire `xl.json`.
|
// Reads entire `xl.json`.
|
||||||
|
|
|
@ -212,99 +212,99 @@ func getSampleXLMeta(totalParts int) xlMetaV1 {
|
||||||
return xlMeta
|
return xlMeta
|
||||||
}
|
}
|
||||||
|
|
||||||
// Compare the unmarshaled XLMetaV1 with the one obtained from fastjson parsing.
|
// Compare the unmarshaled XLMetaV1 with the one obtained from jsoniter parsing.
|
||||||
func compareXLMetaV1(t *testing.T, unMarshalXLMeta, fastjsonXLMeta xlMetaV1) {
|
func compareXLMetaV1(t *testing.T, unMarshalXLMeta, jsoniterXLMeta xlMetaV1) {
|
||||||
// Start comparing the fields of xlMetaV1 obtained from fastjson parsing with one parsed using json unmarshaling.
|
// Start comparing the fields of xlMetaV1 obtained from jsoniter parsing with one parsed using json unmarshaling.
|
||||||
if unMarshalXLMeta.Version != fastjsonXLMeta.Version {
|
if unMarshalXLMeta.Version != jsoniterXLMeta.Version {
|
||||||
t.Errorf("Expected the Version to be \"%s\", but got \"%s\".", unMarshalXLMeta.Version, fastjsonXLMeta.Version)
|
t.Errorf("Expected the Version to be \"%s\", but got \"%s\".", unMarshalXLMeta.Version, jsoniterXLMeta.Version)
|
||||||
}
|
}
|
||||||
if unMarshalXLMeta.Format != fastjsonXLMeta.Format {
|
if unMarshalXLMeta.Format != jsoniterXLMeta.Format {
|
||||||
t.Errorf("Expected the format to be \"%s\", but got \"%s\".", unMarshalXLMeta.Format, fastjsonXLMeta.Format)
|
t.Errorf("Expected the format to be \"%s\", but got \"%s\".", unMarshalXLMeta.Format, jsoniterXLMeta.Format)
|
||||||
}
|
}
|
||||||
if unMarshalXLMeta.Stat.Size != fastjsonXLMeta.Stat.Size {
|
if unMarshalXLMeta.Stat.Size != jsoniterXLMeta.Stat.Size {
|
||||||
t.Errorf("Expected the stat size to be %v, but got %v.", unMarshalXLMeta.Stat.Size, fastjsonXLMeta.Stat.Size)
|
t.Errorf("Expected the stat size to be %v, but got %v.", unMarshalXLMeta.Stat.Size, jsoniterXLMeta.Stat.Size)
|
||||||
}
|
}
|
||||||
if !unMarshalXLMeta.Stat.ModTime.Equal(fastjsonXLMeta.Stat.ModTime) {
|
if !unMarshalXLMeta.Stat.ModTime.Equal(jsoniterXLMeta.Stat.ModTime) {
|
||||||
t.Errorf("Expected the modTime to be \"%v\", but got \"%v\".", unMarshalXLMeta.Stat.ModTime, fastjsonXLMeta.Stat.ModTime)
|
t.Errorf("Expected the modTime to be \"%v\", but got \"%v\".", unMarshalXLMeta.Stat.ModTime, jsoniterXLMeta.Stat.ModTime)
|
||||||
}
|
}
|
||||||
if unMarshalXLMeta.Erasure.Algorithm != fastjsonXLMeta.Erasure.Algorithm {
|
if unMarshalXLMeta.Erasure.Algorithm != jsoniterXLMeta.Erasure.Algorithm {
|
||||||
t.Errorf("Expected the erasure algorithm to be \"%v\", but got \"%v\".", unMarshalXLMeta.Erasure.Algorithm, fastjsonXLMeta.Erasure.Algorithm)
|
t.Errorf("Expected the erasure algorithm to be \"%v\", but got \"%v\".", unMarshalXLMeta.Erasure.Algorithm, jsoniterXLMeta.Erasure.Algorithm)
|
||||||
}
|
}
|
||||||
if unMarshalXLMeta.Erasure.DataBlocks != fastjsonXLMeta.Erasure.DataBlocks {
|
if unMarshalXLMeta.Erasure.DataBlocks != jsoniterXLMeta.Erasure.DataBlocks {
|
||||||
t.Errorf("Expected the erasure data blocks to be %v, but got %v.", unMarshalXLMeta.Erasure.DataBlocks, fastjsonXLMeta.Erasure.DataBlocks)
|
t.Errorf("Expected the erasure data blocks to be %v, but got %v.", unMarshalXLMeta.Erasure.DataBlocks, jsoniterXLMeta.Erasure.DataBlocks)
|
||||||
}
|
}
|
||||||
if unMarshalXLMeta.Erasure.ParityBlocks != fastjsonXLMeta.Erasure.ParityBlocks {
|
if unMarshalXLMeta.Erasure.ParityBlocks != jsoniterXLMeta.Erasure.ParityBlocks {
|
||||||
t.Errorf("Expected the erasure parity blocks to be %v, but got %v.", unMarshalXLMeta.Erasure.ParityBlocks, fastjsonXLMeta.Erasure.ParityBlocks)
|
t.Errorf("Expected the erasure parity blocks to be %v, but got %v.", unMarshalXLMeta.Erasure.ParityBlocks, jsoniterXLMeta.Erasure.ParityBlocks)
|
||||||
}
|
}
|
||||||
if unMarshalXLMeta.Erasure.BlockSize != fastjsonXLMeta.Erasure.BlockSize {
|
if unMarshalXLMeta.Erasure.BlockSize != jsoniterXLMeta.Erasure.BlockSize {
|
||||||
t.Errorf("Expected the erasure block size to be %v, but got %v.", unMarshalXLMeta.Erasure.BlockSize, fastjsonXLMeta.Erasure.BlockSize)
|
t.Errorf("Expected the erasure block size to be %v, but got %v.", unMarshalXLMeta.Erasure.BlockSize, jsoniterXLMeta.Erasure.BlockSize)
|
||||||
}
|
}
|
||||||
if unMarshalXLMeta.Erasure.Index != fastjsonXLMeta.Erasure.Index {
|
if unMarshalXLMeta.Erasure.Index != jsoniterXLMeta.Erasure.Index {
|
||||||
t.Errorf("Expected the erasure index to be %v, but got %v.", unMarshalXLMeta.Erasure.Index, fastjsonXLMeta.Erasure.Index)
|
t.Errorf("Expected the erasure index to be %v, but got %v.", unMarshalXLMeta.Erasure.Index, jsoniterXLMeta.Erasure.Index)
|
||||||
}
|
}
|
||||||
if len(unMarshalXLMeta.Erasure.Distribution) != len(fastjsonXLMeta.Erasure.Distribution) {
|
if len(unMarshalXLMeta.Erasure.Distribution) != len(jsoniterXLMeta.Erasure.Distribution) {
|
||||||
t.Errorf("Expected the size of Erasure Distribution to be %d, but got %d.", len(unMarshalXLMeta.Erasure.Distribution), len(fastjsonXLMeta.Erasure.Distribution))
|
t.Errorf("Expected the size of Erasure Distribution to be %d, but got %d.", len(unMarshalXLMeta.Erasure.Distribution), len(jsoniterXLMeta.Erasure.Distribution))
|
||||||
} else {
|
} else {
|
||||||
for i := 0; i < len(unMarshalXLMeta.Erasure.Distribution); i++ {
|
for i := 0; i < len(unMarshalXLMeta.Erasure.Distribution); i++ {
|
||||||
if unMarshalXLMeta.Erasure.Distribution[i] != fastjsonXLMeta.Erasure.Distribution[i] {
|
if unMarshalXLMeta.Erasure.Distribution[i] != jsoniterXLMeta.Erasure.Distribution[i] {
|
||||||
t.Errorf("Expected the Erasure Distribution to be %d, got %d.", unMarshalXLMeta.Erasure.Distribution[i], fastjsonXLMeta.Erasure.Distribution[i])
|
t.Errorf("Expected the Erasure Distribution to be %d, got %d.", unMarshalXLMeta.Erasure.Distribution[i], jsoniterXLMeta.Erasure.Distribution[i])
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if len(unMarshalXLMeta.Erasure.Checksums) != len(fastjsonXLMeta.Erasure.Checksums) {
|
if len(unMarshalXLMeta.Erasure.Checksums) != len(jsoniterXLMeta.Erasure.Checksums) {
|
||||||
t.Errorf("Expected the size of Erasure Checksums to be %d, but got %d.", len(unMarshalXLMeta.Erasure.Checksums), len(fastjsonXLMeta.Erasure.Checksums))
|
t.Errorf("Expected the size of Erasure Checksums to be %d, but got %d.", len(unMarshalXLMeta.Erasure.Checksums), len(jsoniterXLMeta.Erasure.Checksums))
|
||||||
} else {
|
} else {
|
||||||
for i := 0; i < len(unMarshalXLMeta.Erasure.Checksums); i++ {
|
for i := 0; i < len(unMarshalXLMeta.Erasure.Checksums); i++ {
|
||||||
if unMarshalXLMeta.Erasure.Checksums[i].Name != fastjsonXLMeta.Erasure.Checksums[i].Name {
|
if unMarshalXLMeta.Erasure.Checksums[i].Name != jsoniterXLMeta.Erasure.Checksums[i].Name {
|
||||||
t.Errorf("Expected the Erasure Checksum Name to be \"%s\", got \"%s\".", unMarshalXLMeta.Erasure.Checksums[i].Name, fastjsonXLMeta.Erasure.Checksums[i].Name)
|
t.Errorf("Expected the Erasure Checksum Name to be \"%s\", got \"%s\".", unMarshalXLMeta.Erasure.Checksums[i].Name, jsoniterXLMeta.Erasure.Checksums[i].Name)
|
||||||
}
|
}
|
||||||
if unMarshalXLMeta.Erasure.Checksums[i].Algorithm != fastjsonXLMeta.Erasure.Checksums[i].Algorithm {
|
if unMarshalXLMeta.Erasure.Checksums[i].Algorithm != jsoniterXLMeta.Erasure.Checksums[i].Algorithm {
|
||||||
t.Errorf("Expected the Erasure Checksum Algorithm to be \"%s\", got \"%s\".", unMarshalXLMeta.Erasure.Checksums[i].Algorithm, fastjsonXLMeta.Erasure.Checksums[i].Algorithm)
|
t.Errorf("Expected the Erasure Checksum Algorithm to be \"%s\", got \"%s\".", unMarshalXLMeta.Erasure.Checksums[i].Algorithm, jsoniterXLMeta.Erasure.Checksums[i].Algorithm)
|
||||||
}
|
}
|
||||||
if !bytes.Equal(unMarshalXLMeta.Erasure.Checksums[i].Hash, fastjsonXLMeta.Erasure.Checksums[i].Hash) {
|
if !bytes.Equal(unMarshalXLMeta.Erasure.Checksums[i].Hash, jsoniterXLMeta.Erasure.Checksums[i].Hash) {
|
||||||
t.Errorf("Expected the Erasure Checksum Hash to be \"%s\", got \"%s\".", unMarshalXLMeta.Erasure.Checksums[i].Hash, fastjsonXLMeta.Erasure.Checksums[i].Hash)
|
t.Errorf("Expected the Erasure Checksum Hash to be \"%s\", got \"%s\".", unMarshalXLMeta.Erasure.Checksums[i].Hash, jsoniterXLMeta.Erasure.Checksums[i].Hash)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if unMarshalXLMeta.Minio.Release != fastjsonXLMeta.Minio.Release {
|
if unMarshalXLMeta.Minio.Release != jsoniterXLMeta.Minio.Release {
|
||||||
t.Errorf("Expected the Release string to be \"%s\", but got \"%s\".", unMarshalXLMeta.Minio.Release, fastjsonXLMeta.Minio.Release)
|
t.Errorf("Expected the Release string to be \"%s\", but got \"%s\".", unMarshalXLMeta.Minio.Release, jsoniterXLMeta.Minio.Release)
|
||||||
}
|
}
|
||||||
if len(unMarshalXLMeta.Parts) != len(fastjsonXLMeta.Parts) {
|
if len(unMarshalXLMeta.Parts) != len(jsoniterXLMeta.Parts) {
|
||||||
t.Errorf("Expected info of %d parts to be present, but got %d instead.", len(unMarshalXLMeta.Parts), len(fastjsonXLMeta.Parts))
|
t.Errorf("Expected info of %d parts to be present, but got %d instead.", len(unMarshalXLMeta.Parts), len(jsoniterXLMeta.Parts))
|
||||||
} else {
|
} else {
|
||||||
for i := 0; i < len(unMarshalXLMeta.Parts); i++ {
|
for i := 0; i < len(unMarshalXLMeta.Parts); i++ {
|
||||||
if unMarshalXLMeta.Parts[i].Name != fastjsonXLMeta.Parts[i].Name {
|
if unMarshalXLMeta.Parts[i].Name != jsoniterXLMeta.Parts[i].Name {
|
||||||
t.Errorf("Expected the name of part %d to be \"%s\", got \"%s\".", i+1, unMarshalXLMeta.Parts[i].Name, fastjsonXLMeta.Parts[i].Name)
|
t.Errorf("Expected the name of part %d to be \"%s\", got \"%s\".", i+1, unMarshalXLMeta.Parts[i].Name, jsoniterXLMeta.Parts[i].Name)
|
||||||
}
|
}
|
||||||
if unMarshalXLMeta.Parts[i].ETag != fastjsonXLMeta.Parts[i].ETag {
|
if unMarshalXLMeta.Parts[i].ETag != jsoniterXLMeta.Parts[i].ETag {
|
||||||
t.Errorf("Expected the ETag of part %d to be \"%s\", got \"%s\".", i+1, unMarshalXLMeta.Parts[i].ETag, fastjsonXLMeta.Parts[i].ETag)
|
t.Errorf("Expected the ETag of part %d to be \"%s\", got \"%s\".", i+1, unMarshalXLMeta.Parts[i].ETag, jsoniterXLMeta.Parts[i].ETag)
|
||||||
}
|
}
|
||||||
if unMarshalXLMeta.Parts[i].Number != fastjsonXLMeta.Parts[i].Number {
|
if unMarshalXLMeta.Parts[i].Number != jsoniterXLMeta.Parts[i].Number {
|
||||||
t.Errorf("Expected the number of part %d to be \"%d\", got \"%d\".", i+1, unMarshalXLMeta.Parts[i].Number, fastjsonXLMeta.Parts[i].Number)
|
t.Errorf("Expected the number of part %d to be \"%d\", got \"%d\".", i+1, unMarshalXLMeta.Parts[i].Number, jsoniterXLMeta.Parts[i].Number)
|
||||||
}
|
}
|
||||||
if unMarshalXLMeta.Parts[i].Size != fastjsonXLMeta.Parts[i].Size {
|
if unMarshalXLMeta.Parts[i].Size != jsoniterXLMeta.Parts[i].Size {
|
||||||
t.Errorf("Expected the size of part %d to be %v, got %v.", i+1, unMarshalXLMeta.Parts[i].Size, fastjsonXLMeta.Parts[i].Size)
|
t.Errorf("Expected the size of part %d to be %v, got %v.", i+1, unMarshalXLMeta.Parts[i].Size, jsoniterXLMeta.Parts[i].Size)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
for key, val := range unMarshalXLMeta.Meta {
|
for key, val := range unMarshalXLMeta.Meta {
|
||||||
fastjsonVal, exists := fastjsonXLMeta.Meta[key]
|
jsoniterVal, exists := jsoniterXLMeta.Meta[key]
|
||||||
if !exists {
|
if !exists {
|
||||||
t.Errorf("No meta data entry for Key \"%s\" exists.", key)
|
t.Errorf("No meta data entry for Key \"%s\" exists.", key)
|
||||||
}
|
}
|
||||||
if val != fastjsonVal {
|
if val != jsoniterVal {
|
||||||
t.Errorf("Expected the value for Meta data key \"%s\" to be \"%s\", but got \"%s\".", key, val, fastjsonVal)
|
t.Errorf("Expected the value for Meta data key \"%s\" to be \"%s\", but got \"%s\".", key, val, jsoniterVal)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Tests the correctness of constructing XLMetaV1 using fastjson lib.
|
// Tests the correctness of constructing XLMetaV1 using jsoniter lib.
|
||||||
// The result will be compared with the result obtained from json.unMarshal of the byte data.
|
// The result will be compared with the result obtained from json.unMarshal of the byte data.
|
||||||
func TestGetXLMetaV1Fastjson1(t *testing.T) {
|
func TestGetXLMetaV1Jsoniter1(t *testing.T) {
|
||||||
xlMetaJSON := getXLMetaBytes(1)
|
xlMetaJSON := getXLMetaBytes(1)
|
||||||
|
|
||||||
var unMarshalXLMeta xlMetaV1
|
var unMarshalXLMeta xlMetaV1
|
||||||
|
@ -312,16 +312,16 @@ func TestGetXLMetaV1Fastjson1(t *testing.T) {
|
||||||
t.Errorf("Unmarshalling failed: %v", err)
|
t.Errorf("Unmarshalling failed: %v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
fastjsonXLMeta, err := xlMetaV1UnmarshalJSON(context.Background(), xlMetaJSON)
|
jsoniterXLMeta, err := xlMetaV1UnmarshalJSON(context.Background(), xlMetaJSON)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Errorf("fastjson parsing of XLMeta failed: %v", err)
|
t.Errorf("jsoniter parsing of XLMeta failed: %v", err)
|
||||||
}
|
}
|
||||||
compareXLMetaV1(t, unMarshalXLMeta, fastjsonXLMeta)
|
compareXLMetaV1(t, unMarshalXLMeta, jsoniterXLMeta)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Tests the correctness of constructing XLMetaV1 using fastjson lib for XLMetaV1 of size 10 parts.
|
// Tests the correctness of constructing XLMetaV1 using jsoniter lib for XLMetaV1 of size 10 parts.
|
||||||
// The result will be compared with the result obtained from json.unMarshal of the byte data.
|
// The result will be compared with the result obtained from json.unMarshal of the byte data.
|
||||||
func TestGetXLMetaV1Fastjson10(t *testing.T) {
|
func TestGetXLMetaV1Jsoniter10(t *testing.T) {
|
||||||
|
|
||||||
xlMetaJSON := getXLMetaBytes(10)
|
xlMetaJSON := getXLMetaBytes(10)
|
||||||
|
|
||||||
|
@ -329,11 +329,11 @@ func TestGetXLMetaV1Fastjson10(t *testing.T) {
|
||||||
if err := json.Unmarshal(xlMetaJSON, &unMarshalXLMeta); err != nil {
|
if err := json.Unmarshal(xlMetaJSON, &unMarshalXLMeta); err != nil {
|
||||||
t.Errorf("Unmarshalling failed: %v", err)
|
t.Errorf("Unmarshalling failed: %v", err)
|
||||||
}
|
}
|
||||||
fastjsonXLMeta, err := xlMetaV1UnmarshalJSON(context.Background(), xlMetaJSON)
|
jsoniterXLMeta, err := xlMetaV1UnmarshalJSON(context.Background(), xlMetaJSON)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Errorf("fastjson parsing of XLMeta failed: %v", err)
|
t.Errorf("jsoniter parsing of XLMeta failed: %v", err)
|
||||||
}
|
}
|
||||||
compareXLMetaV1(t, unMarshalXLMeta, fastjsonXLMeta)
|
compareXLMetaV1(t, unMarshalXLMeta, jsoniterXLMeta)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Test the predicted part size from the part index
|
// Test the predicted part size from the part index
|
||||||
|
|
15
go.mod
15
go.mod
|
@ -43,6 +43,7 @@ require (
|
||||||
github.com/hashicorp/raft v1.1.0 // indirect
|
github.com/hashicorp/raft v1.1.0 // indirect
|
||||||
github.com/hashicorp/vault v1.1.0
|
github.com/hashicorp/vault v1.1.0
|
||||||
github.com/inconshreveable/go-update v0.0.0-20160112193335-8152e7eb6ccf
|
github.com/inconshreveable/go-update v0.0.0-20160112193335-8152e7eb6ccf
|
||||||
|
github.com/json-iterator/go v1.1.7
|
||||||
github.com/klauspost/compress v1.5.0 // indirect
|
github.com/klauspost/compress v1.5.0 // indirect
|
||||||
github.com/klauspost/cpuid v1.2.1 // indirect
|
github.com/klauspost/cpuid v1.2.1 // indirect
|
||||||
github.com/klauspost/pgzip v1.2.1
|
github.com/klauspost/pgzip v1.2.1
|
||||||
|
@ -66,6 +67,8 @@ require (
|
||||||
github.com/minio/sha256-simd v0.1.0
|
github.com/minio/sha256-simd v0.1.0
|
||||||
github.com/minio/sio v0.2.0
|
github.com/minio/sio v0.2.0
|
||||||
github.com/mitchellh/go-homedir v1.1.0
|
github.com/mitchellh/go-homedir v1.1.0
|
||||||
|
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect
|
||||||
|
github.com/modern-go/reflect2 v1.0.1 // indirect
|
||||||
github.com/nats-io/gnatsd v1.4.1 // indirect
|
github.com/nats-io/gnatsd v1.4.1 // indirect
|
||||||
github.com/nats-io/go-nats-streaming v0.4.4 // indirect
|
github.com/nats-io/go-nats-streaming v0.4.4 // indirect
|
||||||
github.com/nats-io/nats-server v1.4.1 // indirect
|
github.com/nats-io/nats-server v1.4.1 // indirect
|
||||||
|
@ -89,7 +92,6 @@ require (
|
||||||
github.com/tidwall/match v1.0.1 // indirect
|
github.com/tidwall/match v1.0.1 // indirect
|
||||||
github.com/tidwall/pretty v1.0.0 // indirect
|
github.com/tidwall/pretty v1.0.0 // indirect
|
||||||
github.com/tidwall/sjson v1.0.4
|
github.com/tidwall/sjson v1.0.4
|
||||||
github.com/valyala/fastjson v1.4.1
|
|
||||||
github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a
|
github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a
|
||||||
go.etcd.io/bbolt v1.3.3 // indirect
|
go.etcd.io/bbolt v1.3.3 // indirect
|
||||||
go.uber.org/atomic v1.3.2
|
go.uber.org/atomic v1.3.2
|
||||||
|
@ -104,3 +106,14 @@ require (
|
||||||
|
|
||||||
// Added for go1.13 migration https://github.com/golang/go/issues/32805
|
// Added for go1.13 migration https://github.com/golang/go/issues/32805
|
||||||
replace github.com/gorilla/rpc v1.2.0+incompatible => github.com/gorilla/rpc v1.2.0
|
replace github.com/gorilla/rpc v1.2.0+incompatible => github.com/gorilla/rpc v1.2.0
|
||||||
|
|
||||||
|
// Allow this for offline builds
|
||||||
|
replace github.com/eapache/go-xerial-snappy => github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21
|
||||||
|
|
||||||
|
replace github.com/eapache/queue => github.com/eapache/queue v1.1.0
|
||||||
|
|
||||||
|
replace github.com/mattn/go-runewidth => github.com/mattn/go-runewidth v0.0.4
|
||||||
|
|
||||||
|
replace github.com/mitchellh/mapstructure => github.com/mitchellh/mapstructure v1.1.2
|
||||||
|
|
||||||
|
replace git.apache.org/thrift.git => github.com/apache/thrift v0.12.0
|
||||||
|
|
8
go.sum
8
go.sum
|
@ -44,6 +44,7 @@ github.com/aliyun/aliyun-oss-go-sdk v0.0.0-20170925032315-6fe16293d6b7/go.mod h1
|
||||||
github.com/aliyun/aliyun-oss-go-sdk v0.0.0-20190307165228-86c17b95fcd5 h1:nWDRPCyCltiTsANwC/n3QZH7Vww33Npq9MKqlwRzI/c=
|
github.com/aliyun/aliyun-oss-go-sdk v0.0.0-20190307165228-86c17b95fcd5 h1:nWDRPCyCltiTsANwC/n3QZH7Vww33Npq9MKqlwRzI/c=
|
||||||
github.com/aliyun/aliyun-oss-go-sdk v0.0.0-20190307165228-86c17b95fcd5/go.mod h1:T/Aws4fEfogEE9v+HPhhw+CntffsBHJ8nXQCwKr0/g8=
|
github.com/aliyun/aliyun-oss-go-sdk v0.0.0-20190307165228-86c17b95fcd5/go.mod h1:T/Aws4fEfogEE9v+HPhhw+CntffsBHJ8nXQCwKr0/g8=
|
||||||
github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239/go.mod h1:2FmKhYUyUczH0OGQWaF5ceTx0UBShxjsH6f8oGKYe2c=
|
github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239/go.mod h1:2FmKhYUyUczH0OGQWaF5ceTx0UBShxjsH6f8oGKYe2c=
|
||||||
|
github.com/apache/thrift v0.12.0 h1:pODnxUFNcjP9UTLZGTdeh+j16A8lJbRvD3rOtrk/7bs=
|
||||||
github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ=
|
github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ=
|
||||||
github.com/araddon/gou v0.0.0-20190110011759-c797efecbb61/go.mod h1:ikc1XA58M+Rx7SEbf0bLJCfBkwayZ8T5jBo5FXK8Uz8=
|
github.com/araddon/gou v0.0.0-20190110011759-c797efecbb61/go.mod h1:ikc1XA58M+Rx7SEbf0bLJCfBkwayZ8T5jBo5FXK8Uz8=
|
||||||
github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o=
|
github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o=
|
||||||
|
@ -198,6 +199,7 @@ github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5a
|
||||||
github.com/google/go-github v17.0.0+incompatible/go.mod h1:zLgOLi98H3fifZn+44m+umXrS52loVEgC2AApnigrVQ=
|
github.com/google/go-github v17.0.0+incompatible/go.mod h1:zLgOLi98H3fifZn+44m+umXrS52loVEgC2AApnigrVQ=
|
||||||
github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck=
|
github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck=
|
||||||
github.com/google/gofuzz v0.0.0-20170612174753-24818f796faf/go.mod h1:HP5RmnzzSNb993RKQDq4+1A4ia9nllfqcQFTQJedwGI=
|
github.com/google/gofuzz v0.0.0-20170612174753-24818f796faf/go.mod h1:HP5RmnzzSNb993RKQDq4+1A4ia9nllfqcQFTQJedwGI=
|
||||||
|
github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
|
||||||
github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no=
|
github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no=
|
||||||
github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs=
|
github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs=
|
||||||
github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
|
github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
|
||||||
|
@ -328,6 +330,8 @@ github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0
|
||||||
github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo=
|
github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo=
|
||||||
github.com/json-iterator/go v1.1.6 h1:MrUvLMLTMxbqFJ9kzlvat/rYZqZnW3u4wkLzWTaFwKs=
|
github.com/json-iterator/go v1.1.6 h1:MrUvLMLTMxbqFJ9kzlvat/rYZqZnW3u4wkLzWTaFwKs=
|
||||||
github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
|
github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
|
||||||
|
github.com/json-iterator/go v1.1.7 h1:KfgG9LzI+pYjr4xvmz/5H4FXjokeP+rlHLhv3iH62Fo=
|
||||||
|
github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
|
||||||
github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU=
|
github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU=
|
||||||
github.com/jtolds/gls v4.2.1+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU=
|
github.com/jtolds/gls v4.2.1+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU=
|
||||||
github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo=
|
github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo=
|
||||||
|
@ -450,8 +454,10 @@ github.com/mitchellh/mapstructure v1.1.2 h1:fmNYVwqnSfB9mZU6OS2O6GsXM+wcskZDuKQz
|
||||||
github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y=
|
github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y=
|
||||||
github.com/mitchellh/pointerstructure v0.0.0-20170205204203-f2329fcfa9e2/go.mod h1:KMNPMpc0BU/kZEgyDhBplsDn/mjnJMhyMjq4MWboN20=
|
github.com/mitchellh/pointerstructure v0.0.0-20170205204203-f2329fcfa9e2/go.mod h1:KMNPMpc0BU/kZEgyDhBplsDn/mjnJMhyMjq4MWboN20=
|
||||||
github.com/mitchellh/reflectwalk v1.0.0/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw=
|
github.com/mitchellh/reflectwalk v1.0.0/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw=
|
||||||
|
github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
|
||||||
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg=
|
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg=
|
||||||
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
|
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
|
||||||
|
github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
|
||||||
github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI=
|
github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI=
|
||||||
github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
|
github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
|
||||||
github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
|
github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
|
||||||
|
@ -612,8 +618,6 @@ github.com/ugorji/go v1.1.2/go.mod h1:hnLbHMwcvSihnDhEfx2/BzKp2xb0Y+ErdfYcrs9tkJ
|
||||||
github.com/ugorji/go v1.1.4 h1:j4s+tAvLfL3bZyefP2SEWmhBzmuIlH/eqNuPdFPgngw=
|
github.com/ugorji/go v1.1.4 h1:j4s+tAvLfL3bZyefP2SEWmhBzmuIlH/eqNuPdFPgngw=
|
||||||
github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc=
|
github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc=
|
||||||
github.com/ugorji/go/codec v0.0.0-20190320090025-2dc34c0b8780/go.mod h1:iT03XoTwV7xq/+UGwKO3UbC1nNNlopQiY61beSdrtOA=
|
github.com/ugorji/go/codec v0.0.0-20190320090025-2dc34c0b8780/go.mod h1:iT03XoTwV7xq/+UGwKO3UbC1nNNlopQiY61beSdrtOA=
|
||||||
github.com/valyala/fastjson v1.4.1 h1:hrltpHpIpkaxll8QltMU8c3QZ5+qIiCL8yKqPFJI/yE=
|
|
||||||
github.com/valyala/fastjson v1.4.1/go.mod h1:nV6MsjxL2IMJQUoHDIrjEI7oLyeqK6aBD7EFWPsvP8o=
|
|
||||||
github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a h1:0R4NLDRDZX6JcmhJgXi5E4b8Wg84ihbmUKp/GvSPEzc=
|
github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a h1:0R4NLDRDZX6JcmhJgXi5E4b8Wg84ihbmUKp/GvSPEzc=
|
||||||
github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio=
|
github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio=
|
||||||
github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8=
|
github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8=
|
||||||
|
|
|
@ -35,7 +35,7 @@ type LogInfo struct {
|
||||||
|
|
||||||
// SendLog returns true if log pertains to node specified in args.
|
// SendLog returns true if log pertains to node specified in args.
|
||||||
func (l LogInfo) SendLog(node string) bool {
|
func (l LogInfo) SendLog(node string) bool {
|
||||||
return node == "" || strings.ToLower(node) == strings.ToLower(l.NodeName)
|
return node == "" || strings.EqualFold(node, l.NodeName)
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetLogs - listen on console log messages.
|
// GetLogs - listen on console log messages.
|
||||||
|
|
Loading…
Reference in New Issue