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:
Harshavardhana 2019-09-05 15:51:27 -07:00 committed by kannappanr
parent 428836d4e1
commit b52a3e523c
15 changed files with 169 additions and 430 deletions

View file

@ -1854,7 +1854,7 @@ func (a adminAPIHandlers) ConsoleLogHandler(w http.ResponseWriter, r *http.Reque
globalConsoleSys.Subscribe(logCh, doneCh, node, limitLines, nil)
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)
}
}

View file

@ -20,6 +20,7 @@ import (
"bytes"
"context"
"encoding/json"
"fmt"
"path"
"runtime"
"strings"
@ -136,18 +137,23 @@ func (sys *ConfigSys) Init(objAPI ObjectLayer) error {
// of the object layer.
// - Write quorum not met when upgrading configuration
// version is needed.
for range newRetryTimerSimple(doneCh) {
if err := initConfig(objAPI); err != nil {
if strings.Contains(err.Error(), InsufficientReadQuorum{}.Error()) ||
strings.Contains(err.Error(), InsufficientWriteQuorum{}.Error()) {
logger.Info("Waiting for configuration to be initialized..")
continue
retryTimerCh := newRetryTimerSimple(doneCh)
for {
select {
case <-retryTimerCh:
if err := initConfig(objAPI); err != nil {
if strings.Contains(err.Error(), InsufficientReadQuorum{}.Error()) ||
strings.Contains(err.Error(), InsufficientWriteQuorum{}.Error()) {
logger.Info("Waiting for configuration to be initialized..")
continue
}
return err
}
return err
return nil
case <-globalOSSignalCh:
return fmt.Errorf("Initializing config sub-system gracefully stopped")
}
break
}
return nil
}
// NewConfigSys - creates new config system object.

View file

@ -27,10 +27,10 @@ import (
pathutil "path"
"time"
jsoniter "github.com/json-iterator/go"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/lock"
"github.com/minio/minio/pkg/mimedb"
"github.com/valyala/fastjson"
)
// FS format, and object metadata.
@ -203,37 +203,6 @@ func (m *fsMetaV1) WriteTo(lk *lock.LockedFile) (n int64, err error) {
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) {
var fsMetaBuf []byte
fi, err := lk.Stat()
@ -253,18 +222,11 @@ func (m *fsMetaV1) ReadFrom(ctx context.Context, lk *lock.LockedFile) (n int64,
return 0, io.EOF
}
parser := fsParserPool.Get()
defer fsParserPool.Put(parser)
var v *fastjson.Value
v, err = parser.ParseBytes(fsMetaBuf)
if err != nil {
var json = jsoniter.ConfigCompatibleWithStandardLibrary
if err = json.Unmarshal(fsMetaBuf, m); err != nil {
return 0, err
}
// obtain version.
m.Version = parseFSVersion(v)
// Verify if the format is valid, return corrupted format
// for unrecognized formats.
if !isFSMetaValid(m.Version) {
@ -273,12 +235,6 @@ func (m *fsMetaV1) ReadFrom(ctx context.Context, lk *lock.LockedFile) (n int64,
return 0, errCorruptedFormat
}
// obtain parts information
m.Parts = parseFSPartsArray(v)
// obtain metadata.
m.Meta = parseFSMetaMap(v)
// Success.
return int64(len(fsMetaBuf)), nil
}

View file

@ -28,9 +28,9 @@ import (
"strings"
"time"
jsoniter "github.com/json-iterator/go"
"github.com/minio/minio/cmd/logger"
mioutil "github.com/minio/minio/pkg/ioutil"
"github.com/valyala/fastjson"
)
// Returns EXPORT/.minio.sys/multipart/SHA256/UPLOADID
@ -472,16 +472,13 @@ func (fs *FSObjects) ListObjectParts(ctx context.Context, bucket, object, upload
return result, err
}
parser := fsParserPool.Get()
defer fsParserPool.Put(parser)
var v *fastjson.Value
v, err = parser.ParseBytes(fsMetaBytes)
if err != nil {
var fsMeta fsMetaV1
var json = jsoniter.ConfigCompatibleWithStandardLibrary
if err = json.Unmarshal(fsMetaBytes, &fsMeta); err != nil {
return result, err
}
result.UserDefined = parseFSMetaMap(v)
result.UserDefined = fsMeta.Meta
return result, nil
}

View file

@ -30,6 +30,7 @@ import (
"sync/atomic"
"time"
jsoniter "github.com/json-iterator/go"
"github.com/minio/minio-go/v6/pkg/s3utils"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/lifecycle"
@ -38,7 +39,6 @@ import (
"github.com/minio/minio/pkg/mimedb"
"github.com/minio/minio/pkg/mountinfo"
"github.com/minio/minio/pkg/policy"
"github.com/valyala/fastjson"
)
// 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)
}
parser := fsParserPool.Get()
defer fsParserPool.Put(parser)
var v *fastjson.Value
v, err = parser.ParseBytes(fsMetaBuf)
if err != nil {
return "", toObjectErr(err, bucket, entry)
var fsMeta fsMetaV1
var json = jsoniter.ConfigCompatibleWithStandardLibrary
if err = json.Unmarshal(fsMetaBuf, &fsMeta); err != nil {
return "", err
}
// Check if FS metadata is valid, if not return error.
if !isFSMetaValid(parseFSVersion(v)) {
if !isFSMetaValid(fsMeta.Version) {
logger.LogIf(ctx, errCorruptedFormat)
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

View file

@ -24,10 +24,10 @@ import (
"net/http"
"time"
jsoniter "github.com/json-iterator/go"
minio "github.com/minio/minio/cmd"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/hash"
"github.com/valyala/fastjson"
)
var (
@ -137,101 +137,11 @@ func (m gwMetaV1) ObjectToPartOffset(ctx context.Context, offset int64) (partInd
return 0, 0, minio.InvalidRange{}
}
// parses gateway metadata stat info from metadata json
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.
// Constructs GWMetaV1 using `jsoniter` lib to retrieve each field.
func gwMetaUnmarshalJSON(ctx context.Context, gwMetaBuf []byte) (gwMeta gwMetaV1, err error) {
parser := gwParserPool.Get()
defer gwParserPool.Put(parser)
var v *fastjson.Value
v, err = parser.ParseBytes(gwMetaBuf)
if err != nil {
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
var json = jsoniter.ConfigCompatibleWithStandardLibrary
err = json.Unmarshal(gwMetaBuf, &gwMeta)
return gwMeta, err
}
// readGWMeta reads `dare.meta` and returns back GW metadata structure.

View file

@ -20,6 +20,7 @@ import (
"bytes"
"context"
"encoding/xml"
"fmt"
"path"
"strings"
"sync"
@ -134,20 +135,25 @@ func (sys *LifecycleSys) Init(objAPI ObjectLayer) error {
// the following reasons:
// - Read quorum is lost just after the initialization
// of the object layer.
for range newRetryTimerSimple(doneCh) {
// Load LifecycleSys once during boot.
if err := sys.refresh(objAPI); err != nil {
if err == errDiskNotFound ||
strings.Contains(err.Error(), InsufficientReadQuorum{}.Error()) ||
strings.Contains(err.Error(), InsufficientWriteQuorum{}.Error()) {
logger.Info("Waiting for lifecycle subsystem to be initialized..")
continue
retryTimerCh := newRetryTimerSimple(doneCh)
for {
select {
case <-retryTimerCh:
// Load LifecycleSys once during boot.
if err := sys.refresh(objAPI); err != nil {
if err == errDiskNotFound ||
strings.Contains(err.Error(), InsufficientReadQuorum{}.Error()) ||
strings.Contains(err.Error(), InsufficientWriteQuorum{}.Error()) {
logger.Info("Waiting for lifecycle subsystem to be initialized..")
continue
}
return err
}
return err
return nil
case <-globalOSSignalCh:
return fmt.Errorf("Initializing Lifecycle sub-system gracefully stopped")
}
break
}
return nil
}
// Refresh LifecycleSys.

View file

@ -808,19 +808,24 @@ func (sys *NotificationSys) Init(objAPI ObjectLayer) error {
// the following reasons:
// - Read quorum is lost just after the initialization
// of the object layer.
for range newRetryTimerSimple(doneCh) {
if err := sys.refresh(objAPI); err != nil {
if err == errDiskNotFound ||
strings.Contains(err.Error(), InsufficientReadQuorum{}.Error()) ||
strings.Contains(err.Error(), InsufficientWriteQuorum{}.Error()) {
logger.Info("Waiting for notification subsystem to be initialized..")
continue
retryTimerCh := newRetryTimerSimple(doneCh)
for {
select {
case <-retryTimerCh:
if err := sys.refresh(objAPI); err != nil {
if err == errDiskNotFound ||
strings.Contains(err.Error(), InsufficientReadQuorum{}.Error()) ||
strings.Contains(err.Error(), InsufficientWriteQuorum{}.Error()) {
logger.Info("Waiting for notification subsystem to be initialized..")
continue
}
return err
}
return err
return nil
case <-globalOSSignalCh:
return fmt.Errorf("Initializing Notification sub-system gracefully stopped")
}
break
}
return nil
}
// AddRulesMap - adds rules map for bucket name.

View file

@ -163,20 +163,25 @@ func (sys *PolicySys) Init(objAPI ObjectLayer) error {
// the following reasons:
// - Read quorum is lost just after the initialization
// of the object layer.
for range newRetryTimerSimple(doneCh) {
// Load PolicySys once during boot.
if err := sys.refresh(objAPI); err != nil {
if err == errDiskNotFound ||
strings.Contains(err.Error(), InsufficientReadQuorum{}.Error()) ||
strings.Contains(err.Error(), InsufficientWriteQuorum{}.Error()) {
logger.Info("Waiting for policy subsystem to be initialized..")
continue
retryTimerCh := newRetryTimerSimple(doneCh)
for {
select {
case <-retryTimerCh:
// Load PolicySys once during boot.
if err := sys.refresh(objAPI); err != nil {
if err == errDiskNotFound ||
strings.Contains(err.Error(), InsufficientReadQuorum{}.Error()) ||
strings.Contains(err.Error(), InsufficientWriteQuorum{}.Error()) {
logger.Info("Waiting for policy subsystem to be initialized..")
continue
}
return err
}
return err
return nil
case <-globalOSSignalCh:
return fmt.Errorf("Initializing Policy sub-system gracefully stopped")
}
break
}
return nil
}
// NewPolicySys - creates new policy system.

View file

@ -75,8 +75,6 @@ func (c ChecksumInfo) MarshalJSON() ([]byte, error) {
// UnmarshalJSON - should never be called, instead xlMetaV1UnmarshalJSON() should be used.
func (c *ChecksumInfo) UnmarshalJSON(data []byte) error {
logger.LogIf(context.Background(), errUnexpected)
var info checksumInfoJSON
if err := json.Unmarshal(data, &info); err != nil {
return err

View file

@ -18,15 +18,13 @@ package cmd
import (
"context"
"encoding/hex"
"errors"
"hash/crc32"
"path"
"sync"
"time"
jsoniter "github.com/json-iterator/go"
"github.com/minio/minio/cmd/logger"
"github.com/valyala/fastjson"
)
// 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
}
func parseXLStat(v *fastjson.Value) (si 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
}
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.
// Constructs xlMetaV1 using `jsoniter` lib.
func xlMetaV1UnmarshalJSON(ctx context.Context, xlMetaBuf []byte) (xlMeta xlMetaV1, err error) {
parser := xlParserPool.Get()
defer xlParserPool.Put(parser)
var v *fastjson.Value
v, err = parser.ParseBytes(xlMetaBuf)
if err != nil {
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
var json = jsoniter.ConfigCompatibleWithStandardLibrary
err = json.Unmarshal(xlMetaBuf, &xlMeta)
return xlMeta, err
}
// 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
}
// 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,
mp map[string]string, e error) {
// Reads entire `xl.json`.

View file

@ -212,99 +212,99 @@ func getSampleXLMeta(totalParts int) xlMetaV1 {
return xlMeta
}
// Compare the unmarshaled XLMetaV1 with the one obtained from fastjson parsing.
func compareXLMetaV1(t *testing.T, unMarshalXLMeta, fastjsonXLMeta xlMetaV1) {
// Start comparing the fields of xlMetaV1 obtained from fastjson parsing with one parsed using json unmarshaling.
if unMarshalXLMeta.Version != fastjsonXLMeta.Version {
t.Errorf("Expected the Version to be \"%s\", but got \"%s\".", unMarshalXLMeta.Version, fastjsonXLMeta.Version)
// Compare the unmarshaled XLMetaV1 with the one obtained from jsoniter parsing.
func compareXLMetaV1(t *testing.T, unMarshalXLMeta, jsoniterXLMeta xlMetaV1) {
// Start comparing the fields of xlMetaV1 obtained from jsoniter parsing with one parsed using json unmarshaling.
if unMarshalXLMeta.Version != jsoniterXLMeta.Version {
t.Errorf("Expected the Version to be \"%s\", but got \"%s\".", unMarshalXLMeta.Version, jsoniterXLMeta.Version)
}
if unMarshalXLMeta.Format != fastjsonXLMeta.Format {
t.Errorf("Expected the format to be \"%s\", but got \"%s\".", unMarshalXLMeta.Format, fastjsonXLMeta.Format)
if unMarshalXLMeta.Format != jsoniterXLMeta.Format {
t.Errorf("Expected the format to be \"%s\", but got \"%s\".", unMarshalXLMeta.Format, jsoniterXLMeta.Format)
}
if unMarshalXLMeta.Stat.Size != fastjsonXLMeta.Stat.Size {
t.Errorf("Expected the stat size to be %v, but got %v.", 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, jsoniterXLMeta.Stat.Size)
}
if !unMarshalXLMeta.Stat.ModTime.Equal(fastjsonXLMeta.Stat.ModTime) {
t.Errorf("Expected the modTime to be \"%v\", but got \"%v\".", unMarshalXLMeta.Stat.ModTime, 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, jsoniterXLMeta.Stat.ModTime)
}
if unMarshalXLMeta.Erasure.Algorithm != fastjsonXLMeta.Erasure.Algorithm {
t.Errorf("Expected the erasure algorithm to be \"%v\", but got \"%v\".", 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, jsoniterXLMeta.Erasure.Algorithm)
}
if unMarshalXLMeta.Erasure.DataBlocks != fastjsonXLMeta.Erasure.DataBlocks {
t.Errorf("Expected the erasure data blocks to be %v, but got %v.", 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, jsoniterXLMeta.Erasure.DataBlocks)
}
if unMarshalXLMeta.Erasure.ParityBlocks != fastjsonXLMeta.Erasure.ParityBlocks {
t.Errorf("Expected the erasure parity blocks to be %v, but got %v.", 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, jsoniterXLMeta.Erasure.ParityBlocks)
}
if unMarshalXLMeta.Erasure.BlockSize != fastjsonXLMeta.Erasure.BlockSize {
t.Errorf("Expected the erasure block size to be %v, but got %v.", 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, jsoniterXLMeta.Erasure.BlockSize)
}
if unMarshalXLMeta.Erasure.Index != fastjsonXLMeta.Erasure.Index {
t.Errorf("Expected the erasure index to be %v, but got %v.", 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, jsoniterXLMeta.Erasure.Index)
}
if 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(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(jsoniterXLMeta.Erasure.Distribution))
} else {
for i := 0; i < len(unMarshalXLMeta.Erasure.Distribution); i++ {
if unMarshalXLMeta.Erasure.Distribution[i] != fastjsonXLMeta.Erasure.Distribution[i] {
t.Errorf("Expected the Erasure Distribution to be %d, got %d.", 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], jsoniterXLMeta.Erasure.Distribution[i])
}
}
}
if 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(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(jsoniterXLMeta.Erasure.Checksums))
} else {
for i := 0; i < len(unMarshalXLMeta.Erasure.Checksums); i++ {
if 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, 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, jsoniterXLMeta.Erasure.Checksums[i].Name)
}
if 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, 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, jsoniterXLMeta.Erasure.Checksums[i].Algorithm)
}
if !bytes.Equal(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, 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, jsoniterXLMeta.Erasure.Checksums[i].Hash)
}
}
}
if unMarshalXLMeta.Minio.Release != fastjsonXLMeta.Minio.Release {
t.Errorf("Expected the Release string to be \"%s\", but got \"%s\".", 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, jsoniterXLMeta.Minio.Release)
}
if len(unMarshalXLMeta.Parts) != len(fastjsonXLMeta.Parts) {
t.Errorf("Expected info of %d parts to be present, but got %d instead.", 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(jsoniterXLMeta.Parts))
} else {
for i := 0; i < len(unMarshalXLMeta.Parts); i++ {
if 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, 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, jsoniterXLMeta.Parts[i].Name)
}
if 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, 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, jsoniterXLMeta.Parts[i].ETag)
}
if 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, 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, jsoniterXLMeta.Parts[i].Number)
}
if 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, 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, jsoniterXLMeta.Parts[i].Size)
}
}
}
for key, val := range unMarshalXLMeta.Meta {
fastjsonVal, exists := fastjsonXLMeta.Meta[key]
jsoniterVal, exists := jsoniterXLMeta.Meta[key]
if !exists {
t.Errorf("No meta data entry for Key \"%s\" exists.", key)
}
if val != fastjsonVal {
t.Errorf("Expected the value for Meta data key \"%s\" to be \"%s\", but got \"%s\".", key, val, fastjsonVal)
if val != jsoniterVal {
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.
func TestGetXLMetaV1Fastjson1(t *testing.T) {
func TestGetXLMetaV1Jsoniter1(t *testing.T) {
xlMetaJSON := getXLMetaBytes(1)
var unMarshalXLMeta xlMetaV1
@ -312,16 +312,16 @@ func TestGetXLMetaV1Fastjson1(t *testing.T) {
t.Errorf("Unmarshalling failed: %v", err)
}
fastjsonXLMeta, err := xlMetaV1UnmarshalJSON(context.Background(), xlMetaJSON)
jsoniterXLMeta, err := xlMetaV1UnmarshalJSON(context.Background(), xlMetaJSON)
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.
func TestGetXLMetaV1Fastjson10(t *testing.T) {
func TestGetXLMetaV1Jsoniter10(t *testing.T) {
xlMetaJSON := getXLMetaBytes(10)
@ -329,11 +329,11 @@ func TestGetXLMetaV1Fastjson10(t *testing.T) {
if err := json.Unmarshal(xlMetaJSON, &unMarshalXLMeta); err != nil {
t.Errorf("Unmarshalling failed: %v", err)
}
fastjsonXLMeta, err := xlMetaV1UnmarshalJSON(context.Background(), xlMetaJSON)
jsoniterXLMeta, err := xlMetaV1UnmarshalJSON(context.Background(), xlMetaJSON)
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

15
go.mod
View file

@ -43,6 +43,7 @@ require (
github.com/hashicorp/raft v1.1.0 // indirect
github.com/hashicorp/vault v1.1.0
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/cpuid v1.2.1 // indirect
github.com/klauspost/pgzip v1.2.1
@ -66,6 +67,8 @@ require (
github.com/minio/sha256-simd v0.1.0
github.com/minio/sio v0.2.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/go-nats-streaming v0.4.4 // 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/pretty v1.0.0 // indirect
github.com/tidwall/sjson v1.0.4
github.com/valyala/fastjson v1.4.1
github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a
go.etcd.io/bbolt v1.3.3 // indirect
go.uber.org/atomic v1.3.2
@ -104,3 +106,14 @@ require (
// 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
// 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
View file

@ -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/go.mod h1:T/Aws4fEfogEE9v+HPhhw+CntffsBHJ8nXQCwKr0/g8=
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/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=
@ -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-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 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/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs=
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/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.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/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=
@ -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/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/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/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/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
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/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc=
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/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio=
github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8=

View file

@ -35,7 +35,7 @@ type LogInfo struct {
// SendLog returns true if log pertains to node specified in args.
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.