Merge branch 'master' into mq

This commit is contained in:
chrislu
2024-05-20 11:04:53 -07:00
54 changed files with 2301 additions and 1656 deletions

View File

@@ -32,12 +32,15 @@ var cmdFix = &Command{
var (
fixVolumeCollection = cmdFix.Flag.String("collection", "", "an optional volume collection name, if specified only it will be processed")
fixVolumeId = cmdFix.Flag.Int64("volumeId", 0, "an optional volume id, if not 0 (default) only it will be processed")
fixIncludeDeleted = cmdFix.Flag.Bool("includeDeleted", true, "include deleted entries in the index file")
fixIgnoreError = cmdFix.Flag.Bool("ignoreError", false, "an optional, if true will be processed despite errors")
)
type VolumeFileScanner4Fix struct {
version needle.Version
nm *needle_map.MemDb
version needle.Version
nm *needle_map.MemDb
nmDeleted *needle_map.MemDb
includeDeleted bool
}
func (scanner *VolumeFileScanner4Fix) VisitSuperBlock(superBlock super_block.SuperBlock) error {
@@ -50,13 +53,20 @@ func (scanner *VolumeFileScanner4Fix) ReadNeedleBody() bool {
}
func (scanner *VolumeFileScanner4Fix) VisitNeedle(n *needle.Needle, offset int64, needleHeader, needleBody []byte) error {
glog.V(2).Infof("key %d offset %d size %d disk_size %d compressed %v", n.Id, offset, n.Size, n.DiskSize(scanner.version), n.IsCompressed())
glog.V(2).Infof("key %v offset %d size %d disk_size %d compressed %v", n.Id, offset, n.Size, n.DiskSize(scanner.version), n.IsCompressed())
if n.Size.IsValid() {
pe := scanner.nm.Set(n.Id, types.ToOffset(offset), n.Size)
glog.V(2).Infof("saved %d with error %v", n.Size, pe)
if pe := scanner.nm.Set(n.Id, types.ToOffset(offset), n.Size); pe != nil {
return fmt.Errorf("saved %d with error %v", n.Size, pe)
}
} else {
glog.V(2).Infof("skipping deleted file ...")
return scanner.nm.Delete(n.Id)
if scanner.includeDeleted {
if pe := scanner.nmDeleted.Set(n.Id, types.ToOffset(offset), types.TombstoneFileSize); pe != nil {
return fmt.Errorf("saved deleted %d with error %v", n.Size, pe)
}
} else {
glog.V(2).Infof("skipping deleted file ...")
return scanner.nm.Delete(n.Id)
}
}
return nil
}
@@ -109,21 +119,45 @@ func runFix(cmd *Command, args []string) bool {
if *fixVolumeId != 0 && *fixVolumeId != volumeId {
continue
}
doFixOneVolume(basePath, baseFileName, collection, volumeId)
doFixOneVolume(basePath, baseFileName, collection, volumeId, *fixIncludeDeleted)
}
}
return true
}
func doFixOneVolume(basepath string, baseFileName string, collection string, volumeId int64) {
func SaveToIdx(scaner *VolumeFileScanner4Fix, idxName string) (ret error) {
idxFile, err := os.OpenFile(idxName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644)
if err != nil {
return
}
defer func() {
idxFile.Close()
}()
return scaner.nm.AscendingVisit(func(value needle_map.NeedleValue) error {
_, err := idxFile.Write(value.ToBytes())
if scaner.includeDeleted && err == nil {
if deleted, ok := scaner.nmDeleted.Get(value.Key); ok {
_, err = idxFile.Write(deleted.ToBytes())
}
}
return err
})
}
func doFixOneVolume(basepath string, baseFileName string, collection string, volumeId int64, fixIncludeDeleted bool) {
indexFileName := path.Join(basepath, baseFileName+".idx")
nm := needle_map.NewMemDb()
nmDeleted := needle_map.NewMemDb()
defer nm.Close()
defer nmDeleted.Close()
vid := needle.VolumeId(volumeId)
scanner := &VolumeFileScanner4Fix{
nm: nm,
nm: nm,
nmDeleted: nmDeleted,
includeDeleted: fixIncludeDeleted,
}
if err := storage.ScanVolumeFile(basepath, collection, vid, storage.NeedleMapInMemory, scanner); err != nil {
@@ -135,12 +169,12 @@ func doFixOneVolume(basepath string, baseFileName string, collection string, vol
}
}
if err := nm.SaveToIdx(indexFileName); err != nil {
os.Remove(indexFileName)
if err := SaveToIdx(scanner, indexFileName); err != nil {
err := fmt.Errorf("save to .idx File: %v", err)
if *fixIgnoreError {
glog.Error(err)
} else {
os.Remove(indexFileName)
glog.Fatal(err)
}
}

View File

@@ -280,6 +280,13 @@ tls_client_key_file=""
[mongodb]
enabled = false
uri = "mongodb://localhost:27017"
username = ""
password = ""
ssl = false
ssl_ca_file = ""
ssl_cert_file = ""
ssl_key_file = "
insecure_skip_verify = false
option_pool_size = 0
database = "seaweedfs"

View File

@@ -2,7 +2,12 @@ package mongodb
import (
"context"
"crypto/tls"
"crypto/x509"
"fmt"
"os"
"time"
"github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
@@ -10,7 +15,6 @@ import (
"go.mongodb.org/mongo-driver/bson"
"go.mongodb.org/mongo-driver/mongo"
"go.mongodb.org/mongo-driver/mongo/options"
"time"
)
func init() {
@@ -37,17 +41,44 @@ func (store *MongodbStore) Initialize(configuration util.Configuration, prefix s
store.database = configuration.GetString(prefix + "database")
store.collectionName = "filemeta"
poolSize := configuration.GetInt(prefix + "option_pool_size")
return store.connection(configuration.GetString(prefix+"uri"), uint64(poolSize))
uri := configuration.GetString(prefix + "uri")
ssl := configuration.GetBool(prefix + "ssl")
sslCAFile := configuration.GetString(prefix + "ssl_ca_file")
sslCertFile := configuration.GetString(prefix + "ssl_cert_file")
sslKeyFile := configuration.GetString(prefix + "ssl_key_file")
username := configuration.GetString(prefix + "username")
password := configuration.GetString(prefix + "password")
insecure_skip_verify := configuration.GetBool(prefix + "insecure_skip_verify")
return store.connection(uri, uint64(poolSize), ssl, sslCAFile, sslCertFile, sslKeyFile, username, password, insecure_skip_verify)
}
func (store *MongodbStore) connection(uri string, poolSize uint64) (err error) {
ctx, _ := context.WithTimeout(context.Background(), 10*time.Second)
func (store *MongodbStore) connection(uri string, poolSize uint64, ssl bool, sslCAFile, sslCertFile, sslKeyFile string, username, password string, insecure bool) (err error) {
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
defer cancel()
opts := options.Client().ApplyURI(uri)
if poolSize > 0 {
opts.SetMaxPoolSize(poolSize)
}
if ssl {
tlsConfig, err := configureTLS(sslCAFile, sslCertFile, sslKeyFile, insecure)
if err != nil {
return err
}
opts.SetTLSConfig(tlsConfig)
}
if username != "" && password != "" {
creds := options.Credential{
Username: username,
Password: password,
}
opts.SetAuth(creds)
}
client, err := mongo.Connect(ctx, opts)
if err != nil {
return err
@@ -55,10 +86,36 @@ func (store *MongodbStore) connection(uri string, poolSize uint64) (err error) {
c := client.Database(store.database).Collection(store.collectionName)
err = store.indexUnique(c)
store.connect = client
return err
}
func configureTLS(caFile, certFile, keyFile string, insecure bool) (*tls.Config, error) {
cert, err := tls.LoadX509KeyPair(certFile, keyFile)
if err != nil {
return nil, fmt.Errorf("could not load client key pair: %s", err)
}
caCert, err := os.ReadFile(caFile)
if err != nil {
return nil, fmt.Errorf("could not read CA certificate: %s", err)
}
caCertPool := x509.NewCertPool()
if !caCertPool.AppendCertsFromPEM(caCert) {
return nil, fmt.Errorf("failed to append CA certificate")
}
tlsConfig := &tls.Config{
Certificates: []tls.Certificate{cert},
RootCAs: caCertPool,
InsecureSkipVerify: insecure,
}
return tlsConfig, nil
}
func (store *MongodbStore) createIndex(c *mongo.Collection, index mongo.IndexModel, opts *options.CreateIndexesOptions) error {
_, err := c.Indexes().CreateOne(context.Background(), index, opts)
return err
@@ -93,13 +150,10 @@ func (store *MongodbStore) RollbackTransaction(ctx context.Context) error {
}
func (store *MongodbStore) InsertEntry(ctx context.Context, entry *filer.Entry) (err error) {
return store.UpdateEntry(ctx, entry)
}
func (store *MongodbStore) UpdateEntry(ctx context.Context, entry *filer.Entry) (err error) {
dir, name := entry.FullPath.DirAndName()
meta, err := entry.EncodeAttributesAndChunks()
if err != nil {
@@ -126,7 +180,6 @@ func (store *MongodbStore) UpdateEntry(ctx context.Context, entry *filer.Entry)
}
func (store *MongodbStore) FindEntry(ctx context.Context, fullpath util.FullPath) (entry *filer.Entry, err error) {
dir, name := fullpath.DirAndName()
var data Model
@@ -154,7 +207,6 @@ func (store *MongodbStore) FindEntry(ctx context.Context, fullpath util.FullPath
}
func (store *MongodbStore) DeleteEntry(ctx context.Context, fullpath util.FullPath) error {
dir, name := fullpath.DirAndName()
where := bson.M{"directory": dir, "name": name}
@@ -167,7 +219,6 @@ func (store *MongodbStore) DeleteEntry(ctx context.Context, fullpath util.FullPa
}
func (store *MongodbStore) DeleteFolderChildren(ctx context.Context, fullpath util.FullPath) error {
where := bson.M{"directory": fullpath}
_, err := store.connect.Database(store.database).Collection(store.collectionName).DeleteMany(ctx, where)
if err != nil {
@@ -182,7 +233,6 @@ func (store *MongodbStore) ListDirectoryPrefixedEntries(ctx context.Context, dir
}
func (store *MongodbStore) ListDirectoryEntries(ctx context.Context, dirPath util.FullPath, startFileName string, includeStartFile bool, limit int64, eachEntryFunc filer.ListEachEntryFunc) (lastFileName string, err error) {
var where = bson.M{"directory": string(dirPath), "name": bson.M{"$gt": startFileName}}
if includeStartFile {
where["name"] = bson.M{

View File

@@ -33,6 +33,7 @@ const (
StatementActionReadAcp = "GetBucketAcl"
StatementActionList = "List*"
StatementActionTagging = "Tagging*"
StatementActionDelete = "DeleteBucket*"
)
var (
@@ -58,6 +59,8 @@ func MapToStatementAction(action string) string {
return s3_constants.ACTION_LIST
case StatementActionTagging:
return s3_constants.ACTION_TAGGING
case StatementActionDelete:
return s3_constants.ACTION_DELETE_BUCKET
default:
return ""
}
@@ -79,6 +82,8 @@ func MapToIdentitiesAction(action string) string {
return StatementActionList
case s3_constants.ACTION_TAGGING:
return StatementActionTagging
case s3_constants.ACTION_DELETE_BUCKET:
return StatementActionDelete
default:
return ""
}

View File

@@ -67,7 +67,7 @@ func (mc *MemChunk) ReadDataAt(p []byte, off int64, tsNs int64) (maxStop int64)
maxStop = max(maxStop, logicStop)
if t.TsNs >= tsNs {
println("read new data1", t.TsNs - tsNs, "ns")
println("read new data1", t.TsNs-tsNs, "ns")
}
}
}

View File

@@ -137,7 +137,7 @@ func (sc *SwapFileChunk) ReadDataAt(p []byte, off int64, tsNs int64) (maxStop in
maxStop = max(maxStop, logicStop)
if t.TsNs >= tsNs {
println("read new data2", t.TsNs - tsNs, "ns")
println("read new data2", t.TsNs-tsNs, "ns")
}
}
}

View File

@@ -35,7 +35,6 @@ func (b *MessageQueueBroker) ConfigureTopic(ctx context.Context, request *mq_pb.
}
}
t := topic.FromPbTopic(request.Topic)
var readErr, assignErr error
resp, readErr = b.fca.ReadTopicConfFromFiler(t)

View File

@@ -69,7 +69,7 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
return stream.Send(response)
}
var receivedSequence, acknowledgedSequence int64
var receivedSequence, acknowledgedSequence int64
var isClosed bool
// start sending ack to publisher
@@ -85,7 +85,7 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
lastAckTime := time.Now()
for !isClosed {
receivedSequence = atomic.LoadInt64(&localTopicPartition.AckTsNs)
if acknowledgedSequence < receivedSequence && (receivedSequence - acknowledgedSequence >= ackInterval || time.Since(lastAckTime) > 1*time.Second){
if acknowledgedSequence < receivedSequence && (receivedSequence-acknowledgedSequence >= ackInterval || time.Since(lastAckTime) > 1*time.Second) {
acknowledgedSequence = receivedSequence
response := &mq_pb.PublishMessageResponse{
AckSequence: acknowledgedSequence,
@@ -101,7 +101,6 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
}
}()
// process each published messages
clientName := fmt.Sprintf("%v-%4d/%s/%v", findClientAddress(stream.Context()), rand.Intn(10000), initMessage.Topic, initMessage.Partition)
localTopicPartition.Publishers.AddPublisher(clientName, topic.NewLocalPublisher())

View File

@@ -13,10 +13,11 @@ import (
)
type memBuffer struct {
buf []byte
startTime time.Time
stopTime time.Time
buf []byte
startTime time.Time
stopTime time.Time
}
func (b *MessageQueueBroker) PublishFollowMe(stream mq_pb.SeaweedMessaging_PublishFollowMeServer) (err error) {
var req *mq_pb.PublishFollowMeRequest
req, err = stream.Recv()
@@ -84,7 +85,6 @@ func (b *MessageQueueBroker) PublishFollowMe(stream mq_pb.SeaweedMessaging_Publi
}
}
t, p := topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.Partition)
logBuffer.ShutdownLogBuffer()
@@ -97,7 +97,6 @@ func (b *MessageQueueBroker) PublishFollowMe(stream mq_pb.SeaweedMessaging_Publi
partitionGeneration := time.Unix(0, p.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, p.RangeStart, p.RangeStop)
// flush the remaining messages
inMemoryBuffers.CloseInput()
for mem, found := inMemoryBuffers.Dequeue(); found; mem, found = inMemoryBuffers.Dequeue() {

View File

@@ -45,7 +45,7 @@ func (b *MessageQueueBroker) genLogFlushFunc(t topic.Topic, partition *mq_pb.Par
b.accessLock.Lock()
defer b.accessLock.Unlock()
p := topic.FromPbPartition(partition)
if localPartition:=b.localTopicManager.GetLocalPartition(t, p); localPartition!=nil {
if localPartition := b.localTopicManager.GetLocalPartition(t, p); localPartition != nil {
localPartition.NotifyLogFlushed(logBuffer.LastFlushTsNs)
}

View File

@@ -16,7 +16,7 @@ var (
concurrency = flag.Int("c", 4, "concurrent publishers")
partitionCount = flag.Int("p", 6, "partition count")
clientName = flag.String("client", "c1", "client name")
clientName = flag.String("client", "c1", "client name")
namespace = flag.String("ns", "test", "namespace")
t = flag.String("t", "test", "t")

View File

@@ -19,7 +19,7 @@ var (
concurrency = flag.Int("c", 4, "concurrent publishers")
partitionCount = flag.Int("p", 6, "partition count")
clientName = flag.String("client", "c1", "client name")
clientName = flag.String("client", "c1", "client name")
namespace = flag.String("ns", "test", "namespace")
t = flag.String("t", "test", "t")

View File

@@ -48,7 +48,7 @@ func (p *TopicPublisher) FinishPublish() error {
if inputBuffers, found := p.partition2Buffer.AllIntersections(0, pub_balancer.MaxPartitionCount); found {
for _, inputBuffer := range inputBuffers {
inputBuffer.Enqueue(&mq_pb.DataMessage{
TsNs: time.Now().UnixNano(),
TsNs: time.Now().UnixNano(),
Ctrl: &mq_pb.ControlMessage{
IsClose: true,
},

View File

@@ -17,7 +17,7 @@ type BrokerStats struct {
}
type TopicPartitionStats struct {
topic.TopicPartition
PublisherCount int32
PublisherCount int32
SubscriberCount int32
}
@@ -48,7 +48,7 @@ func (bs *BrokerStats) UpdateStats(stats *mq_pb.BrokerStats) {
UnixTimeNs: topicPartitionStats.Partition.UnixTimeNs,
},
},
PublisherCount: topicPartitionStats.PublisherCount,
PublisherCount: topicPartitionStats.PublisherCount,
SubscriberCount: topicPartitionStats.SubscriberCount,
}
publisherCount += topicPartitionStats.PublisherCount
@@ -76,7 +76,7 @@ func (bs *BrokerStats) RegisterAssignment(t *mq_pb.Topic, partition *mq_pb.Parti
UnixTimeNs: partition.UnixTimeNs,
},
},
PublisherCount: 0,
PublisherCount: 0,
SubscriberCount: 0,
}
key := tps.TopicPartition.String()

View File

@@ -6,17 +6,17 @@ import (
type Schema struct {
RecordType *schema_pb.RecordType
fieldMap map[string]*schema_pb.Field
fieldMap map[string]*schema_pb.Field
}
func NewSchema(recordType *schema_pb.RecordType) (*Schema, error) {
fieldMap := make( map[string]*schema_pb.Field)
fieldMap := make(map[string]*schema_pb.Field)
for _, field := range recordType.Fields {
fieldMap[field.Name] = field
}
return &Schema{
RecordType: recordType,
fieldMap: fieldMap,
fieldMap: fieldMap,
}, nil
}

View File

@@ -8,9 +8,9 @@ import (
var (
TypeBoolean = &schema_pb.Type{Kind: &schema_pb.Type_ScalarType{schema_pb.ScalarType_BOOL}}
TypeInt32 = &schema_pb.Type{Kind: &schema_pb.Type_ScalarType{schema_pb.ScalarType_INT32}}
TypeInt64 = &schema_pb.Type{Kind: &schema_pb.Type_ScalarType{schema_pb.ScalarType_INT64}}
TypeFloat = &schema_pb.Type{Kind: &schema_pb.Type_ScalarType{schema_pb.ScalarType_FLOAT}}
TypeDouble = &schema_pb.Type{Kind: &schema_pb.Type_ScalarType{schema_pb.ScalarType_DOUBLE}}
TypeInt64 = &schema_pb.Type{Kind: &schema_pb.Type_ScalarType{schema_pb.ScalarType_INT64}}
TypeFloat = &schema_pb.Type{Kind: &schema_pb.Type_ScalarType{schema_pb.ScalarType_FLOAT}}
TypeDouble = &schema_pb.Type{Kind: &schema_pb.Type_ScalarType{schema_pb.ScalarType_DOUBLE}}
TypeBytes = &schema_pb.Type{Kind: &schema_pb.Type_ScalarType{schema_pb.ScalarType_BYTES}}
TypeString = &schema_pb.Type{Kind: &schema_pb.Type_ScalarType{schema_pb.ScalarType_STRING}}
)

View File

@@ -32,10 +32,10 @@ func TestEnumScalarType(t *testing.T) {
func TestField(t *testing.T) {
field := &Field{
Name: "field_name",
Type: &Type{Kind: &Type_ScalarType{ScalarType: ScalarType_INT32}},
FieldIndex: 1,
IsRepeated: false,
Name: "field_name",
Type: &Type{Kind: &Type_ScalarType{ScalarType: ScalarType_INT32}},
FieldIndex: 1,
IsRepeated: false,
}
assert.NotNil(t, field)
}
@@ -44,32 +44,32 @@ func TestRecordType(t *testing.T) {
subRecord := &RecordType{
Fields: []*Field{
{
Name: "field_1",
Type: &Type{Kind: &Type_ScalarType{ScalarType: ScalarType_INT32}},
FieldIndex: 1,
IsRepeated: false,
Name: "field_1",
Type: &Type{Kind: &Type_ScalarType{ScalarType: ScalarType_INT32}},
FieldIndex: 1,
IsRepeated: false,
},
{
Name: "field_2",
Type: &Type{Kind: &Type_ScalarType{ScalarType: ScalarType_STRING}},
FieldIndex: 2,
IsRepeated: false,
Name: "field_2",
Type: &Type{Kind: &Type_ScalarType{ScalarType: ScalarType_STRING}},
FieldIndex: 2,
IsRepeated: false,
},
},
}
record := &RecordType{
Fields: []*Field{
{
Name: "field_key",
Type: &Type{Kind: &Type_ScalarType{ScalarType: ScalarType_INT32}},
FieldIndex: 1,
IsRepeated: false,
Name: "field_key",
Type: &Type{Kind: &Type_ScalarType{ScalarType: ScalarType_INT32}},
FieldIndex: 1,
IsRepeated: false,
},
{
Name: "field_record",
Type: &Type{Kind: &Type_RecordType{RecordType: subRecord}},
FieldIndex: 2,
IsRepeated: false,
Name: "field_record",
Type: &Type{Kind: &Type_RecordType{RecordType: subRecord}},
FieldIndex: 2,
IsRepeated: false,
},
},
}

View File

@@ -76,7 +76,7 @@ func TestStructToSchema(t *testing.T) {
RecordTypeBegin().
WithField("Field3", TypeString).
WithField("Field4", TypeInt32).
RecordTypeEnd(),
RecordTypeEnd(),
).
RecordTypeEnd(),
},
@@ -104,7 +104,7 @@ func TestStructToSchema(t *testing.T) {
RecordTypeBegin().
WithField("Field6", TypeString).
WithField("Field7", TypeBytes).
RecordTypeEnd(),
RecordTypeEnd(),
).RecordTypeEnd(),
).
RecordTypeEnd(),

View File

@@ -7,9 +7,9 @@ import (
type ParquetLevels struct {
startColumnIndex int
endColumnIndex int
definitionDepth int
levels map[string]*ParquetLevels
endColumnIndex int
definitionDepth int
levels map[string]*ParquetLevels
}
func ToParquetLevels(recordType *schema_pb.RecordType) (*ParquetLevels, error) {
@@ -19,7 +19,7 @@ func ToParquetLevels(recordType *schema_pb.RecordType) (*ParquetLevels, error) {
func toFieldTypeLevels(fieldType *schema_pb.Type, startColumnIndex, definitionDepth int) (*ParquetLevels, error) {
switch fieldType.Kind.(type) {
case *schema_pb.Type_ScalarType:
return toFieldTypeScalarLevels(fieldType.GetScalarType(), startColumnIndex, definitionDepth)
return toFieldTypeScalarLevels(fieldType.GetScalarType(), startColumnIndex, definitionDepth)
case *schema_pb.Type_RecordType:
return toRecordTypeLevels(fieldType.GetRecordType(), startColumnIndex, definitionDepth)
case *schema_pb.Type_ListType:
@@ -35,15 +35,15 @@ func toFieldTypeListLevels(listType *schema_pb.ListType, startColumnIndex, defin
func toFieldTypeScalarLevels(scalarType schema_pb.ScalarType, startColumnIndex, definitionDepth int) (*ParquetLevels, error) {
return &ParquetLevels{
startColumnIndex: startColumnIndex,
endColumnIndex: startColumnIndex + 1,
definitionDepth: definitionDepth,
endColumnIndex: startColumnIndex + 1,
definitionDepth: definitionDepth,
}, nil
}
func toRecordTypeLevels(recordType *schema_pb.RecordType, startColumnIndex, definitionDepth int) (*ParquetLevels, error) {
recordTypeLevels := &ParquetLevels{
startColumnIndex: startColumnIndex,
definitionDepth: definitionDepth,
levels: make(map[string]*ParquetLevels),
definitionDepth: definitionDepth,
levels: make(map[string]*ParquetLevels),
}
for _, field := range recordType.Fields {
fieldTypeLevels, err := toFieldTypeLevels(field.Type, startColumnIndex, definitionDepth+1)

View File

@@ -11,9 +11,9 @@ func TestToParquetLevels(t *testing.T) {
recordType *schema_pb.RecordType
}
tests := []struct {
name string
args args
want *ParquetLevels
name string
args args
want *ParquetLevels
}{
{
name: "nested type",
@@ -25,13 +25,13 @@ func TestToParquetLevels(t *testing.T) {
RecordTypeBegin().
WithField("zName", TypeString).
WithField("emails", ListOf(TypeString)).
RecordTypeEnd()).
RecordTypeEnd()).
WithField("Company", TypeString).
WithRecordField("Address",
RecordTypeBegin().
WithField("Street", TypeString).
WithField("City", TypeString).
RecordTypeEnd()).
RecordTypeEnd()).
RecordTypeEnd(),
},
want: &ParquetLevels{

View File

@@ -31,7 +31,6 @@ func toParquetFieldType(fieldType *schema_pb.Type) (dataType parquet.Node, err e
return nil, fmt.Errorf("unknown field type: %T", fieldType.Kind)
}
return dataType, err
}

View File

@@ -70,7 +70,7 @@ func doVisitValue(fieldType *schema_pb.Type, levels *ParquetLevels, fieldValue *
return
}
func toParquetValue(value *schema_pb.Value) (parquet.Value, error) {
func toParquetValue(value *schema_pb.Value) (parquet.Value, error) {
switch value.Kind.(type) {
case *schema_pb.Value_BoolValue:
return parquet.BooleanValue(value.GetBoolValue()), nil

View File

@@ -47,7 +47,7 @@ func toRecordValue(recordType *schema_pb.RecordType, levels *ParquetLevels, valu
func toListValue(listType *schema_pb.ListType, levels *ParquetLevels, values []parquet.Value, valueIndex int) (listValue *schema_pb.Value, endValueIndex int, err error) {
listValues := make([]*schema_pb.Value, 0)
var value *schema_pb.Value
for ;valueIndex < len(values); {
for valueIndex < len(values) {
if values[valueIndex].Column() != levels.startColumnIndex {
break
}
@@ -67,19 +67,19 @@ func toScalarValue(scalarType schema_pb.ScalarType, levels *ParquetLevels, value
}
switch scalarType {
case schema_pb.ScalarType_BOOL:
return &schema_pb.Value{Kind: &schema_pb.Value_BoolValue{BoolValue: value.Boolean()}}, valueIndex+1, nil
return &schema_pb.Value{Kind: &schema_pb.Value_BoolValue{BoolValue: value.Boolean()}}, valueIndex + 1, nil
case schema_pb.ScalarType_INT32:
return &schema_pb.Value{Kind: &schema_pb.Value_Int32Value{Int32Value: value.Int32()}}, valueIndex+1, nil
return &schema_pb.Value{Kind: &schema_pb.Value_Int32Value{Int32Value: value.Int32()}}, valueIndex + 1, nil
case schema_pb.ScalarType_INT64:
return &schema_pb.Value{Kind: &schema_pb.Value_Int64Value{Int64Value: value.Int64()}}, valueIndex+1, nil
return &schema_pb.Value{Kind: &schema_pb.Value_Int64Value{Int64Value: value.Int64()}}, valueIndex + 1, nil
case schema_pb.ScalarType_FLOAT:
return &schema_pb.Value{Kind: &schema_pb.Value_FloatValue{FloatValue: value.Float()}}, valueIndex+1, nil
return &schema_pb.Value{Kind: &schema_pb.Value_FloatValue{FloatValue: value.Float()}}, valueIndex + 1, nil
case schema_pb.ScalarType_DOUBLE:
return &schema_pb.Value{Kind: &schema_pb.Value_DoubleValue{DoubleValue: value.Double()}}, valueIndex+1, nil
return &schema_pb.Value{Kind: &schema_pb.Value_DoubleValue{DoubleValue: value.Double()}}, valueIndex + 1, nil
case schema_pb.ScalarType_BYTES:
return &schema_pb.Value{Kind: &schema_pb.Value_BytesValue{BytesValue: value.ByteArray()}}, valueIndex+1, nil
return &schema_pb.Value{Kind: &schema_pb.Value_BytesValue{BytesValue: value.ByteArray()}}, valueIndex + 1, nil
case schema_pb.ScalarType_STRING:
return &schema_pb.Value{Kind: &schema_pb.Value_StringValue{StringValue: string(value.ByteArray())}}, valueIndex+1, nil
return &schema_pb.Value{Kind: &schema_pb.Value_StringValue{StringValue: string(value.ByteArray())}}, valueIndex + 1, nil
}
return nil, valueIndex, fmt.Errorf("unsupported scalar type: %v", scalarType)
}

View File

@@ -19,13 +19,13 @@ func TestWriteReadParquet(t *testing.T) {
RecordTypeBegin().
WithField("zName", TypeString).
WithField("emails", ListOf(TypeString)).
RecordTypeEnd()).
RecordTypeEnd()).
WithField("Company", TypeString).
WithRecordField("Address",
RecordTypeBegin().
WithField("Street", TypeString).
WithField("City", TypeString).
RecordTypeEnd()).
RecordTypeEnd()).
RecordTypeEnd()
fmt.Printf("RecordType: %v\n", recordType)
@@ -85,9 +85,9 @@ func testWritingParquetFile(t *testing.T, count int, filename string, parquetSch
fmt.Sprintf("john_%d@c.com", i),
fmt.Sprintf("john_%d@d.com", i),
fmt.Sprintf("john_%d@e.com", i)).
RecordEnd()).
RecordEnd()).
SetString("Company", fmt.Sprintf("company_%d", i)).
RecordEnd()
RecordEnd()
AddRecordValue(rowBuilder, recordType, parquetLevels, recordValue)
if count < 10 {

View File

@@ -16,17 +16,17 @@ import (
)
type LocalPartition struct {
ListenersWaits int64
AckTsNs int64
ListenersWaits int64
AckTsNs int64
// notifying clients
ListenersLock sync.Mutex
ListenersCond *sync.Cond
Partition
LogBuffer *log_buffer.LogBuffer
Publishers *LocalPartitionPublishers
Subscribers *LocalPartitionSubscribers
LogBuffer *log_buffer.LogBuffer
Publishers *LocalPartitionPublishers
Subscribers *LocalPartitionSubscribers
publishFolloweMeStream mq_pb.SeaweedMessaging_PublishFollowMeClient
followerGrpcConnection *grpc.ClientConn
@@ -37,7 +37,7 @@ var TIME_FORMAT = "2006-01-02-15-04-05"
func NewLocalPartition(partition Partition, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
lp := &LocalPartition{
Partition: partition,
Partition: partition,
Publishers: NewLocalPartitionPublishers(),
Subscribers: NewLocalPartitionSubscribers(),
}
@@ -155,8 +155,8 @@ func (p *LocalPartition) MaybeConnectToFollowers(initMessage *mq_pb.PublishMessa
if err = p.publishFolloweMeStream.Send(&mq_pb.PublishFollowMeRequest{
Message: &mq_pb.PublishFollowMeRequest_Init{
Init: &mq_pb.PublishFollowMeRequest_InitMessage{
Topic: initMessage.Topic,
Partition: initMessage.Partition,
Topic: initMessage.Topic,
Partition: initMessage.Partition,
},
},
}); err != nil {

View File

@@ -11,6 +11,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/s3api/s3_constants"
"strconv"
"strings"
"encoding/base64"
"github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/glog"
@@ -202,7 +203,7 @@ func (s3sink *S3Sink) CreateEntry(key string, entry *filer_pb.Entry, signatures
Tagging: aws.String(tags),
}
if len(entry.Attributes.Md5) > 0 {
uploadInput.ContentMD5 = aws.String(fmt.Sprintf("%x", entry.Attributes.Md5))
uploadInput.ContentMD5 = aws.String(base64.StdEncoding.EncodeToString([]byte(entry.Attributes.Md5)))
}
_, err = uploader.Upload(&uploadInput)

View File

@@ -317,6 +317,7 @@ func (iam *IdentityAccessManagement) Auth(f http.HandlerFunc, action Action) htt
}
identity, errCode := iam.authRequest(r, action)
glog.V(3).Infof("auth error: %v", errCode)
if errCode == s3err.ErrNone {
if identity != nil && identity.Name != "" {
r.Header.Set(s3_constants.AmzIdentityId, identity.Name)
@@ -453,6 +454,7 @@ func (identity *Identity) canDo(action Action, bucket string, objectKey string)
}
}
if bucket == "" {
glog.V(3).Infof("identity %s is not allowed to perform action %s on %s -- bucket is empty", identity.Name, action, bucket+objectKey)
return false
}
target := string(action) + ":" + bucket + objectKey
@@ -477,6 +479,8 @@ func (identity *Identity) canDo(action Action, bucket string, objectKey string)
}
}
}
//log error
glog.V(3).Infof("identity %s is not allowed to perform action %s on %s", identity.Name, action, bucket+objectKey)
return false
}

View File

@@ -1,11 +1,12 @@
package s3api
import (
. "github.com/seaweedfs/seaweedfs/weed/s3api/s3_constants"
"github.com/stretchr/testify/assert"
"reflect"
"testing"
. "github.com/seaweedfs/seaweedfs/weed/s3api/s3_constants"
"github.com/stretchr/testify/assert"
"github.com/seaweedfs/seaweedfs/weed/pb/iam_pb"
jsonpb "google.golang.org/protobuf/encoding/protojson"
)
@@ -79,6 +80,7 @@ func TestCanDo(t *testing.T) {
}
// object specific
assert.Equal(t, true, ident1.canDo(ACTION_WRITE, "bucket1", "/a/b/c/d.txt"))
assert.Equal(t, false, ident1.canDo(ACTION_DELETE_BUCKET, "bucket1", ""))
assert.Equal(t, false, ident1.canDo(ACTION_WRITE, "bucket1", "/a/b/other/some"), "action without *")
// bucket specific
@@ -141,6 +143,15 @@ func TestCanDo(t *testing.T) {
},
}
assert.Equal(t, true, ident6.canDo(ACTION_READ, "anything_bucket", "/a/b/c/d.txt"))
//test deleteBucket operation
ident7 := &Identity{
Name: "anything",
Actions: []Action{
"DeleteBucket:bucket1",
},
}
assert.Equal(t, true, ident7.canDo(ACTION_DELETE_BUCKET, "bucket1", ""))
}
type LoadS3ApiConfigurationTestCase struct {

View File

@@ -1,13 +1,14 @@
package s3_constants
const (
ACTION_READ = "Read"
ACTION_READ_ACP = "ReadAcp"
ACTION_WRITE = "Write"
ACTION_WRITE_ACP = "WriteAcp"
ACTION_ADMIN = "Admin"
ACTION_TAGGING = "Tagging"
ACTION_LIST = "List"
ACTION_READ = "Read"
ACTION_READ_ACP = "ReadAcp"
ACTION_WRITE = "Write"
ACTION_WRITE_ACP = "WriteAcp"
ACTION_ADMIN = "Admin"
ACTION_TAGGING = "Tagging"
ACTION_LIST = "List"
ACTION_DELETE_BUCKET = "DeleteBucket"
SeaweedStorageDestinationHeader = "x-seaweedfs-destination"
MultipartUploadsFolder = ".uploads"

View File

@@ -7,7 +7,7 @@ import (
var (
CircuitBreakerConfigDir = "/etc/s3"
CircuitBreakerConfigFile = "circuit_breaker.json"
AllowedActions = []string{ACTION_READ, ACTION_READ_ACP, ACTION_WRITE, ACTION_WRITE_ACP, ACTION_LIST, ACTION_TAGGING, ACTION_ADMIN}
AllowedActions = []string{ACTION_READ, ACTION_READ_ACP, ACTION_WRITE, ACTION_WRITE_ACP, ACTION_LIST, ACTION_TAGGING, ACTION_ADMIN, ACTION_DELETE_BUCKET}
LimitTypeCount = "Count"
LimitTypeBytes = "MB"
Separator = ":"

View File

@@ -6,14 +6,15 @@ import (
"encoding/xml"
"errors"
"fmt"
"github.com/aws/aws-sdk-go/private/protocol/xml/xmlutil"
"github.com/seaweedfs/seaweedfs/weed/s3api/s3bucket"
"github.com/seaweedfs/seaweedfs/weed/util"
"math"
"net/http"
"strings"
"time"
"github.com/aws/aws-sdk-go/private/protocol/xml/xmlutil"
"github.com/seaweedfs/seaweedfs/weed/s3api/s3bucket"
"github.com/seaweedfs/seaweedfs/weed/util"
"github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/s3api/s3_constants"
"github.com/seaweedfs/seaweedfs/weed/storage/needle"
@@ -218,6 +219,10 @@ func (s3a *S3ApiServer) checkBucket(r *http.Request, bucket string) s3err.ErrorC
return s3err.ErrNoSuchBucket
}
//if iam is enabled, the access was already checked before
if s3a.iam.isEnabled() {
return s3err.ErrNone
}
if !s3a.hasAccess(r, entry) {
return s3err.ErrAccessDenied
}
@@ -236,6 +241,7 @@ func (s3a *S3ApiServer) hasAccess(r *http.Request, entry *filer_pb.Entry) bool {
identityId := r.Header.Get(s3_constants.AmzIdentityId)
if id, ok := entry.Extended[s3_constants.AmzIdentityId]; ok {
if identityId != string(id) {
glog.V(3).Infof("hasAccess: %s != %s (entry.Extended = %v)", identityId, id, entry.Extended)
return false
}
}

View File

@@ -279,7 +279,7 @@ func (s3a *S3ApiServer) registerRouter(router *mux.Router) {
bucket.Methods("PUT").HandlerFunc(track(s3a.iam.Auth(s3a.cb.Limit(s3a.PutBucketHandler, ACTION_ADMIN)), "PUT"))
// DeleteBucket
bucket.Methods("DELETE").HandlerFunc(track(s3a.iam.Auth(s3a.cb.Limit(s3a.DeleteBucketHandler, ACTION_ADMIN)), "DELETE"))
bucket.Methods("DELETE").HandlerFunc(track(s3a.iam.Auth(s3a.cb.Limit(s3a.DeleteBucketHandler, ACTION_DELETE_BUCKET)), "DELETE"))
// ListObjectsV1 (Legacy)
bucket.Methods("GET").HandlerFunc(track(s3a.iam.Auth(s3a.cb.Limit(s3a.ListObjectsV1Handler, ACTION_LIST)), "LIST"))

View File

@@ -278,7 +278,7 @@ func TestDeleteEmptySelection(t *testing.T) {
eachDataNode(topologyInfo, func(dc string, rack RackId, dn *master_pb.DataNodeInfo) {
for _, diskInfo := range dn.DiskInfos {
for _, v := range diskInfo.VolumeInfos {
if v.Size <= super_block.SuperBlockSize && v.ModifiedAtSecond > 0 {
if v.Size <= super_block.SuperBlockSize && v.ModifiedAtSecond > 0 {
fmt.Printf("empty volume %d from %s\n", v.Id, dn.Id)
}
}

View File

@@ -1,5 +1,5 @@
//go:build openbsd || netbsd || plan9 || solaris
// +build openbsd netbsd plan9 solaris
//go:build netbsd || plan9 || solaris
// +build netbsd plan9 solaris
package stats

View File

@@ -0,0 +1,25 @@
//go:build openbsd
// +build openbsd
package stats
import (
"syscall"
"github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb"
)
func fillInDiskStatus(disk *volume_server_pb.DiskStatus) {
fs := syscall.Statfs_t{}
err := syscall.Statfs(disk.Dir, &fs)
if err != nil {
return
}
disk.All = fs.F_blocks * uint64(fs.F_bsize)
disk.Free = fs.F_bfree * uint64(fs.F_bsize)
disk.Used = disk.All - disk.Free
disk.PercentFree = float32((float64(disk.Free) / float64(disk.All)) * 100)
disk.PercentUsed = float32((float64(disk.Used) / float64(disk.All)) * 100)
return
}

View File

@@ -36,8 +36,8 @@ func LoadCompactNeedleMap(file *os.File) (*NeedleMap, error) {
func doLoading(file *os.File, nm *NeedleMap) (*NeedleMap, error) {
e := idx.WalkIndexFile(file, 0, func(key NeedleId, offset Offset, size Size) error {
nm.MaybeSetMaxFileKey(key)
nm.FileCounter++
if !offset.IsZero() && size.IsValid() {
nm.FileCounter++
nm.FileByteCounter = nm.FileByteCounter + uint64(size)
oldOffset, oldSize := nm.m.Set(NeedleId(key), offset, size)
if !oldOffset.IsZero() && oldSize.IsValid() {
@@ -51,7 +51,7 @@ func doLoading(file *os.File, nm *NeedleMap) (*NeedleMap, error) {
}
return nil
})
glog.V(1).Infof("max file key: %d for file: %s", nm.MaxFileKey(), file.Name())
glog.V(1).Infof("max file key: %v count: %d deleted: %d for file: %s", nm.MaxFileKey(), nm.FileCount(), nm.DeletedCount(), file.Name())
return nm, e
}

View File

@@ -109,9 +109,6 @@ func verifyNeedleIntegrity(datFile backend.BackendStorageFile, v needle.Version,
return 0, fmt.Errorf("verifyNeedleIntegrity check %s entry offset %d size %d: %v", datFile.Name(), offset, size, err)
}
n.AppendAtNs = util.BytesToUint64(bytes)
if n.HasTtl() {
return n.AppendAtNs, nil
}
fileTailOffset := offset + needle.GetActualSize(size, v)
fileSize, _, err := datFile.GetStat()
if err != nil {
@@ -130,7 +127,7 @@ func verifyNeedleIntegrity(datFile backend.BackendStorageFile, v needle.Version,
return n.AppendAtNs, fmt.Errorf("read data [%d,%d) : %v", offset, offset+int64(size), err)
}
if n.Id != key {
return n.AppendAtNs, fmt.Errorf("index key %#x does not match needle's Id %#x", key, n.Id)
return n.AppendAtNs, fmt.Errorf("index key %v does not match needle's Id %v", key, n.Id)
}
return n.AppendAtNs, err
}
@@ -147,7 +144,7 @@ func verifyDeletedNeedleIntegrity(datFile backend.BackendStorageFile, v needle.V
return n.AppendAtNs, fmt.Errorf("read data [%d,%d) : %v", fileSize-size, size, err)
}
if n.Id != key {
return n.AppendAtNs, fmt.Errorf("index key %#x does not match needle's Id %#x", key, n.Id)
return n.AppendAtNs, fmt.Errorf("index key %v does not match needle's Id %v", key, n.Id)
}
return n.AppendAtNs, err
}

View File

@@ -487,19 +487,21 @@ func (v *Volume) copyDataBasedOnIndexFile(srcDatName, srcIdxName, dstDatName, da
if err != nil {
return err
}
dstDatSize, _, err := dstDatBackend.GetStat()
if err != nil {
return err
}
if v.nm.ContentSize() > v.nm.DeletedSize() {
expectedContentSize := v.nm.ContentSize() - v.nm.DeletedSize()
if expectedContentSize > uint64(dstDatSize) {
return fmt.Errorf("volume %s unexpected new data size: %d does not match size of content minus deleted: %d",
v.Id.String(), dstDatSize, expectedContentSize)
if v.Ttl.String() == "" {
dstDatSize, _, err := dstDatBackend.GetStat()
if err != nil {
return err
}
if v.nm.ContentSize() > v.nm.DeletedSize() {
expectedContentSize := v.nm.ContentSize() - v.nm.DeletedSize()
if expectedContentSize > uint64(dstDatSize) {
return fmt.Errorf("volume %s unexpected new data size: %d does not match size of content minus deleted: %d",
v.Id.String(), dstDatSize, expectedContentSize)
}
} else {
glog.Warningf("volume %s content size: %d less deleted size: %d, new size: %d",
v.Id.String(), v.nm.ContentSize(), v.nm.DeletedSize(), dstDatSize)
}
} else {
glog.Warningf("volume %s content size: %d less deleted size: %d, new size: %d",
v.Id.String(), v.nm.ContentSize(), v.nm.DeletedSize(), dstDatSize)
}
err = newNm.SaveToIdx(datIdxName)
if err != nil {

View File

@@ -31,20 +31,20 @@ type VolumeGrowRequest struct {
}
type volumeGrowthStrategy struct {
Copy1Count int
Copy2Count int
Copy3Count int
Copy1Count int
Copy2Count int
Copy3Count int
CopyOtherCount int
Threshold float64
Threshold float64
}
var (
VolumeGrowStrategy = volumeGrowthStrategy{
Copy1Count: 7,
Copy2Count: 6,
Copy3Count: 3,
Copy1Count: 7,
Copy2Count: 6,
Copy3Count: 3,
CopyOtherCount: 1,
Threshold: 0.9,
Threshold: 0.9,
}
)
@@ -77,7 +77,8 @@ func NewDefaultVolumeGrowth() *VolumeGrowth {
// given copyCount, how many logical volumes to create
func (vg *VolumeGrowth) findVolumeCount(copyCount int) (count int) {
switch copyCount {
case 1: count = VolumeGrowStrategy.Copy1Count
case 1:
count = VolumeGrowStrategy.Copy1Count
case 2:
count = VolumeGrowStrategy.Copy2Count
case 3:

View File

@@ -381,7 +381,7 @@ func (vl *VolumeLayout) GetActiveVolumeCount(option *VolumeGrowOption) (total, a
}
active++
info, _ := dn.GetVolumesById(v)
if float64(info.Size) > float64(vl.volumeSizeLimit)* VolumeGrowStrategy.Threshold{
if float64(info.Size) > float64(vl.volumeSizeLimit)*VolumeGrowStrategy.Threshold {
crowded++
}
}

View File

@@ -19,7 +19,7 @@ func TestNewLogBufferFirstBuffer(t *testing.T) {
}, nil, func() {
})
startTime := MessagePosition{Time:time.Now()}
startTime := MessagePosition{Time: time.Now()}
messageSize := 1024
messageCount := 5000
@@ -38,7 +38,7 @@ func TestNewLogBufferFirstBuffer(t *testing.T) {
println("processed all messages")
return true, io.EOF
}
return false,nil
return false, nil
})
fmt.Printf("before flush: sent %d received %d\n", messageCount, receivedMessageCount)