From 9992e3da622895519d2f1e28190b83247151aff1 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sat, 4 Jul 2020 14:08:06 -0400 Subject: [PATCH 01/38] [dbnode] Add namespace runtime options manager --- .../generated/proto/namespace/namespace.pb.go | 357 +++++++++++++++--- .../generated/proto/namespace/namespace.proto | 28 +- src/dbnode/namespace/convert.go | 59 ++- src/dbnode/namespace/namespace_mock.go | 36 +- .../namespace/namespace_runtime_options.go | 252 +++++++++++++ src/dbnode/namespace/options.go | 19 +- src/dbnode/namespace/types.go | 9 +- src/dbnode/storage/database.go | 31 +- src/dbnode/storage/index.go | 185 +++++---- src/dbnode/storage/index/block.go | 18 + src/dbnode/storage/index/block_prop_test.go | 3 +- src/dbnode/storage/index/block_test.go | 2 + src/dbnode/storage/index/mutable_segments.go | 72 +++- src/dbnode/storage/index/types.go | 30 +- src/dbnode/storage/namespace.go | 4 +- src/m3ninx/index/segment/builder/builder.go | 96 ++++- src/m3ninx/index/segment/mem/segment.go | 8 + src/m3ninx/index/segment/types.go | 6 + 18 files changed, 1026 insertions(+), 189 deletions(-) create mode 100644 src/dbnode/namespace/namespace_runtime_options.go diff --git a/src/dbnode/generated/proto/namespace/namespace.pb.go b/src/dbnode/generated/proto/namespace/namespace.pb.go index 4e817778c8..8d4ff1f104 100644 --- a/src/dbnode/generated/proto/namespace/namespace.pb.go +++ b/src/dbnode/generated/proto/namespace/namespace.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: github.com/m3db/m3/src/dbnode/generated/proto/namespace/namespace.proto -// Copyright (c) 2019 Uber Technologies, Inc. +// Copyright (c) 2020 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy // of this software and associated documentation files (the "Software"), to deal @@ -33,6 +33,7 @@ IndexOptions NamespaceOptions Registry + NamespaceRuntimeOptions SchemaOptions SchemaHistory FileDescriptorSet @@ -42,6 +43,7 @@ package namespace import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" +import google_protobuf "github.com/gogo/protobuf/types" import io "io" @@ -145,16 +147,17 @@ func (m *IndexOptions) GetBlockSizeNanos() int64 { } type NamespaceOptions struct { - BootstrapEnabled bool `protobuf:"varint,1,opt,name=bootstrapEnabled,proto3" json:"bootstrapEnabled,omitempty"` - FlushEnabled bool `protobuf:"varint,2,opt,name=flushEnabled,proto3" json:"flushEnabled,omitempty"` - WritesToCommitLog bool `protobuf:"varint,3,opt,name=writesToCommitLog,proto3" json:"writesToCommitLog,omitempty"` - CleanupEnabled bool `protobuf:"varint,4,opt,name=cleanupEnabled,proto3" json:"cleanupEnabled,omitempty"` - RepairEnabled bool `protobuf:"varint,5,opt,name=repairEnabled,proto3" json:"repairEnabled,omitempty"` - RetentionOptions *RetentionOptions `protobuf:"bytes,6,opt,name=retentionOptions" json:"retentionOptions,omitempty"` - SnapshotEnabled bool `protobuf:"varint,7,opt,name=snapshotEnabled,proto3" json:"snapshotEnabled,omitempty"` - IndexOptions *IndexOptions `protobuf:"bytes,8,opt,name=indexOptions" json:"indexOptions,omitempty"` - SchemaOptions *SchemaOptions `protobuf:"bytes,9,opt,name=schemaOptions" json:"schemaOptions,omitempty"` - ColdWritesEnabled bool `protobuf:"varint,10,opt,name=coldWritesEnabled,proto3" json:"coldWritesEnabled,omitempty"` + BootstrapEnabled bool `protobuf:"varint,1,opt,name=bootstrapEnabled,proto3" json:"bootstrapEnabled,omitempty"` + FlushEnabled bool `protobuf:"varint,2,opt,name=flushEnabled,proto3" json:"flushEnabled,omitempty"` + WritesToCommitLog bool `protobuf:"varint,3,opt,name=writesToCommitLog,proto3" json:"writesToCommitLog,omitempty"` + CleanupEnabled bool `protobuf:"varint,4,opt,name=cleanupEnabled,proto3" json:"cleanupEnabled,omitempty"` + RepairEnabled bool `protobuf:"varint,5,opt,name=repairEnabled,proto3" json:"repairEnabled,omitempty"` + RetentionOptions *RetentionOptions `protobuf:"bytes,6,opt,name=retentionOptions" json:"retentionOptions,omitempty"` + SnapshotEnabled bool `protobuf:"varint,7,opt,name=snapshotEnabled,proto3" json:"snapshotEnabled,omitempty"` + IndexOptions *IndexOptions `protobuf:"bytes,8,opt,name=indexOptions" json:"indexOptions,omitempty"` + SchemaOptions *SchemaOptions `protobuf:"bytes,9,opt,name=schemaOptions" json:"schemaOptions,omitempty"` + ColdWritesEnabled bool `protobuf:"varint,10,opt,name=coldWritesEnabled,proto3" json:"coldWritesEnabled,omitempty"` + RuntimeOptions *NamespaceRuntimeOptions `protobuf:"bytes,11,opt,name=runtimeOptions" json:"runtimeOptions,omitempty"` } func (m *NamespaceOptions) Reset() { *m = NamespaceOptions{} } @@ -232,6 +235,13 @@ func (m *NamespaceOptions) GetColdWritesEnabled() bool { return false } +func (m *NamespaceOptions) GetRuntimeOptions() *NamespaceRuntimeOptions { + if m != nil { + return m.RuntimeOptions + } + return nil +} + type Registry struct { Namespaces map[string]*NamespaceOptions `protobuf:"bytes,1,rep,name=namespaces" json:"namespaces,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` } @@ -248,11 +258,36 @@ func (m *Registry) GetNamespaces() map[string]*NamespaceOptions { return nil } +type NamespaceRuntimeOptions struct { + WriteIndexingPerCPUConcurrency *google_protobuf.DoubleValue `protobuf:"bytes,1,opt,name=writeIndexingPerCPUConcurrency" json:"writeIndexingPerCPUConcurrency,omitempty"` + FlushIndexingPerCPUConcurrency *google_protobuf.DoubleValue `protobuf:"bytes,2,opt,name=flushIndexingPerCPUConcurrency" json:"flushIndexingPerCPUConcurrency,omitempty"` +} + +func (m *NamespaceRuntimeOptions) Reset() { *m = NamespaceRuntimeOptions{} } +func (m *NamespaceRuntimeOptions) String() string { return proto.CompactTextString(m) } +func (*NamespaceRuntimeOptions) ProtoMessage() {} +func (*NamespaceRuntimeOptions) Descriptor() ([]byte, []int) { return fileDescriptorNamespace, []int{4} } + +func (m *NamespaceRuntimeOptions) GetWriteIndexingPerCPUConcurrency() *google_protobuf.DoubleValue { + if m != nil { + return m.WriteIndexingPerCPUConcurrency + } + return nil +} + +func (m *NamespaceRuntimeOptions) GetFlushIndexingPerCPUConcurrency() *google_protobuf.DoubleValue { + if m != nil { + return m.FlushIndexingPerCPUConcurrency + } + return nil +} + func init() { proto.RegisterType((*RetentionOptions)(nil), "namespace.RetentionOptions") proto.RegisterType((*IndexOptions)(nil), "namespace.IndexOptions") proto.RegisterType((*NamespaceOptions)(nil), "namespace.NamespaceOptions") proto.RegisterType((*Registry)(nil), "namespace.Registry") + proto.RegisterType((*NamespaceRuntimeOptions)(nil), "namespace.NamespaceRuntimeOptions") } func (m *RetentionOptions) Marshal() (dAtA []byte, err error) { size := m.Size() @@ -460,6 +495,16 @@ func (m *NamespaceOptions) MarshalTo(dAtA []byte) (int, error) { } i++ } + if m.RuntimeOptions != nil { + dAtA[i] = 0x5a + i++ + i = encodeVarintNamespace(dAtA, i, uint64(m.RuntimeOptions.Size())) + n4, err := m.RuntimeOptions.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n4 + } return i, nil } @@ -498,17 +543,55 @@ func (m *Registry) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintNamespace(dAtA, i, uint64(v.Size())) - n4, err := v.MarshalTo(dAtA[i:]) + n5, err := v.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n4 + i += n5 } } } return i, nil } +func (m *NamespaceRuntimeOptions) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *NamespaceRuntimeOptions) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.WriteIndexingPerCPUConcurrency != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintNamespace(dAtA, i, uint64(m.WriteIndexingPerCPUConcurrency.Size())) + n6, err := m.WriteIndexingPerCPUConcurrency.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n6 + } + if m.FlushIndexingPerCPUConcurrency != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintNamespace(dAtA, i, uint64(m.FlushIndexingPerCPUConcurrency.Size())) + n7, err := m.FlushIndexingPerCPUConcurrency.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n7 + } + return i, nil +} + func encodeVarintNamespace(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -593,6 +676,10 @@ func (m *NamespaceOptions) Size() (n int) { if m.ColdWritesEnabled { n += 2 } + if m.RuntimeOptions != nil { + l = m.RuntimeOptions.Size() + n += 1 + l + sovNamespace(uint64(l)) + } return n } @@ -615,6 +702,20 @@ func (m *Registry) Size() (n int) { return n } +func (m *NamespaceRuntimeOptions) Size() (n int) { + var l int + _ = l + if m.WriteIndexingPerCPUConcurrency != nil { + l = m.WriteIndexingPerCPUConcurrency.Size() + n += 1 + l + sovNamespace(uint64(l)) + } + if m.FlushIndexingPerCPUConcurrency != nil { + l = m.FlushIndexingPerCPUConcurrency.Size() + n += 1 + l + sovNamespace(uint64(l)) + } + return n +} + func sovNamespace(x uint64) (n int) { for { n++ @@ -1169,6 +1270,39 @@ func (m *NamespaceOptions) Unmarshal(dAtA []byte) error { } } m.ColdWritesEnabled = bool(v != 0) + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RuntimeOptions", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowNamespace + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthNamespace + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RuntimeOptions == nil { + m.RuntimeOptions = &NamespaceRuntimeOptions{} + } + if err := m.RuntimeOptions.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipNamespace(dAtA[iNdEx:]) @@ -1363,6 +1497,122 @@ func (m *Registry) Unmarshal(dAtA []byte) error { } return nil } +func (m *NamespaceRuntimeOptions) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowNamespace + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: NamespaceRuntimeOptions: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: NamespaceRuntimeOptions: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WriteIndexingPerCPUConcurrency", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowNamespace + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthNamespace + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WriteIndexingPerCPUConcurrency == nil { + m.WriteIndexingPerCPUConcurrency = &google_protobuf.DoubleValue{} + } + if err := m.WriteIndexingPerCPUConcurrency.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FlushIndexingPerCPUConcurrency", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowNamespace + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthNamespace + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.FlushIndexingPerCPUConcurrency == nil { + m.FlushIndexingPerCPUConcurrency = &google_protobuf.DoubleValue{} + } + if err := m.FlushIndexingPerCPUConcurrency.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipNamespace(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthNamespace + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipNamespace(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 @@ -1473,41 +1723,48 @@ func init() { } var fileDescriptorNamespace = []byte{ - // 575 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x94, 0x51, 0x6b, 0xd3, 0x50, - 0x14, 0xc7, 0x4d, 0xb3, 0xad, 0xed, 0x59, 0xe7, 0xe2, 0x45, 0xb0, 0x54, 0x28, 0xa3, 0x8a, 0x14, - 0x91, 0x06, 0xdb, 0x17, 0x51, 0x18, 0xcc, 0xad, 0x0e, 0x41, 0x6a, 0xb9, 0x13, 0x84, 0xbd, 0xdd, - 0x24, 0xa7, 0x6d, 0x58, 0x92, 0x1b, 0xee, 0xbd, 0xd1, 0xd5, 0xcf, 0xe0, 0x83, 0xdf, 0xc3, 0x2f, - 0xe2, 0xa3, 0x1f, 0x41, 0xea, 0xd7, 0xf0, 0x41, 0x72, 0x63, 0xba, 0x24, 0x1d, 0x32, 0xf6, 0x52, - 0x6e, 0xff, 0xe7, 0x77, 0xce, 0xb9, 0x3d, 0xff, 0x73, 0x0b, 0xa7, 0x73, 0x5f, 0x2d, 0x12, 0x67, - 0xe0, 0xf2, 0xd0, 0x0e, 0x47, 0x9e, 0x63, 0x87, 0x23, 0x5b, 0x0a, 0xd7, 0xf6, 0x9c, 0x88, 0x7b, - 0x68, 0xcf, 0x31, 0x42, 0xc1, 0x14, 0x7a, 0x76, 0x2c, 0xb8, 0xe2, 0x76, 0xc4, 0x42, 0x94, 0x31, - 0x73, 0xf1, 0xea, 0x34, 0xd0, 0x11, 0xd2, 0x5c, 0x0b, 0x9d, 0x93, 0xdb, 0xd6, 0x94, 0xee, 0x02, - 0x43, 0x96, 0x15, 0xec, 0x7d, 0x35, 0xc1, 0xa2, 0xa8, 0x30, 0x52, 0x3e, 0x8f, 0xde, 0xc7, 0xe9, - 0xa7, 0x24, 0x43, 0xb8, 0x2f, 0x72, 0x6d, 0x8a, 0xc2, 0xe7, 0xde, 0x84, 0x45, 0x5c, 0xb6, 0x8d, - 0x03, 0xa3, 0x6f, 0xd2, 0x6b, 0x63, 0xe4, 0x09, 0xdc, 0x75, 0x02, 0xee, 0x5e, 0x9c, 0xf9, 0x5f, - 0x30, 0xa3, 0x6b, 0x9a, 0xae, 0xa8, 0xe4, 0x19, 0xdc, 0x73, 0x92, 0xd9, 0x0c, 0xc5, 0x9b, 0x44, - 0x25, 0xe2, 0x1f, 0x6a, 0x6a, 0x74, 0x33, 0x40, 0xfa, 0xb0, 0x9f, 0x89, 0x53, 0x26, 0x55, 0xc6, - 0x6e, 0x69, 0xb6, 0x2a, 0x6b, 0x32, 0xed, 0x74, 0xc2, 0x14, 0x1b, 0x5f, 0xc6, 0xbe, 0x58, 0xb6, - 0xb7, 0x0f, 0x8c, 0x7e, 0x83, 0x56, 0x65, 0x72, 0x0e, 0xfd, 0x8a, 0x74, 0x34, 0x53, 0x28, 0x26, - 0x5c, 0x1d, 0xb9, 0x2e, 0x4a, 0x59, 0xfc, 0xc5, 0x3b, 0xba, 0xd9, 0x8d, 0x79, 0x72, 0x08, 0x9d, - 0x99, 0xbe, 0x3e, 0xbd, 0x6e, 0x7e, 0x75, 0x5d, 0xed, 0x3f, 0x44, 0x6f, 0x0a, 0xad, 0xb7, 0x91, - 0x87, 0x97, 0xb9, 0x13, 0x6d, 0xa8, 0x63, 0xc4, 0x9c, 0x00, 0x3d, 0x3d, 0xfc, 0x06, 0xcd, 0xbf, - 0xde, 0x74, 0xde, 0xbd, 0x3f, 0x26, 0x58, 0x93, 0xdc, 0xfb, 0xbc, 0xec, 0x53, 0xb0, 0x1c, 0xce, - 0x95, 0x54, 0x82, 0xc5, 0xe3, 0x52, 0xfd, 0x0d, 0x9d, 0xf4, 0xa0, 0x35, 0x0b, 0x12, 0xb9, 0xc8, - 0xb9, 0x9a, 0xe6, 0x4a, 0x5a, 0x6a, 0xea, 0x67, 0xe1, 0x2b, 0x94, 0x1f, 0xf8, 0x31, 0x0f, 0x43, - 0x5f, 0xbd, 0xe3, 0x73, 0x6d, 0x6a, 0x83, 0x6e, 0x06, 0xd2, 0xab, 0xbb, 0x01, 0xb2, 0x28, 0x59, - 0xf7, 0xde, 0xd2, 0x68, 0x45, 0x25, 0x8f, 0x61, 0x4f, 0x60, 0xcc, 0x7c, 0x91, 0x63, 0x99, 0xa1, - 0x65, 0x91, 0x9c, 0x82, 0x25, 0x2a, 0x0b, 0xac, 0x6d, 0xdb, 0x1d, 0x3e, 0x1c, 0x5c, 0x3d, 0x9f, - 0xea, 0x8e, 0xd3, 0x8d, 0xa4, 0x74, 0x83, 0x64, 0xc4, 0x62, 0xb9, 0xe0, 0x2a, 0x6f, 0x58, 0xcf, - 0x36, 0xa8, 0x22, 0x93, 0x57, 0xd0, 0xf2, 0x0b, 0x2e, 0xb5, 0x1b, 0xba, 0xdd, 0x83, 0x42, 0xbb, - 0xa2, 0x89, 0xb4, 0x04, 0x93, 0x43, 0xd8, 0xcb, 0x5e, 0x60, 0x9e, 0xdd, 0xd4, 0xd9, 0xed, 0x42, - 0xf6, 0x59, 0x31, 0x4e, 0xcb, 0x78, 0x3a, 0x6b, 0x97, 0x07, 0xde, 0x47, 0x3d, 0xd6, 0xfc, 0xa2, - 0x90, 0xcd, 0x7a, 0x23, 0xd0, 0xfb, 0x6e, 0x40, 0x83, 0xe2, 0xdc, 0x97, 0x4a, 0x2c, 0xc9, 0x31, - 0xc0, 0xba, 0x49, 0xfa, 0x9a, 0xcd, 0xfe, 0xee, 0xf0, 0x51, 0x69, 0x48, 0x19, 0x38, 0x58, 0x2f, - 0x8c, 0x1c, 0x47, 0x4a, 0x2c, 0x69, 0x21, 0xad, 0x73, 0x0e, 0xfb, 0x95, 0x30, 0xb1, 0xc0, 0xbc, - 0xc0, 0xa5, 0xde, 0xa0, 0x26, 0x4d, 0x8f, 0xe4, 0x39, 0x6c, 0x7f, 0x62, 0x41, 0x82, 0x7a, 0x5b, - 0xca, 0x4e, 0x54, 0x97, 0x91, 0x66, 0xe4, 0xcb, 0xda, 0x0b, 0xe3, 0xb5, 0xf5, 0x63, 0xd5, 0x35, - 0x7e, 0xae, 0xba, 0xc6, 0xaf, 0x55, 0xd7, 0xf8, 0xf6, 0xbb, 0x7b, 0xc7, 0xd9, 0xd1, 0x7f, 0x53, - 0xa3, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x60, 0xfa, 0xeb, 0x0c, 0x42, 0x05, 0x00, 0x00, + // 679 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x54, 0x5d, 0x6e, 0xd3, 0x4c, + 0x14, 0xfd, 0x9c, 0xf4, 0x27, 0x9d, 0xf4, 0x27, 0xdf, 0x08, 0xa9, 0x51, 0x40, 0x51, 0x65, 0x10, + 0x8a, 0x10, 0x8a, 0x45, 0xfa, 0x82, 0x40, 0xaa, 0x54, 0xd2, 0x52, 0x81, 0x50, 0x89, 0xa6, 0xfc, + 0x48, 0x7d, 0x1b, 0xdb, 0x37, 0x8e, 0x55, 0x7b, 0xc6, 0x9a, 0x19, 0xd3, 0x86, 0x35, 0xf0, 0xc0, + 0x3e, 0xd8, 0x48, 0x1f, 0x59, 0x02, 0x2a, 0x62, 0x1f, 0xc8, 0x33, 0x38, 0xb5, 0x9d, 0xb6, 0x54, + 0xbc, 0x44, 0xce, 0xb9, 0xe7, 0xde, 0x63, 0xdf, 0x73, 0x66, 0xd0, 0x41, 0x10, 0xaa, 0x49, 0xea, + 0xf6, 0x3d, 0x1e, 0x3b, 0xf1, 0xb6, 0xef, 0x3a, 0xf1, 0xb6, 0x23, 0x85, 0xe7, 0xf8, 0x2e, 0xe3, + 0x3e, 0x38, 0x01, 0x30, 0x10, 0x54, 0x81, 0xef, 0x24, 0x82, 0x2b, 0xee, 0x30, 0x1a, 0x83, 0x4c, + 0xa8, 0x07, 0x97, 0x4f, 0x7d, 0x5d, 0xc1, 0x2b, 0x33, 0xa0, 0xd3, 0x0d, 0x38, 0x0f, 0x22, 0x30, + 0x2d, 0x6e, 0x3a, 0x76, 0x4e, 0x05, 0x4d, 0x12, 0x10, 0xd2, 0x50, 0x3b, 0x7b, 0xff, 0xaa, 0x29, + 0xbd, 0x09, 0xc4, 0xd4, 0x4c, 0xb1, 0xbf, 0xd4, 0x51, 0x8b, 0x80, 0x02, 0xa6, 0x42, 0xce, 0xde, + 0x26, 0xd9, 0xaf, 0xc4, 0x03, 0x74, 0x47, 0xe4, 0xd8, 0x08, 0x44, 0xc8, 0xfd, 0x43, 0xca, 0xb8, + 0x6c, 0x5b, 0x5b, 0x56, 0xaf, 0x4e, 0xae, 0xac, 0xe1, 0x87, 0x68, 0xdd, 0x8d, 0xb8, 0x77, 0x72, + 0x14, 0x7e, 0x06, 0xc3, 0xae, 0x69, 0x76, 0x05, 0xc5, 0x8f, 0xd1, 0xff, 0x6e, 0x3a, 0x1e, 0x83, + 0x78, 0x99, 0xaa, 0x54, 0xfc, 0xa1, 0xd6, 0x35, 0x75, 0xbe, 0x80, 0x7b, 0x68, 0xc3, 0x80, 0x23, + 0x2a, 0x95, 0xe1, 0x2e, 0x68, 0x6e, 0x15, 0xd6, 0xcc, 0x4c, 0x69, 0x8f, 0x2a, 0xba, 0x7f, 0x96, + 0x84, 0x62, 0xda, 0x5e, 0xdc, 0xb2, 0x7a, 0x0d, 0x52, 0x85, 0xf1, 0x31, 0xea, 0x55, 0xa0, 0xdd, + 0xb1, 0x02, 0x71, 0xc8, 0xd5, 0xae, 0xe7, 0x81, 0x94, 0xc5, 0x2f, 0x5e, 0xd2, 0x62, 0xb7, 0xe6, + 0xe3, 0x1d, 0xd4, 0x19, 0xeb, 0xd7, 0x27, 0x57, 0xed, 0x6f, 0x59, 0x4f, 0xbb, 0x81, 0x61, 0x8f, + 0xd0, 0xea, 0x2b, 0xe6, 0xc3, 0x59, 0xee, 0x44, 0x1b, 0x2d, 0x03, 0xa3, 0x6e, 0x04, 0xbe, 0x5e, + 0x7e, 0x83, 0xe4, 0x7f, 0x6f, 0xbb, 0x6f, 0xfb, 0x7c, 0x01, 0xb5, 0x0e, 0x73, 0xef, 0xf3, 0xb1, + 0x8f, 0x50, 0xcb, 0xe5, 0x5c, 0x49, 0x25, 0x68, 0xb2, 0x5f, 0x9a, 0x3f, 0x87, 0x63, 0x1b, 0xad, + 0x8e, 0xa3, 0x54, 0x4e, 0x72, 0x5e, 0x4d, 0xf3, 0x4a, 0x58, 0x66, 0xea, 0xa9, 0x08, 0x15, 0xc8, + 0x77, 0x7c, 0xc8, 0xe3, 0x38, 0x54, 0x6f, 0x78, 0xa0, 0x4d, 0x6d, 0x90, 0xf9, 0x42, 0xf6, 0xea, + 0x5e, 0x04, 0x94, 0xa5, 0x33, 0xed, 0x05, 0x4d, 0xad, 0xa0, 0xf8, 0x01, 0x5a, 0x13, 0x90, 0xd0, + 0x50, 0xe4, 0x34, 0x63, 0x68, 0x19, 0xc4, 0x07, 0xa8, 0x25, 0x2a, 0x01, 0xd6, 0xb6, 0x35, 0x07, + 0x77, 0xfb, 0x97, 0xc7, 0xab, 0x9a, 0x71, 0x32, 0xd7, 0x94, 0x25, 0x48, 0x32, 0x9a, 0xc8, 0x09, + 0x57, 0xb9, 0xe0, 0xb2, 0x49, 0x50, 0x05, 0xc6, 0xcf, 0xd1, 0x6a, 0x58, 0x70, 0xa9, 0xdd, 0xd0, + 0x72, 0x9b, 0x05, 0xb9, 0xa2, 0x89, 0xa4, 0x44, 0xc6, 0x3b, 0x68, 0xcd, 0x9c, 0xc0, 0xbc, 0x7b, + 0x45, 0x77, 0xb7, 0x0b, 0xdd, 0x47, 0xc5, 0x3a, 0x29, 0xd3, 0xb3, 0x5d, 0x7b, 0x3c, 0xf2, 0x3f, + 0xea, 0xb5, 0xe6, 0x2f, 0x8a, 0xcc, 0xae, 0xe7, 0x0a, 0xf8, 0x35, 0x5a, 0x17, 0x29, 0x53, 0x61, + 0x9c, 0x7b, 0xdf, 0x6e, 0x6a, 0x39, 0xbb, 0x20, 0x37, 0x8b, 0x07, 0x29, 0x31, 0x49, 0xa5, 0xd3, + 0xfe, 0x66, 0xa1, 0x06, 0x81, 0x20, 0x94, 0x4a, 0x4c, 0xf1, 0x10, 0xa1, 0xd9, 0x84, 0xec, 0x66, + 0xa8, 0xf7, 0x9a, 0x83, 0xfb, 0xa5, 0x85, 0x1b, 0xe2, 0xe5, 0x74, 0xb9, 0xcf, 0x94, 0x98, 0x92, + 0x42, 0x5b, 0xe7, 0x18, 0x6d, 0x54, 0xca, 0xb8, 0x85, 0xea, 0x27, 0x30, 0xd5, 0x69, 0x5c, 0x21, + 0xd9, 0x23, 0x7e, 0x82, 0x16, 0x3f, 0xd1, 0x28, 0x05, 0x9d, 0xbc, 0xb2, 0xab, 0xd5, 0x60, 0x13, + 0xc3, 0x7c, 0x56, 0x7b, 0x6a, 0xd9, 0xbf, 0x2c, 0xb4, 0x79, 0xcd, 0x97, 0x61, 0x1f, 0x75, 0x75, + 0x2c, 0xb5, 0x4d, 0x21, 0x0b, 0x46, 0x20, 0x86, 0xa3, 0xf7, 0x43, 0xce, 0xbc, 0x54, 0x08, 0x60, + 0x9e, 0xd1, 0x6f, 0x0e, 0xee, 0xf5, 0xcd, 0x25, 0xdc, 0xcf, 0x2f, 0xe1, 0xfe, 0x1e, 0x4f, 0xdd, + 0x08, 0x3e, 0x64, 0x2a, 0xe4, 0x2f, 0x33, 0x32, 0x15, 0x7d, 0x4a, 0xae, 0x57, 0xa9, 0xdd, 0x46, + 0xe5, 0xe6, 0x19, 0x2f, 0x5a, 0xe7, 0x17, 0x5d, 0xeb, 0xfb, 0x45, 0xd7, 0xfa, 0x71, 0xd1, 0xb5, + 0xbe, 0xfe, 0xec, 0xfe, 0xe7, 0x2e, 0xe9, 0x39, 0xdb, 0xbf, 0x03, 0x00, 0x00, 0xff, 0xff, 0x18, + 0x2e, 0x5a, 0xd3, 0x96, 0x06, 0x00, 0x00, } diff --git a/src/dbnode/generated/proto/namespace/namespace.proto b/src/dbnode/generated/proto/namespace/namespace.proto index 5839b7bb7c..17e82a57dd 100644 --- a/src/dbnode/generated/proto/namespace/namespace.proto +++ b/src/dbnode/generated/proto/namespace/namespace.proto @@ -1,6 +1,8 @@ syntax = "proto3"; + package namespace; +import "google/protobuf/wrappers.proto"; import "github.com/m3db/m3/src/dbnode/generated/proto/namespace/schema.proto"; message RetentionOptions { @@ -19,18 +21,24 @@ message IndexOptions { } message NamespaceOptions { - bool bootstrapEnabled = 1; - bool flushEnabled = 2; - bool writesToCommitLog = 3; - bool cleanupEnabled = 4; - bool repairEnabled = 5; - RetentionOptions retentionOptions = 6; - bool snapshotEnabled = 7; - IndexOptions indexOptions = 8; - SchemaOptions schemaOptions = 9; - bool coldWritesEnabled = 10; + bool bootstrapEnabled = 1; + bool flushEnabled = 2; + bool writesToCommitLog = 3; + bool cleanupEnabled = 4; + bool repairEnabled = 5; + RetentionOptions retentionOptions = 6; + bool snapshotEnabled = 7; + IndexOptions indexOptions = 8; + SchemaOptions schemaOptions = 9; + bool coldWritesEnabled = 10; + NamespaceRuntimeOptions runtimeOptions = 11; } message Registry { map namespaces = 1; } + +message NamespaceRuntimeOptions { + google.protobuf.DoubleValue writeIndexingPerCPUConcurrency = 1; + google.protobuf.DoubleValue flushIndexingPerCPUConcurrency = 2; +} diff --git a/src/dbnode/namespace/convert.go b/src/dbnode/namespace/convert.go index b8a1874f81..7c8dee3a72 100644 --- a/src/dbnode/namespace/convert.go +++ b/src/dbnode/namespace/convert.go @@ -28,6 +28,8 @@ import ( "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" + + protobuftypes "github.com/gogo/protobuf/types" ) var ( @@ -80,6 +82,25 @@ func ToIndexOptions( return iopts, nil } +// ToRuntimeOptions converts nsproto.NamespaceRuntimeOptions to RuntimeOptions. +func ToRuntimeOptions( + opts *nsproto.NamespaceRuntimeOptions, +) (RuntimeOptions, error) { + runtimeOpts := NewRuntimeOptions() + if opts == nil { + return runtimeOpts, nil + } + if v := opts.WriteIndexingPerCPUConcurrency; v != nil { + newValue := v.Value + runtimeOpts = runtimeOpts.SetWriteIndexingPerCPUConcurrency(&newValue) + } + if v := opts.FlushIndexingPerCPUConcurrency; v != nil { + newValue := v.Value + runtimeOpts = runtimeOpts.SetFlushIndexingPerCPUConcurrency(&newValue) + } + return runtimeOpts, nil +} + // ToMetadata converts nsproto.Options to Metadata func ToMetadata( id string, @@ -104,6 +125,11 @@ func ToMetadata( return nil, err } + runtimeOpts, err := ToRuntimeOptions(opts.RuntimeOptions) + if err != nil { + return nil, err + } + mopts := NewOptions(). SetBootstrapEnabled(opts.BootstrapEnabled). SetFlushEnabled(opts.FlushEnabled). @@ -114,7 +140,12 @@ func ToMetadata( SetSchemaHistory(sr). SetRetentionOptions(ropts). SetIndexOptions(iopts). - SetColdWritesEnabled(opts.ColdWritesEnabled) + SetColdWritesEnabled(opts.ColdWritesEnabled). + SetRuntimeOptions(runtimeOpts) + + if err := mopts.Validate(); err != nil { + return nil, err + } return NewMetadata(ident.StringID(id), mopts) } @@ -172,5 +203,31 @@ func OptionsToProto(opts Options) *nsproto.NamespaceOptions { BlockSizeNanos: iopts.BlockSize().Nanoseconds(), }, ColdWritesEnabled: opts.ColdWritesEnabled(), + RuntimeOptions: toRuntimeOptions(opts.RuntimeOptions()), + } +} + +// toRuntimeOptions returns the corresponding RuntimeOptions proto. +func toRuntimeOptions(opts RuntimeOptions) *nsproto.NamespaceRuntimeOptions { + if opts == nil || opts.IsDefaults() { + return nil + } + var ( + writeIndexingPerCPUConcurrency *protobuftypes.DoubleValue + flushIndexingPerCPUConcurrency *protobuftypes.DoubleValue + ) + if v := opts.WriteIndexingPerCPUConcurrency(); v != nil { + writeIndexingPerCPUConcurrency = &protobuftypes.DoubleValue{ + Value: *v, + } + } + if v := opts.FlushIndexingPerCPUConcurrency(); v != nil { + flushIndexingPerCPUConcurrency = &protobuftypes.DoubleValue{ + Value: *v, + } + } + return &nsproto.NamespaceRuntimeOptions{ + WriteIndexingPerCPUConcurrency: writeIndexingPerCPUConcurrency, + FlushIndexingPerCPUConcurrency: flushIndexingPerCPUConcurrency, } } diff --git a/src/dbnode/namespace/namespace_mock.go b/src/dbnode/namespace/namespace_mock.go index 158c448417..e8d41e1dce 100644 --- a/src/dbnode/namespace/namespace_mock.go +++ b/src/dbnode/namespace/namespace_mock.go @@ -368,6 +368,34 @@ func (mr *MockOptionsMockRecorder) SchemaHistory() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SchemaHistory", reflect.TypeOf((*MockOptions)(nil).SchemaHistory)) } +// SetRuntimeOptions mocks base method +func (m *MockOptions) SetRuntimeOptions(value RuntimeOptions) Options { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SetRuntimeOptions", value) + ret0, _ := ret[0].(Options) + return ret0 +} + +// SetRuntimeOptions indicates an expected call of SetRuntimeOptions +func (mr *MockOptionsMockRecorder) SetRuntimeOptions(value interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetRuntimeOptions", reflect.TypeOf((*MockOptions)(nil).SetRuntimeOptions), value) +} + +// RuntimeOptions mocks base method +func (m *MockOptions) RuntimeOptions() RuntimeOptions { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RuntimeOptions") + ret0, _ := ret[0].(RuntimeOptions) + return ret0 +} + +// RuntimeOptions indicates an expected call of RuntimeOptions +func (mr *MockOptionsMockRecorder) RuntimeOptions() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RuntimeOptions", reflect.TypeOf((*MockOptions)(nil).RuntimeOptions)) +} + // MockIndexOptions is a mock of IndexOptions interface type MockIndexOptions struct { ctrl *gomock.Controller @@ -709,18 +737,18 @@ func (mr *MockSchemaRegistryMockRecorder) GetLatestSchema(id interface{}) *gomoc } // GetSchema mocks base method -func (m *MockSchemaRegistry) GetSchema(id ident.ID, schemaId string) (SchemaDescr, error) { +func (m *MockSchemaRegistry) GetSchema(id ident.ID, schemaID string) (SchemaDescr, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "GetSchema", id, schemaId) + ret := m.ctrl.Call(m, "GetSchema", id, schemaID) ret0, _ := ret[0].(SchemaDescr) ret1, _ := ret[1].(error) return ret0, ret1 } // GetSchema indicates an expected call of GetSchema -func (mr *MockSchemaRegistryMockRecorder) GetSchema(id, schemaId interface{}) *gomock.Call { +func (mr *MockSchemaRegistryMockRecorder) GetSchema(id, schemaID interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSchema", reflect.TypeOf((*MockSchemaRegistry)(nil).GetSchema), id, schemaId) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSchema", reflect.TypeOf((*MockSchemaRegistry)(nil).GetSchema), id, schemaID) } // SetSchemaHistory mocks base method diff --git a/src/dbnode/namespace/namespace_runtime_options.go b/src/dbnode/namespace/namespace_runtime_options.go new file mode 100644 index 0000000000..3952c13e75 --- /dev/null +++ b/src/dbnode/namespace/namespace_runtime_options.go @@ -0,0 +1,252 @@ +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package namespace + +import ( + "sync" + + xclose "github.com/m3db/m3/src/x/close" + "github.com/m3db/m3/src/x/watch" +) + +const ( + defaultWriteIndexingPerCPUConcurrency = 0.75 + defaultFlushIndexingPerCPUConcurrency = 0.25 +) + +// RuntimeOptions is a set of runtime options that can +// be set per namespace. +type RuntimeOptions interface { + // IsDefaults returns whether the runtime options are purely defaults + // with no values explicitly set. + IsDefaults() bool + + // Equal will return whether it's equal to another runtime options. + Equal(other RuntimeOptions) bool + + // SetWriteIndexingPerCPUConcurrency sets the write + // indexing per CPU concurrency. + SetWriteIndexingPerCPUConcurrency(value *float64) RuntimeOptions + + // WriteIndexingPerCPUConcurrency returns the write + // indexing per CPU concurrency. + WriteIndexingPerCPUConcurrency() *float64 + + // WriteIndexingPerCPUConcurrencyOrDefault returns the write + // indexing per CPU concurrency or default. + WriteIndexingPerCPUConcurrencyOrDefault() float64 + + // SetFlushIndexingPerCPUConcurrency sets the flush + // indexing per CPU concurrency. + SetFlushIndexingPerCPUConcurrency(value *float64) RuntimeOptions + + // FlushIndexingPerCPUConcurrency returns the flush + // indexing per CPU concurrency. + FlushIndexingPerCPUConcurrency() *float64 + + // FlushIndexingPerCPUConcurrencyOrDefault returns the flush + // indexing per CPU concurrency. + FlushIndexingPerCPUConcurrencyOrDefault() float64 +} + +// RuntimeOptionsManagerRegistry is a registry of runtime options managers. +type RuntimeOptionsManagerRegistry interface { + // RuntimeOptionsManager returns a namespace runtime options manager + // for the given namespace. + RuntimeOptionsManager(namespace string) RuntimeOptionsManager + + // Close closes the watcher and all descendent watches. + Close() +} + +// RuntimeOptionsManager is a runtime options manager. +type RuntimeOptionsManager interface { + // Update updates the current runtime options. + Update(value RuntimeOptions) error + + // Get returns the current values. + Get() RuntimeOptions + + // RegisterListener registers a listener for updates to runtime options, + // it will synchronously call back the listener when this method is called + // to deliver the current set of runtime options. + RegisterListener(l RuntimeOptionsListener) xclose.SimpleCloser + + // Close closes the watcher and all descendent watches. + Close() +} + +// RuntimeOptionsListener listens for updates to runtime options. +type RuntimeOptionsListener interface { + // SetNamespaceRuntimeOptions is called when the listener is registered + // and when any updates occurred passing the new runtime options. + SetNamespaceRuntimeOptions(value RuntimeOptions) +} + +// runtimeOptions should always use pointer value types for it's options +// and provide a "ValueOrDefault()" method so that we can be sure whether +// the options are all defaults or not with the "AllDefaults" method. +type runtimeOptions struct { + writeIndexingPerCPUConcurrency *float64 + flushIndexingPerCPUConcurrency *float64 +} + +// NewRuntimeOptions returns a new namespace runtime options. +func NewRuntimeOptions() RuntimeOptions { + return newRuntimeOptions() +} + +func newRuntimeOptions() *runtimeOptions { + return &runtimeOptions{} +} + +func (o *runtimeOptions) IsDefaults() bool { + defaults := newRuntimeOptions() + return *o == *defaults +} + +func (o *runtimeOptions) Equal(other RuntimeOptions) bool { + return o.writeIndexingPerCPUConcurrency == other.WriteIndexingPerCPUConcurrency() && + o.flushIndexingPerCPUConcurrency == other.FlushIndexingPerCPUConcurrency() +} + +func (o *runtimeOptions) SetWriteIndexingPerCPUConcurrency(value *float64) RuntimeOptions { + opts := *o + opts.writeIndexingPerCPUConcurrency = value + return &opts +} + +func (o *runtimeOptions) WriteIndexingPerCPUConcurrency() *float64 { + return o.writeIndexingPerCPUConcurrency +} + +func (o *runtimeOptions) WriteIndexingPerCPUConcurrencyOrDefault() float64 { + value := o.writeIndexingPerCPUConcurrency + if value == nil { + return defaultWriteIndexingPerCPUConcurrency + } + return *value +} + +func (o *runtimeOptions) SetFlushIndexingPerCPUConcurrency(value *float64) RuntimeOptions { + opts := *o + opts.flushIndexingPerCPUConcurrency = value + return &opts +} + +func (o *runtimeOptions) FlushIndexingPerCPUConcurrency() *float64 { + return o.flushIndexingPerCPUConcurrency +} + +func (o *runtimeOptions) FlushIndexingPerCPUConcurrencyOrDefault() float64 { + value := o.flushIndexingPerCPUConcurrency + if value == nil { + return defaultFlushIndexingPerCPUConcurrency + } + return *value +} + +type runtimeOptionsManagerRegistry struct { + sync.RWMutex + managers map[string]RuntimeOptionsManager +} + +func NewRuntimeOptionsManagerRegistry() RuntimeOptionsManagerRegistry { + return &runtimeOptionsManagerRegistry{ + managers: make(map[string]RuntimeOptionsManager), + } +} + +func (r *runtimeOptionsManagerRegistry) RuntimeOptionsManager( + namespace string, +) RuntimeOptionsManager { + r.Lock() + defer r.Unlock() + manager, ok := r.managers[namespace] + if ok { + return manager + } + + manager = NewRuntimeOptionsManager(namespace) + r.managers[namespace] = manager + + return manager +} + +func (r *runtimeOptionsManagerRegistry) Close() { + r.Lock() + defer r.Unlock() + + for k, v := range r.managers { + v.Close() + delete(r.managers, k) + } +} + +type runtimeOptionsManager struct { + namespace string + watchable watch.Watchable +} + +func NewRuntimeOptionsManager(namespace string) RuntimeOptionsManager { + watchable := watch.NewWatchable() + watchable.Update(NewRuntimeOptions()) + return &runtimeOptionsManager{ + namespace: namespace, + watchable: watchable, + } +} + +func (w *runtimeOptionsManager) Update(value RuntimeOptions) error { + w.watchable.Update(value) + return nil +} + +func (w *runtimeOptionsManager) Get() RuntimeOptions { + return w.watchable.Get().(RuntimeOptions) +} + +func (w *runtimeOptionsManager) RegisterListener( + listener RuntimeOptionsListener, +) xclose.SimpleCloser { + _, watch, _ := w.watchable.Watch() + + // We always initialize the watchable so always read + // the first notification value + <-watch.C() + + // Deliver the current runtime options + listener.SetNamespaceRuntimeOptions(watch.Get().(RuntimeOptions)) + + // Spawn a new goroutine that will terminate when the + // watchable terminates on the close of the runtime options manager + go func() { + for range watch.C() { + listener.SetNamespaceRuntimeOptions(watch.Get().(RuntimeOptions)) + } + }() + + return watch +} + +func (w *runtimeOptionsManager) Close() { + w.watchable.Close() +} diff --git a/src/dbnode/namespace/options.go b/src/dbnode/namespace/options.go index 6f146bc88c..f8cc25f312 100644 --- a/src/dbnode/namespace/options.go +++ b/src/dbnode/namespace/options.go @@ -53,6 +53,7 @@ var ( errIndexBlockSizePositive = errors.New("index block size must positive") errIndexBlockSizeTooLarge = errors.New("index block size needs to be <= namespace retention period") errIndexBlockSizeMustBeAMultipleOfDataBlockSize = errors.New("index block size must be a multiple of data block size") + errNamespaceRuntimeOptionsNotSet = errors.New("namespace runtime options is not set") ) type options struct { @@ -66,6 +67,7 @@ type options struct { retentionOpts retention.Options indexOpts IndexOptions schemaHis SchemaHistory + runtimeOpts RuntimeOptions } // NewSchemaHistory returns an empty schema history. @@ -86,6 +88,7 @@ func NewOptions() Options { retentionOpts: retention.NewOptions(), indexOpts: NewIndexOptions(), schemaHis: NewSchemaHistory(), + runtimeOpts: NewRuntimeOptions(), } } @@ -111,6 +114,9 @@ func (o *options) Validate() error { if indexBlockSize%dataBlockSize != 0 { return errIndexBlockSizeMustBeAMultipleOfDataBlockSize } + if o.runtimeOpts == nil { + return errNamespaceRuntimeOptionsNotSet + } return nil } @@ -124,7 +130,8 @@ func (o *options) Equal(value Options) bool { o.coldWritesEnabled == value.ColdWritesEnabled() && o.retentionOpts.Equal(value.RetentionOptions()) && o.indexOpts.Equal(value.IndexOptions()) && - o.schemaHis.Equal(value.SchemaHistory()) + o.schemaHis.Equal(value.SchemaHistory()) && + o.runtimeOpts.Equal(value.RuntimeOptions()) } func (o *options) SetBootstrapEnabled(value bool) Options { @@ -226,3 +233,13 @@ func (o *options) SetSchemaHistory(value SchemaHistory) Options { func (o *options) SchemaHistory() SchemaHistory { return o.schemaHis } + +func (o *options) SetRuntimeOptions(value RuntimeOptions) Options { + opts := *o + opts.runtimeOpts = value + return &opts +} + +func (o *options) RuntimeOptions() RuntimeOptions { + return o.runtimeOpts +} diff --git a/src/dbnode/namespace/types.go b/src/dbnode/namespace/types.go index ef51b4cbd8..acd61360fe 100644 --- a/src/dbnode/namespace/types.go +++ b/src/dbnode/namespace/types.go @@ -97,6 +97,12 @@ type Options interface { // SchemaHistory returns the schema registry for this namespace. SchemaHistory() SchemaHistory + + // SetRuntimeOptions sets the RuntimeOptions. + SetRuntimeOptions(value RuntimeOptions) Options + + // RuntimeOptions returns the RuntimeOptions. + RuntimeOptions() RuntimeOptions } // IndexOptions controls the indexing options for a namespace. @@ -163,7 +169,7 @@ type SchemaRegistry interface { // GetSchema gets the latest schema for the namespace. // If proto is not enabled, nil, nil is returned - GetSchema(id ident.ID, schemaId string) (SchemaDescr, error) + GetSchema(id ident.ID, schemaID string) (SchemaDescr, error) // SetSchemaHistory sets the schema history for the namespace. // If proto is not enabled, nil is returned @@ -277,4 +283,5 @@ type NamespaceWatch interface { Close() error } +// NamespaceUpdater is a namespace updater function. type NamespaceUpdater func(Map) error diff --git a/src/dbnode/storage/database.go b/src/dbnode/storage/database.go index 3754b85d27..467e313fb1 100644 --- a/src/dbnode/storage/database.go +++ b/src/dbnode/storage/database.go @@ -94,8 +94,9 @@ type db struct { opts Options nowFn clock.NowFn - nsWatch namespace.NamespaceWatch - namespaces *databaseNamespacesMap + nsWatch namespace.NamespaceWatch + namespaces *databaseNamespacesMap + runtimeOptionsRegistry namespace.RuntimeOptionsManagerRegistry commitLog commitlog.CommitLog @@ -172,16 +173,17 @@ func NewDatabase( ) d := &db{ - opts: opts, - nowFn: nowFn, - shardSet: shardSet, - lastReceivedNewShards: nowFn(), - namespaces: newDatabaseNamespacesMap(databaseNamespacesMapOptions{}), - commitLog: commitLog, - scope: scope, - metrics: newDatabaseMetrics(scope), - log: logger, - writeBatchPool: opts.WriteBatchPool(), + opts: opts, + nowFn: nowFn, + shardSet: shardSet, + lastReceivedNewShards: nowFn(), + namespaces: newDatabaseNamespacesMap(databaseNamespacesMapOptions{}), + runtimeOptionsRegistry: namespace.NewRuntimeOptionsManagerRegistry(), + commitLog: commitLog, + scope: scope, + metrics: newDatabaseMetrics(scope), + log: logger, + writeBatchPool: opts.WriteBatchPool(), } databaseIOpts := iopts.SetMetricsScope(scope) @@ -375,7 +377,10 @@ func (d *db) newDatabaseNamespaceWithLock( return nil, err } } - return newDatabaseNamespace(md, d.shardSet, retriever, d, d.commitLog, d.opts) + nsID := md.ID().String() + runtimeOptsMgr := d.runtimeOptionsRegistry.RuntimeOptionsManager(nsID) + return newDatabaseNamespace(md, runtimeOptsMgr, + d.shardSet, retriever, d, d.commitLog, d.opts) } func (d *db) Options() Options { diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index e3a3b0e4b8..e39ad0df46 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -102,11 +102,12 @@ type nsIndex struct { bufferFuture time.Duration coldWritesEnabled bool - indexFilesetsBeforeFn indexFilesetsBeforeFn - deleteFilesFn deleteFilesFn - readIndexInfoFilesFn readIndexInfoFilesFn + namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager + indexFilesetsBeforeFn indexFilesetsBeforeFn + deleteFilesFn deleteFilesFn + readIndexInfoFilesFn readIndexInfoFilesFn - newBlockFn newBlockFn + newBlockFn index.NewBlockFn logger *zap.Logger opts Options nsMetadata namespace.Metadata @@ -175,13 +176,6 @@ type nsIndexRuntimeOptions struct { defaultQueryTimeout time.Duration } -type newBlockFn func( - time.Time, - namespace.Metadata, - index.BlockOptions, - index.Options, -) (index.Block, error) - // NB(prateek): the returned filesets are strictly before the given time, i.e. they // live in the period (-infinity, exclusiveTime). type indexFilesetsBeforeFn func(dir string, @@ -195,11 +189,12 @@ type readIndexInfoFilesFn func(filePathPrefix string, ) []fs.ReadIndexInfoFileResult type newNamespaceIndexOpts struct { - md namespace.Metadata - shardSet sharding.ShardSet - opts Options - newIndexQueueFn newNamespaceIndexInsertQueueFn - newBlockFn newBlockFn + md namespace.Metadata + namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager + shardSet sharding.ShardSet + opts Options + newIndexQueueFn newNamespaceIndexInsertQueueFn + newBlockFn index.NewBlockFn } // execBlockQueryFn executes a query against the given block whilst tracking state. @@ -224,47 +219,53 @@ type asyncQueryExecState struct { // newNamespaceIndex returns a new namespaceIndex for the provided namespace. func newNamespaceIndex( nsMD namespace.Metadata, + namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager, shardSet sharding.ShardSet, opts Options, ) (NamespaceIndex, error) { return newNamespaceIndexWithOptions(newNamespaceIndexOpts{ - md: nsMD, - shardSet: shardSet, - opts: opts, - newIndexQueueFn: newNamespaceIndexInsertQueue, - newBlockFn: index.NewBlock, + md: nsMD, + namespaceRuntimeOptsMgr: namespaceRuntimeOptsMgr, + shardSet: shardSet, + opts: opts, + newIndexQueueFn: newNamespaceIndexInsertQueue, + newBlockFn: index.NewBlock, }) } // newNamespaceIndexWithInsertQueueFn is a ctor used in tests to override the insert queue. func newNamespaceIndexWithInsertQueueFn( nsMD namespace.Metadata, + namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager, shardSet sharding.ShardSet, newIndexQueueFn newNamespaceIndexInsertQueueFn, opts Options, ) (NamespaceIndex, error) { return newNamespaceIndexWithOptions(newNamespaceIndexOpts{ - md: nsMD, - shardSet: shardSet, - opts: opts, - newIndexQueueFn: newIndexQueueFn, - newBlockFn: index.NewBlock, + md: nsMD, + namespaceRuntimeOptsMgr: namespaceRuntimeOptsMgr, + shardSet: shardSet, + opts: opts, + newIndexQueueFn: newIndexQueueFn, + newBlockFn: index.NewBlock, }) } // newNamespaceIndexWithNewBlockFn is a ctor used in tests to inject blocks. func newNamespaceIndexWithNewBlockFn( nsMD namespace.Metadata, + namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager, shardSet sharding.ShardSet, - newBlockFn newBlockFn, + newBlockFn index.NewBlockFn, opts Options, ) (NamespaceIndex, error) { return newNamespaceIndexWithOptions(newNamespaceIndexOpts{ - md: nsMD, - shardSet: shardSet, - opts: opts, - newIndexQueueFn: newNamespaceIndexInsertQueue, - newBlockFn: newBlockFn, + md: nsMD, + namespaceRuntimeOptsMgr: namespaceRuntimeOptsMgr, + shardSet: shardSet, + opts: opts, + newIndexQueueFn: newNamespaceIndexInsertQueue, + newBlockFn: newBlockFn, }) } @@ -324,9 +325,10 @@ func newNamespaceIndexWithOptions( bufferFuture: nsMD.Options().RetentionOptions().BufferFuture(), coldWritesEnabled: nsMD.Options().ColdWritesEnabled(), - indexFilesetsBeforeFn: fs.IndexFileSetsBefore, - readIndexInfoFilesFn: fs.ReadIndexInfoFiles, - deleteFilesFn: fs.DeleteFiles, + namespaceRuntimeOptsMgr: newIndexOpts.namespaceRuntimeOptsMgr, + indexFilesetsBeforeFn: fs.IndexFileSetsBefore, + readIndexInfoFilesFn: fs.ReadIndexInfoFiles, + deleteFilesFn: fs.DeleteFiles, newBlockFn: newBlockFn, opts: newIndexOpts.opts, @@ -435,6 +437,61 @@ func (i *nsIndex) reportStats() error { flushedLevels := i.metrics.blockMetrics.FlushedSegments.Levels flushedLevelStats := make([]nsIndexCompactionLevelStats, len(flushedLevels)) + minIndexConcurrency := 0 + maxIndexConcurrency := 0 + sumIndexConcurrency := 0 + numIndexingStats := 0 + reporter := index.NewBlockStatsReporter( + func(s index.BlockSegmentStats) { + var ( + levels []nsIndexBlocksSegmentsLevelMetrics + levelStats []nsIndexCompactionLevelStats + ) + switch s.Type { + case index.ActiveForegroundSegment: + levels = foregroundLevels + levelStats = foregroundLevelStats + case index.ActiveBackgroundSegment: + levels = backgroundLevels + levelStats = backgroundLevelStats + case index.FlushedSegment: + levels = flushedLevels + levelStats = flushedLevelStats + } + + for i, l := range levels { + contained := s.Size >= l.MinSizeInclusive && s.Size < l.MaxSizeExclusive + if !contained { + continue + } + + l.SegmentsAge.Record(s.Age) + levelStats[i].numSegments++ + levelStats[i].numTotalDocs += s.Size + + break + } + }, + func(s index.BlockIndexingStats) { + first := numIndexingStats == 0 + numIndexingStats++ + + if first { + minIndexConcurrency = s.IndexConcurrency + maxIndexConcurrency = s.IndexConcurrency + sumIndexConcurrency = s.IndexConcurrency + return + } + + if v := s.IndexConcurrency; v < minIndexConcurrency { + minIndexConcurrency = v + } + if v := s.IndexConcurrency; v > maxIndexConcurrency { + maxIndexConcurrency = v + } + sumIndexConcurrency += s.IndexConcurrency + }) + // iterate known blocks in a defined order of time (newest first) // for debug log ordering for _, start := range i.state.blockStartsDescOrder { @@ -443,37 +500,7 @@ func (i *nsIndex) reportStats() error { return i.missingBlockInvariantError(start) } - err := block.Stats( - index.BlockStatsReporterFn(func(s index.BlockSegmentStats) { - var ( - levels []nsIndexBlocksSegmentsLevelMetrics - levelStats []nsIndexCompactionLevelStats - ) - switch s.Type { - case index.ActiveForegroundSegment: - levels = foregroundLevels - levelStats = foregroundLevelStats - case index.ActiveBackgroundSegment: - levels = backgroundLevels - levelStats = backgroundLevelStats - case index.FlushedSegment: - levels = flushedLevels - levelStats = flushedLevelStats - } - - for i, l := range levels { - contained := s.Size >= l.MinSizeInclusive && s.Size < l.MaxSizeExclusive - if !contained { - continue - } - - l.SegmentsAge.Record(s.Age) - levelStats[i].numSegments++ - levelStats[i].numTotalDocs += s.Size - - break - } - })) + err := block.Stats(reporter) if err == index.ErrUnableReportStatsBlockClosed { // Closed blocks are temporarily in the list still continue @@ -483,6 +510,7 @@ func (i *nsIndex) reportStats() error { } } + // Update level stats. for _, elem := range []struct { levels []nsIndexBlocksSegmentsLevelMetrics levelStats []nsIndexCompactionLevelStats @@ -496,6 +524,12 @@ func (i *nsIndex) reportStats() error { } } + // Update the indexing stats. + i.metrics.indexingConcurrencyMin.Update(float64(minIndexConcurrency)) + i.metrics.indexingConcurrencyMax.Update(float64(maxIndexConcurrency)) + avgIndexConcurrency := float64(sumIndexConcurrency) / float64(numIndexingStats) + i.metrics.indexingConcurrencyAvg.Update(avgIndexConcurrency) + return nil } @@ -1676,7 +1710,7 @@ func (i *nsIndex) ensureBlockPresentWithRLock(blockStart time.Time) (index.Block // ok now we know for sure we have to alloc block, err := i.newBlockFn(blockStart, i.nsMetadata, - index.BlockOptions{}, i.opts.IndexOptions()) + index.BlockOptions{}, i.namespaceRuntimeOptsMgr, i.opts.IndexOptions()) if err != nil { // unable to allocate the block, should never happen. return nil, i.unableToAllocBlockInvariantError(err) } @@ -1961,6 +1995,9 @@ type nsIndexMetrics struct { insertEndToEndLatency tally.Timer blocksEvictedMutableSegments tally.Counter blockMetrics nsIndexBlocksMetrics + indexingConcurrencyMin tally.Gauge + indexingConcurrencyMax tally.Gauge + indexingConcurrencyAvg tally.Gauge loadedDocsPerQuery tally.Histogram queryExhaustiveSuccess tally.Counter @@ -1977,8 +2014,9 @@ func newNamespaceIndexMetrics( iopts instrument.Options, ) nsIndexMetrics { const ( - indexAttemptName = "index-attempt" - forwardIndexName = "forward-index" + indexAttemptName = "index-attempt" + forwardIndexName = "forward-index" + indexingConcurrency = "indexing-concurrency" ) scope := iopts.MetricsScope() blocksScope := scope.SubScope("blocks") @@ -2015,6 +2053,15 @@ func newNamespaceIndexMetrics( "insert-end-to-end-latency", iopts.TimerOptions()), blocksEvictedMutableSegments: scope.Counter("blocks-evicted-mutable-segments"), blockMetrics: newNamespaceIndexBlocksMetrics(opts, blocksScope), + indexingConcurrencyMin: scope.Tagged(map[string]string{ + "stat": "min", + }).Gauge(indexingConcurrency), + indexingConcurrencyMax: scope.Tagged(map[string]string{ + "stat": "max", + }).Gauge(indexingConcurrency), + indexingConcurrencyAvg: scope.Tagged(map[string]string{ + "stat": "avg", + }).Gauge(indexingConcurrency), loadedDocsPerQuery: scope.Histogram( "loaded-docs-per-query", tally.MustMakeExponentialValueBuckets(10, 2, 16), diff --git a/src/dbnode/storage/index/block.go b/src/dbnode/storage/index/block.go index 1c1a7cd603..70e40f3d0c 100644 --- a/src/dbnode/storage/index/block.go +++ b/src/dbnode/storage/index/block.go @@ -139,6 +139,7 @@ type block struct { iopts instrument.Options blockOpts BlockOptions nsMD namespace.Metadata + namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager queryStats stats.QueryStats metrics blockMetrics @@ -190,12 +191,25 @@ type BlockOptions struct { BackgroundCompactorMmapDocsData bool } +// NewBlockFn is a new block constructor. +type NewBlockFn func( + blockStart time.Time, + md namespace.Metadata, + blockOpts BlockOptions, + namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager, + opts Options, +) (Block, error) + +// Ensure NewBlock implements NewBlockFn. +var _ NewBlockFn = NewBlock + // NewBlock returns a new Block, representing a complete reverse index for the // duration of time specified. It is backed by one or more segments. func NewBlock( blockStart time.Time, md namespace.Metadata, blockOpts BlockOptions, + namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager, opts Options, ) (Block, error) { blockSize := md.Options().IndexOptions().BlockSize() @@ -206,6 +220,7 @@ func NewBlock( blockStart, opts, blockOpts, + namespaceRuntimeOptsMgr, iopts, ) // NB(bodu): The length of coldMutableSegments is always at least 1. @@ -214,6 +229,7 @@ func NewBlock( blockStart, opts, blockOpts, + namespaceRuntimeOptsMgr, iopts, ), } @@ -229,6 +245,7 @@ func NewBlock( opts: opts, iopts: iopts, nsMD: md, + namespaceRuntimeOptsMgr: namespaceRuntimeOptsMgr, metrics: newBlockMetrics(scope), logger: iopts.Logger(), queryStats: opts.QueryStats(), @@ -1066,6 +1083,7 @@ func (b *block) RotateColdMutableSegments() { b.blockStart, b.opts, b.blockOpts, + b.namespaceRuntimeOptsMgr, b.iopts, )) } diff --git a/src/dbnode/storage/index/block_prop_test.go b/src/dbnode/storage/index/block_prop_test.go index 740ac0da45..b676eab1ef 100644 --- a/src/dbnode/storage/index/block_prop_test.go +++ b/src/dbnode/storage/index/block_prop_test.go @@ -178,7 +178,8 @@ func TestPostingsListCacheDoesNotAffectBlockQueryResults(t *testing.T) { } func newPropTestBlock(t *testing.T, blockStart time.Time, nsMeta namespace.Metadata, opts Options) (Block, error) { - blk, err := NewBlock(blockStart, nsMeta, BlockOptions{}, opts) + blockOpts := BlockOptions{WriteIndexingPerCPUConurrency: 1} + blk, err := NewBlock(blockStart, nsMeta, blockOpts, opts) require.NoError(t, err) var ( diff --git a/src/dbnode/storage/index/block_test.go b/src/dbnode/storage/index/block_test.go index 53cb68a0f0..1bebfd4949 100644 --- a/src/dbnode/storage/index/block_test.go +++ b/src/dbnode/storage/index/block_test.go @@ -1303,6 +1303,7 @@ func TestBlockE2EInsertQuery(t *testing.T) { blk, err := NewBlock(blockStart, testMD, BlockOptions{ + WriteIndexingPerCPUConcurrency: 1, ForegroundCompactorMmapDocsData: true, BackgroundCompactorMmapDocsData: true, }, testOpts) @@ -1914,6 +1915,7 @@ func TestBlockE2EInsertAggregate(t *testing.T) { blk, err := NewBlock(blockStart, testMD, BlockOptions{ + WriteIndexingPerCPUConcurrency: 1, ForegroundCompactorMmapDocsData: true, BackgroundCompactorMmapDocsData: true, }, testOpts) diff --git a/src/dbnode/storage/index/mutable_segments.go b/src/dbnode/storage/index/mutable_segments.go index 5d01ee939a..9c1664be01 100644 --- a/src/dbnode/storage/index/mutable_segments.go +++ b/src/dbnode/storage/index/mutable_segments.go @@ -23,15 +23,19 @@ package index import ( "errors" "fmt" + "math" + "runtime" "sync" "time" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/storage/index/compaction" "github.com/m3db/m3/src/dbnode/storage/index/segments" m3ninxindex "github.com/m3db/m3/src/m3ninx/index" "github.com/m3db/m3/src/m3ninx/index/segment" "github.com/m3db/m3/src/m3ninx/index/segment/builder" "github.com/m3db/m3/src/m3ninx/index/segment/fst" + xclose "github.com/m3db/m3/src/x/close" "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/instrument" "github.com/m3db/m3/src/x/mmap" @@ -64,11 +68,13 @@ type mutableSegments struct { foregroundSegments []*readableSeg backgroundSegments []*readableSeg - compact mutableSegmentsCompact - blockStart time.Time - blockOpts BlockOptions - opts Options - iopts instrument.Options + compact mutableSegmentsCompact + blockStart time.Time + blockOpts BlockOptions + opts Options + iopts instrument.Options + optsListener xclose.SimpleCloser + writeIndexingConcurrency int metrics mutableSegmentsMetrics logger *zap.Logger @@ -98,6 +104,7 @@ func newMutableSegments( blockStart time.Time, opts Options, blockOpts BlockOptions, + namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager, iopts instrument.Options, ) *mutableSegments { m := &mutableSegments{ @@ -108,9 +115,27 @@ func newMutableSegments( metrics: newMutableSegmentsMetrics(iopts.MetricsScope()), logger: iopts.Logger(), } + m.optsListener = namespaceRuntimeOptsMgr.RegisterListener(m) return m } +func (m *mutableSegments) SetNamespaceRuntimeOptions(opts namespace.RuntimeOptions) { + m.Lock() + // Update current runtime opts for segment builders created in future. + perCPUFraction := opts.WriteIndexingPerCPUConcurrencyOrDefault() + cpus := math.Ceil(perCPUFraction * float64(runtime.NumCPU())) + m.writeIndexingConcurrency = int(math.Max(1, cpus)) + builder := m.compact.segmentBuilder + m.Unlock() + + // Reset any existing segment builder to new concurrency, do this + // out of the lock since builder can be used for foreground compaction + // outside the lock and does it's own locking. + if builder != nil { + builder.SetIndexConcurrency(m.writeIndexingConcurrency) + } +} + func (m *mutableSegments) WriteBatch(inserts *WriteBatch) error { m.Lock() if m.state == mutableSegmentsStateClosed { @@ -121,8 +146,10 @@ func (m *mutableSegments) WriteBatch(inserts *WriteBatch) error { m.Unlock() return errUnableToWriteBlockConcurrent } - // Lazily allocate the segment builder and compactors - err := m.compact.allocLazyBuilderAndCompactors(m.blockOpts, m.opts) + + // Lazily allocate the segment builder and compactors. + err := m.compact.allocLazyBuilderAndCompactorsWithLock(m.writeIndexingConcurrency, + m.blockOpts, m.opts) if err != nil { m.Unlock() return err @@ -243,6 +270,9 @@ func (m *mutableSegments) NeedsEviction() bool { } func (m *mutableSegments) NumSegmentsAndDocs() (int64, int64) { + m.RLock() + defer m.RUnlock() + var ( numSegments, numDocs int64 ) @@ -258,6 +288,9 @@ func (m *mutableSegments) NumSegmentsAndDocs() (int64, int64) { } func (m *mutableSegments) Stats(reporter BlockStatsReporter) { + m.RLock() + defer m.RUnlock() + for _, seg := range m.foregroundSegments { _, mutable := seg.Segment().(segment.MutableSegment) reporter.ReportSegmentStats(BlockSegmentStats{ @@ -276,6 +309,10 @@ func (m *mutableSegments) Stats(reporter BlockStatsReporter) { Size: seg.Segment().Size(), }) } + + reporter.ReportIndexingStats(BlockIndexingStats{ + IndexConcurrency: m.writeIndexingConcurrency, + }) } func (m *mutableSegments) Close() { @@ -283,6 +320,7 @@ func (m *mutableSegments) Close() { defer m.Unlock() m.state = mutableSegmentsStateClosed m.cleanupCompactWithLock() + m.optsListener.Close() } func (m *mutableSegments) maybeBackgroundCompactWithLock() { @@ -343,7 +381,7 @@ func (m *mutableSegments) cleanupBackgroundCompactWithLock() { } // Close compacted segments. - m.closeCompactedSegments(m.backgroundSegments) + m.closeCompactedSegmentsWithLock(m.backgroundSegments) m.backgroundSegments = nil // Free compactor resources. @@ -359,7 +397,7 @@ func (m *mutableSegments) cleanupBackgroundCompactWithLock() { m.compact.backgroundCompactor = nil } -func (m *mutableSegments) closeCompactedSegments(segments []*readableSeg) { +func (m *mutableSegments) closeCompactedSegmentsWithLock(segments []*readableSeg) { for _, seg := range segments { err := seg.Segment().Close() if err != nil { @@ -454,14 +492,14 @@ func (m *mutableSegments) backgroundCompactWithTask( m.Lock() defer m.Unlock() - result := m.addCompactedSegmentFromSegments(m.backgroundSegments, + result := m.addCompactedSegmentFromSegmentsWithLock(m.backgroundSegments, segments, compacted) m.backgroundSegments = result return nil } -func (m *mutableSegments) addCompactedSegmentFromSegments( +func (m *mutableSegments) addCompactedSegmentFromSegmentsWithLock( current []*readableSeg, segmentsJustCompacted []segment.Segment, compacted segment.Segment, @@ -675,7 +713,7 @@ func (m *mutableSegments) foregroundCompactWithTask( m.Lock() defer m.Unlock() - result := m.addCompactedSegmentFromSegments(m.foregroundSegments, + result := m.addCompactedSegmentFromSegmentsWithLock(m.foregroundSegments, segments, compacted) m.foregroundSegments = result @@ -690,7 +728,7 @@ func (m *mutableSegments) cleanupForegroundCompactWithLock() { } // Close compacted segments. - m.closeCompactedSegments(m.foregroundSegments) + m.closeCompactedSegmentsWithLock(m.foregroundSegments) m.foregroundSegments = nil // Free compactor resources. @@ -736,7 +774,8 @@ type mutableSegmentsCompact struct { numBackground int } -func (m *mutableSegmentsCompact) allocLazyBuilderAndCompactors( +func (m *mutableSegmentsCompact) allocLazyBuilderAndCompactorsWithLock( + concurrency int, blockOpts BlockOptions, opts Options, ) error { @@ -745,7 +784,10 @@ func (m *mutableSegmentsCompact) allocLazyBuilderAndCompactors( docsPool = opts.DocumentArrayPool() ) if m.segmentBuilder == nil { - m.segmentBuilder, err = builder.NewBuilderFromDocuments(opts.SegmentBuilderOptions()) + builderOpts := opts.SegmentBuilderOptions(). + SetConcurrency(concurrency) + + m.segmentBuilder, err = builder.NewBuilderFromDocuments(builderOpts) if err != nil { return err } diff --git a/src/dbnode/storage/index/types.go b/src/dbnode/storage/index/types.go index d32425c66e..cdf613245f 100644 --- a/src/dbnode/storage/index/types.go +++ b/src/dbnode/storage/index/types.go @@ -409,15 +409,33 @@ func (e *EvictMutableSegmentResults) Add(o EvictMutableSegmentResults) { // block and get an immutable list of segments back). type BlockStatsReporter interface { ReportSegmentStats(stats BlockSegmentStats) + ReportIndexingStats(stats BlockIndexingStats) } -// BlockStatsReporterFn implements the block stats reporter using -// a callback function. -type BlockStatsReporterFn func(stats BlockSegmentStats) +type blockStatsReporter struct { + reportSegmentStats func(stats BlockSegmentStats) + reportIndexingStats func(stats BlockIndexingStats) +} + +// NewBlockStatsReporter returns a new block stats reporter. +func NewBlockStatsReporter( + reportSegmentStats func(stats BlockSegmentStats), + reportIndexingStats func(stats BlockIndexingStats), +) BlockStatsReporter { + return blockStatsReporter{} +} + +func (r blockStatsReporter) ReportSegmentStats(stats BlockSegmentStats) { + r.reportSegmentStats(stats) +} + +func (r blockStatsReporter) ReportIndexingStats(stats BlockIndexingStats) { + r.reportIndexingStats(stats) +} -// ReportSegmentStats implements the BlockStatsReporter interface. -func (f BlockStatsReporterFn) ReportSegmentStats(stats BlockSegmentStats) { - f(stats) +// BlockIndexingStats is stats about a block's indexing stats. +type BlockIndexingStats struct { + IndexConcurrency int } // BlockSegmentStats has segment stats. diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index d2c9048e39..147edb325f 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -287,6 +287,7 @@ func newDatabaseNamespaceMetrics( func newDatabaseNamespace( metadata namespace.Metadata, + namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager, shardSet sharding.ShardSet, blockRetriever block.DatabaseBlockRetriever, increasingIndex increasingIndex, @@ -329,7 +330,8 @@ func newDatabaseNamespace( err error ) if metadata.Options().IndexOptions().Enabled() { - index, err = newNamespaceIndex(metadata, shardSet, opts) + index, err = newNamespaceIndex(metadata, namespaceRuntimeOptsMgr, + shardSet, opts) if err != nil { return nil, err } diff --git a/src/m3ninx/index/segment/builder/builder.go b/src/m3ninx/index/segment/builder/builder.go index b4af8fb744..25c64b4252 100644 --- a/src/m3ninx/index/segment/builder/builder.go +++ b/src/m3ninx/index/segment/builder/builder.go @@ -66,11 +66,13 @@ type builder struct { offset postings.ID - batchSizeOne index.Batch - docs []doc.Document - idSet *IDsMap - fields *shardedFieldsMap - uniqueFields [][]uniqueField + batchSizeOne index.Batch + docs []doc.Document + idSet *IDsMap + fields *shardedFieldsMap + uniqueFields [][]uniqueField + concurrency int + fieldsSetOpts fieldsMapSetUnsafeOptions indexQueues []chan indexJob status builderStatus @@ -80,7 +82,6 @@ type builder struct { // not thread safe and is optimized for insertion speed and a // final build step when documents are indexed. func NewBuilderFromDocuments(opts Options) (segment.CloseableDocumentsBuilder, error) { - concurrency := opts.Concurrency() b := &builder{ opts: opts, newUUIDFn: opts.NewUUIDFn(), @@ -90,26 +91,89 @@ func NewBuilderFromDocuments(opts Options) (segment.CloseableDocumentsBuilder, e idSet: NewIDsMap(IDsMapOptions{ InitialSize: opts.InitialCapacity(), }), - uniqueFields: make([][]uniqueField, 0, concurrency), - indexQueues: make([]chan indexJob, 0, concurrency), + fieldsSetOpts: fieldsMapSetUnsafeOptions{ + // Builder takes ownership of keys and docs so it's ok + // to avoid copying and finalizing keys. + NoCopyKey: true, + NoFinalizeKey: true, + }, } + b.SetIndexConcurrency(opts.Concurrency()) + return b, nil +} - for i := 0; i < concurrency; i++ { +func (b *builder) SetIndexConcurrency(value int) { + b.status.Lock() + defer b.status.Unlock() + + if b.concurrency == value { + return // No-op + } + + b.concurrency = value + + for _, indexQueue := range b.indexQueues { + close(indexQueue) + } + + // Take refs to existing fields to migrate. + existingUniqueFields := b.uniqueFields + existingFields := b.fields + + b.uniqueFields = make([][]uniqueField, 0, b.concurrency) + b.fields = newShardedFieldsMap(b.concurrency, b.opts.InitialCapacity()) + b.indexQueues = make([]chan indexJob, 0, b.concurrency) + + for i := 0; i < b.concurrency; i++ { indexQueue := make(chan indexJob, indexQueueSize) b.indexQueues = append(b.indexQueues, indexQueue) go b.indexWorker(indexQueue) // Give each shard a fraction of the configured initial capacity. - shardInitialCapacity := opts.InitialCapacity() + shardInitialCapacity := b.opts.InitialCapacity() if shardInitialCapacity > 0 { - shardInitialCapacity /= concurrency + shardInitialCapacity /= b.concurrency } + shardUniqueFields := make([]uniqueField, 0, shardInitialCapacity) b.uniqueFields = append(b.uniqueFields, shardUniqueFields) - b.fields = newShardedFieldsMap(concurrency, shardInitialCapacity) } - return b, nil + // Migrate data from existing unique fields. + if existingUniqueFields != nil { + for _, fields := range existingUniqueFields { + for _, field := range fields { + // Calculate the new shard for the field. + newShard := b.calculateShard(field.field) + + // Append to the correct shard. + b.uniqueFields[newShard] = append(b.uniqueFields[newShard], field) + } + } + } + + // Migrate from fields. + if existingFields != nil { + for _, fields := range existingFields.data { + for _, entry := range fields.Iter() { + field := entry.Key() + terms := entry.Value() + + // Calculate the new shard for the field. + newShard := b.calculateShard(field) + + // Set with new correct shard. + b.fields.ShardedSetUnsafe(newShard, field, terms, b.fieldsSetOpts) + } + } + } +} + +func (b *builder) IndexConcurrency() int { + b.status.RLock() + defer b.status.RUnlock() + + return b.concurrency } func (b *builder) Reset(offset postings.ID) { @@ -262,10 +326,8 @@ func (b *builder) indexWorker(indexQueue chan indexJob) { if !ok { // NB(bodu): Check again within the lock to make sure we aren't making concurrent map writes. terms = newTerms(b.opts) - b.fields.ShardedSetUnsafe(job.shard, job.field.Name, terms, fieldsMapSetUnsafeOptions{ - NoCopyKey: true, - NoFinalizeKey: true, - }) + b.fields.ShardedSetUnsafe(job.shard, job.field.Name, + terms, b.fieldsSetOpts) } // If empty field, track insertion of this key into the fields diff --git a/src/m3ninx/index/segment/mem/segment.go b/src/m3ninx/index/segment/mem/segment.go index bedc8f90d9..dd6c4c1fd6 100644 --- a/src/m3ninx/index/segment/mem/segment.go +++ b/src/m3ninx/index/segment/mem/segment.go @@ -84,6 +84,14 @@ func NewSegment(offset postings.ID, opts Options) (sgmt.MutableSegment, error) { return s, nil } +func (s *segment) SetIndexConcurrency(value int) { + // No-op, does not support concurrent indexing. +} + +func (s *segment) IndexConcurrency() int { + return 1 +} + func (s *segment) Reset(offset postings.ID) { s.state.Lock() defer s.state.Unlock() diff --git a/src/m3ninx/index/segment/types.go b/src/m3ninx/index/segment/types.go index 09a7624a0c..b4af5cdb80 100644 --- a/src/m3ninx/index/segment/types.go +++ b/src/m3ninx/index/segment/types.go @@ -192,6 +192,12 @@ type Builder interface { type DocumentsBuilder interface { Builder index.Writer + + // SetIndexConcurrency sets the concurrency used for building the segment. + SetIndexConcurrency(value int) + + // IndexConcurrency returns the concurrency used for building the segment. + IndexConcurrency() int } // CloseableDocumentsBuilder is a builder that has documents written to it and has freeable resources. From aac419881d07547d3078e4d63c903baf580d8a4a Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 6 Jul 2020 03:55:44 -0400 Subject: [PATCH 02/38] Respect flush indexing concurrency runtime options --- src/dbnode/storage/index.go | 64 +++++++++++++++++++++++++++++-------- src/x/serialize/encoder.go | 40 +++++++++++++++-------- 2 files changed, 78 insertions(+), 26 deletions(-) diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index e39ad0df46..ce3ab49b22 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -24,6 +24,8 @@ import ( "bytes" "errors" "fmt" + "math" + goruntime "runtime" "sort" "strconv" "sync" @@ -790,15 +792,9 @@ func (i *nsIndex) writeBatchForBlockStart( i.metrics.asyncInsertSuccess.Inc(n) } - if err != nil { - // NB: dropping duplicate id error messages from logs as they're expected when we see - // repeated inserts. as long as a block has an ID, it's not an error so we don't need - // to pollute the logs with these messages. - if partialError, ok := err.(*m3ninxindex.BatchPartialError); ok { - err = partialError.FilterDuplicateIDErrors() - } - } - if err != nil { + // Allow for duplicate write errors since due to re-indexing races + // we may try to re-index a series more than once. + if err := i.allowDuplicatesWriteError(err); err != nil { numErrors := numPending - int(result.NumSuccess) if partialError, ok := err.(*m3ninxindex.BatchPartialError); ok { // If it was a batch partial error we know exactly how many failed @@ -908,7 +904,14 @@ func (i *nsIndex) WarmFlush( return err } - builderOpts := i.opts.IndexOptions().SegmentBuilderOptions() + // Determine the current flush indexing concurrency. + namespaceRuntimeOpts := i.namespaceRuntimeOptsMgr.Get() + perCPUFraction := namespaceRuntimeOpts.FlushIndexingPerCPUConcurrencyOrDefault() + cpus := math.Ceil(perCPUFraction * float64(goruntime.NumCPU())) + concurrency := int(math.Max(1, cpus)) + + builderOpts := i.opts.IndexOptions().SegmentBuilderOptions(). + SetConcurrency(concurrency) builder, err := builder.NewBuilderFromDocuments(builderOpts) if err != nil { return err @@ -1133,7 +1136,14 @@ func (i *nsIndex) flushBlockSegment( // Reset the builder builder.Reset(0) - ctx := i.opts.ContextPool().Get() + var ( + ctx = i.opts.ContextPool().Get() + docsPool = i.opts.IndexOptions().DocumentArrayPool() + batch = m3ninxindex.Batch{Docs: docsPool.Get(), AllowPartialUpdates: true} + batchSize = cap(batch.Docs) + ) + defer docsPool.Put(batch.Docs) + for _, shard := range shards { var ( first = true @@ -1169,8 +1179,21 @@ func (i *nsIndex) flushBlockSegment( return err } - _, err = builder.Insert(doc) - if err != nil && err != m3ninxindex.ErrDuplicateID { + batch.Docs = append(batch.Docs, doc) + if len(batch.Docs) < batchSize { + continue + } + + err = i.allowDuplicatesWriteError(builder.InsertBatch(batch)) + if err != nil { + return err + } + } + + // Add last batch if remaining. + if len(batch.Docs) > 0 { + err := i.allowDuplicatesWriteError(builder.InsertBatch(batch)) + if err != nil { return err } } @@ -1187,6 +1210,21 @@ func (i *nsIndex) flushBlockSegment( return preparedPersist.Persist(builder) } +func (i *nsIndex) allowDuplicatesWriteError(err error) error { + if err == nil { + return nil + } + + // NB: dropping duplicate id error messages from logs as they're expected when we see + // repeated inserts. as long as a block has an ID, it's not an error so we don't need + // to pollute the logs with these messages. + if partialError, ok := err.(*m3ninxindex.BatchPartialError); ok { + err = partialError.FilterDuplicateIDErrors() + } + + return err +} + func (i *nsIndex) AssignShardSet(shardSet sharding.ShardSet) { // NB(r): Allocate the filter function once, it can be used outside // of locks as it depends on no internal state. diff --git a/src/x/serialize/encoder.go b/src/x/serialize/encoder.go index c54859ea2e..f38d4baf73 100644 --- a/src/x/serialize/encoder.go +++ b/src/x/serialize/encoder.go @@ -52,9 +52,13 @@ import ( var ( byteOrder binary.ByteOrder = binary.LittleEndian - headerMagicBytes = encodeUInt16(headerMagicNumber) + headerMagicBytes = make([]byte, 2) ) +func init() { + encodeUInt16(headerMagicNumber, headerMagicBytes) +} + var ( errTagEncoderInUse = errors.New("encoder already in use") errTagLiteralTooLong = errors.New("literal is too long") @@ -66,8 +70,10 @@ type newCheckedBytesFn func([]byte, checked.BytesOptions) checked.Bytes var defaultNewCheckedBytesFn = checked.NewBytes type encoder struct { - buf *bytes.Buffer - checkedBytes checked.Bytes + buf *bytes.Buffer + checkedBytes checked.Bytes + staticBuffer [2]byte + staticBufferSlice []byte opts TagEncoderOptions pool TagEncoderPool @@ -80,21 +86,23 @@ func newTagEncoder( ) TagEncoder { b := make([]byte, 0, opts.InitialCapacity()) cb := newFn(nil, nil) - return &encoder{ + e := &encoder{ buf: bytes.NewBuffer(b), checkedBytes: cb, opts: opts, pool: pool, } + e.staticBufferSlice = e.staticBuffer[:] + return e } -func (e *encoder) Encode(srcTags ident.TagIterator) error { +func (e *encoder) Encode(tags ident.TagIterator) error { if e.checkedBytes.NumRef() > 0 { return errTagEncoderInUse } - tags := srcTags.Duplicate() - defer tags.Close() + tags.Rewind() + defer tags.Rewind() numTags := tags.Remaining() max := int(e.opts.TagSerializationLimits().MaxNumberTags()) @@ -107,7 +115,7 @@ func (e *encoder) Encode(srcTags ident.TagIterator) error { return err } - if _, err := e.buf.Write(encodeUInt16(uint16(numTags))); err != nil { + if _, err := e.buf.Write(e.encodeUInt16(uint16(numTags))); err != nil { e.buf.Reset() return err } @@ -177,7 +185,7 @@ func (e *encoder) encodeID(i ident.ID) error { } ld := uint16(len(d)) - if _, err := e.buf.Write(encodeUInt16(ld)); err != nil { + if _, err := e.buf.Write(e.encodeUInt16(ld)); err != nil { return err } @@ -188,10 +196,16 @@ func (e *encoder) encodeID(i ident.ID) error { return nil } -func encodeUInt16(v uint16) []byte { - var bytes [2]byte - byteOrder.PutUint16(bytes[:], v) - return bytes[:] +func (e *encoder) encodeUInt16(v uint16) []byte { + // NB(r): Use static buffer on the struct for encoding, otherwise if it's + // statically defined inline in the function it will escape to heap. + dest := e.staticBufferSlice[:2] + encodeUInt16(v, dest) + return dest +} + +func encodeUInt16(v uint16, dest []byte) { + byteOrder.PutUint16(dest, v) } func decodeUInt16(b []byte) uint16 { From c321ffcb36a9dd7be6f14611c8902ded0c885063 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 6 Jul 2020 04:09:17 -0400 Subject: [PATCH 03/38] Fix fn pointer assignment --- src/dbnode/storage/index/types.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/dbnode/storage/index/types.go b/src/dbnode/storage/index/types.go index cdf613245f..9537c5bc02 100644 --- a/src/dbnode/storage/index/types.go +++ b/src/dbnode/storage/index/types.go @@ -422,7 +422,10 @@ func NewBlockStatsReporter( reportSegmentStats func(stats BlockSegmentStats), reportIndexingStats func(stats BlockIndexingStats), ) BlockStatsReporter { - return blockStatsReporter{} + return blockStatsReporter{ + reportSegmentStats: reportSegmentStats, + reportIndexingStats: reportIndexingStats, + } } func (r blockStatsReporter) ReportSegmentStats(stats BlockSegmentStats) { From 8779db12054bba9501cfa1025153c506f49bf73d Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 6 Jul 2020 04:14:24 -0400 Subject: [PATCH 04/38] Revert test files --- src/dbnode/storage/index/block_prop_test.go | 3 +-- src/dbnode/storage/index/block_test.go | 2 -- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/dbnode/storage/index/block_prop_test.go b/src/dbnode/storage/index/block_prop_test.go index b676eab1ef..740ac0da45 100644 --- a/src/dbnode/storage/index/block_prop_test.go +++ b/src/dbnode/storage/index/block_prop_test.go @@ -178,8 +178,7 @@ func TestPostingsListCacheDoesNotAffectBlockQueryResults(t *testing.T) { } func newPropTestBlock(t *testing.T, blockStart time.Time, nsMeta namespace.Metadata, opts Options) (Block, error) { - blockOpts := BlockOptions{WriteIndexingPerCPUConurrency: 1} - blk, err := NewBlock(blockStart, nsMeta, blockOpts, opts) + blk, err := NewBlock(blockStart, nsMeta, BlockOptions{}, opts) require.NoError(t, err) var ( diff --git a/src/dbnode/storage/index/block_test.go b/src/dbnode/storage/index/block_test.go index 1bebfd4949..53cb68a0f0 100644 --- a/src/dbnode/storage/index/block_test.go +++ b/src/dbnode/storage/index/block_test.go @@ -1303,7 +1303,6 @@ func TestBlockE2EInsertQuery(t *testing.T) { blk, err := NewBlock(blockStart, testMD, BlockOptions{ - WriteIndexingPerCPUConcurrency: 1, ForegroundCompactorMmapDocsData: true, BackgroundCompactorMmapDocsData: true, }, testOpts) @@ -1915,7 +1914,6 @@ func TestBlockE2EInsertAggregate(t *testing.T) { blk, err := NewBlock(blockStart, testMD, BlockOptions{ - WriteIndexingPerCPUConcurrency: 1, ForegroundCompactorMmapDocsData: true, BackgroundCompactorMmapDocsData: true, }, testOpts) From c228639d02e0b56406265f8bb735c581e21d7f5f Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 6 Jul 2020 05:08:31 -0400 Subject: [PATCH 05/38] Update runtime options --- src/query/api/v1/handler/namespace/update.go | 39 +++++++++++++++----- 1 file changed, 30 insertions(+), 9 deletions(-) diff --git a/src/query/api/v1/handler/namespace/update.go b/src/query/api/v1/handler/namespace/update.go index ff50cf755d..03bbd016eb 100644 --- a/src/query/api/v1/handler/namespace/update.go +++ b/src/query/api/v1/handler/namespace/update.go @@ -52,6 +52,7 @@ var ( fieldNameRetentionOptions = "RetentionOptions" fieldNameRetetionPeriod = "RetentionPeriodNanos" + fieldNameRuntimeOptions = "RuntimeOptions" errEmptyNamespaceName = errors.New("must specify namespace name") errEmptyNamespaceOptions = errors.New("update options cannot be empty") @@ -144,7 +145,11 @@ func validateUpdateRequest(req *admin.NamespaceUpdateRequest) error { for i := 0; i < optsVal.NumField(); i++ { field := optsVal.Field(i) fieldName := optsVal.Type().Field(i).Name - if !field.IsZero() && fieldName != fieldNameRetentionOptions { + if field.IsZero() { + continue + } + + if fieldName != fieldNameRetentionOptions && fieldName != fieldNameRuntimeOptions { return fmt.Errorf("%s: %w", fieldName, errNamespaceFieldImmutable) } } @@ -190,18 +195,34 @@ func (h *UpdateHandler) Update( // Replace targeted namespace with modified retention. if newNanos := updateReq.Options.RetentionOptions.RetentionPeriodNanos; newNanos != 0 { dur := namespace.FromNanos(newNanos) - opts := ns.Options().SetRetentionOptions( - ns.Options().RetentionOptions().SetRetentionPeriod(dur), - ) - newMD, err := namespace.NewMetadata(ns.ID(), opts) + retentionOpts := ns.Options().RetentionOptions(). + SetRetentionPeriod(dur) + opts := ns.Options(). + SetRetentionOptions(retentionOpts) + ns, err = namespace.NewMetadata(ns.ID(), opts) + if err != nil { + return emptyReg, fmt.Errorf("error constructing new metadata: %w", err) + } + } + + // Update runtime options. + if newRuntimeOpts := updateReq.Options.RuntimeOptions; newRuntimeOpts != nil { + runtimeOpts := ns.Options().RuntimeOptions() + if v := newRuntimeOpts.WriteIndexingPerCPUConcurrency; v != nil { + runtimeOpts = runtimeOpts.SetWriteIndexingPerCPUConcurrency(&v.Value) + } + if v := newRuntimeOpts.FlushIndexingPerCPUConcurrency; v != nil { + runtimeOpts = runtimeOpts.SetFlushIndexingPerCPUConcurrency(&v.Value) + } + opts := ns.Options(). + SetRuntimeOptions(runtimeOpts) + ns, err = namespace.NewMetadata(ns.ID(), opts) if err != nil { return emptyReg, fmt.Errorf("error constructing new metadata: %w", err) } - newMDs = append(newMDs, newMD) - } else { - // If not modifying, keep the original NS. - newMDs = append(newMDs, ns) } + + newMDs = append(newMDs, ns) } nsMap, err := namespace.NewMap(newMDs) From fbe1cec994dfa0fe7c6427013daea91870fd67ca Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 6 Jul 2020 05:09:59 -0400 Subject: [PATCH 06/38] Add note --- src/dbnode/storage/database.go | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/dbnode/storage/database.go b/src/dbnode/storage/database.go index 467e313fb1..1d1928e56e 100644 --- a/src/dbnode/storage/database.go +++ b/src/dbnode/storage/database.go @@ -173,11 +173,12 @@ func NewDatabase( ) d := &db{ - opts: opts, - nowFn: nowFn, - shardSet: shardSet, - lastReceivedNewShards: nowFn(), - namespaces: newDatabaseNamespacesMap(databaseNamespacesMapOptions{}), + opts: opts, + nowFn: nowFn, + shardSet: shardSet, + lastReceivedNewShards: nowFn(), + namespaces: newDatabaseNamespacesMap(databaseNamespacesMapOptions{}), + // TODO: wire to etcd, move this to main.server run runtimeOptionsRegistry: namespace.NewRuntimeOptionsManagerRegistry(), commitLog: commitLog, scope: scope, From dd4fcc8af107d3cfd3f2f779fa86ab72639cbe03 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 6 Jul 2020 16:09:17 -0400 Subject: [PATCH 07/38] Share underlying indexing resources across segment builders to bound concurrency, update runtime options when it changes per namespace --- src/dbnode/storage/database.go | 30 +- src/dbnode/storage/index.go | 31 +- src/dbnode/storage/index/mutable_segments.go | 11 +- src/dbnode/storage/options.go | 124 ++++---- src/dbnode/storage/types.go | 6 + src/m3ninx/index/segment/builder/builder.go | 304 ++++++++++++++----- 6 files changed, 350 insertions(+), 156 deletions(-) diff --git a/src/dbnode/storage/database.go b/src/dbnode/storage/database.go index 1d1928e56e..b2a6c7b608 100644 --- a/src/dbnode/storage/database.go +++ b/src/dbnode/storage/database.go @@ -173,13 +173,12 @@ func NewDatabase( ) d := &db{ - opts: opts, - nowFn: nowFn, - shardSet: shardSet, - lastReceivedNewShards: nowFn(), - namespaces: newDatabaseNamespacesMap(databaseNamespacesMapOptions{}), - // TODO: wire to etcd, move this to main.server run - runtimeOptionsRegistry: namespace.NewRuntimeOptionsManagerRegistry(), + opts: opts, + nowFn: nowFn, + shardSet: shardSet, + lastReceivedNewShards: nowFn(), + namespaces: newDatabaseNamespacesMap(databaseNamespacesMapOptions{}), + runtimeOptionsRegistry: opts.NamespaceRuntimeOptionsManagerRegistry(), commitLog: commitLog, scope: scope, metrics: newDatabaseMetrics(scope), @@ -246,6 +245,19 @@ func (d *db) UpdateOwnedNamespaces(newNamespaces namespace.Map) error { d.log.Error("failed to update schema registry", zap.Error(err)) } + // Always update the runtime options if they were set so that correct + // runtime options are set in the runtime options registry before namespaces + // are actually created. + for _, namespaceMetadata := range newNamespaces.Metadatas() { + id := namespaceMetadata.ID().String() + runtimeOptsMgr := d.runtimeOptionsRegistry.RuntimeOptionsManager(id) + currRuntimeOpts := runtimeOptsMgr.Get() + setRuntimeOpts := namespaceMetadata.Options().RuntimeOptions() + if !currRuntimeOpts.Equal(setRuntimeOpts) { + runtimeOptsMgr.Update(setRuntimeOpts) + } + } + d.Lock() defer d.Unlock() @@ -265,7 +277,9 @@ func (d *db) UpdateOwnedNamespaces(newNamespaces namespace.Map) error { // log that updates and removals are skipped if len(removes) > 0 || len(updates) > 0 { - d.log.Warn("skipping namespace removals and updates (except schema updates), restart process if you want changes to take effect.") + d.log.Warn("skipping namespace removals and updates " + + "(except schema updates and runtime options), " + + "restart the process if you want changes to take effect") } // enqueue bootstraps if new namespaces diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index ce3ab49b22..67b09c7266 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -912,12 +912,18 @@ func (i *nsIndex) WarmFlush( builderOpts := i.opts.IndexOptions().SegmentBuilderOptions(). SetConcurrency(concurrency) + builder, err := builder.NewBuilderFromDocuments(builderOpts) if err != nil { return err } defer builder.Close() + // Emit concurrency, then reset gauge to zero to show time + // active during flushing broken down per namespace. + i.metrics.flushIndexingConcurrency.Update(float64(concurrency)) + defer i.metrics.flushIndexingConcurrency.Update(0) + var evicted int for _, block := range flushable { immutableSegments, err := i.flushBlock(flush, block, shards, builder) @@ -1139,8 +1145,9 @@ func (i *nsIndex) flushBlockSegment( var ( ctx = i.opts.ContextPool().Get() docsPool = i.opts.IndexOptions().DocumentArrayPool() - batch = m3ninxindex.Batch{Docs: docsPool.Get(), AllowPartialUpdates: true} - batchSize = cap(batch.Docs) + docs = docsPool.Get() + batchSize = cap(docs) + batch = m3ninxindex.Batch{Docs: docs[:0], AllowPartialUpdates: true} ) defer docsPool.Put(batch.Docs) @@ -1173,6 +1180,8 @@ func (i *nsIndex) flushBlockSegment( return err } + // Reset docs batch before use. + batch.Docs = batch.Docs[:0] for _, result := range results.Results() { doc, err := convert.FromSeriesIDAndTagIter(result.ID, result.Tags) if err != nil { @@ -2036,6 +2045,7 @@ type nsIndexMetrics struct { indexingConcurrencyMin tally.Gauge indexingConcurrencyMax tally.Gauge indexingConcurrencyAvg tally.Gauge + flushIndexingConcurrency tally.Gauge loadedDocsPerQuery tally.Histogram queryExhaustiveSuccess tally.Counter @@ -2052,13 +2062,14 @@ func newNamespaceIndexMetrics( iopts instrument.Options, ) nsIndexMetrics { const ( - indexAttemptName = "index-attempt" - forwardIndexName = "forward-index" - indexingConcurrency = "indexing-concurrency" + indexAttemptName = "index-attempt" + forwardIndexName = "forward-index" + indexingConcurrency = "indexing-concurrency" + flushIndexingConcurrency = "flush-indexing-concurrency" ) scope := iopts.MetricsScope() blocksScope := scope.SubScope("blocks") - return nsIndexMetrics{ + m := nsIndexMetrics{ asyncInsertAttemptTotal: scope.Tagged(map[string]string{ "stage": "process", }).Counter(indexAttemptName), @@ -2100,6 +2111,7 @@ func newNamespaceIndexMetrics( indexingConcurrencyAvg: scope.Tagged(map[string]string{ "stat": "avg", }).Gauge(indexingConcurrency), + flushIndexingConcurrency: scope.Gauge(flushIndexingConcurrency), loadedDocsPerQuery: scope.Histogram( "loaded-docs-per-query", tally.MustMakeExponentialValueBuckets(10, 2, 16), @@ -2133,6 +2145,13 @@ func newNamespaceIndexMetrics( "result": "error_docs_require_exhaustive", }).Counter("query"), } + + // Initialize gauges that should default to zero before + // returning results so that they are exported with an + // explicit zero value at process startup. + m.flushIndexingConcurrency.Update(0) + + return m } type nsIndexBlocksMetrics struct { diff --git a/src/dbnode/storage/index/mutable_segments.go b/src/dbnode/storage/index/mutable_segments.go index 9c1664be01..9c01494c1b 100644 --- a/src/dbnode/storage/index/mutable_segments.go +++ b/src/dbnode/storage/index/mutable_segments.go @@ -125,15 +125,20 @@ func (m *mutableSegments) SetNamespaceRuntimeOptions(opts namespace.RuntimeOptio perCPUFraction := opts.WriteIndexingPerCPUConcurrencyOrDefault() cpus := math.Ceil(perCPUFraction * float64(runtime.NumCPU())) m.writeIndexingConcurrency = int(math.Max(1, cpus)) - builder := m.compact.segmentBuilder + segmentBuilder := m.compact.segmentBuilder m.Unlock() // Reset any existing segment builder to new concurrency, do this // out of the lock since builder can be used for foreground compaction // outside the lock and does it's own locking. - if builder != nil { - builder.SetIndexConcurrency(m.writeIndexingConcurrency) + if segmentBuilder != nil { + segmentBuilder.SetIndexConcurrency(m.writeIndexingConcurrency) } + + // Set the global concurrency control we have (we may need to fork + // github.com/twotwotwo/sorts to control this on a per segment builder + // basis). + builder.SetSortConcurrency(m.writeIndexingConcurrency) } func (m *mutableSegments) WriteBatch(inserts *WriteBatch) error { diff --git a/src/dbnode/storage/options.go b/src/dbnode/storage/options.go index 92200a2823..bfbc758f9c 100644 --- a/src/dbnode/storage/options.go +++ b/src/dbnode/storage/options.go @@ -117,50 +117,51 @@ func NewSeriesOptionsFromOptions(opts Options, ropts retention.Options) series.O } type options struct { - clockOpts clock.Options - instrumentOpts instrument.Options - nsRegistryInitializer namespace.Initializer - blockOpts block.Options - commitLogOpts commitlog.Options - runtimeOptsMgr m3dbruntime.OptionsManager - errWindowForLoad time.Duration - errThresholdForLoad int64 - indexingEnabled bool - repairEnabled bool - truncateType series.TruncateType - transformOptions series.WriteTransformOptions - indexOpts index.Options - repairOpts repair.Options - newEncoderFn encoding.NewEncoderFn - newDecoderFn encoding.NewDecoderFn - bootstrapProcessProvider bootstrap.ProcessProvider - persistManager persist.Manager - blockRetrieverManager block.DatabaseBlockRetrieverManager - poolOpts pool.ObjectPoolOptions - contextPool context.Pool - seriesCachePolicy series.CachePolicy - seriesOpts series.Options - seriesPool series.DatabaseSeriesPool - bytesPool pool.CheckedBytesPool - encoderPool encoding.EncoderPool - segmentReaderPool xio.SegmentReaderPool - readerIteratorPool encoding.ReaderIteratorPool - multiReaderIteratorPool encoding.MultiReaderIteratorPool - identifierPool ident.Pool - fetchBlockMetadataResultsPool block.FetchBlockMetadataResultsPool - fetchBlocksMetadataResultsPool block.FetchBlocksMetadataResultsPool - queryIDsWorkerPool xsync.WorkerPool - writeBatchPool *writes.WriteBatchPool - bufferBucketPool *series.BufferBucketPool - bufferBucketVersionsPool *series.BufferBucketVersionsPool - retrieveRequestPool fs.RetrieveRequestPool - checkedBytesWrapperPool xpool.CheckedBytesWrapperPool - schemaReg namespace.SchemaRegistry - blockLeaseManager block.LeaseManager - onColdFlush OnColdFlush - memoryTracker MemoryTracker - mmapReporter mmap.Reporter - doNotIndexWithFieldsMap map[string]string + clockOpts clock.Options + instrumentOpts instrument.Options + nsRegistryInitializer namespace.Initializer + blockOpts block.Options + commitLogOpts commitlog.Options + runtimeOptsMgr m3dbruntime.OptionsManager + errWindowForLoad time.Duration + errThresholdForLoad int64 + indexingEnabled bool + repairEnabled bool + truncateType series.TruncateType + transformOptions series.WriteTransformOptions + indexOpts index.Options + repairOpts repair.Options + newEncoderFn encoding.NewEncoderFn + newDecoderFn encoding.NewDecoderFn + bootstrapProcessProvider bootstrap.ProcessProvider + persistManager persist.Manager + blockRetrieverManager block.DatabaseBlockRetrieverManager + poolOpts pool.ObjectPoolOptions + contextPool context.Pool + seriesCachePolicy series.CachePolicy + seriesOpts series.Options + seriesPool series.DatabaseSeriesPool + bytesPool pool.CheckedBytesPool + encoderPool encoding.EncoderPool + segmentReaderPool xio.SegmentReaderPool + readerIteratorPool encoding.ReaderIteratorPool + multiReaderIteratorPool encoding.MultiReaderIteratorPool + identifierPool ident.Pool + fetchBlockMetadataResultsPool block.FetchBlockMetadataResultsPool + fetchBlocksMetadataResultsPool block.FetchBlocksMetadataResultsPool + queryIDsWorkerPool xsync.WorkerPool + writeBatchPool *writes.WriteBatchPool + bufferBucketPool *series.BufferBucketPool + bufferBucketVersionsPool *series.BufferBucketVersionsPool + retrieveRequestPool fs.RetrieveRequestPool + checkedBytesWrapperPool xpool.CheckedBytesWrapperPool + schemaReg namespace.SchemaRegistry + blockLeaseManager block.LeaseManager + onColdFlush OnColdFlush + memoryTracker MemoryTracker + mmapReporter mmap.Reporter + doNotIndexWithFieldsMap map[string]string + namespaceRuntimeOptsMgrRegistry namespace.RuntimeOptionsManagerRegistry } // NewOptions creates a new set of storage options with defaults @@ -221,17 +222,18 @@ func newOptions(poolOpts pool.ObjectPoolOptions) Options { TagsPoolOptions: poolOpts, TagsIteratorPoolOptions: poolOpts, }), - fetchBlockMetadataResultsPool: block.NewFetchBlockMetadataResultsPool(poolOpts, 0), - fetchBlocksMetadataResultsPool: block.NewFetchBlocksMetadataResultsPool(poolOpts, 0), - queryIDsWorkerPool: queryIDsWorkerPool, - writeBatchPool: writeBatchPool, - bufferBucketVersionsPool: series.NewBufferBucketVersionsPool(poolOpts), - bufferBucketPool: series.NewBufferBucketPool(poolOpts), - retrieveRequestPool: retrieveRequestPool, - checkedBytesWrapperPool: bytesWrapperPool, - schemaReg: namespace.NewSchemaRegistry(false, nil), - onColdFlush: &noOpColdFlush{}, - memoryTracker: NewMemoryTracker(NewMemoryTrackerOptions(defaultNumLoadedBytesLimit)), + fetchBlockMetadataResultsPool: block.NewFetchBlockMetadataResultsPool(poolOpts, 0), + fetchBlocksMetadataResultsPool: block.NewFetchBlocksMetadataResultsPool(poolOpts, 0), + queryIDsWorkerPool: queryIDsWorkerPool, + writeBatchPool: writeBatchPool, + bufferBucketVersionsPool: series.NewBufferBucketVersionsPool(poolOpts), + bufferBucketPool: series.NewBufferBucketPool(poolOpts), + retrieveRequestPool: retrieveRequestPool, + checkedBytesWrapperPool: bytesWrapperPool, + schemaReg: namespace.NewSchemaRegistry(false, nil), + onColdFlush: &noOpColdFlush{}, + memoryTracker: NewMemoryTracker(NewMemoryTrackerOptions(defaultNumLoadedBytesLimit)), + namespaceRuntimeOptsMgrRegistry: namespace.NewRuntimeOptionsManagerRegistry(), } return o.SetEncodingM3TSZPooled() } @@ -784,6 +786,18 @@ func (o *options) DoNotIndexWithFieldsMap() map[string]string { return o.doNotIndexWithFieldsMap } +func (o *options) SetNamespaceRuntimeOptionsManagerRegistry( + value namespace.RuntimeOptionsManagerRegistry, +) Options { + opts := *o + opts.namespaceRuntimeOptsMgrRegistry = value + return &opts +} + +func (o *options) NamespaceRuntimeOptionsManagerRegistry() namespace.RuntimeOptionsManagerRegistry { + return o.namespaceRuntimeOptsMgrRegistry +} + type noOpColdFlush struct{} func (n *noOpColdFlush) ColdFlushNamespace(ns Namespace) (OnColdFlushNamespace, error) { diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 4068e69f19..602048d3f8 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -1143,6 +1143,12 @@ type Options interface { // DoNotIndexWithFieldsMap returns a map which if fields match it // will not index those metrics. DoNotIndexWithFieldsMap() map[string]string + + // SetNamespaceRuntimeOptionsManagerRegistry sets the namespace runtime options manager. + SetNamespaceRuntimeOptionsManagerRegistry(value namespace.RuntimeOptionsManagerRegistry) Options + + // NamespaceRuntimeOptionsManagerRegistry returns the namespace runtime options manager. + NamespaceRuntimeOptionsManagerRegistry() namespace.RuntimeOptionsManagerRegistry } // MemoryTracker tracks memory. diff --git a/src/m3ninx/index/segment/builder/builder.go b/src/m3ninx/index/segment/builder/builder.go index 25c64b4252..054d4efa59 100644 --- a/src/m3ninx/index/segment/builder/builder.go +++ b/src/m3ninx/index/segment/builder/builder.go @@ -23,6 +23,7 @@ package builder import ( "errors" "fmt" + "runtime" "sync" "github.com/m3db/m3/src/m3ninx/doc" @@ -32,6 +33,7 @@ import ( "github.com/m3db/m3/src/m3ninx/util" "github.com/cespare/xxhash/v2" + "github.com/twotwotwo/sorts" ) var ( @@ -41,20 +43,131 @@ var ( const ( // Slightly buffer the work to avoid blocking main thread. - indexQueueSize = 2 << 9 // 1024 + indexQueueSize = 2 << 9 // 1024 + entriesPerIndexJob = 32 ) +var ( + globalIndexWorkers = &indexWorkers{} + fieldsMapSetOptions = fieldsMapSetUnsafeOptions{ + // Builder takes ownership of keys and docs so it's ok + // to avoid copying and finalizing keys. + NoCopyKey: true, + NoFinalizeKey: true, + } +) + +type indexWorkers struct { + sync.RWMutex + builders int + queues []chan indexJob +} + type indexJob struct { wg *sync.WaitGroup - id postings.ID - field doc.Field + opts Options + + entries [entriesPerIndexJob]indexJobEntry + usedEntries int + + shard int + shardedFields *shardedFields - shard int - idx int batchErr *index.BatchPartialError } +type indexJobEntry struct { + id postings.ID + field doc.Field + docIdx int +} + +func (w *indexWorkers) registerBuilder() { + w.Lock() + defer w.Unlock() + + preIncBuilders := w.builders + w.builders++ + + if preIncBuilders != 0 { + return // Already initialized. + } + + // Need to initialize structures, prepare all num CPU + // worker queues, even if we don't use all of them. + n := runtime.NumCPU() + if cap(w.queues) == 0 { + w.queues = make([]chan indexJob, 0, n) + } else { + // Reuse existing queues slice. + w.queues = w.queues[:0] + } + + // Start the workers. + for i := 0; i < n; i++ { + indexQueue := make(chan indexJob, indexQueueSize) + w.queues = append(w.queues, indexQueue) + go w.indexWorker(indexQueue) + } +} + +func (w *indexWorkers) indexWorker(indexQueue chan indexJob) { + for job := range indexQueue { + for i := 0; i < job.usedEntries; i++ { + entry := job.entries[i] + terms, ok := job.shardedFields.fields.ShardedGet(job.shard, entry.field.Name) + if !ok { + // NB(bodu): Check again within the lock to make sure we aren't making concurrent map writes. + terms = newTerms(job.opts) + job.shardedFields.fields.ShardedSetUnsafe(job.shard, entry.field.Name, + terms, fieldsMapSetOptions) + } + + // If empty field, track insertion of this key into the fields + // collection for correct response when retrieving all fields. + newField := terms.size() == 0 + // NB(bodu): Bulk of the cpu time during insertion is spent inside of terms.post(). + err := terms.post(entry.field.Value, entry.id) + if err != nil { + job.batchErr.AddWithLock(index.BatchError{Err: err, Idx: entry.docIdx}) + } + if err == nil && newField { + newEntry := uniqueField{ + field: entry.field.Name, + postingsList: terms.postingsListUnion, + } + job.shardedFields.uniqueFields[job.shard] = + append(job.shardedFields.uniqueFields[job.shard], newEntry) + } + } + + job.wg.Done() + } +} + +func (w *indexWorkers) indexJob(job indexJob) { + w.queues[job.shard] <- job +} + +func (w *indexWorkers) unregisterBuilder() { + w.Lock() + defer w.Unlock() + + w.builders-- + + if w.builders != 0 { + return // Still have registered builders, cannot spin down yet. + } + + // Close the workers. + for i := range w.queues { + close(w.queues[i]) + w.queues[i] = nil + } + w.queues = w.queues[:0] +} + type builderStatus struct { sync.RWMutex closed bool @@ -69,13 +182,16 @@ type builder struct { batchSizeOne index.Batch docs []doc.Document idSet *IDsMap - fields *shardedFieldsMap - uniqueFields [][]uniqueField + shardedJobs []indexJob + shardedFields *shardedFields concurrency int - fieldsSetOpts fieldsMapSetUnsafeOptions - indexQueues []chan indexJob - status builderStatus + status builderStatus +} + +type shardedFields struct { + fields *shardedFieldsMap + uniqueFields [][]uniqueField } // NewBuilderFromDocuments returns a builder from documents, it is @@ -91,13 +207,10 @@ func NewBuilderFromDocuments(opts Options) (segment.CloseableDocumentsBuilder, e idSet: NewIDsMap(IDsMapOptions{ InitialSize: opts.InitialCapacity(), }), - fieldsSetOpts: fieldsMapSetUnsafeOptions{ - // Builder takes ownership of keys and docs so it's ok - // to avoid copying and finalizing keys. - NoCopyKey: true, - NoFinalizeKey: true, - }, + shardedFields: &shardedFields{}, } + // Indiciate we need to spin up workers if we haven't already. + globalIndexWorkers.registerBuilder() b.SetIndexConcurrency(opts.Concurrency()) return b, nil } @@ -112,23 +225,17 @@ func (b *builder) SetIndexConcurrency(value int) { b.concurrency = value - for _, indexQueue := range b.indexQueues { - close(indexQueue) - } + // Nothing to migrate, jobs only used during a batch insertion. + b.shardedJobs = make([]indexJob, b.concurrency) // Take refs to existing fields to migrate. - existingUniqueFields := b.uniqueFields - existingFields := b.fields + existingUniqueFields := b.shardedFields.uniqueFields + existingFields := b.shardedFields.fields - b.uniqueFields = make([][]uniqueField, 0, b.concurrency) - b.fields = newShardedFieldsMap(b.concurrency, b.opts.InitialCapacity()) - b.indexQueues = make([]chan indexJob, 0, b.concurrency) + b.shardedFields.uniqueFields = make([][]uniqueField, 0, b.concurrency) + b.shardedFields.fields = newShardedFieldsMap(b.concurrency, b.opts.InitialCapacity()) for i := 0; i < b.concurrency; i++ { - indexQueue := make(chan indexJob, indexQueueSize) - b.indexQueues = append(b.indexQueues, indexQueue) - go b.indexWorker(indexQueue) - // Give each shard a fraction of the configured initial capacity. shardInitialCapacity := b.opts.InitialCapacity() if shardInitialCapacity > 0 { @@ -136,7 +243,8 @@ func (b *builder) SetIndexConcurrency(value int) { } shardUniqueFields := make([]uniqueField, 0, shardInitialCapacity) - b.uniqueFields = append(b.uniqueFields, shardUniqueFields) + b.shardedFields.uniqueFields = + append(b.shardedFields.uniqueFields, shardUniqueFields) } // Migrate data from existing unique fields. @@ -147,7 +255,8 @@ func (b *builder) SetIndexConcurrency(value int) { newShard := b.calculateShard(field.field) // Append to the correct shard. - b.uniqueFields[newShard] = append(b.uniqueFields[newShard], field) + b.shardedFields.uniqueFields[newShard] = + append(b.shardedFields.uniqueFields[newShard], field) } } } @@ -163,7 +272,8 @@ func (b *builder) SetIndexConcurrency(value int) { newShard := b.calculateShard(field) // Set with new correct shard. - b.fields.ShardedSetUnsafe(newShard, field, terms, b.fieldsSetOpts) + b.shardedFields.fields.ShardedSetUnsafe(newShard, field, + terms, fieldsMapSetOptions) } } } @@ -190,14 +300,14 @@ func (b *builder) Reset(offset postings.ID) { b.idSet.Reset() // Keep fields around, just reset the terms set for each one. - b.fields.ResetTermsSets() + b.shardedFields.fields.ResetTermsSets() // Reset the unique fields slice - for i, shardUniqueFields := range b.uniqueFields { + for i, shardUniqueFields := range b.shardedFields.uniqueFields { for i := range shardUniqueFields { shardUniqueFields[i] = uniqueField{} } - b.uniqueFields[i] = shardUniqueFields[:0] + b.shardedFields.uniqueFields[i] = shardUniqueFields[:0] } } @@ -238,11 +348,25 @@ func (b *builder) InsertBatch(batch index.Batch) error { return nil } +func (b *builder) resetShardedJobs() { + // Reset sharded jobs using memset optimization. + var jobZeroed indexJob + for i := range b.shardedJobs { + b.shardedJobs[i] = jobZeroed + } +} + func (b *builder) insertBatchWithRLock(batch index.Batch) *index.BatchPartialError { // NB(r): This is all kept in a single method to make the - // insertion path fast. - var wg sync.WaitGroup + // insertion path avoid too much function call overhead. + wg := &sync.WaitGroup{} batchErr := index.NewBatchPartialError() + + // Reset shared resources and at cleanup too to remove refs. + b.resetShardedJobs() + defer b.resetShardedJobs() + + // Enqueue docs for indexing. for i, d := range batch.Docs { // Validate doc if err := d.Validate(); err != nil { @@ -282,14 +406,21 @@ func (b *builder) insertBatchWithRLock(batch index.Batch) *index.BatchPartialErr // Index the terms. for _, f := range d.Fields { - b.index(&wg, postings.ID(postingsListID), f, i, batchErr) + b.queueIndexJobEntry(wg, postings.ID(postingsListID), f, i, batchErr) } - b.index(&wg, postings.ID(postingsListID), doc.Field{ + b.queueIndexJobEntry(wg, postings.ID(postingsListID), doc.Field{ Name: doc.IDReservedFieldName, Value: d.ID, }, i, batchErr) } + // Enqueue any partially filled sharded jobs. + for shard := 0; shard < b.concurrency; shard++ { + if b.shardedJobs[shard].usedEntries > 0 { + b.flushShardedIndexJob(shard, wg, batchErr) + } + } + // Wait for all the concurrent indexing jobs to finish. wg.Wait() @@ -299,57 +430,48 @@ func (b *builder) insertBatchWithRLock(batch index.Batch) *index.BatchPartialErr return nil } -func (b *builder) index( +func (b *builder) queueIndexJobEntry( wg *sync.WaitGroup, id postings.ID, - f doc.Field, - i int, + field doc.Field, + docIdx int, batchErr *index.BatchPartialError, ) { - wg.Add(1) - // NB(bodu): To avoid locking inside of the terms, we shard the work - // by field name. - shard := b.calculateShard(f.Name) - b.indexQueues[shard] <- indexJob{ - wg: wg, - id: id, - field: f, - shard: shard, - idx: i, - batchErr: batchErr, + shard := b.calculateShard(field.Name) + entryIndex := b.shardedJobs[shard].usedEntries + b.shardedJobs[shard].usedEntries++ + b.shardedJobs[shard].entries[entryIndex].id = id + b.shardedJobs[shard].entries[entryIndex].field = field + b.shardedJobs[shard].entries[entryIndex].docIdx = docIdx + + numEntries := b.shardedJobs[shard].usedEntries + if numEntries != entriesPerIndexJob { + return } -} -func (b *builder) indexWorker(indexQueue chan indexJob) { - for job := range indexQueue { - terms, ok := b.fields.ShardedGet(job.shard, job.field.Name) - if !ok { - // NB(bodu): Check again within the lock to make sure we aren't making concurrent map writes. - terms = newTerms(b.opts) - b.fields.ShardedSetUnsafe(job.shard, job.field.Name, - terms, b.fieldsSetOpts) - } + // Ready to flush this job since all entries are used. + b.flushShardedIndexJob(shard, wg, batchErr) +} - // If empty field, track insertion of this key into the fields - // collection for correct response when retrieving all fields. - newField := terms.size() == 0 - // NB(bodu): Bulk of the cpu time during insertion is spent inside of terms.post(). - err := terms.post(job.field.Value, job.id) - if err != nil { - job.batchErr.AddWithLock(index.BatchError{Err: err, Idx: job.idx}) - } - if err == nil && newField { - b.uniqueFields[job.shard] = append(b.uniqueFields[job.shard], uniqueField{ - field: job.field.Name, - postingsList: terms.postingsListUnion, - }) - } - job.wg.Done() - } +func (b *builder) flushShardedIndexJob( + shard int, + wg *sync.WaitGroup, + batchErr *index.BatchPartialError, +) { + // Set common fields. + b.shardedJobs[shard].shard = shard + b.shardedJobs[shard].wg = wg + b.shardedJobs[shard].batchErr = batchErr + b.shardedJobs[shard].shardedFields = b.shardedFields + b.shardedJobs[shard].opts = b.opts + + // Enqueue job. + wg.Add(1) + globalIndexWorkers.indexJob(b.shardedJobs[shard]) } func (b *builder) calculateShard(field []byte) int { - return int(xxhash.Sum64(field) % uint64(len(b.indexQueues))) + return int(xxhash.Sum64(field) % uint64(b.concurrency)) } func (b *builder) AllDocs() (index.IDDocIterator, error) { @@ -380,11 +502,12 @@ func (b *builder) TermsIterable() segment.TermsIterable { } func (b *builder) FieldsPostingsList() (segment.FieldsPostingsListIterator, error) { - return newOrderedFieldsPostingsListIter(b.uniqueFields), nil + return newOrderedFieldsPostingsListIter(b.shardedFields.uniqueFields), nil } func (b *builder) Terms(field []byte) (segment.TermsIterator, error) { - terms, ok := b.fields.ShardedGet(b.calculateShard(field), field) + shard := b.calculateShard(field) + terms, ok := b.shardedFields.fields.ShardedGet(shard, field) if !ok { return nil, fmt.Errorf("field not found: %s", string(field)) } @@ -399,9 +522,22 @@ func (b *builder) Terms(field []byte) (segment.TermsIterator, error) { func (b *builder) Close() error { b.status.Lock() defer b.status.Unlock() - for _, q := range b.indexQueues { - close(q) - } b.status.closed = true + // Indiciate we could possibly spin down workers if no builders open. + globalIndexWorkers.unregisterBuilder() return nil } + +var ( + sortConcurrencyLock sync.Mutex +) + +// SetSortConcurrency sets the sort concurrency for when +// building segments, unfortunately this must be set globally +// since github.com/twotwotwo/sorts does not provide an +// ability to set parallelism on call to sort. +func SetSortConcurrency(value int) { + sortConcurrencyLock.Lock() + sorts.MaxProcs = value + sortConcurrencyLock.Unlock() +} From ef415252a64ab911b85ac0c9d7486be0b5e53598 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 6 Jul 2020 16:19:48 -0400 Subject: [PATCH 08/38] Reset sharded jobs after use --- src/m3ninx/index/segment/builder/builder.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/m3ninx/index/segment/builder/builder.go b/src/m3ninx/index/segment/builder/builder.go index 054d4efa59..c0305b5c07 100644 --- a/src/m3ninx/index/segment/builder/builder.go +++ b/src/m3ninx/index/segment/builder/builder.go @@ -451,6 +451,9 @@ func (b *builder) queueIndexJobEntry( // Ready to flush this job since all entries are used. b.flushShardedIndexJob(shard, wg, batchErr) + + // Reset for reuse. + b.shardedJobs[shard] = indexJob{} } func (b *builder) flushShardedIndexJob( From f8e5049df7bfa6e7673698c710344d0a28fee7db Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 6 Jul 2020 16:39:07 -0400 Subject: [PATCH 09/38] Fix upsert options endpoint --- src/query/api/v1/handler/namespace/update.go | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/src/query/api/v1/handler/namespace/update.go b/src/query/api/v1/handler/namespace/update.go index 03bbd016eb..a3fd7af2df 100644 --- a/src/query/api/v1/handler/namespace/update.go +++ b/src/query/api/v1/handler/namespace/update.go @@ -136,12 +136,7 @@ func validateUpdateRequest(req *admin.NamespaceUpdateRequest) error { return errEmptyNamespaceOptions } - if req.Options.RetentionOptions == nil { - return errEmptyRetentionOptions - } - optsVal := reflect.ValueOf(*req.Options) - for i := 0; i < optsVal.NumField(); i++ { field := optsVal.Field(i) fieldName := optsVal.Type().Field(i).Name @@ -154,12 +149,14 @@ func validateUpdateRequest(req *admin.NamespaceUpdateRequest) error { } } - optsVal = reflect.ValueOf(*req.Options.RetentionOptions) - for i := 0; i < optsVal.NumField(); i++ { - field := optsVal.Field(i) - fieldName := optsVal.Type().Field(i).Name - if !field.IsZero() && fieldName != fieldNameRetetionPeriod { - return fmt.Errorf("%s.%s: %w", fieldNameRetentionOptions, fieldName, errNamespaceFieldImmutable) + if opts := req.Options.RetentionOptions; opts != nil { + optsVal := reflect.ValueOf(*opts) + for i := 0; i < optsVal.NumField(); i++ { + field := optsVal.Field(i) + fieldName := optsVal.Type().Field(i).Name + if !field.IsZero() && fieldName != fieldNameRetetionPeriod { + return fmt.Errorf("%s.%s: %w", fieldNameRetentionOptions, fieldName, errNamespaceFieldImmutable) + } } } From b069000f880959d5e3131ff15b6c823ffaeb690b Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 6 Jul 2020 17:08:33 -0400 Subject: [PATCH 10/38] Reset docs batch between flushing batch during warm index flush --- src/dbnode/storage/index.go | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 67b09c7266..213a3dd18e 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -84,6 +84,8 @@ var ( const ( defaultFlushReadDataBlocksBatchSize = int64(4096) nsIndexReportStatsInterval = 10 * time.Second + + defaultFlushDocsBatchSize = 256 ) var ( @@ -1149,7 +1151,18 @@ func (i *nsIndex) flushBlockSegment( batchSize = cap(docs) batch = m3ninxindex.Batch{Docs: docs[:0], AllowPartialUpdates: true} ) - defer docsPool.Put(batch.Docs) + if batchSize == 0 { + batchSize = defaultFlushDocsBatchSize + } + defer func() { + // Memset optimization to clear refs before returning to pool. + var docZeroed doc.Document + for i := range batch.Docs { + batch.Docs[i] = docZeroed + } + // Return to pool. + docsPool.Put(batch.Docs) + }() for _, shard := range shards { var ( @@ -1197,6 +1210,9 @@ func (i *nsIndex) flushBlockSegment( if err != nil { return err } + + // Reset docs after insertions. + batch.Docs = batch.Docs[:0] } // Add last batch if remaining. From 94564b8e6e691f9c344641ffca70098ebfad2abc Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Wed, 8 Jul 2020 01:43:58 -0400 Subject: [PATCH 11/38] Fix update namespace flow and also do not attempt index flush when node owns no shards --- src/dbnode/storage/index.go | 10 ++ .../api/v1/handler/namespace/add_test.go | 39 ++++++- .../api/v1/handler/namespace/get_test.go | 72 +++++++++++- src/query/api/v1/handler/namespace/update.go | 103 +++++++++++------- .../api/v1/handler/namespace/update_test.go | 97 +++++++++++++++-- 5 files changed, 268 insertions(+), 53 deletions(-) diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 9735841d15..86b30e0129 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -901,6 +901,11 @@ func (i *nsIndex) WarmFlush( flush persist.IndexFlush, shards []databaseShard, ) error { + if len(shards) == 0 { + // No-op if no shards currently owned. + return nil + } + flushable, err := i.flushableBlocks(shards, series.WarmWrite) if err != nil { return err @@ -961,6 +966,11 @@ func (i *nsIndex) WarmFlush( } func (i *nsIndex) ColdFlush(shards []databaseShard) (OnColdFlushDone, error) { + if len(shards) == 0 { + // No-op if no shards currently owned. + return func() error { return nil }, nil + } + flushable, err := i.flushableBlocks(shards, series.ColdWrite) if err != nil { return nil, err diff --git a/src/query/api/v1/handler/namespace/add_test.go b/src/query/api/v1/handler/namespace/add_test.go index a797f5ffb2..3655083719 100644 --- a/src/query/api/v1/handler/namespace/add_test.go +++ b/src/query/api/v1/handler/namespace/add_test.go @@ -31,6 +31,7 @@ import ( nsproto "github.com/m3db/m3/src/dbnode/generated/proto/namespace" "github.com/m3db/m3/src/x/instrument" xjson "github.com/m3db/m3/src/x/json" + xtest "github.com/m3db/m3/src/x/test" "github.com/golang/mock/gomock" "github.com/stretchr/testify/assert" @@ -105,7 +106,43 @@ func TestNamespaceAddHandler(t *testing.T) { resp = w.Result() body, _ = ioutil.ReadAll(resp.Body) assert.Equal(t, http.StatusOK, resp.StatusCode) - assert.Equal(t, "{\"registry\":{\"namespaces\":{\"testNamespace\":{\"bootstrapEnabled\":true,\"flushEnabled\":true,\"writesToCommitLog\":true,\"cleanupEnabled\":true,\"repairEnabled\":true,\"retentionOptions\":{\"retentionPeriodNanos\":\"172800000000000\",\"blockSizeNanos\":\"7200000000000\",\"bufferFutureNanos\":\"600000000000\",\"bufferPastNanos\":\"600000000000\",\"blockDataExpiry\":true,\"blockDataExpiryAfterNotAccessPeriodNanos\":\"300000000000\",\"futureRetentionPeriodNanos\":\"0\"},\"snapshotEnabled\":true,\"indexOptions\":{\"enabled\":true,\"blockSizeNanos\":\"7200000000000\"},\"schemaOptions\":null,\"coldWritesEnabled\":false}}}}", string(body)) + + expected := xtest.MustPrettyJSONMap(t, + xjson.Map{ + "registry": xjson.Map{ + "namespaces": xjson.Map{ + "testNamespace": xjson.Map{ + "bootstrapEnabled": true, + "flushEnabled": true, + "writesToCommitLog": true, + "cleanupEnabled": true, + "repairEnabled": true, + "retentionOptions": xjson.Map{ + "retentionPeriodNanos": "172800000000000", + "blockSizeNanos": "7200000000000", + "bufferFutureNanos": "600000000000", + "bufferPastNanos": "600000000000", + "blockDataExpiry": true, + "blockDataExpiryAfterNotAccessPeriodNanos": "300000000000", + "futureRetentionPeriodNanos": "0", + }, + "snapshotEnabled": true, + "indexOptions": xjson.Map{ + "enabled": true, + "blockSizeNanos": "7200000000000", + }, + "runtimeOptions": nil, + "schemaOptions": nil, + "coldWritesEnabled": false, + }, + }, + }, + }) + + actual := xtest.MustPrettyJSONString(t, string(body)) + + assert.Equal(t, expected, actual, + xtest.Diff(expected, actual)) } func TestNamespaceAddHandler_Conflict(t *testing.T) { diff --git a/src/query/api/v1/handler/namespace/get_test.go b/src/query/api/v1/handler/namespace/get_test.go index c9e4a55a49..260e3b3d52 100644 --- a/src/query/api/v1/handler/namespace/get_test.go +++ b/src/query/api/v1/handler/namespace/get_test.go @@ -30,6 +30,8 @@ import ( "github.com/m3db/m3/src/cluster/kv" nsproto "github.com/m3db/m3/src/dbnode/generated/proto/namespace" "github.com/m3db/m3/src/x/instrument" + xjson "github.com/m3db/m3/src/x/json" + xtest "github.com/m3db/m3/src/x/test" "github.com/golang/mock/gomock" "github.com/stretchr/testify/assert" @@ -105,7 +107,40 @@ func TestNamespaceGetHandler(t *testing.T) { resp = w.Result() body, _ = ioutil.ReadAll(resp.Body) assert.Equal(t, http.StatusOK, resp.StatusCode) - assert.Equal(t, "{\"registry\":{\"namespaces\":{\"test\":{\"bootstrapEnabled\":true,\"flushEnabled\":true,\"writesToCommitLog\":true,\"cleanupEnabled\":false,\"repairEnabled\":false,\"retentionOptions\":{\"retentionPeriodNanos\":\"172800000000000\",\"blockSizeNanos\":\"7200000000000\",\"bufferFutureNanos\":\"600000000000\",\"bufferPastNanos\":\"600000000000\",\"blockDataExpiry\":true,\"blockDataExpiryAfterNotAccessPeriodNanos\":\"3600000000000\",\"futureRetentionPeriodNanos\":\"0\"},\"snapshotEnabled\":true,\"indexOptions\":null,\"schemaOptions\":null,\"coldWritesEnabled\":false}}}}", string(body)) + + expected := xtest.MustPrettyJSONMap(t, + xjson.Map{ + "registry": xjson.Map{ + "namespaces": xjson.Map{ + "test": xjson.Map{ + "bootstrapEnabled": true, + "cleanupEnabled": false, + "coldWritesEnabled": false, + "flushEnabled": true, + "indexOptions": nil, + "repairEnabled": false, + "retentionOptions": xjson.Map{ + "blockDataExpiry": true, + "blockDataExpiryAfterNotAccessPeriodNanos": "3600000000000", + "blockSizeNanos": "7200000000000", + "bufferFutureNanos": "600000000000", + "bufferPastNanos": "600000000000", + "futureRetentionPeriodNanos": "0", + "retentionPeriodNanos": "172800000000000", + }, + "runtimeOptions": nil, + "schemaOptions": nil, + "snapshotEnabled": true, + "writesToCommitLog": true, + }, + }, + }, + }) + + actual := xtest.MustPrettyJSONString(t, string(body)) + + assert.Equal(t, expected, actual, + xtest.Diff(expected, actual)) } func TestNamespaceGetHandlerWithDebug(t *testing.T) { @@ -151,5 +186,38 @@ func TestNamespaceGetHandlerWithDebug(t *testing.T) { resp := w.Result() body, _ := ioutil.ReadAll(resp.Body) assert.Equal(t, http.StatusOK, resp.StatusCode) - assert.Equal(t, "{\"registry\":{\"namespaces\":{\"test\":{\"bootstrapEnabled\":true,\"cleanupEnabled\":false,\"coldWritesEnabled\":false,\"flushEnabled\":true,\"indexOptions\":null,\"repairEnabled\":false,\"retentionOptions\":{\"blockDataExpiry\":true,\"blockDataExpiryAfterNotAccessPeriodDuration\":\"1h0m0s\",\"blockSizeDuration\":\"2h0m0s\",\"bufferFutureDuration\":\"10m0s\",\"bufferPastDuration\":\"10m0s\",\"futureRetentionPeriodDuration\":\"0s\",\"retentionPeriodDuration\":\"48h0m0s\"},\"schemaOptions\":null,\"snapshotEnabled\":true,\"writesToCommitLog\":true}}}}", string(body)) + + expected := xtest.MustPrettyJSONMap(t, + xjson.Map{ + "registry": xjson.Map{ + "namespaces": xjson.Map{ + "test": xjson.Map{ + "bootstrapEnabled": true, + "cleanupEnabled": false, + "coldWritesEnabled": false, + "flushEnabled": true, + "indexOptions": nil, + "repairEnabled": false, + "retentionOptions": xjson.Map{ + "blockDataExpiry": true, + "blockDataExpiryAfterNotAccessPeriodDuration": "1h0m0s", + "blockSizeDuration": "2h0m0s", + "bufferFutureDuration": "10m0s", + "bufferPastDuration": "10m0s", + "futureRetentionPeriodDuration": "0s", + "retentionPeriodDuration": "48h0m0s", + }, + "runtimeOptions": nil, + "schemaOptions": nil, + "snapshotEnabled": true, + "writesToCommitLog": true, + }, + }, + }, + }) + + actual := xtest.MustPrettyJSONString(t, string(body)) + + assert.Equal(t, expected, actual, + xtest.Diff(expected, actual)) } diff --git a/src/query/api/v1/handler/namespace/update.go b/src/query/api/v1/handler/namespace/update.go index a3fd7af2df..ed4417d08e 100644 --- a/src/query/api/v1/handler/namespace/update.go +++ b/src/query/api/v1/handler/namespace/update.go @@ -35,7 +35,6 @@ import ( "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" "github.com/m3db/m3/src/query/generated/proto/admin" "github.com/m3db/m3/src/query/util/logging" - "github.com/m3db/m3/src/x/ident" "github.com/m3db/m3/src/x/instrument" xhttp "github.com/m3db/m3/src/x/net/http" @@ -56,8 +55,12 @@ var ( errEmptyNamespaceName = errors.New("must specify namespace name") errEmptyNamespaceOptions = errors.New("update options cannot be empty") - errEmptyRetentionOptions = errors.New("retention options must be set") errNamespaceFieldImmutable = errors.New("namespace option field is immutable") + + allowedUpdateOptionsFields = map[string]struct{}{ + fieldNameRetentionOptions: struct{}{}, + fieldNameRuntimeOptions: struct{}{}, + } ) // UpdateHandler is the handler for namespace updates. @@ -84,13 +87,18 @@ func (h *UpdateHandler) ServeHTTP( md, rErr := h.parseRequest(r) if rErr != nil { - logger.Error("unable to parse request", zap.Error(rErr)) + logger.Warn("unable to parse request", zap.Error(rErr)) xhttp.Error(w, rErr.Inner(), rErr.Code()) return } opts := handleroptions.NewServiceOptions(svc, r.Header, nil) - nsRegistry, err := h.Update(md, opts) + nsRegistry, parseErr, err := h.Update(md, opts) + if parseErr != nil { + logger.Warn("update namespace bad request", zap.Error(parseErr)) + xhttp.Error(w, parseErr.Inner(), parseErr.Code()) + return + } if err != nil { logger.Error("unable to update namespace", zap.Error(err)) xhttp.Error(w, err, http.StatusInternalServerError) @@ -137,6 +145,7 @@ func validateUpdateRequest(req *admin.NamespaceUpdateRequest) error { } optsVal := reflect.ValueOf(*req.Options) + allNonZeroFields := true for i := 0; i < optsVal.NumField(); i++ { field := optsVal.Field(i) fieldName := optsVal.Type().Field(i).Name @@ -144,11 +153,18 @@ func validateUpdateRequest(req *admin.NamespaceUpdateRequest) error { continue } - if fieldName != fieldNameRetentionOptions && fieldName != fieldNameRuntimeOptions { + allNonZeroFields = false + + _, ok := allowedUpdateOptionsFields[fieldName] + if !ok { return fmt.Errorf("%s: %w", fieldName, errNamespaceFieldImmutable) } } + if allNonZeroFields { + return errEmptyNamespaceOptions + } + if opts := req.Options.RetentionOptions; opts != nil { optsVal := reflect.ValueOf(*opts) for i := 0; i < optsVal.NumField(); i++ { @@ -167,30 +183,35 @@ func validateUpdateRequest(req *admin.NamespaceUpdateRequest) error { func (h *UpdateHandler) Update( updateReq *admin.NamespaceUpdateRequest, opts handleroptions.ServiceOptions, -) (nsproto.Registry, error) { +) (nsproto.Registry, *xhttp.ParseError, error) { var emptyReg = nsproto.Registry{} store, err := h.client.Store(opts.KVOverrideOptions()) if err != nil { - return emptyReg, err + return emptyReg, nil, err } currentMetadata, version, err := Metadata(store) if err != nil { - return emptyReg, err + return emptyReg, nil, err } - updateID := ident.StringID(updateReq.Name) - newMDs := make([]namespace.Metadata, 0, len(currentMetadata)) + newMetadata := make(map[string]namespace.Metadata) for _, ns := range currentMetadata { - // Don't modify any other namespaces. - if !ns.ID().Equal(updateID) { - newMDs = append(newMDs, ns) - continue - } + newMetadata[ns.ID().String()] = ns + } + + ns, ok := newMetadata[updateReq.Name] + if !ok { + parseErr := xhttp.NewParseError( + fmt.Errorf("namespace not found: err=%s", updateReq.Name), + http.StatusNotFound) + return emptyReg, parseErr, nil + } - // Replace targeted namespace with modified retention. - if newNanos := updateReq.Options.RetentionOptions.RetentionPeriodNanos; newNanos != 0 { + // Replace targeted namespace with modified retention. + if newRetentionOpts := updateReq.Options.RetentionOptions; newRetentionOpts != nil { + if newNanos := newRetentionOpts.RetentionPeriodNanos; newNanos != 0 { dur := namespace.FromNanos(newNanos) retentionOpts := ns.Options().RetentionOptions(). SetRetentionPeriod(dur) @@ -198,40 +219,46 @@ func (h *UpdateHandler) Update( SetRetentionOptions(retentionOpts) ns, err = namespace.NewMetadata(ns.ID(), opts) if err != nil { - return emptyReg, fmt.Errorf("error constructing new metadata: %w", err) + return emptyReg, nil, fmt.Errorf("error constructing new metadata: %w", err) } } + } - // Update runtime options. - if newRuntimeOpts := updateReq.Options.RuntimeOptions; newRuntimeOpts != nil { - runtimeOpts := ns.Options().RuntimeOptions() - if v := newRuntimeOpts.WriteIndexingPerCPUConcurrency; v != nil { - runtimeOpts = runtimeOpts.SetWriteIndexingPerCPUConcurrency(&v.Value) - } - if v := newRuntimeOpts.FlushIndexingPerCPUConcurrency; v != nil { - runtimeOpts = runtimeOpts.SetFlushIndexingPerCPUConcurrency(&v.Value) - } - opts := ns.Options(). - SetRuntimeOptions(runtimeOpts) - ns, err = namespace.NewMetadata(ns.ID(), opts) - if err != nil { - return emptyReg, fmt.Errorf("error constructing new metadata: %w", err) - } + // Update runtime options. + if newRuntimeOpts := updateReq.Options.RuntimeOptions; newRuntimeOpts != nil { + runtimeOpts := ns.Options().RuntimeOptions() + if v := newRuntimeOpts.WriteIndexingPerCPUConcurrency; v != nil { + runtimeOpts = runtimeOpts.SetWriteIndexingPerCPUConcurrency(&v.Value) + } + if v := newRuntimeOpts.FlushIndexingPerCPUConcurrency; v != nil { + runtimeOpts = runtimeOpts.SetFlushIndexingPerCPUConcurrency(&v.Value) + } + opts := ns.Options(). + SetRuntimeOptions(runtimeOpts) + ns, err = namespace.NewMetadata(ns.ID(), opts) + if err != nil { + return emptyReg, nil, fmt.Errorf("error constructing new metadata: %w", err) } - - newMDs = append(newMDs, ns) } + // Update the namespace in case an update occurred. + newMetadata[updateReq.Name] = ns + + // Set the new slice and update. + newMDs := make([]namespace.Metadata, 0, len(newMetadata)) + for _, elem := range newMetadata { + newMDs = append(newMDs, elem) + } nsMap, err := namespace.NewMap(newMDs) if err != nil { - return emptyReg, err + return emptyReg, nil, err } protoRegistry := namespace.ToProto(nsMap) _, err = store.CheckAndSet(M3DBNodeNamespacesKey, version, protoRegistry) if err != nil { - return emptyReg, fmt.Errorf("failed to update namespace: %w", err) + return emptyReg, nil, fmt.Errorf("failed to update namespace: %w", err) } - return *protoRegistry, nil + return *protoRegistry, nil, nil } diff --git a/src/query/api/v1/handler/namespace/update_test.go b/src/query/api/v1/handler/namespace/update_test.go index 2411fa8f85..3ffd063f59 100644 --- a/src/query/api/v1/handler/namespace/update_test.go +++ b/src/query/api/v1/handler/namespace/update_test.go @@ -22,6 +22,7 @@ package namespace import ( "errors" + "fmt" "io/ioutil" "net/http" "net/http/httptest" @@ -33,6 +34,7 @@ import ( "github.com/m3db/m3/src/query/generated/proto/admin" "github.com/m3db/m3/src/x/instrument" xjson "github.com/m3db/m3/src/x/json" + xtest "github.com/m3db/m3/src/x/test" "github.com/golang/mock/gomock" "github.com/stretchr/testify/assert" @@ -87,7 +89,7 @@ func TestNamespaceUpdateHandler(t *testing.T) { body, err := ioutil.ReadAll(resp.Body) assert.NoError(t, err) assert.Equal(t, http.StatusBadRequest, resp.StatusCode) - assert.Equal(t, "{\"error\":\"unable to validate update request: retention options must be set\"}\n", string(body)) + assert.Equal(t, "{\"error\":\"unable to validate update request: update options cannot be empty\"}\n", string(body)) // Test good case. Note: there is no way to tell the difference between a boolean // being false and it not being set by a user. @@ -128,7 +130,43 @@ func TestNamespaceUpdateHandler(t *testing.T) { resp = w.Result() body, _ = ioutil.ReadAll(resp.Body) assert.Equal(t, http.StatusOK, resp.StatusCode) - assert.Equal(t, "{\"registry\":{\"namespaces\":{\"testNamespace\":{\"bootstrapEnabled\":true,\"flushEnabled\":true,\"writesToCommitLog\":true,\"cleanupEnabled\":false,\"repairEnabled\":false,\"retentionOptions\":{\"retentionPeriodNanos\":\"345600000000000\",\"blockSizeNanos\":\"7200000000000\",\"bufferFutureNanos\":\"600000000000\",\"bufferPastNanos\":\"600000000000\",\"blockDataExpiry\":true,\"blockDataExpiryAfterNotAccessPeriodNanos\":\"3600000000000\",\"futureRetentionPeriodNanos\":\"0\"},\"snapshotEnabled\":true,\"indexOptions\":{\"enabled\":false,\"blockSizeNanos\":\"7200000000000\"},\"schemaOptions\":null,\"coldWritesEnabled\":false}}}}", string(body)) + + expected := xtest.MustPrettyJSONMap(t, + xjson.Map{ + "registry": xjson.Map{ + "namespaces": xjson.Map{ + "testNamespace": xjson.Map{ + "bootstrapEnabled": true, + "flushEnabled": true, + "writesToCommitLog": true, + "cleanupEnabled": false, + "repairEnabled": false, + "retentionOptions": xjson.Map{ + "retentionPeriodNanos": "345600000000000", + "blockSizeNanos": "7200000000000", + "bufferFutureNanos": "600000000000", + "bufferPastNanos": "600000000000", + "blockDataExpiry": true, + "blockDataExpiryAfterNotAccessPeriodNanos": "3600000000000", + "futureRetentionPeriodNanos": "0", + }, + "snapshotEnabled": true, + "indexOptions": xjson.Map{ + "enabled": false, + "blockSizeNanos": "7200000000000", + }, + "runtimeOptions": nil, + "schemaOptions": nil, + "coldWritesEnabled": false, + }, + }, + }, + }) + + actual := xtest.MustPrettyJSONString(t, string(body)) + + assert.Equal(t, expected, actual, + xtest.Diff(expected, actual)) // Ensure an empty request respects existing namespaces. w = httptest.NewRecorder() @@ -146,7 +184,43 @@ func TestNamespaceUpdateHandler(t *testing.T) { resp = w.Result() body, _ = ioutil.ReadAll(resp.Body) assert.Equal(t, http.StatusOK, resp.StatusCode) - assert.Equal(t, "{\"registry\":{\"namespaces\":{\"testNamespace\":{\"bootstrapEnabled\":true,\"flushEnabled\":true,\"writesToCommitLog\":true,\"cleanupEnabled\":false,\"repairEnabled\":false,\"retentionOptions\":{\"retentionPeriodNanos\":\"172800000000000\",\"blockSizeNanos\":\"7200000000000\",\"bufferFutureNanos\":\"600000000000\",\"bufferPastNanos\":\"600000000000\",\"blockDataExpiry\":true,\"blockDataExpiryAfterNotAccessPeriodNanos\":\"3600000000000\",\"futureRetentionPeriodNanos\":\"0\"},\"snapshotEnabled\":true,\"indexOptions\":{\"enabled\":false,\"blockSizeNanos\":\"7200000000000\"},\"schemaOptions\":null,\"coldWritesEnabled\":false}}}}", string(body)) + + expected = xtest.MustPrettyJSONMap(t, + xjson.Map{ + "registry": xjson.Map{ + "namespaces": xjson.Map{ + "testNamespace": xjson.Map{ + "bootstrapEnabled": true, + "flushEnabled": true, + "writesToCommitLog": true, + "cleanupEnabled": false, + "repairEnabled": false, + "retentionOptions": xjson.Map{ + "retentionPeriodNanos": "172800000000000", + "blockSizeNanos": "7200000000000", + "bufferFutureNanos": "600000000000", + "bufferPastNanos": "600000000000", + "blockDataExpiry": true, + "blockDataExpiryAfterNotAccessPeriodNanos": "3600000000000", + "futureRetentionPeriodNanos": "0", + }, + "snapshotEnabled": true, + "indexOptions": xjson.Map{ + "enabled": false, + "blockSizeNanos": "7200000000000", + }, + "runtimeOptions": nil, + "schemaOptions": nil, + "coldWritesEnabled": false, + }, + }, + }, + }) + + actual = xtest.MustPrettyJSONString(t, string(body)) + + assert.Equal(t, expected, actual, + xtest.Diff(expected, actual)) } func TestValidateUpdateRequest(t *testing.T) { @@ -161,11 +235,9 @@ func TestValidateUpdateRequest(t *testing.T) { Name: "foo", } - reqEmptyRetention = &admin.NamespaceUpdateRequest{ - Name: "foo", - Options: &nsproto.NamespaceOptions{ - BootstrapEnabled: true, - }, + reqNoNonZeroFields = &admin.NamespaceUpdateRequest{ + Name: "foo", + Options: &nsproto.NamespaceOptions{}, } reqNonZeroBootstrap = &admin.NamespaceUpdateRequest{ @@ -213,9 +285,9 @@ func TestValidateUpdateRequest(t *testing.T) { expErr: errEmptyNamespaceOptions, }, { - name: "emptyRetention", - request: reqEmptyRetention, - expErr: errEmptyRetentionOptions, + name: "emptyNoNonZeroFields", + request: reqNoNonZeroFields, + expErr: errEmptyNamespaceOptions, }, { name: "nonZeroBootstrapField", @@ -236,7 +308,8 @@ func TestValidateUpdateRequest(t *testing.T) { t.Run(test.name, func(t *testing.T) { err := validateUpdateRequest(test.request) if err != nil { - assert.True(t, errors.Is(err, test.expErr)) + assert.True(t, errors.Is(err, test.expErr), + fmt.Sprintf("expected=%s, actual=%s", test.expErr, err)) return } From a26448411553a135f00cb6677a83aedb67d87510 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Wed, 8 Jul 2020 02:12:29 -0400 Subject: [PATCH 12/38] Add log for updating index runtime options --- src/dbnode/namespace/dynamic.go | 6 ++++-- src/dbnode/storage/index.go | 25 +++++++++++++++++-------- 2 files changed, 21 insertions(+), 10 deletions(-) diff --git a/src/dbnode/namespace/dynamic.go b/src/dbnode/namespace/dynamic.go index b2bb058073..d76a9f7a6d 100644 --- a/src/dbnode/namespace/dynamic.go +++ b/src/dbnode/namespace/dynamic.go @@ -199,11 +199,13 @@ func (r *dynamicRegistry) run() { if m.Equal(r.maps()) { r.metrics.numInvalidUpdates.Inc(1) - r.logger.Warn("dynamic namespace registry received identical update, skipping") + r.logger.Warn("dynamic namespace registry received identical update, skipping", + zap.Int("version", val.Version())) continue } - r.logger.Info("dynamic namespace registry updated to version", zap.Int("version", val.Version())) + r.logger.Info("dynamic namespace registry updated to version", + zap.Int("version", val.Version())) r.Lock() r.currentValue = val r.currentMap = m diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 86b30e0129..1dac8c49f1 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -111,11 +111,12 @@ type nsIndex struct { deleteFilesFn deleteFilesFn readIndexInfoFilesFn readIndexInfoFilesFn - newBlockFn index.NewBlockFn - logger *zap.Logger - opts Options - nsMetadata namespace.Metadata - runtimeOptsListener xclose.SimpleCloser + newBlockFn index.NewBlockFn + logger *zap.Logger + opts Options + nsMetadata namespace.Metadata + runtimeOptsListener xclose.SimpleCloser + runtimeNsOptsListener xclose.SimpleCloser resultsPool index.QueryResultsPool aggregateResultsPool index.AggregateResultsPool @@ -351,9 +352,8 @@ func newNamespaceIndexWithOptions( // Assign shard set upfront. idx.AssignShardSet(shardSet) - if runtimeOptsMgr != nil { - idx.runtimeOptsListener = runtimeOptsMgr.RegisterListener(idx) - } + idx.runtimeOptsListener = runtimeOptsMgr.RegisterListener(idx) + idx.runtimeNsOptsListener = idx.namespaceRuntimeOptsMgr.RegisterListener(idx) // set up forward index dice. dice, err := newForwardIndexDice(newIndexOpts.opts) @@ -406,6 +406,15 @@ func (i *nsIndex) SetRuntimeOptions(value runtime.Options) { i.state.Unlock() } +func (i *nsIndex) SetNamespaceRuntimeOptions(opts namespace.RuntimeOptions) { + // We don't like to log from every single index segment that has + // settings updated so we log the changes here. + i.logger.Info("set namespace runtime index options", + zap.Stringer("namespace", i.nsMetadata.ID()), + zap.Any("writeIndexingPerCPUConcurrency", opts.WriteIndexingPerCPUConcurrency()), + zap.Any("flushIndexingPerCPUConcurrency", opts.FlushIndexingPerCPUConcurrency())) +} + func (i *nsIndex) reportStatsUntilClosed() { ticker := time.NewTicker(nsIndexReportStatsInterval) defer ticker.Stop() From dd0012723be7610802a7dcd0d7b4ad1d00a734d1 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Thu, 9 Jul 2020 05:02:51 -0400 Subject: [PATCH 13/38] Use context pool for FST readers, also fix avoid unnecessary locking for series read write ref --- src/dbnode/storage/shard.go | 19 ++++++++++++------- src/m3ninx/index/segment/fst/segment.go | 5 +++-- src/x/context/finalizeable_list_gen.go | 24 +++++++++++++++++++++++- 3 files changed, 38 insertions(+), 10 deletions(-) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 1df1b562f1..d19665c5f6 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1028,18 +1028,14 @@ func (s *dbShard) SeriesReadWriteRef( }, nil } - // NB(r): Insert synchronously so caller has access to the series + // NB(r): Insert then wait so caller has access to the series // immediately, otherwise calls to LoadBlock(..) etc on the series itself // may have no effect if a collision with the same series // being put in the insert queue may cause a block to be loaded to a // series which gets discarded. - // TODO(r): Probably can't insert series sync otherwise we stall a ton - // of writes... need a better solution for bootstrapping. - // This is what causes writes to degrade during bootstrap. at := s.nowFn() - entry, err = s.insertSeriesSync(id, newTagsIterArg(tags), insertSyncOptions{ - insertType: insertSyncIncReaderWriterCount, - hasPendingIndex: opts.ReverseIndex, + result, err := s.insertSeriesAsyncBatched(id, tags, dbShardInsertAsyncOptions{ + hasPendingIndexing: opts.ReverseIndex, pendingIndex: dbShardPendingIndex{ timestamp: at, enqueuedAt: at, @@ -1049,6 +1045,15 @@ func (s *dbShard) SeriesReadWriteRef( return SeriesReadWriteRef{}, err } + // Wait for the insert to be batched together and inserted + result.wg.Wait() + + // Retrieve the inserted entry + entry, err = s.writableSeries(id, tags) + if err != nil { + return SeriesReadWriteRef{}, err + } + return SeriesReadWriteRef{ Series: entry.Series, Shard: s.shard, diff --git a/src/m3ninx/index/segment/fst/segment.go b/src/m3ninx/index/segment/fst/segment.go index f5786532d1..a95daa657e 100644 --- a/src/m3ninx/index/segment/fst/segment.go +++ b/src/m3ninx/index/segment/fst/segment.go @@ -258,7 +258,7 @@ func (r *fsSegment) Reader() (index.Reader, error) { return nil, errReaderClosed } - reader := newReader(r) + reader := newReader(r, r.opts) // NB(r): Ensure that we do not release, mmaps, etc // until all readers have been closed. @@ -845,9 +845,10 @@ type fsSegmentReader struct { func newReader( fsSegment *fsSegment, + opts Options, ) *fsSegmentReader { return &fsSegmentReader{ - ctx: context.NewContext(), + ctx: opts.ContextPool().Get(), fsSegment: fsSegment, } } diff --git a/src/x/context/finalizeable_list_gen.go b/src/x/context/finalizeable_list_gen.go index 5240da8e84..25d9a17cdc 100644 --- a/src/x/context/finalizeable_list_gen.go +++ b/src/x/context/finalizeable_list_gen.go @@ -25,6 +25,8 @@ package context import ( + "sync" + "github.com/m3db/m3/src/x/pool" ) @@ -115,6 +117,11 @@ func (e *finalizeableElement) Prev() *finalizeableElement { return nil } +var ( + defaultFinalizeableElementsPoolLock sync.RWMutex + defaultFinalizeableElementsPool *finalizeableElementPool +) + // finalizeableList represents a doubly linked list. // The zero value for finalizeableList is an empty list ready to use. type finalizeableList struct { @@ -129,7 +136,22 @@ func (l *finalizeableList) Init() *finalizeableList { l.root.prev = &l.root l.len = 0 if l.Pool == nil { - l.Pool = newFinalizeableElementPool(nil) + // Use a static pool at least, otherwise each time + // we create a list with no pool we create a wholly + // new pool of finalizeables (4096 of them). + defaultFinalizeableElementsPoolLock.RLock() + l.Pool = defaultFinalizeableElementsPool + defaultFinalizeableElementsPoolLock.RUnlock() + if l.Pool == nil { + defaultFinalizeableElementsPoolLock.Lock() + if defaultFinalizeableElementsPool == nil { + // Still not set, allocate pool once. + defaultFinalizeableElementsPool = newFinalizeableElementPool(nil) + } + // Take ref to guaranteed allocated pool. + l.Pool = defaultFinalizeableElementsPool + defaultFinalizeableElementsPoolLock.Unlock() + } } return l } From 09bc443d9e2242668014474fdf526543b9cbe88d Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Fri, 17 Jul 2020 16:31:22 -0400 Subject: [PATCH 14/38] Address feedback --- src/dbnode/namespace/convert.go | 2 +- .../namespace/namespace_runtime_options.go | 18 +++++++----- src/dbnode/storage/index.go | 8 ++--- src/x/context/finalizeable_list_gen.go | 29 ++++++++----------- 4 files changed, 27 insertions(+), 30 deletions(-) diff --git a/src/dbnode/namespace/convert.go b/src/dbnode/namespace/convert.go index 7c8dee3a72..853485a8bc 100644 --- a/src/dbnode/namespace/convert.go +++ b/src/dbnode/namespace/convert.go @@ -209,7 +209,7 @@ func OptionsToProto(opts Options) *nsproto.NamespaceOptions { // toRuntimeOptions returns the corresponding RuntimeOptions proto. func toRuntimeOptions(opts RuntimeOptions) *nsproto.NamespaceRuntimeOptions { - if opts == nil || opts.IsDefaults() { + if opts == nil || opts.IsDefault() { return nil } var ( diff --git a/src/dbnode/namespace/namespace_runtime_options.go b/src/dbnode/namespace/namespace_runtime_options.go index 3952c13e75..6c2918d58e 100644 --- a/src/dbnode/namespace/namespace_runtime_options.go +++ b/src/dbnode/namespace/namespace_runtime_options.go @@ -35,9 +35,9 @@ const ( // RuntimeOptions is a set of runtime options that can // be set per namespace. type RuntimeOptions interface { - // IsDefaults returns whether the runtime options are purely defaults + // IsDefault returns whether the runtime options are purely defaults // with no values explicitly set. - IsDefaults() bool + IsDefault() bool // Equal will return whether it's equal to another runtime options. Equal(other RuntimeOptions) bool @@ -118,7 +118,7 @@ func newRuntimeOptions() *runtimeOptions { return &runtimeOptions{} } -func (o *runtimeOptions) IsDefaults() bool { +func (o *runtimeOptions) IsDefault() bool { defaults := newRuntimeOptions() return *o == *defaults } @@ -169,6 +169,8 @@ type runtimeOptionsManagerRegistry struct { managers map[string]RuntimeOptionsManager } +// NewRuntimeOptionsManagerRegistry returns a new runtime options +// manager registry. func NewRuntimeOptionsManagerRegistry() RuntimeOptionsManagerRegistry { return &runtimeOptionsManagerRegistry{ managers: make(map[string]RuntimeOptionsManager), @@ -180,14 +182,13 @@ func (r *runtimeOptionsManagerRegistry) RuntimeOptionsManager( ) RuntimeOptionsManager { r.Lock() defer r.Unlock() + manager, ok := r.managers[namespace] - if ok { - return manager + if !ok { + manager = NewRuntimeOptionsManager(namespace) + r.managers[namespace] = manager } - manager = NewRuntimeOptionsManager(namespace) - r.managers[namespace] = manager - return manager } @@ -206,6 +207,7 @@ type runtimeOptionsManager struct { watchable watch.Watchable } +// NewRuntimeOptionsManager returns a new runtime options manager. func NewRuntimeOptionsManager(namespace string) RuntimeOptionsManager { watchable := watch.NewWatchable() watchable.Update(NewRuntimeOptions()) diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 3ebee8ea50..97773890c9 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -805,7 +805,7 @@ func (i *nsIndex) writeBatchForBlockStart( // Allow for duplicate write errors since due to re-indexing races // we may try to re-index a series more than once. - if err := i.allowDuplicatesWriteError(err); err != nil { + if err := i.sanitizeAllowDuplicatesWriteError(err); err != nil { numErrors := numPending - int(result.NumSuccess) if partialError, ok := err.(*m3ninxindex.BatchPartialError); ok { // If it was a batch partial error we know exactly how many failed @@ -1234,7 +1234,7 @@ func (i *nsIndex) flushBlockSegment( continue } - err = i.allowDuplicatesWriteError(builder.InsertBatch(batch)) + err = i.sanitizeAllowDuplicatesWriteError(builder.InsertBatch(batch)) if err != nil { return err } @@ -1245,7 +1245,7 @@ func (i *nsIndex) flushBlockSegment( // Add last batch if remaining. if len(batch.Docs) > 0 { - err := i.allowDuplicatesWriteError(builder.InsertBatch(batch)) + err := i.sanitizeAllowDuplicatesWriteError(builder.InsertBatch(batch)) if err != nil { return err } @@ -1263,7 +1263,7 @@ func (i *nsIndex) flushBlockSegment( return preparedPersist.Persist(builder) } -func (i *nsIndex) allowDuplicatesWriteError(err error) error { +func (i *nsIndex) sanitizeAllowDuplicatesWriteError(err error) error { if err == nil { return nil } diff --git a/src/x/context/finalizeable_list_gen.go b/src/x/context/finalizeable_list_gen.go index 25d9a17cdc..0c989bc2ae 100644 --- a/src/x/context/finalizeable_list_gen.go +++ b/src/x/context/finalizeable_list_gen.go @@ -117,11 +117,6 @@ func (e *finalizeableElement) Prev() *finalizeableElement { return nil } -var ( - defaultFinalizeableElementsPoolLock sync.RWMutex - defaultFinalizeableElementsPool *finalizeableElementPool -) - // finalizeableList represents a doubly linked list. // The zero value for finalizeableList is an empty list ready to use. type finalizeableList struct { @@ -139,23 +134,23 @@ func (l *finalizeableList) Init() *finalizeableList { // Use a static pool at least, otherwise each time // we create a list with no pool we create a wholly // new pool of finalizeables (4096 of them). - defaultFinalizeableElementsPoolLock.RLock() + defaultFinalizeableElementsPoolOnce.Do(initFinalizeableElementsPool) l.Pool = defaultFinalizeableElementsPool - defaultFinalizeableElementsPoolLock.RUnlock() - if l.Pool == nil { - defaultFinalizeableElementsPoolLock.Lock() - if defaultFinalizeableElementsPool == nil { - // Still not set, allocate pool once. - defaultFinalizeableElementsPool = newFinalizeableElementPool(nil) - } - // Take ref to guaranteed allocated pool. - l.Pool = defaultFinalizeableElementsPool - defaultFinalizeableElementsPoolLock.Unlock() - } } return l } +var ( + defaultFinalizeableElementsPoolOnce sync.Once + defaultFinalizeableElementsPool *finalizeableElementPool +) + +// define as a static method so lambda alloc not required +// when passing function pointer to sync.Once.Do. +func initFinalizeableElementsPool() { + defaultFinalizeableElementsPool = newFinalizeableElementPool(nil) +} + // newFinalizeableList returns an initialized list. func newFinalizeableList(p *finalizeableElementPool) *finalizeableList { l := &finalizeableList{Pool: p} From bbfb16ab3de1763a1d3900004479a47e2dfb4fcf Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sat, 18 Jul 2020 10:37:20 -0400 Subject: [PATCH 15/38] Address feedback --- .../downsample/downsample_mock.go | 12 ++-- .../m3coordinator/downsample/downsampler.go | 49 +++++++------- .../downsample/downsampler_test.go | 34 ++++++---- .../downsample/metrics_appender.go | 65 +++++++++++++++---- .../m3coordinator/downsample/options.go | 57 +++++++++------- .../services/m3coordinator/downsample/tags.go | 26 +++++--- .../services/m3coordinator/ingest/write.go | 5 +- .../namespace/namespace_runtime_options.go | 2 +- src/dbnode/storage/index.go | 19 +----- src/dbnode/storage/shard.go | 23 +++---- src/m3ninx/index/segment/builder/builder.go | 59 +++++++++++------ src/query/server/query.go | 27 ++++---- 12 files changed, 231 insertions(+), 147 deletions(-) diff --git a/src/cmd/services/m3coordinator/downsample/downsample_mock.go b/src/cmd/services/m3coordinator/downsample/downsample_mock.go index 9899c1a94b..2d04b1a4c8 100644 --- a/src/cmd/services/m3coordinator/downsample/downsample_mock.go +++ b/src/cmd/services/m3coordinator/downsample/downsample_mock.go @@ -116,16 +116,16 @@ func (mr *MockMetricsAppenderMockRecorder) Finalize() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Finalize", reflect.TypeOf((*MockMetricsAppender)(nil).Finalize)) } -// Reset mocks base method -func (m *MockMetricsAppender) Reset() { +// NextMetric mocks base method +func (m *MockMetricsAppender) NextMetric() { m.ctrl.T.Helper() - m.ctrl.Call(m, "Reset") + m.ctrl.Call(m, "NextMetric") } -// Reset indicates an expected call of Reset -func (mr *MockMetricsAppenderMockRecorder) Reset() *gomock.Call { +// NextMetric indicates an expected call of NextMetric +func (mr *MockMetricsAppenderMockRecorder) NextMetric() *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Reset", reflect.TypeOf((*MockMetricsAppender)(nil).Reset)) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NextMetric", reflect.TypeOf((*MockMetricsAppender)(nil).NextMetric)) } // SamplesAppender mocks base method diff --git a/src/cmd/services/m3coordinator/downsample/downsampler.go b/src/cmd/services/m3coordinator/downsample/downsampler.go index 5213bf3820..ee19bd85bc 100644 --- a/src/cmd/services/m3coordinator/downsample/downsampler.go +++ b/src/cmd/services/m3coordinator/downsample/downsampler.go @@ -23,7 +23,6 @@ package downsample import ( "time" - "go.uber.org/zap" "go.uber.org/zap/zapcore" ) @@ -35,9 +34,14 @@ type Downsampler interface { // MetricsAppender is a metrics appender that can build a samples // appender, only valid to use with a single caller at a time. type MetricsAppender interface { + // NextMetric progresses to building the next metric. + NextMetric() + // AddTag adds a tag to the current metric being built. AddTag(name, value []byte) + // SamplesAppender returns a samples appender for the current + // metric built with the tags that have been set. SamplesAppender(opts SampleAppenderOptions) (SamplesAppenderResult, error) - Reset() + // Finalize finalizes the entire metrics appender for reuse. Finalize() } @@ -72,11 +76,9 @@ type SamplesAppender interface { } type downsampler struct { - opts DownsamplerOptions - agg agg - - debugLogging bool - logger *zap.Logger + opts DownsamplerOptions + agg agg + metricsAppenderOpts metricsAppenderOptions } type downsamplerOptions struct { @@ -95,24 +97,27 @@ func newDownsampler(opts downsamplerOptions) (*downsampler, error) { debugLogging = true } + metricsAppenderOpts := metricsAppenderOptions{ + agg: opts.agg.aggregator, + clientRemote: opts.agg.clientRemote, + defaultStagedMetadatasProtos: opts.agg.defaultStagedMetadatasProtos, + clockOpts: opts.agg.clockOpts, + tagEncoderPool: opts.agg.pools.tagEncoderPool, + matcher: opts.agg.matcher, + metricTagsIteratorPool: opts.agg.pools.metricTagsIteratorPool, + debugLogging: debugLogging, + logger: logger, + } + return &downsampler{ - opts: opts.opts, - agg: opts.agg, - debugLogging: debugLogging, - logger: logger, + opts: opts.opts, + agg: opts.agg, + metricsAppenderOpts: metricsAppenderOpts, }, nil } func (d *downsampler) NewMetricsAppender() (MetricsAppender, error) { - return newMetricsAppender(metricsAppenderOptions{ - agg: d.agg.aggregator, - clientRemote: d.agg.clientRemote, - defaultStagedMetadatasProtos: d.agg.defaultStagedMetadatasProtos, - clockOpts: d.agg.clockOpts, - tagEncoder: d.agg.pools.tagEncoderPool.Get(), - matcher: d.agg.matcher, - metricTagsIteratorPool: d.agg.pools.metricTagsIteratorPool, - debugLogging: d.debugLogging, - logger: d.logger, - }), nil + metricsAppender := d.agg.pools.metricsAppenderPool.Get() + metricsAppender.reset(d.metricsAppenderOpts) + return metricsAppender, nil } diff --git a/src/cmd/services/m3coordinator/downsample/downsampler_test.go b/src/cmd/services/m3coordinator/downsample/downsampler_test.go index 6146f90650..833bc3f8c7 100644 --- a/src/cmd/services/m3coordinator/downsample/downsampler_test.go +++ b/src/cmd/services/m3coordinator/downsample/downsampler_test.go @@ -989,7 +989,8 @@ func testDownsamplerAggregationIngest( opts = *testOpts.sampleAppenderOpts } for _, metric := range testCounterMetrics { - appender.Reset() + appender.NextMetric() + for name, value := range metric.tags { appender.AddTag([]byte(name), []byte(value)) } @@ -1016,7 +1017,8 @@ func testDownsamplerAggregationIngest( } } for _, metric := range testGaugeMetrics { - appender.Reset() + appender.NextMetric() + for name, value := range metric.tags { appender.AddTag([]byte(name), []byte(value)) } @@ -1116,13 +1118,20 @@ func newTestDownsampler(t *testing.T, opts testDownsamplerOptions) testDownsampl tagEncoderOptions := serialize.NewTagEncoderOptions() tagDecoderOptions := serialize.NewTagDecoderOptions(serialize.TagDecoderOptionsConfig{}) tagEncoderPoolOptions := pool.NewObjectPoolOptions(). + SetSize(2). SetInstrumentOptions(instrumentOpts. SetMetricsScope(instrumentOpts.MetricsScope(). SubScope("tag-encoder-pool"))) tagDecoderPoolOptions := pool.NewObjectPoolOptions(). + SetSize(2). SetInstrumentOptions(instrumentOpts. SetMetricsScope(instrumentOpts.MetricsScope(). SubScope("tag-decoder-pool"))) + metricsAppenderPoolOptions := pool.NewObjectPoolOptions(). + SetSize(2). + SetInstrumentOptions(instrumentOpts. + SetMetricsScope(instrumentOpts.MetricsScope(). + SubScope("metrics-appender-pool"))) var cfg Configuration if opts.remoteClientMock != nil { @@ -1137,16 +1146,17 @@ func newTestDownsampler(t *testing.T, opts testDownsamplerOptions) testDownsampl } instance, err := cfg.NewDownsampler(DownsamplerOptions{ - Storage: storage, - ClusterClient: clusterclient.NewMockClient(gomock.NewController(t)), - RulesKVStore: rulesKVStore, - AutoMappingRules: opts.autoMappingRules, - ClockOptions: clockOpts, - InstrumentOptions: instrumentOpts, - TagEncoderOptions: tagEncoderOptions, - TagDecoderOptions: tagDecoderOptions, - TagEncoderPoolOptions: tagEncoderPoolOptions, - TagDecoderPoolOptions: tagDecoderPoolOptions, + Storage: storage, + ClusterClient: clusterclient.NewMockClient(gomock.NewController(t)), + RulesKVStore: rulesKVStore, + AutoMappingRules: opts.autoMappingRules, + ClockOptions: clockOpts, + InstrumentOptions: instrumentOpts, + TagEncoderOptions: tagEncoderOptions, + TagDecoderOptions: tagDecoderOptions, + TagEncoderPoolOptions: tagEncoderPoolOptions, + TagDecoderPoolOptions: tagDecoderPoolOptions, + MetricsAppenderPoolOptions: metricsAppenderPoolOptions, }) require.NoError(t, err) diff --git a/src/cmd/services/m3coordinator/downsample/metrics_appender.go b/src/cmd/services/m3coordinator/downsample/metrics_appender.go index 0bc1cf5f7c..aef17e76e2 100644 --- a/src/cmd/services/m3coordinator/downsample/metrics_appender.go +++ b/src/cmd/services/m3coordinator/downsample/metrics_appender.go @@ -34,6 +34,7 @@ import ( "github.com/m3db/m3/src/metrics/metadata" "github.com/m3db/m3/src/metrics/policy" "github.com/m3db/m3/src/x/clock" + "github.com/m3db/m3/src/x/pool" "github.com/m3db/m3/src/x/serialize" "github.com/golang/protobuf/jsonpb" @@ -41,14 +42,39 @@ import ( "go.uber.org/zap/zapcore" ) +type metricsAppenderPool struct { + pool pool.ObjectPool +} + +func newMetricsAppenderPool(opts pool.ObjectPoolOptions) *metricsAppenderPool { + p := &metricsAppenderPool{ + pool: pool.NewObjectPool(opts), + } + p.pool.Init(func() interface{} { + return newMetricsAppender(p) + }) + return p +} + +func (p *metricsAppenderPool) Get() *metricsAppender { + return p.pool.Get().(*metricsAppender) +} + +func (p *metricsAppenderPool) Put(v *metricsAppender) { + p.pool.Put(v) +} + type metricsAppender struct { metricsAppenderOptions + pool *metricsAppenderPool + tags *tags multiSamplesAppender *multiSamplesAppender curr metadata.StagedMetadata defaultStagedMetadatasCopies []metadata.StagedMetadatas mappingRuleStoragePolicies []policy.StoragePolicy + tagEncoder serialize.TagEncoder } // metricsAppenderOptions will have one of agg or clientRemote set. @@ -57,8 +83,8 @@ type metricsAppenderOptions struct { clientRemote client.Client defaultStagedMetadatasProtos []metricpb.StagedMetadatas - tagEncoder serialize.TagEncoder matcher matcher.Matcher + tagEncoderPool serialize.TagEncoderPool metricTagsIteratorPool serialize.MetricTagsIteratorPool clockOpts clock.Options @@ -66,14 +92,31 @@ type metricsAppenderOptions struct { logger *zap.Logger } -func newMetricsAppender(opts metricsAppenderOptions) *metricsAppender { - stagedMetadatasCopies := make([]metadata.StagedMetadatas, - len(opts.defaultStagedMetadatasProtos)) +func newMetricsAppender(pool *metricsAppenderPool) *metricsAppender { return &metricsAppender{ - metricsAppenderOptions: opts, - tags: newTags(), - multiSamplesAppender: newMultiSamplesAppender(), - defaultStagedMetadatasCopies: stagedMetadatasCopies, + pool: pool, + tags: newTags(), + multiSamplesAppender: newMultiSamplesAppender(), + } +} + +// reset is called when pulled from the pool. +func (a *metricsAppender) reset(opts metricsAppenderOptions) { + a.metricsAppenderOptions = opts + if a.tagEncoder == nil { + a.tagEncoder = opts.tagEncoderPool.Get() + } + + // Make sure a.defaultStagedMetadatasCopies is right length. + capRequired := len(opts.defaultStagedMetadatasProtos) + if cap(a.defaultStagedMetadatasCopies) < capRequired { + // Too short, reallocate. + slice := make([]metadata.StagedMetadatas, capRequired) + a.defaultStagedMetadatasCopies = slice + } else { + // Has enough capacity, take subslice. + slice := a.defaultStagedMetadatasCopies[:capRequired] + a.defaultStagedMetadatasCopies = slice } } @@ -317,14 +360,14 @@ func (a *metricsAppender) debugLogMatch(str string, opts debugLogMatchOptions) { a.logger.Debug(str, fields...) } -func (a *metricsAppender) Reset() { +func (a *metricsAppender) NextMetric() { a.tags.names = a.tags.names[:0] a.tags.values = a.tags.values[:0] } func (a *metricsAppender) Finalize() { - a.tagEncoder.Finalize() - a.tagEncoder = nil + // Return to pool. + a.pool.Put(a) } func stagedMetadatasLogField(sm metadata.StagedMetadatas) zapcore.Field { diff --git a/src/cmd/services/m3coordinator/downsample/options.go b/src/cmd/services/m3coordinator/downsample/options.go index 5b16551a97..a51285adf8 100644 --- a/src/cmd/services/m3coordinator/downsample/options.go +++ b/src/cmd/services/m3coordinator/downsample/options.go @@ -83,34 +83,36 @@ const ( var ( numShards = runtime.NumCPU() - errNoStorage = errors.New("dynamic downsampling enabled with storage not set") - errNoClusterClient = errors.New("dynamic downsampling enabled with cluster client not set") - errNoRulesStore = errors.New("dynamic downsampling enabled with rules store not set") - errNoClockOptions = errors.New("dynamic downsampling enabled with clock options not set") - errNoInstrumentOptions = errors.New("dynamic downsampling enabled with instrument options not set") - errNoTagEncoderOptions = errors.New("dynamic downsampling enabled with tag encoder options not set") - errNoTagDecoderOptions = errors.New("dynamic downsampling enabled with tag decoder options not set") - errNoTagEncoderPoolOptions = errors.New("dynamic downsampling enabled with tag encoder pool options not set") - errNoTagDecoderPoolOptions = errors.New("dynamic downsampling enabled with tag decoder pool options not set") - errRollupRuleNoTransforms = errors.New("rollup rule has no transforms set") + errNoStorage = errors.New("downsampling enabled with storage not set") + errNoClusterClient = errors.New("downsampling enabled with cluster client not set") + errNoRulesStore = errors.New("downsampling enabled with rules store not set") + errNoClockOptions = errors.New("downsampling enabled with clock options not set") + errNoInstrumentOptions = errors.New("downsampling enabled with instrument options not set") + errNoTagEncoderOptions = errors.New("downsampling enabled with tag encoder options not set") + errNoTagDecoderOptions = errors.New("downsampling enabled with tag decoder options not set") + errNoTagEncoderPoolOptions = errors.New("downsampling enabled with tag encoder pool options not set") + errNoTagDecoderPoolOptions = errors.New("downsampling enabled with tag decoder pool options not set") + errNoMetricsAppenderPoolOptions = errors.New("downsampling enabled with metrics appender pool options not set") + errRollupRuleNoTransforms = errors.New("rollup rule has no transforms set") ) // DownsamplerOptions is a set of required downsampler options. type DownsamplerOptions struct { - Storage storage.Storage - StorageFlushConcurrency int - ClusterClient clusterclient.Client - RulesKVStore kv.Store - AutoMappingRules []AutoMappingRule - NameTag string - ClockOptions clock.Options - InstrumentOptions instrument.Options - TagEncoderOptions serialize.TagEncoderOptions - TagDecoderOptions serialize.TagDecoderOptions - TagEncoderPoolOptions pool.ObjectPoolOptions - TagDecoderPoolOptions pool.ObjectPoolOptions - OpenTimeout time.Duration - TagOptions models.TagOptions + Storage storage.Storage + StorageFlushConcurrency int + ClusterClient clusterclient.Client + RulesKVStore kv.Store + AutoMappingRules []AutoMappingRule + NameTag string + ClockOptions clock.Options + InstrumentOptions instrument.Options + TagEncoderOptions serialize.TagEncoderOptions + TagDecoderOptions serialize.TagDecoderOptions + TagEncoderPoolOptions pool.ObjectPoolOptions + TagDecoderPoolOptions pool.ObjectPoolOptions + OpenTimeout time.Duration + TagOptions models.TagOptions + MetricsAppenderPoolOptions pool.ObjectPoolOptions } // AutoMappingRule is a mapping rule to apply to metrics. @@ -169,6 +171,9 @@ func (o DownsamplerOptions) validate() error { if o.TagDecoderPoolOptions == nil { return errNoTagDecoderPoolOptions } + if o.MetricsAppenderPoolOptions == nil { + return errNoMetricsAppenderPoolOptions + } return nil } @@ -874,6 +879,7 @@ type aggPools struct { tagEncoderPool serialize.TagEncoderPool tagDecoderPool serialize.TagDecoderPool metricTagsIteratorPool serialize.MetricTagsIteratorPool + metricsAppenderPool *metricsAppenderPool } func (o DownsamplerOptions) newAggregatorPools() aggPools { @@ -889,10 +895,13 @@ func (o DownsamplerOptions) newAggregatorPools() aggPools { o.TagDecoderPoolOptions) metricTagsIteratorPool.Init() + metricsAppenderPool := newMetricsAppenderPool(o.MetricsAppenderPoolOptions) + return aggPools{ tagEncoderPool: tagEncoderPool, tagDecoderPool: tagDecoderPool, metricTagsIteratorPool: metricTagsIteratorPool, + metricsAppenderPool: metricsAppenderPool, } } diff --git a/src/cmd/services/m3coordinator/downsample/tags.go b/src/cmd/services/m3coordinator/downsample/tags.go index b9c94f60df..c0a3a1c3ec 100644 --- a/src/cmd/services/m3coordinator/downsample/tags.go +++ b/src/cmd/services/m3coordinator/downsample/tags.go @@ -33,11 +33,13 @@ const ( ) type tags struct { - names [][]byte - values [][]byte - idx int - nameBuf []byte - valueBuf []byte + names [][]byte + values [][]byte + idx int + nameBuf []byte + valueBuf []byte + reuseableTagName *ident.ReuseableBytesID + reuseableTagValue *ident.ReuseableBytesID } // Ensure tags implements TagIterator and sort Interface @@ -48,9 +50,11 @@ var ( func newTags() *tags { return &tags{ - names: make([][]byte, 0, initAllocTagsSliceCapacity), - values: make([][]byte, 0, initAllocTagsSliceCapacity), - idx: -1, + names: make([][]byte, 0, initAllocTagsSliceCapacity), + values: make([][]byte, 0, initAllocTagsSliceCapacity), + idx: -1, + reuseableTagName: ident.NewReuseableBytesID(), + reuseableTagValue: ident.NewReuseableBytesID(), } } @@ -90,9 +94,11 @@ func (t *tags) CurrentIndex() int { func (t *tags) Current() ident.Tag { t.nameBuf = append(t.nameBuf[:0], t.names[t.idx]...) t.valueBuf = append(t.valueBuf[:0], t.values[t.idx]...) + t.reuseableTagName.Reset(t.nameBuf) + t.reuseableTagValue.Reset(t.valueBuf) return ident.Tag{ - Name: ident.BytesID(t.nameBuf), - Value: ident.BytesID(t.valueBuf), + Name: t.reuseableTagName, + Value: t.reuseableTagValue, } } diff --git a/src/cmd/services/m3coordinator/ingest/write.go b/src/cmd/services/m3coordinator/ingest/write.go index 975670c574..cd658a9f78 100644 --- a/src/cmd/services/m3coordinator/ingest/write.go +++ b/src/cmd/services/m3coordinator/ingest/write.go @@ -229,8 +229,6 @@ func (d *downsamplerAndWriter) writeToDownsampler( unit xtime.Unit, overrides WriteOptions, ) (bool, error) { - // TODO(rartoul): MetricsAppender has a Finalize() method, but it does not actually reuse many - // resources. If we can pool this properly we can get a nice speedup. appender, err := d.downsampler.NewMetricsAppender() if err != nil { return false, err @@ -432,7 +430,8 @@ func (d *downsamplerAndWriter) writeAggregatedBatch( defer appender.Finalize() for iter.Next() { - appender.Reset() + appender.NextMetric() + value := iter.Current() for _, tag := range value.Tags.Tags { appender.AddTag(tag.Name, tag.Value) diff --git a/src/dbnode/namespace/namespace_runtime_options.go b/src/dbnode/namespace/namespace_runtime_options.go index 6c2918d58e..8531cc6eaa 100644 --- a/src/dbnode/namespace/namespace_runtime_options.go +++ b/src/dbnode/namespace/namespace_runtime_options.go @@ -103,7 +103,7 @@ type RuntimeOptionsListener interface { // runtimeOptions should always use pointer value types for it's options // and provide a "ValueOrDefault()" method so that we can be sure whether -// the options are all defaults or not with the "AllDefaults" method. +// the options are all defaults or not with the "IsDefault" method. type runtimeOptions struct { writeIndexingPerCPUConcurrency *float64 flushIndexingPerCPUConcurrency *float64 diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 97773890c9..2b2f88ca46 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -1165,24 +1165,9 @@ func (i *nsIndex) flushBlockSegment( var ( ctx = i.opts.ContextPool().Get() - docsPool = i.opts.IndexOptions().DocumentArrayPool() - docs = docsPool.Get() - batchSize = cap(docs) - batch = m3ninxindex.Batch{Docs: docs[:0], AllowPartialUpdates: true} - ) - if batchSize == 0 { + batch = m3ninxindex.Batch{AllowPartialUpdates: true} batchSize = defaultFlushDocsBatchSize - } - defer func() { - // Memset optimization to clear refs before returning to pool. - var docZeroed doc.Document - for i := range batch.Docs { - batch.Docs[i] = docZeroed - } - // Return to pool. - docsPool.Put(batch.Docs) - }() - + ) for _, shard := range shards { var ( first = true diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 80d2377b6f..d2577f77b8 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1031,26 +1031,27 @@ func (s *dbShard) SeriesReadWriteRef( } // NB(r): Insert then wait so caller has access to the series - // immediately, otherwise calls to LoadBlock(..) etc on the series itself - // may have no effect if a collision with the same series + // immediately, otherwise calls to LoadBlock(..) etc on the series + // itself may have no effect if a collision with the same series // being put in the insert queue may cause a block to be loaded to a // series which gets discarded. at := s.nowFn() - result, err := s.insertSeriesAsyncBatched(id, tags, dbShardInsertAsyncOptions{ - hasPendingIndexing: opts.ReverseIndex, - pendingIndex: dbShardPendingIndex{ - timestamp: at, - enqueuedAt: at, - }, - }) + result, err := s.insertSeriesAsyncBatched(id, tags, + dbShardInsertAsyncOptions{ + hasPendingIndexing: opts.ReverseIndex, + pendingIndex: dbShardPendingIndex{ + timestamp: at, + enqueuedAt: at, + }, + }) if err != nil { return SeriesReadWriteRef{}, err } - // Wait for the insert to be batched together and inserted + // Wait for the insert to be batched together and inserted. result.wg.Wait() - // Retrieve the inserted entry + // Retrieve the inserted entry. entry, err = s.writableSeries(id, tags) if err != nil { return SeriesReadWriteRef{}, err diff --git a/src/m3ninx/index/segment/builder/builder.go b/src/m3ninx/index/segment/builder/builder.go index c0305b5c07..cc39df3ef0 100644 --- a/src/m3ninx/index/segment/builder/builder.go +++ b/src/m3ninx/index/segment/builder/builder.go @@ -252,7 +252,7 @@ func (b *builder) SetIndexConcurrency(value int) { for _, fields := range existingUniqueFields { for _, field := range fields { // Calculate the new shard for the field. - newShard := b.calculateShard(field.field) + newShard := b.calculateShardWithRLock(field.field) // Append to the correct shard. b.shardedFields.uniqueFields[newShard] = @@ -269,7 +269,7 @@ func (b *builder) SetIndexConcurrency(value int) { terms := entry.Value() // Calculate the new shard for the field. - newShard := b.calculateShard(field) + newShard := b.calculateShardWithRLock(field) // Set with new correct shard. b.shardedFields.fields.ShardedSetUnsafe(newShard, field, @@ -287,6 +287,9 @@ func (b *builder) IndexConcurrency() int { } func (b *builder) Reset(offset postings.ID) { + b.status.Lock() + defer b.status.Unlock() + b.offset = offset // Reset the documents slice. @@ -303,22 +306,23 @@ func (b *builder) Reset(offset postings.ID) { b.shardedFields.fields.ResetTermsSets() // Reset the unique fields slice + var emptyField uniqueField for i, shardUniqueFields := range b.shardedFields.uniqueFields { for i := range shardUniqueFields { - shardUniqueFields[i] = uniqueField{} + shardUniqueFields[i] = emptyField } b.shardedFields.uniqueFields[i] = shardUniqueFields[:0] } } func (b *builder) Insert(d doc.Document) ([]byte, error) { - b.status.RLock() - defer b.status.RUnlock() + b.status.Lock() + defer b.status.Unlock() // Use a preallocated slice to make insert able to avoid alloc // a slice to call insert batch with. b.batchSizeOne.Docs[0] = d - err := b.insertBatchWithRLock(b.batchSizeOne) + err := b.insertBatchWithLock(b.batchSizeOne) if err != nil { if errs := err.Errs(); len(errs) == 1 { // Return concrete error instead of the batch partial error. @@ -332,8 +336,8 @@ func (b *builder) Insert(d doc.Document) ([]byte, error) { } func (b *builder) InsertBatch(batch index.Batch) error { - b.status.RLock() - defer b.status.RUnlock() + b.status.Lock() + defer b.status.Unlock() if b.status.closed { return errClosed @@ -342,7 +346,7 @@ func (b *builder) InsertBatch(batch index.Batch) error { // NB(r): This switch is required or else *index.BatchPartialError // is returned as a non-nil wrapped "error" even though it is not // an error and underlying error is nil. - if err := b.insertBatchWithRLock(batch); err != nil { + if err := b.insertBatchWithLock(batch); err != nil { return err } return nil @@ -356,7 +360,7 @@ func (b *builder) resetShardedJobs() { } } -func (b *builder) insertBatchWithRLock(batch index.Batch) *index.BatchPartialError { +func (b *builder) insertBatchWithLock(batch index.Batch) *index.BatchPartialError { // NB(r): This is all kept in a single method to make the // insertion path avoid too much function call overhead. wg := &sync.WaitGroup{} @@ -406,9 +410,9 @@ func (b *builder) insertBatchWithRLock(batch index.Batch) *index.BatchPartialErr // Index the terms. for _, f := range d.Fields { - b.queueIndexJobEntry(wg, postings.ID(postingsListID), f, i, batchErr) + b.queueIndexJobEntryWithLock(wg, postings.ID(postingsListID), f, i, batchErr) } - b.queueIndexJobEntry(wg, postings.ID(postingsListID), doc.Field{ + b.queueIndexJobEntryWithLock(wg, postings.ID(postingsListID), doc.Field{ Name: doc.IDReservedFieldName, Value: d.ID, }, i, batchErr) @@ -417,7 +421,7 @@ func (b *builder) insertBatchWithRLock(batch index.Batch) *index.BatchPartialErr // Enqueue any partially filled sharded jobs. for shard := 0; shard < b.concurrency; shard++ { if b.shardedJobs[shard].usedEntries > 0 { - b.flushShardedIndexJob(shard, wg, batchErr) + b.flushShardedIndexJobWithLock(shard, wg, batchErr) } } @@ -430,14 +434,14 @@ func (b *builder) insertBatchWithRLock(batch index.Batch) *index.BatchPartialErr return nil } -func (b *builder) queueIndexJobEntry( +func (b *builder) queueIndexJobEntryWithLock( wg *sync.WaitGroup, id postings.ID, field doc.Field, docIdx int, batchErr *index.BatchPartialError, ) { - shard := b.calculateShard(field.Name) + shard := b.calculateShardWithRLock(field.Name) entryIndex := b.shardedJobs[shard].usedEntries b.shardedJobs[shard].usedEntries++ b.shardedJobs[shard].entries[entryIndex].id = id @@ -450,13 +454,13 @@ func (b *builder) queueIndexJobEntry( } // Ready to flush this job since all entries are used. - b.flushShardedIndexJob(shard, wg, batchErr) + b.flushShardedIndexJobWithLock(shard, wg, batchErr) // Reset for reuse. b.shardedJobs[shard] = indexJob{} } -func (b *builder) flushShardedIndexJob( +func (b *builder) flushShardedIndexJobWithLock( shard int, wg *sync.WaitGroup, batchErr *index.BatchPartialError, @@ -473,17 +477,23 @@ func (b *builder) flushShardedIndexJob( globalIndexWorkers.indexJob(b.shardedJobs[shard]) } -func (b *builder) calculateShard(field []byte) int { +func (b *builder) calculateShardWithRLock(field []byte) int { return int(xxhash.Sum64(field) % uint64(b.concurrency)) } func (b *builder) AllDocs() (index.IDDocIterator, error) { + b.status.RLock() + defer b.status.RUnlock() + rangeIter := postings.NewRangeIterator(b.offset, b.offset+postings.ID(len(b.docs))) return index.NewIDDocIterator(b, rangeIter), nil } func (b *builder) Doc(id postings.ID) (doc.Document, error) { + b.status.RLock() + defer b.status.RUnlock() + idx := int(id - b.offset) if idx < 0 || idx >= len(b.docs) { return doc.Document{}, errDocNotFound @@ -493,6 +503,9 @@ func (b *builder) Doc(id postings.ID) (doc.Document, error) { } func (b *builder) Docs() []doc.Document { + b.status.RLock() + defer b.status.RUnlock() + return b.docs } @@ -505,11 +518,18 @@ func (b *builder) TermsIterable() segment.TermsIterable { } func (b *builder) FieldsPostingsList() (segment.FieldsPostingsListIterator, error) { + b.status.RLock() + defer b.status.RUnlock() + return newOrderedFieldsPostingsListIter(b.shardedFields.uniqueFields), nil } func (b *builder) Terms(field []byte) (segment.TermsIterator, error) { - shard := b.calculateShard(field) + // NB(r): Need write lock since sort if required below. + b.status.Lock() + defer b.status.Unlock() + + shard := b.calculateShardWithRLock(field) terms, ok := b.shardedFields.fields.ShardedGet(shard, field) if !ok { return nil, fmt.Errorf("field not found: %s", string(field)) @@ -525,6 +545,7 @@ func (b *builder) Terms(field []byte) (segment.TermsIterator, error) { func (b *builder) Close() error { b.status.Lock() defer b.status.Unlock() + b.status.closed = true // Indiciate we could possibly spin down workers if no builders open. globalIndexWorkers.unregisterBuilder() diff --git a/src/query/server/query.go b/src/query/server/query.go index fbbdedd40f..ae63909c20 100644 --- a/src/query/server/query.go +++ b/src/query/server/query.go @@ -701,19 +701,24 @@ func newDownsampler( SetInstrumentOptions(instrumentOpts. SetMetricsScope(instrumentOpts.MetricsScope(). SubScope("tag-decoder-pool"))) + metricsAppenderPoolOptions := pool.NewObjectPoolOptions(). + SetInstrumentOptions(instrumentOpts. + SetMetricsScope(instrumentOpts.MetricsScope(). + SubScope("metrics-appender-pool"))) downsampler, err := cfg.NewDownsampler(downsample.DownsamplerOptions{ - Storage: storage, - ClusterClient: clusterManagementClient, - RulesKVStore: kvStore, - AutoMappingRules: autoMappingRules, - ClockOptions: clock.NewOptions(), - InstrumentOptions: instrumentOpts, - TagEncoderOptions: tagEncoderOptions, - TagDecoderOptions: tagDecoderOptions, - TagEncoderPoolOptions: tagEncoderPoolOptions, - TagDecoderPoolOptions: tagDecoderPoolOptions, - TagOptions: tagOptions, + Storage: storage, + ClusterClient: clusterManagementClient, + RulesKVStore: kvStore, + AutoMappingRules: autoMappingRules, + ClockOptions: clock.NewOptions(), + InstrumentOptions: instrumentOpts, + TagEncoderOptions: tagEncoderOptions, + TagDecoderOptions: tagDecoderOptions, + TagEncoderPoolOptions: tagEncoderPoolOptions, + TagDecoderPoolOptions: tagDecoderPoolOptions, + TagOptions: tagOptions, + MetricsAppenderPoolOptions: metricsAppenderPoolOptions, }) if err != nil { return nil, fmt.Errorf("unable to create downsampler: %v", err) From 5e3e9c6e66741628966160e12746cc5281ef0b72 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sat, 18 Jul 2020 12:02:58 -0400 Subject: [PATCH 16/38] Fix concurrency issue --- ...ommitlog_bootstrap_index_perf_speed_test.go | 8 ++++---- src/dbnode/storage/index.go | 4 +++- src/dbnode/storage/shard.go | 18 +++++++++++++++--- src/m3ninx/index/segment/builder/builder.go | 2 +- .../index/segment/builder/bytes_slice_iter.go | 3 +++ src/m3ninx/index/segment/builder/terms.go | 4 ++++ 6 files changed, 30 insertions(+), 9 deletions(-) diff --git a/src/dbnode/integration/commitlog_bootstrap_index_perf_speed_test.go b/src/dbnode/integration/commitlog_bootstrap_index_perf_speed_test.go index 66154a1278..5d056e9750 100644 --- a/src/dbnode/integration/commitlog_bootstrap_index_perf_speed_test.go +++ b/src/dbnode/integration/commitlog_bootstrap_index_perf_speed_test.go @@ -145,21 +145,21 @@ func TestCommitLogIndexPerfSpeedBootstrap(t *testing.T) { require.NoError(t, err) require.NoError(t, commitLog.Open()) - // NB(r): Write points using no up front series metadata or point - // generation so that the memory usage is constant during the write phase ctx := context.NewContext() defer ctx.Close() + shardSet := setup.ShardSet() idPrefix := "test.id.test.id.test.id.test.id.test.id.test.id.test.id.test.id" idPrefixBytes := []byte(idPrefix) - checkedBytes := checked.NewBytes(nil, nil) - seriesID := ident.BinaryID(checkedBytes) numBytes := make([]byte, 8) numHexBytes := make([]byte, hex.EncodedLen(len(numBytes))) tagEncoderPool := commitLogOpts.FilesystemOptions().TagEncoderPool() tagSliceIter := ident.NewTagsIterator(ident.Tags{}) for i := 0; i < numPoints; i++ { for j := 0; j < numSeries; j++ { + checkedBytes := checked.NewBytes(nil, nil) + seriesID := ident.BinaryID(checkedBytes) + // Write the ID prefix checkedBytes.Resize(0) checkedBytes.AppendAll(idPrefixBytes) diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 2b2f88ca46..d056841132 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -1164,10 +1164,12 @@ func (i *nsIndex) flushBlockSegment( builder.Reset(0) var ( - ctx = i.opts.ContextPool().Get() batch = m3ninxindex.Batch{AllowPartialUpdates: true} batchSize = defaultFlushDocsBatchSize ) + ctx := i.opts.ContextPool().Get() + defer ctx.Close() + for _, shard := range shards { var ( first = true diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index d2577f77b8..ef28b25b9f 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1051,12 +1051,24 @@ func (s *dbShard) SeriesReadWriteRef( // Wait for the insert to be batched together and inserted. result.wg.Wait() - // Retrieve the inserted entry. - entry, err = s.writableSeries(id, tags) + // Guaranteed now to be available in the shard map, otherwise + // an invariant is not held. + s.RLock() + defer s.RUnlock() + + entry, _, err = s.lookupEntryWithLock(result.copiedID) if err != nil { + instrument.EmitAndLogInvariantViolation(s.opts.InstrumentOptions(), + func(l *zap.Logger) { + l.Error("read write ref inserted series not found", zap.Error(err)) + }) return SeriesReadWriteRef{}, err } + // Increment before releasing lock so won't be expired until operation + // complete by calling ReleaseReadWriteRef. + entry.IncrementReaderWriterCount() + return SeriesReadWriteRef{ Series: entry.Series, Shard: s.shard, @@ -1306,7 +1318,7 @@ func (s *dbShard) insertSeriesAsyncBatched( }) return insertAsyncResult{ wg: wg, - // Make sure to return the copied ID from the new series + // Make sure to return the copied ID from the new series. copiedID: entry.Series.ID(), entry: entry, }, err diff --git a/src/m3ninx/index/segment/builder/builder.go b/src/m3ninx/index/segment/builder/builder.go index cc39df3ef0..bcda153c63 100644 --- a/src/m3ninx/index/segment/builder/builder.go +++ b/src/m3ninx/index/segment/builder/builder.go @@ -553,7 +553,7 @@ func (b *builder) Close() error { } var ( - sortConcurrencyLock sync.Mutex + sortConcurrencyLock sync.RWMutex ) // SetSortConcurrency sets the sort concurrency for when diff --git a/src/m3ninx/index/segment/builder/bytes_slice_iter.go b/src/m3ninx/index/segment/builder/bytes_slice_iter.go index b7076f4eaf..f627388d89 100644 --- a/src/m3ninx/index/segment/builder/bytes_slice_iter.go +++ b/src/m3ninx/index/segment/builder/bytes_slice_iter.go @@ -52,7 +52,10 @@ func newOrderedFieldsPostingsListIter( maybeUnorderedFields [][]uniqueField, ) *orderedFieldsPostingsListIter { sortable := &sortableSliceOfSliceOfUniqueFieldsAsc{data: maybeUnorderedFields} + // NB(r): See SetSortConcurrency why this RLock is required. + sortConcurrencyLock.RLock() sorts.ByBytes(sortable) + sortConcurrencyLock.RUnlock() return &orderedFieldsPostingsListIter{ currentIdx: -1, backingSlices: sortable, diff --git a/src/m3ninx/index/segment/builder/terms.go b/src/m3ninx/index/segment/builder/terms.go index 5fb10a04a1..4fde080672 100644 --- a/src/m3ninx/index/segment/builder/terms.go +++ b/src/m3ninx/index/segment/builder/terms.go @@ -96,7 +96,11 @@ func (t *terms) sortIfRequired() { return } + // NB(r): See SetSortConcurrency why this RLock is required. + sortConcurrencyLock.RLock() sorts.ByBytes(t) + sortConcurrencyLock.RUnlock() + t.uniqueTermsIsSorted = true } From a687e0f7a5bbe406fb33835e48a086c5885155d0 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sat, 18 Jul 2020 12:27:39 -0400 Subject: [PATCH 17/38] Fix tests --- Makefile | 2 +- go.mod | 2 +- go.sum | 4 +- src/dbnode/storage/index/block_bench_test.go | 5 +- src/dbnode/storage/index/block_test.go | 137 +++++++++++++------ src/dbnode/storage/index/index_mock.go | 12 ++ src/dbnode/storage/storage_mock.go | 28 ++++ src/m3ninx/index/segment/segment_mock.go | 78 +++++++++++ src/x/context/finalizeable_list_gen.go | 12 +- src/x/generated-source-files.mk | 2 +- src/x/generics/list/list.go | 19 ++- tools.go | 2 +- 12 files changed, 243 insertions(+), 60 deletions(-) diff --git a/Makefile b/Makefile index 88b3059761..3154f1a593 100644 --- a/Makefile +++ b/Makefile @@ -187,7 +187,7 @@ install-tools: GOBIN=$(tools_bin_path) go install github.com/mauricelam/genny GOBIN=$(tools_bin_path) go install github.com/mjibson/esc GOBIN=$(tools_bin_path) go install github.com/pointlander/peg - GOBIN=$(tools_bin_path) go install github.com/prateek/gorename + GOBIN=$(tools_bin_path) go install github.com/robskillington/gorename GOBIN=$(tools_bin_path) go install github.com/rakyll/statik GOBIN=$(tools_bin_path) go install github.com/garethr/kubeval diff --git a/go.mod b/go.mod index 3fadc10cae..9baf5f79c2 100644 --- a/go.mod +++ b/go.mod @@ -82,13 +82,13 @@ require ( github.com/pointlander/jetset v1.0.0 // indirect github.com/pointlander/peg v1.0.0 github.com/prashantv/protectmem v0.0.0-20171002184600-e20412882b3a // indirect - github.com/prateek/gorename v0.0.0-20180424020013-52c7307cddd2 github.com/prometheus/client_golang v1.5.1 github.com/prometheus/common v0.9.1 github.com/prometheus/prometheus v1.8.2-0.20200420081721-18254838fbe2 github.com/rakyll/statik v0.1.6 github.com/remeh/sizedwaitgroup v1.0.0 // indirect github.com/rhysd/go-github-selfupdate v1.2.2 // indirect + github.com/robskillington/gorename v0.0.0-20180424020013-52c7307cddd2 github.com/russross/blackfriday v2.0.0+incompatible github.com/rveen/ogdl v0.0.0-20200522080342-eeeda1a978e7 // indirect github.com/satori/go.uuid v1.2.0 diff --git a/go.sum b/go.sum index 8c9347929b..fc3b9cee8e 100644 --- a/go.sum +++ b/go.sum @@ -604,8 +604,6 @@ github.com/pointlander/peg v1.0.0/go.mod h1:WJTMcgeWYr6fZz4CwHnY1oWZCXew8GWCF93F github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/prashantv/protectmem v0.0.0-20171002184600-e20412882b3a h1:AA9vgIBDjMHPC2McaGPojgV2dcI78ZC0TLNhYCXEKH8= github.com/prashantv/protectmem v0.0.0-20171002184600-e20412882b3a/go.mod h1:lzZQ3Noex5pfAy7mkAeCjcBDteYU85uWWnJ/y6gKU8k= -github.com/prateek/gorename v0.0.0-20180424020013-52c7307cddd2 h1:/pJs9wFXnmhD12W+dnwoNJXPtLsxa78y+vzC9i/Hs+A= -github.com/prateek/gorename v0.0.0-20180424020013-52c7307cddd2/go.mod h1:nw9dXugFBQe1pshgrdeRjyYrY+RxNZckdWkiGHX8URE= github.com/prometheus/alertmanager v0.20.0/go.mod h1:9g2i48FAyZW6BtbsnvHtMHQXl2aVtrORKwKVCQ+nbrg= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod h1:p2iRAGwDERtqlqzRXnrOVns+ignqQo//hLXqYxZYVNs= @@ -641,6 +639,8 @@ github.com/remeh/sizedwaitgroup v1.0.0 h1:VNGGFwNo/R5+MJBf6yrsr110p0m4/OX4S3DCy7 github.com/remeh/sizedwaitgroup v1.0.0/go.mod h1:3j2R4OIe/SeS6YDhICBy22RWjJC5eNCJ1V+9+NVNYlo= github.com/rhysd/go-github-selfupdate v1.2.2 h1:G+mNzkc1wEtpmM6sFS/Ghkeq+ad4Yp6EZEHyp//wGEo= github.com/rhysd/go-github-selfupdate v1.2.2/go.mod h1:khesvSyKcXDUxeySCedFh621iawCks0dS/QnHPcpCws= +github.com/robskillington/gorename v0.0.0-20180424020013-52c7307cddd2 h1:t+C9QFlvAI+evRn96lz7eKyzo1CgDx3YVx3N/GJIetk= +github.com/robskillington/gorename v0.0.0-20180424020013-52c7307cddd2/go.mod h1:CVTJ4xwzb/4H98jrd7NFgNoTAiL63scr2Pl7kqOcQAQ= github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= github.com/rogpeppe/fastuuid v1.1.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= diff --git a/src/dbnode/storage/index/block_bench_test.go b/src/dbnode/storage/index/block_bench_test.go index c25f70221c..180b5048d3 100644 --- a/src/dbnode/storage/index/block_bench_test.go +++ b/src/dbnode/storage/index/block_bench_test.go @@ -28,6 +28,7 @@ import ( "testing" "time" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/m3ninx/doc" xtime "github.com/m3db/m3/src/x/time" @@ -46,8 +47,8 @@ func BenchmarkBlockWrite(b *testing.B) { now := time.Now() blockStart := now.Truncate(blockSize) - bl, err := NewBlock(blockStart, testMD, - BlockOptions{}, testOpts) + bl, err := NewBlock(blockStart, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(b, err) defer func() { require.NoError(b, bl.Close()) diff --git a/src/dbnode/storage/index/block_test.go b/src/dbnode/storage/index/block_test.go index 53cb68a0f0..48c821bd1d 100644 --- a/src/dbnode/storage/index/block_test.go +++ b/src/dbnode/storage/index/block_test.go @@ -76,7 +76,8 @@ func newTestNSMetadata(t require.TestingT) namespace.Metadata { func TestBlockCtor(t *testing.T) { md := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - b, err := NewBlock(start, md, BlockOptions{}, testOpts) + b, err := NewBlock(start, md, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) require.Equal(t, start, b.StartTime()) @@ -99,7 +100,8 @@ func TestBlockWriteAfterClose(t *testing.T) { Truncate(blockSize). Add(time.Minute) - b, err := NewBlock(blockStart, testMD, BlockOptions{}, testOpts) + b, err := NewBlock(blockStart, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) require.NoError(t, b.Close()) @@ -147,7 +149,8 @@ func TestBlockWriteAfterSeal(t *testing.T) { Truncate(blockSize). Add(time.Minute) - b, err := NewBlock(blockStart, testMD, BlockOptions{}, testOpts) + b, err := NewBlock(blockStart, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) require.NoError(t, b.Seal()) @@ -195,7 +198,8 @@ func TestBlockWrite(t *testing.T) { Truncate(blockSize). Add(time.Minute) - blk, err := NewBlock(blockStart, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(blockStart, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) defer func() { require.NoError(t, blk.Close()) @@ -244,7 +248,8 @@ func TestBlockWriteActualSegmentPartialFailure(t *testing.T) { Truncate(blockSize). Add(time.Minute) - blk, err := NewBlock(blockStart, md, BlockOptions{}, testOpts) + blk, err := NewBlock(blockStart, md, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) @@ -304,7 +309,8 @@ func TestBlockWritePartialFailure(t *testing.T) { Truncate(blockSize). Add(time.Minute) - blk, err := NewBlock(blockStart, md, BlockOptions{}, testOpts) + blk, err := NewBlock(blockStart, md, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) @@ -353,7 +359,8 @@ func TestBlockWritePartialFailure(t *testing.T) { func TestBlockQueryAfterClose(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - b, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + b, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) require.Equal(t, start, b.StartTime()) @@ -368,7 +375,8 @@ func TestBlockQueryAfterClose(t *testing.T) { func TestBlockQueryWithCancelledQuery(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - b, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + b, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) require.Equal(t, start, b.StartTime()) @@ -387,7 +395,8 @@ func TestBlockQueryWithCancelledQuery(t *testing.T) { func TestBlockQueryExecutorError(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -408,7 +417,8 @@ func TestBlockQuerySegmentReaderError(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -430,7 +440,8 @@ func TestBlockQueryAddResultsSegmentsError(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -469,7 +480,8 @@ func TestBlockMockQueryExecutorExecError(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -495,7 +507,8 @@ func TestBlockMockQueryExecutorExecIterErr(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -535,7 +548,8 @@ func TestBlockMockQueryExecutorExecLimit(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -585,7 +599,8 @@ func TestBlockMockQueryExecutorExecIterCloseErr(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -623,7 +638,8 @@ func TestBlockMockQuerySeriesLimitNonExhaustive(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -672,7 +688,8 @@ func TestBlockMockQuerySeriesLimitExhaustive(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -723,7 +740,8 @@ func TestBlockMockQueryDocsLimitNonExhaustive(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -772,7 +790,8 @@ func TestBlockMockQueryDocsLimitExhaustive(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -823,7 +842,8 @@ func TestBlockMockQueryMergeResultsMapLimit(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -876,7 +896,8 @@ func TestBlockMockQueryMergeResultsDupeID(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -936,7 +957,8 @@ func TestBlockAddResultsAddsSegment(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -959,7 +981,8 @@ func TestBlockAddResultsAfterCloseFails(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) require.NoError(t, blk.Close()) @@ -976,7 +999,8 @@ func TestBlockAddResultsAfterSealWorks(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) require.NoError(t, blk.Seal()) @@ -1000,7 +1024,8 @@ func TestBlockTickSingleSegment(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -1022,7 +1047,8 @@ func TestBlockTickMultipleSegment(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -1051,7 +1077,8 @@ func TestBlockTickAfterSeal(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) require.NoError(t, blk.Seal()) @@ -1074,7 +1101,8 @@ func TestBlockTickAfterClose(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) require.NoError(t, blk.Close()) @@ -1088,7 +1116,8 @@ func TestBlockAddResultsRangeCheck(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -1112,7 +1141,8 @@ func TestBlockAddResultsCoversCurrentData(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -1142,7 +1172,8 @@ func TestBlockAddResultsDoesNotCoverCurrentData(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -1173,7 +1204,8 @@ func TestBlockNeedsMutableSegmentsEvicted(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -1207,7 +1239,8 @@ func TestBlockNeedsMutableSegmentsEvictedMutableSegments(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -1239,7 +1272,8 @@ func TestBlockEvictMutableSegmentsSimple(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) err = blk.EvictMutableSegments() require.Error(t, err) @@ -1255,7 +1289,8 @@ func TestBlockEvictMutableSegmentsAddResults(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -1305,7 +1340,9 @@ func TestBlockE2EInsertQuery(t *testing.T) { BlockOptions{ ForegroundCompactorMmapDocsData: true, BackgroundCompactorMmapDocsData: true, - }, testOpts) + }, + namespace.NewRuntimeOptionsManager("foo"), + testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) @@ -1384,7 +1421,8 @@ func TestBlockE2EInsertQueryLimit(t *testing.T) { Truncate(blockSize). Add(time.Minute) - blk, err := NewBlock(blockStart, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(blockStart, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) @@ -1461,7 +1499,8 @@ func TestBlockE2EInsertAddResultsQuery(t *testing.T) { Truncate(blockSize). Add(time.Minute) - blk, err := NewBlock(blockStart, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(blockStart, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) @@ -1546,7 +1585,8 @@ func TestBlockE2EInsertAddResultsMergeQuery(t *testing.T) { Truncate(blockSize). Add(time.Minute) - blk, err := NewBlock(blockStart, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(blockStart, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) @@ -1628,7 +1668,8 @@ func TestBlockWriteBackgroundCompact(t *testing.T) { testOpts = testOpts.SetInstrumentOptions( testOpts.InstrumentOptions().SetLogger(logger)) - blk, err := NewBlock(blockStart, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(blockStart, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) defer func() { require.NoError(t, blk.Close()) @@ -1718,7 +1759,8 @@ func TestBlockWriteBackgroundCompact(t *testing.T) { func TestBlockAggregateAfterClose(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - b, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + b, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) require.Equal(t, start, b.StartTime()) @@ -1736,7 +1778,8 @@ func TestBlockAggregateIterationErr(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -1774,7 +1817,8 @@ func TestBlockAggregate(t *testing.T) { testMD := newTestNSMetadata(t) start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, testMD, BlockOptions{}, testOpts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), testOpts) require.NoError(t, err) b, ok := blk.(*block) @@ -1845,7 +1889,8 @@ func TestBlockAggregateNotExhaustive(t *testing.T) { aggResultsEntryArrayPool.Init() opts := testOpts.SetAggregateResultsEntryArrayPool(aggResultsEntryArrayPool) - blk, err := NewBlock(start, testMD, BlockOptions{}, opts) + blk, err := NewBlock(start, testMD, BlockOptions{}, + namespace.NewRuntimeOptionsManager("foo"), opts) require.NoError(t, err) b, ok := blk.(*block) @@ -1916,7 +1961,9 @@ func TestBlockE2EInsertAggregate(t *testing.T) { BlockOptions{ ForegroundCompactorMmapDocsData: true, BackgroundCompactorMmapDocsData: true, - }, testOpts) + }, + namespace.NewRuntimeOptionsManager("foo"), + testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) diff --git a/src/dbnode/storage/index/index_mock.go b/src/dbnode/storage/index/index_mock.go index 11ec3fe479..e5953e4a34 100644 --- a/src/dbnode/storage/index/index_mock.go +++ b/src/dbnode/storage/index/index_mock.go @@ -964,6 +964,18 @@ func (mr *MockBlockStatsReporterMockRecorder) ReportSegmentStats(stats interface return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReportSegmentStats", reflect.TypeOf((*MockBlockStatsReporter)(nil).ReportSegmentStats), stats) } +// ReportIndexingStats mocks base method +func (m *MockBlockStatsReporter) ReportIndexingStats(stats BlockIndexingStats) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "ReportIndexingStats", stats) +} + +// ReportIndexingStats indicates an expected call of ReportIndexingStats +func (mr *MockBlockStatsReporterMockRecorder) ReportIndexingStats(stats interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReportIndexingStats", reflect.TypeOf((*MockBlockStatsReporter)(nil).ReportIndexingStats), stats) +} + // MockfieldsAndTermsIterator is a mock of fieldsAndTermsIterator interface type MockfieldsAndTermsIterator struct { ctrl *gomock.Controller diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index 6e1a2f47cf..b00171847d 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -4310,6 +4310,34 @@ func (mr *MockOptionsMockRecorder) DoNotIndexWithFieldsMap() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DoNotIndexWithFieldsMap", reflect.TypeOf((*MockOptions)(nil).DoNotIndexWithFieldsMap)) } +// SetNamespaceRuntimeOptionsManagerRegistry mocks base method +func (m *MockOptions) SetNamespaceRuntimeOptionsManagerRegistry(value namespace.RuntimeOptionsManagerRegistry) Options { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SetNamespaceRuntimeOptionsManagerRegistry", value) + ret0, _ := ret[0].(Options) + return ret0 +} + +// SetNamespaceRuntimeOptionsManagerRegistry indicates an expected call of SetNamespaceRuntimeOptionsManagerRegistry +func (mr *MockOptionsMockRecorder) SetNamespaceRuntimeOptionsManagerRegistry(value interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetNamespaceRuntimeOptionsManagerRegistry", reflect.TypeOf((*MockOptions)(nil).SetNamespaceRuntimeOptionsManagerRegistry), value) +} + +// NamespaceRuntimeOptionsManagerRegistry mocks base method +func (m *MockOptions) NamespaceRuntimeOptionsManagerRegistry() namespace.RuntimeOptionsManagerRegistry { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "NamespaceRuntimeOptionsManagerRegistry") + ret0, _ := ret[0].(namespace.RuntimeOptionsManagerRegistry) + return ret0 +} + +// NamespaceRuntimeOptionsManagerRegistry indicates an expected call of NamespaceRuntimeOptionsManagerRegistry +func (mr *MockOptionsMockRecorder) NamespaceRuntimeOptionsManagerRegistry() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NamespaceRuntimeOptionsManagerRegistry", reflect.TypeOf((*MockOptions)(nil).NamespaceRuntimeOptionsManagerRegistry)) +} + // MockMemoryTracker is a mock of MemoryTracker interface type MockMemoryTracker struct { ctrl *gomock.Controller diff --git a/src/m3ninx/index/segment/segment_mock.go b/src/m3ninx/index/segment/segment_mock.go index d2071999d2..aaefd82525 100644 --- a/src/m3ninx/index/segment/segment_mock.go +++ b/src/m3ninx/index/segment/segment_mock.go @@ -879,6 +879,32 @@ func (mr *MockMutableSegmentMockRecorder) InsertBatch(b interface{}) *gomock.Cal return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "InsertBatch", reflect.TypeOf((*MockMutableSegment)(nil).InsertBatch), b) } +// SetIndexConcurrency mocks base method +func (m *MockMutableSegment) SetIndexConcurrency(value int) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "SetIndexConcurrency", value) +} + +// SetIndexConcurrency indicates an expected call of SetIndexConcurrency +func (mr *MockMutableSegmentMockRecorder) SetIndexConcurrency(value interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetIndexConcurrency", reflect.TypeOf((*MockMutableSegment)(nil).SetIndexConcurrency), value) +} + +// IndexConcurrency mocks base method +func (m *MockMutableSegment) IndexConcurrency() int { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "IndexConcurrency") + ret0, _ := ret[0].(int) + return ret0 +} + +// IndexConcurrency indicates an expected call of IndexConcurrency +func (mr *MockMutableSegmentMockRecorder) IndexConcurrency() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexConcurrency", reflect.TypeOf((*MockMutableSegment)(nil).IndexConcurrency)) +} + // Fields mocks base method func (m *MockMutableSegment) Fields() (FieldsIterator, error) { m.ctrl.T.Helper() @@ -1291,6 +1317,32 @@ func (mr *MockDocumentsBuilderMockRecorder) InsertBatch(b interface{}) *gomock.C return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "InsertBatch", reflect.TypeOf((*MockDocumentsBuilder)(nil).InsertBatch), b) } +// SetIndexConcurrency mocks base method +func (m *MockDocumentsBuilder) SetIndexConcurrency(value int) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "SetIndexConcurrency", value) +} + +// SetIndexConcurrency indicates an expected call of SetIndexConcurrency +func (mr *MockDocumentsBuilderMockRecorder) SetIndexConcurrency(value interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetIndexConcurrency", reflect.TypeOf((*MockDocumentsBuilder)(nil).SetIndexConcurrency), value) +} + +// IndexConcurrency mocks base method +func (m *MockDocumentsBuilder) IndexConcurrency() int { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "IndexConcurrency") + ret0, _ := ret[0].(int) + return ret0 +} + +// IndexConcurrency indicates an expected call of IndexConcurrency +func (mr *MockDocumentsBuilderMockRecorder) IndexConcurrency() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexConcurrency", reflect.TypeOf((*MockDocumentsBuilder)(nil).IndexConcurrency)) +} + // MockCloseableDocumentsBuilder is a mock of CloseableDocumentsBuilder interface type MockCloseableDocumentsBuilder struct { ctrl *gomock.Controller @@ -1414,6 +1466,32 @@ func (mr *MockCloseableDocumentsBuilderMockRecorder) InsertBatch(b interface{}) return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "InsertBatch", reflect.TypeOf((*MockCloseableDocumentsBuilder)(nil).InsertBatch), b) } +// SetIndexConcurrency mocks base method +func (m *MockCloseableDocumentsBuilder) SetIndexConcurrency(value int) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "SetIndexConcurrency", value) +} + +// SetIndexConcurrency indicates an expected call of SetIndexConcurrency +func (mr *MockCloseableDocumentsBuilderMockRecorder) SetIndexConcurrency(value interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetIndexConcurrency", reflect.TypeOf((*MockCloseableDocumentsBuilder)(nil).SetIndexConcurrency), value) +} + +// IndexConcurrency mocks base method +func (m *MockCloseableDocumentsBuilder) IndexConcurrency() int { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "IndexConcurrency") + ret0, _ := ret[0].(int) + return ret0 +} + +// IndexConcurrency indicates an expected call of IndexConcurrency +func (mr *MockCloseableDocumentsBuilderMockRecorder) IndexConcurrency() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexConcurrency", reflect.TypeOf((*MockCloseableDocumentsBuilder)(nil).IndexConcurrency)) +} + // Close mocks base method func (m *MockCloseableDocumentsBuilder) Close() error { m.ctrl.T.Helper() diff --git a/src/x/context/finalizeable_list_gen.go b/src/x/context/finalizeable_list_gen.go index 0c989bc2ae..f8d7eadd34 100644 --- a/src/x/context/finalizeable_list_gen.go +++ b/src/x/context/finalizeable_list_gen.go @@ -134,21 +134,21 @@ func (l *finalizeableList) Init() *finalizeableList { // Use a static pool at least, otherwise each time // we create a list with no pool we create a wholly // new pool of finalizeables (4096 of them). - defaultFinalizeableElementsPoolOnce.Do(initFinalizeableElementsPool) - l.Pool = defaultFinalizeableElementsPool + defaultElementPoolOnce.Do(initElementPool) + l.Pool = defaultElementPool } return l } var ( - defaultFinalizeableElementsPoolOnce sync.Once - defaultFinalizeableElementsPool *finalizeableElementPool + defaultElementPoolOnce sync.Once + defaultElementPool *finalizeableElementPool ) // define as a static method so lambda alloc not required // when passing function pointer to sync.Once.Do. -func initFinalizeableElementsPool() { - defaultFinalizeableElementsPool = newFinalizeableElementPool(nil) +func initElementPool() { + defaultElementPool = newFinalizeableElementPool(nil) } // newFinalizeableList returns an initialized list. diff --git a/src/x/generated-source-files.mk b/src/x/generated-source-files.mk index 44e186b634..6da7c3f652 100644 --- a/src/x/generated-source-files.mk +++ b/src/x/generated-source-files.mk @@ -2,7 +2,7 @@ gopath_prefix := $(GOPATH)/src m3x_package := github.com/m3db/m3/src/x m3x_package_path := $(gopath_prefix)/$(m3x_package) temp_suffix := _temp -gorename_package := github.com/prateek/gorename +gorename_package := github.com/robskillington/gorename gorename_package_version := 52c7307cddd221bb98f0a3215216789f3c821b10 # Tests that all currently generated types match their contents if they were regenerated diff --git a/src/x/generics/list/list.go b/src/x/generics/list/list.go index c1ce49c559..5e7a99c799 100644 --- a/src/x/generics/list/list.go +++ b/src/x/generics/list/list.go @@ -56,6 +56,8 @@ package list import ( + "sync" + "github.com/m3db/m3/src/x/pool" "github.com/mauricelam/genny/generic" @@ -110,11 +112,26 @@ func (l *List) Init() *List { l.root.prev = &l.root l.len = 0 if l.Pool == nil { - l.Pool = newElementPool(nil) + // Use a static pool at least, otherwise each time + // we create a list with no pool we create a wholly + // new pool of finalizeables (4096 of them). + defaultElementPoolOnce.Do(initElementPool) + l.Pool = defaultElementPool } return l } +var ( + defaultElementPoolOnce sync.Once + defaultElementPool *ElementPool +) + +// define as a static method so lambda alloc not required +// when passing function pointer to sync.Once.Do. +func initElementPool() { + defaultElementPool = newElementPool(nil) +} + // newList returns an initialized list. func newList(p *ElementPool) *List { l := &List{Pool: p} diff --git a/tools.go b/tools.go index a63eebc0ef..10eb26c156 100644 --- a/tools.go +++ b/tools.go @@ -14,6 +14,6 @@ import ( _ "github.com/mauricelam/genny" _ "github.com/mjibson/esc" _ "github.com/pointlander/peg" - _ "github.com/prateek/gorename" _ "github.com/rakyll/statik" + _ "github.com/robskillington/gorename" ) From 8acd924d44ea0e45651daf5675568620b4ae68c5 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sat, 18 Jul 2020 12:40:25 -0400 Subject: [PATCH 18/38] Fix tests --- .../m3coordinator/ingest/write_test.go | 8 ++--- .../api/v1/handler/database/create_test.go | 6 ++++ src/x/serialize/decoder_test.go | 36 +++++++++---------- src/x/serialize/encoder.go | 6 ++-- src/x/serialize/encoder_test.go | 35 +++++++++--------- 5 files changed, 47 insertions(+), 44 deletions(-) diff --git a/src/cmd/services/m3coordinator/ingest/write_test.go b/src/cmd/services/m3coordinator/ingest/write_test.go index 8b046797cc..2080cf05e1 100644 --- a/src/cmd/services/m3coordinator/ingest/write_test.go +++ b/src/cmd/services/m3coordinator/ingest/write_test.go @@ -441,7 +441,7 @@ func TestDownsampleAndWriteBatch(t *testing.T) { } downsampler.EXPECT().NewMetricsAppender().Return(mockMetricsAppender, nil) - mockMetricsAppender.EXPECT().Reset().Times(2) + mockMetricsAppender.EXPECT().NextMetric().Times(2) mockMetricsAppender.EXPECT().Finalize() for _, entry := range testEntries { @@ -487,7 +487,7 @@ func TestDownsampleAndWriteBatchDifferentTypes(t *testing.T) { } downsampler.EXPECT().NewMetricsAppender().Return(mockMetricsAppender, nil) - mockMetricsAppender.EXPECT().Reset().Times(2) + mockMetricsAppender.EXPECT().NextMetric().Times(2) mockMetricsAppender.EXPECT().Finalize() for _, entry := range testEntries2 { @@ -537,7 +537,7 @@ func TestDownsampleAndWriteBatchSingleDrop(t *testing.T) { } downsampler.EXPECT().NewMetricsAppender().Return(mockMetricsAppender, nil) - mockMetricsAppender.EXPECT().Reset().Times(2) + mockMetricsAppender.EXPECT().NextMetric().Times(2) mockMetricsAppender.EXPECT().Finalize() for _, dp := range testEntries[1].datapoints { @@ -615,7 +615,7 @@ func TestDownsampleAndWriteBatchOverrideDownsampleRules(t *testing.T) { } downsampler.EXPECT().NewMetricsAppender().Return(mockMetricsAppender, nil) - mockMetricsAppender.EXPECT().Reset() + mockMetricsAppender.EXPECT().NextMetric() mockMetricsAppender.EXPECT().Finalize() iter := newTestIter(entries) diff --git a/src/query/api/v1/handler/database/create_test.go b/src/query/api/v1/handler/database/create_test.go index 47bcfb5d3f..4f2bfba557 100644 --- a/src/query/api/v1/handler/database/create_test.go +++ b/src/query/api/v1/handler/database/create_test.go @@ -170,6 +170,7 @@ func testLocalType(t *testing.T, providedType string, placementExists bool) { "enabled": true, "blockSizeNanos": "3600000000000" }, + "runtimeOptions": null, "schemaOptions": null, "coldWritesEnabled": false } @@ -332,6 +333,7 @@ func TestLocalTypeWithNumShards(t *testing.T) { "enabled": true, "blockSizeNanos": "3600000000000" }, + "runtimeOptions": null, "schemaOptions": null, "coldWritesEnabled": false } @@ -446,6 +448,7 @@ func TestLocalWithBlockSizeNanos(t *testing.T) { "enabled": true, "blockSizeNanos": "10800000000000" }, + "runtimeOptions": null, "schemaOptions": null, "coldWritesEnabled": false } @@ -566,6 +569,7 @@ func TestLocalWithBlockSizeExpectedSeriesDatapointsPerHour(t *testing.T) { "enabled": true, "blockSizeNanos": "%d" }, + "runtimeOptions": null, "schemaOptions": null, "coldWritesEnabled": false } @@ -816,6 +820,7 @@ func testClusterTypeHosts(t *testing.T, placementExists bool) { "enabled": true, "blockSizeNanos": "3600000000000" }, + "runtimeOptions": null, "schemaOptions": null, "coldWritesEnabled": false } @@ -959,6 +964,7 @@ func TestClusterTypeHostsWithIsolationGroup(t *testing.T) { "enabled": true, "blockSizeNanos": "3600000000000" }, + "runtimeOptions": null, "schemaOptions": null, "coldWritesEnabled": false } diff --git a/src/x/serialize/decoder_test.go b/src/x/serialize/decoder_test.go index 61665d092a..81f7a0a58a 100644 --- a/src/x/serialize/decoder_test.go +++ b/src/x/serialize/decoder_test.go @@ -52,9 +52,9 @@ func TestEmptyDecode(t *testing.T) { func TestEmptyTagNameDecode(t *testing.T) { var b []byte b = append(b, headerMagicBytes...) - b = append(b, encodeUInt16(uint16(1))...) /* num tags */ - b = append(b, encodeUInt16(0)...) /* len empty string */ - b = append(b, encodeUInt16(4)...) /* len defg */ + b = append(b, encodeUInt16(1, make([]byte, 2))...) /* num tags */ + b = append(b, encodeUInt16(0, make([]byte, 2))...) /* len empty string */ + b = append(b, encodeUInt16(4, make([]byte, 2))...) /* len defg */ b = append(b, []byte("defg")...) d := newTagDecoder(testDecodeOpts, nil) @@ -66,10 +66,10 @@ func TestEmptyTagNameDecode(t *testing.T) { func TestEmptyTagValueDecode(t *testing.T) { var b []byte b = append(b, headerMagicBytes...) - b = append(b, encodeUInt16(uint16(1))...) /* num tags */ - b = append(b, encodeUInt16(1)...) /* len "1" */ - b = append(b, []byte("a")...) /* tag name */ - b = append(b, encodeUInt16(0)...) /* len tag value */ + b = append(b, encodeUInt16(1, make([]byte, 2))...) /* num tags */ + b = append(b, encodeUInt16(1, make([]byte, 2))...) /* len "1" */ + b = append(b, []byte("a")...) /* tag name */ + b = append(b, encodeUInt16(0, make([]byte, 2))...) /* len tag value */ d := newTagDecoder(testDecodeOpts, nil) d.Reset(wrapAsCheckedBytes(b)) @@ -189,7 +189,7 @@ func TestDecodeLiteralTooLong(t *testing.T) { func TestDecodeMissingTags(t *testing.T) { var b []byte b = append(b, headerMagicBytes...) - b = append(b, encodeUInt16(uint16(2))...) /* num tags */ + b = append(b, encodeUInt16(2, make([]byte, 2))...) /* num tags */ d := newTestTagDecoder() d.Reset(wrapAsCheckedBytes(b)) @@ -202,7 +202,7 @@ func TestDecodeMissingTags(t *testing.T) { func TestDecodeOwnershipFinalize(t *testing.T) { var b []byte b = append(b, headerMagicBytes...) - b = append(b, encodeUInt16(uint16(2))...) /* num tags */ + b = append(b, encodeUInt16(2, make([]byte, 2))...) /* num tags */ wrappedBytes := wrapAsCheckedBytes(b) require.Equal(t, 0, wrappedBytes.NumRef()) @@ -224,14 +224,14 @@ func TestDecodeOwnershipFinalize(t *testing.T) { func TestDecodeMissingValue(t *testing.T) { var b []byte b = append(b, headerMagicBytes...) - b = append(b, encodeUInt16(uint16(2))...) /* num tags */ - b = append(b, encodeUInt16(3)...) /* len abc */ + b = append(b, encodeUInt16(2, make([]byte, 2))...) /* num tags */ + b = append(b, encodeUInt16(3, make([]byte, 2))...) /* len abc */ b = append(b, []byte("abc")...) - b = append(b, encodeUInt16(4)...) /* len defg */ + b = append(b, encodeUInt16(4, make([]byte, 2))...) /* len defg */ b = append(b, []byte("defg")...) - b = append(b, encodeUInt16(1)...) /* len x */ + b = append(b, encodeUInt16(1, make([]byte, 2))...) /* len x */ b = append(b, []byte("x")...) d := newTestTagDecoder() @@ -365,18 +365,18 @@ func wrapAsCheckedBytes(b []byte) checked.Bytes { func testTagDecoderBytesRaw() []byte { var b []byte b = append(b, headerMagicBytes...) - b = append(b, encodeUInt16(uint16(2))...) /* num tags */ + b = append(b, encodeUInt16(2, make([]byte, 2))...) /* num tags */ - b = append(b, encodeUInt16(3)...) /* len abc */ + b = append(b, encodeUInt16(3, make([]byte, 2))...) /* len abc */ b = append(b, []byte("abc")...) - b = append(b, encodeUInt16(4)...) /* len defg */ + b = append(b, encodeUInt16(4, make([]byte, 2))...) /* len defg */ b = append(b, []byte("defg")...) - b = append(b, encodeUInt16(1)...) /* len x */ + b = append(b, encodeUInt16(1, make([]byte, 2))...) /* len x */ b = append(b, []byte("x")...) - b = append(b, encodeUInt16(3)...) /* len bar */ + b = append(b, encodeUInt16(3, make([]byte, 2))...) /* len bar */ b = append(b, []byte("bar")...) return b } diff --git a/src/x/serialize/encoder.go b/src/x/serialize/encoder.go index f38d4baf73..a682a639ee 100644 --- a/src/x/serialize/encoder.go +++ b/src/x/serialize/encoder.go @@ -200,12 +200,12 @@ func (e *encoder) encodeUInt16(v uint16) []byte { // NB(r): Use static buffer on the struct for encoding, otherwise if it's // statically defined inline in the function it will escape to heap. dest := e.staticBufferSlice[:2] - encodeUInt16(v, dest) - return dest + return encodeUInt16(v, dest) } -func encodeUInt16(v uint16, dest []byte) { +func encodeUInt16(v uint16, dest []byte) []byte { byteOrder.PutUint16(dest, v) + return dest } func decodeUInt16(b []byte) uint16 { diff --git a/src/x/serialize/encoder_test.go b/src/x/serialize/encoder_test.go index fe4d81651d..074f71a76f 100644 --- a/src/x/serialize/encoder_test.go +++ b/src/x/serialize/encoder_test.go @@ -108,7 +108,7 @@ func TestSimpleEncode(t *testing.T) { 2 /* bar length */ + len("bar") require.Len(t, b, numExpectedBytes) require.Equal(t, headerMagicBytes, b[:2]) - require.Equal(t, encodeUInt16(2), b[2:4]) + require.Equal(t, encodeUInt16(2, make([]byte, 2)), b[2:4]) require.Equal(t, uint16(3), decodeUInt16(b[4:6])) /* len abc */ require.Equal(t, "abc", string(b[6:9])) require.Equal(t, uint16(4), decodeUInt16(b[9:11])) /* len defg */ @@ -147,18 +147,17 @@ func TestEmptyTagIterEncode(t *testing.T) { return mockBytes } - clonedIter := ident.NewMockTagIterator(ctrl) iter := ident.NewMockTagIterator(ctrl) mockBytes.EXPECT().IncRef() mockBytes.EXPECT().Reset(gomock.Any()) gomock.InOrder( mockBytes.EXPECT().NumRef().Return(0), - iter.EXPECT().Duplicate().Return(clonedIter), - clonedIter.EXPECT().Remaining().Return(0), - clonedIter.EXPECT().Next().Return(false), - clonedIter.EXPECT().Err().Return(nil), - clonedIter.EXPECT().Close(), + iter.EXPECT().Rewind(), + iter.EXPECT().Remaining().Return(0), + iter.EXPECT().Next().Return(false), + iter.EXPECT().Err().Return(nil), + iter.EXPECT().Rewind(), ) enc := newTagEncoder(newBytesFn, newTestEncoderOpts(), nil) @@ -169,15 +168,14 @@ func TestTooManyTags(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - clonedIter := ident.NewMockTagIterator(ctrl) iter := ident.NewMockTagIterator(ctrl) testOpts := newTestEncoderOpts() maxNumTags := testOpts.TagSerializationLimits().MaxNumberTags() gomock.InOrder( - iter.EXPECT().Duplicate().Return(clonedIter), - clonedIter.EXPECT().Remaining().Return(1+int(maxNumTags)), - clonedIter.EXPECT().Close(), + iter.EXPECT().Rewind(), + iter.EXPECT().Remaining().Return(1+int(maxNumTags)), + iter.EXPECT().Rewind(), ) enc := newTagEncoder(defaultNewCheckedBytesFn, testOpts, nil) @@ -193,22 +191,21 @@ func TestSingleValueTagIterEncode(t *testing.T) { return mockBytes } - clonedIter := ident.NewMockTagIterator(ctrl) iter := ident.NewMockTagIterator(ctrl) mockBytes.EXPECT().IncRef() mockBytes.EXPECT().Reset(gomock.Any()) gomock.InOrder( mockBytes.EXPECT().NumRef().Return(0), - iter.EXPECT().Duplicate().Return(clonedIter), - clonedIter.EXPECT().Remaining().Return(1), - clonedIter.EXPECT().Next().Return(true), - clonedIter.EXPECT().Current().Return( + iter.EXPECT().Rewind(), + iter.EXPECT().Remaining().Return(1), + iter.EXPECT().Next().Return(true), + iter.EXPECT().Current().Return( ident.StringTag("some", "tag"), ), - clonedIter.EXPECT().Next().Return(false), - clonedIter.EXPECT().Err().Return(nil), - clonedIter.EXPECT().Close(), + iter.EXPECT().Next().Return(false), + iter.EXPECT().Err().Return(nil), + iter.EXPECT().Rewind(), ) enc := newTagEncoder(newBytesFn, newTestEncoderOpts(), nil) From 2ef0f02ebcaa12bfc861517ed94fd0c6b3aae4a0 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sat, 18 Jul 2020 13:22:54 -0400 Subject: [PATCH 19/38] Fix test build --- src/dbnode/storage/index_block_test.go | 68 +++++++++++++++---- .../storage/index_queue_forward_write_test.go | 19 ++++-- src/dbnode/storage/index_queue_test.go | 27 ++++++-- src/dbnode/storage/index_test.go | 20 ++++-- src/dbnode/storage/namespace_test.go | 28 ++++++-- src/dbnode/storage/shard_ref_count_test.go | 8 ++- 6 files changed, 130 insertions(+), 40 deletions(-) diff --git a/src/dbnode/storage/index_block_test.go b/src/dbnode/storage/index_block_test.go index fad3bf2073..b08ee1e972 100644 --- a/src/dbnode/storage/index_block_test.go +++ b/src/dbnode/storage/index_block_test.go @@ -145,13 +145,16 @@ func TestNamespaceIndexNewBlockFn(t *testing.T) { ts time.Time, md namespace.Metadata, _ index.BlockOptions, + _ namespace.RuntimeOptionsManager, io index.Options, ) (index.Block, error) { require.Equal(t, now.Truncate(blockSize), ts) return mockBlock, nil } md := testNamespaceMetadata(blockSize, 4*time.Hour) - index, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + index, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.NoError(t, err) defer func() { @@ -186,12 +189,15 @@ func TestNamespaceIndexNewBlockFnRandomErr(t *testing.T) { ts time.Time, md namespace.Metadata, _ index.BlockOptions, + _ namespace.RuntimeOptionsManager, io index.Options, ) (index.Block, error) { return nil, fmt.Errorf("randomerr") } md := testNamespaceMetadata(blockSize, 4*time.Hour) - _, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + _, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.Error(t, err) } @@ -213,13 +219,16 @@ func TestNamespaceIndexWrite(t *testing.T) { ts time.Time, md namespace.Metadata, _ index.BlockOptions, + _ namespace.RuntimeOptionsManager, io index.Options, ) (index.Block, error) { require.Equal(t, now.Truncate(blockSize), ts) return mockBlock, nil } md := testNamespaceMetadata(blockSize, 4*time.Hour) - idx, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + idx, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.NoError(t, err) defer func() { @@ -281,6 +290,7 @@ func TestNamespaceIndexWriteCreatesBlock(t *testing.T) { ts time.Time, md namespace.Metadata, _ index.BlockOptions, + _ namespace.RuntimeOptionsManager, io index.Options, ) (index.Block, error) { if ts.Equal(t0) { @@ -292,7 +302,9 @@ func TestNamespaceIndexWriteCreatesBlock(t *testing.T) { panic("should never get here") } md := testNamespaceMetadata(blockSize, 4*time.Hour) - idx, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + idx, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.NoError(t, err) defer func() { @@ -358,6 +370,7 @@ func TestNamespaceIndexBootstrap(t *testing.T) { ts time.Time, md namespace.Metadata, _ index.BlockOptions, + _ namespace.RuntimeOptionsManager, io index.Options, ) (index.Block, error) { if ts.Equal(t0) { @@ -369,7 +382,9 @@ func TestNamespaceIndexBootstrap(t *testing.T) { panic("should never get here") } md := testNamespaceMetadata(blockSize, 4*time.Hour) - idx, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + idx, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.NoError(t, err) seg1 := segment.NewMockSegment(ctrl) @@ -415,6 +430,7 @@ func TestNamespaceIndexTickExpire(t *testing.T) { ts time.Time, md namespace.Metadata, _ index.BlockOptions, + _ namespace.RuntimeOptionsManager, io index.Options, ) (index.Block, error) { if ts.Equal(t0) { @@ -423,7 +439,9 @@ func TestNamespaceIndexTickExpire(t *testing.T) { panic("should never get here") } md := testNamespaceMetadata(blockSize, retentionPeriod) - idx, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + idx, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.NoError(t, err) nowLock.Lock() @@ -464,6 +482,7 @@ func TestNamespaceIndexTick(t *testing.T) { ts time.Time, md namespace.Metadata, _ index.BlockOptions, + _ namespace.RuntimeOptionsManager, io index.Options, ) (index.Block, error) { if ts.Equal(t0) { @@ -472,7 +491,9 @@ func TestNamespaceIndexTick(t *testing.T) { panic("should never get here") } md := testNamespaceMetadata(blockSize, retentionPeriod) - idx, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + idx, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.NoError(t, err) defer func() { @@ -560,6 +581,7 @@ func TestNamespaceIndexBlockQuery(t *testing.T) { ts time.Time, md namespace.Metadata, _ index.BlockOptions, + _ namespace.RuntimeOptionsManager, io index.Options, ) (index.Block, error) { if ts.Equal(t0) { @@ -571,7 +593,9 @@ func TestNamespaceIndexBlockQuery(t *testing.T) { panic("should never get here") } md := testNamespaceMetadata(blockSize, retention) - idx, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + idx, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.NoError(t, err) defer func() { @@ -685,6 +709,7 @@ func TestLimits(t *testing.T) { ts time.Time, md namespace.Metadata, _ index.BlockOptions, + _ namespace.RuntimeOptionsManager, io index.Options, ) (index.Block, error) { if ts.Equal(t0) { @@ -693,7 +718,9 @@ func TestLimits(t *testing.T) { panic("should never get here") } md := testNamespaceMetadata(blockSize, retention) - idx, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + idx, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.NoError(t, err) defer func() { @@ -854,6 +881,7 @@ func TestNamespaceIndexBlockQueryReleasingContext(t *testing.T) { ts time.Time, md namespace.Metadata, _ index.BlockOptions, + _ namespace.RuntimeOptionsManager, io index.Options, ) (index.Block, error) { if ts.Equal(t0) { @@ -871,7 +899,9 @@ func TestNamespaceIndexBlockQueryReleasingContext(t *testing.T) { stubResult := index.NewQueryResults(ident.StringID("ns"), index.QueryResultsOptions{}, iopts) md := testNamespaceMetadata(blockSize, retention) - idxIface, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + idxIface, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.NoError(t, err) idx, ok := idxIface.(*nsIndex) @@ -952,6 +982,7 @@ func TestNamespaceIndexBlockAggregateQuery(t *testing.T) { ts time.Time, md namespace.Metadata, _ index.BlockOptions, + _ namespace.RuntimeOptionsManager, io index.Options, ) (index.Block, error) { if ts.Equal(t0) { @@ -963,7 +994,9 @@ func TestNamespaceIndexBlockAggregateQuery(t *testing.T) { panic("should never get here") } md := testNamespaceMetadata(blockSize, retention) - idx, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + idx, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.NoError(t, err) defer func() { @@ -1092,6 +1125,7 @@ func TestNamespaceIndexBlockAggregateQueryReleasingContext(t *testing.T) { ts time.Time, md namespace.Metadata, _ index.BlockOptions, + _ namespace.RuntimeOptionsManager, io index.Options, ) (index.Block, error) { if ts.Equal(t0) { @@ -1106,10 +1140,13 @@ func TestNamespaceIndexBlockAggregateQueryReleasingContext(t *testing.T) { iopts := opts.IndexOptions() mockPool := index.NewMockAggregateResultsPool(ctrl) iopts = iopts.SetAggregateResultsPool(mockPool) - stubResult := index.NewAggregateResults(ident.StringID("ns"), index.AggregateResultsOptions{}, iopts) + stubResult := index.NewAggregateResults(ident.StringID("ns"), + index.AggregateResultsOptions{}, iopts) md := testNamespaceMetadata(blockSize, retention) - idxIface, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + idxIface, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.NoError(t, err) idx, ok := idxIface.(*nsIndex) @@ -1195,6 +1232,7 @@ func TestNamespaceIndexBlockAggregateQueryAggPath(t *testing.T) { ts time.Time, md namespace.Metadata, _ index.BlockOptions, + _ namespace.RuntimeOptionsManager, io index.Options, ) (index.Block, error) { if ts.Equal(t0) { @@ -1206,7 +1244,9 @@ func TestNamespaceIndexBlockAggregateQueryAggPath(t *testing.T) { panic("should never get here") } md := testNamespaceMetadata(blockSize, retention) - idx, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + idx, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.NoError(t, err) defer func() { diff --git a/src/dbnode/storage/index_queue_forward_write_test.go b/src/dbnode/storage/index_queue_forward_write_test.go index e823c14876..e414b29317 100644 --- a/src/dbnode/storage/index_queue_forward_write_test.go +++ b/src/dbnode/storage/index_queue_forward_write_test.go @@ -91,7 +91,9 @@ func setupForwardIndex( require.NoError(t, err) opts, now, blockSize := generateOptionsNowAndBlockSize() - idx, err := newNamespaceIndexWithInsertQueueFn(md, testShardSet, newFn, opts) + idx, err := newNamespaceIndexWithInsertQueueFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newFn, opts) require.NoError(t, err) var ( @@ -233,7 +235,7 @@ func createMockBlocks( ctrl *gomock.Controller, blockStart time.Time, nextBlockStart time.Time, -) (*index.MockBlock, *index.MockBlock, newBlockFn) { +) (*index.MockBlock, *index.MockBlock, index.NewBlockFn) { mockBlock := index.NewMockBlock(ctrl) mockBlock.EXPECT().Stats(gomock.Any()).Return(nil).AnyTimes() mockBlock.EXPECT().Close().Return(nil) @@ -249,6 +251,7 @@ func createMockBlocks( ts time.Time, md namespace.Metadata, _ index.BlockOptions, + _ namespace.RuntimeOptionsManager, io index.Options, ) (index.Block, error) { if ts.Equal(blockStart) { @@ -281,7 +284,9 @@ func TestNamespaceIndexForwardWrite(t *testing.T) { mockBlock, futureBlock, newBlockFn := createMockBlocks(ctrl, blockStart, futureStart) md := testNamespaceMetadata(blockSize, 4*time.Hour) - idx, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + idx, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.NoError(t, err) defer func() { @@ -321,7 +326,9 @@ func TestNamespaceIndexForwardWriteCreatesBlock(t *testing.T) { mockBlock, futureBlock, newBlockFn := createMockBlocks(ctrl, blockStart, futureStart) md := testNamespaceMetadata(blockSize, 4*time.Hour) - idx, err := newNamespaceIndexWithNewBlockFn(md, testShardSet, newBlockFn, opts) + idx, err := newNamespaceIndexWithNewBlockFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newBlockFn, opts) require.NoError(t, err) defer func() { @@ -379,7 +386,9 @@ func testShardForwardWriteTaggedRefCountIndex( opts, now, blockSize := generateOptionsNowAndBlockSize() opts = opts.SetIndexOptions(opts.IndexOptions().SetInsertMode(syncType)) - idx, err := newNamespaceIndexWithInsertQueueFn(md, testShardSet, newFn, opts) + idx, err := newNamespaceIndexWithInsertQueueFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newFn, opts) require.NoError(t, err) defer func() { diff --git a/src/dbnode/storage/index_queue_test.go b/src/dbnode/storage/index_queue_test.go index 12c6e3e44a..4b8f57135e 100644 --- a/src/dbnode/storage/index_queue_test.go +++ b/src/dbnode/storage/index_queue_test.go @@ -55,7 +55,9 @@ func newTestNamespaceIndex(t *testing.T, ctrl *gomock.Controller) (NamespaceInde q.EXPECT().Start().Return(nil) md, err := namespace.NewMetadata(defaultTestNs1ID, defaultTestNs1Opts) require.NoError(t, err) - idx, err := newNamespaceIndexWithInsertQueueFn(md, testShardSet, newFn, DefaultTestOptions()) + idx, err := newNamespaceIndexWithInsertQueueFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newFn, DefaultTestOptions()) assert.NoError(t, err) return idx, q } @@ -72,7 +74,9 @@ func TestNamespaceIndexHappyPath(t *testing.T) { md, err := namespace.NewMetadata(defaultTestNs1ID, defaultTestNs1Opts) require.NoError(t, err) - idx, err := newNamespaceIndexWithInsertQueueFn(md, testShardSet, newFn, DefaultTestOptions()) + idx, err := newNamespaceIndexWithInsertQueueFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newFn, DefaultTestOptions()) assert.NoError(t, err) assert.NotNil(t, idx) @@ -91,7 +95,9 @@ func TestNamespaceIndexStartErr(t *testing.T) { q.EXPECT().Start().Return(fmt.Errorf("random err")) md, err := namespace.NewMetadata(defaultTestNs1ID, defaultTestNs1Opts) require.NoError(t, err) - idx, err := newNamespaceIndexWithInsertQueueFn(md, testShardSet, newFn, DefaultTestOptions()) + idx, err := newNamespaceIndexWithInsertQueueFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newFn, DefaultTestOptions()) assert.Error(t, err) assert.Nil(t, idx) } @@ -108,7 +114,9 @@ func TestNamespaceIndexStopErr(t *testing.T) { md, err := namespace.NewMetadata(defaultTestNs1ID, defaultTestNs1Opts) require.NoError(t, err) - idx, err := newNamespaceIndexWithInsertQueueFn(md, testShardSet, newFn, DefaultTestOptions()) + idx, err := newNamespaceIndexWithInsertQueueFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newFn, DefaultTestOptions()) assert.NoError(t, err) assert.NotNil(t, idx) @@ -187,7 +195,9 @@ func TestNamespaceIndexInsertOlderThanRetentionPeriod(t *testing.T) { opts := testNamespaceIndexOptions().SetInsertMode(index.InsertSync) opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(nowFn)) - dbIdx, err := newNamespaceIndex(md, testShardSet, DefaultTestOptions().SetIndexOptions(opts)) + dbIdx, err := newNamespaceIndex(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, DefaultTestOptions().SetIndexOptions(opts)) assert.NoError(t, err) idx, ok := dbIdx.(*nsIndex) @@ -285,8 +295,11 @@ func setupIndex(t *testing.T, } md, err := namespace.NewMetadata(defaultTestNs1ID, defaultTestNs1Opts) require.NoError(t, err) - idx, err := newNamespaceIndexWithInsertQueueFn(md, testShardSet, newFn, DefaultTestOptions(). - SetIndexOptions(testNamespaceIndexOptions().SetInsertMode(index.InsertSync))) + idx, err := newNamespaceIndexWithInsertQueueFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newFn, DefaultTestOptions(). + SetIndexOptions(testNamespaceIndexOptions(). + SetInsertMode(index.InsertSync))) assert.NoError(t, err) var ( diff --git a/src/dbnode/storage/index_test.go b/src/dbnode/storage/index_test.go index 7588db9ef8..c58bc9eb4a 100644 --- a/src/dbnode/storage/index_test.go +++ b/src/dbnode/storage/index_test.go @@ -51,7 +51,9 @@ import ( func TestNamespaceIndexCleanupExpiredFilesets(t *testing.T) { md := testNamespaceMetadata(time.Hour, time.Hour*8) - nsIdx, err := newNamespaceIndex(md, testShardSet, DefaultTestOptions()) + nsIdx, err := newNamespaceIndex(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, DefaultTestOptions()) require.NoError(t, err) now := time.Now().Truncate(time.Hour) @@ -73,7 +75,9 @@ func TestNamespaceIndexCleanupExpiredFilesets(t *testing.T) { func TestNamespaceIndexCleanupDuplicateFilesets(t *testing.T) { md := testNamespaceMetadata(time.Hour, time.Hour*8) - nsIdx, err := newNamespaceIndex(md, testShardSet, DefaultTestOptions()) + nsIdx, err := newNamespaceIndex(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, DefaultTestOptions()) require.NoError(t, err) idx := nsIdx.(*nsIndex) @@ -150,7 +154,9 @@ func TestNamespaceIndexCleanupDuplicateFilesets(t *testing.T) { func TestNamespaceIndexCleanupDuplicateFilesetsNoop(t *testing.T) { md := testNamespaceMetadata(time.Hour, time.Hour*8) - nsIdx, err := newNamespaceIndex(md, testShardSet, DefaultTestOptions()) + nsIdx, err := newNamespaceIndex(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, DefaultTestOptions()) require.NoError(t, err) idx := nsIdx.(*nsIndex) @@ -213,7 +219,9 @@ func TestNamespaceIndexCleanupExpiredFilesetsWithBlocks(t *testing.T) { defer ctrl.Finish() md := testNamespaceMetadata(time.Hour, time.Hour*8) - nsIdx, err := newNamespaceIndex(md, testShardSet, DefaultTestOptions()) + nsIdx, err := newNamespaceIndex(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, DefaultTestOptions()) require.NoError(t, err) defer func() { @@ -463,7 +471,9 @@ func newTestIndex(t *testing.T, ctrl *gomock.Controller) testIndex { md, err := namespace.NewMetadata(ident.StringID("testns"), nopts) require.NoError(t, err) opts := DefaultTestOptions() - index, err := newNamespaceIndex(md, testShardSet, opts) + index, err := newNamespaceIndex(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, opts) require.NoError(t, err) return testIndex{ diff --git a/src/dbnode/storage/namespace_test.go b/src/dbnode/storage/namespace_test.go index 6f6018d0ef..382536952f 100644 --- a/src/dbnode/storage/namespace_test.go +++ b/src/dbnode/storage/namespace_test.go @@ -90,7 +90,9 @@ func newTestNamespaceWithIDOpts( shardSet, err := sharding.NewShardSet(testShardIDs, hashFn) require.NoError(t, err) dopts := DefaultTestOptions().SetRuntimeOptionsManager(runtime.NewOptionsManager()) - ns, err := newDatabaseNamespace(metadata, shardSet, nil, nil, nil, dopts) + ns, err := newDatabaseNamespace(metadata, + namespace.NewRuntimeOptionsManager(metadata.ID().String()), + shardSet, nil, nil, nil, dopts) require.NoError(t, err) closer := dopts.RuntimeOptionsManager().Close return ns.(*dbNamespace), closer @@ -105,7 +107,9 @@ func newTestNamespaceWithOpts( hashFn := func(identifier ident.ID) uint32 { return testShardIDs[0].ID() } shardSet, err := sharding.NewShardSet(testShardIDs, hashFn) require.NoError(t, err) - ns, err := newDatabaseNamespace(metadata, shardSet, nil, nil, nil, dopts) + ns, err := newDatabaseNamespace(metadata, + namespace.NewRuntimeOptionsManager(metadata.ID().String()), + shardSet, nil, nil, nil, dopts) require.NoError(t, err) closer := dopts.RuntimeOptionsManager().Close return ns.(*dbNamespace), closer @@ -698,7 +702,9 @@ func TestNamespaceAssignShardSet(t *testing.T) { dopts = dopts.SetInstrumentOptions(dopts.InstrumentOptions(). SetMetricsScope(scope)) - oNs, err := newDatabaseNamespace(metadata, shardSet, nil, nil, nil, dopts) + oNs, err := newDatabaseNamespace(metadata, + namespace.NewRuntimeOptionsManager(metadata.ID().String()), + shardSet, nil, nil, nil, dopts) require.NoError(t, err) ns := oNs.(*dbNamespace) @@ -771,7 +777,9 @@ func newNeedsFlushNamespace(t *testing.T, shardNumbers []uint32) *dbNamespace { return at })) - ns, err := newDatabaseNamespace(metadata, shardSet, nil, nil, nil, dopts) + ns, err := newDatabaseNamespace(metadata, + namespace.NewRuntimeOptionsManager(metadata.ID().String()), + shardSet, nil, nil, nil, dopts) require.NoError(t, err) return ns.(*dbNamespace) } @@ -916,7 +924,9 @@ func TestNamespaceNeedsFlushAllSuccess(t *testing.T) { blockStart := retention.FlushTimeEnd(ropts, at) - oNs, err := newDatabaseNamespace(metadata, shardSet, nil, nil, nil, dopts) + oNs, err := newDatabaseNamespace(metadata, + namespace.NewRuntimeOptionsManager(metadata.ID().String()), + shardSet, nil, nil, nil, dopts) require.NoError(t, err) ns := oNs.(*dbNamespace) @@ -957,7 +967,9 @@ func TestNamespaceNeedsFlushAnyFailed(t *testing.T) { blockStart := retention.FlushTimeEnd(ropts, at) - oNs, err := newDatabaseNamespace(testNs, shardSet, nil, nil, nil, dopts) + oNs, err := newDatabaseNamespace(testNs, + namespace.NewRuntimeOptionsManager(testNs.ID().String()), + shardSet, nil, nil, nil, dopts) require.NoError(t, err) ns := oNs.(*dbNamespace) for _, s := range shards { @@ -1009,7 +1021,9 @@ func TestNamespaceNeedsFlushAnyNotStarted(t *testing.T) { blockStart := retention.FlushTimeEnd(ropts, at) - oNs, err := newDatabaseNamespace(testNs, shardSet, nil, nil, nil, dopts) + oNs, err := newDatabaseNamespace(testNs, + namespace.NewRuntimeOptionsManager(testNs.ID().String()), + shardSet, nil, nil, nil, dopts) require.NoError(t, err) ns := oNs.(*dbNamespace) for _, s := range shards { diff --git a/src/dbnode/storage/shard_ref_count_test.go b/src/dbnode/storage/shard_ref_count_test.go index c320b1a56c..1ca60a44ae 100644 --- a/src/dbnode/storage/shard_ref_count_test.go +++ b/src/dbnode/storage/shard_ref_count_test.go @@ -164,7 +164,9 @@ func TestShardWriteTaggedSyncRefCountSyncIndex(t *testing.T) { ) opts = opts.SetIndexOptions(indexOpts) - idx, err := newNamespaceIndexWithInsertQueueFn(md, testShardSet, newFn, opts) + idx, err := newNamespaceIndexWithInsertQueueFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newFn, opts) assert.NoError(t, err) defer func() { @@ -370,7 +372,9 @@ func TestShardWriteTaggedAsyncRefCountSyncIndex(t *testing.T) { SetClockOptions(opts.ClockOptions().SetNowFn(nowFn)) opts = opts.SetIndexOptions(indexOpts) - idx, err := newNamespaceIndexWithInsertQueueFn(md, testShardSet, newFn, opts) + idx, err := newNamespaceIndexWithInsertQueueFn(md, + namespace.NewRuntimeOptionsManager(md.ID().String()), + testShardSet, newFn, opts) assert.NoError(t, err) defer func() { From 296d83443b0128ed6e5d7a77f81a107f074215c2 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sat, 18 Jul 2020 13:27:58 -0400 Subject: [PATCH 20/38] Fix not closing listener --- src/dbnode/storage/id_list_gen.go | 19 ++++++++++++++++++- src/dbnode/storage/index.go | 5 +++++ 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/src/dbnode/storage/id_list_gen.go b/src/dbnode/storage/id_list_gen.go index 4b786afaa6..027e24acbc 100644 --- a/src/dbnode/storage/id_list_gen.go +++ b/src/dbnode/storage/id_list_gen.go @@ -25,6 +25,8 @@ package storage import ( + "sync" + "github.com/m3db/m3/src/m3ninx/doc" "github.com/m3db/m3/src/x/pool" ) @@ -130,11 +132,26 @@ func (l *idList) Init() *idList { l.root.prev = &l.root l.len = 0 if l.Pool == nil { - l.Pool = newIDElementPool(nil) + // Use a static pool at least, otherwise each time + // we create a list with no pool we create a wholly + // new pool of finalizeables (4096 of them). + defaultElementPoolOnce.Do(initElementPool) + l.Pool = defaultElementPool } return l } +var ( + defaultElementPoolOnce sync.Once + defaultElementPool *idElementPool +) + +// define as a static method so lambda alloc not required +// when passing function pointer to sync.Once.Do. +func initElementPool() { + defaultElementPool = newIDElementPool(nil) +} + // newIDList returns an initialized list. func newIDList(p *idElementPool) *idList { l := &idList{Pool: p} diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index d056841132..d32c8b0853 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -2029,6 +2029,11 @@ func (i *nsIndex) Close() error { i.runtimeOptsListener = nil } + if i.runtimeNsOptsListener != nil { + i.runtimeNsOptsListener.Close() + i.runtimeNsOptsListener = nil + } + // Can now unlock after collecting blocks to close and setting closed state. i.state.Unlock() From 38039ac27d63be978999928f7ee6f1278da198f9 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sat, 18 Jul 2020 14:59:44 -0400 Subject: [PATCH 21/38] Fix build --- src/dbnode/storage/index/block_prop_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/dbnode/storage/index/block_prop_test.go b/src/dbnode/storage/index/block_prop_test.go index 740ac0da45..009cc9672d 100644 --- a/src/dbnode/storage/index/block_prop_test.go +++ b/src/dbnode/storage/index/block_prop_test.go @@ -178,7 +178,8 @@ func TestPostingsListCacheDoesNotAffectBlockQueryResults(t *testing.T) { } func newPropTestBlock(t *testing.T, blockStart time.Time, nsMeta namespace.Metadata, opts Options) (Block, error) { - blk, err := NewBlock(blockStart, nsMeta, BlockOptions{}, opts) + blk, err := NewBlock(blockStart, nsMeta, BlockOptions{}, + namespace.NewRuntimeOptionsManager(nsMeta.ID().String()), opts) require.NoError(t, err) var ( From 8f4daeead31be21466cccf7bdd70189b2c9f9b4c Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 19 Jul 2020 02:50:23 -0400 Subject: [PATCH 22/38] Use per CPU queue for shard insert queue --- src/dbnode/storage/index_insert_queue.go | 1 - src/dbnode/storage/shard.go | 4 + src/dbnode/storage/shard_insert_queue.go | 213 +++++++++++++++++------ 3 files changed, 167 insertions(+), 51 deletions(-) diff --git a/src/dbnode/storage/index_insert_queue.go b/src/dbnode/storage/index_insert_queue.go index 1e0dd9209d..df001bfbf4 100644 --- a/src/dbnode/storage/index_insert_queue.go +++ b/src/dbnode/storage/index_insert_queue.go @@ -354,7 +354,6 @@ func (b *nsIndexInsertBatch) Rotate(target *nsIndexInsertBatch) *sync.WaitGroup b.wg.Add(1) // Rotate to target if we need to. - for idx, inserts := range b.insertsByCPUCore { if target == nil { // No target to rotate with. diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index ef28b25b9f..b2f5a20915 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1038,6 +1038,10 @@ func (s *dbShard) SeriesReadWriteRef( at := s.nowFn() result, err := s.insertSeriesAsyncBatched(id, tags, dbShardInsertAsyncOptions{ + // Make sure to skip the rate limit since this is called + // from the bootstrapper and does not need to be rate + // limited. + skipRateLimit: true, hasPendingIndexing: opts.ReverseIndex, pendingIndex: dbShardPendingIndex{ timestamp: at, diff --git a/src/dbnode/storage/shard_insert_queue.go b/src/dbnode/storage/shard_insert_queue.go index 3ae74def52..cca2fee8e5 100644 --- a/src/dbnode/storage/shard_insert_queue.go +++ b/src/dbnode/storage/shard_insert_queue.go @@ -22,6 +22,7 @@ package storage import ( "errors" + "strconv" "sync" "time" @@ -33,12 +34,17 @@ import ( "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/x/checked" "github.com/m3db/m3/src/x/ident" + xsync "github.com/m3db/m3/src/x/sync" xtime "github.com/m3db/m3/src/x/time" "github.com/uber-go/tally" "go.uber.org/zap" ) +const ( + resetShardInsertsEvery = 30 * time.Second +) + var ( errShardInsertQueueNotOpen = errors.New("shard insert queue is not open") errShardInsertQueueAlreadyOpenOrClosed = errors.New("shard insert queue already open or is closed") @@ -99,11 +105,136 @@ func newDatabaseShardInsertQueueMetrics( } type dbShardInsertBatch struct { + nowFn clock.NowFn + wg *sync.WaitGroup + // Note: since inserts by CPU core is allocated when + // nsIndexInsertBatch is constructed and then never modified + // it is safe to concurently read (but not modify obviously). + insertsByCPUCore []*dbShardInsertsByCPUCore + lastReset time.Time +} + +func newDbShardInsertBatch( + nowFn clock.NowFn, + scope tally.Scope, +) *dbShardInsertBatch { + b := &dbShardInsertBatch{ + nowFn: nowFn, + wg: &sync.WaitGroup{}, + } + numCores := xsync.NumCores() + for i := 0; i < numCores; i++ { + b.insertsByCPUCore = append(b.insertsByCPUCore, &dbShardInsertsByCPUCore{ + wg: b.wg, + metrics: newDBShardInsertsByCPUCoreMetrics(i, scope), + }) + } + b.Rotate(nil) + return b +} + +type dbShardInsertsByCPUCore struct { + sync.Mutex + wg *sync.WaitGroup inserts []dbShardInsert + metrics dbShardInsertsByCPUCoreMetrics +} + +type dbShardInsertsByCPUCoreMetrics struct { + rotateInserts tally.Counter +} + +func newDBShardInsertsByCPUCoreMetrics( + cpuIndex int, + scope tally.Scope, +) dbShardInsertsByCPUCoreMetrics { + scope = scope.Tagged(map[string]string{ + "cpu-index": strconv.Itoa(cpuIndex), + }) + + return dbShardInsertsByCPUCoreMetrics{ + rotateInserts: scope.Counter("rotate-inserts"), + } +} + +func (b *dbShardInsertBatch) Rotate(target *dbShardInsertBatch) *sync.WaitGroup { + prevWg := b.wg + + // We always expect to be waiting for an index. + b.wg = &sync.WaitGroup{} + b.wg.Add(1) + + reset := false + now := b.nowFn() + if now.Sub(b.lastReset) > resetShardInsertsEvery { + // NB(r): Sometimes this can grow very high, so we reset it + // relatively frequently. + reset = true + b.lastReset = now + } + + // Rotate to target if we need to. + for idx, inserts := range b.insertsByCPUCore { + if target == nil { + // No target to rotate with. + inserts.Lock() + // Reset + inserts.inserts = inserts.inserts[:0] + // Use new wait group. + inserts.wg = b.wg + inserts.Unlock() + continue + } + + // First prepare the target to take the current batch's inserts. + targetInserts := target.insertsByCPUCore[idx] + targetInserts.Lock() + + // Reset the target inserts since we'll take ref to them in a second. + var prevTargetInserts []dbShardInsert + if !reset { + // Only reuse if not resetting the allocation. + // memset optimization. + var zeroDbShardInsert dbShardInsert + for i := range targetInserts.inserts { + targetInserts.inserts[i] = zeroDbShardInsert + } + prevTargetInserts = targetInserts.inserts[:0] + } + + // Lock the current batch inserts now ready to rotate to the target. + inserts.Lock() + + // Update current slice refs to take target's inserts. + targetInserts.inserts = inserts.inserts + targetInserts.wg = inserts.wg + + // Reuse the target's old slices. + inserts.inserts = prevTargetInserts + + // Use new wait group. + inserts.wg = b.wg + + // Unlock as early as possible for writes to keep enqueuing. + inserts.Unlock() + + numTargetInserts := len(targetInserts.inserts) + + // Now can unlock target inserts too. + targetInserts.Unlock() + + if n := numTargetInserts; n > 0 { + inserts.metrics.rotateInserts.Inc(int64(n)) + } + } + + return prevWg } type dbShardInsertAsyncOptions struct { + skipRateLimit bool + pendingWrite dbShardPendingWrite pendingRetrievedBlock dbShardPendingRetrievedBlock pendingIndex dbShardPendingIndex @@ -148,16 +279,6 @@ type dbShardPendingRetrievedBlock struct { nsCtx namespace.Context } -func (b *dbShardInsertBatch) reset() { - b.wg = &sync.WaitGroup{} - // We always expect to be waiting for an insert - b.wg.Add(1) - for i := range b.inserts { - b.inserts[i] = dbShardInsertZeroed - } - b.inserts = b.inserts[:0] -} - type dbShardInsertEntryBatchFn func(inserts []dbShardInsert) error // newDatabaseShardInsertQueue creates a new shard insert queue. The shard @@ -181,9 +302,8 @@ func newDatabaseShardInsertQueue( scope tally.Scope, logger *zap.Logger, ) *dbShardInsertQueue { - currBatch := &dbShardInsertBatch{} - currBatch.reset() - subscope := scope.SubScope("insert-queue") + scope = scope.SubScope("insert-queue") + currBatch := newDbShardInsertBatch(nowFn, scope) return &dbShardInsertQueue{ nowFn: nowFn, insertEntryBatchFn: insertEntryBatchFn, @@ -191,7 +311,7 @@ func newDatabaseShardInsertQueue( currBatch: currBatch, notifyInsert: make(chan struct{}, 1), closeCh: make(chan struct{}, 1), - metrics: newDatabaseShardInsertQueueMetrics(subscope), + metrics: newDatabaseShardInsertQueueMetrics(scope), logger: logger, } } @@ -209,8 +329,7 @@ func (q *dbShardInsertQueue) insertLoop() { }() var lastInsert time.Time - freeBatch := &dbShardInsertBatch{} - freeBatch.reset() + batch := newDbShardInsertBatch(q.nowFn, tally.NoopScope) for range q.notifyInsert { // Check if inserting too fast elapsedSinceLastInsert := q.nowFn().Sub(lastInsert) @@ -219,41 +338,33 @@ func (q *dbShardInsertQueue) insertLoop() { var ( state dbShardInsertQueueState backoff time.Duration - batch *dbShardInsertBatch ) q.Lock() state = q.state if elapsedSinceLastInsert < q.insertBatchBackoff { // Need to backoff before rotate and insert backoff = q.insertBatchBackoff - elapsedSinceLastInsert - } else { - // No backoff required, rotate and go - batch = q.currBatch - q.currBatch = freeBatch } q.Unlock() if backoff > 0 { q.sleepFn(backoff) - q.Lock() - // Rotate after backoff - batch = q.currBatch - q.currBatch = freeBatch - q.Unlock() } - if len(batch.inserts) > 0 { - if err := q.insertEntryBatchFn(batch.inserts); err != nil { + batchWg := q.currBatch.Rotate(batch) + + for _, batchByCPUCore := range batch.insertsByCPUCore { + batchByCPUCore.Lock() + err := q.insertEntryBatchFn(batchByCPUCore.inserts) + batchByCPUCore.Unlock() + if err != nil { q.metrics.insertsBatchErrors.Inc(1) q.logger.Error("shard insert queue batch insert failed", zap.Error(err)) } } - batch.wg.Done() - // Set the free batch - batch.reset() - freeBatch = batch + batchWg.Done() lastInsert = q.nowFn() @@ -303,26 +414,28 @@ func (q *dbShardInsertQueue) Stop() error { func (q *dbShardInsertQueue) Insert(insert dbShardInsert) (*sync.WaitGroup, error) { windowNanos := q.nowFn().Truncate(time.Second).UnixNano() - q.Lock() - if q.state != dbShardInsertQueueStateOpen { - q.Unlock() - return nil, errShardInsertQueueNotOpen - } - if limit := q.insertPerSecondLimit; limit > 0 { - if q.insertPerSecondLimitWindowNanos != windowNanos { - // Rolled into a new window - q.insertPerSecondLimitWindowNanos = windowNanos - q.insertPerSecondLimitWindowValues = 0 - } - q.insertPerSecondLimitWindowValues++ - if q.insertPerSecondLimitWindowValues > limit { - q.Unlock() - return nil, errNewSeriesInsertRateLimitExceeded + if !insert.opts.skipRateLimit { + q.Lock() + if limit := q.insertPerSecondLimit; limit > 0 { + if q.insertPerSecondLimitWindowNanos != windowNanos { + // Rolled into a new window + q.insertPerSecondLimitWindowNanos = windowNanos + q.insertPerSecondLimitWindowValues = 0 + } + q.insertPerSecondLimitWindowValues++ + if q.insertPerSecondLimitWindowValues > limit { + q.Unlock() + return nil, errNewSeriesInsertRateLimitExceeded + } } + q.Unlock() } - q.currBatch.inserts = append(q.currBatch.inserts, insert) - wg := q.currBatch.wg - q.Unlock() + + inserts := q.currBatch.insertsByCPUCore[xsync.CPUCore()] + inserts.Lock() + inserts.inserts = append(inserts.inserts, insert) + wg := inserts.wg + inserts.Unlock() // Notify insert loop select { From aa07507010b773ab0d3a1e822f4e386f647c6d67 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 19 Jul 2020 17:27:52 -0400 Subject: [PATCH 23/38] Less locks in shard insert queue --- src/dbnode/storage/index_insert_queue.go | 39 +++++++++----- src/dbnode/storage/shard_insert_queue.go | 69 ++++++++++++++---------- 2 files changed, 67 insertions(+), 41 deletions(-) diff --git a/src/dbnode/storage/index_insert_queue.go b/src/dbnode/storage/index_insert_queue.go index df001bfbf4..642ff2b0c8 100644 --- a/src/dbnode/storage/index_insert_queue.go +++ b/src/dbnode/storage/index_insert_queue.go @@ -102,10 +102,13 @@ func newNamespaceIndexInsertQueue( indexBatchFn: indexBatchFn, nowFn: nowFn, sleepFn: time.Sleep, - notifyInsert: make(chan struct{}, 1), - closeCh: make(chan struct{}, 1), - scope: subscope, - metrics: newNamespaceIndexInsertQueueMetrics(subscope), + // NB(r): Use 2 * num cores so that each CPU insert queue which + // is 1 per num CPU core can always enqueue a notification without + // it being lost. + notifyInsert: make(chan struct{}, 2*xsync.NumCores()), + closeCh: make(chan struct{}, 1), + scope: subscope, + metrics: newNamespaceIndexInsertQueueMetrics(subscope), } q.currBatch = q.newBatch(newBatchOptions{instrumented: true}) return q @@ -180,15 +183,19 @@ func (q *nsIndexInsertQueue) InsertBatch( // it is safe to concurently read (but not modify obviously). inserts := q.currBatch.insertsByCPUCore[xsync.CPUCore()] inserts.Lock() + firstInsert := len(inserts.shardInserts) == 0 inserts.shardInserts = append(inserts.shardInserts, batch) wg := inserts.wg inserts.Unlock() - // Notify insert loop. - select { - case q.notifyInsert <- struct{}{}: - default: - // Loop busy, already ready to consume notification. + // Notify insert loop, only required if first to insert for this + // this CPU core. + if firstInsert { + select { + case q.notifyInsert <- struct{}{}: + default: + // Loop busy, already ready to consume notification. + } } q.metrics.numPending.Inc(int64(batchLen)) @@ -206,15 +213,19 @@ func (q *nsIndexInsertQueue) InsertPending( // it is safe to concurently read (but not modify obviously). inserts := q.currBatch.insertsByCPUCore[xsync.CPUCore()] inserts.Lock() + firstInsert := len(inserts.batchInserts) == 0 inserts.batchInserts = append(inserts.batchInserts, pending...) wg := inserts.wg inserts.Unlock() - // Notify insert loop. - select { - case q.notifyInsert <- struct{}{}: - default: - // Loop busy, already ready to consume notification. + // Notify insert loop, only required if first to insert for this + // this CPU core. + if firstInsert { + select { + case q.notifyInsert <- struct{}{}: + default: + // Loop busy, already ready to consume notification. + } } q.metrics.numPending.Inc(int64(batchLen)) diff --git a/src/dbnode/storage/shard_insert_queue.go b/src/dbnode/storage/shard_insert_queue.go index cca2fee8e5..454488d6a1 100644 --- a/src/dbnode/storage/shard_insert_queue.go +++ b/src/dbnode/storage/shard_insert_queue.go @@ -38,6 +38,7 @@ import ( xtime "github.com/m3db/m3/src/x/time" "github.com/uber-go/tally" + "go.uber.org/atomic" "go.uber.org/zap" ) @@ -69,10 +70,10 @@ type dbShardInsertQueue struct { // rate limits, protected by mutex insertBatchBackoff time.Duration - insertPerSecondLimit int + insertPerSecondLimit *atomic.Uint64 - insertPerSecondLimitWindowNanos int64 - insertPerSecondLimitWindowValues int + insertPerSecondLimitWindowNanos *atomic.Uint64 + insertPerSecondLimitWindowValues *atomic.Uint64 currBatch *dbShardInsertBatch notifyInsert chan struct{} @@ -309,18 +310,26 @@ func newDatabaseShardInsertQueue( insertEntryBatchFn: insertEntryBatchFn, sleepFn: time.Sleep, currBatch: currBatch, - notifyInsert: make(chan struct{}, 1), - closeCh: make(chan struct{}, 1), - metrics: newDatabaseShardInsertQueueMetrics(scope), - logger: logger, + // NB(r): Use 2 * num cores so that each CPU insert queue which + // is 1 per num CPU core can always enqueue a notification without + // it being lost. + notifyInsert: make(chan struct{}, 2*xsync.NumCores()), + closeCh: make(chan struct{}, 1), + insertPerSecondLimitWindowNanos: atomic.NewUint64(0), + insertPerSecondLimitWindowValues: atomic.NewUint64(0), + metrics: newDatabaseShardInsertQueueMetrics(scope), + logger: logger, } } func (q *dbShardInsertQueue) SetRuntimeOptions(value runtime.Options) { q.Lock() q.insertBatchBackoff = value.WriteNewSeriesBackoffDuration() - q.insertPerSecondLimit = value.WriteNewSeriesLimitPerShardPerSecond() q.Unlock() + + // Use atomics so no locks outside of per CPU core lock used. + v := uint64(value.WriteNewSeriesLimitPerShardPerSecond()) + q.insertPerSecondLimit.Store(v) } func (q *dbShardInsertQueue) insertLoop() { @@ -398,11 +407,11 @@ func (q *dbShardInsertQueue) Stop() error { q.state = dbShardInsertQueueStateClosed q.Unlock() - // Final flush + // Final flush. select { case q.notifyInsert <- struct{}{}: default: - // Loop busy, already ready to consume notification + // Loop busy, already ready to consume notification. } // wait till other go routine is done @@ -412,36 +421,42 @@ func (q *dbShardInsertQueue) Stop() error { } func (q *dbShardInsertQueue) Insert(insert dbShardInsert) (*sync.WaitGroup, error) { - windowNanos := q.nowFn().Truncate(time.Second).UnixNano() - if !insert.opts.skipRateLimit { - q.Lock() - if limit := q.insertPerSecondLimit; limit > 0 { - if q.insertPerSecondLimitWindowNanos != windowNanos { - // Rolled into a new window - q.insertPerSecondLimitWindowNanos = windowNanos - q.insertPerSecondLimitWindowValues = 0 + if limit := q.insertPerSecondLimit.Load(); limit > 0 { + windowNanos := uint64(q.nowFn().Truncate(time.Second).UnixNano()) + currLimitWindowNanos := q.insertPerSecondLimitWindowNanos.Load() + if currLimitWindowNanos != windowNanos { + // Rolled into a new window. + if q.insertPerSecondLimitWindowNanos.CAS(currLimitWindowNanos, windowNanos) { + // If managed to set it to the new window, reset the counter + // otherwise another goroutine got to it first and + // will zero the counter. + q.insertPerSecondLimitWindowValues.Store(0) + } } - q.insertPerSecondLimitWindowValues++ - if q.insertPerSecondLimitWindowValues > limit { - q.Unlock() + if q.insertPerSecondLimitWindowValues.Inc() > uint64(limit) { return nil, errNewSeriesInsertRateLimitExceeded } } - q.Unlock() } inserts := q.currBatch.insertsByCPUCore[xsync.CPUCore()] inserts.Lock() + // Track if first insert, if so then we need to notify insert loop, + // otherwise we already have a pending notification. + firstInsert := len(inserts.inserts) == 0 inserts.inserts = append(inserts.inserts, insert) wg := inserts.wg inserts.Unlock() - // Notify insert loop - select { - case q.notifyInsert <- struct{}{}: - default: - // Loop busy, already ready to consume notification + // Notify insert loop, only required if first to insert for this + // this CPU core. + if firstInsert { + select { + case q.notifyInsert <- struct{}{}: + default: + // Loop busy, already ready to consume notification. + } } if insert.opts.hasPendingWrite { From 71167687e18e71d27949464456f99c8d8d8da9c0 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 19 Jul 2020 17:35:20 -0400 Subject: [PATCH 24/38] Fix uninit var --- src/dbnode/storage/shard_insert_queue.go | 1 + 1 file changed, 1 insertion(+) diff --git a/src/dbnode/storage/shard_insert_queue.go b/src/dbnode/storage/shard_insert_queue.go index 454488d6a1..76301ae906 100644 --- a/src/dbnode/storage/shard_insert_queue.go +++ b/src/dbnode/storage/shard_insert_queue.go @@ -315,6 +315,7 @@ func newDatabaseShardInsertQueue( // it being lost. notifyInsert: make(chan struct{}, 2*xsync.NumCores()), closeCh: make(chan struct{}, 1), + insertPerSecondLimit: atomic.NewUint64(0), insertPerSecondLimitWindowNanos: atomic.NewUint64(0), insertPerSecondLimitWindowValues: atomic.NewUint64(0), metrics: newDatabaseShardInsertQueueMetrics(scope), From 8ec78e0cc24c40c336ca71931d8acb55c5233f09 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 19 Jul 2020 18:23:11 -0400 Subject: [PATCH 25/38] Single insertion for shard insert queue batch exec --- src/dbnode/storage/index_insert_queue.go | 2 +- src/dbnode/storage/shard_insert_queue.go | 38 +++++++++++++++++++----- 2 files changed, 31 insertions(+), 9 deletions(-) diff --git a/src/dbnode/storage/index_insert_queue.go b/src/dbnode/storage/index_insert_queue.go index 642ff2b0c8..f3edbbc7c8 100644 --- a/src/dbnode/storage/index_insert_queue.go +++ b/src/dbnode/storage/index_insert_queue.go @@ -50,7 +50,7 @@ const ( // TODO(prateek): runtime options for this stuff defaultIndexBatchBackoff = 2 * time.Millisecond - indexResetAllInsertsEvery = 30 * time.Second + indexResetAllInsertsEvery = 3 * time.Minute ) type nsIndexInsertQueue struct { diff --git a/src/dbnode/storage/shard_insert_queue.go b/src/dbnode/storage/shard_insert_queue.go index 76301ae906..19a53fadc6 100644 --- a/src/dbnode/storage/shard_insert_queue.go +++ b/src/dbnode/storage/shard_insert_queue.go @@ -43,7 +43,7 @@ import ( ) const ( - resetShardInsertsEvery = 30 * time.Second + resetShardInsertsEvery = 3 * time.Minute ) var ( @@ -338,7 +338,11 @@ func (q *dbShardInsertQueue) insertLoop() { close(q.closeCh) }() - var lastInsert time.Time + var ( + lastInsert time.Time + allInserts []dbShardInsert + allInsertsLastReset time.Time + ) batch := newDbShardInsertBatch(q.nowFn, tally.NoopScope) for range q.notifyInsert { // Check if inserting too fast @@ -363,19 +367,37 @@ func (q *dbShardInsertQueue) insertLoop() { batchWg := q.currBatch.Rotate(batch) + // NB(r): Either reset (to avoid spikey allocations sticking around + // forever) or reuse existing slice. + now := q.nowFn() + if now.Sub(allInsertsLastReset) > resetShardInsertsEvery { + allInserts = nil + allInsertsLastReset = now + } else { + allInserts = allInserts[:0] + } + // Batch together for single insertion. for _, batchByCPUCore := range batch.insertsByCPUCore { batchByCPUCore.Lock() - err := q.insertEntryBatchFn(batchByCPUCore.inserts) + allInserts = append(allInserts, batchByCPUCore.inserts...) batchByCPUCore.Unlock() - if err != nil { - q.metrics.insertsBatchErrors.Inc(1) - q.logger.Error("shard insert queue batch insert failed", - zap.Error(err)) - } + } + + err := q.insertEntryBatchFn(allInserts) + if err != nil { + q.metrics.insertsBatchErrors.Inc(1) + q.logger.Error("shard insert queue batch insert failed", + zap.Error(err)) } batchWg.Done() + // Memset optimization to clear inserts holding refs to objects. + var insertZeroValue dbShardInsert + for i := range allInserts { + allInserts[i] = insertZeroValue + } + lastInsert = q.nowFn() if state != dbShardInsertQueueStateOpen { From 09fcc34826f44e5beb466e6b5395a3e89df4985c Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 19 Jul 2020 23:47:31 -0400 Subject: [PATCH 26/38] Write bootstrap data to side-buffer then rotate/merge after bootstrap done --- .../bootstrapper/commitlog/source.go | 12 +- src/dbnode/storage/namespace.go | 3 +- src/dbnode/storage/series/buffer.go | 38 +++++ src/dbnode/storage/series/buffer_mock.go | 14 ++ src/dbnode/storage/series/series.go | 136 ++++++++++++++---- src/dbnode/storage/series/series_mock.go | 14 ++ src/dbnode/storage/series/types.go | 12 +- src/dbnode/storage/shard.go | 25 +++- src/dbnode/storage/storage_mock.go | 8 +- src/dbnode/storage/types.go | 2 +- 10 files changed, 209 insertions(+), 55 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index f95bad14b6..be0a2e2a6a 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -891,15 +891,9 @@ func (s *commitLogSource) startAccumulateWorker(worker *accumulateWorker) { _, _, err := entry.Series.Write(ctx, dp.Timestamp, dp.Value, unit, annotation, series.WriteOptions{ - SchemaDesc: namespace.namespaceContext.Schema, - // NB(r): Make sure this is the series we originally - // checked out for writing too (which should be guaranteed - // by the fact during shard tick we do not expire any - // series unless they are bootstrapped). - MatchUniqueIndex: true, - MatchUniqueIndexValue: entry.UniqueIndex, - BootstrapWrite: true, - SkipOutOfRetention: true, + SchemaDesc: namespace.namespaceContext.Schema, + BootstrapWrite: true, + SkipOutOfRetention: true, }) if err != nil { // NB(r): Only log first error per worker since this could be very diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index 5b601e4527..47de4c9d35 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -921,6 +921,7 @@ func (n *dbNamespace) Bootstrap( return errNamespaceIsBootstrapping } n.bootstrapState = Bootstrapping + nsCtx := n.nsContextWithRLock() n.Unlock() n.metrics.bootstrapStart.Inc(1) @@ -986,7 +987,7 @@ func (n *dbNamespace) Bootstrap( wg.Add(1) shard := shard workers.Go(func() { - err := shard.Bootstrap(ctx) + err := shard.Bootstrap(ctx, nsCtx) mutex.Lock() multiErr = multiErr.Add(err) diff --git a/src/dbnode/storage/series/buffer.go b/src/dbnode/storage/series/buffer.go index 876a99e2d2..af8c742fcf 100644 --- a/src/dbnode/storage/series/buffer.go +++ b/src/dbnode/storage/series/buffer.go @@ -75,6 +75,11 @@ const ( ) type databaseBuffer interface { + MoveTo( + buffer databaseBuffer, + nsCtx namespace.Context, + ) error + Write( ctx context.Context, id ident.ID, @@ -247,6 +252,39 @@ func (b *dbBuffer) Reset(opts databaseBufferResetOptions) { b.blockRetriever = opts.BlockRetriever } +func (b *dbBuffer) MoveTo( + buffer databaseBuffer, + nsCtx namespace.Context, +) error { + blockSize := b.opts.RetentionOptions().BlockSize() + for _, buckets := range b.bucketsMap { + for _, bucket := range buckets.buckets { + // Load any existing blocks. + for _, block := range bucket.loadedBlocks { + // Load block. + buffer.Load(block, bucket.writeType) + } + + // Load encoders. + for _, elem := range bucket.encoders { + if elem.encoder.Len() == 0 { + // No data. + continue + } + // Take ownership of the encoder. + segment := elem.encoder.Discard() + // Create block and load into new buffer. + block := b.opts.DatabaseBlockOptions().DatabaseBlockPool().Get() + block.Reset(bucket.start, blockSize, segment, nsCtx) + // Load block. + buffer.Load(block, bucket.writeType) + } + } + } + + return nil +} + func (b *dbBuffer) Write( ctx context.Context, id ident.ID, diff --git a/src/dbnode/storage/series/buffer_mock.go b/src/dbnode/storage/series/buffer_mock.go index 09a5c4f4c3..a215f611f4 100644 --- a/src/dbnode/storage/series/buffer_mock.go +++ b/src/dbnode/storage/series/buffer_mock.go @@ -62,6 +62,20 @@ func (m *MockdatabaseBuffer) EXPECT() *MockdatabaseBufferMockRecorder { return m.recorder } +// MoveTo mocks base method +func (m *MockdatabaseBuffer) MoveTo(buffer databaseBuffer, nsCtx namespace.Context) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "MoveTo", buffer, nsCtx) + ret0, _ := ret[0].(error) + return ret0 +} + +// MoveTo indicates an expected call of MoveTo +func (mr *MockdatabaseBufferMockRecorder) MoveTo(buffer, nsCtx interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "MoveTo", reflect.TypeOf((*MockdatabaseBuffer)(nil).MoveTo), buffer, nsCtx) +} + // Write mocks base method func (m *MockdatabaseBuffer) Write(ctx context.Context, id ident.ID, timestamp time.Time, value float64, unit time0.Unit, annotation []byte, wOpts WriteOptions) (bool, WriteType, error) { m.ctrl.T.Helper() diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index 133d8df998..046a199b78 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -43,17 +43,14 @@ import ( var ( // ErrSeriesAllDatapointsExpired is returned on tick when all datapoints are expired ErrSeriesAllDatapointsExpired = errors.New("series datapoints are all expired") - // errSeriesMatchUniqueIndexFailed is returned when MatchUniqueIndex is - // specified for a write but the value does not match the current series - // unique index. - errSeriesMatchUniqueIndexFailed = errors.New("series write failed due to unique index not matched") - // errSeriesMatchUniqueIndexInvalid is returned when MatchUniqueIndex is - // specified for a write but the current series unique index is invalid. - errSeriesMatchUniqueIndexInvalid = errors.New("series write failed due to unique index being invalid") errSeriesAlreadyBootstrapped = errors.New("series is already bootstrapped") errSeriesNotBootstrapped = errors.New("series is not yet bootstrapped") errBlockStateSnapshotNotBootstrapped = errors.New("block state snapshot is not bootstrapped") + + // Placeholder for a timeseries being bootstrapped which does not + // have access to the TS ID. + bootstrapWriteID = ident.StringID("bootstrap_timeseries") ) type dbSeries struct { @@ -72,6 +69,8 @@ type dbSeries struct { metadata doc.Document uniqueIndex uint64 + bootstrap dbSeriesBootstrap + buffer databaseBuffer cachedBlocks block.DatabaseSeriesBlocks blockRetriever QueryableBlockRetriever @@ -80,6 +79,14 @@ type dbSeries struct { pool DatabaseSeriesPool } +type dbSeriesBootstrap struct { + sync.Mutex + + // buffer should be nil unless this series + // has taken bootstrap writes. + buffer databaseBuffer +} + // NewDatabaseSeries creates a new database series. func NewDatabaseSeries(opts DatabaseSeriesOptions) DatabaseSeries { s := newDatabaseSeries() @@ -149,10 +156,21 @@ func (s *dbSeries) Tick(blockStates ShardBlockStateSnapshot, nsCtx namespace.Con s.Unlock() - if update.ActiveBlocks == 0 { - return r, ErrSeriesAllDatapointsExpired + if update.ActiveBlocks > 0 { + return r, nil + } + + // Check if any bootstrap writes that hasn't been merged yet. + s.bootstrap.Lock() + unmergedBootstrapDatapoints := s.bootstrap.buffer != nil + s.bootstrap.Unlock() + + if unmergedBootstrapDatapoints { + return r, nil } - return r, nil + + // Everything expired. + return r, ErrSeriesAllDatapointsExpired } type updateBlocksResult struct { @@ -294,26 +312,70 @@ func (s *dbSeries) Write( annotation []byte, wOpts WriteOptions, ) (bool, WriteType, error) { - var writeType WriteType + if wOpts.BootstrapWrite { + return s.bootstrapWrite(ctx, timestamp, value, unit, annotation, wOpts) + } + s.Lock() - defer s.Unlock() - matchUniqueIndex := wOpts.MatchUniqueIndex - if matchUniqueIndex { - if s.uniqueIndex == 0 { - return false, writeType, errSeriesMatchUniqueIndexInvalid + written, writeType, err := s.buffer.Write(ctx, s.id, timestamp, value, + unit, annotation, wOpts) + s.Unlock() + + return written, writeType, err +} + +func (s *dbSeries) bootstrapWrite( + ctx context.Context, + timestamp time.Time, + value float64, + unit xtime.Unit, + annotation []byte, + wOpts WriteOptions, +) (bool, WriteType, error) { + s.bootstrap.Lock() + defer s.bootstrap.Unlock() + + if s.bootstrap.buffer == nil { + // Temporarily release bootstrap lock. + s.bootstrap.Unlock() + + // Get reset opts. + resetOpts, err := s.bufferResetOpts() + + // Re-lock bootstrap lock. + s.bootstrap.Lock() + + if err != nil { + // Abort if failed to get buffer opts. + var writeType WriteType + return false, writeType, err } - if s.uniqueIndex != wOpts.MatchUniqueIndexValue { - // NB(r): Match unique index allows for a caller to - // reliably take a reference to a series and call Write(...) - // later while keeping a direct reference to the series - // while the shard and namespace continues to own and manage - // the lifecycle of the series. - return false, writeType, errSeriesMatchUniqueIndexFailed + + // If buffer still nil then set it. + if s.bootstrap.buffer == nil { + s.bootstrap.buffer = newDatabaseBuffer() + s.bootstrap.buffer.Reset(resetOpts) } } - return s.buffer.Write(ctx, s.id, timestamp, value, - unit, annotation, wOpts) + return s.bootstrap.buffer.Write(ctx, bootstrapWriteID, timestamp, + value, unit, annotation, wOpts) +} + +func (s *dbSeries) bufferResetOpts() (databaseBufferResetOptions, error) { + // Grab series lock. + s.RLock() + defer s.RUnlock() + + if s.id == nil { + // Not active, expired series. + return databaseBufferResetOptions{}, ErrSeriesAllDatapointsExpired + } + + return databaseBufferResetOptions{ + BlockRetriever: s.blockRetriever, + Options: s.opts, + }, nil } func (s *dbSeries) ReadEncoded( @@ -562,7 +624,31 @@ func (s *dbSeries) ColdFlushBlockStarts(blockStates BootstrappedBlockStateSnapsh return s.buffer.ColdFlushBlockStarts(blockStates.Snapshot) } +func (s *dbSeries) Bootstrap(nsCtx namespace.Context) error { + // NB(r): Need to hold the lock the whole time since + // this needs to be consistent view for a tick to see. + s.Lock() + defer s.Unlock() + + s.bootstrap.Lock() + bootstrapBuffer := s.bootstrap.buffer + s.bootstrap.buffer = nil + s.bootstrap.Unlock() + + if bootstrapBuffer == nil { + return nil + } + + return bootstrapBuffer.MoveTo(s.buffer, nsCtx) +} + func (s *dbSeries) Close() { + s.bootstrap.Lock() + if s.bootstrap.buffer != nil { + s.bootstrap.buffer = nil + } + s.bootstrap.Unlock() + s.Lock() defer s.Unlock() diff --git a/src/dbnode/storage/series/series_mock.go b/src/dbnode/storage/series/series_mock.go index ca2240c093..c6fc05e145 100644 --- a/src/dbnode/storage/series/series_mock.go +++ b/src/dbnode/storage/series/series_mock.go @@ -64,6 +64,20 @@ func (m *MockDatabaseSeries) EXPECT() *MockDatabaseSeriesMockRecorder { return m.recorder } +// Bootstrap mocks base method +func (m *MockDatabaseSeries) Bootstrap(arg0 namespace.Context) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Bootstrap", arg0) + ret0, _ := ret[0].(error) + return ret0 +} + +// Bootstrap indicates an expected call of Bootstrap +func (mr *MockDatabaseSeriesMockRecorder) Bootstrap(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Bootstrap", reflect.TypeOf((*MockDatabaseSeries)(nil).Bootstrap), arg0) +} + // Close mocks base method func (m *MockDatabaseSeries) Close() { m.ctrl.T.Helper() diff --git a/src/dbnode/storage/series/types.go b/src/dbnode/storage/series/types.go index ff033aa574..0d5f627183 100644 --- a/src/dbnode/storage/series/types.go +++ b/src/dbnode/storage/series/types.go @@ -141,6 +141,10 @@ type DatabaseSeries interface { // ColdFlushBlockStarts returns the block starts that need cold flushes. ColdFlushBlockStarts(blockStates BootstrappedBlockStateSnapshot) OptimizedTimes + // Bootstrap will moved any bootstrapped data to buffer so series + // is ready for reading. + Bootstrap(nsCtx namespace.Context) error + // Close will close the series and if pooled returned to the pool. Close() @@ -408,14 +412,6 @@ type WriteOptions struct { TruncateType TruncateType // TransformOptions describes transformation options for incoming writes. TransformOptions WriteTransformOptions - // MatchUniqueIndex specifies whether the series unique index - // must match the unique index value specified (to ensure the series - // being written is the same series as previously referenced). - MatchUniqueIndex bool - // MatchUniqueIndexValue is the series unique index value that - // must match the current series unique index value (to ensure series - // being written is the same series as previously referenced). - MatchUniqueIndexValue uint64 // BootstrapWrite allows a warm write outside the time window as long as the // block hasn't already been flushed to disk. This is useful for // bootstrappers filling data that they know has not yet been flushed to diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index b2f5a20915..6da2555c72 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -786,7 +786,7 @@ func (s *dbShard) tickAndExpire( } expired = expired[:0] } - // Continue + // Continue. return true }) @@ -1456,25 +1456,25 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { // for the same ID. entry, _, err := s.lookupEntryWithLock(inserts[i].entry.Series.ID()) if entry != nil { - // Already exists so update the entry we're pointed at for this insert + // Already exists so update the entry we're pointed at for this insert. inserts[i].entry = entry } if hasPendingIndexing || hasPendingWrite || hasPendingRetrievedBlock { // We're definitely writing a value, ensure that the pending write is - // visible before we release the lookup write lock + // visible before we release the lookup write lock. inserts[i].entry.IncrementReaderWriterCount() - // also indicate that we have a ref count on this entry for this operation + // also indicate that we have a ref count on this entry for this operation. inserts[i].opts.entryRefCountIncremented = true } if err == nil { - // Already inserted + // Already inserted. continue } if err != errShardEntryNotFound { - // Shard is not taking inserts + // Shard is not taking inserts. s.Unlock() // FOLLOWUP(prateek): is this an existing bug? why don't we need to release any ref's we've inc'd // on entries in the loop before this point, i.e. in range [0, i). Otherwise, how are those entries @@ -1979,7 +1979,10 @@ func (s *dbShard) UpdateFlushStates() { } } -func (s *dbShard) Bootstrap(ctx context.Context) error { +func (s *dbShard) Bootstrap( + ctx context.Context, + nsCtx namespace.Context, +) error { ctx, span, sampled := ctx.StartSampledTraceSpan(tracepoint.ShardBootstrap) defer span.Finish() @@ -2013,6 +2016,14 @@ func (s *dbShard) Bootstrap(ctx context.Context) error { multiErr = multiErr.Add(err) } + // Move any bootstrap buffers into position for reading. + s.forEachShardEntry(func(entry *lookup.Entry) bool { + if err := entry.Series.Bootstrap(nsCtx); err != nil { + multiErr = multiErr.Add(err) + } + return true + }) + s.Lock() s.bootstrapState = Bootstrapped s.Unlock() diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index b00171847d..864655e539 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -1837,17 +1837,17 @@ func (mr *MockdatabaseShardMockRecorder) PrepareBootstrap(ctx interface{}) *gomo } // Bootstrap mocks base method -func (m *MockdatabaseShard) Bootstrap(ctx context.Context) error { +func (m *MockdatabaseShard) Bootstrap(ctx context.Context, nsCtx namespace.Context) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Bootstrap", ctx) + ret := m.ctrl.Call(m, "Bootstrap", ctx, nsCtx) ret0, _ := ret[0].(error) return ret0 } // Bootstrap indicates an expected call of Bootstrap -func (mr *MockdatabaseShardMockRecorder) Bootstrap(ctx interface{}) *gomock.Call { +func (mr *MockdatabaseShardMockRecorder) Bootstrap(ctx, nsCtx interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Bootstrap", reflect.TypeOf((*MockdatabaseShard)(nil).Bootstrap), ctx) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Bootstrap", reflect.TypeOf((*MockdatabaseShard)(nil).Bootstrap), ctx, nsCtx) } // UpdateFlushStates mocks base method diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 03cdafb230..68a77e2038 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -516,7 +516,7 @@ type databaseShard interface { // Bootstrap bootstraps the shard after all provided data // has been loaded using LoadBootstrapBlocks. - Bootstrap(ctx context.Context) error + Bootstrap(ctx context.Context, nsCtx namespace.Context) error // UpdateFlushStates updates all the flush states for the current shard // by checking the file volumes that exist on disk at a point in time. From e8f25ea3b7ee0c923a73024a6e3a3cfed36834eb Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 20 Jul 2020 01:19:39 -0400 Subject: [PATCH 27/38] With higher tchannel send buffer size --- src/dbnode/storage/shard_insert_queue_test.go | 6 +++--- src/dbnode/x/tchannel/options.go | 6 ++++++ 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/src/dbnode/storage/shard_insert_queue_test.go b/src/dbnode/storage/shard_insert_queue_test.go index be692ba209..df3a76b3c6 100644 --- a/src/dbnode/storage/shard_insert_queue_test.go +++ b/src/dbnode/storage/shard_insert_queue_test.go @@ -48,8 +48,8 @@ func TestShardInsertQueueBatchBackoff(t *testing.T) { currTime = currTime.Add(d) } backoff = 10 * time.Millisecond - insertWgs [3]sync.WaitGroup - insertProgressWgs [3]sync.WaitGroup + insertWgs [64]sync.WaitGroup + insertProgressWgs [64]sync.WaitGroup ) for i := range insertWgs { insertWgs[i].Add(1) @@ -146,7 +146,7 @@ func TestShardInsertQueueRateLimit(t *testing.T) { return currTime }, tally.NoopScope, zap.NewNop()) - q.insertPerSecondLimit = 2 + q.insertPerSecondLimit.Store(2) require.NoError(t, q.Start()) defer func() { diff --git a/src/dbnode/x/tchannel/options.go b/src/dbnode/x/tchannel/options.go index d8e29537c7..1832884935 100644 --- a/src/dbnode/x/tchannel/options.go +++ b/src/dbnode/x/tchannel/options.go @@ -29,6 +29,9 @@ import ( const ( defaultIdleCheckInterval = 5 * time.Minute defaultMaxIdleTime = 5 * time.Minute + // defaultSendBufferSize sets the default send buffer size, + // by default only 512 frames would be buffered. + defaultSendBufferSize = 16384 ) // NewDefaultChannelOptions returns the default tchannel options used. @@ -37,5 +40,8 @@ func NewDefaultChannelOptions() *tchannel.ChannelOptions { Logger: NewNoopLogger(), MaxIdleTime: defaultMaxIdleTime, IdleCheckInterval: defaultIdleCheckInterval, + DefaultConnectionOptions: tchannel.ConnectionOptions{ + SendBufferSize: defaultSendBufferSize, + }, } } From 3cb6f0e406579675c8782f0c2c2e6c8acc5bee1d Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 20 Jul 2020 03:41:00 -0400 Subject: [PATCH 28/38] Rate limit reading index files --- go.mod | 2 ++ go.sum | 3 +++ src/dbnode/digest/digest.go | 6 +++++ src/dbnode/persist/fs/index_read.go | 31 +++++++++++++++++++++++- src/dbnode/persist/fs/persist_manager.go | 5 ++-- 5 files changed, 44 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 9baf5f79c2..9986cbf714 100644 --- a/go.mod +++ b/go.mod @@ -115,11 +115,13 @@ require ( go.etcd.io/etcd v3.4.3+incompatible go.uber.org/atomic v1.5.1 go.uber.org/config v1.4.0 + go.uber.org/ratelimit v0.1.0 go.uber.org/zap v1.13.0 golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f // indirect golang.org/x/net v0.0.0-20200301022130-244492dfa37a golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527 + golang.org/x/time v0.0.0-20191024005414-555d28b269f0 golang.org/x/tools v0.0.0-20200601175630-2caf76543d99 // indirect google.golang.org/grpc v1.27.1 gopkg.in/go-ini/ini.v1 v1.57.0 // indirect diff --git a/go.sum b/go.sum index fc3b9cee8e..1844808e8e 100644 --- a/go.sum +++ b/go.sum @@ -792,6 +792,8 @@ go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/ go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/multierr v1.4.0 h1:f3WCSC2KzAcBXGATIxAB1E2XuCpNU255wNKZ505qi3E= go.uber.org/multierr v1.4.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= +go.uber.org/ratelimit v0.1.0 h1:U2AruXqeTb4Eh9sYQSTrMhH8Cb7M0Ian2ibBOnBcnAw= +go.uber.org/ratelimit v0.1.0/go.mod h1:2X8KaoNd1J0lZV+PxJk/5+DGbO/tpwLR1m++a7FnB/Y= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee h1:0mgffUl7nfd+FpvXMVz4IDEaUSmT1ysygQC7qYo7sG4= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= @@ -914,6 +916,7 @@ golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxb golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0 h1:/5xXl8Y5W96D+TtHSlonuFqGHIWVuyCkGJLwGh9JJFs= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e h1:EHBhcS0mlXEAVwNyO2dLfjToGsyY4j24pTs2ScHnX7s= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20181011042414-1f849cf54d09/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/src/dbnode/digest/digest.go b/src/dbnode/digest/digest.go index 653a50103a..b04296b6eb 100644 --- a/src/dbnode/digest/digest.go +++ b/src/dbnode/digest/digest.go @@ -21,6 +21,7 @@ package digest import ( + "hash" "hash/adler32" "github.com/m3db/stackadler32" @@ -32,6 +33,11 @@ func NewDigest() stackadler32.Digest { return stackadler32.NewDigest() } +// NewDigestWriter returns a stateful digest writer. +func NewDigestWriter() hash.Hash32 { + return adler32.New() +} + // Checksum returns the checksum for a buffer. func Checksum(buf []byte) uint32 { return adler32.Checksum(buf) diff --git a/src/dbnode/persist/fs/index_read.go b/src/dbnode/persist/fs/index_read.go index f86c2d9c1c..6a721a3bc9 100644 --- a/src/dbnode/persist/fs/index_read.go +++ b/src/dbnode/persist/fs/index_read.go @@ -34,6 +34,7 @@ import ( idxpersist "github.com/m3db/m3/src/m3ninx/persist" "github.com/m3db/m3/src/x/mmap" + "go.uber.org/ratelimit" "go.uber.org/zap" ) @@ -257,11 +258,39 @@ func (r *indexReader) ReadSegmentFileSet() ( r.logger.Warn("warning while mmapping files in reader", zap.Error(warning)) } + // Create limiter right before file read in case value just changed. + limits := r.opts.RuntimeOptionsManager().Get().PersistRateLimitOptions() + limiter := ratelimit.NewUnlimited() + if limits.LimitEnabled() { + // We copy 1mb at a time, so set the limit to be how + // many per second we can call. + megaBytesPerSecond := int(limits.LimitMbps() / 8.0) + limiter = ratelimit.New(megaBytesPerSecond) + } + + // Use 1mb batch read size to match the rate limit value. + const batchReadSize = 1024 * 1024 + hash := digest.NewDigestWriter() + reader := bytes.NewReader(desc.Bytes) + for { + // Wait for availability. + limiter.Take() + + // Read batch now rate limiter allowed progression. + _, err := io.CopyN(hash, reader, batchReadSize) + if err == io.EOF { + break + } + if err != nil { + return nil, err + } + } + file := newReadableIndexSegmentFileMmap(segFileType, fd, desc) result.files = append(result.files, file) digests.files = append(digests.files, indexReaderReadSegmentFileDigest{ segmentFileType: segFileType, - digest: digest.Checksum(desc.Bytes), + digest: hash.Sum32(), }) // NB(bodu): Free mmaped bytes after we take the checksum so we don't get memory spikes at bootstrap time. diff --git a/src/dbnode/persist/fs/persist_manager.go b/src/dbnode/persist/fs/persist_manager.go index 867f8fea88..c13f1df00d 100644 --- a/src/dbnode/persist/fs/persist_manager.go +++ b/src/dbnode/persist/fs/persist_manager.go @@ -44,7 +44,8 @@ import ( ) const ( - bytesPerMegabit = 1024 * 1024 / 8 + // BytesPerMegabit is the number of bytes per megabit. + BytesPerMegabit = 1024 * 1024 / 8 ) type persistManagerStatus int @@ -513,7 +514,7 @@ func (pm *persistManager) persist( if pm.start.IsZero() { pm.start = start } else if pm.count >= opts.LimitCheckEvery() { - target := time.Duration(float64(time.Second) * float64(pm.bytesWritten) / (rateLimitMbps * bytesPerMegabit)) + target := time.Duration(float64(time.Second) * float64(pm.bytesWritten) / (rateLimitMbps * BytesPerMegabit)) if elapsed := start.Sub(pm.start); elapsed < target { pm.sleepFn(target - elapsed) // Recapture start for precise timing, might take some time to "wakeup" From 12c467ef8af69d5e42223386e4eb7d8d6e8ab084 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 21 Jul 2020 18:47:15 -0400 Subject: [PATCH 29/38] Avoid applying rate limit currently --- src/dbnode/persist/fs/index_read.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/dbnode/persist/fs/index_read.go b/src/dbnode/persist/fs/index_read.go index 6a721a3bc9..2ba15ecd56 100644 --- a/src/dbnode/persist/fs/index_read.go +++ b/src/dbnode/persist/fs/index_read.go @@ -264,8 +264,8 @@ func (r *indexReader) ReadSegmentFileSet() ( if limits.LimitEnabled() { // We copy 1mb at a time, so set the limit to be how // many per second we can call. - megaBytesPerSecond := int(limits.LimitMbps() / 8.0) - limiter = ratelimit.New(megaBytesPerSecond) + // megaBytesPerSecond := int(limits.LimitMbps() / 8.0) + // limiter = ratelimit.New(megaBytesPerSecond) } // Use 1mb batch read size to match the rate limit value. From 4518df48eb3f90995691690a3df5060b16e1b8f9 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 21 Jul 2020 22:22:29 -0400 Subject: [PATCH 30/38] Revert "Avoid applying rate limit currently" This reverts commit 12c467ef8af69d5e42223386e4eb7d8d6e8ab084. --- src/dbnode/persist/fs/index_read.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/dbnode/persist/fs/index_read.go b/src/dbnode/persist/fs/index_read.go index 2ba15ecd56..6a721a3bc9 100644 --- a/src/dbnode/persist/fs/index_read.go +++ b/src/dbnode/persist/fs/index_read.go @@ -264,8 +264,8 @@ func (r *indexReader) ReadSegmentFileSet() ( if limits.LimitEnabled() { // We copy 1mb at a time, so set the limit to be how // many per second we can call. - // megaBytesPerSecond := int(limits.LimitMbps() / 8.0) - // limiter = ratelimit.New(megaBytesPerSecond) + megaBytesPerSecond := int(limits.LimitMbps() / 8.0) + limiter = ratelimit.New(megaBytesPerSecond) } // Use 1mb batch read size to match the rate limit value. From 9ab89445de9a32fa29bda3e651fc98849e869230 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 21 Jul 2020 22:22:37 -0400 Subject: [PATCH 31/38] Revert "Rate limit reading index files" This reverts commit 3cb6f0e406579675c8782f0c2c2e6c8acc5bee1d. --- go.mod | 2 -- go.sum | 3 --- src/dbnode/digest/digest.go | 6 ----- src/dbnode/persist/fs/index_read.go | 31 +----------------------- src/dbnode/persist/fs/persist_manager.go | 5 ++-- 5 files changed, 3 insertions(+), 44 deletions(-) diff --git a/go.mod b/go.mod index 9986cbf714..9baf5f79c2 100644 --- a/go.mod +++ b/go.mod @@ -115,13 +115,11 @@ require ( go.etcd.io/etcd v3.4.3+incompatible go.uber.org/atomic v1.5.1 go.uber.org/config v1.4.0 - go.uber.org/ratelimit v0.1.0 go.uber.org/zap v1.13.0 golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f // indirect golang.org/x/net v0.0.0-20200301022130-244492dfa37a golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527 - golang.org/x/time v0.0.0-20191024005414-555d28b269f0 golang.org/x/tools v0.0.0-20200601175630-2caf76543d99 // indirect google.golang.org/grpc v1.27.1 gopkg.in/go-ini/ini.v1 v1.57.0 // indirect diff --git a/go.sum b/go.sum index 1844808e8e..fc3b9cee8e 100644 --- a/go.sum +++ b/go.sum @@ -792,8 +792,6 @@ go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/ go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/multierr v1.4.0 h1:f3WCSC2KzAcBXGATIxAB1E2XuCpNU255wNKZ505qi3E= go.uber.org/multierr v1.4.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= -go.uber.org/ratelimit v0.1.0 h1:U2AruXqeTb4Eh9sYQSTrMhH8Cb7M0Ian2ibBOnBcnAw= -go.uber.org/ratelimit v0.1.0/go.mod h1:2X8KaoNd1J0lZV+PxJk/5+DGbO/tpwLR1m++a7FnB/Y= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee h1:0mgffUl7nfd+FpvXMVz4IDEaUSmT1ysygQC7qYo7sG4= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= @@ -916,7 +914,6 @@ golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxb golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0 h1:/5xXl8Y5W96D+TtHSlonuFqGHIWVuyCkGJLwGh9JJFs= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e h1:EHBhcS0mlXEAVwNyO2dLfjToGsyY4j24pTs2ScHnX7s= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20181011042414-1f849cf54d09/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/src/dbnode/digest/digest.go b/src/dbnode/digest/digest.go index b04296b6eb..653a50103a 100644 --- a/src/dbnode/digest/digest.go +++ b/src/dbnode/digest/digest.go @@ -21,7 +21,6 @@ package digest import ( - "hash" "hash/adler32" "github.com/m3db/stackadler32" @@ -33,11 +32,6 @@ func NewDigest() stackadler32.Digest { return stackadler32.NewDigest() } -// NewDigestWriter returns a stateful digest writer. -func NewDigestWriter() hash.Hash32 { - return adler32.New() -} - // Checksum returns the checksum for a buffer. func Checksum(buf []byte) uint32 { return adler32.Checksum(buf) diff --git a/src/dbnode/persist/fs/index_read.go b/src/dbnode/persist/fs/index_read.go index 6a721a3bc9..f86c2d9c1c 100644 --- a/src/dbnode/persist/fs/index_read.go +++ b/src/dbnode/persist/fs/index_read.go @@ -34,7 +34,6 @@ import ( idxpersist "github.com/m3db/m3/src/m3ninx/persist" "github.com/m3db/m3/src/x/mmap" - "go.uber.org/ratelimit" "go.uber.org/zap" ) @@ -258,39 +257,11 @@ func (r *indexReader) ReadSegmentFileSet() ( r.logger.Warn("warning while mmapping files in reader", zap.Error(warning)) } - // Create limiter right before file read in case value just changed. - limits := r.opts.RuntimeOptionsManager().Get().PersistRateLimitOptions() - limiter := ratelimit.NewUnlimited() - if limits.LimitEnabled() { - // We copy 1mb at a time, so set the limit to be how - // many per second we can call. - megaBytesPerSecond := int(limits.LimitMbps() / 8.0) - limiter = ratelimit.New(megaBytesPerSecond) - } - - // Use 1mb batch read size to match the rate limit value. - const batchReadSize = 1024 * 1024 - hash := digest.NewDigestWriter() - reader := bytes.NewReader(desc.Bytes) - for { - // Wait for availability. - limiter.Take() - - // Read batch now rate limiter allowed progression. - _, err := io.CopyN(hash, reader, batchReadSize) - if err == io.EOF { - break - } - if err != nil { - return nil, err - } - } - file := newReadableIndexSegmentFileMmap(segFileType, fd, desc) result.files = append(result.files, file) digests.files = append(digests.files, indexReaderReadSegmentFileDigest{ segmentFileType: segFileType, - digest: hash.Sum32(), + digest: digest.Checksum(desc.Bytes), }) // NB(bodu): Free mmaped bytes after we take the checksum so we don't get memory spikes at bootstrap time. diff --git a/src/dbnode/persist/fs/persist_manager.go b/src/dbnode/persist/fs/persist_manager.go index c13f1df00d..867f8fea88 100644 --- a/src/dbnode/persist/fs/persist_manager.go +++ b/src/dbnode/persist/fs/persist_manager.go @@ -44,8 +44,7 @@ import ( ) const ( - // BytesPerMegabit is the number of bytes per megabit. - BytesPerMegabit = 1024 * 1024 / 8 + bytesPerMegabit = 1024 * 1024 / 8 ) type persistManagerStatus int @@ -514,7 +513,7 @@ func (pm *persistManager) persist( if pm.start.IsZero() { pm.start = start } else if pm.count >= opts.LimitCheckEvery() { - target := time.Duration(float64(time.Second) * float64(pm.bytesWritten) / (rateLimitMbps * BytesPerMegabit)) + target := time.Duration(float64(time.Second) * float64(pm.bytesWritten) / (rateLimitMbps * bytesPerMegabit)) if elapsed := start.Sub(pm.start); elapsed < target { pm.sleepFn(target - elapsed) // Recapture start for precise timing, might take some time to "wakeup" From 63f1f472a66ddb135ffc0a904ca007c47cb1d973 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 21 Jul 2020 22:52:34 -0400 Subject: [PATCH 32/38] Do not autovalidate index segments on boot --- src/cmd/tools/read_index_files/main/main.go | 23 ++++++--- src/dbnode/persist/fs/index_read.go | 20 ++++++-- .../persist/fs/index_read_write_test.go | 51 +++++++++++++++++-- src/dbnode/persist/fs/index_write_test.go | 3 +- src/dbnode/persist/fs/options.go | 17 +++++++ src/dbnode/persist/fs/types.go | 10 +++- 6 files changed, 107 insertions(+), 17 deletions(-) diff --git a/src/cmd/tools/read_index_files/main/main.go b/src/cmd/tools/read_index_files/main/main.go index 7f1fc7de36..5366a0ea70 100644 --- a/src/cmd/tools/read_index_files/main/main.go +++ b/src/cmd/tools/read_index_files/main/main.go @@ -44,12 +44,13 @@ import ( func main() { var ( - optPathPrefix = getopt.StringLong("path-prefix", 'p', "/var/lib/m3db", "Path prefix [e.g. /var/lib/m3db]") - optNamespace = getopt.StringLong("namespace", 'n', "metrics", "Namespace [e.g. metrics]") - optBlockstart = getopt.Int64Long("block-start", 'b', 0, "Block Start Time [in nsec]") - optVolumeIndex = getopt.Int64Long("volume-index", 'v', 0, "Volume index") - optLargeFieldLimit = getopt.Int64Long("large-field-limit", 'l', 0, "Large Field Limit (non-zero to display fields with num terms > limit)") - optOutputIdsPrefix = getopt.StringLong("output-ids-prefix", 'o', "", "If set, it emits all terms for the _m3ninx_id field.") + optPathPrefix = getopt.StringLong("path-prefix", 'p', "/var/lib/m3db", "Path prefix [e.g. /var/lib/m3db]") + optNamespace = getopt.StringLong("namespace", 'n', "metrics", "Namespace [e.g. metrics]") + optBlockstart = getopt.Int64Long("block-start", 'b', 0, "Block Start Time [in nsec]") + optVolumeIndex = getopt.Int64Long("volume-index", 'v', 0, "Volume index") + optLargeFieldLimit = getopt.Int64Long("large-field-limit", 'l', 0, "Large Field Limit (non-zero to display fields with num terms > limit)") + optOutputIdsPrefix = getopt.StringLong("output-ids-prefix", 'o', "", "If set, it emits all terms for the _m3ninx_id field.") + optSkipValidateIntegrity = getopt.BoolLong("skip-validate-integrity", 's', "If set will not skip integrity validation on segment open") ) getopt.Parse() @@ -66,7 +67,9 @@ func main() { os.Exit(1) } - fsOpts := fs.NewOptions().SetFilePathPrefix(*optPathPrefix) + fsOpts := fs.NewOptions(). + SetFilePathPrefix(*optPathPrefix). + SetIndexReaderAutovalidateIndexSegments(!*optSkipValidateIntegrity) reader, err := fs.NewIndexReader(fsOpts) if err != nil { log.Fatalf("could not create new index reader: %v", err) @@ -89,6 +92,7 @@ func main() { i := 0 for { i++ + log.Info("opening index segment file set") fileset, err := reader.ReadSegmentFileSet() if err == io.EOF { break @@ -97,6 +101,11 @@ func main() { log.Fatalf("unable to retrieve fileset: %v", err) } + log.Info("validating index segment file set") + if err := reader.Validate(); err != nil { + log.Fatalf("error validating segment file set: %v", err) + } + seg, err := m3ninxpersist.NewSegment(fileset, fsOpts.FSTOptions()) if err != nil { log.Fatalf("unable to open segment reader: %v", err) diff --git a/src/dbnode/persist/fs/index_read.go b/src/dbnode/persist/fs/index_read.go index f86c2d9c1c..88712236dc 100644 --- a/src/dbnode/persist/fs/index_read.go +++ b/src/dbnode/persist/fs/index_read.go @@ -259,12 +259,18 @@ func (r *indexReader) ReadSegmentFileSet() ( file := newReadableIndexSegmentFileMmap(segFileType, fd, desc) result.files = append(result.files, file) - digests.files = append(digests.files, indexReaderReadSegmentFileDigest{ - segmentFileType: segFileType, - digest: digest.Checksum(desc.Bytes), - }) - // NB(bodu): Free mmaped bytes after we take the checksum so we don't get memory spikes at bootstrap time. + if r.opts.IndexReaderAutovalidateIndexSegments() { + // Only checksum the file if we are autovalidating the index + // segments on open. + digests.files = append(digests.files, indexReaderReadSegmentFileDigest{ + segmentFileType: segFileType, + digest: digest.Checksum(desc.Bytes), + }) + } + + // NB(bodu): Free mmaped bytes after we take the checksum so we don't + // get memory spikes at bootstrap time. if err := mmap.MadviseDontNeed(desc); err != nil { return nil, err } @@ -283,6 +289,10 @@ func (r *indexReader) Validate() error { if err := r.validateInfoFileDigest(); err != nil { return err } + if !r.opts.IndexReaderAutovalidateIndexSegments() { + // Do not validate on segment open. + return nil + } for i, segment := range r.info.Segments { for j := range segment.Files { if err := r.validateSegmentFileDigest(i, j); err != nil { diff --git a/src/dbnode/persist/fs/index_read_write_test.go b/src/dbnode/persist/fs/index_read_write_test.go index bd5709863f..f2049a1260 100644 --- a/src/dbnode/persist/fs/index_read_write_test.go +++ b/src/dbnode/persist/fs/index_read_write_test.go @@ -24,6 +24,7 @@ import ( "bufio" "bytes" "crypto/rand" + "encoding/json" "io" "io/ioutil" "os" @@ -83,7 +84,41 @@ func (s indexWriteTestSetup) cleanup() { os.RemoveAll(s.rootDir) } +type testIndexReadWriteOptions struct { + IndexReaderOptions testIndexReaderOptions +} + func TestIndexSimpleReadWrite(t *testing.T) { + tests := []struct { + TestOptions testIndexReadWriteOptions + }{ + { + TestOptions: testIndexReadWriteOptions{ + IndexReaderOptions: testIndexReaderOptions{ + AutovalidateIndexSegments: true, + }, + }, + }, + { + TestOptions: testIndexReadWriteOptions{ + IndexReaderOptions: testIndexReaderOptions{ + AutovalidateIndexSegments: true, + }, + }, + }, + } + + for _, test := range tests { + test := test + name, err := json.Marshal(test) + require.NoError(t, err) + t.Run(string(name), func(t *testing.T) { + testIndexSimpleReadWrite(t, test.TestOptions) + }) + } +} + +func testIndexSimpleReadWrite(t *testing.T, testOpts testIndexReadWriteOptions) { ctrl := gomock.NewController(t) defer ctrl.Finish() @@ -125,7 +160,8 @@ func TestIndexSimpleReadWrite(t *testing.T) { err = writer.Close() require.NoError(t, err) - reader := newTestIndexReader(t, test.filePathPrefix) + reader := newTestIndexReader(t, test.filePathPrefix, + testOpts.IndexReaderOptions) result, err := reader.Open(IndexReaderOpenOptions{ Identifier: test.fileSetID, FileSetType: persist.FileSetFlushType, @@ -150,9 +186,18 @@ func newTestIndexWriter(t *testing.T, filePathPrefix string) IndexFileSetWriter return writer } -func newTestIndexReader(t *testing.T, filePathPrefix string) IndexFileSetReader { +type testIndexReaderOptions struct { + AutovalidateIndexSegments bool +} + +func newTestIndexReader( + t *testing.T, + filePathPrefix string, + opts testIndexReaderOptions, +) IndexFileSetReader { reader, err := NewIndexReader(testDefaultOpts. - SetFilePathPrefix(filePathPrefix)) + SetFilePathPrefix(filePathPrefix). + SetIndexReaderAutovalidateIndexSegments(opts.AutovalidateIndexSegments)) require.NoError(t, err) return reader } diff --git a/src/dbnode/persist/fs/index_write_test.go b/src/dbnode/persist/fs/index_write_test.go index b8384481bb..a5431b990d 100644 --- a/src/dbnode/persist/fs/index_write_test.go +++ b/src/dbnode/persist/fs/index_write_test.go @@ -130,7 +130,8 @@ func TestSnapshotIndexWriter(t *testing.T) { }, actualFiles) // Verify can read them - reader := newTestIndexReader(t, test.filePathPrefix) + reader := newTestIndexReader(t, test.filePathPrefix, + testIndexReaderOptions{}) for _, snapshot := range testSnapshotSegments { // Add the snapshot index to the file set ID fileSetID := test.fileSetID diff --git a/src/dbnode/persist/fs/options.go b/src/dbnode/persist/fs/options.go index deec323002..042494b963 100644 --- a/src/dbnode/persist/fs/options.go +++ b/src/dbnode/persist/fs/options.go @@ -67,6 +67,11 @@ const ( // defaultForceIndexBloomFilterMmapMemory is the default configuration for whether the bytes for the bloom filter // should be mmap'd as an anonymous region (forced completely into memory) or mmap'd as a file. defaultForceIndexBloomFilterMmapMemory = false + + // defaultIndexReaderAutovalidateIndexSegments is the default configuration for + // whether or not the index reader should autovalidate the index segments when + // opening segments. This is an expensive operation and should be done post-open. + defaultIndexReaderAutovalidateIndexSegments = false ) var ( @@ -100,6 +105,7 @@ type options struct { forceBloomFilterMmapMemory bool mmapEnableHugePages bool mmapReporter mmap.Reporter + indexReaderAutovalidateIndexSegments bool } // NewOptions creates a new set of fs options @@ -134,6 +140,7 @@ func NewOptions() Options { tagEncoderPool: tagEncoderPool, tagDecoderPool: tagDecoderPool, fstOptions: fstOptions, + indexReaderAutovalidateIndexSegments: defaultIndexReaderAutovalidateIndexSegments, } } @@ -366,3 +373,13 @@ func (o *options) SetMmapReporter(mmapReporter mmap.Reporter) Options { func (o *options) MmapReporter() mmap.Reporter { return o.mmapReporter } + +func (o *options) SetIndexReaderAutovalidateIndexSegments(value bool) Options { + opts := *o + opts.indexReaderAutovalidateIndexSegments = value + return &opts +} + +func (o *options) IndexReaderAutovalidateIndexSegments() bool { + return o.indexReaderAutovalidateIndexSegments +} diff --git a/src/dbnode/persist/fs/types.go b/src/dbnode/persist/fs/types.go index cded4b51dc..45ceb6f70c 100644 --- a/src/dbnode/persist/fs/types.go +++ b/src/dbnode/persist/fs/types.go @@ -484,10 +484,18 @@ type Options interface { FSTOptions() fst.Options // SetMmapReporter sets the mmap reporter. - SetMmapReporter(mmapReporter mmap.Reporter) Options + SetMmapReporter(value mmap.Reporter) Options // MmapReporter returns the mmap reporter. MmapReporter() mmap.Reporter + + // SetIndexReaderAutovalidateIndexSegments sets the index reader to + // autovalidate index segments data integrity on file open. + SetIndexReaderAutovalidateIndexSegments(value bool) Options + + // IndexReaderAutovalidateIndexSegments returns the index reader to + // autovalidate index segments data integrity on file open. + IndexReaderAutovalidateIndexSegments() bool } // BlockRetrieverOptions represents the options for block retrieval From 83a89d19dedc54e203f0a8acc22cac9366fe3f57 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 21 Jul 2020 23:04:28 -0400 Subject: [PATCH 33/38] Check is sealed directly rather than inferring block is already closed --- src/dbnode/storage/index.go | 23 ++++++++++------------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index d32c8b0853..84306eafba 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -1023,18 +1023,20 @@ func (i *nsIndex) flushableBlocks( currentBlockStart := now.Truncate(i.blockSize) // Check for flushable blocks by iterating through all block starts w/in retention. for blockStart := earliestBlockStartToRetain; blockStart.Before(currentBlockStart); blockStart = blockStart.Add(i.blockSize) { - canFlush, err := i.canFlushBlockWithRLock(infoFiles, now, blockStart, shards, flushType) + block, err := i.ensureBlockPresentWithRLock(blockStart) if err != nil { return nil, err } - if !canFlush { - continue - } - // Ensure all flushable blocks exist. - block, err := i.ensureBlockPresentWithRLock(blockStart) + + canFlush, err := i.canFlushBlockWithRLock(infoFiles, now, blockStart, + block, shards, flushType) if err != nil { return nil, err } + if !canFlush { + continue + } + flushable = append(flushable, block) } return flushable, nil @@ -1044,6 +1046,7 @@ func (i *nsIndex) canFlushBlockWithRLock( infoFiles []fs.ReadIndexInfoFileResult, startTime time.Time, blockStart time.Time, + block index.Block, shards []databaseShard, flushType series.WriteType, ) (bool, error) { @@ -1054,16 +1057,10 @@ func (i *nsIndex) canFlushBlockWithRLock( // `block.NeedsMutableSegmentsEvicted()` since bootstrap writes for cold block starts // get marked as warm writes if there doesn't already exist data on disk and need to // properly go through the warm flush lifecycle. - isSealed := !blockStart.After(i.lastSealableBlockStart(startTime)) - if !isSealed || i.hasIndexWarmFlushedToDisk(infoFiles, blockStart) { + if !block.IsSealed() || i.hasIndexWarmFlushedToDisk(infoFiles, blockStart) { return false, nil } case series.ColdWrite: - block, ok := i.state.blocksByTime[xtime.ToUnixNano(blockStart)] - // If there is no allocated block then there are definitely no pending cold writes to flush. - if !ok { - return false, nil - } if !block.NeedsColdMutableSegmentsEvicted() { return false, nil } From 3421885d6a4bd58d4dec5ff5ba3731efcd5fdc95 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 21 Jul 2020 23:39:14 -0400 Subject: [PATCH 34/38] Synchronous lock on bootstrap series insert for faster bootstrap --- src/dbnode/storage/shard.go | 95 ++++++++++----------- src/m3ninx/index/segment/builder/builder.go | 2 +- 2 files changed, 44 insertions(+), 53 deletions(-) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 6da2555c72..d1b106fb8b 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -486,6 +486,9 @@ func (s *dbShard) OnRetrieveBlock( s.insertQueue.Insert(dbShardInsert{ entry: entry, opts: dbShardInsertAsyncOptions{ + // NB(r): Caching blocks should not be considered for + // new series insert rate limit. + skipRateLimit: true, hasPendingRetrievedBlock: true, pendingRetrievedBlock: dbShardPendingRetrievedBlock{ id: copiedID, @@ -1030,49 +1033,34 @@ func (s *dbShard) SeriesReadWriteRef( }, nil } - // NB(r): Insert then wait so caller has access to the series - // immediately, otherwise calls to LoadBlock(..) etc on the series - // itself may have no effect if a collision with the same series + // NB(r): Insert synchronously so caller has access to the series + // immediately, otherwise calls to LoadBlock(..) etc on the series itself + // may have no effect if a collision with the same series // being put in the insert queue may cause a block to be loaded to a // series which gets discarded. + // TODO(r): Probably can't insert series sync otherwise we stall a ton + // of writes... need a better solution for bootstrapping. + // This is what can cause writes to degrade during bootstrap if + // write lock is super contended + // Having said that, now that writes are kept in a separate "bootstrap" + // buffer in the series itself to normal writes then merged at tick + // it somewhat mitigates some lock contention since the shard lock + // is still contended but at least series writes due to commit log + // bootstrapping do not interrupt normal writes waiting for ability + // to write to an individual series. at := s.nowFn() - result, err := s.insertSeriesAsyncBatched(id, tags, - dbShardInsertAsyncOptions{ - // Make sure to skip the rate limit since this is called - // from the bootstrapper and does not need to be rate - // limited. - skipRateLimit: true, - hasPendingIndexing: opts.ReverseIndex, - pendingIndex: dbShardPendingIndex{ - timestamp: at, - enqueuedAt: at, - }, - }) - if err != nil { - return SeriesReadWriteRef{}, err - } - - // Wait for the insert to be batched together and inserted. - result.wg.Wait() - - // Guaranteed now to be available in the shard map, otherwise - // an invariant is not held. - s.RLock() - defer s.RUnlock() - - entry, _, err = s.lookupEntryWithLock(result.copiedID) + entry, err = s.insertSeriesSync(id, newTagsIterArg(tags), insertSyncOptions{ + insertType: insertSyncIncReaderWriterCount, + hasPendingIndex: opts.ReverseIndex, + pendingIndex: dbShardPendingIndex{ + timestamp: at, + enqueuedAt: at, + }, + }) if err != nil { - instrument.EmitAndLogInvariantViolation(s.opts.InstrumentOptions(), - func(l *zap.Logger) { - l.Error("read write ref inserted series not found", zap.Error(err)) - }) return SeriesReadWriteRef{}, err } - // Increment before releasing lock so won't be expired until operation - // complete by calling ReleaseReadWriteRef. - entry.IncrementReaderWriterCount() - return SeriesReadWriteRef{ Series: entry.Series, Shard: s.shard, @@ -1274,6 +1262,9 @@ func (s *dbShard) insertSeriesForIndexingAsyncBatched( wg, err := s.insertQueue.Insert(dbShardInsert{ entry: entry, opts: dbShardInsertAsyncOptions{ + // NB(r): Just indexing, should not be considered for new + // series insert rate limiting. + skipRateLimit: true, hasPendingIndexing: true, pendingIndex: dbShardPendingIndex{ timestamp: timestamp, @@ -1347,10 +1338,19 @@ func (s *dbShard) insertSeriesSync( tagsArgOpts tagsArgOptions, opts insertSyncOptions, ) (*lookup.Entry, error) { - var ( - entry *lookup.Entry - err error - ) + // NB(r): Create new shard entry outside of write lock to reduce + // time using write lock. + entry, err := s.newShardEntry(id, tagsArgOpts) + if err != nil { + // should never happen + instrument.EmitAndLogInvariantViolation(s.opts.InstrumentOptions(), + func(logger *zap.Logger) { + logger.Error("insertSeriesSync error creating shard entry", + zap.String("id", id.String()), + zap.Error(err)) + }) + return nil, err + } s.Lock() unlocked := false @@ -1370,18 +1370,6 @@ func (s *dbShard) insertSeriesSync( return entry, nil } - entry, err = s.newShardEntry(id, tagsArgOpts) - if err != nil { - // should never happen - instrument.EmitAndLogInvariantViolation(s.opts.InstrumentOptions(), - func(logger *zap.Logger) { - logger.Error("insertSeriesSync error creating shard entry", - zap.String("id", id.String()), - zap.Error(err)) - }) - return nil, err - } - s.insertNewShardEntryWithLock(entry) // Track unlocking. @@ -1393,6 +1381,9 @@ func (s *dbShard) insertSeriesSync( if _, err := s.insertQueue.Insert(dbShardInsert{ entry: entry, opts: dbShardInsertAsyncOptions{ + // NB(r): Just indexing, should not be considered for new + // series insert rate limiting. + skipRateLimit: true, hasPendingIndexing: opts.hasPendingIndex, pendingIndex: opts.pendingIndex, }, diff --git a/src/m3ninx/index/segment/builder/builder.go b/src/m3ninx/index/segment/builder/builder.go index bcda153c63..a5df8f1fd4 100644 --- a/src/m3ninx/index/segment/builder/builder.go +++ b/src/m3ninx/index/segment/builder/builder.go @@ -112,7 +112,7 @@ func (w *indexWorkers) registerBuilder() { } } -func (w *indexWorkers) indexWorker(indexQueue chan indexJob) { +func (w *indexWorkers) indexWorker(indexQueue <-chan indexJob) { for job := range indexQueue { for i := 0; i < job.usedEntries; i++ { entry := job.entries[i] From 95a9b058e98ba3e51d4a6df41204960722faf55c Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 21 Jul 2020 23:47:45 -0400 Subject: [PATCH 35/38] Fix insertSeriesSync --- src/dbnode/storage/shard.go | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index d1b106fb8b..79ee11c573 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1340,7 +1340,7 @@ func (s *dbShard) insertSeriesSync( ) (*lookup.Entry, error) { // NB(r): Create new shard entry outside of write lock to reduce // time using write lock. - entry, err := s.newShardEntry(id, tagsArgOpts) + newEntry, err := s.newShardEntry(id, tagsArgOpts) if err != nil { // should never happen instrument.EmitAndLogInvariantViolation(s.opts.InstrumentOptions(), @@ -1360,17 +1360,17 @@ func (s *dbShard) insertSeriesSync( } }() - entry, _, err = s.lookupEntryWithLock(id) + existingEntry, _, err := s.lookupEntryWithLock(id) if err != nil && err != errShardEntryNotFound { // Shard not taking inserts likely. return nil, err } - if entry != nil { - // Already inserted. - return entry, nil + if existingEntry != nil { + // Already inserted, likely a race. + return existingEntry, nil } - s.insertNewShardEntryWithLock(entry) + s.insertNewShardEntryWithLock(newEntry) // Track unlocking. unlocked = true @@ -1379,7 +1379,7 @@ func (s *dbShard) insertSeriesSync( // Be sure to enqueue for indexing if requires a pending index. if opts.hasPendingIndex { if _, err := s.insertQueue.Insert(dbShardInsert{ - entry: entry, + entry: newEntry, opts: dbShardInsertAsyncOptions{ // NB(r): Just indexing, should not be considered for new // series insert rate limiting. @@ -1396,10 +1396,10 @@ func (s *dbShard) insertSeriesSync( // to increment the writer count so it's visible when we release // the lock. if opts.insertType == insertSyncIncReaderWriterCount { - entry.IncrementReaderWriterCount() + newEntry.IncrementReaderWriterCount() } - return entry, nil + return newEntry, nil } func (s *dbShard) insertNewShardEntryWithLock(entry *lookup.Entry) { From 2a38d80202ed97556ae4c6e5e23d8f0dd63245ca Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Wed, 22 Jul 2020 00:25:45 -0400 Subject: [PATCH 36/38] Address feedback --- src/cmd/tools/read_index_files/main/main.go | 2 +- src/dbnode/storage/index.go | 2 +- src/dbnode/storage/series/series.go | 11 + src/dbnode/storage/shard.go | 8 +- src/dbnode/storage/shard_insert_queue.go | 348 ++++++++++---------- 5 files changed, 190 insertions(+), 181 deletions(-) diff --git a/src/cmd/tools/read_index_files/main/main.go b/src/cmd/tools/read_index_files/main/main.go index 5366a0ea70..317dc9e935 100644 --- a/src/cmd/tools/read_index_files/main/main.go +++ b/src/cmd/tools/read_index_files/main/main.go @@ -50,7 +50,7 @@ func main() { optVolumeIndex = getopt.Int64Long("volume-index", 'v', 0, "Volume index") optLargeFieldLimit = getopt.Int64Long("large-field-limit", 'l', 0, "Large Field Limit (non-zero to display fields with num terms > limit)") optOutputIdsPrefix = getopt.StringLong("output-ids-prefix", 'o', "", "If set, it emits all terms for the _m3ninx_id field.") - optSkipValidateIntegrity = getopt.BoolLong("skip-validate-integrity", 's', "If set will not skip integrity validation on segment open") + optSkipValidateIntegrity = getopt.BoolLong("skip-validate-integrity", 's', "If set will skip integrity validation on segment open") ) getopt.Parse() diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 84306eafba..a9dbcf69b3 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -85,7 +85,7 @@ const ( defaultFlushReadDataBlocksBatchSize = int64(4096) nsIndexReportStatsInterval = 10 * time.Second - defaultFlushDocsBatchSize = 256 + defaultFlushDocsBatchSize = 8192 ) var ( diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index 046a199b78..4e5931ca67 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -313,6 +313,11 @@ func (s *dbSeries) Write( wOpts WriteOptions, ) (bool, WriteType, error) { if wOpts.BootstrapWrite { + // NB(r): If this is a bootstrap write we store this in a + // side buffer so that we don't need to take the series lock + // and contend with normal writes that are flowing into the DB + // while bootstrapping which can significantly interrupt + // write latency and cause entire DB to stall/degrade in performance. return s.bootstrapWrite(ctx, timestamp, value, unit, annotation, wOpts) } @@ -639,6 +644,12 @@ func (s *dbSeries) Bootstrap(nsCtx namespace.Context) error { return nil } + // NB(r): Now bootstrapped need to move bootstrap writes to the + // normal series buffer to make them visible to DB. + // We store these bootstrap writes in a side buffer so that we don't + // need to take the series lock and contend with normal writes + // that flow into the DB while bootstrapping which can significantly + // interrupt write latency and cause entire DB to stall/degrade in performance. return bootstrapBuffer.MoveTo(s.buffer, nsCtx) } diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 79ee11c573..ab8dd5dfb2 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1041,11 +1041,11 @@ func (s *dbShard) SeriesReadWriteRef( // TODO(r): Probably can't insert series sync otherwise we stall a ton // of writes... need a better solution for bootstrapping. // This is what can cause writes to degrade during bootstrap if - // write lock is super contended + // write lock is super contended. // Having said that, now that writes are kept in a separate "bootstrap" - // buffer in the series itself to normal writes then merged at tick - // it somewhat mitigates some lock contention since the shard lock - // is still contended but at least series writes due to commit log + // buffer in the series itself to normal writes then merged at end of + // bootstrap it somewhat mitigates some lock contention since the shard + // lock is still contended but at least series writes due to commit log // bootstrapping do not interrupt normal writes waiting for ability // to write to an individual series. at := s.nowFn() diff --git a/src/dbnode/storage/shard_insert_queue.go b/src/dbnode/storage/shard_insert_queue.go index 19a53fadc6..fe66bfd45a 100644 --- a/src/dbnode/storage/shard_insert_queue.go +++ b/src/dbnode/storage/shard_insert_queue.go @@ -105,181 +105,6 @@ func newDatabaseShardInsertQueueMetrics( } } -type dbShardInsertBatch struct { - nowFn clock.NowFn - wg *sync.WaitGroup - // Note: since inserts by CPU core is allocated when - // nsIndexInsertBatch is constructed and then never modified - // it is safe to concurently read (but not modify obviously). - insertsByCPUCore []*dbShardInsertsByCPUCore - lastReset time.Time -} - -func newDbShardInsertBatch( - nowFn clock.NowFn, - scope tally.Scope, -) *dbShardInsertBatch { - b := &dbShardInsertBatch{ - nowFn: nowFn, - wg: &sync.WaitGroup{}, - } - numCores := xsync.NumCores() - for i := 0; i < numCores; i++ { - b.insertsByCPUCore = append(b.insertsByCPUCore, &dbShardInsertsByCPUCore{ - wg: b.wg, - metrics: newDBShardInsertsByCPUCoreMetrics(i, scope), - }) - } - b.Rotate(nil) - return b -} - -type dbShardInsertsByCPUCore struct { - sync.Mutex - - wg *sync.WaitGroup - inserts []dbShardInsert - metrics dbShardInsertsByCPUCoreMetrics -} - -type dbShardInsertsByCPUCoreMetrics struct { - rotateInserts tally.Counter -} - -func newDBShardInsertsByCPUCoreMetrics( - cpuIndex int, - scope tally.Scope, -) dbShardInsertsByCPUCoreMetrics { - scope = scope.Tagged(map[string]string{ - "cpu-index": strconv.Itoa(cpuIndex), - }) - - return dbShardInsertsByCPUCoreMetrics{ - rotateInserts: scope.Counter("rotate-inserts"), - } -} - -func (b *dbShardInsertBatch) Rotate(target *dbShardInsertBatch) *sync.WaitGroup { - prevWg := b.wg - - // We always expect to be waiting for an index. - b.wg = &sync.WaitGroup{} - b.wg.Add(1) - - reset := false - now := b.nowFn() - if now.Sub(b.lastReset) > resetShardInsertsEvery { - // NB(r): Sometimes this can grow very high, so we reset it - // relatively frequently. - reset = true - b.lastReset = now - } - - // Rotate to target if we need to. - for idx, inserts := range b.insertsByCPUCore { - if target == nil { - // No target to rotate with. - inserts.Lock() - // Reset - inserts.inserts = inserts.inserts[:0] - // Use new wait group. - inserts.wg = b.wg - inserts.Unlock() - continue - } - - // First prepare the target to take the current batch's inserts. - targetInserts := target.insertsByCPUCore[idx] - targetInserts.Lock() - - // Reset the target inserts since we'll take ref to them in a second. - var prevTargetInserts []dbShardInsert - if !reset { - // Only reuse if not resetting the allocation. - // memset optimization. - var zeroDbShardInsert dbShardInsert - for i := range targetInserts.inserts { - targetInserts.inserts[i] = zeroDbShardInsert - } - prevTargetInserts = targetInserts.inserts[:0] - } - - // Lock the current batch inserts now ready to rotate to the target. - inserts.Lock() - - // Update current slice refs to take target's inserts. - targetInserts.inserts = inserts.inserts - targetInserts.wg = inserts.wg - - // Reuse the target's old slices. - inserts.inserts = prevTargetInserts - - // Use new wait group. - inserts.wg = b.wg - - // Unlock as early as possible for writes to keep enqueuing. - inserts.Unlock() - - numTargetInserts := len(targetInserts.inserts) - - // Now can unlock target inserts too. - targetInserts.Unlock() - - if n := numTargetInserts; n > 0 { - inserts.metrics.rotateInserts.Inc(int64(n)) - } - } - - return prevWg -} - -type dbShardInsertAsyncOptions struct { - skipRateLimit bool - - pendingWrite dbShardPendingWrite - pendingRetrievedBlock dbShardPendingRetrievedBlock - pendingIndex dbShardPendingIndex - - hasPendingWrite bool - hasPendingRetrievedBlock bool - hasPendingIndexing bool - - // NB(prateek): `entryRefCountIncremented` indicates if the - // entry provided along with the dbShardInsertAsyncOptions - // already has it's ref count incremented. It's used to - // correctly manage the lifecycle of the entry across the - // shard -> shard Queue -> shard boundaries. - entryRefCountIncremented bool -} - -type dbShardInsert struct { - entry *lookup.Entry - opts dbShardInsertAsyncOptions -} - -var dbShardInsertZeroed = dbShardInsert{} - -type dbShardPendingWrite struct { - timestamp time.Time - value float64 - unit xtime.Unit - annotation checked.Bytes - opts series.WriteOptions -} - -type dbShardPendingIndex struct { - timestamp time.Time - enqueuedAt time.Time -} - -type dbShardPendingRetrievedBlock struct { - id ident.ID - tags ident.TagIterator - start time.Time - segment ts.Segment - nsCtx namespace.Context -} - type dbShardInsertEntryBatchFn func(inserts []dbShardInsert) error // newDatabaseShardInsertQueue creates a new shard insert queue. The shard @@ -490,3 +315,176 @@ func (q *dbShardInsertQueue) Insert(insert dbShardInsert) (*sync.WaitGroup, erro return wg, nil } + +type dbShardInsertBatch struct { + nowFn clock.NowFn + wg *sync.WaitGroup + // Note: since inserts by CPU core is allocated when + // nsIndexInsertBatch is constructed and then never modified + // it is safe to concurently read (but not modify obviously). + insertsByCPUCore []*dbShardInsertsByCPUCore + lastReset time.Time +} + +type dbShardInsertsByCPUCore struct { + sync.Mutex + + wg *sync.WaitGroup + inserts []dbShardInsert + metrics dbShardInsertsByCPUCoreMetrics +} + +type dbShardInsert struct { + entry *lookup.Entry + opts dbShardInsertAsyncOptions +} + +type dbShardInsertAsyncOptions struct { + skipRateLimit bool + + pendingWrite dbShardPendingWrite + pendingRetrievedBlock dbShardPendingRetrievedBlock + pendingIndex dbShardPendingIndex + + hasPendingWrite bool + hasPendingRetrievedBlock bool + hasPendingIndexing bool + + // NB(prateek): `entryRefCountIncremented` indicates if the + // entry provided along with the dbShardInsertAsyncOptions + // already has it's ref count incremented. It's used to + // correctly manage the lifecycle of the entry across the + // shard -> shard Queue -> shard boundaries. + entryRefCountIncremented bool +} + +type dbShardPendingWrite struct { + timestamp time.Time + value float64 + unit xtime.Unit + annotation checked.Bytes + opts series.WriteOptions +} + +type dbShardPendingIndex struct { + timestamp time.Time + enqueuedAt time.Time +} + +type dbShardPendingRetrievedBlock struct { + id ident.ID + tags ident.TagIterator + start time.Time + segment ts.Segment + nsCtx namespace.Context +} + +func newDbShardInsertBatch( + nowFn clock.NowFn, + scope tally.Scope, +) *dbShardInsertBatch { + b := &dbShardInsertBatch{ + nowFn: nowFn, + wg: &sync.WaitGroup{}, + } + numCores := xsync.NumCores() + for i := 0; i < numCores; i++ { + b.insertsByCPUCore = append(b.insertsByCPUCore, &dbShardInsertsByCPUCore{ + wg: b.wg, + metrics: newDBShardInsertsByCPUCoreMetrics(i, scope), + }) + } + b.Rotate(nil) + return b +} + +type dbShardInsertsByCPUCoreMetrics struct { + rotateInserts tally.Counter +} + +func newDBShardInsertsByCPUCoreMetrics( + cpuIndex int, + scope tally.Scope, +) dbShardInsertsByCPUCoreMetrics { + scope = scope.Tagged(map[string]string{ + "cpu-index": strconv.Itoa(cpuIndex), + }) + + return dbShardInsertsByCPUCoreMetrics{ + rotateInserts: scope.Counter("rotate-inserts"), + } +} + +func (b *dbShardInsertBatch) Rotate(target *dbShardInsertBatch) *sync.WaitGroup { + prevWg := b.wg + + // We always expect to be waiting for an index. + b.wg = &sync.WaitGroup{} + b.wg.Add(1) + + reset := false + now := b.nowFn() + if now.Sub(b.lastReset) > resetShardInsertsEvery { + // NB(r): Sometimes this can grow very high, so we reset it + // relatively frequently. + reset = true + b.lastReset = now + } + + // Rotate to target if we need to. + for idx, inserts := range b.insertsByCPUCore { + if target == nil { + // No target to rotate with. + inserts.Lock() + // Reset + inserts.inserts = inserts.inserts[:0] + // Use new wait group. + inserts.wg = b.wg + inserts.Unlock() + continue + } + + // First prepare the target to take the current batch's inserts. + targetInserts := target.insertsByCPUCore[idx] + targetInserts.Lock() + + // Reset the target inserts since we'll take ref to them in a second. + var prevTargetInserts []dbShardInsert + if !reset { + // Only reuse if not resetting the allocation. + // memset optimization. + var zeroDbShardInsert dbShardInsert + for i := range targetInserts.inserts { + targetInserts.inserts[i] = zeroDbShardInsert + } + prevTargetInserts = targetInserts.inserts[:0] + } + + // Lock the current batch inserts now ready to rotate to the target. + inserts.Lock() + + // Update current slice refs to take target's inserts. + targetInserts.inserts = inserts.inserts + targetInserts.wg = inserts.wg + + // Reuse the target's old slices. + inserts.inserts = prevTargetInserts + + // Use new wait group. + inserts.wg = b.wg + + // Unlock as early as possible for writes to keep enqueuing. + inserts.Unlock() + + numTargetInserts := len(targetInserts.inserts) + + // Now can unlock target inserts too. + targetInserts.Unlock() + + if n := numTargetInserts; n > 0 { + inserts.metrics.rotateInserts.Inc(int64(n)) + } + } + + return prevWg +} From bc253faff0c1690752ffb51be2aae375f73a6581 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Wed, 22 Jul 2020 01:26:13 -0400 Subject: [PATCH 37/38] Add FST writer options to FST writer --- src/dbnode/integration/integration.go | 2 +- src/dbnode/persist/fs/options.go | 13 +++++++++++++ src/dbnode/persist/fs/persist_manager.go | 4 +++- src/dbnode/persist/fs/types.go | 8 +++++++- src/m3ninx/persist/writer.go | 10 +++++++--- 5 files changed, 31 insertions(+), 6 deletions(-) diff --git a/src/dbnode/integration/integration.go b/src/dbnode/integration/integration.go index 995ec6715a..17681602c8 100644 --- a/src/dbnode/integration/integration.go +++ b/src/dbnode/integration/integration.go @@ -448,7 +448,7 @@ func writeTestIndexDataToDisk( if err != nil { return err } - segmentWriter, err := idxpersist.NewMutableSegmentFileSetWriter() + segmentWriter, err := idxpersist.NewMutableSegmentFileSetWriter(fst.WriterOptions{}) if err != nil { return err } diff --git a/src/dbnode/persist/fs/options.go b/src/dbnode/persist/fs/options.go index 042494b963..3fd6aa95cc 100644 --- a/src/dbnode/persist/fs/options.go +++ b/src/dbnode/persist/fs/options.go @@ -78,6 +78,7 @@ var ( defaultFilePathPrefix = os.TempDir() defaultNewFileMode = os.FileMode(0666) defaultNewDirectoryMode = os.ModeDir | os.FileMode(0755) + defaultFSTWriterOptions = fst.WriterOptions{} errTagEncoderPoolNotSet = errors.New("tag encoder pool is not set") errTagDecoderPoolNotSet = errors.New("tag decoder pool is not set") @@ -101,6 +102,7 @@ type options struct { tagEncoderPool serialize.TagEncoderPool tagDecoderPool serialize.TagDecoderPool fstOptions fst.Options + fstWriterOptions fst.WriterOptions forceIndexSummariesMmapMemory bool forceBloomFilterMmapMemory bool mmapEnableHugePages bool @@ -140,6 +142,7 @@ func NewOptions() Options { tagEncoderPool: tagEncoderPool, tagDecoderPool: tagDecoderPool, fstOptions: fstOptions, + fstWriterOptions: defaultFSTWriterOptions, indexReaderAutovalidateIndexSegments: defaultIndexReaderAutovalidateIndexSegments, } } @@ -364,6 +367,16 @@ func (o *options) FSTOptions() fst.Options { return o.fstOptions } +func (o *options) SetFSTWriterOptions(value fst.WriterOptions) Options { + opts := *o + opts.fstWriterOptions = value + return &opts +} + +func (o *options) FSTWriterOptions() fst.WriterOptions { + return o.fstWriterOptions +} + func (o *options) SetMmapReporter(mmapReporter mmap.Reporter) Options { opts := *o opts.mmapReporter = mmapReporter diff --git a/src/dbnode/persist/fs/persist_manager.go b/src/dbnode/persist/fs/persist_manager.go index 867f8fea88..752e48f656 100644 --- a/src/dbnode/persist/fs/persist_manager.go +++ b/src/dbnode/persist/fs/persist_manager.go @@ -167,7 +167,9 @@ func NewPersistManager(opts Options) (persist.Manager, error) { if err != nil { return nil, err } - segmentWriter, err := m3ninxpersist.NewMutableSegmentFileSetWriter() + + segmentWriter, err := m3ninxpersist.NewMutableSegmentFileSetWriter( + opts.FSTWriterOptions()) if err != nil { return nil, err } diff --git a/src/dbnode/persist/fs/types.go b/src/dbnode/persist/fs/types.go index 45ceb6f70c..7c15b6d56e 100644 --- a/src/dbnode/persist/fs/types.go +++ b/src/dbnode/persist/fs/types.go @@ -477,12 +477,18 @@ type Options interface { // TagDecoderPool returns the tag decoder pool. TagDecoderPool() serialize.TagDecoderPool - // SetFStOptions sets the fst options. + // SetFSTOptions sets the fst options. SetFSTOptions(value fst.Options) Options // FSTOptions returns the fst options. FSTOptions() fst.Options + // SetFStWriterOptions sets the fst writer options. + SetFSTWriterOptions(value fst.WriterOptions) Options + + // FSTWriterOptions returns the fst writer options. + FSTWriterOptions() fst.WriterOptions + // SetMmapReporter sets the mmap reporter. SetMmapReporter(value mmap.Reporter) Options diff --git a/src/m3ninx/persist/writer.go b/src/m3ninx/persist/writer.go index 0f7526072a..f709c8be24 100644 --- a/src/m3ninx/persist/writer.go +++ b/src/m3ninx/persist/writer.go @@ -36,15 +36,19 @@ var ( // NewMutableSegmentFileSetWriter returns a new IndexSegmentFileSetWriter for writing // out the provided Mutable Segment. -func NewMutableSegmentFileSetWriter() (MutableSegmentFileSetWriter, error) { - w, err := fst.NewWriter(fst.WriterOptions{}) +func NewMutableSegmentFileSetWriter( + fstOpts fst.WriterOptions, +) (MutableSegmentFileSetWriter, error) { + w, err := fst.NewWriter(fstOpts) if err != nil { return nil, err } return newMutableSegmentFileSetWriter(w) } -func newMutableSegmentFileSetWriter(fsWriter fst.Writer) (MutableSegmentFileSetWriter, error) { +func newMutableSegmentFileSetWriter( + fsWriter fst.Writer, +) (MutableSegmentFileSetWriter, error) { return &writer{ fsWriter: fsWriter, }, nil From 1b9a46359037cab5165b7e3642a63dd7e6adf992 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Wed, 22 Jul 2020 02:31:10 -0400 Subject: [PATCH 38/38] Fix test and revise tchannel send frame default value --- src/dbnode/storage/index_test.go | 2 + src/dbnode/storage/namespace_test.go | 10 ++-- src/dbnode/storage/shard_insert_queue_test.go | 14 +++-- src/dbnode/storage/shard_race_prop_test.go | 2 +- src/dbnode/storage/shard_test.go | 53 ++++++++++++------- src/dbnode/x/tchannel/options.go | 2 +- 6 files changed, 54 insertions(+), 29 deletions(-) diff --git a/src/dbnode/storage/index_test.go b/src/dbnode/storage/index_test.go index c58bc9eb4a..1cf5388481 100644 --- a/src/dbnode/storage/index_test.go +++ b/src/dbnode/storage/index_test.go @@ -308,6 +308,7 @@ func TestNamespaceIndexFlushShardStateNotSuccess(t *testing.T) { mockBlock.EXPECT().EndTime().Return(blockTime.Add(test.indexBlockSize)).AnyTimes() idx.state.blocksByTime[xtime.ToUnixNano(blockTime)] = mockBlock + mockBlock.EXPECT().IsSealed().Return(true) mockBlock.EXPECT().Close().Return(nil) mockShard := NewMockdatabaseShard(ctrl) @@ -440,6 +441,7 @@ func verifyFlushForShards( gomock.Any(), gomock.Any(), block.FetchBlocksMetadataOptions{OnlyDisk: true}).Return(results, nil, nil) } + mockBlock.EXPECT().IsSealed().Return(true) mockBlock.EXPECT().AddResults(gomock.Any()).Return(nil) mockBlock.EXPECT().EvictMutableSegments().Return(nil) } diff --git a/src/dbnode/storage/namespace_test.go b/src/dbnode/storage/namespace_test.go index 382536952f..7db0854009 100644 --- a/src/dbnode/storage/namespace_test.go +++ b/src/dbnode/storage/namespace_test.go @@ -362,7 +362,7 @@ func TestNamespaceBootstrapAllShards(t *testing.T) { shard := NewMockdatabaseShard(ctrl) shard.EXPECT().IsBootstrapped().Return(false) shard.EXPECT().ID().Return(shardID) - shard.EXPECT().Bootstrap(gomock.Any()).Return(errs[i]) + shard.EXPECT().Bootstrap(gomock.Any(), gomock.Any()).Return(errs[i]) ns.shards[testShardIDs[i].ID()] = shard shardIDs = append(shardIDs, shardID) } @@ -407,7 +407,7 @@ func TestNamespaceBootstrapOnlyNonBootstrappedShards(t *testing.T) { shard := NewMockdatabaseShard(ctrl) shard.EXPECT().IsBootstrapped().Return(false) shard.EXPECT().ID().Return(testShard.ID()) - shard.EXPECT().Bootstrap(gomock.Any()).Return(nil) + shard.EXPECT().Bootstrap(gomock.Any(), gomock.Any()).Return(nil) ns.shards[testShard.ID()] = shard shardIDs = append(shardIDs, testShard.ID()) } @@ -1215,12 +1215,16 @@ func TestNamespaceTicksIndex(t *testing.T) { ns, closer := newTestNamespaceWithIndex(t, idx) defer closer() + ns.RLock() + nsCtx := ns.nsContextWithRLock() + ns.RUnlock() + ctx := context.NewContext() defer ctx.Close() for _, s := range ns.shards { if s != nil { - s.Bootstrap(ctx) + s.Bootstrap(ctx, nsCtx) } } diff --git a/src/dbnode/storage/shard_insert_queue_test.go b/src/dbnode/storage/shard_insert_queue_test.go index df3a76b3c6..1190a80bb3 100644 --- a/src/dbnode/storage/shard_insert_queue_test.go +++ b/src/dbnode/storage/shard_insert_queue_test.go @@ -48,8 +48,8 @@ func TestShardInsertQueueBatchBackoff(t *testing.T) { currTime = currTime.Add(d) } backoff = 10 * time.Millisecond - insertWgs [64]sync.WaitGroup - insertProgressWgs [64]sync.WaitGroup + insertWgs [3]sync.WaitGroup + insertProgressWgs [3]sync.WaitGroup ) for i := range insertWgs { insertWgs[i].Add(1) @@ -58,6 +58,10 @@ func TestShardInsertQueueBatchBackoff(t *testing.T) { insertProgressWgs[i].Add(1) } q := newDatabaseShardInsertQueue(func(value []dbShardInsert) error { + if len(inserts) == len(insertWgs) { + return nil // Overflow. + } + inserts = append(inserts, value) insertWgs[len(inserts)-1].Done() insertProgressWgs[len(inserts)-1].Wait() @@ -187,9 +191,9 @@ func TestShardInsertQueueRateLimit(t *testing.T) { require.NoError(t, err) q.Lock() - expectedCurrWindow := currTime.Truncate(time.Second).UnixNano() - assert.Equal(t, expectedCurrWindow, q.insertPerSecondLimitWindowNanos) - assert.Equal(t, 1, q.insertPerSecondLimitWindowValues) + expectedCurrWindow := uint64(currTime.Truncate(time.Second).UnixNano()) + assert.Equal(t, expectedCurrWindow, q.insertPerSecondLimitWindowNanos.Load()) + assert.Equal(t, uint64(1), q.insertPerSecondLimitWindowValues.Load()) q.Unlock() } diff --git a/src/dbnode/storage/shard_race_prop_test.go b/src/dbnode/storage/shard_race_prop_test.go index 294f17aa16..3d90be156b 100644 --- a/src/dbnode/storage/shard_race_prop_test.go +++ b/src/dbnode/storage/shard_race_prop_test.go @@ -276,7 +276,7 @@ func TestShardTickBootstrapWriteRace(t *testing.T) { ctx := context.NewContext() defer ctx.Close() - assert.NoError(t, shard.Bootstrap(ctx)) + assert.NoError(t, shard.Bootstrap(ctx, namespace.Context{ID: ident.StringID("foo")})) for _, id := range writeIDs { id := id go func() { diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index 3b15b7e4b0..21791b7283 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -132,8 +132,9 @@ func TestShardBootstrapState(t *testing.T) { ctx := context.NewContext() defer ctx.Close() - require.NoError(t, s.Bootstrap(ctx)) - require.Error(t, s.Bootstrap(ctx)) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + require.NoError(t, s.Bootstrap(ctx, nsCtx)) + require.Error(t, s.Bootstrap(ctx, nsCtx)) } func TestShardFlushStateNotStarted(t *testing.T) { @@ -163,7 +164,8 @@ func TestShardFlushStateNotStarted(t *testing.T) { ctx := context.NewContext() defer ctx.Close() - s.Bootstrap(ctx) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + s.Bootstrap(ctx, nsCtx) notStarted := fileOpState{WarmStatus: fileOpNotStarted} for st := earliest; !st.After(latest); st = st.Add(ropts.BlockSize()) { @@ -201,6 +203,7 @@ func TestShardBootstrapWithFlushVersion(t *testing.T) { mockSeries := series.NewMockDatabaseSeries(ctrl) mockSeries.EXPECT().ID().Return(mockSeriesID).AnyTimes() mockSeries.EXPECT().IsEmpty().Return(false).AnyTimes() + mockSeries.EXPECT().Bootstrap(gomock.Any()) // Load the mock into the shard as an expected series so that we can assert // on the call to its Bootstrap() method below. @@ -233,7 +236,8 @@ func TestShardBootstrapWithFlushVersion(t *testing.T) { ctx := context.NewContext() defer ctx.Close() - err = s.Bootstrap(ctx) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + err = s.Bootstrap(ctx, nsCtx) require.NoError(t, err) require.Equal(t, Bootstrapped, s.bootstrapState) @@ -297,7 +301,8 @@ func TestShardBootstrapWithFlushVersionNoCleanUp(t *testing.T) { ctx := context.NewContext() defer ctx.Close() - err = s.Bootstrap(ctx) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + err = s.Bootstrap(ctx, nsCtx) require.NoError(t, err) require.Equal(t, Bootstrapped, s.bootstrapState) @@ -335,7 +340,8 @@ func TestShardBootstrapWithCacheShardIndices(t *testing.T) { ctx := context.NewContext() defer ctx.Close() - err = s.Bootstrap(ctx) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + err = s.Bootstrap(ctx, nsCtx) require.NoError(t, err) require.Equal(t, Bootstrapped, s.bootstrapState) } @@ -386,7 +392,8 @@ func testShardLoadLimit(t *testing.T, limit int64, shouldReturnError bool) { ctx := context.NewContext() defer ctx.Close() - require.NoError(t, s.Bootstrap(ctx)) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + require.NoError(t, s.Bootstrap(ctx, nsCtx)) // First load will never trigger the limit. require.NoError(t, s.LoadBlocks(seriesMap)) @@ -410,7 +417,8 @@ func TestShardFlushSeriesFlushError(t *testing.T) { ctx := context.NewContext() defer ctx.Close() - s.Bootstrap(ctx) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + s.Bootstrap(ctx, nsCtx) s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] = fileOpState{ WarmStatus: fileOpFailed, @@ -487,7 +495,8 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { ctx := context.NewContext() defer ctx.Close() - s.Bootstrap(ctx) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + s.Bootstrap(ctx, nsCtx) s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] = fileOpState{ WarmStatus: fileOpFailed, @@ -581,7 +590,8 @@ func TestShardColdFlush(t *testing.T) { ctx := context.NewContext() defer ctx.Close() - require.NoError(t, shard.Bootstrap(ctx)) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + require.NoError(t, shard.Bootstrap(ctx, nsCtx)) shard.newMergerFn = newMergerTestFn shard.newFSMergeWithMemFn = newFSMergeWithMemTestFn @@ -628,7 +638,6 @@ func TestShardColdFlush(t *testing.T) { idElementPool: newIDElementPool(nil), fsReader: fsReader, } - nsCtx := namespace.Context{} // Assert that flush state cold versions all start at 0. for i := t0; i.Before(t7.Add(blockSize)); i = i.Add(blockSize) { @@ -667,7 +676,8 @@ func TestShardColdFlushNoMergeIfNothingDirty(t *testing.T) { ctx := context.NewContext() defer ctx.Close() - require.NoError(t, shard.Bootstrap(ctx)) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + require.NoError(t, shard.Bootstrap(ctx, nsCtx)) shard.newMergerFn = newMergerTestFn shard.newFSMergeWithMemFn = newFSMergeWithMemTestFn @@ -701,7 +711,6 @@ func TestShardColdFlushNoMergeIfNothingDirty(t *testing.T) { idElementPool: idElementPool, fsReader: fsReader, } - nsCtx := namespace.Context{} shardColdFlush, err := shard.ColdFlush(preparer, resources, nsCtx, &persist.NoOpColdFlushNamespace{}) require.NoError(t, err) @@ -929,7 +938,8 @@ func TestShardTick(t *testing.T) { defer ctx.Close() shard := testDatabaseShard(t, opts) - shard.Bootstrap(ctx) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + shard.Bootstrap(ctx, nsCtx) shard.SetRuntimeOptions(runtime.NewOptions(). SetTickPerSeriesSleepDuration(sleepPerSeries). SetTickSeriesBatchSize(1)) @@ -1098,7 +1108,8 @@ func testShardWriteAsync(t *testing.T, writes []testWrite) { defer ctx.Close() shard := testDatabaseShard(t, opts) - shard.Bootstrap(ctx) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + shard.Bootstrap(ctx, nsCtx) shard.SetRuntimeOptions(runtime.NewOptions(). SetWriteNewSeriesAsync(true). SetTickPerSeriesSleepDuration(sleepPerSeries). @@ -1156,7 +1167,8 @@ func TestShardTickRace(t *testing.T) { ctx := context.NewContext() defer ctx.Close() - shard.Bootstrap(ctx) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + shard.Bootstrap(ctx, nsCtx) addTestSeries(shard, ident.StringID("foo")) var wg sync.WaitGroup @@ -1189,7 +1201,8 @@ func TestShardTickCleanupSmallBatchSize(t *testing.T) { defer ctx.Close() shard := testDatabaseShard(t, opts) - shard.Bootstrap(ctx) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + shard.Bootstrap(ctx, nsCtx) addTestSeries(shard, ident.StringID("foo")) shard.Tick(context.NewNoOpCanncellable(), time.Now(), namespace.Context{}) @@ -1216,7 +1229,8 @@ func TestShardReturnsErrorForConcurrentTicks(t *testing.T) { defer ctx.Close() shard := testDatabaseShard(t, opts) - shard.Bootstrap(ctx) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + shard.Bootstrap(ctx, nsCtx) shard.currRuntimeOptions.tickSleepSeriesBatchSize = 1 shard.currRuntimeOptions.tickSleepPerSeries = time.Millisecond @@ -1571,7 +1585,8 @@ func TestShardReadEncodedCachesSeriesWithRecentlyReadPolicy(t *testing.T) { ctx := context.NewContext() defer ctx.Close() - require.NoError(t, shard.Bootstrap(ctx)) + nsCtx := namespace.Context{ID: ident.StringID("foo")} + require.NoError(t, shard.Bootstrap(ctx, nsCtx)) ropts := shard.seriesOpts.RetentionOptions() end := opts.ClockOptions().NowFn()().Truncate(ropts.BlockSize()) diff --git a/src/dbnode/x/tchannel/options.go b/src/dbnode/x/tchannel/options.go index 1832884935..d3f9775c58 100644 --- a/src/dbnode/x/tchannel/options.go +++ b/src/dbnode/x/tchannel/options.go @@ -31,7 +31,7 @@ const ( defaultMaxIdleTime = 5 * time.Minute // defaultSendBufferSize sets the default send buffer size, // by default only 512 frames would be buffered. - defaultSendBufferSize = 16384 + defaultSendBufferSize = 4096 ) // NewDefaultChannelOptions returns the default tchannel options used.