From e76a41b7b7c305f4f9565fd7a72d742f6f9d7f1c Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Wed, 8 Apr 2020 16:03:48 -0400 Subject: [PATCH 01/91] Initial disk queue skeleton --- libbeat/publisher/queue/diskqueue/config.go | 35 +++++++++++ libbeat/publisher/queue/diskqueue/consumer.go | 52 ++++++++++++++++ libbeat/publisher/queue/diskqueue/producer.go | 47 +++++++++++++++ libbeat/publisher/queue/diskqueue/queue.go | 59 +++++++++++++++++++ 4 files changed, 193 insertions(+) create mode 100644 libbeat/publisher/queue/diskqueue/config.go create mode 100644 libbeat/publisher/queue/diskqueue/consumer.go create mode 100644 libbeat/publisher/queue/diskqueue/producer.go create mode 100644 libbeat/publisher/queue/diskqueue/queue.go diff --git a/libbeat/publisher/queue/diskqueue/config.go b/libbeat/publisher/queue/diskqueue/config.go new file mode 100644 index 00000000000..146a8e09cc0 --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/config.go @@ -0,0 +1,35 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +import ( + "errors" +) + +type config struct { +} + +var defaultConfig = config{} + +func (c *config) Validate() error { + if false { + return errors.New("something is wrong") + } + + return nil +} diff --git a/libbeat/publisher/queue/diskqueue/consumer.go b/libbeat/publisher/queue/diskqueue/consumer.go new file mode 100644 index 00000000000..979fbcc6b70 --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/consumer.go @@ -0,0 +1,52 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +import ( + "github.com/elastic/beats/v7/libbeat/publisher" + "github.com/elastic/beats/v7/libbeat/publisher/queue" +) + +type diskQueueConsumer struct { +} + +type diskQueueBatch struct { +} + +// +// diskQueueConsumer implementation of the queue.Consumer interface +// + +func (consumer *diskQueueConsumer) Get(eventCount int) (queue.Batch, error) { + return &diskQueueBatch{}, nil +} + +func (consumer *diskQueueConsumer) Close() error { + return nil +} + +// +// diskQueueBatch implementation of the queue.Batch interface +// + +func (batch *diskQueueBatch) Events() []publisher.Event { + return nil +} + +func (batch *diskQueueBatch) ACK() { +} diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go new file mode 100644 index 00000000000..f2bf2965435 --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -0,0 +1,47 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +import ( + "github.com/elastic/beats/v7/libbeat/publisher" + "github.com/elastic/beats/v7/libbeat/publisher/queue" +) + +type diskQueueProducer struct { + // The disk queue that created this producer. + queue *diskQueue + + // The configuration this producer was created with. + config queue.ProducerConfig +} + +// +// diskQueueProducer implementation of the queue.Producer interface +// + +func (producer *diskQueueProducer) Publish(event publisher.Event) bool { + return false +} + +func (producer *diskQueueProducer) TryPublish(event publisher.Event) bool { + return false +} + +func (producer *diskQueueProducer) Cancel() int { + return 0 +} diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go new file mode 100644 index 00000000000..48630243dd9 --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -0,0 +1,59 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +import ( + "github.com/elastic/beats/v7/libbeat/logp" + "github.com/elastic/beats/v7/libbeat/publisher/queue" +) + +type diskQueue struct { +} + +// NewQueue returns a disk-based queue configured with the given logger +// and settings. +func NewQueue( + logger *logp.Logger, + ackListener queue.ACKListener, +) queue.Queue { + + return &diskQueue{} +} + +// +// diskQueue mplementation of the queue.Queue interface +// + +func (dq *diskQueue) Close() error { + return nil +} + +func (dq *diskQueue) BufferConfig() queue.BufferConfig { + return queue.BufferConfig{Events: 0} +} + +func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { + return &diskQueueProducer{ + queue: dq, + config: cfg, + } +} + +func (dq *diskQueue) Consumer() queue.Consumer { + return &diskQueueConsumer{} +} From e8c81280b48092433d8d385200d75cb940695f9c Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Fri, 10 Apr 2020 12:34:24 -0400 Subject: [PATCH 02/91] Sketching out some top-level disk queue data structures --- libbeat/publisher/queue/diskqueue/queue.go | 102 +++++++++++++++++++-- 1 file changed, 95 insertions(+), 7 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 48630243dd9..720bc8350b9 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -22,17 +22,105 @@ import ( "github.com/elastic/beats/v7/libbeat/publisher/queue" ) +// Settings contains the configuration fields to create a new disk queue. +type Settings struct { + // The destination for log messages related to the disk queue. + Logger *logp.Logger + + // A listener that receives ACKs when events are written to the queue's + // disk buffer. + WriteToDiskACKListener queue.ACKListener + + // A listener that receives ACKs when events are removed from the queue + // and written to their output. + WriteToOutputACKListener queue.ACKListener + + // The size in bytes of one data page in the on-disk buffer. To minimize + // data loss if there is an error, this should match the page size of the + // target filesystem. + PageSize uint32 + + // MaxBufferSize is the maximum number of bytes that the queue should + // ever occupy on disk. A value of 0 means the queue can grow until the + // disk is full. + MaxBufferSize uint64 +} + +type bufferPosition struct { + // The segment index of this position within the overall buffer. + segmentIndex uint64 + + // The page index of this position within its segment. + pageIndex uint64 + + // The byte index of this position within its page's data region. + byteIndex uint32 +} + +type diskQueueState struct { + // The page size of the queue. This is originally derived from + // Settings.PageSize, and the two must match during normal queue operation. + // They can only differ during data recovery / page size migration. + pageSize uint32 + + // The oldest position in the queue. This is advanced as we receive ACKs from + // downstream consumers indicating it is safe to remove old events. + firstPosition bufferPosition + + // The position of the next (unwritten) byte in the queue buffer. When an + // event is added to the queue, this position is advanced to point to the + // first byte after its end. + lastPosition bufferPosition + + // The maximum number of pages that can be used for the queue buffer. + // This is derived by dividing Settings.MaxBufferSize by pageSize and + // rounding down. + maxPageCount uint64 + + // The number of pages currently occupied by the queue buffer. This can't + // be derived from firstPosition and lastPosition because segment length + // varies with the size of their last event. + // This can be greater than maxPageCount if the maximum buffer size is + // reduced on an already-full queue. + allocatedPageCount uint64 +} + +// diskQueue is the internal type representing a disk-based implementation +// of queue.Queue. type diskQueue struct { + settings Settings + + // The persistent queue state. After a filesystem sync this should be + // identical to the queue's metadata file. + state diskQueueState + + // The position of the next event to read from the queue. If this equals + // state.lastPosition, then there are no events left to read. + // This is initialized to state.firstPosition, but generally the two differ: + // readPosition is advanced when an event is read, but firstPosition is + // only advanced when the event has been read _and_ its consumer receives + // an acknowledgement (meaning it has been transmitted and can be removed + // from the queue). + // This is part of diskQueue and not diskQueueState since it represents + // in-memory state that should not persist through a restart. + readPosition bufferPosition } +/*var _ = queue.Feature("disk", create, + feature.MakeDetails( + "Disk queue", + "Buffer events on disk before sending to the output.", + feature.Beta), +) + +func init() { + queue.RegisterType("disk", create) +}*/ + // NewQueue returns a disk-based queue configured with the given logger // and settings. -func NewQueue( - logger *logp.Logger, - ackListener queue.ACKListener, -) queue.Queue { - - return &diskQueue{} +func NewQueue(settings Settings) queue.Queue { + return &diskQueue{settings: settings} } // @@ -44,7 +132,7 @@ func (dq *diskQueue) Close() error { } func (dq *diskQueue) BufferConfig() queue.BufferConfig { - return queue.BufferConfig{Events: 0} + return queue.BufferConfig{MaxEvents: 0} } func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { From 97a7ed53d0e512317d9223e3add9db03469b6312 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 16 Apr 2020 10:18:40 -0400 Subject: [PATCH 03/91] add queue type registration --- libbeat/publisher/queue/diskqueue/queue.go | 64 +++++++++++++++------- 1 file changed, 43 insertions(+), 21 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 720bc8350b9..9a382c4c90b 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -18,22 +18,19 @@ package diskqueue import ( + "github.com/elastic/beats/v7/libbeat/common" "github.com/elastic/beats/v7/libbeat/logp" + "github.com/elastic/beats/v7/libbeat/paths" "github.com/elastic/beats/v7/libbeat/publisher/queue" ) -// Settings contains the configuration fields to create a new disk queue. +// Settings contains the configuration fields to create a new disk queue +// or open an existing one. type Settings struct { - // The destination for log messages related to the disk queue. - Logger *logp.Logger - - // A listener that receives ACKs when events are written to the queue's - // disk buffer. - WriteToDiskACKListener queue.ACKListener - - // A listener that receives ACKs when events are removed from the queue - // and written to their output. - WriteToOutputACKListener queue.ACKListener + // The path on disk of the queue's main metadata file. + // Paths for data segment files are created by appending ".{segmentIndex}" to + // this path. + Path string // The size in bytes of one data page in the on-disk buffer. To minimize // data loss if there is an error, this should match the page size of the @@ -44,6 +41,17 @@ type Settings struct { // ever occupy on disk. A value of 0 means the queue can grow until the // disk is full. MaxBufferSize uint64 + + // The destination for log messages related to the disk queue. + Logger *logp.Logger + + // A listener that receives ACKs when events are written to the queue's + // disk buffer. + WriteToDiskACKListener queue.ACKListener + + // A listener that receives ACKs when events are removed from the queue + // and written to their output. + WriteToOutputACKListener queue.ACKListener } type bufferPosition struct { @@ -106,21 +114,35 @@ type diskQueue struct { readPosition bufferPosition } -/*var _ = queue.Feature("disk", create, - feature.MakeDetails( - "Disk queue", - "Buffer events on disk before sending to the output.", - feature.Beta), -) +/* +func init() { + queue.RegisterQueueType( + "disk", + create, + feature.MakeDetails( + "Disk queue", + "Buffer events on disk before sending to the output.", + feature.Beta)) +} +*/ func init() { - queue.RegisterType("disk", create) -}*/ + queue.RegisterType("disk", queueFactory) +} + +func queueFactory( + ackListener queue.ACKListener, logger *logp.Logger, cfg *common.Config, +) (queue.Queue, error) { + return nil, nil +} // NewQueue returns a disk-based queue configured with the given logger // and settings. -func NewQueue(settings Settings) queue.Queue { - return &diskQueue{settings: settings} +func NewQueue(settings Settings) (queue.Queue, error) { + if settings.Path == "" { + settings.Path = paths.Resolve(paths.Data, "queue.dat") + } + return &diskQueue{settings: settings}, nil } // From a78b85c54f7841276ab07bb887a77c2d2db70176 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 16 Apr 2020 10:37:21 -0400 Subject: [PATCH 04/91] use new registry helper --- libbeat/publisher/queue/diskqueue/queue.go | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 9a382c4c90b..ca5c33e0aef 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -19,6 +19,7 @@ package diskqueue import ( "github.com/elastic/beats/v7/libbeat/common" + "github.com/elastic/beats/v7/libbeat/feature" "github.com/elastic/beats/v7/libbeat/logp" "github.com/elastic/beats/v7/libbeat/paths" "github.com/elastic/beats/v7/libbeat/publisher/queue" @@ -114,21 +115,15 @@ type diskQueue struct { readPosition bufferPosition } -/* func init() { queue.RegisterQueueType( "disk", - create, + queueFactory, feature.MakeDetails( "Disk queue", "Buffer events on disk before sending to the output.", feature.Beta)) } -*/ - -func init() { - queue.RegisterType("disk", queueFactory) -} func queueFactory( ackListener queue.ACKListener, logger *logp.Logger, cfg *common.Config, From b12020c90f5b3458dbe02ca87ac6521730844848 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 16 Apr 2020 11:36:14 -0400 Subject: [PATCH 05/91] connect external user config to the queue's Settings struct --- libbeat/publisher/queue/diskqueue/config.go | 24 +++++++++++++++++---- libbeat/publisher/queue/diskqueue/queue.go | 9 +++++++- 2 files changed, 28 insertions(+), 5 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/config.go b/libbeat/publisher/queue/diskqueue/config.go index 146a8e09cc0..7dc73c63be8 100644 --- a/libbeat/publisher/queue/diskqueue/config.go +++ b/libbeat/publisher/queue/diskqueue/config.go @@ -19,17 +19,33 @@ package diskqueue import ( "errors" + + "github.com/elastic/beats/v7/libbeat/common" ) -type config struct { +// userConfig holds the parameters for a disk queue that are configurable +// by the end user in the beats yml file. +type userConfig struct { + PageSize uint32 `config:"page_size" validate:"min=128"` } -var defaultConfig = config{} - -func (c *config) Validate() error { +func (c *userConfig) Validate() error { if false { return errors.New("something is wrong") } return nil } + +// SettingsForUserConfig returns a Settings struct initialized with the +// end-user-configurable settings in the given config tree. +func SettingsForUserConfig(config *common.Config) (Settings, error) { + userConfig := userConfig{} + if err := config.Unpack(&userConfig); err != nil { + return Settings{}, err + } + + return Settings{ + PageSize: userConfig.PageSize, + }, nil +} diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index ca5c33e0aef..f9d31c5eb6f 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -31,6 +31,7 @@ type Settings struct { // The path on disk of the queue's main metadata file. // Paths for data segment files are created by appending ".{segmentIndex}" to // this path. + // If blank, the default is "queue.dat" within the beat's data directory. Path string // The size in bytes of one data page in the on-disk buffer. To minimize @@ -125,10 +126,16 @@ func init() { feature.Beta)) } +// queueFactory matches the queue.Factory type, and is used to add the disk +// queue to the registry. func queueFactory( ackListener queue.ACKListener, logger *logp.Logger, cfg *common.Config, ) (queue.Queue, error) { - return nil, nil + settings, err := SettingsForUserConfig(cfg) + if err != nil { + return nil, err + } + return NewQueue(settings) } // NewQueue returns a disk-based queue configured with the given logger From 67540d69be3f34de0327c7fd893f8c55d47450e5 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 16 Apr 2020 11:44:52 -0400 Subject: [PATCH 06/91] Fill out more default settings --- libbeat/publisher/queue/diskqueue/queue.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index f9d31c5eb6f..2df6a518ee4 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -135,6 +135,12 @@ func queueFactory( if err != nil { return nil, err } + settings.Logger = logger + // For now, incoming messages are acked when they are written to disk + // (rather than transmitted to the output, as with the memory queue). This + // can produce unexpected behavior in some contexts and we might want to + // make it configurable later. + settings.WriteToDiskACKListener = ackListener return NewQueue(settings) } From 94f125c7219364db31df6f4cbfb9146e2af09884 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 23 Apr 2020 15:12:16 -0400 Subject: [PATCH 07/91] review comments --- libbeat/publisher/queue/diskqueue/queue.go | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 2df6a518ee4..55f0a187822 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -28,10 +28,15 @@ import ( // Settings contains the configuration fields to create a new disk queue // or open an existing one. type Settings struct { - // The path on disk of the queue's main metadata file. - // Paths for data segment files are created by appending ".{segmentIndex}" to - // this path. - // If blank, the default is "queue.dat" within the beat's data directory. + // The destination for log messages related to the disk queue. + Logger *logp.Logger + + // The path on disk of the queue's containing directory, which will be + // created if it doesn't exist. Within the directory, the queue's state + // is stored in queue.dat and each segment's data is stored in + // segment.{segmentIndex} + // If blank, the default directory is "diskqueue" within the beat's data + // directory. Path string // The size in bytes of one data page in the on-disk buffer. To minimize @@ -44,9 +49,6 @@ type Settings struct { // disk is full. MaxBufferSize uint64 - // The destination for log messages related to the disk queue. - Logger *logp.Logger - // A listener that receives ACKs when events are written to the queue's // disk buffer. WriteToDiskACKListener queue.ACKListener From f29a96ff426616a6aafcf015db3015d6417c145d Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 23 Apr 2020 15:14:40 -0400 Subject: [PATCH 08/91] review comments (add panic to unimplemented functions) --- libbeat/publisher/queue/diskqueue/consumer.go | 7 ++++--- libbeat/publisher/queue/diskqueue/producer.go | 6 +++--- libbeat/publisher/queue/diskqueue/queue.go | 9 +++++---- 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/consumer.go b/libbeat/publisher/queue/diskqueue/consumer.go index 979fbcc6b70..eba7e89a61e 100644 --- a/libbeat/publisher/queue/diskqueue/consumer.go +++ b/libbeat/publisher/queue/diskqueue/consumer.go @@ -33,11 +33,11 @@ type diskQueueBatch struct { // func (consumer *diskQueueConsumer) Get(eventCount int) (queue.Batch, error) { - return &diskQueueBatch{}, nil + panic("TODO: not implemented") } func (consumer *diskQueueConsumer) Close() error { - return nil + panic("TODO: not implemented") } // @@ -45,8 +45,9 @@ func (consumer *diskQueueConsumer) Close() error { // func (batch *diskQueueBatch) Events() []publisher.Event { - return nil + panic("TODO: not implemented") } func (batch *diskQueueBatch) ACK() { + panic("TODO: not implemented") } diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index f2bf2965435..b4ed703e095 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -35,13 +35,13 @@ type diskQueueProducer struct { // func (producer *diskQueueProducer) Publish(event publisher.Event) bool { - return false + panic("TODO: not implemented") } func (producer *diskQueueProducer) TryPublish(event publisher.Event) bool { - return false + panic("TODO: not implemented") } func (producer *diskQueueProducer) Cancel() int { - return 0 + panic("TODO: not implemented") } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 55f0a187822..7e2d159c514 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -160,7 +160,7 @@ func NewQueue(settings Settings) (queue.Queue, error) { // func (dq *diskQueue) Close() error { - return nil + panic("TODO: not implemented") } func (dq *diskQueue) BufferConfig() queue.BufferConfig { @@ -168,12 +168,13 @@ func (dq *diskQueue) BufferConfig() queue.BufferConfig { } func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { - return &diskQueueProducer{ + /*return &diskQueueProducer{ queue: dq, config: cfg, - } + }*/ + panic("TODO: not implemented") } func (dq *diskQueue) Consumer() queue.Consumer { - return &diskQueueConsumer{} + panic("TODO: not implemented") } From a04980e3153c08429e67354d0c83e30a8d8ec8b9 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Wed, 27 May 2020 09:46:48 -0400 Subject: [PATCH 09/91] some state file stuff --- libbeat/publisher/queue/diskqueue/queue.go | 86 +++++++++++++++++++--- 1 file changed, 74 insertions(+), 12 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 7e2d159c514..1f9acd33728 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -18,6 +18,10 @@ package diskqueue import ( + "fmt" + "os" + "path/filepath" + "github.com/elastic/beats/v7/libbeat/common" "github.com/elastic/beats/v7/libbeat/feature" "github.com/elastic/beats/v7/libbeat/logp" @@ -84,16 +88,11 @@ type diskQueueState struct { // first byte after its end. lastPosition bufferPosition - // The maximum number of pages that can be used for the queue buffer. - // This is derived by dividing Settings.MaxBufferSize by pageSize and - // rounding down. - maxPageCount uint64 - // The number of pages currently occupied by the queue buffer. This can't // be derived from firstPosition and lastPosition because segment length // varies with the size of their last event. - // This can be greater than maxPageCount if the maximum buffer size is - // reduced on an already-full queue. + // This can be greater than diskQueue.maxPageCount if the maximum buffer + // size is reduced on an already-full queue. allocatedPageCount uint64 } @@ -104,7 +103,13 @@ type diskQueue struct { // The persistent queue state. After a filesystem sync this should be // identical to the queue's metadata file. - state diskQueueState + state diskQueueState + stateFile *stateFile + + // The maximum number of pages that can be used for the queue buffer. + // This is derived by dividing Settings.MaxBufferSize by pageSize and + // rounding down. + maxPageCount uint64 // The position of the next event to read from the queue. If this equals // state.lastPosition, then there are no events left to read. @@ -118,6 +123,8 @@ type diskQueue struct { readPosition bufferPosition } +type stateFile os.File + func init() { queue.RegisterQueueType( "disk", @@ -147,16 +154,35 @@ func queueFactory( } // NewQueue returns a disk-based queue configured with the given logger -// and settings. +// and settings, creating it if it doesn't exist. func NewQueue(settings Settings) (queue.Queue, error) { - if settings.Path == "" { - settings.Path = paths.Resolve(paths.Data, "queue.dat") + // Create the given directory path if it doesn't exist. + err := os.MkdirAll(settings.directoryPath(), os.ModePerm) + if err != nil { + return nil, fmt.Errorf("Couldn't create disk queue directory: %w", err) } + + // Load the file handle for the queue state. + stateFile, err := stateFileForPath(settings.stateFilePath()) + if err != nil { + return nil, fmt.Errorf("Couldn't open disk queue metadata file: %w", err) + } + defer func() { + if err != nil { + // If the function is returning because of an error, close the + // file handle. + stateFile.Close() + } + }() + + // Decode the state from the file. + stateFile.Seek(0) + return &diskQueue{settings: settings}, nil } // -// diskQueue mplementation of the queue.Queue interface +// diskQueue implementation of the queue.Queue interface // func (dq *diskQueue) Close() error { @@ -178,3 +204,39 @@ func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { func (dq *diskQueue) Consumer() queue.Consumer { panic("TODO: not implemented") } + +func (settings diskQueueSettings) directoryPath() string { + if settings.Path == "" { + return paths.Resolve(paths.Data, "diskqueue") + } + return settings.Path +} + +func (settings diskQueueSettings) stateFilePath() string { + return filepath.Join(settings.directoryPath(), "queue-state.dat") +} + +func (settings diskQueueSettings) segmentFilePath(segmentID uint64) string { + return filepath.Join(settings.directoryPath(), "segment.%v", segmentID) +} + +func queueStateFromPath(path string) (diskQueueState, error) { + return diskQueueState{}, nil +} + +func stateFileForPath(path string) (*stateFile, error) { + file, err := os.OpenFile(path, os.O_RDWR|os.O_CREATE, 0600) + if err != nil { + return nil, fmt.Errorf("Couldn't open disk queue metadata file: %w", err) + } + fileInfo, err := file.Stat() + if err != nil { + return nil, fmt.Errorf("Couldn't stat disk queue metadata file: %w", err) + } + if + return file, nil +} + +func (stateFile *stateFile) saveState(state diskQueueState) error { + return nil +} From 26b4248499668d2f299a382d35570fdd4acfe845 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Mon, 1 Jun 2020 13:27:58 -0400 Subject: [PATCH 10/91] revising code to match new design --- libbeat/publisher/queue/diskqueue/queue.go | 61 +++++++--------------- 1 file changed, 20 insertions(+), 41 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 1f9acd33728..549d6108c26 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -37,20 +37,15 @@ type Settings struct { // The path on disk of the queue's containing directory, which will be // created if it doesn't exist. Within the directory, the queue's state - // is stored in queue.dat and each segment's data is stored in - // segment.{segmentIndex} + // is stored in state.dat and each segment's data is stored in + // {segmentIndex}.seg // If blank, the default directory is "diskqueue" within the beat's data // directory. Path string - // The size in bytes of one data page in the on-disk buffer. To minimize - // data loss if there is an error, this should match the page size of the - // target filesystem. - PageSize uint32 - // MaxBufferSize is the maximum number of bytes that the queue should // ever occupy on disk. A value of 0 means the queue can grow until the - // disk is full. + // disk is full (this is not recommended on a primary system disk). MaxBufferSize uint64 // A listener that receives ACKs when events are written to the queue's @@ -63,37 +58,23 @@ type Settings struct { } type bufferPosition struct { - // The segment index of this position within the overall buffer. + // The index of this position's segment within the overall buffer. segmentIndex uint64 - // The page index of this position within its segment. - pageIndex uint64 - - // The byte index of this position within its page's data region. - byteIndex uint32 + // The byte offset of this position within its segment. + byteIndex uint64 } -type diskQueueState struct { - // The page size of the queue. This is originally derived from - // Settings.PageSize, and the two must match during normal queue operation. - // They can only differ during data recovery / page size migration. - pageSize uint32 - +// This is the queue metadata that is saved to disk. Currently it only +// tracks the read position in the queue; all other data is contained +// in the segment files. +type diskQueuePersistentState struct { + // The schema version for the state file (currently always 0). + version uint32 + // The oldest position in the queue. This is advanced as we receive ACKs from // downstream consumers indicating it is safe to remove old events. firstPosition bufferPosition - - // The position of the next (unwritten) byte in the queue buffer. When an - // event is added to the queue, this position is advanced to point to the - // first byte after its end. - lastPosition bufferPosition - - // The number of pages currently occupied by the queue buffer. This can't - // be derived from firstPosition and lastPosition because segment length - // varies with the size of their last event. - // This can be greater than diskQueue.maxPageCount if the maximum buffer - // size is reduced on an already-full queue. - allocatedPageCount uint64 } // diskQueue is the internal type representing a disk-based implementation @@ -103,16 +84,14 @@ type diskQueue struct { // The persistent queue state. After a filesystem sync this should be // identical to the queue's metadata file. - state diskQueueState + //state diskQueuePersistentState stateFile *stateFile - // The maximum number of pages that can be used for the queue buffer. - // This is derived by dividing Settings.MaxBufferSize by pageSize and - // rounding down. - maxPageCount uint64 + // + firstPosition bufferPosition // The position of the next event to read from the queue. If this equals - // state.lastPosition, then there are no events left to read. + // writePosition, then there are no events left to read. // This is initialized to state.firstPosition, but generally the two differ: // readPosition is advanced when an event is read, but firstPosition is // only advanced when the event has been read _and_ its consumer receives @@ -163,7 +142,7 @@ func NewQueue(settings Settings) (queue.Queue, error) { } // Load the file handle for the queue state. - stateFile, err := stateFileForPath(settings.stateFilePath()) + stateFile, state, err := stateFileForPath(settings.stateFilePath()) if err != nil { return nil, fmt.Errorf("Couldn't open disk queue metadata file: %w", err) } @@ -213,11 +192,11 @@ func (settings diskQueueSettings) directoryPath() string { } func (settings diskQueueSettings) stateFilePath() string { - return filepath.Join(settings.directoryPath(), "queue-state.dat") + return filepath.Join(settings.directoryPath(), "state.dat") } func (settings diskQueueSettings) segmentFilePath(segmentID uint64) string { - return filepath.Join(settings.directoryPath(), "segment.%v", segmentID) + return filepath.Join(settings.directoryPath(), "%v.seg", segmentID) } func queueStateFromPath(path string) (diskQueueState, error) { From f30f30b8730e327377ea9b320efacfa9c4741e97 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Mon, 1 Jun 2020 16:10:29 -0400 Subject: [PATCH 11/91] more state file handling --- libbeat/publisher/queue/diskqueue/config.go | 6 +- libbeat/publisher/queue/diskqueue/queue.go | 80 +++------ libbeat/publisher/queue/diskqueue/segments.go | 8 + .../publisher/queue/diskqueue/state_file.go | 161 ++++++++++++++++++ 4 files changed, 195 insertions(+), 60 deletions(-) create mode 100644 libbeat/publisher/queue/diskqueue/segments.go create mode 100644 libbeat/publisher/queue/diskqueue/state_file.go diff --git a/libbeat/publisher/queue/diskqueue/config.go b/libbeat/publisher/queue/diskqueue/config.go index 7dc73c63be8..66931924a16 100644 --- a/libbeat/publisher/queue/diskqueue/config.go +++ b/libbeat/publisher/queue/diskqueue/config.go @@ -26,7 +26,7 @@ import ( // userConfig holds the parameters for a disk queue that are configurable // by the end user in the beats yml file. type userConfig struct { - PageSize uint32 `config:"page_size" validate:"min=128"` + Path string `config:"path"` } func (c *userConfig) Validate() error { @@ -45,7 +45,5 @@ func SettingsForUserConfig(config *common.Config) (Settings, error) { return Settings{}, err } - return Settings{ - PageSize: userConfig.PageSize, - }, nil + return Settings{}, nil } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 549d6108c26..dc703cf8b94 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -65,18 +65,6 @@ type bufferPosition struct { byteIndex uint64 } -// This is the queue metadata that is saved to disk. Currently it only -// tracks the read position in the queue; all other data is contained -// in the segment files. -type diskQueuePersistentState struct { - // The schema version for the state file (currently always 0). - version uint32 - - // The oldest position in the queue. This is advanced as we receive ACKs from - // downstream consumers indicating it is safe to remove old events. - firstPosition bufferPosition -} - // diskQueue is the internal type representing a disk-based implementation // of queue.Queue. type diskQueue struct { @@ -87,7 +75,7 @@ type diskQueue struct { //state diskQueuePersistentState stateFile *stateFile - // + // firstPosition bufferPosition // The position of the next event to read from the queue. If this equals @@ -102,8 +90,6 @@ type diskQueue struct { readPosition bufferPosition } -type stateFile os.File - func init() { queue.RegisterQueueType( "disk", @@ -136,13 +122,13 @@ func queueFactory( // and settings, creating it if it doesn't exist. func NewQueue(settings Settings) (queue.Queue, error) { // Create the given directory path if it doesn't exist. - err := os.MkdirAll(settings.directoryPath(), os.ModePerm) + err := os.MkdirAll(settings.Path, os.ModePerm) if err != nil { return nil, fmt.Errorf("Couldn't create disk queue directory: %w", err) } // Load the file handle for the queue state. - stateFile, state, err := stateFileForPath(settings.stateFilePath()) + stateFile, err := stateFileForPath(settings.stateFilePath()) if err != nil { return nil, fmt.Errorf("Couldn't open disk queue metadata file: %w", err) } @@ -154,12 +140,30 @@ func NewQueue(settings Settings) (queue.Queue, error) { } }() - // Decode the state from the file. - stateFile.Seek(0) - return &diskQueue{settings: settings}, nil } +// +// bookkeeping helpers to locate queue data on disk +// + +func (settings Settings) directoryPath() string { + if settings.Path == "" { + return paths.Resolve(paths.Data, "diskqueue") + } + return settings.Path +} + +func (settings Settings) stateFilePath() string { + return filepath.Join(settings.directoryPath(), "state.dat") +} + +func (settings Settings) segmentFilePath(segmentID uint64) string { + return filepath.Join( + settings.directoryPath(), + fmt.Sprintf("%v.seg", segmentID)) +} + // // diskQueue implementation of the queue.Queue interface // @@ -183,39 +187,3 @@ func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { func (dq *diskQueue) Consumer() queue.Consumer { panic("TODO: not implemented") } - -func (settings diskQueueSettings) directoryPath() string { - if settings.Path == "" { - return paths.Resolve(paths.Data, "diskqueue") - } - return settings.Path -} - -func (settings diskQueueSettings) stateFilePath() string { - return filepath.Join(settings.directoryPath(), "state.dat") -} - -func (settings diskQueueSettings) segmentFilePath(segmentID uint64) string { - return filepath.Join(settings.directoryPath(), "%v.seg", segmentID) -} - -func queueStateFromPath(path string) (diskQueueState, error) { - return diskQueueState{}, nil -} - -func stateFileForPath(path string) (*stateFile, error) { - file, err := os.OpenFile(path, os.O_RDWR|os.O_CREATE, 0600) - if err != nil { - return nil, fmt.Errorf("Couldn't open disk queue metadata file: %w", err) - } - fileInfo, err := file.Stat() - if err != nil { - return nil, fmt.Errorf("Couldn't stat disk queue metadata file: %w", err) - } - if - return file, nil -} - -func (stateFile *stateFile) saveState(state diskQueueState) error { - return nil -} diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go new file mode 100644 index 00000000000..3a8cceedd41 --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -0,0 +1,8 @@ +package diskqueue + +type segmentTable struct { +} + +func segmentTableForPath(path string) (*segmentTable, error) { + return nil, nil +} diff --git a/libbeat/publisher/queue/diskqueue/state_file.go b/libbeat/publisher/queue/diskqueue/state_file.go new file mode 100644 index 00000000000..1b7ee667b67 --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/state_file.go @@ -0,0 +1,161 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +import ( + "bufio" + "encoding/binary" + "fmt" + "os" +) + +// This is the queue metadata that is saved to disk. Currently it only +// tracks the read position in the queue; all other data is contained +// in the segment files. +type diskQueuePersistentState struct { + // The schema version for the state file (currently always 0). + version uint32 + + // The oldest position in the queue. This is advanced as we receive ACKs from + // downstream consumers indicating it is safe to remove old events. + firstPosition bufferPosition +} + +// A wrapper around os.File that caches the most recently read / written +// state data. +type stateFile struct { + // An open file handle to the queue's state file. + file *os.File + + // A pointer to the disk queue state that was read when this queue was + // opened, or nil if a new state file was created. + loadedState *diskQueuePersistentState + + // If there was a non-fatal error loading the queue state, it is stored + // here. In this case, the queue overwrites the existing state file with + // a valid starting state. + stateErr error +} + +// Given an open file handle, decode the file as a diskQueuePersistentState +// and return the result if successful, otherwise an error. +func persistentStateFromHandle( + file *os.File, +) (*diskQueuePersistentState, error) { + _, err := file.Seek(0, 0) + if err != nil { + return nil, err + } + state := diskQueuePersistentState{} + + reader := bufio.NewReader(file) + err = binary.Read(reader, binary.LittleEndian, + &state.version) + if err != nil { + return nil, err + } + + err = binary.Read(reader, binary.LittleEndian, + &state.firstPosition.segmentIndex) + if err != nil { + return nil, err + } + + err = binary.Read(reader, binary.LittleEndian, + &state.firstPosition.byteIndex) + if err != nil { + return nil, err + } + + return &state, nil +} + +// Given an open file handle and the first remaining position of a disk queue, +// binary encode the corresponding diskQueuePersistentState and overwrite the +// file with the result. Returns nil if successful, otherwise an error. +func writePersistentStateToHandle( + file *os.File, + firstPosition bufferPosition, +) error { + _, err := file.Seek(0, 0) + if err != nil { + return err + } + + var version uint32 = 0 + err = binary.Write(file, binary.LittleEndian, + &version) + if err != nil { + return err + } + + err = binary.Write(file, binary.LittleEndian, + &firstPosition.segmentIndex) + if err != nil { + return err + } + + err = binary.Write(file, binary.LittleEndian, + &firstPosition.byteIndex) + if err != nil { + return err + } + + return nil +} + +func (stateFile *stateFile) Close() error { + return stateFile.file.Close() +} + +func stateFileForPath(path string) (*stateFile, error) { + var state *diskQueuePersistentState + var stateErr error + // Try to open an existing state file. + file, err := os.OpenFile(path, os.O_RDWR, 0600) + if err != nil { + // If we can't open the file, it's likely a new queue, so try to create it. + file, err = os.OpenFile(path, os.O_RDWR|os.O_CREATE, 0600) + if err != nil { + return nil, fmt.Errorf("Couldn't open disk queue metadata file: %w", err) + } + } else { + // Read the existing state. + state, stateErr = persistentStateFromHandle(file) + // This + if err != nil { + // TODO: this shouldn't be a fatal error. If the state file exists but + // its contents are invalid, then we should log a warning and overwrite + // it with metadata derived from the segment files instead. + return nil, err + } + } + result := &stateFile{ + file: file, + loadedState: state, + stateErr: stateErr, + } + if state == nil { + // Initialize with new zero state. + err = writePersistentStateToHandle(file, bufferPosition{0, 0}) + if err != nil { + return nil, fmt.Errorf("Couldn't write queue state to disk: %w", err) + } + } + return result, nil +} From c312d69530efdd4cefc98093f750b090423fb35e Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 2 Jun 2020 14:45:55 -0400 Subject: [PATCH 12/91] reading data frames from segments --- libbeat/publisher/queue/diskqueue/consumer.go | 25 ++- libbeat/publisher/queue/diskqueue/producer.go | 2 + libbeat/publisher/queue/diskqueue/queue.go | 18 +- libbeat/publisher/queue/diskqueue/segments.go | 173 +++++++++++++++++- .../publisher/queue/diskqueue/serialize.go | 28 +++ .../publisher/queue/diskqueue/state_file.go | 24 +-- 6 files changed, 242 insertions(+), 28 deletions(-) create mode 100644 libbeat/publisher/queue/diskqueue/serialize.go diff --git a/libbeat/publisher/queue/diskqueue/consumer.go b/libbeat/publisher/queue/diskqueue/consumer.go index eba7e89a61e..f27c6eaea57 100644 --- a/libbeat/publisher/queue/diskqueue/consumer.go +++ b/libbeat/publisher/queue/diskqueue/consumer.go @@ -18,14 +18,19 @@ package diskqueue import ( + "fmt" + "github.com/elastic/beats/v7/libbeat/publisher" "github.com/elastic/beats/v7/libbeat/publisher/queue" ) type diskQueueConsumer struct { + queue *diskQueue + closed bool } type diskQueueBatch struct { + events []publisher.Event } // @@ -33,11 +38,27 @@ type diskQueueBatch struct { // func (consumer *diskQueueConsumer) Get(eventCount int) (queue.Batch, error) { + if consumer.closed { + return nil, fmt.Errorf("Tried to read from closed disk queue consumer") + } + // lock mutex ---> + /*position := consumer.queue.readPosition + segment, err := consumer.queue.getSegment(position.segment) + if err != nil { + + } + + // <-- lock mutex + //reader, err := consumer.queue.segments.reader() + if err != nil { + return nil, fmt.Errorf("Couldn't read from queue: %w", err) + }*/ panic("TODO: not implemented") } func (consumer *diskQueueConsumer) Close() error { - panic("TODO: not implemented") + consumer.closed = true + return nil } // @@ -45,7 +66,7 @@ func (consumer *diskQueueConsumer) Close() error { // func (batch *diskQueueBatch) Events() []publisher.Event { - panic("TODO: not implemented") + return batch.events } func (batch *diskQueueBatch) ACK() { diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index b4ed703e095..eeb1b4ab0bf 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -35,6 +35,8 @@ type diskQueueProducer struct { // func (producer *diskQueueProducer) Publish(event publisher.Event) bool { + //data := dataForEvent(event) + panic("TODO: not implemented") } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index dc703cf8b94..c065e118eee 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -57,9 +57,11 @@ type Settings struct { WriteToOutputACKListener queue.ACKListener } +type segmentID uint64 + type bufferPosition struct { // The index of this position's segment within the overall buffer. - segmentIndex uint64 + segment segmentID // The byte offset of this position within its segment. byteIndex uint64 @@ -70,11 +72,11 @@ type bufferPosition struct { type diskQueue struct { settings Settings - // The persistent queue state. After a filesystem sync this should be - // identical to the queue's metadata file. - //state diskQueuePersistentState + // The persistent queue state (wraps diskQueuePersistentState on disk). stateFile *stateFile + segments *segmentManager + // firstPosition bufferPosition @@ -143,6 +145,10 @@ func NewQueue(settings Settings) (queue.Queue, error) { return &diskQueue{settings: settings}, nil } +func (dq *diskQueue) getSegment(id segmentID) (*segmentFile, error) { + panic("TODO: not implemented") +} + // // bookkeeping helpers to locate queue data on disk // @@ -177,10 +183,6 @@ func (dq *diskQueue) BufferConfig() queue.BufferConfig { } func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { - /*return &diskQueueProducer{ - queue: dq, - config: cfg, - }*/ panic("TODO: not implemented") } diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 3a8cceedd41..36e169623cd 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -1,8 +1,175 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package diskqueue -type segmentTable struct { +import ( + "bufio" + "encoding/binary" + "fmt" + "hash/crc32" + "io" + "io/ioutil" + "os" + "sort" + "strconv" + "strings" +) + +type segmentManager struct { + segments []segmentFile +} + +// A wrapper around the file handle and metadata for a single segment file. +type segmentFile struct { + id segmentID + size int64 + file *os.File +} + +// Each data frame has 2 32-bit lengths and 1 32-bit checksum. +const frameMetadataSize = 12 + +// Sort order: we store loaded segments in ascending order by their id. +type bySegmentID []segmentFile + +func (s bySegmentID) Len() int { return len(s) } +func (s bySegmentID) Swap(i, j int) { s[i], s[j] = s[j], s[i] } +func (s bySegmentID) Less(i, j int) bool { return s[i].size < s[j].size } + +func segmentManagerForPath(path string) (*segmentManager, error) { + files, err := ioutil.ReadDir(path) + if err != nil { + return nil, fmt.Errorf("Couldn't read queue directory '%s': %w", path, err) + } + + segments := []segmentFile{} + for _, file := range files { + components := strings.Split(file.Name(), ".") + if len(components) == 2 && strings.ToLower(components[1]) == "seg" { + // Parse the id as base-10 64-bit unsigned int. We ignore file names that + // don't match the "[uint64].seg" pattern. + if id, err := strconv.ParseUint(components[0], 10, 64); err == nil { + segments = append(segments, + segmentFile{id: segmentID(id), size: file.Size()}) + } + } + } + sort.Sort(bySegmentID(segments)) + return &segmentManager{ + segments: segments, + }, nil +} + +type segmentReader struct { + *bufio.Reader + + // The current byte offset of the reader within the file + curPosition int64 + + // The position at which this reader should stop reading. This is often + // the end of the file, but it may be earlier when the queue is reading + // and writing to the same segment. + endPosition int64 + + checksumType checksumType +} + +type checksumType int + +const ( + checksumTypeNone = iota + checksumTypeCRC32 +) + +// A nil data frame with no error means this reader has no more frames. +func (reader *segmentReader) nextDataFrame() ([]byte, error) { + if reader.curPosition >= reader.endPosition { + return nil, nil + } + var frameLength uint32 + err := binary.Read(reader.Reader, binary.LittleEndian, &frameLength) + if err != nil { + return nil, fmt.Errorf( + "Disk queue couldn't read next frame length: %w", err) + } + + // Bounds checking to make sure we can read this frame. + if reader.curPosition+int64(frameLength) > reader.endPosition { + // This frame extends past the end of our data region, which + // should never happen unless there is data corruption. + return nil, fmt.Errorf( + "Data frame length (%d) exceeds remaining data (%d)", + frameLength, reader.endPosition-reader.curPosition) + } + if frameLength <= frameMetadataSize { + // Actual enqueued data must have positive length + return nil, fmt.Errorf( + "Data frame with no data (length %d)", frameLength) + } + + // Read the actual frame data + dataLength := frameLength - frameMetadataSize + data := make([]byte, dataLength) + _, err = io.ReadFull(reader.Reader, data) + if err != nil { + return nil, fmt.Errorf( + "Couldn't read data frame from disk: %w", err) + } + + // Read the footer (length + checksum) + var duplicateLength uint32 + err = binary.Read(reader.Reader, binary.LittleEndian, &duplicateLength) + if err != nil { + return nil, fmt.Errorf( + "Disk queue couldn't read trailing frame length: %w", err) + } + if duplicateLength != frameLength { + return nil, fmt.Errorf( + "Disk queue: inconsistent frame length (%d vs %d)", + frameLength, duplicateLength) + } + + // Validate the checksum + var checksum uint32 + err = binary.Read(reader.Reader, binary.LittleEndian, &checksum) + if err != nil { + return nil, fmt.Errorf( + "Disk queue couldn't read data frame's checksum: %w", err) + } + if computeChecksum(data, reader.checksumType) != checksum { + return nil, fmt.Errorf("Disk queue: bad data frame checksum") + } + + reader.curPosition += int64(frameLength) + return data, nil } -func segmentTableForPath(path string) (*segmentTable, error) { - return nil, nil +func computeChecksum(data []byte, checksumType checksumType) uint32 { + switch checksumType { + case checksumTypeNone: + return 0 + case checksumTypeCRC32: + hash := crc32.NewIEEE() + frameLength := uint32(len(data) + frameMetadataSize) + binary.Write(hash, binary.LittleEndian, &frameLength) + hash.Write(data) + return hash.Sum32() + default: + panic("segmentReader: invalid checksum type") + } } diff --git a/libbeat/publisher/queue/diskqueue/serialize.go b/libbeat/publisher/queue/diskqueue/serialize.go new file mode 100644 index 00000000000..f395c19d313 --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/serialize.go @@ -0,0 +1,28 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +import "github.com/elastic/beats/v7/libbeat/publisher" + +func dataForEvent(event publisher.Event) []byte { + panic("TODO: not implemented") +} + +func eventForData(data []byte) publisher.Event { + panic("TODO: not implemented") +} diff --git a/libbeat/publisher/queue/diskqueue/state_file.go b/libbeat/publisher/queue/diskqueue/state_file.go index 1b7ee667b67..e6b572ffa5f 100644 --- a/libbeat/publisher/queue/diskqueue/state_file.go +++ b/libbeat/publisher/queue/diskqueue/state_file.go @@ -36,14 +36,13 @@ type diskQueuePersistentState struct { firstPosition bufferPosition } -// A wrapper around os.File that caches the most recently read / written -// state data. +// A wrapper around os.File that saves and loads the queue state. type stateFile struct { // An open file handle to the queue's state file. file *os.File // A pointer to the disk queue state that was read when this queue was - // opened, or nil if a new state file was created. + // opened, or nil if there was no preexisting state file. loadedState *diskQueuePersistentState // If there was a non-fatal error loading the queue state, it is stored @@ -64,20 +63,18 @@ func persistentStateFromHandle( state := diskQueuePersistentState{} reader := bufio.NewReader(file) - err = binary.Read(reader, binary.LittleEndian, - &state.version) + err = binary.Read(reader, binary.LittleEndian, &state.version) if err != nil { return nil, err } - err = binary.Read(reader, binary.LittleEndian, - &state.firstPosition.segmentIndex) + err = binary.Read(reader, binary.LittleEndian, &state.firstPosition.segment) if err != nil { return nil, err } - err = binary.Read(reader, binary.LittleEndian, - &state.firstPosition.byteIndex) + err = binary.Read( + reader, binary.LittleEndian, &state.firstPosition.byteIndex) if err != nil { return nil, err } @@ -98,20 +95,17 @@ func writePersistentStateToHandle( } var version uint32 = 0 - err = binary.Write(file, binary.LittleEndian, - &version) + err = binary.Write(file, binary.LittleEndian, &version) if err != nil { return err } - err = binary.Write(file, binary.LittleEndian, - &firstPosition.segmentIndex) + err = binary.Write(file, binary.LittleEndian, &firstPosition.segment) if err != nil { return err } - err = binary.Write(file, binary.LittleEndian, - &firstPosition.byteIndex) + err = binary.Write(file, binary.LittleEndian, &firstPosition.byteIndex) if err != nil { return err } From 1a40b06e628f735efdb2d3ee7c268704c1d20658 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Wed, 3 Jun 2020 16:03:47 -0400 Subject: [PATCH 13/91] fleshing out segment logic --- libbeat/publisher/queue/diskqueue/queue.go | 80 ++++++++- libbeat/publisher/queue/diskqueue/segments.go | 154 +++++++++++++----- 2 files changed, 190 insertions(+), 44 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index c065e118eee..c19990d26ab 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -54,7 +54,10 @@ type Settings struct { // A listener that receives ACKs when events are removed from the queue // and written to their output. - WriteToOutputACKListener queue.ACKListener + // This can only be effective for events that are added to the queue + // after it is opened (there is no way to acknowledge input from a + // previous execution). It is ignored for events before that. + //WriteToOutputACKListener queue.ACKListener } type segmentID uint64 @@ -67,6 +70,16 @@ type bufferPosition struct { byteIndex uint64 } +type diskQueueOutput struct { + data []byte + + // The segment file this data was read from. + segment *segmentFile + + // The index of this data's frame within its segment. + frameIndex int +} + // diskQueue is the internal type representing a disk-based implementation // of queue.Queue. type diskQueue struct { @@ -75,9 +88,30 @@ type diskQueue struct { // The persistent queue state (wraps diskQueuePersistentState on disk). stateFile *stateFile - segments *segmentManager + // A list of all segments that have been completely written but have + // not yet been handed off to a segmentReader. + // Sorted by increasing segment ID. + segments []segmentFile + + // The total bytes occupied by all segment files. This is the value + // we check to see if there is enough space to add an incoming event + // to the queue. + bytesOnDisk uint64 + + // The memory queue of data blobs waiting to be written to disk. + // To add something to the queue internally, send it to this channel. + inChan chan byte[] - // + outChan chan diskQueueOutput + + // The currently active segment reader, or nil if there is none. + reader *segmentReader + + // The currently active segment writer. When the corresponding segment + // is full it is appended to segments. + writer *segmentWriter + + // The ol firstPosition bufferPosition // The position of the next event to read from the queue. If this equals @@ -142,11 +176,44 @@ func NewQueue(settings Settings) (queue.Queue, error) { } }() - return &diskQueue{settings: settings}, nil + segments, err := segmentFilesForPath(settings.directoryPath()) + if err != nil { + return nil, err + } + + return &diskQueue{ + settings: settings, + segments: segments, + }, nil } -func (dq *diskQueue) getSegment(id segmentID) (*segmentFile, error) { - panic("TODO: not implemented") +func (dq *diskQueue) nextSegmentReader() (*segmentReader, error) { + if len(dq.segments) > 0 { + return nil, nil + } + nextSegment := dq.segments[0] + +} + +// readNextFrame reads the next pending data frame in the queue +// and returns its contents. +func (dq *diskQueue) readNextFrame() ([]byte, error) { + // READER LOCK ---> + if dq.reader != nil { + frameData, err := dq.reader.nextDataFrame() + if err != nil { + return nil, err + } + if frameData != nil { + return frameData, nil + } + // If we made it here then the active reader was empty and + // we need to fetch a new one. + } + reader, err := dq.nextSegmentReader() + dq.reader = reader + return reader.nextDataFrame() + // <--- READER LOCK } // @@ -157,6 +224,7 @@ func (settings Settings) directoryPath() string { if settings.Path == "" { return paths.Resolve(paths.Data, "diskqueue") } + return settings.Path } diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 36e169623cd..589ae5a0a12 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -18,7 +18,6 @@ package diskqueue import ( - "bufio" "encoding/binary" "fmt" "hash/crc32" @@ -28,22 +27,79 @@ import ( "sort" "strconv" "strings" -) + "sync" -type segmentManager struct { - segments []segmentFile -} + "github.com/elastic/beats/v7/libbeat/logp" +) -// A wrapper around the file handle and metadata for a single segment file. +// The metadata for a single segment file. type segmentFile struct { - id segmentID + logger *logp.Logger + + lock sync.Mutex + + id segmentID + + // The length in bytes of the segment file on disk. This is updated when + // the segment is written to, and should always correspond to the end of + // a complete data frame. size int64 - file *os.File + + // The number of data frames in this segment file. + // This is used for ack handling: when a consumer reads an event, the + // the resulting diskQueueBatch encodes the event's index. It is safe + // to delete a segment file when all indices from 0...(frameCount-1) + // have been acknowledged. + // This value may be zero for segment files that already existed when + // the queue was opened; in that case it is not populated until the + // segment file has been completely read. In particular, we will not + // delete the file for a segment if frameCount == 0. + frameCount int + + // The lowest frame index that has not yet been acknowledged. + ackedUpTo int + + // A map of all acked indices that are above ackedUpTo (and thus + // can't yet be acknowledged as a continuous block). + acked map[int]bool +} + +// segmentReader is a wrapper around io.Reader that provides helpers and +// metadata for decoding segment files. +type segmentReader struct { + // The underlying data reader + raw io.Reader + + // The current byte offset of the reader within the file + curPosition int64 + + // The position at which this reader should stop reading. This is often + // the end of the file, but it may be earlier when the queue is reading + // and writing to the same segment. + endPosition int64 + + // The checksumType field from this segment file's header. + checksumType checksumType +} + +type segmentWriter struct { + *os.File + curPosition int64 } +type checksumType int + +const ( + checksumTypeNone = iota + checksumTypeCRC32 +) + // Each data frame has 2 32-bit lengths and 1 32-bit checksum. const frameMetadataSize = 12 +// Each segment header has a 32-bit version and a 32-bit checksum type. +const segmentHeaderSize = 8 + // Sort order: we store loaded segments in ascending order by their id. type bySegmentID []segmentFile @@ -51,7 +107,9 @@ func (s bySegmentID) Len() int { return len(s) } func (s bySegmentID) Swap(i, j int) { s[i], s[j] = s[j], s[i] } func (s bySegmentID) Less(i, j int) bool { return s[i].size < s[j].size } -func segmentManagerForPath(path string) (*segmentManager, error) { +func segmentFilesForPath( + path string, logger *logp.Logger, +) ([]segmentFile, error) { files, err := ioutil.ReadDir(path) if err != nil { return nil, fmt.Errorf("Couldn't read queue directory '%s': %w", path, err) @@ -65,44 +123,27 @@ func segmentManagerForPath(path string) (*segmentManager, error) { // don't match the "[uint64].seg" pattern. if id, err := strconv.ParseUint(components[0], 10, 64); err == nil { segments = append(segments, - segmentFile{id: segmentID(id), size: file.Size()}) + segmentFile{ + logger: logger, + id: segmentID(id), + size: file.Size(), + }) } } } sort.Sort(bySegmentID(segments)) - return &segmentManager{ - segments: segments, - }, nil -} - -type segmentReader struct { - *bufio.Reader - - // The current byte offset of the reader within the file - curPosition int64 - - // The position at which this reader should stop reading. This is often - // the end of the file, but it may be earlier when the queue is reading - // and writing to the same segment. - endPosition int64 - - checksumType checksumType + return segments, nil } -type checksumType int - -const ( - checksumTypeNone = iota - checksumTypeCRC32 -) - // A nil data frame with no error means this reader has no more frames. +// If nextDataFrame returns an error, it should be logged and the +// corresponding segment should be dropped. func (reader *segmentReader) nextDataFrame() ([]byte, error) { if reader.curPosition >= reader.endPosition { return nil, nil } var frameLength uint32 - err := binary.Read(reader.Reader, binary.LittleEndian, &frameLength) + err := binary.Read(reader.raw, binary.LittleEndian, &frameLength) if err != nil { return nil, fmt.Errorf( "Disk queue couldn't read next frame length: %w", err) @@ -125,7 +166,7 @@ func (reader *segmentReader) nextDataFrame() ([]byte, error) { // Read the actual frame data dataLength := frameLength - frameMetadataSize data := make([]byte, dataLength) - _, err = io.ReadFull(reader.Reader, data) + _, err = io.ReadFull(reader.raw, data) if err != nil { return nil, fmt.Errorf( "Couldn't read data frame from disk: %w", err) @@ -133,7 +174,7 @@ func (reader *segmentReader) nextDataFrame() ([]byte, error) { // Read the footer (length + checksum) var duplicateLength uint32 - err = binary.Read(reader.Reader, binary.LittleEndian, &duplicateLength) + err = binary.Read(reader.raw, binary.LittleEndian, &duplicateLength) if err != nil { return nil, fmt.Errorf( "Disk queue couldn't read trailing frame length: %w", err) @@ -146,7 +187,7 @@ func (reader *segmentReader) nextDataFrame() ([]byte, error) { // Validate the checksum var checksum uint32 - err = binary.Read(reader.Reader, binary.LittleEndian, &checksum) + err = binary.Read(reader.raw, binary.LittleEndian, &checksum) if err != nil { return nil, fmt.Errorf( "Disk queue couldn't read data frame's checksum: %w", err) @@ -159,6 +200,19 @@ func (reader *segmentReader) nextDataFrame() ([]byte, error) { return data, nil } +// returns the number of indices by which ackedUpTo was advanced. +func (s *segmentFile) ack(index int) int { + s.lock.Lock() + defer s.lock.Unlock() + s.acked[index] = true + ackedCount := 0 + for ; s.acked[s.ackedUpTo]; s.ackedUpTo++ { + delete(s.acked, s.ackedUpTo) + ackedCount++ + } + return ackedCount +} + func computeChecksum(data []byte, checksumType checksumType) uint32 { switch checksumType { case checksumTypeNone: @@ -173,3 +227,27 @@ func computeChecksum(data []byte, checksumType checksumType) uint32 { panic("segmentReader: invalid checksum type") } } + +func (dq *diskQueue) segmentReaderForPosition( + pos bufferPosition, +) (*segmentReader, error) { + panic("TODO: not implemented") +} + +/* +func (sm *segmentManager) segmentReaderForPosition(pos bufferPosition) (*segmentReader, error) { + segment = getSegment(pos.segment) + + dataSize := segment.size - segmentHeaderSize + file, err := os.Open(pathForSegmentId(pos.segment)) + // ...read segment header... + checksumType := checksumTypeNone + file.Seek(segmentHeaderSize+pos.byteIndex, 0) + reader := bufio.NewReader(file) + return &segmentReader{ + raw: reader, + curPosition: pos.byteIndex, + endPosition: dataSize, + checksumType: checksumType, + }, nil +}*/ From ce65718ab52360190318eed593641ba2d9ae5ac8 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 9 Jun 2020 12:15:37 -0400 Subject: [PATCH 14/91] lots of partial work on reader and writer --- libbeat/publisher/queue/diskqueue/queue.go | 174 +++++++++++++++--- .../publisher/queue/diskqueue/reader_loop.go | 113 ++++++++++++ libbeat/publisher/queue/diskqueue/segments.go | 60 +++--- 3 files changed, 297 insertions(+), 50 deletions(-) create mode 100644 libbeat/publisher/queue/diskqueue/reader_loop.go diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index c19990d26ab..ccc171629ce 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -18,11 +18,15 @@ package diskqueue import ( + "bufio" "fmt" + "io" "os" "path/filepath" + "sync" "github.com/elastic/beats/v7/libbeat/common" + "github.com/elastic/beats/v7/libbeat/common/atomic" "github.com/elastic/beats/v7/libbeat/feature" "github.com/elastic/beats/v7/libbeat/logp" "github.com/elastic/beats/v7/libbeat/paths" @@ -48,6 +52,10 @@ type Settings struct { // disk is full (this is not recommended on a primary system disk). MaxBufferSize uint64 + // MaxSegmentSize is the maximum number of bytes that should be written + // to a single segment file before creating a new one. + MaxSegmentSize uint64 + // A listener that receives ACKs when events are written to the queue's // disk buffer. WriteToDiskACKListener queue.ACKListener @@ -76,8 +84,9 @@ type diskQueueOutput struct { // The segment file this data was read from. segment *segmentFile - // The index of this data's frame within its segment. - frameIndex int + // The index of this data's frame (the sequential read order + // of all frames during this execution). + frame frameID } // diskQueue is the internal type representing a disk-based implementation @@ -88,10 +97,29 @@ type diskQueue struct { // The persistent queue state (wraps diskQueuePersistentState on disk). stateFile *stateFile - // A list of all segments that have been completely written but have - // not yet been handed off to a segmentReader. - // Sorted by increasing segment ID. - segments []segmentFile + // segmentLock must be held to read or write segmentFile, readingSegments + // and completedSegments. + segmentLock sync.Mutex + + // The segment that is currently being written. + writingSegment *segmentFile + + // A list of the segments that have been completely written but have + // not yet been processed by the reader loop, sorted by increasing + // segment ID. Segments are always read in order. When a segment has + // been read completely, it is removed from the front of this list and + // appended to completedSegments. + readingSegments []*segmentFile + + // A list of the segments that have been read but have not yet been + // completely acknowledged, sorted by increasing segment ID. When the + // first entry of this list is completely acknowledged, it is removed + // from this list and the underlying file is deleted. + completedSegments []*segmentFile + + // The next sequential unused segment ID. This is what will be assigned + // to the next segmentFile we create. + nextSegmentID segmentID // The total bytes occupied by all segment files. This is the value // we check to see if there is enough space to add an incoming event @@ -100,16 +128,16 @@ type diskQueue struct { // The memory queue of data blobs waiting to be written to disk. // To add something to the queue internally, send it to this channel. - inChan chan byte[] + inChan chan []byte outChan chan diskQueueOutput // The currently active segment reader, or nil if there is none. - reader *segmentReader + //reader *segmentReader // The currently active segment writer. When the corresponding segment // is full it is appended to segments. - writer *segmentWriter + //writer *segmentWriter // The ol firstPosition bufferPosition @@ -124,6 +152,43 @@ type diskQueue struct { // This is part of diskQueue and not diskQueueState since it represents // in-memory state that should not persist through a restart. readPosition bufferPosition + + // A condition that is signalled when a segment file is deleted. + // Used by writerLoop when the queue is full, to detect when to try again. + // When the queue is closed, this condition will receive a broadcast after + // diskQueue.closed is set to true. + segmentDeletedCond sync.Cond + + // A condition that is signalled when a frame has been completely + // written to disk. + // Used by readerLoop when the queue is empty, to detect when to try again. + // When the queue is closed, this condition will receive a broadcast after + // diskQueue.closed is set to true. + frameWrittenCond sync.Cond + + // The oldest frame id that is still stored on disk. + // This will usually be less than ackedUpTo, since oldestFrame can't + // advance until the entire segment file has been acknowledged and + // deleted. + oldestFrame frameID + + // This lock must be held to read and write acked and ackedUpTo. + ackLock sync.Mutex + + // The lowest frame id that has not yet been acknowledged. + ackedUpTo frameID + + // A map of all acked indices that are above ackedUpTo (and thus + // can't yet be acknowledged as a continuous block). + acked map[frameID]bool + + // Whether the queue has been closed. Code that can't check the done + // channel (e.g. code that must wait on a condition variable) should + // always check this value when waking up. + closed atomic.Bool + + // The channel to signal our goroutines to shut down. + done chan struct{} } func init() { @@ -176,28 +241,90 @@ func NewQueue(settings Settings) (queue.Queue, error) { } }() - segments, err := segmentFilesForPath(settings.directoryPath()) + segments, err := segmentFilesForPath( + settings.directoryPath(), settings.Logger) if err != nil { return nil, err } return &diskQueue{ - settings: settings, - segments: segments, + settings: settings, + readingSegments: segments, + closed: atomic.MakeBool(false), + done: make(chan struct{}), }, nil } -func (dq *diskQueue) nextSegmentReader() (*segmentReader, error) { - if len(dq.segments) > 0 { - return nil, nil +// This is only called by readerLoop. +func (dq *diskQueue) nextSegmentReader() (*segmentReader, []error) { + // TODO: make sure we hold the right locks to mess with segments here. + errors := []error{} + for len(dq.readingSegments) > 0 { + segment := dq.readingSegments[0] + segmentPath := dq.settings.segmentFilePath(segment.id) + reader, err := tryLoad(segment, segmentPath) + if err != nil { + // TODO: Handle this: depending on the type of error, either delete + // the segment or log an error and leave it alone, then skip to the + // next one. + errors = append(errors, err) + dq.readingSegments = dq.readingSegments[1:] + continue + } + // Remove the segment from the active list and move it to + // completedSegments until all its data has been acknowledged. + dq.readingSegments = dq.readingSegments[1:] + dq.completedSegments = append(dq.completedSegments, segment) + return reader, errors } - nextSegment := dq.segments[0] + // TODO: if readingSegments is empty we may still be able to + // read partial data from writingSegment which is still being + // written. + return nil, errors +} +func tryLoad(segment *segmentFile, path string) (*segmentReader, error) { + // this is a strangely fine-grained lock maybe? + segment.lock.Lock() + defer segment.lock.Unlock() + + // dataSize is guaranteed to be positive because we don't add + // anything to the segments list unless it is. + dataSize := segment.size - segmentHeaderSize + file, err := os.Open(path) + if err != nil { + return nil, fmt.Errorf( + "Couldn't open segment %d: %w", segment.id, err) + } + reader := bufio.NewReader(file) + header, err := readSegmentHeader(reader) + if err != nil { + return nil, fmt.Errorf("Couldn't read segment header: %w", err) + } + if header.version != 0 { + return nil, fmt.Errorf("Segment %d: unrecognized schema version %d", + segment.id, header.version) + } + return &segmentReader{ + raw: reader, + curPosition: 0, + endPosition: dataSize, + checksumType: header.checksumType, + }, nil +} + +type segmentHeader struct { + version uint32 + checksumType checksumType +} + +func readSegmentHeader(in io.Reader) (*segmentHeader, error) { + return nil, nil } // readNextFrame reads the next pending data frame in the queue // and returns its contents. -func (dq *diskQueue) readNextFrame() ([]byte, error) { +/*func (dq *diskQueue) readNextFrame() ([]byte, error) { // READER LOCK ---> if dq.reader != nil { frameData, err := dq.reader.nextDataFrame() @@ -210,11 +337,11 @@ func (dq *diskQueue) readNextFrame() ([]byte, error) { // If we made it here then the active reader was empty and // we need to fetch a new one. } - reader, err := dq.nextSegmentReader() + reader, _ := dq.nextSegmentReader() dq.reader = reader return reader.nextDataFrame() // <--- READER LOCK -} +}*/ // // bookkeeping helpers to locate queue data on disk @@ -232,7 +359,7 @@ func (settings Settings) stateFilePath() string { return filepath.Join(settings.directoryPath(), "state.dat") } -func (settings Settings) segmentFilePath(segmentID uint64) string { +func (settings Settings) segmentFilePath(segmentID segmentID) string { return filepath.Join( settings.directoryPath(), fmt.Sprintf("%v.seg", segmentID)) @@ -243,7 +370,12 @@ func (settings Settings) segmentFilePath(segmentID uint64) string { // func (dq *diskQueue) Close() error { - panic("TODO: not implemented") + if dq.closed.Swap(true) { + return fmt.Errorf("Can't close disk queue: queue already closed") + } + // TODO: wait for worker threads? + close(dq.done) + return nil } func (dq *diskQueue) BufferConfig() queue.BufferConfig { diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go new file mode 100644 index 00000000000..b7f4762e6d8 --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -0,0 +1,113 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +func (dq *diskQueue) readerLoop() { + curFrameID := frameID(0) + for { + dq.frameWrittenCond.Wait() + reader, errs := dq.nextSegmentReader() + for _, err := range errs { + // Errors encountered while reading should be logged. + dq.settings.Logger.Error(err) + } + if reader == nil { + // We couldn't find a readable segment, wait for a new + // data frame to be written. + dq.frameWrittenCond.Wait() + if dq.closed.Load() { + // The queue has been closed, shut down. + // TODO: cleanup (write the final read position) + return + } + continue + } + // If we made it here, we have a nonempty reader and we want + // to send all its frames to dq.outChan. + framesRead := int64(0) + for { + bytes, err := reader.nextDataFrame() + if err != nil { + // An error at this stage generally means there has been + // data corruption. For now, in this case we log the + // error, then discard any remaining frames. When all + // successfully read frames have been acknowledged, we + // delete the underlying file. + break + } + if bytes == nil { + // If bytes is nil with no error, we've reached the end + // of this segmentReader. Update the segment's frame count. + break + } + framesRead++ + output := diskQueueOutput{ + data: bytes, + segment: reader.segment, + frame: curFrameID, + } + select { + case dq.outChan <- output: + curFrameID++ + case <-dq.done: + } + } + reader.segment.framesRead += framesRead + } +} + +func (dq *diskQueue) writerLoop() { + for { + var frameBytes []byte + select { + case frameBytes = <-dq.inChan: + + case <-dq.done: + break + } + } +} + +// This is only called from the writer loop. +func (dq *diskQueue) writeFrameData(bytes []byte) error { + frameSize := len(bytes) + frameMetadataSize + + // We can check segment size without holding a lock because the writer + // loop (i.e. the current thread) is the only place that modifies it + // after segmentFile creation. + if dq.writingSegment == nil || + dq.writingSegment.size + frameSize > dq.settings.MaxSegmentSize { + // There is no writing segment, or the incoming data frame doesn't + // fit in the current one, so we need to create a new one. + } + + .capacity() < frameSize { + + } != nil && dq.writingSegment. + + // while (free bytes) < frameSize { + // block + // } + + if dq.writingSegment == nil { + // There is no current output segment, create a new one. + + } + + return nil +} \ No newline at end of file diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 589ae5a0a12..3fd8a6dd99a 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -32,6 +32,11 @@ import ( "github.com/elastic/beats/v7/libbeat/logp" ) +// Every data frame read from the queue is assigned a unique sequential +// integer, which is used to keep track of which frames have been +// acknowledged. +type frameID uint64 + // The metadata for a single segment file. type segmentFile struct { logger *logp.Logger @@ -45,32 +50,23 @@ type segmentFile struct { // a complete data frame. size int64 - // The number of data frames in this segment file. - // This is used for ack handling: when a consumer reads an event, the - // the resulting diskQueueBatch encodes the event's index. It is safe - // to delete a segment file when all indices from 0...(frameCount-1) - // have been acknowledged. - // This value may be zero for segment files that already existed when - // the queue was opened; in that case it is not populated until the - // segment file has been completely read. In particular, we will not - // delete the file for a segment if frameCount == 0. - frameCount int - - // The lowest frame index that has not yet been acknowledged. - ackedUpTo int - - // A map of all acked indices that are above ackedUpTo (and thus - // can't yet be acknowledged as a continuous block). - acked map[int]bool + // The number of frames read from this segment, or zero if it has not + // yet been completely read. + // It is safe to delete a segment when framesRead > 0 and all those + // frames have been acknowledged. + framesRead int64 } // segmentReader is a wrapper around io.Reader that provides helpers and // metadata for decoding segment files. type segmentReader struct { - // The underlying data reader + // The segment this reader was generated from. + segment *segmentFile + + // The underlying data reader. raw io.Reader - // The current byte offset of the reader within the file + // The current byte offset of the reader within the file. curPosition int64 // The position at which this reader should stop reading. This is often @@ -101,7 +97,7 @@ const frameMetadataSize = 12 const segmentHeaderSize = 8 // Sort order: we store loaded segments in ascending order by their id. -type bySegmentID []segmentFile +type bySegmentID []*segmentFile func (s bySegmentID) Len() int { return len(s) } func (s bySegmentID) Swap(i, j int) { s[i], s[j] = s[j], s[i] } @@ -109,21 +105,27 @@ func (s bySegmentID) Less(i, j int) bool { return s[i].size < s[j].size } func segmentFilesForPath( path string, logger *logp.Logger, -) ([]segmentFile, error) { +) ([]*segmentFile, error) { files, err := ioutil.ReadDir(path) if err != nil { return nil, fmt.Errorf("Couldn't read queue directory '%s': %w", path, err) } - segments := []segmentFile{} + segments := []*segmentFile{} for _, file := range files { + if file.Size() <= segmentHeaderSize { + // Ignore segments that don't have at least some data beyond the + // header (this will always be true of segments we write unless there + // is an error). + continue + } components := strings.Split(file.Name(), ".") if len(components) == 2 && strings.ToLower(components[1]) == "seg" { // Parse the id as base-10 64-bit unsigned int. We ignore file names that // don't match the "[uint64].seg" pattern. if id, err := strconv.ParseUint(components[0], 10, 64); err == nil { segments = append(segments, - segmentFile{ + &segmentFile{ logger: logger, id: segmentID(id), size: file.Size(), @@ -201,13 +203,13 @@ func (reader *segmentReader) nextDataFrame() ([]byte, error) { } // returns the number of indices by which ackedUpTo was advanced. -func (s *segmentFile) ack(index int) int { - s.lock.Lock() - defer s.lock.Unlock() - s.acked[index] = true +func (dq *diskQueue) ack(frame frameID) int { + dq.ackLock.Lock() + defer dq.ackLock.Unlock() + dq.acked[frame] = true ackedCount := 0 - for ; s.acked[s.ackedUpTo]; s.ackedUpTo++ { - delete(s.acked, s.ackedUpTo) + for ; dq.acked[dq.ackedUpTo]; dq.ackedUpTo++ { + delete(dq.acked, dq.ackedUpTo) ackedCount++ } return ackedCount From 22ae14874cf5383d9689b96bc830fb226851b308 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 16 Jun 2020 16:10:52 -0400 Subject: [PATCH 15/91] reworking segments --- libbeat/publisher/queue/diskqueue/queue.go | 98 +++++++++++-------- .../publisher/queue/diskqueue/reader_loop.go | 58 ++++++++--- libbeat/publisher/queue/diskqueue/segments.go | 30 +++--- 3 files changed, 113 insertions(+), 73 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index ccc171629ce..787994cdf4d 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -82,7 +82,7 @@ type diskQueueOutput struct { data []byte // The segment file this data was read from. - segment *segmentFile + segment *queueSegment // The index of this data's frame (the sequential read order // of all frames during this execution). @@ -97,29 +97,8 @@ type diskQueue struct { // The persistent queue state (wraps diskQueuePersistentState on disk). stateFile *stateFile - // segmentLock must be held to read or write segmentFile, readingSegments - // and completedSegments. - segmentLock sync.Mutex - - // The segment that is currently being written. - writingSegment *segmentFile - - // A list of the segments that have been completely written but have - // not yet been processed by the reader loop, sorted by increasing - // segment ID. Segments are always read in order. When a segment has - // been read completely, it is removed from the front of this list and - // appended to completedSegments. - readingSegments []*segmentFile - - // A list of the segments that have been read but have not yet been - // completely acknowledged, sorted by increasing segment ID. When the - // first entry of this list is completely acknowledged, it is removed - // from this list and the underlying file is deleted. - completedSegments []*segmentFile - - // The next sequential unused segment ID. This is what will be assigned - // to the next segmentFile we create. - nextSegmentID segmentID + // Metadata related to the segment files. + segments diskQueueSegments // The total bytes occupied by all segment files. This is the value // we check to see if there is enough space to add an incoming event @@ -191,6 +170,35 @@ type diskQueue struct { done chan struct{} } +// diskQueueSegments encapsulates segment-related queue metadata. +type diskQueueSegments struct { + // The lock should be held to read or write any of the fields below. + sync.Mutex + + // The segment that is currently being written. + writing *queueSegment + + writer *segmentWriter + reader *segmentReader + + // A list of the segments that have been completely written but have + // not yet been processed by the reader loop, sorted by increasing + // segment ID. Segments are always read in order. When a segment has + // been read completely, it is removed from the front of this list and + // appended to completedSegments. + reading []*queueSegment + + // A list of the segments that have been read but have not yet been + // completely acknowledged, sorted by increasing segment ID. When the + // first entry of this list is completely acknowledged, it is removed + // from this list and the underlying file is deleted. + completed []*queueSegment + + // The next sequential unused segment ID. This is what will be assigned + // to the next queueSegment we create. + nextID segmentID +} + func init() { queue.RegisterQueueType( "disk", @@ -241,52 +249,56 @@ func NewQueue(settings Settings) (queue.Queue, error) { } }() - segments, err := segmentFilesForPath( + segments, err := queueSegmentsForPath( settings.directoryPath(), settings.Logger) if err != nil { return nil, err } return &diskQueue{ - settings: settings, - readingSegments: segments, - closed: atomic.MakeBool(false), - done: make(chan struct{}), + settings: settings, + segments: diskQueueSegments{ + reading: segments, + }, + closed: atomic.MakeBool(false), + done: make(chan struct{}), }, nil } // This is only called by readerLoop. func (dq *diskQueue) nextSegmentReader() (*segmentReader, []error) { - // TODO: make sure we hold the right locks to mess with segments here. + dq.segments.Lock() + defer dq.segments.Unlock() + errors := []error{} - for len(dq.readingSegments) > 0 { - segment := dq.readingSegments[0] - segmentPath := dq.settings.segmentFilePath(segment.id) + for len(dq.segments.reading) > 0 { + segment := dq.segments.reading[0] + segmentPath := dq.settings.segmentPath(segment.id) reader, err := tryLoad(segment, segmentPath) if err != nil { // TODO: Handle this: depending on the type of error, either delete // the segment or log an error and leave it alone, then skip to the // next one. errors = append(errors, err) - dq.readingSegments = dq.readingSegments[1:] + dq.segments.reading = dq.segments.reading[1:] continue } // Remove the segment from the active list and move it to // completedSegments until all its data has been acknowledged. - dq.readingSegments = dq.readingSegments[1:] - dq.completedSegments = append(dq.completedSegments, segment) + dq.segments.reading = dq.segments.reading[1:] + dq.segments.completed = append(dq.segments.completed, segment) return reader, errors } - // TODO: if readingSegments is empty we may still be able to - // read partial data from writingSegment which is still being + // TODO: if segments.reading is empty we may still be able to + // read partial data from segments.writing which is still being // written. return nil, errors } -func tryLoad(segment *segmentFile, path string) (*segmentReader, error) { +func tryLoad(segment *queueSegment, path string) (*segmentReader, error) { // this is a strangely fine-grained lock maybe? - segment.lock.Lock() - defer segment.lock.Unlock() + segment.Lock() + defer segment.Unlock() // dataSize is guaranteed to be positive because we don't add // anything to the segments list unless it is. @@ -308,7 +320,7 @@ func tryLoad(segment *segmentFile, path string) (*segmentReader, error) { return &segmentReader{ raw: reader, curPosition: 0, - endPosition: dataSize, + endPosition: int64(dataSize), checksumType: header.checksumType, }, nil } @@ -359,7 +371,7 @@ func (settings Settings) stateFilePath() string { return filepath.Join(settings.directoryPath(), "state.dat") } -func (settings Settings) segmentFilePath(segmentID segmentID) string { +func (settings Settings) segmentPath(segmentID segmentID) string { return filepath.Join( settings.directoryPath(), fmt.Sprintf("%v.seg", segmentID)) diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index b7f4762e6d8..6271523aa8a 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -17,6 +17,8 @@ package diskqueue +import "os" + func (dq *diskQueue) readerLoop() { curFrameID := frameID(0) for { @@ -83,31 +85,59 @@ func (dq *diskQueue) writerLoop() { } } +func (dq *diskQueue) newSegmentWriter() (*segmentWriter, error) { + var err error + dq.segments.Lock() + defer dq.segments.Unlock() + + id := dq.segments.nextID + defer func() { + // If we were successful, update nextID + if err == nil { + dq.segments.nextID++ + } + }() + + segment := &queueSegment{id: id} + + path := dq.settings.segmentPath(id) + file, err := os.OpenFile(path, os.O_CREATE|os.O_TRUNC, 0600) + if err != nil { + return nil, err + } + + return &segmentWriter{ + segment: segment, + file: file, + }, nil +} + // This is only called from the writer loop. func (dq *diskQueue) writeFrameData(bytes []byte) error { - frameSize := len(bytes) + frameMetadataSize - - // We can check segment size without holding a lock because the writer - // loop (i.e. the current thread) is the only place that modifies it - // after segmentFile creation. - if dq.writingSegment == nil || - dq.writingSegment.size + frameSize > dq.settings.MaxSegmentSize { - // There is no writing segment, or the incoming data frame doesn't - // fit in the current one, so we need to create a new one. + frameSize := uint64(len(bytes) + frameMetadataSize) + + dq.segments.Lock() + defer dq.segments.Unlock() + if dq.segments.writer != nil && + dq.segments.writer.position+frameSize > dq.settings.MaxSegmentSize { + // There is a writing segment, but the incoming data frame doesn't + // fit, so we need to finalize it and create a new one. + //dq.segments.writer = + //dq.segments.writing } - - .capacity() < frameSize { - } != nil && dq.writingSegment. + /*.capacity() < frameSize { + + } != nil && dq.writingSegment.*/ // while (free bytes) < frameSize { // block // } - if dq.writingSegment == nil { + if dq.segments.writing == nil { // There is no current output segment, create a new one. } return nil -} \ No newline at end of file +} diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 3fd8a6dd99a..a140ebe8674 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -38,17 +38,15 @@ import ( type frameID uint64 // The metadata for a single segment file. -type segmentFile struct { - logger *logp.Logger - - lock sync.Mutex +type queueSegment struct { + sync.Mutex id segmentID // The length in bytes of the segment file on disk. This is updated when // the segment is written to, and should always correspond to the end of // a complete data frame. - size int64 + size uint64 // The number of frames read from this segment, or zero if it has not // yet been completely read. @@ -61,7 +59,7 @@ type segmentFile struct { // metadata for decoding segment files. type segmentReader struct { // The segment this reader was generated from. - segment *segmentFile + segment *queueSegment // The underlying data reader. raw io.Reader @@ -79,8 +77,9 @@ type segmentReader struct { } type segmentWriter struct { - *os.File - curPosition int64 + segment *queueSegment + file *os.File + position int64 } type checksumType int @@ -97,21 +96,21 @@ const frameMetadataSize = 12 const segmentHeaderSize = 8 // Sort order: we store loaded segments in ascending order by their id. -type bySegmentID []*segmentFile +type bySegmentID []*queueSegment func (s bySegmentID) Len() int { return len(s) } func (s bySegmentID) Swap(i, j int) { s[i], s[j] = s[j], s[i] } func (s bySegmentID) Less(i, j int) bool { return s[i].size < s[j].size } -func segmentFilesForPath( +func queueSegmentsForPath( path string, logger *logp.Logger, -) ([]*segmentFile, error) { +) ([]*queueSegment, error) { files, err := ioutil.ReadDir(path) if err != nil { return nil, fmt.Errorf("Couldn't read queue directory '%s': %w", path, err) } - segments := []*segmentFile{} + segments := []*queueSegment{} for _, file := range files { if file.Size() <= segmentHeaderSize { // Ignore segments that don't have at least some data beyond the @@ -125,10 +124,9 @@ func segmentFilesForPath( // don't match the "[uint64].seg" pattern. if id, err := strconv.ParseUint(components[0], 10, 64); err == nil { segments = append(segments, - &segmentFile{ - logger: logger, - id: segmentID(id), - size: file.Size(), + &queueSegment{ + id: segmentID(id), + size: uint64(file.Size()), }) } } From 3f5f8fe9e366ff1cf05e2c43458226fec06350ea Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 18 Jun 2020 11:06:36 -0400 Subject: [PATCH 16/91] reworking reader code --- libbeat/publisher/queue/diskqueue/queue.go | 19 +++--- .../publisher/queue/diskqueue/reader_loop.go | 59 ++++++++++++++++++- libbeat/publisher/queue/diskqueue/segments.go | 25 ++------ 3 files changed, 71 insertions(+), 32 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 787994cdf4d..20d97a5c099 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -70,12 +70,12 @@ type Settings struct { type segmentID uint64 -type bufferPosition struct { - // The index of this position's segment within the overall buffer. +type queuePosition struct { + // The index of this position's segment within the queue. segment segmentID // The byte offset of this position within its segment. - byteIndex uint64 + byteIndex segmentPos } type diskQueueOutput struct { @@ -313,14 +313,10 @@ func tryLoad(segment *queueSegment, path string) (*segmentReader, error) { if err != nil { return nil, fmt.Errorf("Couldn't read segment header: %w", err) } - if header.version != 0 { - return nil, fmt.Errorf("Segment %d: unrecognized schema version %d", - segment.id, header.version) - } return &segmentReader{ raw: reader, curPosition: 0, - endPosition: int64(dataSize), + endPosition: segmentPos(dataSize), checksumType: header.checksumType, }, nil } @@ -331,7 +327,12 @@ type segmentHeader struct { } func readSegmentHeader(in io.Reader) (*segmentHeader, error) { - return nil, nil + header := segmentHeader{} + if header.version != 0 { + return nil, fmt.Errorf("Unrecognized schema version %d", header.version) + } + panic("TODO: not implemented") + //return nil, nil } // readNextFrame reads the next pending data frame in the queue diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 6271523aa8a..9f51367cc6e 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -17,7 +17,12 @@ package diskqueue -import "os" +import ( + "bufio" + "fmt" + "io" + "os" +) func (dq *diskQueue) readerLoop() { curFrameID := frameID(0) @@ -118,13 +123,13 @@ func (dq *diskQueue) writeFrameData(bytes []byte) error { dq.segments.Lock() defer dq.segments.Unlock() - if dq.segments.writer != nil && + /*if dq.segments.writer != nil && dq.segments.writer.position+frameSize > dq.settings.MaxSegmentSize { // There is a writing segment, but the incoming data frame doesn't // fit, so we need to finalize it and create a new one. //dq.segments.writer = //dq.segments.writing - } + }*/ /*.capacity() < frameSize { @@ -141,3 +146,51 @@ func (dq *diskQueue) writeFrameData(bytes []byte) error { return nil } + +func readSegment( + reader io.Reader, checksumType checksumType, + start segmentPos, stop segmentPos, + output chan diskQueueOutput, cancel chan struct{}, +) (int, error) { + return 0, nil +} + +func (dq *diskQueue) nextSegmentForReading() *queueSegment { + dq.segments.Lock() + defer dq.segments.Unlock() + if len(dq.segments.reading) > 0 { + return dq.segments.reading[0] + } + if dq.segments.writing != nil { + return dq.segments.writing + } + return nil +} + +func (dq *diskQueue) altReaderLoop() { + curFrameID := frameID(0) + for { + segment := dq.nextSegmentForReading() + if segment == nil { + // TODO: wait + continue + } + // this is a strangely fine-grained lock maybe? + segment.Lock() + defer segment.Unlock() + + // dataSize is guaranteed to be positive because we don't add + // anything to the segments list unless it is. + dataSize := segment.size - segmentHeaderSize + file, err := os.Open(path) + if err != nil { + return nil, fmt.Errorf( + "Couldn't open segment %d: %w", segment.id, err) + } + reader := bufio.NewReader(file) + header, err := readSegmentHeader(reader) + if err != nil { + return nil, fmt.Errorf("Couldn't read segment header: %w", err) + } + } +} diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index a140ebe8674..ba0da434c6c 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -37,6 +37,9 @@ import ( // acknowledged. type frameID uint64 +// segmentPos is a byte index into the segment's data region. +type segmentPos uint64 + // The metadata for a single segment file. type queueSegment struct { sync.Mutex @@ -65,12 +68,12 @@ type segmentReader struct { raw io.Reader // The current byte offset of the reader within the file. - curPosition int64 + curPosition segmentPos // The position at which this reader should stop reading. This is often // the end of the file, but it may be earlier when the queue is reading // and writing to the same segment. - endPosition int64 + endPosition segmentPos // The checksumType field from this segment file's header. checksumType checksumType @@ -233,21 +236,3 @@ func (dq *diskQueue) segmentReaderForPosition( ) (*segmentReader, error) { panic("TODO: not implemented") } - -/* -func (sm *segmentManager) segmentReaderForPosition(pos bufferPosition) (*segmentReader, error) { - segment = getSegment(pos.segment) - - dataSize := segment.size - segmentHeaderSize - file, err := os.Open(pathForSegmentId(pos.segment)) - // ...read segment header... - checksumType := checksumTypeNone - file.Seek(segmentHeaderSize+pos.byteIndex, 0) - reader := bufio.NewReader(file) - return &segmentReader{ - raw: reader, - curPosition: pos.byteIndex, - endPosition: dataSize, - checksumType: checksumType, - }, nil -}*/ From 61fa5d77fa728d803192ed39b55ed1cf8532bf5a Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Fri, 17 Jul 2020 11:38:18 -0400 Subject: [PATCH 17/91] working on writer loop --- libbeat/publisher/queue/diskqueue/queue.go | 20 +++-- .../publisher/queue/diskqueue/reader_loop.go | 12 --- libbeat/publisher/queue/diskqueue/segments.go | 9 +- .../publisher/queue/diskqueue/writer_loop.go | 83 +++++++++++++++++++ 4 files changed, 102 insertions(+), 22 deletions(-) create mode 100644 libbeat/publisher/queue/diskqueue/writer_loop.go diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 20d97a5c099..8ce1164f945 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -98,7 +98,7 @@ type diskQueue struct { stateFile *stateFile // Metadata related to the segment files. - segments diskQueueSegments + segments *diskQueueSegments // The total bytes occupied by all segment files. This is the value // we check to see if there is enough space to add an incoming event @@ -178,21 +178,27 @@ type diskQueueSegments struct { // The segment that is currently being written. writing *queueSegment - writer *segmentWriter - reader *segmentReader + //writer *segmentWriter + //reader *segmentReader // A list of the segments that have been completely written but have - // not yet been processed by the reader loop, sorted by increasing + // not yet been completely processed by the reader loop, sorted by increasing // segment ID. Segments are always read in order. When a segment has // been read completely, it is removed from the front of this list and - // appended to completedSegments. + // appended to waiting. reading []*queueSegment // A list of the segments that have been read but have not yet been // completely acknowledged, sorted by increasing segment ID. When the // first entry of this list is completely acknowledged, it is removed - // from this list and the underlying file is deleted. - completed []*queueSegment + // from this list and added to finished. + waiting []*queueSegment + + // A list of the segments that have been completely processed and are + // ready to be deleted. The writer loop always tries to delete segments + // in this list before writing new data. When a segment is successfully + // deleted, it is removed from this list. + finished []*queueSegment // The next sequential unused segment ID. This is what will be assigned // to the next queueSegment we create. diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 9f51367cc6e..d26aaa04f64 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -78,18 +78,6 @@ func (dq *diskQueue) readerLoop() { } } -func (dq *diskQueue) writerLoop() { - for { - var frameBytes []byte - select { - case frameBytes = <-dq.inChan: - - case <-dq.done: - break - } - } -} - func (dq *diskQueue) newSegmentWriter() (*segmentWriter, error) { var err error dq.segments.Lock() diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index ba0da434c6c..99b0fe61f69 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -35,6 +35,8 @@ import ( // Every data frame read from the queue is assigned a unique sequential // integer, which is used to keep track of which frames have been // acknowledged. +// This id is not stable between restarts; the value 0 is always assigned +// to the oldest remaining frame on startup. type frameID uint64 // segmentPos is a byte index into the segment's data region. @@ -138,9 +140,10 @@ func queueSegmentsForPath( return segments, nil } -// A nil data frame with no error means this reader has no more frames. -// If nextDataFrame returns an error, it should be logged and the -// corresponding segment should be dropped. +// nextDataFrame returns the bytes of the next data frame, or an error if the +// frame couldn't be read. If an error is returned, the caller should log it +// and drop the containing segment. A nil return value with no error means +// there are no frames to read. func (reader *segmentReader) nextDataFrame() ([]byte, error) { if reader.curPosition >= reader.endPosition { return nil, nil diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go new file mode 100644 index 00000000000..74d4ebc5e34 --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -0,0 +1,83 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +import "os" + +// The writer loop's job is to continuously read a data frame from the +// queue's intake channel, if there is one, and write it to disk. +func (dq *diskQueue) writerLoop() { + // The open file handle for the segment currently being written. + // This should be non-nil if and only if dq.segments.writing is. + var file *os.File + var filePosition int64 + for { + var frameBytes []byte + select { + case frameBytes = <-dq.inChan: + + case <-dq.done: + break + } + dq.segments.Lock() + defer dq.segments.Unlock() + // TODO: try to delete dq.segments.finished + + var newFrameSize = uint64(len(frameBytes) + frameMetadataSize) + + if dq.segments.writing != nil && + dq.segments.writing.size+newFrameSize > dq.settings.MaxSegmentSize { + // This segment is full. Close the file handle and move it to the + // reading list. + // TODO: make reasonable efforts to sync to disk. + file.Close() + dq.segments.reading = append(dq.segments.reading, dq.segments.writing) + dq.segments.writing = nil + } + + if dq.segments.writing == nil { + // There is no active writing segment, create one. + // TODO: (actually create one) + } + + currentSize := dq.segments.sizeOnDiskWithLock() + // TODO: block (releasing dq.segments) until + // currentSize + newFrameSize <= dq.settings.MaxBufferSize + + // We now have a frame we want to write to disk, and enough free capacity + // to write it. + + } +} + +func (segments *diskQueueSegments) sizeOnDiskWithLock() uint64 { + total := uint64(0) + if segments.writing != nil { + total += segments.writing.size + } + for _, segment := range segments.reading { + total += segment.size + } + for _, segment := range segments.waiting { + total += segment.size + } + for _, segment := range segments.finished { + total += segment.size + } + return total +} From 3bf35ffbb2fba12d9d97ae62bfd895bb73dd4669 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Fri, 17 Jul 2020 16:46:22 -0400 Subject: [PATCH 18/91] fix most build errors --- libbeat/publisher/queue/diskqueue/config.go | 4 ++ libbeat/publisher/queue/diskqueue/queue.go | 18 ++++--- .../publisher/queue/diskqueue/reader_loop.go | 19 +++---- libbeat/publisher/queue/diskqueue/segments.go | 34 ++++++++----- .../publisher/queue/diskqueue/state_file.go | 10 ++-- .../publisher/queue/diskqueue/writer_loop.go | 51 ++++++++++++++++--- 6 files changed, 91 insertions(+), 45 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/config.go b/libbeat/publisher/queue/diskqueue/config.go index 66931924a16..3344cda8b49 100644 --- a/libbeat/publisher/queue/diskqueue/config.go +++ b/libbeat/publisher/queue/diskqueue/config.go @@ -44,6 +44,10 @@ func SettingsForUserConfig(config *common.Config) (Settings, error) { if err := config.Unpack(&userConfig); err != nil { return Settings{}, err } + settings := Settings{ + ChecksumType: ChecksumTypeCRC32, + } + settings.Path = userConfig.Path return Settings{}, nil } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 8ce1164f945..ad2da740c98 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -66,6 +66,8 @@ type Settings struct { // after it is opened (there is no way to acknowledge input from a // previous execution). It is ignored for events before that. //WriteToOutputACKListener queue.ACKListener + + ChecksumType ChecksumType } type segmentID uint64 @@ -75,7 +77,9 @@ type queuePosition struct { segment segmentID // The byte offset of this position within its segment. - byteIndex segmentPos + // This is specified relative to the start of the segment's data region, i.e. + // an offset of 0 means the first byte after the end of the segment header. + offset segmentOffset } type diskQueueOutput struct { @@ -119,7 +123,7 @@ type diskQueue struct { //writer *segmentWriter // The ol - firstPosition bufferPosition + //firstPosition bufferPosition // The position of the next event to read from the queue. If this equals // writePosition, then there are no events left to read. @@ -130,7 +134,7 @@ type diskQueue struct { // from the queue). // This is part of diskQueue and not diskQueueState since it represents // in-memory state that should not persist through a restart. - readPosition bufferPosition + //readPosition bufferPosition // A condition that is signalled when a segment file is deleted. // Used by writerLoop when the queue is full, to detect when to try again. @@ -263,7 +267,7 @@ func NewQueue(settings Settings) (queue.Queue, error) { return &diskQueue{ settings: settings, - segments: diskQueueSegments{ + segments: &diskQueueSegments{ reading: segments, }, closed: atomic.MakeBool(false), @@ -292,7 +296,7 @@ func (dq *diskQueue) nextSegmentReader() (*segmentReader, []error) { // Remove the segment from the active list and move it to // completedSegments until all its data has been acknowledged. dq.segments.reading = dq.segments.reading[1:] - dq.segments.completed = append(dq.segments.completed, segment) + dq.segments.waiting = append(dq.segments.waiting, segment) return reader, errors } // TODO: if segments.reading is empty we may still be able to @@ -322,14 +326,14 @@ func tryLoad(segment *queueSegment, path string) (*segmentReader, error) { return &segmentReader{ raw: reader, curPosition: 0, - endPosition: segmentPos(dataSize), + endPosition: segmentOffset(dataSize), checksumType: header.checksumType, }, nil } type segmentHeader struct { version uint32 - checksumType checksumType + checksumType ChecksumType } func readSegmentHeader(in io.Reader) (*segmentHeader, error) { diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index d26aaa04f64..ac573845007 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -18,8 +18,6 @@ package diskqueue import ( - "bufio" - "fmt" "io" "os" ) @@ -107,21 +105,18 @@ func (dq *diskQueue) newSegmentWriter() (*segmentWriter, error) { // This is only called from the writer loop. func (dq *diskQueue) writeFrameData(bytes []byte) error { - frameSize := uint64(len(bytes) + frameMetadataSize) + /*frameSize := uint64(len(bytes) + frameMetadataSize) dq.segments.Lock() defer dq.segments.Unlock() - /*if dq.segments.writer != nil && + if dq.segments.writer != nil && dq.segments.writer.position+frameSize > dq.settings.MaxSegmentSize { // There is a writing segment, but the incoming data frame doesn't // fit, so we need to finalize it and create a new one. //dq.segments.writer = //dq.segments.writing - }*/ - - /*.capacity() < frameSize { + } - } != nil && dq.writingSegment.*/ // while (free bytes) < frameSize { // block @@ -130,14 +125,14 @@ func (dq *diskQueue) writeFrameData(bytes []byte) error { if dq.segments.writing == nil { // There is no current output segment, create a new one. - } + }*/ return nil } func readSegment( reader io.Reader, checksumType checksumType, - start segmentPos, stop segmentPos, + start segmentOffset, stop segmentOffset, output chan diskQueueOutput, cancel chan struct{}, ) (int, error) { return 0, nil @@ -156,7 +151,7 @@ func (dq *diskQueue) nextSegmentForReading() *queueSegment { } func (dq *diskQueue) altReaderLoop() { - curFrameID := frameID(0) + /*curFrameID := frameID(0) for { segment := dq.nextSegmentForReading() if segment == nil { @@ -180,5 +175,5 @@ func (dq *diskQueue) altReaderLoop() { if err != nil { return nil, fmt.Errorf("Couldn't read segment header: %w", err) } - } + }*/ } diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 99b0fe61f69..5bcc749fa1f 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -39,8 +39,8 @@ import ( // to the oldest remaining frame on startup. type frameID uint64 -// segmentPos is a byte index into the segment's data region. -type segmentPos uint64 +// segmentOffset is a byte index into the segment's data region. +type segmentOffset uint64 // The metadata for a single segment file. type queueSegment struct { @@ -70,15 +70,15 @@ type segmentReader struct { raw io.Reader // The current byte offset of the reader within the file. - curPosition segmentPos + curPosition segmentOffset // The position at which this reader should stop reading. This is often // the end of the file, but it may be earlier when the queue is reading // and writing to the same segment. - endPosition segmentPos + endPosition segmentOffset // The checksumType field from this segment file's header. - checksumType checksumType + checksumType ChecksumType } type segmentWriter struct { @@ -87,11 +87,17 @@ type segmentWriter struct { position int64 } -type checksumType int +// ChecksumType specifies what checksum algorithm the queue should use to +// verify its data frames. +type ChecksumType int +// ChecksumTypeNone: Don't compute or verify checksums. +// ChecksumTypeCRC32: Compute the checksum with the Go standard library's +// "hash/crc32" package. const ( - checksumTypeNone = iota - checksumTypeCRC32 + ChecksumTypeNone = iota + + ChecksumTypeCRC32 ) // Each data frame has 2 32-bit lengths and 1 32-bit checksum. @@ -156,7 +162,7 @@ func (reader *segmentReader) nextDataFrame() ([]byte, error) { } // Bounds checking to make sure we can read this frame. - if reader.curPosition+int64(frameLength) > reader.endPosition { + if reader.curPosition+segmentOffset(frameLength) > reader.endPosition { // This frame extends past the end of our data region, which // should never happen unless there is data corruption. return nil, fmt.Errorf( @@ -202,7 +208,7 @@ func (reader *segmentReader) nextDataFrame() ([]byte, error) { return nil, fmt.Errorf("Disk queue: bad data frame checksum") } - reader.curPosition += int64(frameLength) + reader.curPosition += segmentOffset(frameLength) return data, nil } @@ -219,11 +225,11 @@ func (dq *diskQueue) ack(frame frameID) int { return ackedCount } -func computeChecksum(data []byte, checksumType checksumType) uint32 { +func computeChecksum(data []byte, checksumType ChecksumType) uint32 { switch checksumType { - case checksumTypeNone: + case ChecksumTypeNone: return 0 - case checksumTypeCRC32: + case ChecksumTypeCRC32: hash := crc32.NewIEEE() frameLength := uint32(len(data) + frameMetadataSize) binary.Write(hash, binary.LittleEndian, &frameLength) @@ -234,8 +240,10 @@ func computeChecksum(data []byte, checksumType checksumType) uint32 { } } +/* func (dq *diskQueue) segmentReaderForPosition( pos bufferPosition, ) (*segmentReader, error) { panic("TODO: not implemented") } +*/ diff --git a/libbeat/publisher/queue/diskqueue/state_file.go b/libbeat/publisher/queue/diskqueue/state_file.go index e6b572ffa5f..c29e1a528cb 100644 --- a/libbeat/publisher/queue/diskqueue/state_file.go +++ b/libbeat/publisher/queue/diskqueue/state_file.go @@ -33,7 +33,7 @@ type diskQueuePersistentState struct { // The oldest position in the queue. This is advanced as we receive ACKs from // downstream consumers indicating it is safe to remove old events. - firstPosition bufferPosition + firstPosition queuePosition } // A wrapper around os.File that saves and loads the queue state. @@ -74,7 +74,7 @@ func persistentStateFromHandle( } err = binary.Read( - reader, binary.LittleEndian, &state.firstPosition.byteIndex) + reader, binary.LittleEndian, &state.firstPosition.offset) if err != nil { return nil, err } @@ -87,7 +87,7 @@ func persistentStateFromHandle( // file with the result. Returns nil if successful, otherwise an error. func writePersistentStateToHandle( file *os.File, - firstPosition bufferPosition, + firstPosition queuePosition, ) error { _, err := file.Seek(0, 0) if err != nil { @@ -105,7 +105,7 @@ func writePersistentStateToHandle( return err } - err = binary.Write(file, binary.LittleEndian, &firstPosition.byteIndex) + err = binary.Write(file, binary.LittleEndian, &firstPosition.offset) if err != nil { return err } @@ -146,7 +146,7 @@ func stateFileForPath(path string) (*stateFile, error) { } if state == nil { // Initialize with new zero state. - err = writePersistentStateToHandle(file, bufferPosition{0, 0}) + err = writePersistentStateToHandle(file, queuePosition{0, 0}) if err != nil { return nil, fmt.Errorf("Couldn't write queue state to disk: %w", err) } diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 74d4ebc5e34..8cd3c4f25e4 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -17,7 +17,27 @@ package diskqueue -import "os" +import ( + "bytes" + "io" + "os" + "syscall" +) + +// frameForContent wraps the given content buffer in a +// frame header / footer and returns the resulting larger buffer. +func frameForContent( + frameContent []byte, checksumType ChecksumType, +) bytes.Buffer { + buf := bytes.Buffer{} + //checksum := computeChecksum(frameContent, checksumType) + /*buf + frameLength := len(frameContent) + frameMetadataSize; + frameBytes := make([]byte, frameLength) + frameWriter := + binary.Write(reader.raw, binary.LittleEndian, &frameLength)*/ + return buf +} // The writer loop's job is to continuously read a data frame from the // queue's intake channel, if there is one, and write it to disk. @@ -26,11 +46,11 @@ func (dq *diskQueue) writerLoop() { // This should be non-nil if and only if dq.segments.writing is. var file *os.File var filePosition int64 + var frameBuffer bytes.Buffer for { - var frameBytes []byte select { - case frameBytes = <-dq.inChan: - + case frameContent := <-dq.inChan: + frameBuffer = frameForContent(frameContent, dq.settings.ChecksumType) case <-dq.done: break } @@ -38,10 +58,9 @@ func (dq *diskQueue) writerLoop() { defer dq.segments.Unlock() // TODO: try to delete dq.segments.finished - var newFrameSize = uint64(len(frameBytes) + frameMetadataSize) - + frameLen := uint64(frameBuffer.Len()) if dq.segments.writing != nil && - dq.segments.writing.size+newFrameSize > dq.settings.MaxSegmentSize { + dq.segments.writing.size+frameLen > dq.settings.MaxSegmentSize { // This segment is full. Close the file handle and move it to the // reading list. // TODO: make reasonable efforts to sync to disk. @@ -61,7 +80,7 @@ func (dq *diskQueue) writerLoop() { // We now have a frame we want to write to disk, and enough free capacity // to write it. - + writeAll(file, frameBuffer.Bytes()) } } @@ -81,3 +100,19 @@ func (segments *diskQueueSegments) sizeOnDiskWithLock() uint64 { } return total } + +func writeAll(writer io.Writer, p []byte) (int, error) { + var N int + for len(p) > 0 { + n, err := writer.Write(p) + N, p = N+n, p[n:] + if err != nil && isRetryErr(err) { + return N, err + } + } + return N, nil +} + +func isRetryErr(err error) bool { + return err == syscall.EINTR || err == syscall.EAGAIN +} From 50bd450f2303e3cc996d8366c0333a1061df52c9 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 21 Jul 2020 14:52:59 -0400 Subject: [PATCH 19/91] checksumType -> ChecksumType --- libbeat/publisher/queue/diskqueue/reader_loop.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index ac573845007..ba5d0a86437 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -131,7 +131,7 @@ func (dq *diskQueue) writeFrameData(bytes []byte) error { } func readSegment( - reader io.Reader, checksumType checksumType, + reader io.Reader, checksumType ChecksumType, start segmentOffset, stop segmentOffset, output chan diskQueueOutput, cancel chan struct{}, ) (int, error) { From 04c9b60fafb6104e956e007232e0d2d62fe082ac Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 30 Jul 2020 16:46:37 -0400 Subject: [PATCH 20/91] working on read / write loops --- libbeat/publisher/queue/diskqueue/queue.go | 69 ++++++++---- .../publisher/queue/diskqueue/reader_loop.go | 5 +- libbeat/publisher/queue/diskqueue/segments.go | 5 +- .../publisher/queue/diskqueue/writer_loop.go | 106 ++++++++++++------ 4 files changed, 126 insertions(+), 59 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index ad2da740c98..8a2d7d8cc77 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -107,7 +107,7 @@ type diskQueue struct { // The total bytes occupied by all segment files. This is the value // we check to see if there is enough space to add an incoming event // to the queue. - bytesOnDisk uint64 + //bytesOnDisk uint64 // The memory queue of data blobs waiting to be written to disk. // To add something to the queue internally, send it to this channel. @@ -153,7 +153,7 @@ type diskQueue struct { // This will usually be less than ackedUpTo, since oldestFrame can't // advance until the entire segment file has been acknowledged and // deleted. - oldestFrame frameID + //oldestFrame frameID // This lock must be held to read and write acked and ackedUpTo. ackLock sync.Mutex @@ -165,10 +165,27 @@ type diskQueue struct { // can't yet be acknowledged as a continuous block). acked map[frameID]bool - // Whether the queue has been closed. Code that can't check the done - // channel (e.g. code that must wait on a condition variable) should - // always check this value when waking up. - closed atomic.Bool + // Whether the queue has been closed for write. When this is true, data that + // has already been added to the queue intake should continue being written + // to disk, but no new data should be accepted and the writerLoop routine + // should terminate when all data has been written. + closedForWrite atomic.Bool + + // Whether the queue has been closed for read. When this is true, no more + // frames can be read from the queue, and the readerLoop routine should + // terminate. The queue will continue to accept acks for already-read + // frames and update the on-disk state accordingly. + closedForRead atomic.Bool + + // If true, goroutines should return as soon as possible without waiting for + // data updates to complete. The queue will no longer write buffered frames + // to disk or update the on-disk state to reflect incoming acks. + // If abort is true, then closedForRead and closedForWrite must be true. + abort atomic.Bool + + // Wait group for shutdown of the goroutines associated with this queue: + // reader loop, writer loop. + waitGroup *sync.WaitGroup // The channel to signal our goroutines to shut down. done chan struct{} @@ -184,25 +201,28 @@ type diskQueueSegments struct { //writer *segmentWriter //reader *segmentReader + segmentDeletedCond *sync.Cond + frameWrittenCond *sync.Cond // A list of the segments that have been completely written but have // not yet been completely processed by the reader loop, sorted by increasing // segment ID. Segments are always read in order. When a segment has // been read completely, it is removed from the front of this list and - // appended to waiting. + // appended to read. reading []*queueSegment // A list of the segments that have been read but have not yet been // completely acknowledged, sorted by increasing segment ID. When the // first entry of this list is completely acknowledged, it is removed - // from this list and added to finished. - waiting []*queueSegment + // from this list and added to acked. + acking []*queueSegment // A list of the segments that have been completely processed and are // ready to be deleted. The writer loop always tries to delete segments // in this list before writing new data. When a segment is successfully - // deleted, it is removed from this list. - finished []*queueSegment + // deleted, it is removed from this list and the queue's + // segmentDeletedCond is signalled. + acked []*queueSegment // The next sequential unused segment ID. This is what will be assigned // to the next queueSegment we create. @@ -259,19 +279,26 @@ func NewQueue(settings Settings) (queue.Queue, error) { } }() - segments, err := queueSegmentsForPath( + initialSegments, err := queueSegmentsForPath( settings.directoryPath(), settings.Logger) if err != nil { return nil, err } + // segments needs to be created separately so its condition variables can + // point at its sync.Mutex. + segments := &diskQueueSegments{ + reading: initialSegments, + } + segments.segmentDeletedCond = sync.NewCond(segments) + segments.frameWrittenCond = sync.NewCond(segments) + return &diskQueue{ - settings: settings, - segments: &diskQueueSegments{ - reading: segments, - }, - closed: atomic.MakeBool(false), - done: make(chan struct{}), + settings: settings, + segments: segments, + closedForRead: atomic.MakeBool(false), + closedForWrite: atomic.MakeBool(false), + done: make(chan struct{}), }, nil } @@ -296,7 +323,7 @@ func (dq *diskQueue) nextSegmentReader() (*segmentReader, []error) { // Remove the segment from the active list and move it to // completedSegments until all its data has been acknowledged. dq.segments.reading = dq.segments.reading[1:] - dq.segments.waiting = append(dq.segments.waiting, segment) + dq.segments.acking = append(dq.segments.acking, segment) return reader, errors } // TODO: if segments.reading is empty we may still be able to @@ -393,7 +420,9 @@ func (settings Settings) segmentPath(segmentID segmentID) string { // func (dq *diskQueue) Close() error { - if dq.closed.Swap(true) { + closedForRead := dq.closedForRead.Swap(true) + closedForWrite := dq.closedForWrite.Swap(true) + if closedForRead && closedForWrite { return fmt.Errorf("Can't close disk queue: queue already closed") } // TODO: wait for worker threads? diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index ba5d0a86437..161ce3cbd51 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -24,18 +24,19 @@ import ( func (dq *diskQueue) readerLoop() { curFrameID := frameID(0) + logger := dq.settings.Logger.Named("readerLoop") for { dq.frameWrittenCond.Wait() reader, errs := dq.nextSegmentReader() for _, err := range errs { // Errors encountered while reading should be logged. - dq.settings.Logger.Error(err) + logger.Error(err) } if reader == nil { // We couldn't find a readable segment, wait for a new // data frame to be written. dq.frameWrittenCond.Wait() - if dq.closed.Load() { + if dq.closedForRead.Load() { // The queue has been closed, shut down. // TODO: cleanup (write the final read position) return diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 5bcc749fa1f..7088feacb70 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -50,7 +50,10 @@ type queueSegment struct { // The length in bytes of the segment file on disk. This is updated when // the segment is written to, and should always correspond to the end of - // a complete data frame. + // a complete data frame. If a write fails after writing part of a frame, + // size does not change, even though the on-disk size increases, because + // the later bytes will be overwritten by the next write and should not + // be counted against the number of writable bytes. size uint64 // The number of frames read from this segment, or zero if it has not diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 8cd3c4f25e4..1c9fe6e4d6c 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -39,51 +39,85 @@ func frameForContent( return buf } +type writerState struct { + // The open file handle for the segment currently being written. + // This should be non-nil if and only if diskQueue.segments.writing is. + file *os.File + filePosition int64 +} + +func handleFrame(dq *diskQueue, state *writerState, frame bytes.Buffer) { + dq.segments.Lock() + defer dq.segments.Unlock() + + frameLen := uint64(frame.Len()) + // If there isn't enough space left in the current segment, close the + // segment's file handle and move it to the reading list. + if dq.segments.writing != nil && + dq.segments.writing.size+frameLen > dq.settings.MaxSegmentSize { + // TODO: make reasonable efforts to sync to disk. + state.file.Close() + dq.segments.reading = append(dq.segments.reading, dq.segments.writing) + dq.segments.writing = nil + } + + if dq.segments.writing == nil { + // There is no active writing segment, create one. + // TODO: (actually create one) + } + + // TODO: try to delete dq.segments.acked + + currentSize := dq.segments.sizeOnDiskWithLock() + // Block (releasing the dq.segments lock) until + // currentSize + frameLen <= dq.settings.MaxBufferSize + for currentSize+frameLen > dq.settings.MaxBufferSize { + // Wait for some space to be freed. + dq.segments.segmentDeletedCond.Wait() + if dq.closedForWrite.Load() { + // The queue is closed, abort + } + } + + // We now have a frame we want to write to disk, and enough free capacity + // to write it. + writeAll(state.file, frame.Bytes()) +} + // The writer loop's job is to continuously read a data frame from the // queue's intake channel, if there is one, and write it to disk. +// It continues until the intake channel is empty or func (dq *diskQueue) writerLoop() { - // The open file handle for the segment currently being written. - // This should be non-nil if and only if dq.segments.writing is. - var file *os.File - var filePosition int64 - var frameBuffer bytes.Buffer + defer dq.waitGroup.Done() + //logger := dq.settings.Logger.Named("writerLoop") + state := &writerState{} + for { + if dq.abort.Load() { + // We are aborting, ignore any remaining buffered frames. + return + } select { case frameContent := <-dq.inChan: - frameBuffer = frameForContent(frameContent, dq.settings.ChecksumType) + if frameContent == nil { + // The channel has been drained, the writer loop should shut down. + return + } + frameBuffer := frameForContent(frameContent, dq.settings.ChecksumType) + handleFrame(dq, state, frameBuffer) + if !dq.abort.Load() { + // As long as we aren't aborting, continue processing any pending + // frames. + continue + } case <-dq.done: - break } - dq.segments.Lock() - defer dq.segments.Unlock() - // TODO: try to delete dq.segments.finished - - frameLen := uint64(frameBuffer.Len()) - if dq.segments.writing != nil && - dq.segments.writing.size+frameLen > dq.settings.MaxSegmentSize { - // This segment is full. Close the file handle and move it to the - // reading list. - // TODO: make reasonable efforts to sync to disk. - file.Close() - dq.segments.reading = append(dq.segments.reading, dq.segments.writing) - dq.segments.writing = nil - } - - if dq.segments.writing == nil { - // There is no active writing segment, create one. - // TODO: (actually create one) - } - - currentSize := dq.segments.sizeOnDiskWithLock() - // TODO: block (releasing dq.segments) until - // currentSize + newFrameSize <= dq.settings.MaxBufferSize - - // We now have a frame we want to write to disk, and enough free capacity - // to write it. - writeAll(file, frameBuffer.Bytes()) + // We've processed } } +// The number of bytes occupied by all the queue's segment files. Must +// be called with segments.Mutex held. func (segments *diskQueueSegments) sizeOnDiskWithLock() uint64 { total := uint64(0) if segments.writing != nil { @@ -92,10 +126,10 @@ func (segments *diskQueueSegments) sizeOnDiskWithLock() uint64 { for _, segment := range segments.reading { total += segment.size } - for _, segment := range segments.waiting { + for _, segment := range segments.acking { total += segment.size } - for _, segment := range segments.finished { + for _, segment := range segments.acked { total += segment.size } return total From 7a2e09a63c9bb71e8945aacb8d86ee80667c9bf6 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 30 Jul 2020 16:47:17 -0400 Subject: [PATCH 21/91] replace filebeat with a queue wrapper for testing --- filebeat/main.go | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/filebeat/main.go b/filebeat/main.go index e8f36acde01..57fc5bc6703 100644 --- a/filebeat/main.go +++ b/filebeat/main.go @@ -18,12 +18,15 @@ package main import ( + "fmt" "os" - "github.com/elastic/beats/v7/filebeat/cmd" - inputs "github.com/elastic/beats/v7/filebeat/input/default-inputs" + "github.com/elastic/beats/v7/libbeat/publisher/queue/diskqueue" ) +//"github.com/elastic/beats/v7/filebeat/cmd" +//inputs "github.com/elastic/beats/v7/filebeat/input/default-inputs" + // The basic model of execution: // - input: finds files in paths/globs to harvest, starts harvesters // - harvester: reads a file, sends events to the spooler @@ -33,7 +36,15 @@ import ( // Finally, input uses the registrar information, on restart, to // determine where in each file to restart a harvester. func main() { - if err := cmd.Filebeat(inputs.Init).Execute(); err != nil { + //if err := cmd.Filebeat(inputs.Init).Execute(); err != nil { + // os.Exit(1) + //} + settings := diskqueue.Settings{} + queue, err := diskqueue.NewQueue(settings) + if err != nil { + fmt.Printf("queue creation error: %v", err) os.Exit(1) } + queue.Close() + fmt.Printf("Hello") } From 132ba8e34e6acbdc570ef46c15915d0467292d64 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Wed, 12 Aug 2020 09:02:05 -0400 Subject: [PATCH 22/91] adapting encoder stuff from the disk spool --- .../publisher/queue/diskqueue/core_loop.go | 59 +++++++ libbeat/publisher/queue/diskqueue/producer.go | 30 +++- libbeat/publisher/queue/diskqueue/queue.go | 33 +++- .../publisher/queue/diskqueue/serialize.go | 144 +++++++++++++++++- 4 files changed, 257 insertions(+), 9 deletions(-) create mode 100644 libbeat/publisher/queue/diskqueue/core_loop.go diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go new file mode 100644 index 00000000000..3cc85f77ce6 --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -0,0 +1,59 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +import "github.com/elastic/beats/v7/libbeat/publisher" + +// A frame waiting to be written to disk +type writeFrame struct { + // The original event provided by the client to diskQueueProducer + event publisher.Event + + // The event, serialized for writing to disk and wrapped in a frame + // header / footer. + serialized []byte +} + +// A frame that has been read from disk +type readFrame struct { +} + +// A request sent from a producer to the core loop to add a frame to the queue. +// +type writeRequest struct { + frame *writeFrame + shouldBlock bool + responseChan chan bool +} + +// A readRequest is sent from the reader loop to the core loop when it +// needs a new segment file to read. +type readRequest struct { + responseChan chan *readResponse +} + +type readResponse struct { +} + +type cancelRequest struct { + producer *diskQueueProducer + // If producer.config.DropOnCancel is true, then the core loop will respond + // on responseChan with the number of dropped events. + // Otherwise, this field may be nil. + responseChan chan int +} diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index eeb1b4ab0bf..2e607983164 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -35,13 +35,35 @@ type diskQueueProducer struct { // func (producer *diskQueueProducer) Publish(event publisher.Event) bool { - //data := dataForEvent(event) - - panic("TODO: not implemented") + return producer.publish(event, true) } func (producer *diskQueueProducer) TryPublish(event publisher.Event) bool { - panic("TODO: not implemented") + return producer.publish(event, false) +} + +func (producer *diskQueueProducer) publish( + event publisher.Event, shouldBlock bool, +) bool { + serialized := dataFrameForEvent(event, producer.queue.settings.ChecksumType) + request := &writeRequest{ + frame: &writeFrame{ + event: event, + serialized: serialized, + }, + shouldBlock: shouldBlock, + responseChan: make(chan bool), + } + + select { + case producer.queue.writeRequestChan <- request: + // The request has been sent, and we are now guaranteed to get a result on + // the response channel, so we must read from it immediately to avoid + // blocking the core loop. + return <-request.responseChan + case <-producer.queue.done: + return false + } } func (producer *diskQueueProducer) Cancel() int { diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 8a2d7d8cc77..d11354bf53f 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -30,6 +30,7 @@ import ( "github.com/elastic/beats/v7/libbeat/feature" "github.com/elastic/beats/v7/libbeat/logp" "github.com/elastic/beats/v7/libbeat/paths" + "github.com/elastic/beats/v7/libbeat/publisher" "github.com/elastic/beats/v7/libbeat/publisher/queue" ) @@ -101,6 +102,9 @@ type diskQueue struct { // The persistent queue state (wraps diskQueuePersistentState on disk). stateFile *stateFile + // Data frames waiting to be written to disk. + incoming *pendingFrameData + // Metadata related to the segment files. segments *diskQueueSegments @@ -111,7 +115,11 @@ type diskQueue struct { // The memory queue of data blobs waiting to be written to disk. // To add something to the queue internally, send it to this channel. - inChan chan []byte + // inChan chan []byte + + writeRequestChan chan *writeRequest + readRequestChan chan *readRequest + cancelRequestChan chan *cancelRequest outChan chan diskQueueOutput @@ -191,6 +199,22 @@ type diskQueue struct { done chan struct{} } +// pendingFrame stores a single incoming event waiting to be written to disk, +// along with its serialization and metadata needed to notify its originating +// producer of ack / cancel state. +type pendingFrame struct { + event publisher.Event + producer *diskQueueProducer +} + +// pendingFrameData stores data frames waiting to be written to disk, with +// metadata to handle acks / cancellation if needed. +type pendingFrameData struct { + sync.Mutex + + frames []pendingFrame +} + // diskQueueSegments encapsulates segment-related queue metadata. type diskQueueSegments struct { // The lock should be held to read or write any of the fields below. @@ -261,7 +285,7 @@ func queueFactory( // and settings, creating it if it doesn't exist. func NewQueue(settings Settings) (queue.Queue, error) { // Create the given directory path if it doesn't exist. - err := os.MkdirAll(settings.Path, os.ModePerm) + err := os.MkdirAll(settings.directoryPath(), os.ModePerm) if err != nil { return nil, fmt.Errorf("Couldn't create disk queue directory: %w", err) } @@ -435,7 +459,10 @@ func (dq *diskQueue) BufferConfig() queue.BufferConfig { } func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { - panic("TODO: not implemented") + return &diskQueueProducer{ + queue: dq, + config: cfg, + } } func (dq *diskQueue) Consumer() queue.Consumer { diff --git a/libbeat/publisher/queue/diskqueue/serialize.go b/libbeat/publisher/queue/diskqueue/serialize.go index f395c19d313..0aa09ed19ad 100644 --- a/libbeat/publisher/queue/diskqueue/serialize.go +++ b/libbeat/publisher/queue/diskqueue/serialize.go @@ -15,14 +15,154 @@ // specific language governing permissions and limitations // under the License. +// Encoding / decoding routines adapted from +// libbeat/publisher/queue/spool/codec.go. + package diskqueue -import "github.com/elastic/beats/v7/libbeat/publisher" +import ( + "bytes" + "time" + + "github.com/elastic/beats/v7/libbeat/beat" + "github.com/elastic/beats/v7/libbeat/common" + "github.com/elastic/beats/v7/libbeat/outputs/codec" + "github.com/elastic/beats/v7/libbeat/publisher" + "github.com/elastic/go-structform/gotype" + "github.com/elastic/go-structform/json" +) -func dataForEvent(event publisher.Event) []byte { +func dataFrameForEvent( + event publisher.Event, checksumType ChecksumType, +) []byte { + buf := bytes.Buffer{} + //checksum := computeChecksum(frameContent, checksumType) panic("TODO: not implemented") } func eventForData(data []byte) publisher.Event { panic("TODO: not implemented") } + +type encoder struct { + buf bytes.Buffer + folder *gotype.Iterator + checksumType ChecksumType +} + +type decoder struct { + buf []byte + + parser *json.Parser + unfolder *gotype.Unfolder +} + +type entry struct { + Timestamp int64 + Flags uint8 + Meta common.MapStr + Fields common.MapStr +} + +func newEncoder(checksumType ChecksumType) (*encoder, error) { + e := &encoder{checksumType: checksumType} + e.reset() + return e, nil +} + +func (e *encoder) reset() { + e.folder = nil + + visitor := json.NewVisitor(&e.buf) + folder, err := gotype.NewIterator(visitor, + gotype.Folders( + codec.MakeTimestampEncoder(), + codec.MakeBCTimestampEncoder(), + ), + ) + if err != nil { + panic(err) + } + + e.folder = folder +} + +func (e *encoder) encode(event *publisher.Event) ([]byte, error) { + e.buf.Reset() + + var flags uint8 + /*if (event.Flags & publisher.GuaranteedSend) == publisher.GuaranteedSend { + flags = flagGuaranteed + }*/ + + err := e.folder.Fold(entry{ + Timestamp: event.Content.Timestamp.UTC().UnixNano(), + Flags: flags, + Meta: event.Content.Meta, + Fields: event.Content.Fields, + }) + if err != nil { + e.reset() + return nil, err + } + + return e.buf.Bytes(), nil +} + +func newDecoder() *decoder { + d := &decoder{} + d.reset() + return d +} + +func (d *decoder) reset() { + unfolder, err := gotype.NewUnfolder(nil) + if err != nil { + panic(err) // can not happen + } + + d.unfolder = unfolder + d.parser = json.NewParser(unfolder) +} + +// Buffer prepares the read buffer to hold the next event of n bytes. +func (d *decoder) Buffer(n int) []byte { + if cap(d.buf) > n { + d.buf = d.buf[:n] + } else { + d.buf = make([]byte, n) + } + return d.buf +} + +func (d *decoder) Decode() (publisher.Event, error) { + var ( + to entry + err error + contents = d.buf[1:] + ) + + d.unfolder.SetTarget(&to) + defer d.unfolder.Reset() + + err = d.parser.Parse(contents) + + if err != nil { + d.reset() // reset parser just in case + return publisher.Event{}, err + } + + var flags publisher.EventFlags + /*if (to.Flags & flagGuaranteed) != 0 { + flags |= publisher.GuaranteedSend + }*/ + + return publisher.Event{ + Flags: flags, + Content: beat.Event{ + Timestamp: time.Unix(0, to.Timestamp), + Fields: to.Fields, + Meta: to.Meta, + }, + }, nil +} From e73f55fc130bb2b400e3f5c24019fad998abce08 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Wed, 12 Aug 2020 16:18:29 -0400 Subject: [PATCH 23/91] add most of the api logic for the reader / writer loops --- .../publisher/queue/diskqueue/core_loop.go | 47 +++++++++ libbeat/publisher/queue/diskqueue/producer.go | 2 + libbeat/publisher/queue/diskqueue/queue.go | 27 ++---- .../publisher/queue/diskqueue/reader_loop.go | 97 ++++++++++++++++++- libbeat/publisher/queue/diskqueue/segments.go | 62 +++++++----- .../publisher/queue/diskqueue/serialize.go | 10 +- .../publisher/queue/diskqueue/writer_loop.go | 50 +++++++++- 7 files changed, 245 insertions(+), 50 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 3cc85f77ce6..e641d554d2f 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -57,3 +57,50 @@ type cancelRequest struct { // Otherwise, this field may be nil. responseChan chan int } + +func (dq *diskQueue) coreLoop() { + // writing is true if a writeRequest is currently being processed by the + // writer loop, false otherwise. + writing := false + + // writeRequests is a list of all write requests that have been accepted + // by the queue and are waiting to be written to disk. + writeRequests := []*writeRequest{} + + for { + select { + case <-dq.writerLoop.finishedWriting: + if len(writeRequests) > 0 { + dq.forwardWriteRequest(writeRequests[0]) + //dq.writerLoop.input <- writeRequests[0] + writeRequests = writeRequests[1:] + } else { + writing = false + } + case <-dq.writerLoop.nextWriteSegment: + // Create a new segment and send it back to the writer loop. + + } + } +} + +func (dq *diskQueue) forwardWriteRequest(request *writeRequest) { + // First we must decide which segment the new frame should be written to. + data := request.frame.serialized + segment := dq.segments.writing + + if segment != nil && + segment.size+uint64(len(data)) > dq.settings.MaxSegmentSize { + // The new frame is too big to fit in this segment, so close it and + // move it to the read queue. + segment.writer.Close() + // TODO: make reasonable attempts to sync the closed file. + dq.segments.reading = append(dq.segments.reading, segment) + segment = nil + } + + // If we don't have a segment, we need to create one. + if segment == nil { + segment = &queueSegment{id: dq.segments.nextID} + } +} diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index 2e607983164..1c427939388 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -28,6 +28,8 @@ type diskQueueProducer struct { // The configuration this producer was created with. config queue.ProducerConfig + + encoder frameEncoder } // diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index d11354bf53f..210e1284531 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -108,6 +108,8 @@ type diskQueue struct { // Metadata related to the segment files. segments *diskQueueSegments + writerLoop *writerLoop + // The total bytes occupied by all segment files. This is the value // we check to see if there is enough space to add an incoming event // to the queue. @@ -303,31 +305,22 @@ func NewQueue(settings Settings) (queue.Queue, error) { } }() - initialSegments, err := queueSegmentsForPath( - settings.directoryPath(), settings.Logger) + initialSegments, err := settings.scanExistingSegments() if err != nil { return nil, err } - // segments needs to be created separately so its condition variables can - // point at its sync.Mutex. - segments := &diskQueueSegments{ - reading: initialSegments, - } - segments.segmentDeletedCond = sync.NewCond(segments) - segments.frameWrittenCond = sync.NewCond(segments) - return &diskQueue{ - settings: settings, - segments: segments, - closedForRead: atomic.MakeBool(false), - closedForWrite: atomic.MakeBool(false), - done: make(chan struct{}), + settings: settings, + segments: &diskQueueSegments{ + reading: initialSegments, + }, + done: make(chan struct{}), }, nil } // This is only called by readerLoop. -func (dq *diskQueue) nextSegmentReader() (*segmentReader, []error) { +/*func (dq *diskQueue) nextSegmentReader() (*segmentReader, []error) { dq.segments.Lock() defer dq.segments.Unlock() @@ -354,7 +347,7 @@ func (dq *diskQueue) nextSegmentReader() (*segmentReader, []error) { // read partial data from segments.writing which is still being // written. return nil, errors -} +}*/ func tryLoad(segment *queueSegment, path string) (*segmentReader, error) { // this is a strangely fine-grained lock maybe? diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 161ce3cbd51..1e358b7c72d 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -22,7 +22,98 @@ import ( "os" ) -func (dq *diskQueue) readerLoop() { +type finishedReadingMessage struct { + // The number of frames read from the last file the reader loop was given. + frameCount int + + // If there was an error in the segment file (i.e. inconsistent data), the + // err field is set. + err error +} + +type readBlock struct { + reader io.Reader + length uint64 +} + +type readerLoop struct { + // When there is a block available for reading, it will be sent to + // nextReadBlock. When the reader loop has finished processing it, it + // sends the result to finishedReading. If there is more than one block + // available for reading, the core loop will wait until it gets a + // finishedReadingMessage before it + nextReadBlock chan readBlock + finishedReading chan finishedReadingMessage + + // Frames that have been read from disk are sent to this channel. + // Unlike most of the queue's API channels, this one is buffered to allow + // the reader to read ahead and cache pending frames before a consumer + // explicitly requests them. + output chan *readFrame +} + +func (rl *readerLoop) run() { + for { + block, ok := <-rl.nextReadBlock + if !ok { + // The channel has closed, we are shutting down. + return + } + rl.finishedReading <- rl.processBlock(block) + } +} + +func (rl *readerLoop) processBlock(block readBlock) finishedReadingMessage { + frameCount := 0 + for { + frame, err := block.nextFrame() + if err != nil { + return finishedReadingMessage{ + frameCount: frameCount, + err: err, + } + } + if frame == nil { + // There are no more frames in this block. + return finishedReadingMessage{ + frameCount: frameCount, + err: nil, + } + } + // We've read the frame, try sending it to the output channel. + select { + case rl.output <- frame: + // Success! Increment the total for this block. + frameCount++ + case <-rl.nextReadBlock: + // Since we haven't sent a finishedReading message yet, we can only + // reach this case when the nextReadBlock channel is closed, indicating + // queue shutdown. In this case we immediately return. + return finishedReadingMessage{ + frameCount: frameCount, + err: nil, + } + } + + // If the output channel's buffer is not full, the previous select + // might not recognize when the queue is being closed, so check that + // again separately before we move on to the next data frame. + select { + case <-rl.nextReadBlock: + return finishedReadingMessage{ + frameCount: frameCount, + err: nil, + } + default: + } + } +} + +func (block *readBlock) nextFrame() (*readFrame, error) { + return nil, nil +} + +/*func (dq *diskQueue) readerLoop() { curFrameID := frameID(0) logger := dq.settings.Logger.Named("readerLoop") for { @@ -75,7 +166,7 @@ func (dq *diskQueue) readerLoop() { } reader.segment.framesRead += framesRead } -} +}*/ func (dq *diskQueue) newSegmentWriter() (*segmentWriter, error) { var err error @@ -90,7 +181,7 @@ func (dq *diskQueue) newSegmentWriter() (*segmentWriter, error) { } }() - segment := &queueSegment{id: id} + segment := &queueSegment{queue: dq, id: id} path := dq.settings.segmentPath(id) file, err := os.OpenFile(path, os.O_CREATE|os.O_TRUNC, 0600) diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 7088feacb70..04296b002ab 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -27,9 +27,6 @@ import ( "sort" "strconv" "strings" - "sync" - - "github.com/elastic/beats/v7/libbeat/logp" ) // Every data frame read from the queue is assigned a unique sequential @@ -44,28 +41,44 @@ type segmentOffset uint64 // The metadata for a single segment file. type queueSegment struct { - sync.Mutex - + // A segment id is globally unique within its originating queue. id segmentID - // The length in bytes of the segment file on disk. This is updated when + // The settings for the queue that created this segment. Used for locating + // the queue file on disk and determining its checksum behavior. + queueSettings *Settings + + // Whether the file for this segment exists on disk yet. If it does + // not, then calling getWriter() will create it and return a writer + // positioned at the start of the data region. + created bool + + // The size in bytes of the segment file on disk. This is updated when // the segment is written to, and should always correspond to the end of - // a complete data frame. If a write fails after writing part of a frame, - // size does not change, even though the on-disk size increases, because - // the later bytes will be overwritten by the next write and should not - // be counted against the number of writable bytes. + // a complete data frame. size uint64 - // The number of frames read from this segment, or zero if it has not - // yet been completely read. - // It is safe to delete a segment when framesRead > 0 and all those - // frames have been acknowledged. + // The number of frames read from this segment. framesRead int64 + + // If this segment is being written or read, then reader / writer + // contain the respective file handles. To get a valid reader / writer for + // a segment that may not yet be open, call getReader / getWriter instead. + reader io.ReadCloser + writer io.WriteCloser +} + +func (segment *queueSegment) getReader() (io.ReadCloser, error) { + return nil, nil +} + +func (segment *queueSegment) getWriter() (io.WriteCloser, error) { + return nil, nil } // segmentReader is a wrapper around io.Reader that provides helpers and // metadata for decoding segment files. -type segmentReader struct { +/*type segmentReader struct { // The segment this reader was generated from. segment *queueSegment @@ -82,7 +95,7 @@ type segmentReader struct { // The checksumType field from this segment file's header. checksumType ChecksumType -} +}*/ type segmentWriter struct { segment *queueSegment @@ -103,8 +116,11 @@ const ( ChecksumTypeCRC32 ) -// Each data frame has 2 32-bit lengths and 1 32-bit checksum. -const frameMetadataSize = 12 +// Each data frame has a 32-bit lengths and 1 32-bit checksum +// in the header, and a duplicate 32-bit length in the footer. +const frameHeaderSize = 8 +const frameFooterSize = 4 +const frameMetadataSize = frameHeaderSize + frameFooterSize // Each segment header has a 32-bit version and a 32-bit checksum type. const segmentHeaderSize = 8 @@ -116,9 +132,8 @@ func (s bySegmentID) Len() int { return len(s) } func (s bySegmentID) Swap(i, j int) { s[i], s[j] = s[j], s[i] } func (s bySegmentID) Less(i, j int) bool { return s[i].size < s[j].size } -func queueSegmentsForPath( - path string, logger *logp.Logger, -) ([]*queueSegment, error) { +func (settings *Settings) scanExistingSegments() ([]*queueSegment, error) { + path := settings.directoryPath() files, err := ioutil.ReadDir(path) if err != nil { return nil, fmt.Errorf("Couldn't read queue directory '%s': %w", path, err) @@ -139,8 +154,9 @@ func queueSegmentsForPath( if id, err := strconv.ParseUint(components[0], 10, 64); err == nil { segments = append(segments, &queueSegment{ - id: segmentID(id), - size: uint64(file.Size()), + id: segmentID(id), + created: true, + size: uint64(file.Size()), }) } } diff --git a/libbeat/publisher/queue/diskqueue/serialize.go b/libbeat/publisher/queue/diskqueue/serialize.go index 0aa09ed19ad..1945aa98f4c 100644 --- a/libbeat/publisher/queue/diskqueue/serialize.go +++ b/libbeat/publisher/queue/diskqueue/serialize.go @@ -44,7 +44,7 @@ func eventForData(data []byte) publisher.Event { panic("TODO: not implemented") } -type encoder struct { +type frameEncoder struct { buf bytes.Buffer folder *gotype.Iterator checksumType ChecksumType @@ -64,13 +64,13 @@ type entry struct { Fields common.MapStr } -func newEncoder(checksumType ChecksumType) (*encoder, error) { - e := &encoder{checksumType: checksumType} +func newEncoder(checksumType ChecksumType) (*frameEncoder, error) { + e := &frameEncoder{checksumType: checksumType} e.reset() return e, nil } -func (e *encoder) reset() { +func (e *frameEncoder) reset() { e.folder = nil visitor := json.NewVisitor(&e.buf) @@ -87,7 +87,7 @@ func (e *encoder) reset() { e.folder = folder } -func (e *encoder) encode(event *publisher.Event) ([]byte, error) { +func (e *frameEncoder) encode(event *publisher.Event) ([]byte, error) { e.buf.Reset() var flags uint8 diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 1c9fe6e4d6c..9db2a76011d 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -22,8 +22,54 @@ import ( "io" "os" "syscall" + + "github.com/elastic/beats/v7/libbeat/logp" ) +// One block for the writer loop consists of a write request and the +// segment it should be written to. +type writeBlock struct { + request *writeRequest + segment *queueSegment +} + +type writerLoop struct { + logger *logp.Logger + + // The writer loop listens on the input channel for write blocks, and + // writes them to disk immediately (all queue capacity checking etc. is + // done by the core loop before sending it to the writer). + input chan *writeBlock + + // The writer loop sends to this channel when it has finished writing a + // frame, to signal the core loop that it is ready for the next one. To + // ensure that the core loop doesn't block, the writer loop always reads + // from input immediately after sending to finishedWriting. + finishedWriting chan struct{} + + // The writer loop sends to nextWriteSegment to indicate that it needs a new + // segment file to write its frames to. The core loop must respond on + // nextSegmentResponse with the next segment file structure. + //nextSegmentRequest chan struct{} + //nextSegmentResponse chan *queueSegment +} + +func (wl *writerLoop) run() { + for { + block, ok := <-wl.input + if !ok { + // The input channel is closed, we are done + return + } + wl.processRequest(block) + wl.finishedWriting <- struct{}{} + } +} + +func (wl *writerLoop) processRequest(block *writeBlock) { + writer := block.segment.getWriter() +} + // frameForContent wraps the given content buffer in a // frame header / footer and returns the resulting larger buffer. func frameForContent( @@ -87,7 +133,7 @@ func handleFrame(dq *diskQueue, state *writerState, frame bytes.Buffer) { // The writer loop's job is to continuously read a data frame from the // queue's intake channel, if there is one, and write it to disk. // It continues until the intake channel is empty or -func (dq *diskQueue) writerLoop() { +/*func (dq *diskQueue) writerLoop() { defer dq.waitGroup.Done() //logger := dq.settings.Logger.Named("writerLoop") state := &writerState{} @@ -114,7 +160,7 @@ func (dq *diskQueue) writerLoop() { } // We've processed } -} +}*/ // The number of bytes occupied by all the queue's segment files. Must // be called with segments.Mutex held. From 6d2ca31e04c4d40bf7a270ea778936bebe56181f Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 13 Aug 2020 09:46:01 -0400 Subject: [PATCH 24/91] filling in segment-deletion api --- .../publisher/queue/diskqueue/core_loop.go | 53 ++++++++- .../publisher/queue/diskqueue/deleter_loop.go | 79 +++++++++++++ libbeat/publisher/queue/diskqueue/queue.go | 42 +++---- .../publisher/queue/diskqueue/reader_loop.go | 7 +- libbeat/publisher/queue/diskqueue/segments.go | 109 +++++++++++++++--- .../publisher/queue/diskqueue/writer_loop.go | 25 +--- 6 files changed, 243 insertions(+), 72 deletions(-) create mode 100644 libbeat/publisher/queue/diskqueue/deleter_loop.go diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index e641d554d2f..3aa8741e1ec 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -63,23 +63,70 @@ func (dq *diskQueue) coreLoop() { // writer loop, false otherwise. writing := false + // reading is true if the reader loop is processing a readBlock, false + // otherwise. + reading := false + + // deleting is true if the segment-deletion loop is processing a deletion + // request, false otherwise. + deleting := false + // writeRequests is a list of all write requests that have been accepted // by the queue and are waiting to be written to disk. writeRequests := []*writeRequest{} for { select { + // Endpoints used by the external API + case writeRequest := <-dq.writeRequestChan: + // We will accept this request if there is enough capacity left in + // the queue (after accounting for the pending writes that were + // already accepted). + pendingBytes := 0 + for _, request := range writeRequests { + pendingBytes += len(request.serialized) + } + + case cancelRequest := <-dq.cancelRequestChan: + + // Writer loop handling case <-dq.writerLoop.finishedWriting: if len(writeRequests) > 0 { dq.forwardWriteRequest(writeRequests[0]) - //dq.writerLoop.input <- writeRequests[0] writeRequests = writeRequests[1:] } else { writing = false } - case <-dq.writerLoop.nextWriteSegment: - // Create a new segment and send it back to the writer loop. + // Reader loop handling + case readResponse := <-dq.readerLoop.finishedReading: + + // Deleter loop handling + case deleteResponse := <-dq.deleterLoop.response: + if len(deleteResponse.deleted) > 0 { + // One or more segments were deleted, recompute the outstanding list. + newAckedSegments := []*queueSegment{} + for _, segment := range dq.segments.acked { + if !deleteResponse.deleted[segment] { + // This segment wasn't deleted, so it goes in the new list. + newAckedSegments = append(newAckedSegments, segment) + } + } + dq.segments.acked = newAckedSegments + } + if len(deleteResponse.errors) > 0 { + dq.settings.Logger.Errorw("Couldn't delete old segment files", + "errors", deleteResponse.errors) + } + + if len(dq.segments.acked) > 0 { + // There are still (possibly new) segments to delete, send the + // next batch. + dq.deleterLoop.input <- &deleteRequest{segments: dq.segments.acked} + } else { + // Nothing more to delete for now, update the deleting flag. + deleting = false + } } } } diff --git a/libbeat/publisher/queue/diskqueue/deleter_loop.go b/libbeat/publisher/queue/diskqueue/deleter_loop.go new file mode 100644 index 00000000000..8b990b13da6 --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/deleter_loop.go @@ -0,0 +1,79 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +import ( + "errors" + "os" +) + +type deleterLoop struct { + // The settings for the queue that created this loop. + queueSettings *Settings + + // When one or more segments are ready to delete, they are sent to the + // deleter loop input as a deleteRequest. At most one deleteRequest may be + // outstanding at any time. + input chan *deleteRequest + + // When a deleteRequest has been completely processed, the resulting + // deleteResponse is sent on the response channel. If at least one deletion + // was successful, the response is sent immediately. Otherwise, the deleter + // loop delays for queueSettings.RetryWriteInterval before returning, so + // that delays don't have to be handled by the core loop. + response chan *deleteResponse +} + +type deleteRequest struct { + segments []*queueSegment +} + +type deleteResponse struct { + // The queue segments that were successfully deleted. + deleted map[*queueSegment]bool + + // Errors + errors []error +} + +func (dl *deleterLoop) run() { + for { + request, ok := <-dl.input + if !ok { + // The channel has been closed, time to shut down. + return + } + deleted := make(map[*queueSegment]bool, len(request.segments)) + errorList := []error{} + for _, segment := range request.segments { + path := dl.queueSettings.segmentPath(segment.id) + err := os.Remove(path) + // We ignore errors caused by the file not existing: this shouldn't + // happen, but it is still safe to report it as successfully removed. + if err == nil || errors.Is(err, os.ErrNotExist) { + errorList = append(errorList, err) + } else { + deleted[segment] = true + } + } + dl.response <- &deleteResponse{ + deleted: deleted, + errors: errorList, + } + } +} diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 210e1284531..ea6d11ef354 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -18,15 +18,12 @@ package diskqueue import ( - "bufio" "fmt" - "io" "os" "path/filepath" "sync" "github.com/elastic/beats/v7/libbeat/common" - "github.com/elastic/beats/v7/libbeat/common/atomic" "github.com/elastic/beats/v7/libbeat/feature" "github.com/elastic/beats/v7/libbeat/logp" "github.com/elastic/beats/v7/libbeat/paths" @@ -103,12 +100,14 @@ type diskQueue struct { stateFile *stateFile // Data frames waiting to be written to disk. - incoming *pendingFrameData + //incoming *pendingFrameData // Metadata related to the segment files. segments *diskQueueSegments - writerLoop *writerLoop + readerLoop *readerLoop + writerLoop *writerLoop + deleterLoop *deleterLoop // The total bytes occupied by all segment files. This is the value // we check to see if there is enough space to add an incoming event @@ -123,7 +122,7 @@ type diskQueue struct { readRequestChan chan *readRequest cancelRequestChan chan *cancelRequest - outChan chan diskQueueOutput + //outChan chan diskQueueOutput // The currently active segment reader, or nil if there is none. //reader *segmentReader @@ -150,14 +149,14 @@ type diskQueue struct { // Used by writerLoop when the queue is full, to detect when to try again. // When the queue is closed, this condition will receive a broadcast after // diskQueue.closed is set to true. - segmentDeletedCond sync.Cond + //segmentDeletedCond sync.Cond // A condition that is signalled when a frame has been completely // written to disk. // Used by readerLoop when the queue is empty, to detect when to try again. // When the queue is closed, this condition will receive a broadcast after // diskQueue.closed is set to true. - frameWrittenCond sync.Cond + //frameWrittenCond sync.Cond // The oldest frame id that is still stored on disk. // This will usually be less than ackedUpTo, since oldestFrame can't @@ -179,22 +178,22 @@ type diskQueue struct { // has already been added to the queue intake should continue being written // to disk, but no new data should be accepted and the writerLoop routine // should terminate when all data has been written. - closedForWrite atomic.Bool + //closedForWrite atomic.Bool // Whether the queue has been closed for read. When this is true, no more // frames can be read from the queue, and the readerLoop routine should // terminate. The queue will continue to accept acks for already-read // frames and update the on-disk state accordingly. - closedForRead atomic.Bool + //closedForRead atomic.Bool // If true, goroutines should return as soon as possible without waiting for // data updates to complete. The queue will no longer write buffered frames // to disk or update the on-disk state to reflect incoming acks. // If abort is true, then closedForRead and closedForWrite must be true. - abort atomic.Bool + //abort atomic.Bool // Wait group for shutdown of the goroutines associated with this queue: - // reader loop, writer loop. + // core loop, reader loop, writer loop, deleter loop. waitGroup *sync.WaitGroup // The channel to signal our goroutines to shut down. @@ -305,7 +304,7 @@ func NewQueue(settings Settings) (queue.Queue, error) { } }() - initialSegments, err := settings.scanExistingSegments() + initialSegments, err := scanExistingSegments(settings.directoryPath()) if err != nil { return nil, err } @@ -348,7 +347,7 @@ func NewQueue(settings Settings) (queue.Queue, error) { // written. return nil, errors }*/ - +/* func tryLoad(segment *queueSegment, path string) (*segmentReader, error) { // this is a strangely fine-grained lock maybe? segment.Lock() @@ -374,20 +373,7 @@ func tryLoad(segment *queueSegment, path string) (*segmentReader, error) { checksumType: header.checksumType, }, nil } - -type segmentHeader struct { - version uint32 - checksumType ChecksumType -} - -func readSegmentHeader(in io.Reader) (*segmentHeader, error) { - header := segmentHeader{} - if header.version != 0 { - return nil, fmt.Errorf("Unrecognized schema version %d", header.version) - } - panic("TODO: not implemented") - //return nil, nil -} +*/ // readNextFrame reads the next pending data frame in the queue // and returns its contents. diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 1e358b7c72d..3736b0fa731 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -19,7 +19,6 @@ package diskqueue import ( "io" - "os" ) type finishedReadingMessage struct { @@ -168,7 +167,7 @@ func (block *readBlock) nextFrame() (*readFrame, error) { } }*/ -func (dq *diskQueue) newSegmentWriter() (*segmentWriter, error) { +/*func (dq *diskQueue) newSegmentWriter() (*segmentWriter, error) { var err error dq.segments.Lock() defer dq.segments.Unlock() @@ -181,7 +180,7 @@ func (dq *diskQueue) newSegmentWriter() (*segmentWriter, error) { } }() - segment := &queueSegment{queue: dq, id: id} + segment := &queueSegment{queueSettings: &dq.settings, id: id} path := dq.settings.segmentPath(id) file, err := os.OpenFile(path, os.O_CREATE|os.O_TRUNC, 0600) @@ -193,7 +192,7 @@ func (dq *diskQueue) newSegmentWriter() (*segmentWriter, error) { segment: segment, file: file, }, nil -} +}*/ // This is only called from the writer loop. func (dq *diskQueue) writeFrameData(bytes []byte) error { diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 04296b002ab..2cba3076180 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -58,22 +58,29 @@ type queueSegment struct { // a complete data frame. size uint64 - // The number of frames read from this segment. + // The header metadata for this segment file. This field is nil if the + // segment has not yet been opened for reading. It should only be + // accessed by the reader loop. + header *segmentHeader + + // The number of frames read from this segment during this session. This + // does not necessarily equal the number of frames in the segment, even + // after reading is complete, since the segment may have been partially + // read during a previous session. + // + // Used to count how many frames still need to be acknowledged by consumers. framesRead int64 // If this segment is being written or read, then reader / writer // contain the respective file handles. To get a valid reader / writer for // a segment that may not yet be open, call getReader / getWriter instead. - reader io.ReadCloser - writer io.WriteCloser + reader *os.File + writer *os.File } -func (segment *queueSegment) getReader() (io.ReadCloser, error) { - return nil, nil -} - -func (segment *queueSegment) getWriter() (io.WriteCloser, error) { - return nil, nil +type segmentHeader struct { + version uint32 + checksumType ChecksumType } // segmentReader is a wrapper around io.Reader that provides helpers and @@ -97,11 +104,11 @@ func (segment *queueSegment) getWriter() (io.WriteCloser, error) { checksumType ChecksumType }*/ -type segmentWriter struct { +/*type segmentWriter struct { segment *queueSegment file *os.File position int64 -} +}*/ // ChecksumType specifies what checksum algorithm the queue should use to // verify its data frames. @@ -132,8 +139,7 @@ func (s bySegmentID) Len() int { return len(s) } func (s bySegmentID) Swap(i, j int) { s[i], s[j] = s[j], s[i] } func (s bySegmentID) Less(i, j int) bool { return s[i].size < s[j].size } -func (settings *Settings) scanExistingSegments() ([]*queueSegment, error) { - path := settings.directoryPath() +func scanExistingSegments(path string) ([]*queueSegment, error) { files, err := ioutil.ReadDir(path) if err != nil { return nil, fmt.Errorf("Couldn't read queue directory '%s': %w", path, err) @@ -165,11 +171,84 @@ func (settings *Settings) scanExistingSegments() ([]*queueSegment, error) { return segments, nil } +// Should only be called from the reader loop. +func (segment *queueSegment) getReader() (io.ReadCloser, error) { + if segment.reader != nil { + return segment.reader, nil + } + path := segment.queueSettings.segmentPath(segment.id) + file, err := os.Open(path) + if err != nil { + return nil, fmt.Errorf( + "Couldn't open segment %d: %w", segment.id, err) + } + //reader := bufio.NewReader(file) + header, err := readSegmentHeader(file) + if err != nil { + return nil, fmt.Errorf("Couldn't read segment header: %w", err) + } + segment.header = header + + return file, nil +} + +// Should only be called from the writer loop. +func (segment *queueSegment) getWriter() (io.WriteCloser, error) { + if segment.writer != nil { + return segment.writer, nil + } + path := segment.queueSettings.segmentPath(segment.id) + file, err := os.OpenFile(path, os.O_CREATE|os.O_TRUNC, 0600) + if err != nil { + return nil, err + } + segment.writer = file + header := &segmentHeader{ + version: 0, + checksumType: segment.queueSettings.ChecksumType, + } + err = writeSegmentHeader(file, header) + // TODO: write header + return file, nil +} + +func readSegmentHeader(in io.Reader) (*segmentHeader, error) { + header := segmentHeader{} + if header.version != 0 { + return nil, fmt.Errorf("Unrecognized schema version %d", header.version) + } + panic("TODO: not implemented") + //return nil, nil +} + +func writeSegmentHeader(out io.Writer, header *segmentHeader) error { + panic("TODO: not implemented") +} + +// The number of bytes occupied by all the queue's segment files. This +// should only be called from the core loop. +func (segments *diskQueueSegments) sizeOnDisk() uint64 { + total := uint64(0) + if segments.writing != nil { + total += segments.writing.size + } + for _, segment := range segments.reading { + total += segment.size + } + for _, segment := range segments.acking { + total += segment.size + } + for _, segment := range segments.acked { + total += segment.size + } + return total +} + // nextDataFrame returns the bytes of the next data frame, or an error if the // frame couldn't be read. If an error is returned, the caller should log it // and drop the containing segment. A nil return value with no error means // there are no frames to read. -func (reader *segmentReader) nextDataFrame() ([]byte, error) { +/*func (reader *segmentReader) nextDataFrame() ([]byte, error) { if reader.curPosition >= reader.endPosition { return nil, nil } @@ -229,7 +308,7 @@ func (reader *segmentReader) nextDataFrame() ([]byte, error) { reader.curPosition += segmentOffset(frameLength) return data, nil -} +}*/ // returns the number of indices by which ackedUpTo was advanced. func (dq *diskQueue) ack(frame frameID) int { diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 9db2a76011d..87c6dc00288 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -67,7 +67,7 @@ func (wl *writerLoop) run() { } func (wl *writerLoop) processRequest(block *writeBlock) { - writer := block.segment.getWriter() + writer, err := block.segment.getWriter() } // frameForContent wraps the given content buffer in a @@ -92,7 +92,7 @@ type writerState struct { filePosition int64 } -func handleFrame(dq *diskQueue, state *writerState, frame bytes.Buffer) { +/*func handleFrame(dq *diskQueue, state *writerState, frame bytes.Buffer) { dq.segments.Lock() defer dq.segments.Unlock() @@ -128,7 +128,7 @@ func handleFrame(dq *diskQueue, state *writerState, frame bytes.Buffer) { // We now have a frame we want to write to disk, and enough free capacity // to write it. writeAll(state.file, frame.Bytes()) -} +}*/ // The writer loop's job is to continuously read a data frame from the // queue's intake channel, if there is one, and write it to disk. @@ -162,25 +162,6 @@ func handleFrame(dq *diskQueue, state *writerState, frame bytes.Buffer) { } }*/ -// The number of bytes occupied by all the queue's segment files. Must -// be called with segments.Mutex held. -func (segments *diskQueueSegments) sizeOnDiskWithLock() uint64 { - total := uint64(0) - if segments.writing != nil { - total += segments.writing.size - } - for _, segment := range segments.reading { - total += segment.size - } - for _, segment := range segments.acking { - total += segment.size - } - for _, segment := range segments.acked { - total += segment.size - } - return total -} - func writeAll(writer io.Writer, p []byte) (int, error) { var N int for len(p) > 0 { From 988cef65aedbaa9b4ea29e403de367f4fd28815a Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 13 Aug 2020 10:36:10 -0400 Subject: [PATCH 25/91] connect consumer ack endpoints --- .../publisher/queue/diskqueue/core_loop.go | 128 +++++++++++++++--- libbeat/publisher/queue/diskqueue/queue.go | 14 +- 2 files changed, 119 insertions(+), 23 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 3aa8741e1ec..c13b96b0475 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -58,49 +58,73 @@ type cancelRequest struct { responseChan chan int } -func (dq *diskQueue) coreLoop() { +type coreLoop struct { + // The queue that created this coreLoop. The core loop is the only one of + // the main goroutines for the queue that has a pointer to the queue and + // understands its logic / structure. + // Possible TODO: the split between fields in coreLoop and fields in + // diskQueue seems artificial. Maybe the values here should be promoted + // to diskQueue fields, and the core loop should just be a function on + // diskQueue. + queue *diskQueue + // writing is true if a writeRequest is currently being processed by the // writer loop, false otherwise. - writing := false + writing bool // reading is true if the reader loop is processing a readBlock, false // otherwise. - reading := false + reading bool // deleting is true if the segment-deletion loop is processing a deletion // request, false otherwise. - deleting := false + deleting bool - // writeRequests is a list of all write requests that have been accepted + // pendingWrites is a list of all write requests that have been accepted // by the queue and are waiting to be written to disk. - writeRequests := []*writeRequest{} + pendingWrites []*writeRequest + + // blockedWrites is a list of all write requests that are waiting for + // free space in the queue. + blockedWrites []*writeRequest + + // This value represents the oldest frame ID for a segment that has not + // yet been moved to the acked list. It is used to detect when the oldest + // outstanding segment has been fully acknowledged by the consumer. + oldestFrameID frameID +} + +func (cl *coreLoop) run() { + dq := cl.queue for { select { - // Endpoints used by the external API + /////////////////////////////////////// + // Endpoints used by the public API case writeRequest := <-dq.writeRequestChan: - // We will accept this request if there is enough capacity left in - // the queue (after accounting for the pending writes that were - // already accepted). - pendingBytes := 0 - for _, request := range writeRequests { - pendingBytes += len(request.serialized) - } + cl.handleProducerWriteRequest(writeRequest) case cancelRequest := <-dq.cancelRequestChan: + cl.handleProducerCancelRequest(cancelRequest) + case ackedUpTo := <-dq.consumerAckChan: + cl.handleConsumerAck(ackedUpTo) + + /////////////////////////////////////// // Writer loop handling case <-dq.writerLoop.finishedWriting: - if len(writeRequests) > 0 { - dq.forwardWriteRequest(writeRequests[0]) - writeRequests = writeRequests[1:] + if len(cl.pendingWrites) > 0 { + cl.forwardWriteRequest(cl.pendingWrites[0]) + cl.pendingWrites = cl.pendingWrites[1:] } else { - writing = false + cl.writing = false } + /////////////////////////////////////// // Reader loop handling case readResponse := <-dq.readerLoop.finishedReading: + /////////////////////////////////////// // Deleter loop handling case deleteResponse := <-dq.deleterLoop.response: if len(deleteResponse.deleted) > 0 { @@ -125,13 +149,14 @@ func (dq *diskQueue) coreLoop() { dq.deleterLoop.input <- &deleteRequest{segments: dq.segments.acked} } else { // Nothing more to delete for now, update the deleting flag. - deleting = false + cl.deleting = false } } } } -func (dq *diskQueue) forwardWriteRequest(request *writeRequest) { +func (cl *coreLoop) forwardWriteRequest(request *writeRequest) { + dq := cl.queue // First we must decide which segment the new frame should be written to. data := request.frame.serialized segment := dq.segments.writing @@ -151,3 +176,66 @@ func (dq *diskQueue) forwardWriteRequest(request *writeRequest) { segment = &queueSegment{id: dq.segments.nextID} } } + +func (cl *coreLoop) handleProducerWriteRequest(request *writeRequest) { + if len(cl.blockedWrites) > 0 { + // If other requests are still waiting for space, then there + // definitely isn't enough for this one. + if request.shouldBlock { + cl.blockedWrites = append(cl.blockedWrites, request) + } else { + // If the request is non-blocking, send immediate failure and discard it. + request.responseChan <- false + } + return + } + // We will accept this request if there is enough capacity left in + // the queue (after accounting for the pending writes that were + // already accepted). + pendingBytes := uint64(0) + for _, request := range cl.pendingWrites { + pendingBytes += uint64(len(request.frame.serialized)) + } + currentSize := pendingBytes + cl.queue.segments.sizeOnDisk() + frameSize := uint64(len(request.frame.serialized)) + if currentSize+frameSize > cl.queue.settings.MaxBufferSize { + // The queue is too full + } +} + +func (cl *coreLoop) handleProducerCancelRequest(request *cancelRequest) { +} + +func (cl *coreLoop) handleConsumerAck(ackedUpTo frameID) { + acking := cl.queue.segments.acking + if len(acking) == 0 { + return + } + segmentsAcked := 0 + startFrame := cl.oldestFrameID + for ; segmentsAcked < len(acking); segmentsAcked++ { + segment := acking[segmentsAcked] + endFrame := startFrame + frameID(segment.framesRead) + if endFrame > ackedUpTo { + // This segment has not been fully read, we're done. + break + } + } + if segmentsAcked > 0 { + // Move fully acked segments to the acked list and remove them + // from the acking list. + cl.queue.segments.acked = + append(cl.queue.segments.acked, acking[:segmentsAcked]...) + cl.queue.segments.acking = acking[segmentsAcked:] + cl.maybeDeleteAcked() + } +} + +// If the acked list is nonempty, and there are no outstanding deletion +// requests, send one. +func (cl *coreLoop) maybeDeleteAcked() { + if !cl.deleting && len(cl.queue.segments.acked) > 0 { + cl.queue.deleterLoop.input <- &deleteRequest{segments: cl.queue.segments.acked} + cl.deleting = true + } +} diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index ea6d11ef354..9dbd648cb43 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -119,8 +119,16 @@ type diskQueue struct { // inChan chan []byte writeRequestChan chan *writeRequest - readRequestChan chan *readRequest cancelRequestChan chan *cancelRequest + readRequestChan chan *readRequest + + // When a consumer ack increments ackedUpTo, the consumer sends + // its new value to this channel. The core loop then decides whether to + // delete the containing segments. + // The value sent on the channel is redundant with the value of ackedUpTo, + // but we send it anyway so we don't have to worry about the core loop + // waiting on ackLock. + consumerAckChan chan frameID //outChan chan diskQueueOutput @@ -423,11 +431,11 @@ func (settings Settings) segmentPath(segmentID segmentID) string { // func (dq *diskQueue) Close() error { - closedForRead := dq.closedForRead.Swap(true) + /*closedForRead := dq.closedForRead.Swap(true) closedForWrite := dq.closedForWrite.Swap(true) if closedForRead && closedForWrite { return fmt.Errorf("Can't close disk queue: queue already closed") - } + }*/ // TODO: wait for worker threads? close(dq.done) return nil From 7774dc4c817bde94e6927ebfab6432b5948e00cf Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 13 Aug 2020 11:35:15 -0400 Subject: [PATCH 26/91] organize, delete dead code --- .../publisher/queue/diskqueue/core_loop.go | 231 ++++++++++-------- libbeat/publisher/queue/diskqueue/producer.go | 7 +- libbeat/publisher/queue/diskqueue/queue.go | 89 +------ .../publisher/queue/diskqueue/reader_loop.go | 20 +- libbeat/publisher/queue/diskqueue/segments.go | 69 +----- .../publisher/queue/diskqueue/serialize.go | 38 ++- .../publisher/queue/diskqueue/writer_loop.go | 9 +- 7 files changed, 186 insertions(+), 277 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index c13b96b0475..397246814b0 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -19,45 +19,6 @@ package diskqueue import "github.com/elastic/beats/v7/libbeat/publisher" -// A frame waiting to be written to disk -type writeFrame struct { - // The original event provided by the client to diskQueueProducer - event publisher.Event - - // The event, serialized for writing to disk and wrapped in a frame - // header / footer. - serialized []byte -} - -// A frame that has been read from disk -type readFrame struct { -} - -// A request sent from a producer to the core loop to add a frame to the queue. -// -type writeRequest struct { - frame *writeFrame - shouldBlock bool - responseChan chan bool -} - -// A readRequest is sent from the reader loop to the core loop when it -// needs a new segment file to read. -type readRequest struct { - responseChan chan *readResponse -} - -type readResponse struct { -} - -type cancelRequest struct { - producer *diskQueueProducer - // If producer.config.DropOnCancel is true, then the core loop will respond - // on responseChan with the number of dropped events. - // Otherwise, this field may be nil. - responseChan chan int -} - type coreLoop struct { // The queue that created this coreLoop. The core loop is the only one of // the main goroutines for the queue that has a pointer to the queue and @@ -94,12 +55,43 @@ type coreLoop struct { oldestFrameID frameID } +// A frame waiting to be written to disk +type writeFrame struct { + // The original event provided by the client to diskQueueProducer. + // We keep this as well as the serialized form until we are done + // writing, because we may need to send this value back to the producer + // callback if it is cancelled. + event publisher.Event + + // The event, serialized for writing to disk and wrapped in a frame + // header / footer. + serialized []byte +} + +// A frame that has been read from disk +type readFrame struct { +} + +// A request sent from a producer to the core loop to add a frame to the queue. +type writeRequest struct { + frame *writeFrame + shouldBlock bool + responseChan chan bool +} + +type cancelRequest struct { + producer *diskQueueProducer + // If producer.config.DropOnCancel is true, then the core loop will respond + // on responseChan with the number of dropped events. + // Otherwise, this field may be nil. + responseChan chan int +} + func (cl *coreLoop) run() { dq := cl.queue for { select { - /////////////////////////////////////// // Endpoints used by the public API case writeRequest := <-dq.writeRequestChan: cl.handleProducerWriteRequest(writeRequest) @@ -110,73 +102,28 @@ func (cl *coreLoop) run() { case ackedUpTo := <-dq.consumerAckChan: cl.handleConsumerAck(ackedUpTo) - /////////////////////////////////////// + case <-dq.done: + cl.handleShutdown() + return + // Writer loop handling case <-dq.writerLoop.finishedWriting: - if len(cl.pendingWrites) > 0 { - cl.forwardWriteRequest(cl.pendingWrites[0]) - cl.pendingWrites = cl.pendingWrites[1:] - } else { - cl.writing = false - } + // Reset the writing flag and check if there's another frame waiting + // to be written. + cl.writing = false + cl.maybeWritePending() - /////////////////////////////////////// // Reader loop handling case readResponse := <-dq.readerLoop.finishedReading: + cl.handleReadResponse(readResponse) - /////////////////////////////////////// // Deleter loop handling case deleteResponse := <-dq.deleterLoop.response: - if len(deleteResponse.deleted) > 0 { - // One or more segments were deleted, recompute the outstanding list. - newAckedSegments := []*queueSegment{} - for _, segment := range dq.segments.acked { - if !deleteResponse.deleted[segment] { - // This segment wasn't deleted, so it goes in the new list. - newAckedSegments = append(newAckedSegments, segment) - } - } - dq.segments.acked = newAckedSegments - } - if len(deleteResponse.errors) > 0 { - dq.settings.Logger.Errorw("Couldn't delete old segment files", - "errors", deleteResponse.errors) - } - - if len(dq.segments.acked) > 0 { - // There are still (possibly new) segments to delete, send the - // next batch. - dq.deleterLoop.input <- &deleteRequest{segments: dq.segments.acked} - } else { - // Nothing more to delete for now, update the deleting flag. - cl.deleting = false - } + cl.handleDeleteResponse(deleteResponse) } } } -func (cl *coreLoop) forwardWriteRequest(request *writeRequest) { - dq := cl.queue - // First we must decide which segment the new frame should be written to. - data := request.frame.serialized - segment := dq.segments.writing - - if segment != nil && - segment.size+uint64(len(data)) > dq.settings.MaxSegmentSize { - // The new frame is too big to fit in this segment, so close it and - // move it to the read queue. - segment.writer.Close() - // TODO: make reasonable attempts to sync the closed file. - dq.segments.reading = append(dq.segments.reading, segment) - segment = nil - } - - // If we don't have a segment, we need to create one. - if segment == nil { - segment = &queueSegment{id: dq.segments.nextID} - } -} - func (cl *coreLoop) handleProducerWriteRequest(request *writeRequest) { if len(cl.blockedWrites) > 0 { // If other requests are still waiting for space, then there @@ -199,13 +146,29 @@ func (cl *coreLoop) handleProducerWriteRequest(request *writeRequest) { currentSize := pendingBytes + cl.queue.segments.sizeOnDisk() frameSize := uint64(len(request.frame.serialized)) if currentSize+frameSize > cl.queue.settings.MaxBufferSize { - // The queue is too full + // The queue is too full. Either add the request to blockedWrites, + // or send an immediate reject. + if request.shouldBlock { + cl.blockedWrites = append(cl.blockedWrites, request) + } else { + request.responseChan <- false + } + } else { + // There is enough space for the new frame! Add it to the + // pending list and dispatch it to the writer loop if no other + // writes are outstanding. + cl.pendingWrites = append(cl.pendingWrites, request) + cl.maybeWritePending() } } func (cl *coreLoop) handleProducerCancelRequest(request *cancelRequest) { } +func (cl *coreLoop) handleReadResponse(response readResponse) { + +} + func (cl *coreLoop) handleConsumerAck(ackedUpTo frameID) { acking := cl.queue.segments.acking if len(acking) == 0 { @@ -231,6 +194,82 @@ func (cl *coreLoop) handleConsumerAck(ackedUpTo frameID) { } } +func (cl *coreLoop) handleDeleteResponse(response *deleteResponse) { + dq := cl.queue + cl.deleting = false + if len(response.deleted) > 0 { + // One or more segments were deleted, recompute the outstanding list. + newAckedSegments := []*queueSegment{} + for _, segment := range dq.segments.acked { + if !response.deleted[segment] { + // This segment wasn't deleted, so it goes in the new list. + newAckedSegments = append(newAckedSegments, segment) + } + } + dq.segments.acked = newAckedSegments + } + if len(response.errors) > 0 { + dq.settings.Logger.Errorw("Couldn't delete old segment files", + "errors", response.errors) + } + // If there are still files to delete, send the next request. + cl.maybeDeleteAcked() +} + +func (cl *coreLoop) handleShutdown() { + +} + +// If the pendingWrites list is nonempty, and there are no outstanding +// requests to the writer loop, send the next frame. +func (cl *coreLoop) maybeWritePending() { + dq := cl.queue + if cl.writing || len(cl.pendingWrites) == 0 { + // Nothing to do right now + return + } + // We are now definitely going to handle the next request, so + // remove it from pendingWrites. + request := cl.pendingWrites[0] + cl.pendingWrites = cl.pendingWrites[1:] + + // We have a frame to write, but we need to decide which segment + // it should go in. + segment := dq.segments.writing + + // If the new frame exceeds the maximum segment size, close the current + // writing segment. + frameLen := uint64(len(request.frame.serialized)) + if segment != nil && segment.size+frameLen > dq.settings.MaxSegmentSize { + segment.writer.Close() + segment.writer = nil + dq.segments.reading = append(dq.segments.reading, segment) + segment = nil + } + + // If there is no active writing segment need to create a new segment. + if segment == nil { + segment = &queueSegment{ + id: dq.segments.nextID, + queueSettings: &dq.settings, + } + dq.segments.writing = segment + dq.segments.nextID++ + } + + cl.queue.writerLoop.input <- &writeBlock{ + request: cl.pendingWrites[0], + segment: segment, + } + cl.writing = true +} + +// If the reading list is nonempty, and there are no outstanding read +// requests, send one. +func (cl *coreLoop) maybeReadPending() { + +} + // If the acked list is nonempty, and there are no outstanding deletion // requests, send one. func (cl *coreLoop) maybeDeleteAcked() { diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index 1c427939388..4b9e338b9f9 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -47,7 +47,12 @@ func (producer *diskQueueProducer) TryPublish(event publisher.Event) bool { func (producer *diskQueueProducer) publish( event publisher.Event, shouldBlock bool, ) bool { - serialized := dataFrameForEvent(event, producer.queue.settings.ChecksumType) + serialized, err := producer.encoder.encode(&event) + if err != nil { + producer.queue.settings.Logger.Errorf( + "Couldn't serialize incoming event: %v", err) + return false + } request := &writeRequest{ frame: &writeFrame{ event: event, diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 9dbd648cb43..d7881993b16 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -58,13 +58,6 @@ type Settings struct { // disk buffer. WriteToDiskACKListener queue.ACKListener - // A listener that receives ACKs when events are removed from the queue - // and written to their output. - // This can only be effective for events that are added to the queue - // after it is opened (there is no way to acknowledge input from a - // previous execution). It is ignored for events before that. - //WriteToOutputACKListener queue.ACKListener - ChecksumType ChecksumType } @@ -99,9 +92,6 @@ type diskQueue struct { // The persistent queue state (wraps diskQueuePersistentState on disk). stateFile *stateFile - // Data frames waiting to be written to disk. - //incoming *pendingFrameData - // Metadata related to the segment files. segments *diskQueueSegments @@ -109,18 +99,9 @@ type diskQueue struct { writerLoop *writerLoop deleterLoop *deleterLoop - // The total bytes occupied by all segment files. This is the value - // we check to see if there is enough space to add an incoming event - // to the queue. - //bytesOnDisk uint64 - - // The memory queue of data blobs waiting to be written to disk. - // To add something to the queue internally, send it to this channel. - // inChan chan []byte - + // The API channels used by diskQueueProducer to send write / cancel calls. writeRequestChan chan *writeRequest cancelRequestChan chan *cancelRequest - readRequestChan chan *readRequest // When a consumer ack increments ackedUpTo, the consumer sends // its new value to this channel. The core loop then decides whether to @@ -130,48 +111,6 @@ type diskQueue struct { // waiting on ackLock. consumerAckChan chan frameID - //outChan chan diskQueueOutput - - // The currently active segment reader, or nil if there is none. - //reader *segmentReader - - // The currently active segment writer. When the corresponding segment - // is full it is appended to segments. - //writer *segmentWriter - - // The ol - //firstPosition bufferPosition - - // The position of the next event to read from the queue. If this equals - // writePosition, then there are no events left to read. - // This is initialized to state.firstPosition, but generally the two differ: - // readPosition is advanced when an event is read, but firstPosition is - // only advanced when the event has been read _and_ its consumer receives - // an acknowledgement (meaning it has been transmitted and can be removed - // from the queue). - // This is part of diskQueue and not diskQueueState since it represents - // in-memory state that should not persist through a restart. - //readPosition bufferPosition - - // A condition that is signalled when a segment file is deleted. - // Used by writerLoop when the queue is full, to detect when to try again. - // When the queue is closed, this condition will receive a broadcast after - // diskQueue.closed is set to true. - //segmentDeletedCond sync.Cond - - // A condition that is signalled when a frame has been completely - // written to disk. - // Used by readerLoop when the queue is empty, to detect when to try again. - // When the queue is closed, this condition will receive a broadcast after - // diskQueue.closed is set to true. - //frameWrittenCond sync.Cond - - // The oldest frame id that is still stored on disk. - // This will usually be less than ackedUpTo, since oldestFrame can't - // advance until the entire segment file has been acknowledged and - // deleted. - //oldestFrame frameID - // This lock must be held to read and write acked and ackedUpTo. ackLock sync.Mutex @@ -182,24 +121,6 @@ type diskQueue struct { // can't yet be acknowledged as a continuous block). acked map[frameID]bool - // Whether the queue has been closed for write. When this is true, data that - // has already been added to the queue intake should continue being written - // to disk, but no new data should be accepted and the writerLoop routine - // should terminate when all data has been written. - //closedForWrite atomic.Bool - - // Whether the queue has been closed for read. When this is true, no more - // frames can be read from the queue, and the readerLoop routine should - // terminate. The queue will continue to accept acks for already-read - // frames and update the on-disk state accordingly. - //closedForRead atomic.Bool - - // If true, goroutines should return as soon as possible without waiting for - // data updates to complete. The queue will no longer write buffered frames - // to disk or update the on-disk state to reflect incoming acks. - // If abort is true, then closedForRead and closedForWrite must be true. - //abort atomic.Bool - // Wait group for shutdown of the goroutines associated with this queue: // core loop, reader loop, writer loop, deleter loop. waitGroup *sync.WaitGroup @@ -226,17 +147,9 @@ type pendingFrameData struct { // diskQueueSegments encapsulates segment-related queue metadata. type diskQueueSegments struct { - // The lock should be held to read or write any of the fields below. - sync.Mutex - // The segment that is currently being written. writing *queueSegment - //writer *segmentWriter - //reader *segmentReader - segmentDeletedCond *sync.Cond - frameWrittenCond *sync.Cond - // A list of the segments that have been completely written but have // not yet been completely processed by the reader loop, sorted by increasing // segment ID. Segments are always read in order. When a segment has diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 3736b0fa731..5e224c4a85c 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -21,7 +21,7 @@ import ( "io" ) -type finishedReadingMessage struct { +type readResponse struct { // The number of frames read from the last file the reader loop was given. frameCount int @@ -42,7 +42,7 @@ type readerLoop struct { // available for reading, the core loop will wait until it gets a // finishedReadingMessage before it nextReadBlock chan readBlock - finishedReading chan finishedReadingMessage + finishedReading chan readResponse // Frames that have been read from disk are sent to this channel. // Unlike most of the queue's API channels, this one is buffered to allow @@ -62,19 +62,19 @@ func (rl *readerLoop) run() { } } -func (rl *readerLoop) processBlock(block readBlock) finishedReadingMessage { +func (rl *readerLoop) processBlock(block readBlock) readResponse { frameCount := 0 for { frame, err := block.nextFrame() if err != nil { - return finishedReadingMessage{ + return readResponse{ frameCount: frameCount, err: err, } } if frame == nil { // There are no more frames in this block. - return finishedReadingMessage{ + return readResponse{ frameCount: frameCount, err: nil, } @@ -88,7 +88,7 @@ func (rl *readerLoop) processBlock(block readBlock) finishedReadingMessage { // Since we haven't sent a finishedReading message yet, we can only // reach this case when the nextReadBlock channel is closed, indicating // queue shutdown. In this case we immediately return. - return finishedReadingMessage{ + return readResponse{ frameCount: frameCount, err: nil, } @@ -99,7 +99,7 @@ func (rl *readerLoop) processBlock(block readBlock) finishedReadingMessage { // again separately before we move on to the next data frame. select { case <-rl.nextReadBlock: - return finishedReadingMessage{ + return readResponse{ frameCount: frameCount, err: nil, } @@ -229,6 +229,7 @@ func readSegment( return 0, nil } +/* func (dq *diskQueue) nextSegmentForReading() *queueSegment { dq.segments.Lock() defer dq.segments.Unlock() @@ -242,7 +243,7 @@ func (dq *diskQueue) nextSegmentForReading() *queueSegment { } func (dq *diskQueue) altReaderLoop() { - /*curFrameID := frameID(0) + curFrameID := frameID(0) for { segment := dq.nextSegmentForReading() if segment == nil { @@ -266,5 +267,6 @@ func (dq *diskQueue) altReaderLoop() { if err != nil { return nil, fmt.Errorf("Couldn't read segment header: %w", err) } - }*/ + } } +*/ diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 2cba3076180..d53cbb8189e 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -18,9 +18,7 @@ package diskqueue import ( - "encoding/binary" "fmt" - "hash/crc32" "io" "io/ioutil" "os" @@ -83,47 +81,7 @@ type segmentHeader struct { checksumType ChecksumType } -// segmentReader is a wrapper around io.Reader that provides helpers and -// metadata for decoding segment files. -/*type segmentReader struct { - // The segment this reader was generated from. - segment *queueSegment - - // The underlying data reader. - raw io.Reader - - // The current byte offset of the reader within the file. - curPosition segmentOffset - - // The position at which this reader should stop reading. This is often - // the end of the file, but it may be earlier when the queue is reading - // and writing to the same segment. - endPosition segmentOffset - - // The checksumType field from this segment file's header. - checksumType ChecksumType -}*/ - -/*type segmentWriter struct { - segment *queueSegment - file *os.File - position int64 -}*/ - -// ChecksumType specifies what checksum algorithm the queue should use to -// verify its data frames. -type ChecksumType int - -// ChecksumTypeNone: Don't compute or verify checksums. -// ChecksumTypeCRC32: Compute the checksum with the Go standard library's -// "hash/crc32" package. -const ( - ChecksumTypeNone = iota - - ChecksumTypeCRC32 -) - -// Each data frame has a 32-bit lengths and 1 32-bit checksum +// Each data frame has a 32-bit length and a 32-bit checksum // in the header, and a duplicate 32-bit length in the footer. const frameHeaderSize = 8 const frameFooterSize = 4 @@ -139,6 +97,8 @@ func (s bySegmentID) Len() int { return len(s) } func (s bySegmentID) Swap(i, j int) { s[i], s[j] = s[j], s[i] } func (s bySegmentID) Less(i, j int) bool { return s[i].size < s[j].size } +// Scan the given path for segment files, and return them in a list +// ordered by segment id. func scanExistingSegments(path string) ([]*queueSegment, error) { files, err := ioutil.ReadDir(path) if err != nil { @@ -322,26 +282,3 @@ func (dq *diskQueue) ack(frame frameID) int { } return ackedCount } - -func computeChecksum(data []byte, checksumType ChecksumType) uint32 { - switch checksumType { - case ChecksumTypeNone: - return 0 - case ChecksumTypeCRC32: - hash := crc32.NewIEEE() - frameLength := uint32(len(data) + frameMetadataSize) - binary.Write(hash, binary.LittleEndian, &frameLength) - hash.Write(data) - return hash.Sum32() - default: - panic("segmentReader: invalid checksum type") - } -} - -/* -func (dq *diskQueue) segmentReaderForPosition( - pos bufferPosition, -) (*segmentReader, error) { - panic("TODO: not implemented") -} -*/ diff --git a/libbeat/publisher/queue/diskqueue/serialize.go b/libbeat/publisher/queue/diskqueue/serialize.go index 1945aa98f4c..8e360b697f1 100644 --- a/libbeat/publisher/queue/diskqueue/serialize.go +++ b/libbeat/publisher/queue/diskqueue/serialize.go @@ -22,6 +22,8 @@ package diskqueue import ( "bytes" + "encoding/binary" + "hash/crc32" "time" "github.com/elastic/beats/v7/libbeat/beat" @@ -32,17 +34,18 @@ import ( "github.com/elastic/go-structform/json" ) -func dataFrameForEvent( - event publisher.Event, checksumType ChecksumType, -) []byte { - buf := bytes.Buffer{} - //checksum := computeChecksum(frameContent, checksumType) - panic("TODO: not implemented") -} +// ChecksumType specifies what checksum algorithm the queue should use to +// verify its data frames. +type ChecksumType int -func eventForData(data []byte) publisher.Event { - panic("TODO: not implemented") -} +// ChecksumTypeNone: Don't compute or verify checksums. +// ChecksumTypeCRC32: Compute the checksum with the Go standard library's +// "hash/crc32" package. +const ( + ChecksumTypeNone = iota + + ChecksumTypeCRC32 +) type frameEncoder struct { buf bytes.Buffer @@ -166,3 +169,18 @@ func (d *decoder) Decode() (publisher.Event, error) { }, }, nil } + +func computeChecksum(data []byte, checksumType ChecksumType) uint32 { + switch checksumType { + case ChecksumTypeNone: + return 0 + case ChecksumTypeCRC32: + hash := crc32.NewIEEE() + frameLength := uint32(len(data) + frameMetadataSize) + binary.Write(hash, binary.LittleEndian, &frameLength) + hash.Write(data) + return hash.Sum32() + default: + panic("segmentReader: invalid checksum type") + } +} diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 87c6dc00288..683bf02131f 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -46,12 +46,6 @@ type writerLoop struct { // ensure that the core loop doesn't block, the writer loop always reads // from input immediately after sending to finishedWriting. finishedWriting chan struct{} - - // The writer loop sends to nextWriteSegment to indicate that it needs a new - // segment file to write its frames to. The core loop must respond on - // nextSegmentResponse with the next segment file structure. - //nextSegmentRequest chan struct{} - //nextSegmentResponse chan *queueSegment } func (wl *writerLoop) run() { @@ -66,8 +60,9 @@ func (wl *writerLoop) run() { } } +// Write the block data to disk. func (wl *writerLoop) processRequest(block *writeBlock) { - writer, err := block.segment.getWriter() + //writer, err := block.segment.getWriter() } // frameForContent wraps the given content buffer in a From 7146525de922f79f7ba429aa45781d538d15824e Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 13 Aug 2020 12:01:36 -0400 Subject: [PATCH 27/91] comment / delete more old code --- .../publisher/queue/diskqueue/core_loop.go | 28 +++++ libbeat/publisher/queue/diskqueue/queue.go | 58 +--------- .../publisher/queue/diskqueue/reader_loop.go | 104 ------------------ .../publisher/queue/diskqueue/writer_loop.go | 10 +- 4 files changed, 36 insertions(+), 164 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 397246814b0..b0241c67fa1 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -217,7 +217,35 @@ func (cl *coreLoop) handleDeleteResponse(response *deleteResponse) { } func (cl *coreLoop) handleShutdown() { + // We need to close the input channels for all other goroutines and + // wait for any outstanding responses. Order is important: handling + // a read response may require the deleter, so the reader must be + // shut down first. + + close(cl.queue.readerLoop.nextReadBlock) + if cl.reading { + response := <-cl.queue.readerLoop.finishedReading + cl.handleReadResponse(response) + } + + close(cl.queue.writerLoop.input) + if cl.writing { + <-cl.queue.writerLoop.finishedWriting + cl.queue.segments.writing.writer.Close() + } + + close(cl.queue.deleterLoop.input) + if cl.deleting { + response := <-cl.queue.deleterLoop.response + // We can't retry any more if deletion failed, but we still check the + // response so we can log any errors. + if len(response.errors) > 0 { + cl.queue.settings.Logger.Errorw("Couldn't delete old segment files", + "errors", response.errors) + } + } + // TODO: wait (with timeout?) for any outstanding acks? } // If the pendingWrites list is nonempty, and there are no outstanding diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index d7881993b16..7a073c337b5 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -268,54 +268,6 @@ func NewQueue(settings Settings) (queue.Queue, error) { // written. return nil, errors }*/ -/* -func tryLoad(segment *queueSegment, path string) (*segmentReader, error) { - // this is a strangely fine-grained lock maybe? - segment.Lock() - defer segment.Unlock() - - // dataSize is guaranteed to be positive because we don't add - // anything to the segments list unless it is. - dataSize := segment.size - segmentHeaderSize - file, err := os.Open(path) - if err != nil { - return nil, fmt.Errorf( - "Couldn't open segment %d: %w", segment.id, err) - } - reader := bufio.NewReader(file) - header, err := readSegmentHeader(reader) - if err != nil { - return nil, fmt.Errorf("Couldn't read segment header: %w", err) - } - return &segmentReader{ - raw: reader, - curPosition: 0, - endPosition: segmentOffset(dataSize), - checksumType: header.checksumType, - }, nil -} -*/ - -// readNextFrame reads the next pending data frame in the queue -// and returns its contents. -/*func (dq *diskQueue) readNextFrame() ([]byte, error) { - // READER LOCK ---> - if dq.reader != nil { - frameData, err := dq.reader.nextDataFrame() - if err != nil { - return nil, err - } - if frameData != nil { - return frameData, nil - } - // If we made it here then the active reader was empty and - // we need to fetch a new one. - } - reader, _ := dq.nextSegmentReader() - dq.reader = reader - return reader.nextDataFrame() - // <--- READER LOCK -}*/ // // bookkeeping helpers to locate queue data on disk @@ -344,13 +296,11 @@ func (settings Settings) segmentPath(segmentID segmentID) string { // func (dq *diskQueue) Close() error { - /*closedForRead := dq.closedForRead.Swap(true) - closedForWrite := dq.closedForWrite.Swap(true) - if closedForRead && closedForWrite { - return fmt.Errorf("Can't close disk queue: queue already closed") - }*/ - // TODO: wait for worker threads? + // Closing the done channel signals to the core loop that it should + // shut down the other helper goroutines and wrap everything up. close(dq.done) + dq.waitGroup.Wait() + return nil } diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 5e224c4a85c..cb9ec1dcf3c 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -166,107 +166,3 @@ func (block *readBlock) nextFrame() (*readFrame, error) { reader.segment.framesRead += framesRead } }*/ - -/*func (dq *diskQueue) newSegmentWriter() (*segmentWriter, error) { - var err error - dq.segments.Lock() - defer dq.segments.Unlock() - - id := dq.segments.nextID - defer func() { - // If we were successful, update nextID - if err == nil { - dq.segments.nextID++ - } - }() - - segment := &queueSegment{queueSettings: &dq.settings, id: id} - - path := dq.settings.segmentPath(id) - file, err := os.OpenFile(path, os.O_CREATE|os.O_TRUNC, 0600) - if err != nil { - return nil, err - } - - return &segmentWriter{ - segment: segment, - file: file, - }, nil -}*/ - -// This is only called from the writer loop. -func (dq *diskQueue) writeFrameData(bytes []byte) error { - /*frameSize := uint64(len(bytes) + frameMetadataSize) - - dq.segments.Lock() - defer dq.segments.Unlock() - if dq.segments.writer != nil && - dq.segments.writer.position+frameSize > dq.settings.MaxSegmentSize { - // There is a writing segment, but the incoming data frame doesn't - // fit, so we need to finalize it and create a new one. - //dq.segments.writer = - //dq.segments.writing - } - - - // while (free bytes) < frameSize { - // block - // } - - if dq.segments.writing == nil { - // There is no current output segment, create a new one. - - }*/ - - return nil -} - -func readSegment( - reader io.Reader, checksumType ChecksumType, - start segmentOffset, stop segmentOffset, - output chan diskQueueOutput, cancel chan struct{}, -) (int, error) { - return 0, nil -} - -/* -func (dq *diskQueue) nextSegmentForReading() *queueSegment { - dq.segments.Lock() - defer dq.segments.Unlock() - if len(dq.segments.reading) > 0 { - return dq.segments.reading[0] - } - if dq.segments.writing != nil { - return dq.segments.writing - } - return nil -} - -func (dq *diskQueue) altReaderLoop() { - curFrameID := frameID(0) - for { - segment := dq.nextSegmentForReading() - if segment == nil { - // TODO: wait - continue - } - // this is a strangely fine-grained lock maybe? - segment.Lock() - defer segment.Unlock() - - // dataSize is guaranteed to be positive because we don't add - // anything to the segments list unless it is. - dataSize := segment.size - segmentHeaderSize - file, err := os.Open(path) - if err != nil { - return nil, fmt.Errorf( - "Couldn't open segment %d: %w", segment.id, err) - } - reader := bufio.NewReader(file) - header, err := readSegmentHeader(reader) - if err != nil { - return nil, fmt.Errorf("Couldn't read segment header: %w", err) - } - } -} -*/ diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 683bf02131f..786323d388c 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -19,9 +19,6 @@ package diskqueue import ( "bytes" - "io" - "os" - "syscall" "github.com/elastic/beats/v7/libbeat/logp" ) @@ -80,12 +77,12 @@ func frameForContent( return buf } -type writerState struct { +/*type writerState struct { // The open file handle for the segment currently being written. // This should be non-nil if and only if diskQueue.segments.writing is. file *os.File filePosition int64 -} +}*/ /*func handleFrame(dq *diskQueue, state *writerState, frame bytes.Buffer) { dq.segments.Lock() @@ -156,7 +153,7 @@ type writerState struct { // We've processed } }*/ - +/* func writeAll(writer io.Writer, p []byte) (int, error) { var N int for len(p) > 0 { @@ -172,3 +169,4 @@ func writeAll(writer io.Writer, p []byte) (int, error) { func isRetryErr(err error) bool { return err == syscall.EINTR || err == syscall.EAGAIN } +*/ From 1001565b9fc2c4d28b2147e50ddd5b3c5bb16e75 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 13 Aug 2020 12:33:47 -0400 Subject: [PATCH 28/91] cleanup, plug in consumer acks --- libbeat/publisher/queue/diskqueue/consumer.go | 66 +++++++++++++++---- .../publisher/queue/diskqueue/core_loop.go | 19 ++---- libbeat/publisher/queue/diskqueue/producer.go | 16 +++++ libbeat/publisher/queue/diskqueue/queue.go | 61 ++++++++--------- .../publisher/queue/diskqueue/reader_loop.go | 1 + libbeat/publisher/queue/diskqueue/segments.go | 13 ---- 6 files changed, 107 insertions(+), 69 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/consumer.go b/libbeat/publisher/queue/diskqueue/consumer.go index f27c6eaea57..46df08551b7 100644 --- a/libbeat/publisher/queue/diskqueue/consumer.go +++ b/libbeat/publisher/queue/diskqueue/consumer.go @@ -30,7 +30,9 @@ type diskQueueConsumer struct { } type diskQueueBatch struct { + queue *diskQueue events []publisher.Event + ackIDs []frameID } // @@ -41,19 +43,42 @@ func (consumer *diskQueueConsumer) Get(eventCount int) (queue.Batch, error) { if consumer.closed { return nil, fmt.Errorf("Tried to read from closed disk queue consumer") } - // lock mutex ---> - /*position := consumer.queue.readPosition - segment, err := consumer.queue.getSegment(position.segment) - if err != nil { + // Read at least one frame. This is guaranteed to eventually + // succeed unless the queue is closed. + frame, ok := <-consumer.queue.readerLoop.output + if !ok { + return nil, fmt.Errorf("Tried to read from a closed disk queue") + } + frames := []*readFrame{frame} +eventLoop: + for eventCount <= 0 || len(frames) < eventCount { + select { + case frame, ok = <-consumer.queue.readerLoop.output: + if !ok { + // The queue was closed while we were reading it, just send back + // what we have so far. + break eventLoop + } + frames = append(frames, frame) + default: + // We can't read any more frames without blocking, so send back + // what we have now. + break eventLoop + } + } + var ackIDs []frameID + var events []publisher.Event + for _, frame := range frames { + events = append(events, frame.event) + ackIDs = append(ackIDs, frame.id) } - // <-- lock mutex - //reader, err := consumer.queue.segments.reader() - if err != nil { - return nil, fmt.Errorf("Couldn't read from queue: %w", err) - }*/ - panic("TODO: not implemented") + return &diskQueueBatch{ + queue: consumer.queue, + events: events, + ackIDs: ackIDs, + }, nil } func (consumer *diskQueueConsumer) Close() error { @@ -69,6 +94,25 @@ func (batch *diskQueueBatch) Events() []publisher.Event { return batch.events } +// This is the only place that the queue state is changed from +// outside the core loop. This is because ACKs are messy and bursty +// and we don't want the core loop to bottleneck on manipulating +// a potentially large dictionary, so we use a lock and let +// consumer threads handle most of the processing themselves. func (batch *diskQueueBatch) ACK() { - panic("TODO: not implemented") + dq := batch.queue + dq.ackLock.Lock() + defer dq.ackLock.Unlock() + for _, frameID := range batch.ackIDs { + dq.acked[frameID] = true + } + ackedCount := 0 + for ; dq.acked[dq.ackedUpTo]; dq.ackedUpTo++ { + delete(dq.acked, dq.ackedUpTo) + ackedCount++ + } + if ackedCount > 0 { + dq.ackedUpTo += frameID(ackedCount) + dq.consumerAckChan <- dq.ackedUpTo + } } diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index b0241c67fa1..c48fa628781 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -70,21 +70,8 @@ type writeFrame struct { // A frame that has been read from disk type readFrame struct { -} - -// A request sent from a producer to the core loop to add a frame to the queue. -type writeRequest struct { - frame *writeFrame - shouldBlock bool - responseChan chan bool -} - -type cancelRequest struct { - producer *diskQueueProducer - // If producer.config.DropOnCancel is true, then the core loop will respond - // on responseChan with the number of dropped events. - // Otherwise, this field may be nil. - responseChan chan int + event publisher.Event + id frameID } func (cl *coreLoop) run() { @@ -246,6 +233,8 @@ func (cl *coreLoop) handleShutdown() { } // TODO: wait (with timeout?) for any outstanding acks? + + // TODO: write final queue state to the metadata file. } // If the pendingWrites list is nonempty, and there are no outstanding diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index 4b9e338b9f9..d1cc0fbc8a0 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -32,6 +32,22 @@ type diskQueueProducer struct { encoder frameEncoder } +// A request sent from a producer to the core loop to add a frame to the queue. +type writeRequest struct { + frame *writeFrame + shouldBlock bool + responseChan chan bool +} + +// A request to the core loop to cancel the specified producer. +type cancelRequest struct { + producer *diskQueueProducer + // If producer.config.DropOnCancel is true, then the core loop will respond + // on responseChan with the number of dropped events. + // Otherwise, this field may be nil. + responseChan chan int +} + // // diskQueueProducer implementation of the queue.Producer interface // diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 7a073c337b5..3f4d7201fc2 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -95,6 +95,7 @@ type diskQueue struct { // Metadata related to the segment files. segments *diskQueueSegments + coreLoop *coreLoop readerLoop *readerLoop writerLoop *writerLoop deleterLoop *deleterLoop @@ -239,36 +240,6 @@ func NewQueue(settings Settings) (queue.Queue, error) { }, nil } -// This is only called by readerLoop. -/*func (dq *diskQueue) nextSegmentReader() (*segmentReader, []error) { - dq.segments.Lock() - defer dq.segments.Unlock() - - errors := []error{} - for len(dq.segments.reading) > 0 { - segment := dq.segments.reading[0] - segmentPath := dq.settings.segmentPath(segment.id) - reader, err := tryLoad(segment, segmentPath) - if err != nil { - // TODO: Handle this: depending on the type of error, either delete - // the segment or log an error and leave it alone, then skip to the - // next one. - errors = append(errors, err) - dq.segments.reading = dq.segments.reading[1:] - continue - } - // Remove the segment from the active list and move it to - // completedSegments until all its data has been acknowledged. - dq.segments.reading = dq.segments.reading[1:] - dq.segments.acking = append(dq.segments.acking, segment) - return reader, errors - } - // TODO: if segments.reading is empty we may still be able to - // read partial data from segments.writing which is still being - // written. - return nil, errors -}*/ - // // bookkeeping helpers to locate queue data on disk // @@ -318,3 +289,33 @@ func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { func (dq *diskQueue) Consumer() queue.Consumer { panic("TODO: not implemented") } + +// This is only called by readerLoop. +/*func (dq *diskQueue) nextSegmentReader() (*segmentReader, []error) { + dq.segments.Lock() + defer dq.segments.Unlock() + + errors := []error{} + for len(dq.segments.reading) > 0 { + segment := dq.segments.reading[0] + segmentPath := dq.settings.segmentPath(segment.id) + reader, err := tryLoad(segment, segmentPath) + if err != nil { + // TODO: Handle this: depending on the type of error, either delete + // the segment or log an error and leave it alone, then skip to the + // next one. + errors = append(errors, err) + dq.segments.reading = dq.segments.reading[1:] + continue + } + // Remove the segment from the active list and move it to + // completedSegments until all its data has been acknowledged. + dq.segments.reading = dq.segments.reading[1:] + dq.segments.acking = append(dq.segments.acking, segment) + return reader, errors + } + // TODO: if segments.reading is empty we may still be able to + // read partial data from segments.writing which is still being + // written. + return nil, errors +}*/ diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index cb9ec1dcf3c..f9911c1b30c 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -56,6 +56,7 @@ func (rl *readerLoop) run() { block, ok := <-rl.nextReadBlock if !ok { // The channel has closed, we are shutting down. + close(rl.output) return } rl.finishedReading <- rl.processBlock(block) diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index d53cbb8189e..04badccbaa3 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -269,16 +269,3 @@ func (segments *diskQueueSegments) sizeOnDisk() uint64 { reader.curPosition += segmentOffset(frameLength) return data, nil }*/ - -// returns the number of indices by which ackedUpTo was advanced. -func (dq *diskQueue) ack(frame frameID) int { - dq.ackLock.Lock() - defer dq.ackLock.Unlock() - dq.acked[frame] = true - ackedCount := 0 - for ; dq.acked[dq.ackedUpTo]; dq.ackedUpTo++ { - delete(dq.acked, dq.ackedUpTo) - ackedCount++ - } - return ackedCount -} From bb56b8f1d8fe7f280208db7700636b049a2b725d Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Fri, 21 Aug 2020 09:14:46 -0400 Subject: [PATCH 29/91] finish reader loop api / logic --- .../publisher/queue/diskqueue/core_loop.go | 124 +++++++++++++----- libbeat/publisher/queue/diskqueue/queue.go | 6 +- .../publisher/queue/diskqueue/reader_loop.go | 100 ++++++++------ libbeat/publisher/queue/diskqueue/segments.go | 38 +++--- 4 files changed, 178 insertions(+), 90 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index c48fa628781..5998f93f00d 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -41,6 +41,13 @@ type coreLoop struct { // request, false otherwise. deleting bool + // nextReadOffset is the segment offset to start reading at during + // the next read request. This offset always refers to the first read + // segment: either segments.reading[0], if that list is nonempty, or + // segments.writing (if all segments have been read except the one + // currently being written). + nextReadOffset segmentOffset + // pendingWrites is a list of all write requests that have been accepted // by the queue and are waiting to be written to disk. pendingWrites []*writeRequest @@ -77,6 +84,9 @@ type readFrame struct { func (cl *coreLoop) run() { dq := cl.queue + // Wake up the reader loop if there are segments available to read. + cl.maybeReadPending() + for { select { // Endpoints used by the public API @@ -101,7 +111,7 @@ func (cl *coreLoop) run() { cl.maybeWritePending() // Reader loop handling - case readResponse := <-dq.readerLoop.finishedReading: + case readResponse := <-dq.readerLoop.responseChan: cl.handleReadResponse(readResponse) // Deleter loop handling @@ -144,6 +154,11 @@ func (cl *coreLoop) handleProducerWriteRequest(request *writeRequest) { // There is enough space for the new frame! Add it to the // pending list and dispatch it to the writer loop if no other // writes are outstanding. + // Right now we accept any request if there is enough space for it + // on disk. High-throughput inputs may produce events faster than + // they can be written to disk, so it would make sense to + // additionally bound the amount of data in pendingWrites to some + // configurable limit to avoid out-of-memory errors. cl.pendingWrites = append(cl.pendingWrites, request) cl.maybeWritePending() } @@ -152,33 +167,39 @@ func (cl *coreLoop) handleProducerWriteRequest(request *writeRequest) { func (cl *coreLoop) handleProducerCancelRequest(request *cancelRequest) { } -func (cl *coreLoop) handleReadResponse(response readResponse) { - +// Returns the active read segment, or nil if there is none. +func (segments *diskQueueSegments) readingSegment() *queueSegment { + if len(segments.reading) > 0 { + return segments.reading[0] + } + return segments.writing } -func (cl *coreLoop) handleConsumerAck(ackedUpTo frameID) { - acking := cl.queue.segments.acking - if len(acking) == 0 { - return - } - segmentsAcked := 0 - startFrame := cl.oldestFrameID - for ; segmentsAcked < len(acking); segmentsAcked++ { - segment := acking[segmentsAcked] - endFrame := startFrame + frameID(segment.framesRead) - if endFrame > ackedUpTo { - // This segment has not been fully read, we're done. - break - } - } - if segmentsAcked > 0 { - // Move fully acked segments to the acked list and remove them - // from the acking list. - cl.queue.segments.acked = - append(cl.queue.segments.acked, acking[:segmentsAcked]...) - cl.queue.segments.acking = acking[segmentsAcked:] - cl.maybeDeleteAcked() +func (cl *coreLoop) handleReadResponse(response readResponse) { + cl.reading = false + segments := cl.queue.segments + segment := segments.readingSegment() + segment.framesRead += response.frameCount + newOffset := cl.nextReadOffset + segmentOffset(response.byteCount) + + // A (non-writing) segment is finished if we have read all the data, or + // the read response reports an error. + segmentFinished := + segment != segments.writing && + (newOffset >= segment.endOffset || response.err != nil) + if segmentFinished { + // Move to the acking list and reset the read offset. + segments.reading = segments.reading[1:] + segments.acking = append(segments.acking, segment) + cl.nextReadOffset = 0 + } else { + // We aren't done reading this segment; next time we'll start from + // the end of this read response. + cl.nextReadOffset = newOffset } + + // If there is more data to read, start a new read request. + cl.maybeReadPending() } func (cl *coreLoop) handleDeleteResponse(response *deleteResponse) { @@ -203,15 +224,42 @@ func (cl *coreLoop) handleDeleteResponse(response *deleteResponse) { cl.maybeDeleteAcked() } +func (cl *coreLoop) handleConsumerAck(ackedUpTo frameID) { + acking := cl.queue.segments.acking + if len(acking) == 0 { + return + } + startFrame := cl.oldestFrameID + endFrame := startFrame + ackedSegmentCount := 0 + for ; ackedSegmentCount < len(acking); ackedSegmentCount++ { + segment := acking[ackedSegmentCount] + endFrame += frameID(segment.framesRead) + if endFrame > ackedUpTo { + // This segment is still waiting for acks, we're done. + break + } + } + if ackedSegmentCount > 0 { + // Move fully acked segments to the acked list and remove them + // from the acking list. + cl.queue.segments.acked = + append(cl.queue.segments.acked, acking[:ackedSegmentCount]...) + cl.queue.segments.acking = acking[ackedSegmentCount:] + cl.oldestFrameID = endFrame + cl.maybeDeleteAcked() + } +} + func (cl *coreLoop) handleShutdown() { // We need to close the input channels for all other goroutines and // wait for any outstanding responses. Order is important: handling // a read response may require the deleter, so the reader must be // shut down first. - close(cl.queue.readerLoop.nextReadBlock) + close(cl.queue.readerLoop.requestChan) if cl.reading { - response := <-cl.queue.readerLoop.finishedReading + response := <-cl.queue.readerLoop.responseChan cl.handleReadResponse(response) } @@ -256,8 +304,9 @@ func (cl *coreLoop) maybeWritePending() { // If the new frame exceeds the maximum segment size, close the current // writing segment. - frameLen := uint64(len(request.frame.serialized)) - if segment != nil && segment.size+frameLen > dq.settings.MaxSegmentSize { + frameLen := segmentOffset(len(request.frame.serialized)) + newEndOffset := segment.endOffset + frameLen + if segment != nil && newEndOffset > dq.settings.maxSegmentOffset() { segment.writer.Close() segment.writer = nil dq.segments.reading = append(dq.segments.reading, segment) @@ -284,7 +333,22 @@ func (cl *coreLoop) maybeWritePending() { // If the reading list is nonempty, and there are no outstanding read // requests, send one. func (cl *coreLoop) maybeReadPending() { - + if cl.reading { + // A read request is already pending + return + } + segment := cl.queue.segments.readingSegment() + if segment == nil || cl.nextReadOffset >= segmentOffset(segment.endOffset) { + // Nothing to read + return + } + request := readRequest{ + segment: segment, + startOffset: cl.nextReadOffset, + endOffset: segment.endOffset, + } + cl.queue.readerLoop.requestChan <- request + cl.reading = true } // If the acked list is nonempty, and there are no outstanding deletion diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 3f4d7201fc2..9bab59b6c8c 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -241,7 +241,7 @@ func NewQueue(settings Settings) (queue.Queue, error) { } // -// bookkeeping helpers to locate queue data on disk +// bookkeeping helpers // func (settings Settings) directoryPath() string { @@ -262,6 +262,10 @@ func (settings Settings) segmentPath(segmentID segmentID) string { fmt.Sprintf("%v.seg", segmentID)) } +func (settings Settings) maxSegmentOffset() segmentOffset { + return segmentOffset(settings.MaxSegmentSize - segmentHeaderSize) +} + // // diskQueue implementation of the queue.Queue interface // diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index f9911c1b30c..565b45e4fb7 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -17,32 +17,34 @@ package diskqueue -import ( - "io" -) +import "os" + +type readRequest struct { + segment *queueSegment + startOffset segmentOffset + endOffset segmentOffset +} type readResponse struct { - // The number of frames read from the last file the reader loop was given. - frameCount int + // The number of frames successfully read from the requested segment file. + frameCount int64 + + // The number of bytes successfully read from the requested segment file. + byteCount int64 // If there was an error in the segment file (i.e. inconsistent data), the // err field is set. err error } -type readBlock struct { - reader io.Reader - length uint64 -} - type readerLoop struct { // When there is a block available for reading, it will be sent to - // nextReadBlock. When the reader loop has finished processing it, it + // requestChan. When the reader loop has finished processing it, it // sends the result to finishedReading. If there is more than one block // available for reading, the core loop will wait until it gets a // finishedReadingMessage before it - nextReadBlock chan readBlock - finishedReading chan readResponse + requestChan chan readRequest + responseChan chan readResponse // Frames that have been read from disk are sent to this channel. // Unlike most of the queue's API channels, this one is buffered to allow @@ -53,45 +55,61 @@ type readerLoop struct { func (rl *readerLoop) run() { for { - block, ok := <-rl.nextReadBlock + request, ok := <-rl.requestChan if !ok { // The channel has closed, we are shutting down. close(rl.output) return } - rl.finishedReading <- rl.processBlock(block) + rl.responseChan <- rl.processRequest(request) } } -func (rl *readerLoop) processBlock(block readBlock) readResponse { - frameCount := 0 +func (rl *readerLoop) processRequest(request readRequest) readResponse { + frameCount := int64(0) + byteCount := int64(0) + + // Open the file and seek to the starting position. + handle, err := request.segment.getReader() + if err != nil { + return readResponse{err: err} + } + _, err = handle.Seek(segmentHeaderSize+int64(request.startOffset), 0) + if err != nil { + return readResponse{err: err} + } + + targetLength := int64(request.endOffset - request.startOffset) for { - frame, err := block.nextFrame() - if err != nil { - return readResponse{ - frameCount: frameCount, - err: err, - } - } - if frame == nil { - // There are no more frames in this block. - return readResponse{ - frameCount: frameCount, - err: nil, + frame, err := nextFrame(handle) + if frame != nil { + // We've read the frame, try sending it to the output channel. + select { + case rl.output <- frame: + // Success! Increment the total for this request. + frameCount++ + byteCount += frame.bytesOnDisk + case <-rl.requestChan: + // Since we haven't sent a finishedReading message yet, we can only + // reach this case when the nextReadBlock channel is closed, indicating + // queue shutdown. In this case we immediately return. + return readResponse{ + frameCount: frameCount, + byteCount: byteCount, + err: nil, + } } } - // We've read the frame, try sending it to the output channel. - select { - case rl.output <- frame: - // Success! Increment the total for this block. - frameCount++ - case <-rl.nextReadBlock: - // Since we haven't sent a finishedReading message yet, we can only - // reach this case when the nextReadBlock channel is closed, indicating - // queue shutdown. In this case we immediately return. + + // We are done with this request if: + // - there was an error reading the frame, + // - there are no more frames to read, or + // - we have reached the end of the requested region + if err != nil || frame == nil || byteCount >= targetLength { return readResponse{ frameCount: frameCount, - err: nil, + byteCount: byteCount, + err: err, } } @@ -99,7 +117,7 @@ func (rl *readerLoop) processBlock(block readBlock) readResponse { // might not recognize when the queue is being closed, so check that // again separately before we move on to the next data frame. select { - case <-rl.nextReadBlock: + case <-rl.requestChan: return readResponse{ frameCount: frameCount, err: nil, @@ -109,7 +127,7 @@ func (rl *readerLoop) processBlock(block readBlock) readResponse { } } -func (block *readBlock) nextFrame() (*readFrame, error) { +func nextFrame(handle *os.File) (*readFrame, error) { return nil, nil } diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 04badccbaa3..d07d8f20f6c 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -35,6 +35,7 @@ import ( type frameID uint64 // segmentOffset is a byte index into the segment's data region. +// An offset of 0 means the first byte after the segment file header. type segmentOffset uint64 // The metadata for a single segment file. @@ -51,10 +52,11 @@ type queueSegment struct { // positioned at the start of the data region. created bool - // The size in bytes of the segment file on disk. This is updated when - // the segment is written to, and should always correspond to the end of - // a complete data frame. - size uint64 + // The byte offset of the end of the segment's data region. This is + // updated when the segment is written to, and should always correspond + // to the end of a complete data frame. The total size of a segment file + // on disk is segmentHeaderSize + segment.endOffset. + endOffset segmentOffset // The header metadata for this segment file. This field is nil if the // segment has not yet been opened for reading. It should only be @@ -95,7 +97,7 @@ type bySegmentID []*queueSegment func (s bySegmentID) Len() int { return len(s) } func (s bySegmentID) Swap(i, j int) { s[i], s[j] = s[j], s[i] } -func (s bySegmentID) Less(i, j int) bool { return s[i].size < s[j].size } +func (s bySegmentID) Less(i, j int) bool { return s[i].endOffset < s[j].endOffset } // Scan the given path for segment files, and return them in a list // ordered by segment id. @@ -120,9 +122,9 @@ func scanExistingSegments(path string) ([]*queueSegment, error) { if id, err := strconv.ParseUint(components[0], 10, 64); err == nil { segments = append(segments, &queueSegment{ - id: segmentID(id), - created: true, - size: uint64(file.Size()), + id: segmentID(id), + created: true, + endOffset: segmentOffset(file.Size() - segmentHeaderSize), }) } } @@ -131,18 +133,18 @@ func scanExistingSegments(path string) ([]*queueSegment, error) { return segments, nil } +func (segment *queueSegment) sizeOnDisk() uint64 { + return uint64(segment.endOffset) + segmentHeaderSize +} + // Should only be called from the reader loop. -func (segment *queueSegment) getReader() (io.ReadCloser, error) { - if segment.reader != nil { - return segment.reader, nil - } +func (segment *queueSegment) getReader() (*os.File, error) { path := segment.queueSettings.segmentPath(segment.id) file, err := os.Open(path) if err != nil { return nil, fmt.Errorf( "Couldn't open segment %d: %w", segment.id, err) } - //reader := bufio.NewReader(file) header, err := readSegmentHeader(file) if err != nil { return nil, fmt.Errorf("Couldn't read segment header: %w", err) @@ -172,7 +174,7 @@ func (segment *queueSegment) getWriter() (io.WriteCloser, error) { return file, nil } -func readSegmentHeader(in io.Reader) (*segmentHeader, error) { +func readSegmentHeader(in *os.File) (*segmentHeader, error) { header := segmentHeader{} if header.version != 0 { return nil, fmt.Errorf("Unrecognized schema version %d", header.version) @@ -190,16 +192,16 @@ func writeSegmentHeader(out io.Writer, header *segmentHeader) error { func (segments *diskQueueSegments) sizeOnDisk() uint64 { total := uint64(0) if segments.writing != nil { - total += segments.writing.size + total += segments.writing.sizeOnDisk() } for _, segment := range segments.reading { - total += segment.size + total += segment.sizeOnDisk() } for _, segment := range segments.acking { - total += segment.size + total += segment.sizeOnDisk() } for _, segment := range segments.acked { - total += segment.size + total += segment.sizeOnDisk() } return total } From a99e869cb88147e0e326259fa9510a9c3519d0a4 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 3 Sep 2020 13:03:00 -0400 Subject: [PATCH 30/91] make things build --- libbeat/publisher/queue/diskqueue/core_loop.go | 4 ++++ libbeat/publisher/queue/diskqueue/producer.go | 1 + libbeat/publisher/queue/diskqueue/reader_loop.go | 16 ++++++++++++++-- libbeat/publisher/queue/diskqueue/segments.go | 2 +- 4 files changed, 20 insertions(+), 3 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 5998f93f00d..6de399028e8 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -79,6 +79,10 @@ type writeFrame struct { type readFrame struct { event publisher.Event id frameID + + // How much space this frame occupied on disk (before deserialization), + // including the frame header / footer. + bytesOnDisk int64 } func (cl *coreLoop) run() { diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index d1cc0fbc8a0..ebd3a2ab3ef 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -83,6 +83,7 @@ func (producer *diskQueueProducer) publish( // The request has been sent, and we are now guaranteed to get a result on // the response channel, so we must read from it immediately to avoid // blocking the core loop. + // TODO: this should be unblocked by a call to Cancel return <-request.responseChan case <-producer.queue.done: return false diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 565b45e4fb7..fb7e74052c4 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -81,7 +81,19 @@ func (rl *readerLoop) processRequest(request readRequest) readResponse { targetLength := int64(request.endOffset - request.startOffset) for { - frame, err := nextFrame(handle) + remainingLength := targetLength - byteCount + /*if byteCount+frame.bytesOnDisk > targetLength { + // Something is wrong, read requests must end on a segment boundary. + return readResponse{ + frameCount: frameCount, + byteCount: byteCount, + } + }*/ + + // Try to read the next, clipping to the length we were told to read. + // If the next frame extends past this boundary, nextFrame will return + // an error. + frame, err := nextFrame(handle, remainingLength) if frame != nil { // We've read the frame, try sending it to the output channel. select { @@ -127,7 +139,7 @@ func (rl *readerLoop) processRequest(request readRequest) readResponse { } } -func nextFrame(handle *os.File) (*readFrame, error) { +func nextFrame(handle *os.File, maxLength int64) (*readFrame, error) { return nil, nil } diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index d07d8f20f6c..f691f975f39 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -174,7 +174,7 @@ func (segment *queueSegment) getWriter() (io.WriteCloser, error) { return file, nil } -func readSegmentHeader(in *os.File) (*segmentHeader, error) { +func readSegmentHeader(in io.Reader) (*segmentHeader, error) { header := segmentHeader{} if header.version != 0 { return nil, fmt.Errorf("Unrecognized schema version %d", header.version) From 89da2b24e5c0b7c140a64d0d27f30cb6f127b8cc Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 3 Sep 2020 17:06:45 -0400 Subject: [PATCH 31/91] clean up naming, plug in queue initialization, start fixing writer loop --- .../publisher/queue/diskqueue/core_loop.go | 36 ++++++--- libbeat/publisher/queue/diskqueue/producer.go | 11 +-- libbeat/publisher/queue/diskqueue/queue.go | 81 +++++++++++++++++-- .../publisher/queue/diskqueue/serialize.go | 7 +- .../publisher/queue/diskqueue/writer_loop.go | 23 ++++-- 5 files changed, 123 insertions(+), 35 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 6de399028e8..aee6e2367c8 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -17,7 +17,9 @@ package diskqueue -import "github.com/elastic/beats/v7/libbeat/publisher" +import ( + "github.com/elastic/beats/v7/libbeat/publisher" +) type coreLoop struct { // The queue that created this coreLoop. The core loop is the only one of @@ -49,12 +51,12 @@ type coreLoop struct { nextReadOffset segmentOffset // pendingWrites is a list of all write requests that have been accepted - // by the queue and are waiting to be written to disk. - pendingWrites []*writeRequest + // by the queue and are waiting to be sent to the writer loop. + pendingWrites []*producerWriteRequest // blockedWrites is a list of all write requests that are waiting for // free space in the queue. - blockedWrites []*writeRequest + blockedWrites []*producerWriteRequest // This value represents the oldest frame ID for a segment that has not // yet been moved to the acked list. It is used to detect when the oldest @@ -73,6 +75,16 @@ type writeFrame struct { // The event, serialized for writing to disk and wrapped in a frame // header / footer. serialized []byte + + // The producer that created this frame. This is included in the + // frame structure itself because we may need the producer and / or + // its config at any time up until it has been completely written: + // - While the core loop is tracking frames to send to the writer, + // it may receive a Cancel request, which requires us to know + // the producer / config each frame came from. + // - After the writer loop has finished writing the frame to disk, + // it needs to call the ACK function specified in ProducerConfig. + producer *diskQueueProducer } // A frame that has been read from disk @@ -94,10 +106,10 @@ func (cl *coreLoop) run() { for { select { // Endpoints used by the public API - case writeRequest := <-dq.writeRequestChan: - cl.handleProducerWriteRequest(writeRequest) + case producerWriteRequest := <-dq.producerWriteRequestChan: + cl.handleProducerWriteRequest(producerWriteRequest) - case cancelRequest := <-dq.cancelRequestChan: + case cancelRequest := <-dq.producerCancelRequestChan: cl.handleProducerCancelRequest(cancelRequest) case ackedUpTo := <-dq.consumerAckChan: @@ -108,7 +120,7 @@ func (cl *coreLoop) run() { return // Writer loop handling - case <-dq.writerLoop.finishedWriting: + case <-dq.writerLoop.writeResponse: // Reset the writing flag and check if there's another frame waiting // to be written. cl.writing = false @@ -125,7 +137,7 @@ func (cl *coreLoop) run() { } } -func (cl *coreLoop) handleProducerWriteRequest(request *writeRequest) { +func (cl *coreLoop) handleProducerWriteRequest(request *producerWriteRequest) { if len(cl.blockedWrites) > 0 { // If other requests are still waiting for space, then there // definitely isn't enough for this one. @@ -168,7 +180,7 @@ func (cl *coreLoop) handleProducerWriteRequest(request *writeRequest) { } } -func (cl *coreLoop) handleProducerCancelRequest(request *cancelRequest) { +func (cl *coreLoop) handleProducerCancelRequest(request *producerCancelRequest) { } // Returns the active read segment, or nil if there is none. @@ -269,7 +281,7 @@ func (cl *coreLoop) handleShutdown() { close(cl.queue.writerLoop.input) if cl.writing { - <-cl.queue.writerLoop.finishedWriting + <-cl.queue.writerLoop.writeResponse cl.queue.segments.writing.writer.Close() } @@ -327,7 +339,7 @@ func (cl *coreLoop) maybeWritePending() { dq.segments.nextID++ } - cl.queue.writerLoop.input <- &writeBlock{ + cl.queue.writerLoop.input <- &writeRequest{ request: cl.pendingWrites[0], segment: segment, } diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index ebd3a2ab3ef..e63059c9b77 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -29,18 +29,18 @@ type diskQueueProducer struct { // The configuration this producer was created with. config queue.ProducerConfig - encoder frameEncoder + encoder *frameEncoder } // A request sent from a producer to the core loop to add a frame to the queue. -type writeRequest struct { +type producerWriteRequest struct { frame *writeFrame shouldBlock bool responseChan chan bool } // A request to the core loop to cancel the specified producer. -type cancelRequest struct { +type producerCancelRequest struct { producer *diskQueueProducer // If producer.config.DropOnCancel is true, then the core loop will respond // on responseChan with the number of dropped events. @@ -69,17 +69,18 @@ func (producer *diskQueueProducer) publish( "Couldn't serialize incoming event: %v", err) return false } - request := &writeRequest{ + request := &producerWriteRequest{ frame: &writeFrame{ event: event, serialized: serialized, + producer: producer, }, shouldBlock: shouldBlock, responseChan: make(chan bool), } select { - case producer.queue.writeRequestChan <- request: + case producer.queue.producerWriteRequestChan <- request: // The request has been sent, and we are now guaranteed to get a result on // the response channel, so we must read from it immediately to avoid // blocking the core loop. diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 9bab59b6c8c..86451338ef9 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -101,8 +101,8 @@ type diskQueue struct { deleterLoop *deleterLoop // The API channels used by diskQueueProducer to send write / cancel calls. - writeRequestChan chan *writeRequest - cancelRequestChan chan *cancelRequest + producerWriteRequestChan chan *producerWriteRequest + producerCancelRequestChan chan *producerCancelRequest // When a consumer ack increments ackedUpTo, the consumer sends // its new value to this channel. The core loop then decides whether to @@ -120,6 +120,7 @@ type diskQueue struct { // A map of all acked indices that are above ackedUpTo (and thus // can't yet be acknowledged as a continuous block). + // TODO: do this better. acked map[frameID]bool // Wait group for shutdown of the goroutines associated with this queue: @@ -207,6 +208,9 @@ func queueFactory( // NewQueue returns a disk-based queue configured with the given logger // and settings, creating it if it doesn't exist. func NewQueue(settings Settings) (queue.Queue, error) { + settings.Logger.Debugf( + "Initializing disk queue at path %v", settings.directoryPath()) + // Create the given directory path if it doesn't exist. err := os.MkdirAll(settings.directoryPath(), os.ModePerm) if err != nil { @@ -231,13 +235,73 @@ func NewQueue(settings Settings) (queue.Queue, error) { return nil, err } - return &diskQueue{ - settings: settings, + // We wait for four goroutines: core loop, reader loop, writer loop, + // deleter loop. + var waitGroup sync.WaitGroup + waitGroup.Add(4) + + // The helper loops all have an input channel with buffer size 1, to ensure + // that the core loop can never block when sending a request (the core + // loop never sends a request until receiving the response from the + // previous one, so there is never more than one outstanding request for + // any helper loop). + + readerLoop := &readerLoop{ + requestChan: make(chan readRequest, 1), + responseChan: make(chan readResponse), + output: make(chan *readFrame, 20), // TODO: customize this buffer size + } + go func() { + readerLoop.run() + waitGroup.Done() + }() + + writerLoop := &writerLoop{ + logger: settings.Logger, + input: make(chan *writeRequest, 1), + writeResponse: make(chan struct{}), + } + go func() { + writerLoop.run() + waitGroup.Done() + }() + + deleterLoop := &deleterLoop{ + queueSettings: &settings, + input: make(chan *deleteRequest), + response: make(chan *deleteResponse), + } + go func() { + deleterLoop.run() + waitGroup.Done() + }() + + queue := &diskQueue{ + settings: settings, + stateFile: stateFile, segments: &diskQueueSegments{ reading: initialSegments, }, - done: make(chan struct{}), - }, nil + readerLoop: readerLoop, + writerLoop: writerLoop, + deleterLoop: deleterLoop, + waitGroup: &waitGroup, + done: make(chan struct{}), + } + + // The core loop is created last because it's the only one that needs + // to refer back to the queue. (TODO: just merge the core loop fields + // and logic into the queue itself.) + queue.coreLoop = &coreLoop{ + queue: queue, + nextReadOffset: 0, // TODO: initialize this if we're opening an existing queue + } + go func() { + queue.coreLoop.run() + waitGroup.Done() + }() + + return queue, nil } // @@ -285,8 +349,9 @@ func (dq *diskQueue) BufferConfig() queue.BufferConfig { func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { return &diskQueueProducer{ - queue: dq, - config: cfg, + queue: dq, + config: cfg, + encoder: newFrameEncoder(dq.settings.ChecksumType), } } diff --git a/libbeat/publisher/queue/diskqueue/serialize.go b/libbeat/publisher/queue/diskqueue/serialize.go index 8e360b697f1..2efdc55cd65 100644 --- a/libbeat/publisher/queue/diskqueue/serialize.go +++ b/libbeat/publisher/queue/diskqueue/serialize.go @@ -23,6 +23,7 @@ package diskqueue import ( "bytes" "encoding/binary" + "fmt" "hash/crc32" "time" @@ -67,16 +68,17 @@ type entry struct { Fields common.MapStr } -func newEncoder(checksumType ChecksumType) (*frameEncoder, error) { +func newFrameEncoder(checksumType ChecksumType) *frameEncoder { e := &frameEncoder{checksumType: checksumType} e.reset() - return e, nil + return e } func (e *frameEncoder) reset() { e.folder = nil visitor := json.NewVisitor(&e.buf) + fmt.Printf("Creating folder\n") folder, err := gotype.NewIterator(visitor, gotype.Folders( codec.MakeTimestampEncoder(), @@ -94,6 +96,7 @@ func (e *frameEncoder) encode(event *publisher.Event) ([]byte, error) { e.buf.Reset() var flags uint8 + // TODO: handle guaranteed send? /*if (event.Flags & publisher.GuaranteedSend) == publisher.GuaranteedSend { flags = flagGuaranteed }*/ diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 786323d388c..067e7ec43ee 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -23,11 +23,18 @@ import ( "github.com/elastic/beats/v7/libbeat/logp" ) +type frameWriteRequest struct { + // The frame to be written to disk. + frame *writeFrame + + // The segment to which this frame should be written. + segment *queueSegment +} + // One block for the writer loop consists of a write request and the // segment it should be written to. -type writeBlock struct { - request *writeRequest - segment *queueSegment +type writeRequest struct { + frames []frameWriteRequest } type writerLoop struct { @@ -36,13 +43,13 @@ type writerLoop struct { // The writer loop listens on the input channel for write blocks, and // writes them to disk immediately (all queue capacity checking etc. is // done by the core loop before sending it to the writer). - input chan *writeBlock + input chan *writeRequest // The writer loop sends to this channel when it has finished writing a // frame, to signal the core loop that it is ready for the next one. To // ensure that the core loop doesn't block, the writer loop always reads - // from input immediately after sending to finishedWriting. - finishedWriting chan struct{} + // from input immediately after sending to writeResponse. + writeResponse chan struct{} } func (wl *writerLoop) run() { @@ -53,12 +60,12 @@ func (wl *writerLoop) run() { return } wl.processRequest(block) - wl.finishedWriting <- struct{}{} + wl.writeResponse <- struct{}{} } } // Write the block data to disk. -func (wl *writerLoop) processRequest(block *writeBlock) { +func (wl *writerLoop) processRequest(block *writeRequest) { //writer, err := block.segment.getWriter() } From 6fa9d3377dfb2d8842b45b1bf6a40f101621a10b Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Fri, 4 Sep 2020 13:53:36 -0400 Subject: [PATCH 32/91] finish writer loop rework --- .../publisher/queue/diskqueue/core_loop.go | 124 ++++++++++-------- libbeat/publisher/queue/diskqueue/queue.go | 17 ++- libbeat/publisher/queue/diskqueue/segments.go | 22 ++-- .../publisher/queue/diskqueue/writer_loop.go | 35 +++-- 4 files changed, 114 insertions(+), 84 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index aee6e2367c8..24247f62f66 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -46,10 +46,18 @@ type coreLoop struct { // nextReadOffset is the segment offset to start reading at during // the next read request. This offset always refers to the first read // segment: either segments.reading[0], if that list is nonempty, or - // segments.writing (if all segments have been read except the one + // segments.writing[0] (if all segments have been read except the one // currently being written). nextReadOffset segmentOffset + // nextWriteOffset is the segment offset at which the next new frame + // should be written. This offset always refers to the last entry of + // segments.writing. This is distinct from the endOffset field + // within a segment: endOffset tracks how much data _has_ been + // written to disk, while nextWriteOffset also includes all pending + // frames that are scheduled to be written to disk. + nextWriteOffset segmentOffset + // pendingWrites is a list of all write requests that have been accepted // by the queue and are waiting to be sent to the writer loop. pendingWrites []*producerWriteRequest @@ -105,7 +113,7 @@ func (cl *coreLoop) run() { for { select { - // Endpoints used by the public API + // Endpoints used by the producer / consumer API implementation. case producerWriteRequest := <-dq.producerWriteRequestChan: cl.handleProducerWriteRequest(producerWriteRequest) @@ -120,7 +128,7 @@ func (cl *coreLoop) run() { return // Writer loop handling - case <-dq.writerLoop.writeResponse: + case <-dq.writerLoop.responseChan: // Reset the writing flag and check if there's another frame waiting // to be written. cl.writing = false @@ -188,31 +196,38 @@ func (segments *diskQueueSegments) readingSegment() *queueSegment { if len(segments.reading) > 0 { return segments.reading[0] } - return segments.writing + if len(segments.writing) > 0 { + return segments.writing[0] + } + return nil } func (cl *coreLoop) handleReadResponse(response readResponse) { cl.reading = false segments := cl.queue.segments - segment := segments.readingSegment() - segment.framesRead += response.frameCount - newOffset := cl.nextReadOffset + segmentOffset(response.byteCount) - - // A (non-writing) segment is finished if we have read all the data, or - // the read response reports an error. - segmentFinished := - segment != segments.writing && - (newOffset >= segment.endOffset || response.err != nil) - if segmentFinished { - // Move to the acking list and reset the read offset. - segments.reading = segments.reading[1:] - segments.acking = append(segments.acking, segment) - cl.nextReadOffset = 0 + + // Advance the read offset based on what was just completed. + cl.nextReadOffset += segmentOffset(response.byteCount) + + var segment *queueSegment + if len(segments.reading) > 0 { + // A segment is finished if we have read all the data, or + // the read response reports an error. + // Segments in the reading list have been completely written, + // so we can rely on their endOffset field to determine the + // size of the data. + segment = segments.reading[0] + if cl.nextReadOffset >= segment.endOffset || response.err != nil { + segments.reading = segments.reading[1:] + segments.acking = append(segments.acking, segment) + cl.nextReadOffset = 0 + } } else { - // We aren't done reading this segment; next time we'll start from - // the end of this read response. - cl.nextReadOffset = newOffset + // A segment in the writing list can't be finished writing, + // so we don't check the endOffset. + segment = segments.writing[0] } + segment.framesRead += response.frameCount // If there is more data to read, start a new read request. cl.maybeReadPending() @@ -279,10 +294,10 @@ func (cl *coreLoop) handleShutdown() { cl.handleReadResponse(response) } - close(cl.queue.writerLoop.input) + close(cl.queue.writerLoop.requestChan) if cl.writing { - <-cl.queue.writerLoop.writeResponse - cl.queue.segments.writing.writer.Close() + <-cl.queue.writerLoop.responseChan + //cl.queue.segments.writing.writer.Close() } close(cl.queue.deleterLoop.input) @@ -309,39 +324,42 @@ func (cl *coreLoop) maybeWritePending() { // Nothing to do right now return } - // We are now definitely going to handle the next request, so - // remove it from pendingWrites. - request := cl.pendingWrites[0] - cl.pendingWrites = cl.pendingWrites[1:] - - // We have a frame to write, but we need to decide which segment - // it should go in. - segment := dq.segments.writing - - // If the new frame exceeds the maximum segment size, close the current - // writing segment. - frameLen := segmentOffset(len(request.frame.serialized)) - newEndOffset := segment.endOffset + frameLen - if segment != nil && newEndOffset > dq.settings.maxSegmentOffset() { - segment.writer.Close() - segment.writer = nil - dq.segments.reading = append(dq.segments.reading, segment) - segment = nil + // We are now definitely going to handle the queued requests, so + // remove them from pendingWrites. + requests := cl.pendingWrites + cl.pendingWrites = nil + + // We have frames to write, but we need to decide which segments + // they should go in and assemble them into frameWriteRequests + // for the writer loop. Start with the most recent writing segment + // if there is one. + var segment *queueSegment + if len(dq.segments.writing) > 0 { + segment = dq.segments.writing[len(dq.segments.writing)-1] } - - // If there is no active writing segment need to create a new segment. - if segment == nil { - segment = &queueSegment{ - id: dq.segments.nextID, - queueSettings: &dq.settings, + var frameRequests []frameWriteRequest + for _, request := range requests { + frameLen := segmentOffset(len(request.frame.serialized)) + // If segment is nil, or the new segment exceeds its bounds, + // we need to create a new writing segment. + if segment == nil || + cl.nextWriteOffset+frameLen > dq.settings.maxSegmentOffset() { + segment = &queueSegment{ + id: dq.segments.nextID, + queueSettings: &dq.settings, + } + dq.segments.writing = append(dq.segments.writing, segment) + dq.segments.nextID++ + cl.nextWriteOffset = frameLen } - dq.segments.writing = segment - dq.segments.nextID++ + frameRequests = append(frameRequests, frameWriteRequest{ + frame: request.frame, + segment: segment, + }) } - cl.queue.writerLoop.input <- &writeRequest{ - request: cl.pendingWrites[0], - segment: segment, + cl.queue.writerLoop.requestChan <- writeRequest{ + frames: frameRequests, } cl.writing = true } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 86451338ef9..c9b1e6a3d16 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -149,8 +149,15 @@ type pendingFrameData struct { // diskQueueSegments encapsulates segment-related queue metadata. type diskQueueSegments struct { - // The segment that is currently being written. - writing *queueSegment + // The segments that are currently being written. The writer loop + // writes these segments in order. When a segment has been + // completely written, the writer loop notifies the core loop + // in a writeResponse, and it is moved to the reading list. + // If the reading list is empty, the reader loop may read from + // a segment that is still being written, but it will always + // be writing[0], since later entries have generally not been + // created yet. + writing []*queueSegment // A list of the segments that have been completely written but have // not yet been completely processed by the reader loop, sorted by increasing @@ -257,9 +264,9 @@ func NewQueue(settings Settings) (queue.Queue, error) { }() writerLoop := &writerLoop{ - logger: settings.Logger, - input: make(chan *writeRequest, 1), - writeResponse: make(chan struct{}), + logger: settings.Logger, + requestChan: make(chan writeRequest, 1), + responseChan: make(chan writeResponse), } go func() { writerLoop.run() diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index f691f975f39..b5b86d4d155 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -70,12 +70,6 @@ type queueSegment struct { // // Used to count how many frames still need to be acknowledged by consumers. framesRead int64 - - // If this segment is being written or read, then reader / writer - // contain the respective file handles. To get a valid reader / writer for - // a segment that may not yet be open, call getReader / getWriter instead. - reader *os.File - writer *os.File } type segmentHeader struct { @@ -156,21 +150,21 @@ func (segment *queueSegment) getReader() (*os.File, error) { // Should only be called from the writer loop. func (segment *queueSegment) getWriter() (io.WriteCloser, error) { - if segment.writer != nil { - return segment.writer, nil - } path := segment.queueSettings.segmentPath(segment.id) file, err := os.OpenFile(path, os.O_CREATE|os.O_TRUNC, 0600) if err != nil { - return nil, err + return nil, fmt.Errorf( + "Couldn't open segment %d: %w", segment.id, err) } - segment.writer = file header := &segmentHeader{ version: 0, checksumType: segment.queueSettings.ChecksumType, } err = writeSegmentHeader(file, header) - // TODO: write header + if err != nil { + return nil, fmt.Errorf( + "Couldn't write to new segment %d: %w", segment.id, err) + } return file, nil } @@ -191,8 +185,8 @@ func writeSegmentHeader(out io.Writer, header *segmentHeader) error { // should only be called from the core loop. func (segments *diskQueueSegments) sizeOnDisk() uint64 { total := uint64(0) - if segments.writing != nil { - total += segments.writing.sizeOnDisk() + for _, segment := range segments.writing { + total += segment.sizeOnDisk() } for _, segment := range segments.reading { total += segment.sizeOnDisk() diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 067e7ec43ee..fb31f96bbf9 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -37,36 +37,47 @@ type writeRequest struct { frames []frameWriteRequest } +// A writeResponse reports the list of segments that have been +// completely written and can be moved to segments.reading. +// A segment is determined to have been completely written +// (and is then closed by the writer loop) when a frameWriteRequest +// targets a different segment than the previous ones. +type writeResponse struct { + completedSegments []*queueSegment +} + type writerLoop struct { logger *logp.Logger - // The writer loop listens on the input channel for write blocks, and + // The writer loop listens on requestChan for write blocks, and // writes them to disk immediately (all queue capacity checking etc. is // done by the core loop before sending it to the writer). - input chan *writeRequest + requestChan chan writeRequest - // The writer loop sends to this channel when it has finished writing a - // frame, to signal the core loop that it is ready for the next one. To - // ensure that the core loop doesn't block, the writer loop always reads - // from input immediately after sending to writeResponse. - writeResponse chan struct{} + // The writer loop sends to responseChan when it has finished handling a + // request, to signal the core loop that it is ready for the next one. + responseChan chan writeResponse } func (wl *writerLoop) run() { for { - block, ok := <-wl.input + block, ok := <-wl.requestChan if !ok { // The input channel is closed, we are done return } - wl.processRequest(block) - wl.writeResponse <- struct{}{} + completedSegments := wl.processRequest(block) + wl.responseChan <- writeResponse{ + completedSegments: completedSegments, + } } } -// Write the block data to disk. -func (wl *writerLoop) processRequest(block *writeRequest) { +// Write the given data to disk, returns the list of segments that were +// completed in the process. +func (wl *writerLoop) processRequest(request writeRequest) []*queueSegment { //writer, err := block.segment.getWriter() + return nil } // frameForContent wraps the given content buffer in a From e77dfa630bb61996dafceb93bda63c505e0db68b Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Fri, 4 Sep 2020 14:17:46 -0400 Subject: [PATCH 33/91] plug in writing to actual files --- libbeat/publisher/queue/diskqueue/segments.go | 2 +- .../publisher/queue/diskqueue/writer_loop.go | 40 ++++++++++++++++++- 2 files changed, 39 insertions(+), 3 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index b5b86d4d155..a1bccf48d13 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -149,7 +149,7 @@ func (segment *queueSegment) getReader() (*os.File, error) { } // Should only be called from the writer loop. -func (segment *queueSegment) getWriter() (io.WriteCloser, error) { +func (segment *queueSegment) getWriter() (*os.File, error) { path := segment.queueSettings.segmentPath(segment.id) file, err := os.OpenFile(path, os.O_CREATE|os.O_TRUNC, 0600) if err != nil { diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index fb31f96bbf9..eef6fad9cba 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -19,6 +19,7 @@ package diskqueue import ( "bytes" + "os" "github.com/elastic/beats/v7/libbeat/logp" ) @@ -57,6 +58,14 @@ type writerLoop struct { // The writer loop sends to responseChan when it has finished handling a // request, to signal the core loop that it is ready for the next one. responseChan chan writeResponse + + // The most recent segment that has been written to, if there is one. + // This segment + currentSegment *queueSegment + + // The file handle corresponding to currentSegment. When currentSegment + // changes, this handle is closed and a new one is created. + outputFile *os.File } func (wl *writerLoop) run() { @@ -76,8 +85,35 @@ func (wl *writerLoop) run() { // Write the given data to disk, returns the list of segments that were // completed in the process. func (wl *writerLoop) processRequest(request writeRequest) []*queueSegment { - //writer, err := block.segment.getWriter() - return nil + var completedSegments []*queueSegment + for _, frameRequest := range request.frames { + // If the new segment doesn't match the last one, we need to open a new + // file handle and possibly clean up the old one. + if wl.currentSegment != frameRequest.segment { + if wl.outputFile != nil { + completedSegments = append(completedSegments, wl.currentSegment) + wl.outputFile.Close() + wl.outputFile = nil + // TODO: try to sync? + } + wl.currentSegment = frameRequest.segment + file, err := wl.currentSegment.getWriter() + if err != nil { + // TODO: retry, etc + } + wl.outputFile = file + } + + // We have the data and a file to write it to. We are now committed + // to writing this block unless the queue is closed in the meantime. + _, err := wl.outputFile.Write(frameRequest.frame.serialized) + // TODO: retry forever if there is an error or n isn't the right + // length. + if err != nil { + wl.logger.Errorf("Couldn't write pending data to output file: %w", err) + } + } + return completedSegments } // frameForContent wraps the given content buffer in a From d2a65dc108a79287d5bd467e4981bb9ac65acc4b Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Fri, 4 Sep 2020 14:32:56 -0400 Subject: [PATCH 34/91] make diskQueue.logger a top-level field --- .../publisher/queue/diskqueue/core_loop.go | 4 ++-- libbeat/publisher/queue/diskqueue/producer.go | 4 +++- libbeat/publisher/queue/diskqueue/queue.go | 22 +++++++------------ .../publisher/queue/diskqueue/writer_loop.go | 2 +- 4 files changed, 14 insertions(+), 18 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 24247f62f66..990dccc251b 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -248,7 +248,7 @@ func (cl *coreLoop) handleDeleteResponse(response *deleteResponse) { dq.segments.acked = newAckedSegments } if len(response.errors) > 0 { - dq.settings.Logger.Errorw("Couldn't delete old segment files", + dq.logger.Errorw("Couldn't delete old segment files", "errors", response.errors) } // If there are still files to delete, send the next request. @@ -306,7 +306,7 @@ func (cl *coreLoop) handleShutdown() { // We can't retry any more if deletion failed, but we still check the // response so we can log any errors. if len(response.errors) > 0 { - cl.queue.settings.Logger.Errorw("Couldn't delete old segment files", + cl.queue.logger.Errorw("Couldn't delete old segment files", "errors", response.errors) } } diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index e63059c9b77..f9733fe5a1c 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -63,9 +63,11 @@ func (producer *diskQueueProducer) TryPublish(event publisher.Event) bool { func (producer *diskQueueProducer) publish( event publisher.Event, shouldBlock bool, ) bool { + producer.queue.logger.Debugf( + "diskQueueProducer.publish(%v, %v)", event, shouldBlock) serialized, err := producer.encoder.encode(&event) if err != nil { - producer.queue.settings.Logger.Errorf( + producer.queue.logger.Errorf( "Couldn't serialize incoming event: %v", err) return false } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index c9b1e6a3d16..71ce0b40d30 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -34,9 +34,6 @@ import ( // Settings contains the configuration fields to create a new disk queue // or open an existing one. type Settings struct { - // The destination for log messages related to the disk queue. - Logger *logp.Logger - // The path on disk of the queue's containing directory, which will be // created if it doesn't exist. Within the directory, the queue's state // is stored in state.dat and each segment's data is stored in @@ -56,7 +53,7 @@ type Settings struct { // A listener that receives ACKs when events are written to the queue's // disk buffer. - WriteToDiskACKListener queue.ACKListener + //producerACKListener queue.ACKListener ChecksumType ChecksumType } @@ -87,6 +84,7 @@ type diskQueueOutput struct { // diskQueue is the internal type representing a disk-based implementation // of queue.Queue. type diskQueue struct { + logger *logp.Logger settings Settings // The persistent queue state (wraps diskQueuePersistentState on disk). @@ -203,19 +201,14 @@ func queueFactory( if err != nil { return nil, err } - settings.Logger = logger - // For now, incoming messages are acked when they are written to disk - // (rather than transmitted to the output, as with the memory queue). This - // can produce unexpected behavior in some contexts and we might want to - // make it configurable later. - settings.WriteToDiskACKListener = ackListener - return NewQueue(settings) + //settings.producerAckListener = ackListener + return NewQueue(logger, settings) } // NewQueue returns a disk-based queue configured with the given logger // and settings, creating it if it doesn't exist. -func NewQueue(settings Settings) (queue.Queue, error) { - settings.Logger.Debugf( +func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { + logger.Debugf( "Initializing disk queue at path %v", settings.directoryPath()) // Create the given directory path if it doesn't exist. @@ -264,7 +257,7 @@ func NewQueue(settings Settings) (queue.Queue, error) { }() writerLoop := &writerLoop{ - logger: settings.Logger, + logger: logger, requestChan: make(chan writeRequest, 1), responseChan: make(chan writeResponse), } @@ -284,6 +277,7 @@ func NewQueue(settings Settings) (queue.Queue, error) { }() queue := &diskQueue{ + logger: logger, settings: settings, stateFile: stateFile, segments: &diskQueueSegments{ diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index eef6fad9cba..591e3712c38 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -110,7 +110,7 @@ func (wl *writerLoop) processRequest(request writeRequest) []*queueSegment { // TODO: retry forever if there is an error or n isn't the right // length. if err != nil { - wl.logger.Errorf("Couldn't write pending data to output file: %w", err) + wl.logger.Errorf("Couldn't write pending data to disk: %w", err) } } return completedSegments From b93fe83d83ce3beadcd6c11ad089b191127e659c Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Fri, 4 Sep 2020 16:31:20 -0400 Subject: [PATCH 35/91] move segment allocation to the producer handler, clean up naming --- libbeat/publisher/queue/diskqueue/config.go | 14 +- .../publisher/queue/diskqueue/core_loop.go | 140 ++++++++++-------- libbeat/publisher/queue/diskqueue/producer.go | 9 +- libbeat/publisher/queue/diskqueue/queue.go | 43 ++++-- .../publisher/queue/diskqueue/reader_loop.go | 20 +-- libbeat/publisher/queue/diskqueue/segments.go | 18 ++- .../publisher/queue/diskqueue/writer_loop.go | 23 +-- 7 files changed, 161 insertions(+), 106 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/config.go b/libbeat/publisher/queue/diskqueue/config.go index 3344cda8b49..7463e868b2f 100644 --- a/libbeat/publisher/queue/diskqueue/config.go +++ b/libbeat/publisher/queue/diskqueue/config.go @@ -37,6 +37,16 @@ func (c *userConfig) Validate() error { return nil } +// DefaultSettings returns a Settings object with reasonable default values +// for all important fields. +func DefaultSettings() Settings { + return Settings{ + ChecksumType: ChecksumTypeCRC32, + MaxSegmentSize: 100 * (1 << 20), // 100MB + MaxBufferSize: (1 << 30), // 1GB + } +} + // SettingsForUserConfig returns a Settings struct initialized with the // end-user-configurable settings in the given config tree. func SettingsForUserConfig(config *common.Config) (Settings, error) { @@ -44,9 +54,7 @@ func SettingsForUserConfig(config *common.Config) (Settings, error) { if err := config.Unpack(&userConfig); err != nil { return Settings{}, err } - settings := Settings{ - ChecksumType: ChecksumTypeCRC32, - } + settings := DefaultSettings() settings.Path = userConfig.Path return Settings{}, nil diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 990dccc251b..8f614a0e2ba 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -58,13 +58,13 @@ type coreLoop struct { // frames that are scheduled to be written to disk. nextWriteOffset segmentOffset - // pendingWrites is a list of all write requests that have been accepted - // by the queue and are waiting to be sent to the writer loop. - pendingWrites []*producerWriteRequest + // pendingFrames is a list of all incoming data frames that have been + // accepted by the queue and are waiting to be sent to the writer loop. + pendingFrames []segmentedFrame - // blockedWrites is a list of all write requests that are waiting for - // free space in the queue. - blockedWrites []*producerWriteRequest + // blockedProducers is a list of all producer write requests that are + // waiting for free space in the queue. + blockedProducers []producerWriteRequest // This value represents the oldest frame ID for a segment that has not // yet been moved to the acked list. It is used to detect when the oldest @@ -72,7 +72,8 @@ type coreLoop struct { oldestFrameID frameID } -// A frame waiting to be written to disk +// A data frame created through the producer API and waiting to be +// written to disk. type writeFrame struct { // The original event provided by the client to diskQueueProducer. // We keep this as well as the serialized form until we are done @@ -106,6 +107,7 @@ type readFrame struct { } func (cl *coreLoop) run() { + cl.queue.logger.Debug("Core loop starting up...") dq := cl.queue // Wake up the reader loop if there are segments available to read. @@ -145,50 +147,72 @@ func (cl *coreLoop) run() { } } -func (cl *coreLoop) handleProducerWriteRequest(request *producerWriteRequest) { - if len(cl.blockedWrites) > 0 { +func (cl *coreLoop) handleProducerWriteRequest(request producerWriteRequest) { + cl.queue.logger.Debugf( + "Core loop received producer write request (%d bytes)", + len(request.frame.serialized)) + if len(cl.blockedProducers) > 0 { // If other requests are still waiting for space, then there // definitely isn't enough for this one. if request.shouldBlock { - cl.blockedWrites = append(cl.blockedWrites, request) + // Blocked writes don't get a response until there is enough free + // space and they are moved to pendingWrites. + cl.blockedProducers = append(cl.blockedProducers, request) } else { // If the request is non-blocking, send immediate failure and discard it. request.responseChan <- false } return } + + // Pathological case checking: make sure the incoming frame isn't bigger + // than an entire segment all by itself (as long as it isn't, it is + // guaranteed to eventually enter the queue assuming no disk errors). + frameSize := uint64(len(request.frame.serialized)) + if cl.queue.settings.MaxSegmentSize < frameSize { + cl.queue.logger.Warnf( + "Rejecting event with size %v because the maximum segment size is %v", + frameSize, cl.queue.settings.MaxSegmentSize) + request.responseChan <- false + return + } + // We will accept this request if there is enough capacity left in // the queue (after accounting for the pending writes that were // already accepted). pendingBytes := uint64(0) - for _, request := range cl.pendingWrites { + for _, request := range cl.pendingFrames { pendingBytes += uint64(len(request.frame.serialized)) } currentSize := pendingBytes + cl.queue.segments.sizeOnDisk() - frameSize := uint64(len(request.frame.serialized)) - if currentSize+frameSize > cl.queue.settings.MaxBufferSize { + cl.queue.logger.Debugf( + "currentSize: %v frameSize: %v MaxBufferSize: %v", + currentSize, frameSize, cl.queue.settings.MaxBufferSize) + if cl.queue.settings.MaxBufferSize > 0 && + currentSize+frameSize > cl.queue.settings.MaxBufferSize { // The queue is too full. Either add the request to blockedWrites, // or send an immediate reject. if request.shouldBlock { - cl.blockedWrites = append(cl.blockedWrites, request) + cl.blockedProducers = append(cl.blockedProducers, request) } else { request.responseChan <- false } } else { // There is enough space for the new frame! Add it to the - // pending list and dispatch it to the writer loop if no other - // writes are outstanding. + // pending list and report success, then dispatch it to the + // writer loop if no other requests are outstanding. // Right now we accept any request if there is enough space for it // on disk. High-throughput inputs may produce events faster than // they can be written to disk, so it would make sense to // additionally bound the amount of data in pendingWrites to some // configurable limit to avoid out-of-memory errors. - cl.pendingWrites = append(cl.pendingWrites, request) + cl.enqueueProducerFrame(request.frame) + request.responseChan <- true cl.maybeWritePending() } } -func (cl *coreLoop) handleProducerCancelRequest(request *producerCancelRequest) { +func (cl *coreLoop) handleProducerCancelRequest(request producerCancelRequest) { } // Returns the active read segment, or nil if there is none. @@ -202,7 +226,7 @@ func (segments *diskQueueSegments) readingSegment() *queueSegment { return nil } -func (cl *coreLoop) handleReadResponse(response readResponse) { +func (cl *coreLoop) handleReadResponse(response readerLoopResponse) { cl.reading = false segments := cl.queue.segments @@ -317,49 +341,18 @@ func (cl *coreLoop) handleShutdown() { } // If the pendingWrites list is nonempty, and there are no outstanding -// requests to the writer loop, send the next frame. +// requests to the writer loop, send the next batch of frames. func (cl *coreLoop) maybeWritePending() { - dq := cl.queue - if cl.writing || len(cl.pendingWrites) == 0 { + if cl.writing || len(cl.pendingFrames) == 0 { // Nothing to do right now return } - // We are now definitely going to handle the queued requests, so - // remove them from pendingWrites. - requests := cl.pendingWrites - cl.pendingWrites = nil - - // We have frames to write, but we need to decide which segments - // they should go in and assemble them into frameWriteRequests - // for the writer loop. Start with the most recent writing segment - // if there is one. - var segment *queueSegment - if len(dq.segments.writing) > 0 { - segment = dq.segments.writing[len(dq.segments.writing)-1] - } - var frameRequests []frameWriteRequest - for _, request := range requests { - frameLen := segmentOffset(len(request.frame.serialized)) - // If segment is nil, or the new segment exceeds its bounds, - // we need to create a new writing segment. - if segment == nil || - cl.nextWriteOffset+frameLen > dq.settings.maxSegmentOffset() { - segment = &queueSegment{ - id: dq.segments.nextID, - queueSettings: &dq.settings, - } - dq.segments.writing = append(dq.segments.writing, segment) - dq.segments.nextID++ - cl.nextWriteOffset = frameLen - } - frameRequests = append(frameRequests, frameWriteRequest{ - frame: request.frame, - segment: segment, - }) - } + // Remove everything from pendingWrites and forward it to the writer loop. + requests := cl.pendingFrames + cl.pendingFrames = nil - cl.queue.writerLoop.requestChan <- writeRequest{ - frames: frameRequests, + cl.queue.writerLoop.requestChan <- writerLoopRequest{ + frames: requests, } cl.writing = true } @@ -376,7 +369,7 @@ func (cl *coreLoop) maybeReadPending() { // Nothing to read return } - request := readRequest{ + request := readerLoopRequest{ segment: segment, startOffset: cl.nextReadOffset, endOffset: segment.endOffset, @@ -393,3 +386,34 @@ func (cl *coreLoop) maybeDeleteAcked() { cl.deleting = true } } + +// enqueueProducerFrame determines which segment an incoming frame should be +// written to and adds the result to pendingWrites. +func (cl *coreLoop) enqueueProducerFrame(frame *writeFrame) { + dq := cl.queue + + // Start with the most recent writing segment if there is one. + var segment *queueSegment + if len(dq.segments.writing) > 0 { + segment = dq.segments.writing[len(dq.segments.writing)-1] + } + frameLen := segmentOffset(len(frame.serialized)) + // If segment is nil, or the new segment exceeds its bounds, + // we need to create a new writing segment. + if segment == nil || + cl.nextWriteOffset+frameLen > dq.settings.maxSegmentOffset() { + segment = &queueSegment{ + id: dq.segments.nextID, + queueSettings: &dq.settings, + } + dq.segments.writing = append(dq.segments.writing, segment) + dq.segments.nextID++ + cl.nextWriteOffset = 0 + } + + cl.nextWriteOffset += frameLen + cl.pendingFrames = append(cl.pendingFrames, segmentedFrame{ + frame: frame, + segment: segment, + }) +} diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index f9733fe5a1c..cbbfb813a69 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -63,22 +63,20 @@ func (producer *diskQueueProducer) TryPublish(event publisher.Event) bool { func (producer *diskQueueProducer) publish( event publisher.Event, shouldBlock bool, ) bool { - producer.queue.logger.Debugf( - "diskQueueProducer.publish(%v, %v)", event, shouldBlock) serialized, err := producer.encoder.encode(&event) if err != nil { producer.queue.logger.Errorf( "Couldn't serialize incoming event: %v", err) return false } - request := &producerWriteRequest{ + request := producerWriteRequest{ frame: &writeFrame{ event: event, serialized: serialized, producer: producer, }, shouldBlock: shouldBlock, - responseChan: make(chan bool), + responseChan: make(chan bool, 1), } select { @@ -87,7 +85,8 @@ func (producer *diskQueueProducer) publish( // the response channel, so we must read from it immediately to avoid // blocking the core loop. // TODO: this should be unblocked by a call to Cancel - return <-request.responseChan + response := <-request.responseChan + return response case <-producer.queue.done: return false } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 71ce0b40d30..d4f28e302e4 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -51,10 +51,6 @@ type Settings struct { // to a single segment file before creating a new one. MaxSegmentSize uint64 - // A listener that receives ACKs when events are written to the queue's - // disk buffer. - //producerACKListener queue.ACKListener - ChecksumType ChecksumType } @@ -93,14 +89,15 @@ type diskQueue struct { // Metadata related to the segment files. segments *diskQueueSegments + // The queue's helper loops, each of which is run in its own goroutine. coreLoop *coreLoop readerLoop *readerLoop writerLoop *writerLoop deleterLoop *deleterLoop // The API channels used by diskQueueProducer to send write / cancel calls. - producerWriteRequestChan chan *producerWriteRequest - producerCancelRequestChan chan *producerCancelRequest + producerWriteRequestChan chan producerWriteRequest + producerCancelRequestChan chan producerCancelRequest // When a consumer ack increments ackedUpTo, the consumer sends // its new value to this channel. The core loop then decides whether to @@ -211,6 +208,14 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { logger.Debugf( "Initializing disk queue at path %v", settings.directoryPath()) + if settings.MaxBufferSize > 0 && + settings.MaxBufferSize < settings.MaxSegmentSize*2 { + return nil, fmt.Errorf( + "Disk queue buffer size (%v) must be at least "+ + "twice the segment size (%v)", + settings.MaxBufferSize, settings.MaxSegmentSize) + } + // Create the given directory path if it doesn't exist. err := os.MkdirAll(settings.directoryPath(), os.ModePerm) if err != nil { @@ -247,8 +252,8 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { // any helper loop). readerLoop := &readerLoop{ - requestChan: make(chan readRequest, 1), - responseChan: make(chan readResponse), + requestChan: make(chan readerLoopRequest, 1), + responseChan: make(chan readerLoopResponse), output: make(chan *readFrame, 20), // TODO: customize this buffer size } go func() { @@ -258,8 +263,8 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { writerLoop := &writerLoop{ logger: logger, - requestChan: make(chan writeRequest, 1), - responseChan: make(chan writeResponse), + requestChan: make(chan writerLoopRequest, 1), + responseChan: make(chan writerLoopResponse), } go func() { writerLoop.run() @@ -277,17 +282,27 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { }() queue := &diskQueue{ - logger: logger, - settings: settings, + logger: logger, + settings: settings, + stateFile: stateFile, segments: &diskQueueSegments{ reading: initialSegments, }, + readerLoop: readerLoop, writerLoop: writerLoop, deleterLoop: deleterLoop, - waitGroup: &waitGroup, - done: make(chan struct{}), + + // TODO: customize this channel buffer size + producerWriteRequestChan: make(chan producerWriteRequest, 10), + producerCancelRequestChan: make(chan producerCancelRequest), + + consumerAckChan: make(chan frameID), + acked: make(map[frameID]bool), + + waitGroup: &waitGroup, + done: make(chan struct{}), } // The core loop is created last because it's the only one that needs diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index fb7e74052c4..132397c4bb3 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -19,13 +19,13 @@ package diskqueue import "os" -type readRequest struct { +type readerLoopRequest struct { segment *queueSegment startOffset segmentOffset endOffset segmentOffset } -type readResponse struct { +type readerLoopResponse struct { // The number of frames successfully read from the requested segment file. frameCount int64 @@ -43,8 +43,8 @@ type readerLoop struct { // sends the result to finishedReading. If there is more than one block // available for reading, the core loop will wait until it gets a // finishedReadingMessage before it - requestChan chan readRequest - responseChan chan readResponse + requestChan chan readerLoopRequest + responseChan chan readerLoopResponse // Frames that have been read from disk are sent to this channel. // Unlike most of the queue's API channels, this one is buffered to allow @@ -65,18 +65,18 @@ func (rl *readerLoop) run() { } } -func (rl *readerLoop) processRequest(request readRequest) readResponse { +func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopResponse { frameCount := int64(0) byteCount := int64(0) // Open the file and seek to the starting position. handle, err := request.segment.getReader() if err != nil { - return readResponse{err: err} + return readerLoopResponse{err: err} } _, err = handle.Seek(segmentHeaderSize+int64(request.startOffset), 0) if err != nil { - return readResponse{err: err} + return readerLoopResponse{err: err} } targetLength := int64(request.endOffset - request.startOffset) @@ -105,7 +105,7 @@ func (rl *readerLoop) processRequest(request readRequest) readResponse { // Since we haven't sent a finishedReading message yet, we can only // reach this case when the nextReadBlock channel is closed, indicating // queue shutdown. In this case we immediately return. - return readResponse{ + return readerLoopResponse{ frameCount: frameCount, byteCount: byteCount, err: nil, @@ -118,7 +118,7 @@ func (rl *readerLoop) processRequest(request readRequest) readResponse { // - there are no more frames to read, or // - we have reached the end of the requested region if err != nil || frame == nil || byteCount >= targetLength { - return readResponse{ + return readerLoopResponse{ frameCount: frameCount, byteCount: byteCount, err: err, @@ -130,7 +130,7 @@ func (rl *readerLoop) processRequest(request readRequest) readResponse { // again separately before we move on to the next data frame. select { case <-rl.requestChan: - return readResponse{ + return readerLoopResponse{ frameCount: frameCount, err: nil, } diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index a1bccf48d13..fb9ff4f2f5f 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -18,6 +18,7 @@ package diskqueue import ( + "encoding/binary" "fmt" "io" "io/ioutil" @@ -151,10 +152,9 @@ func (segment *queueSegment) getReader() (*os.File, error) { // Should only be called from the writer loop. func (segment *queueSegment) getWriter() (*os.File, error) { path := segment.queueSettings.segmentPath(segment.id) - file, err := os.OpenFile(path, os.O_CREATE|os.O_TRUNC, 0600) + file, err := os.OpenFile(path, os.O_CREATE|os.O_TRUNC|os.O_WRONLY, 0600) if err != nil { - return nil, fmt.Errorf( - "Couldn't open segment %d: %w", segment.id, err) + return nil, err } header := &segmentHeader{ version: 0, @@ -162,8 +162,7 @@ func (segment *queueSegment) getWriter() (*os.File, error) { } err = writeSegmentHeader(file, header) if err != nil { - return nil, fmt.Errorf( - "Couldn't write to new segment %d: %w", segment.id, err) + return nil, fmt.Errorf("Couldn't write segment header: %w", err) } return file, nil } @@ -177,8 +176,13 @@ func readSegmentHeader(in io.Reader) (*segmentHeader, error) { //return nil, nil } -func writeSegmentHeader(out io.Writer, header *segmentHeader) error { - panic("TODO: not implemented") +func writeSegmentHeader(out *os.File, header *segmentHeader) error { + err := binary.Write(out, binary.LittleEndian, header.version) + fmt.Printf("binary.Write result: %v\n", err) + if err == nil { + err = binary.Write(out, binary.LittleEndian, uint32(header.checksumType)) + } + return err } // The number of bytes occupied by all the queue's segment files. This diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 591e3712c38..008c91b2a2a 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -24,7 +24,9 @@ import ( "github.com/elastic/beats/v7/libbeat/logp" ) -type frameWriteRequest struct { +// A segmentedFrame is a data frame waiting to be written to disk along with +// the containing segment it has been assigned to. +type segmentedFrame struct { // The frame to be written to disk. frame *writeFrame @@ -34,16 +36,16 @@ type frameWriteRequest struct { // One block for the writer loop consists of a write request and the // segment it should be written to. -type writeRequest struct { - frames []frameWriteRequest +type writerLoopRequest struct { + frames []segmentedFrame } -// A writeResponse reports the list of segments that have been +// A writerLoopResponse reports the list of segments that have been // completely written and can be moved to segments.reading. // A segment is determined to have been completely written // (and is then closed by the writer loop) when a frameWriteRequest // targets a different segment than the previous ones. -type writeResponse struct { +type writerLoopResponse struct { completedSegments []*queueSegment } @@ -53,11 +55,11 @@ type writerLoop struct { // The writer loop listens on requestChan for write blocks, and // writes them to disk immediately (all queue capacity checking etc. is // done by the core loop before sending it to the writer). - requestChan chan writeRequest + requestChan chan writerLoopRequest // The writer loop sends to responseChan when it has finished handling a // request, to signal the core loop that it is ready for the next one. - responseChan chan writeResponse + responseChan chan writerLoopResponse // The most recent segment that has been written to, if there is one. // This segment @@ -69,6 +71,7 @@ type writerLoop struct { } func (wl *writerLoop) run() { + wl.logger.Debug("Writer loop starting up...") for { block, ok := <-wl.requestChan if !ok { @@ -76,7 +79,7 @@ func (wl *writerLoop) run() { return } completedSegments := wl.processRequest(block) - wl.responseChan <- writeResponse{ + wl.responseChan <- writerLoopResponse{ completedSegments: completedSegments, } } @@ -84,12 +87,13 @@ func (wl *writerLoop) run() { // Write the given data to disk, returns the list of segments that were // completed in the process. -func (wl *writerLoop) processRequest(request writeRequest) []*queueSegment { +func (wl *writerLoop) processRequest(request writerLoopRequest) []*queueSegment { var completedSegments []*queueSegment for _, frameRequest := range request.frames { // If the new segment doesn't match the last one, we need to open a new // file handle and possibly clean up the old one. if wl.currentSegment != frameRequest.segment { + wl.logger.Debugf("") if wl.outputFile != nil { completedSegments = append(completedSegments, wl.currentSegment) wl.outputFile.Close() @@ -99,6 +103,7 @@ func (wl *writerLoop) processRequest(request writeRequest) []*queueSegment { wl.currentSegment = frameRequest.segment file, err := wl.currentSegment.getWriter() if err != nil { + wl.logger.Errorf("Couldn't open new segment file: %w", err) // TODO: retry, etc } wl.outputFile = file From 3a3bfcd76ea1743c41267574e7397e24f8a70d1d Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Fri, 4 Sep 2020 17:03:47 -0400 Subject: [PATCH 36/91] properly handle writer loop responses --- .../publisher/queue/diskqueue/core_loop.go | 53 +++++++++++++------ .../publisher/queue/diskqueue/writer_loop.go | 41 ++++++++++---- 2 files changed, 67 insertions(+), 27 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 8f614a0e2ba..3e8089bbd34 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -130,15 +130,12 @@ func (cl *coreLoop) run() { return // Writer loop handling - case <-dq.writerLoop.responseChan: - // Reset the writing flag and check if there's another frame waiting - // to be written. - cl.writing = false - cl.maybeWritePending() + case writerLoopResponse := <-dq.writerLoop.responseChan: + cl.handleWriterLoopResponse(writerLoopResponse) // Reader loop handling - case readResponse := <-dq.readerLoop.responseChan: - cl.handleReadResponse(readResponse) + case readerLoopResponse := <-dq.readerLoop.responseChan: + cl.handleReaderLoopResponse(readerLoopResponse) // Deleter loop handling case deleteResponse := <-dq.deleterLoop.response: @@ -215,18 +212,29 @@ func (cl *coreLoop) handleProducerWriteRequest(request producerWriteRequest) { func (cl *coreLoop) handleProducerCancelRequest(request producerCancelRequest) { } -// Returns the active read segment, or nil if there is none. -func (segments *diskQueueSegments) readingSegment() *queueSegment { - if len(segments.reading) > 0 { - return segments.reading[0] - } - if len(segments.writing) > 0 { - return segments.writing[0] +func (cl *coreLoop) handleWriterLoopResponse(response writerLoopResponse) { + cl.writing = false + + for _, metadata := range response.segments { + // Update the segments with their new size and, if the writer + // closed them, move them to the reading list. + // TODO: i don't like this. It is redundant, and in brittle ways: + // segments are always written and closed in strict order, and the + // core loop knows what that order is, but we let the writer loop + // report them as independent parameters and then depend on those + // instead? It works for the moment but needs to be fixed soon. + metadata.segment.endOffset += segmentOffset(metadata.bytesWritten) + if metadata.closed { + cl.queue.segments.writing = cl.queue.segments.writing[1:] + cl.queue.segments.reading = + append(cl.queue.segments.reading, metadata.segment) + } } - return nil + + cl.maybeWritePending() } -func (cl *coreLoop) handleReadResponse(response readerLoopResponse) { +func (cl *coreLoop) handleReaderLoopResponse(response readerLoopResponse) { cl.reading = false segments := cl.queue.segments @@ -315,7 +323,7 @@ func (cl *coreLoop) handleShutdown() { close(cl.queue.readerLoop.requestChan) if cl.reading { response := <-cl.queue.readerLoop.responseChan - cl.handleReadResponse(response) + cl.handleReaderLoopResponse(response) } close(cl.queue.writerLoop.requestChan) @@ -357,6 +365,17 @@ func (cl *coreLoop) maybeWritePending() { cl.writing = true } +// Returns the active read segment, or nil if there is none. +func (segments *diskQueueSegments) readingSegment() *queueSegment { + if len(segments.reading) > 0 { + return segments.reading[0] + } + if len(segments.writing) > 0 { + return segments.writing[0] + } + return nil +} + // If the reading list is nonempty, and there are no outstanding read // requests, send one. func (cl *coreLoop) maybeReadPending() { diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 008c91b2a2a..fa3821423fd 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -40,13 +40,23 @@ type writerLoopRequest struct { frames []segmentedFrame } +// writerSegmentMetadata returns the actions taken by the writer loop +// on a single segment: the number of bytes written, and whether or +// not the segment was closed while handling the request (which signals +// to the core loop that the segment can be moved to segments.reading). +type writerSegmentMetadata struct { + segment *queueSegment + bytesWritten int64 + closed bool +} + // A writerLoopResponse reports the list of segments that have been // completely written and can be moved to segments.reading. // A segment is determined to have been completely written // (and is then closed by the writer loop) when a frameWriteRequest // targets a different segment than the previous ones. type writerLoopResponse struct { - completedSegments []*queueSegment + segments []writerSegmentMetadata } type writerLoop struct { @@ -78,27 +88,31 @@ func (wl *writerLoop) run() { // The input channel is closed, we are done return } - completedSegments := wl.processRequest(block) - wl.responseChan <- writerLoopResponse{ - completedSegments: completedSegments, - } + segments := wl.processRequest(block) + wl.responseChan <- writerLoopResponse{segments: segments} } } // Write the given data to disk, returns the list of segments that were // completed in the process. -func (wl *writerLoop) processRequest(request writerLoopRequest) []*queueSegment { - var completedSegments []*queueSegment +func (wl *writerLoop) processRequest(request writerLoopRequest) []writerSegmentMetadata { + var segments []writerSegmentMetadata + bytesWritten := int64(0) // Bytes written to the current segment. for _, frameRequest := range request.frames { // If the new segment doesn't match the last one, we need to open a new // file handle and possibly clean up the old one. if wl.currentSegment != frameRequest.segment { wl.logger.Debugf("") if wl.outputFile != nil { - completedSegments = append(completedSegments, wl.currentSegment) wl.outputFile.Close() wl.outputFile = nil // TODO: try to sync? + segments = append(segments, writerSegmentMetadata{ + segment: wl.currentSegment, + bytesWritten: bytesWritten, + closed: true, + }) + bytesWritten = 0 } wl.currentSegment = frameRequest.segment file, err := wl.currentSegment.getWriter() @@ -111,14 +125,21 @@ func (wl *writerLoop) processRequest(request writerLoopRequest) []*queueSegment // We have the data and a file to write it to. We are now committed // to writing this block unless the queue is closed in the meantime. - _, err := wl.outputFile.Write(frameRequest.frame.serialized) + n, err := wl.outputFile.Write(frameRequest.frame.serialized) + bytesWritten += int64(n) // TODO: retry forever if there is an error or n isn't the right // length. if err != nil { wl.logger.Errorf("Couldn't write pending data to disk: %w", err) } } - return completedSegments + if bytesWritten > 0 { + segments = append(segments, writerSegmentMetadata{ + segment: wl.currentSegment, + bytesWritten: bytesWritten, + }) + } + return segments } // frameForContent wraps the given content buffer in a From cb1107d554da4ddd145f5f76677ee99c393211c7 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Fri, 4 Sep 2020 17:31:40 -0400 Subject: [PATCH 37/91] implement readSegmentHeader, more cleanup / naming changes --- .../publisher/queue/diskqueue/core_loop.go | 26 +++++++++++------- .../publisher/queue/diskqueue/deleter_loop.go | 27 +++++++++---------- libbeat/publisher/queue/diskqueue/queue.go | 4 +-- libbeat/publisher/queue/diskqueue/segments.go | 17 +++++++----- 4 files changed, 43 insertions(+), 31 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 3e8089bbd34..504d9a0eed7 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -138,16 +138,13 @@ func (cl *coreLoop) run() { cl.handleReaderLoopResponse(readerLoopResponse) // Deleter loop handling - case deleteResponse := <-dq.deleterLoop.response: - cl.handleDeleteResponse(deleteResponse) + case deleterLoopResponse := <-dq.deleterLoop.responseChan: + cl.handleDeleterLoopResponse(deleterLoopResponse) } } } func (cl *coreLoop) handleProducerWriteRequest(request producerWriteRequest) { - cl.queue.logger.Debugf( - "Core loop received producer write request (%d bytes)", - len(request.frame.serialized)) if len(cl.blockedProducers) > 0 { // If other requests are still waiting for space, then there // definitely isn't enough for this one. @@ -231,6 +228,9 @@ func (cl *coreLoop) handleWriterLoopResponse(response writerLoopResponse) { } } + // New data is available, so we also check if we should send a new + // read request. + cl.maybeReadPending() cl.maybeWritePending() } @@ -265,7 +265,7 @@ func (cl *coreLoop) handleReaderLoopResponse(response readerLoopResponse) { cl.maybeReadPending() } -func (cl *coreLoop) handleDeleteResponse(response *deleteResponse) { +func (cl *coreLoop) handleDeleterLoopResponse(response deleterLoopResponse) { dq := cl.queue cl.deleting = false if len(response.deleted) > 0 { @@ -285,6 +285,10 @@ func (cl *coreLoop) handleDeleteResponse(response *deleteResponse) { } // If there are still files to delete, send the next request. cl.maybeDeleteAcked() + + // If there are blocked producers waiting for more queue space, this + // deletion might have unblocked them. + cl.maybeUnblockProducers() } func (cl *coreLoop) handleConsumerAck(ackedUpTo frameID) { @@ -332,9 +336,9 @@ func (cl *coreLoop) handleShutdown() { //cl.queue.segments.writing.writer.Close() } - close(cl.queue.deleterLoop.input) + close(cl.queue.deleterLoop.requestChan) if cl.deleting { - response := <-cl.queue.deleterLoop.response + response := <-cl.queue.deleterLoop.responseChan // We can't retry any more if deletion failed, but we still check the // response so we can log any errors. if len(response.errors) > 0 { @@ -401,11 +405,15 @@ func (cl *coreLoop) maybeReadPending() { // requests, send one. func (cl *coreLoop) maybeDeleteAcked() { if !cl.deleting && len(cl.queue.segments.acked) > 0 { - cl.queue.deleterLoop.input <- &deleteRequest{segments: cl.queue.segments.acked} + cl.queue.deleterLoop.requestChan <- deleterLoopRequest{segments: cl.queue.segments.acked} cl.deleting = true } } +func (cl *coreLoop) maybeUnblockProducers() { + // TODO: implement me +} + // enqueueProducerFrame determines which segment an incoming frame should be // written to and adds the result to pendingWrites. func (cl *coreLoop) enqueueProducerFrame(frame *writeFrame) { diff --git a/libbeat/publisher/queue/diskqueue/deleter_loop.go b/libbeat/publisher/queue/diskqueue/deleter_loop.go index 8b990b13da6..9673d325442 100644 --- a/libbeat/publisher/queue/diskqueue/deleter_loop.go +++ b/libbeat/publisher/queue/diskqueue/deleter_loop.go @@ -26,24 +26,23 @@ type deleterLoop struct { // The settings for the queue that created this loop. queueSettings *Settings - // When one or more segments are ready to delete, they are sent to the - // deleter loop input as a deleteRequest. At most one deleteRequest may be - // outstanding at any time. - input chan *deleteRequest + // When one or more segments are ready to delete, they are sent to + // requestChan. At most one deleteRequest may be outstanding at any time. + requestChan chan deleterLoopRequest - // When a deleteRequest has been completely processed, the resulting - // deleteResponse is sent on the response channel. If at least one deletion - // was successful, the response is sent immediately. Otherwise, the deleter - // loop delays for queueSettings.RetryWriteInterval before returning, so - // that delays don't have to be handled by the core loop. - response chan *deleteResponse + // When a request has been completely processed, a response is sent on + // responseChan. If at least one deletion was successful, the response + // is sent immediately. Otherwise, the deleter loop delays for + // queueSettings.RetryWriteInterval before returning, so timed retries + // don't have to be handled by the core loop. + responseChan chan deleterLoopResponse } -type deleteRequest struct { +type deleterLoopRequest struct { segments []*queueSegment } -type deleteResponse struct { +type deleterLoopResponse struct { // The queue segments that were successfully deleted. deleted map[*queueSegment]bool @@ -53,7 +52,7 @@ type deleteResponse struct { func (dl *deleterLoop) run() { for { - request, ok := <-dl.input + request, ok := <-dl.requestChan if !ok { // The channel has been closed, time to shut down. return @@ -71,7 +70,7 @@ func (dl *deleterLoop) run() { deleted[segment] = true } } - dl.response <- &deleteResponse{ + dl.responseChan <- deleterLoopResponse{ deleted: deleted, errors: errorList, } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index d4f28e302e4..71e797a06f2 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -273,8 +273,8 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { deleterLoop := &deleterLoop{ queueSettings: &settings, - input: make(chan *deleteRequest), - response: make(chan *deleteResponse), + requestChan: make(chan deleterLoopRequest), + responseChan: make(chan deleterLoopResponse), } go func() { deleterLoop.run() diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index fb9ff4f2f5f..73cc83cd548 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -20,7 +20,6 @@ package diskqueue import ( "encoding/binary" "fmt" - "io" "io/ioutil" "os" "sort" @@ -167,18 +166,24 @@ func (segment *queueSegment) getWriter() (*os.File, error) { return file, nil } -func readSegmentHeader(in io.Reader) (*segmentHeader, error) { - header := segmentHeader{} +func readSegmentHeader(in *os.File) (*segmentHeader, error) { + header := &segmentHeader{} + err := binary.Read(in, binary.LittleEndian, &header.version) + if err != nil { + return nil, err + } if header.version != 0 { return nil, fmt.Errorf("Unrecognized schema version %d", header.version) } - panic("TODO: not implemented") - //return nil, nil + err = binary.Read(in, binary.LittleEndian, &header.checksumType) + if err != nil { + return nil, err + } + return header, nil } func writeSegmentHeader(out *os.File, header *segmentHeader) error { err := binary.Write(out, binary.LittleEndian, header.version) - fmt.Printf("binary.Write result: %v\n", err) if err == nil { err = binary.Write(out, binary.LittleEndian, uint32(header.checksumType)) } From 3020962326c059eb0c1f121f60040a8c4ad44c85 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 8 Sep 2020 11:09:39 -0400 Subject: [PATCH 38/91] move queueSettings out of queueSegment, fix bugs --- .../publisher/queue/diskqueue/core_loop.go | 11 +++---- .../publisher/queue/diskqueue/deleter_loop.go | 4 +-- libbeat/publisher/queue/diskqueue/queue.go | 30 +++++++++++++------ .../publisher/queue/diskqueue/reader_loop.go | 5 +++- libbeat/publisher/queue/diskqueue/segments.go | 18 ++++++----- .../publisher/queue/diskqueue/serialize.go | 2 -- .../publisher/queue/diskqueue/writer_loop.go | 6 +++- 7 files changed, 47 insertions(+), 29 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 504d9a0eed7..4642b24a01a 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -179,9 +179,9 @@ func (cl *coreLoop) handleProducerWriteRequest(request producerWriteRequest) { pendingBytes += uint64(len(request.frame.serialized)) } currentSize := pendingBytes + cl.queue.segments.sizeOnDisk() - cl.queue.logger.Debugf( - "currentSize: %v frameSize: %v MaxBufferSize: %v", - currentSize, frameSize, cl.queue.settings.MaxBufferSize) + // cl.queue.logger.Debugf( + // "currentSize: %v frameSize: %v MaxBufferSize: %v", + // currentSize, frameSize, cl.queue.settings.MaxBufferSize) if cl.queue.settings.MaxBufferSize > 0 && currentSize+frameSize > cl.queue.settings.MaxBufferSize { // The queue is too full. Either add the request to blockedWrites, @@ -429,10 +429,7 @@ func (cl *coreLoop) enqueueProducerFrame(frame *writeFrame) { // we need to create a new writing segment. if segment == nil || cl.nextWriteOffset+frameLen > dq.settings.maxSegmentOffset() { - segment = &queueSegment{ - id: dq.segments.nextID, - queueSettings: &dq.settings, - } + segment = &queueSegment{id: dq.segments.nextID} dq.segments.writing = append(dq.segments.writing, segment) dq.segments.nextID++ cl.nextWriteOffset = 0 diff --git a/libbeat/publisher/queue/diskqueue/deleter_loop.go b/libbeat/publisher/queue/diskqueue/deleter_loop.go index 9673d325442..9813bc70416 100644 --- a/libbeat/publisher/queue/diskqueue/deleter_loop.go +++ b/libbeat/publisher/queue/diskqueue/deleter_loop.go @@ -24,7 +24,7 @@ import ( type deleterLoop struct { // The settings for the queue that created this loop. - queueSettings *Settings + settings *Settings // When one or more segments are ready to delete, they are sent to // requestChan. At most one deleteRequest may be outstanding at any time. @@ -60,7 +60,7 @@ func (dl *deleterLoop) run() { deleted := make(map[*queueSegment]bool, len(request.segments)) errorList := []error{} for _, segment := range request.segments { - path := dl.queueSettings.segmentPath(segment.id) + path := dl.settings.segmentPath(segment.id) err := os.Remove(path) // We ignore errors caused by the file not existing: this shouldn't // happen, but it is still safe to report it as successfully removed. diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 71e797a06f2..88c89709e2e 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -235,11 +235,6 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { } }() - initialSegments, err := scanExistingSegments(settings.directoryPath()) - if err != nil { - return nil, err - } - // We wait for four goroutines: core loop, reader loop, writer loop, // deleter loop. var waitGroup sync.WaitGroup @@ -252,6 +247,8 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { // any helper loop). readerLoop := &readerLoop{ + settings: &settings, + requestChan: make(chan readerLoopRequest, 1), responseChan: make(chan readerLoopResponse), output: make(chan *readFrame, 20), // TODO: customize this buffer size @@ -262,7 +259,9 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { }() writerLoop := &writerLoop{ - logger: logger, + logger: logger, + settings: &settings, + requestChan: make(chan writerLoopRequest, 1), responseChan: make(chan writerLoopResponse), } @@ -272,15 +271,27 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { }() deleterLoop := &deleterLoop{ - queueSettings: &settings, - requestChan: make(chan deleterLoopRequest), - responseChan: make(chan deleterLoopResponse), + settings: &settings, + + requestChan: make(chan deleterLoopRequest), + responseChan: make(chan deleterLoopResponse), } go func() { deleterLoop.run() waitGroup.Done() }() + // Index any existing data segments to be placed in segments.reading. + initialSegments, err := scanExistingSegments(settings.directoryPath()) + if err != nil { + return nil, err + } + var nextSegmentID segmentID + if len(initialSegments) > 0 { + lastID := initialSegments[len(initialSegments)-1].id + nextSegmentID = lastID + 1 + } + queue := &diskQueue{ logger: logger, settings: settings, @@ -288,6 +299,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { stateFile: stateFile, segments: &diskQueueSegments{ reading: initialSegments, + nextID: nextSegmentID, }, readerLoop: readerLoop, diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 132397c4bb3..e46a39d2040 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -38,6 +38,9 @@ type readerLoopResponse struct { } type readerLoop struct { + // The settings for the queue that created this loop. + settings *Settings + // When there is a block available for reading, it will be sent to // requestChan. When the reader loop has finished processing it, it // sends the result to finishedReading. If there is more than one block @@ -70,7 +73,7 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon byteCount := int64(0) // Open the file and seek to the starting position. - handle, err := request.segment.getReader() + handle, err := request.segment.getReader(rl.settings) if err != nil { return readerLoopResponse{err: err} } diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 73cc83cd548..3296dd5fb1a 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -45,7 +45,7 @@ type queueSegment struct { // The settings for the queue that created this segment. Used for locating // the queue file on disk and determining its checksum behavior. - queueSettings *Settings + //queueSettings *Settings // Whether the file for this segment exists on disk yet. If it does // not, then calling getWriter() will create it and return a writer @@ -91,7 +91,7 @@ type bySegmentID []*queueSegment func (s bySegmentID) Len() int { return len(s) } func (s bySegmentID) Swap(i, j int) { s[i], s[j] = s[j], s[i] } -func (s bySegmentID) Less(i, j int) bool { return s[i].endOffset < s[j].endOffset } +func (s bySegmentID) Less(i, j int) bool { return s[i].id < s[j].id } // Scan the given path for segment files, and return them in a list // ordered by segment id. @@ -132,8 +132,10 @@ func (segment *queueSegment) sizeOnDisk() uint64 { } // Should only be called from the reader loop. -func (segment *queueSegment) getReader() (*os.File, error) { - path := segment.queueSettings.segmentPath(segment.id) +func (segment *queueSegment) getReader( + queueSettings *Settings, +) (*os.File, error) { + path := queueSettings.segmentPath(segment.id) file, err := os.Open(path) if err != nil { return nil, fmt.Errorf( @@ -149,15 +151,17 @@ func (segment *queueSegment) getReader() (*os.File, error) { } // Should only be called from the writer loop. -func (segment *queueSegment) getWriter() (*os.File, error) { - path := segment.queueSettings.segmentPath(segment.id) +func (segment *queueSegment) getWriter( + queueSettings *Settings, +) (*os.File, error) { + path := queueSettings.segmentPath(segment.id) file, err := os.OpenFile(path, os.O_CREATE|os.O_TRUNC|os.O_WRONLY, 0600) if err != nil { return nil, err } header := &segmentHeader{ version: 0, - checksumType: segment.queueSettings.ChecksumType, + checksumType: queueSettings.ChecksumType, } err = writeSegmentHeader(file, header) if err != nil { diff --git a/libbeat/publisher/queue/diskqueue/serialize.go b/libbeat/publisher/queue/diskqueue/serialize.go index 2efdc55cd65..b73cf98e1e9 100644 --- a/libbeat/publisher/queue/diskqueue/serialize.go +++ b/libbeat/publisher/queue/diskqueue/serialize.go @@ -23,7 +23,6 @@ package diskqueue import ( "bytes" "encoding/binary" - "fmt" "hash/crc32" "time" @@ -78,7 +77,6 @@ func (e *frameEncoder) reset() { e.folder = nil visitor := json.NewVisitor(&e.buf) - fmt.Printf("Creating folder\n") folder, err := gotype.NewIterator(visitor, gotype.Folders( codec.MakeTimestampEncoder(), diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index fa3821423fd..18ca406119c 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -60,6 +60,10 @@ type writerLoopResponse struct { } type writerLoop struct { + // The settings for the queue that created this loop. + settings *Settings + + // The logger for the writer loop, assigned when the queue creates it. logger *logp.Logger // The writer loop listens on requestChan for write blocks, and @@ -115,7 +119,7 @@ func (wl *writerLoop) processRequest(request writerLoopRequest) []writerSegmentM bytesWritten = 0 } wl.currentSegment = frameRequest.segment - file, err := wl.currentSegment.getWriter() + file, err := wl.currentSegment.getWriter(wl.settings) if err != nil { wl.logger.Errorf("Couldn't open new segment file: %w", err) // TODO: retry, etc From df31c0e4b818f46fa1a817ac318e89b898de41a1 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 8 Sep 2020 15:00:33 -0400 Subject: [PATCH 39/91] delete old code, reorganize some structures --- .../publisher/queue/diskqueue/core_loop.go | 38 ------ libbeat/publisher/queue/diskqueue/frames.go | 68 ++++++++++ libbeat/publisher/queue/diskqueue/producer.go | 2 +- libbeat/publisher/queue/diskqueue/queue.go | 97 +------------- .../publisher/queue/diskqueue/reader_loop.go | 122 +++++++++++++++++- libbeat/publisher/queue/diskqueue/segments.go | 85 +++++++++--- .../publisher/queue/diskqueue/serialize.go | 35 ++--- .../publisher/queue/diskqueue/writer_loop.go | 2 +- 8 files changed, 269 insertions(+), 180 deletions(-) create mode 100644 libbeat/publisher/queue/diskqueue/frames.go diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 4642b24a01a..c2757269c13 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -17,10 +17,6 @@ package diskqueue -import ( - "github.com/elastic/beats/v7/libbeat/publisher" -) - type coreLoop struct { // The queue that created this coreLoop. The core loop is the only one of // the main goroutines for the queue that has a pointer to the queue and @@ -72,40 +68,6 @@ type coreLoop struct { oldestFrameID frameID } -// A data frame created through the producer API and waiting to be -// written to disk. -type writeFrame struct { - // The original event provided by the client to diskQueueProducer. - // We keep this as well as the serialized form until we are done - // writing, because we may need to send this value back to the producer - // callback if it is cancelled. - event publisher.Event - - // The event, serialized for writing to disk and wrapped in a frame - // header / footer. - serialized []byte - - // The producer that created this frame. This is included in the - // frame structure itself because we may need the producer and / or - // its config at any time up until it has been completely written: - // - While the core loop is tracking frames to send to the writer, - // it may receive a Cancel request, which requires us to know - // the producer / config each frame came from. - // - After the writer loop has finished writing the frame to disk, - // it needs to call the ACK function specified in ProducerConfig. - producer *diskQueueProducer -} - -// A frame that has been read from disk -type readFrame struct { - event publisher.Event - id frameID - - // How much space this frame occupied on disk (before deserialization), - // including the frame header / footer. - bytesOnDisk int64 -} - func (cl *coreLoop) run() { cl.queue.logger.Debug("Core loop starting up...") dq := cl.queue diff --git a/libbeat/publisher/queue/diskqueue/frames.go b/libbeat/publisher/queue/diskqueue/frames.go new file mode 100644 index 00000000000..93671eaa77f --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/frames.go @@ -0,0 +1,68 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +import "github.com/elastic/beats/v7/libbeat/publisher" + +// Every data frame read from the queue is assigned a unique sequential +// integer, which is used to keep track of which frames have been +// acknowledged. +// This id is not stable between restarts; the value 0 is always assigned +// to the oldest remaining frame on startup. +type frameID uint64 + +// A data frame created through the producer API and waiting to be +// written to disk. +type writeFrame struct { + // The original event provided by the client to diskQueueProducer. + // We keep this as well as the serialized form until we are done + // writing, because we may need to send this value back to the producer + // callback if it is cancelled. + event publisher.Event + + // The event, serialized for writing to disk and wrapped in a frame + // header / footer. + serialized []byte + + // The producer that created this frame. This is included in the + // frame structure itself because we may need the producer and / or + // its config at any time up until it has been completely written: + // - While the core loop is tracking frames to send to the writer, + // it may receive a Cancel request, which requires us to know + // the producer / config each frame came from. + // - After the writer loop has finished writing the frame to disk, + // it needs to call the ACK function specified in ProducerConfig. + producer *diskQueueProducer +} + +// A frame that has been read from disk and is waiting to be read / +// acknowledged through the consumer API. +type readFrame struct { + event publisher.Event + id frameID + + // How much space this frame occupied on disk (before deserialization), + // including the frame header / footer. + bytesOnDisk int64 +} + +// Each data frame has a 32-bit length in the header, and a 32-bit checksum +// and a duplicate 32-bit length in the footer. +const frameHeaderSize = 4 +const frameFooterSize = 8 +const frameMetadataSize = frameHeaderSize + frameFooterSize diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index cbbfb813a69..f1aa174c8a2 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -29,7 +29,7 @@ type diskQueueProducer struct { // The configuration this producer was created with. config queue.ProducerConfig - encoder *frameEncoder + encoder *eventEncoder } // A request sent from a producer to the core loop to add a frame to the queue. diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 88c89709e2e..d7b8fff70ad 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -27,7 +27,6 @@ import ( "github.com/elastic/beats/v7/libbeat/feature" "github.com/elastic/beats/v7/libbeat/logp" "github.com/elastic/beats/v7/libbeat/paths" - "github.com/elastic/beats/v7/libbeat/publisher" "github.com/elastic/beats/v7/libbeat/publisher/queue" ) @@ -66,17 +65,6 @@ type queuePosition struct { offset segmentOffset } -type diskQueueOutput struct { - data []byte - - // The segment file this data was read from. - segment *queueSegment - - // The index of this data's frame (the sequential read order - // of all frames during this execution). - frame frameID -} - // diskQueue is the internal type representing a disk-based implementation // of queue.Queue. type diskQueue struct { @@ -126,59 +114,6 @@ type diskQueue struct { done chan struct{} } -// pendingFrame stores a single incoming event waiting to be written to disk, -// along with its serialization and metadata needed to notify its originating -// producer of ack / cancel state. -type pendingFrame struct { - event publisher.Event - producer *diskQueueProducer -} - -// pendingFrameData stores data frames waiting to be written to disk, with -// metadata to handle acks / cancellation if needed. -type pendingFrameData struct { - sync.Mutex - - frames []pendingFrame -} - -// diskQueueSegments encapsulates segment-related queue metadata. -type diskQueueSegments struct { - // The segments that are currently being written. The writer loop - // writes these segments in order. When a segment has been - // completely written, the writer loop notifies the core loop - // in a writeResponse, and it is moved to the reading list. - // If the reading list is empty, the reader loop may read from - // a segment that is still being written, but it will always - // be writing[0], since later entries have generally not been - // created yet. - writing []*queueSegment - - // A list of the segments that have been completely written but have - // not yet been completely processed by the reader loop, sorted by increasing - // segment ID. Segments are always read in order. When a segment has - // been read completely, it is removed from the front of this list and - // appended to read. - reading []*queueSegment - - // A list of the segments that have been read but have not yet been - // completely acknowledged, sorted by increasing segment ID. When the - // first entry of this list is completely acknowledged, it is removed - // from this list and added to acked. - acking []*queueSegment - - // A list of the segments that have been completely processed and are - // ready to be deleted. The writer loop always tries to delete segments - // in this list before writing new data. When a segment is successfully - // deleted, it is removed from this list and the queue's - // segmentDeletedCond is signalled. - acked []*queueSegment - - // The next sequential unused segment ID. This is what will be assigned - // to the next queueSegment we create. - nextID segmentID -} - func init() { queue.RegisterQueueType( "disk", @@ -379,40 +314,10 @@ func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { return &diskQueueProducer{ queue: dq, config: cfg, - encoder: newFrameEncoder(dq.settings.ChecksumType), + encoder: newFrameEncoder(), } } func (dq *diskQueue) Consumer() queue.Consumer { panic("TODO: not implemented") } - -// This is only called by readerLoop. -/*func (dq *diskQueue) nextSegmentReader() (*segmentReader, []error) { - dq.segments.Lock() - defer dq.segments.Unlock() - - errors := []error{} - for len(dq.segments.reading) > 0 { - segment := dq.segments.reading[0] - segmentPath := dq.settings.segmentPath(segment.id) - reader, err := tryLoad(segment, segmentPath) - if err != nil { - // TODO: Handle this: depending on the type of error, either delete - // the segment or log an error and leave it alone, then skip to the - // next one. - errors = append(errors, err) - dq.segments.reading = dq.segments.reading[1:] - continue - } - // Remove the segment from the active list and move it to - // completedSegments until all its data has been acknowledged. - dq.segments.reading = dq.segments.reading[1:] - dq.segments.acking = append(dq.segments.acking, segment) - return reader, errors - } - // TODO: if segments.reading is empty we may still be able to - // read partial data from segments.writing which is still being - // written. - return nil, errors -}*/ diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index e46a39d2040..861c89348df 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -17,7 +17,15 @@ package diskqueue -import "os" +import ( + "encoding/binary" + "errors" + "fmt" + "io" + "os" + "syscall" + "time" +) type readerLoopRequest struct { segment *queueSegment @@ -64,11 +72,20 @@ func (rl *readerLoop) run() { close(rl.output) return } + response := rl.processRequest(request) + fmt.Printf("\033[0;32mread response: read %d frames and %d bytes\033[0m\n", response.frameCount, response.byteCount) + if response.err != nil { + fmt.Printf("\033[0;32mresponse had err: %v\033[0m\n", response.err) + } rl.responseChan <- rl.processRequest(request) } } func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopResponse { + fmt.Printf("\033[0;32mprocessRequest(segment %d from %d to %d)\033[0m\n", request.segment.id, request.startOffset, request.endOffset) + + defer time.Sleep(time.Second) + frameCount := int64(0) byteCount := int64(0) @@ -77,6 +94,7 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon if err != nil { return readerLoopResponse{err: err} } + defer handle.Close() _, err = handle.Seek(segmentHeaderSize+int64(request.startOffset), 0) if err != nil { return readerLoopResponse{err: err} @@ -96,7 +114,8 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon // Try to read the next, clipping to the length we were told to read. // If the next frame extends past this boundary, nextFrame will return // an error. - frame, err := nextFrame(handle, remainingLength) + frame, err := nextFrame( + handle, remainingLength, request.segment.header.checksumType) if frame != nil { // We've read the frame, try sending it to the output channel. select { @@ -135,6 +154,7 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon case <-rl.requestChan: return readerLoopResponse{ frameCount: frameCount, + byteCount: byteCount, err: nil, } default: @@ -142,8 +162,102 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon } } -func nextFrame(handle *os.File, maxLength int64) (*readFrame, error) { - return nil, nil +func nextFrame( + handle *os.File, maxLength int64, checksumType ChecksumType, +) (*readFrame, error) { + // Ensure we are allowed to read the frame header. + if maxLength < frameHeaderSize { + return nil, fmt.Errorf( + "Can't read next frame: remaining length %d is too low", maxLength) + } + // Wrap the handle to retry non-fatal errors and always return the full + // requested data length if possible. + reader := autoRetryReader{handle} + var frameLength uint32 + err := binary.Read(reader, binary.LittleEndian, &frameLength) + if err != nil { + return nil, fmt.Errorf( + "Couldn't read next frame length: %w", err) + } + + // If the frame extends past the area we were told to read, return an error. + // This should never happen unless the segment file is corrupted. + if maxLength < int64(frameLength) { + return nil, fmt.Errorf( + "Can't read next frame: frame size is %v but remaining data is only %v", + frameLength, maxLength) + } + if frameLength <= frameMetadataSize { + // Valid enqueued data must have positive length + return nil, fmt.Errorf( + "Data frame with no data (length %d)", frameLength) + } + + // Read the actual frame data + dataLength := frameLength - frameMetadataSize + bytes := make([]byte, dataLength) + _, err = reader.Read(bytes) + if err != nil { + return nil, fmt.Errorf("Couldn't read data frame: %w", err) + } + + // Read the footer (checksum + duplicate length) + var checksum uint32 + err = binary.Read(reader, binary.LittleEndian, &checksum) + if err != nil { + return nil, fmt.Errorf("Couldn't read data frame checksum: %w", err) + } + // TODO: validate checksum + /*if computeChecksum(data, reader.checksumType) != checksum { + return nil, fmt.Errorf("Disk queue: bad data frame checksum") + }*/ + + var duplicateLength uint32 + err = binary.Read(reader, binary.LittleEndian, &duplicateLength) + if err != nil { + return nil, fmt.Errorf( + "Disk queue couldn't read trailing frame length: %w", err) + } + if duplicateLength != frameLength { + return nil, fmt.Errorf( + "Disk queue: inconsistent frame length (%d vs %d)", + frameLength, duplicateLength) + } + + // TODO: deserialize + frame := &readFrame{ + bytesOnDisk: int64(frameLength), + } + + return frame, nil +} + +// A wrapper for an io.Reader that tries to read the full number of bytes +// requested, retrying on EAGAIN and EINTR, and returns an error if +// and only if the number of bytes read is less than requested. +// This is similar to io.ReadFull but with retrying. +type autoRetryReader struct { + wrapped io.Reader +} + +func (r autoRetryReader) Read(p []byte) (int, error) { + bytesRead := 0 + reader := r.wrapped + n, err := reader.Read(p) + for n < len(p) { + if err != nil && !readErrorIsRetriable(err) { + return bytesRead + n, err + } + // If there is an error, it is retriable, so advance p and try again. + bytesRead += n + p = p[n:] + n, err = reader.Read(p) + } + return bytesRead + n, nil +} + +func readErrorIsRetriable(err error) bool { + return errors.Is(err, syscall.EINTR) || errors.Is(err, syscall.EAGAIN) } /*func (dq *diskQueue) readerLoop() { diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 3296dd5fb1a..4bf88af7e95 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -27,6 +27,43 @@ import ( "strings" ) +// diskQueueSegments encapsulates segment-related queue metadata. +type diskQueueSegments struct { + // The segments that are currently being written. The writer loop + // writes these segments in order. When a segment has been + // completely written, the writer loop notifies the core loop + // in a writeResponse, and it is moved to the reading list. + // If the reading list is empty, the reader loop may read from + // a segment that is still being written, but it will always + // be writing[0], since later entries have generally not been + // created yet. + writing []*queueSegment + + // A list of the segments that have been completely written but have + // not yet been completely processed by the reader loop, sorted by increasing + // segment ID. Segments are always read in order. When a segment has + // been read completely, it is removed from the front of this list and + // appended to read. + reading []*queueSegment + + // A list of the segments that have been read but have not yet been + // completely acknowledged, sorted by increasing segment ID. When the + // first entry of this list is completely acknowledged, it is removed + // from this list and added to acked. + acking []*queueSegment + + // A list of the segments that have been completely processed and are + // ready to be deleted. The writer loop always tries to delete segments + // in this list before writing new data. When a segment is successfully + // deleted, it is removed from this list and the queue's + // segmentDeletedCond is signalled. + acked []*queueSegment + + // The next sequential unused segment ID. This is what will be assigned + // to the next queueSegment we create. + nextID segmentID +} + // Every data frame read from the queue is assigned a unique sequential // integer, which is used to keep track of which frames have been // acknowledged. @@ -43,15 +80,6 @@ type queueSegment struct { // A segment id is globally unique within its originating queue. id segmentID - // The settings for the queue that created this segment. Used for locating - // the queue file on disk and determining its checksum behavior. - //queueSettings *Settings - - // Whether the file for this segment exists on disk yet. If it does - // not, then calling getWriter() will create it and return a writer - // positioned at the start of the data region. - created bool - // The byte offset of the end of the segment's data region. This is // updated when the segment is written to, and should always correspond // to the end of a complete data frame. The total size of a segment file @@ -77,10 +105,23 @@ type segmentHeader struct { checksumType ChecksumType } -// Each data frame has a 32-bit length and a 32-bit checksum -// in the header, and a duplicate 32-bit length in the footer. -const frameHeaderSize = 8 -const frameFooterSize = 4 +// ChecksumType specifies what checksum algorithm the queue should use to +// verify its data frames. +type ChecksumType uint32 + +// ChecksumTypeNone: Don't compute or verify checksums. +// ChecksumTypeCRC32: Compute the checksum with the Go standard library's +// "hash/crc32" package. +const ( + ChecksumTypeNone = iota + + ChecksumTypeCRC32 +) + +// Each data frame has a 32-bit length in the header, and a 32-bit checksum +// and a duplicate 32-bit length in the footer. +const frameHeaderSize = 4 +const frameFooterSize = 8 const frameMetadataSize = frameHeaderSize + frameFooterSize // Each segment header has a 32-bit version and a 32-bit checksum type. @@ -117,7 +158,6 @@ func scanExistingSegments(path string) ([]*queueSegment, error) { segments = append(segments, &queueSegment{ id: segmentID(id), - created: true, endOffset: segmentOffset(file.Size() - segmentHeaderSize), }) } @@ -135,18 +175,24 @@ func (segment *queueSegment) sizeOnDisk() uint64 { func (segment *queueSegment) getReader( queueSettings *Settings, ) (*os.File, error) { + fmt.Printf("\033[94mgetReader(%v)\033[0m\n", segment.id) + path := queueSettings.segmentPath(segment.id) file, err := os.Open(path) if err != nil { + fmt.Printf("\033[94mfailed: %v\033[0m\n", err) return nil, fmt.Errorf( "Couldn't open segment %d: %w", segment.id, err) } header, err := readSegmentHeader(file) if err != nil { + fmt.Printf("\033[94mfailed (header): %v\033[0m\n", err) + file.Close() return nil, fmt.Errorf("Couldn't read segment header: %w", err) } segment.header = header + fmt.Printf("\033[94msuccess\033[0m\n") return file, nil } @@ -154,9 +200,11 @@ func (segment *queueSegment) getReader( func (segment *queueSegment) getWriter( queueSettings *Settings, ) (*os.File, error) { + fmt.Printf("\033[0;32mgetWriter(%v)\033[0m\n", segment.id) path := queueSettings.segmentPath(segment.id) - file, err := os.OpenFile(path, os.O_CREATE|os.O_TRUNC|os.O_WRONLY, 0600) + file, err := os.OpenFile(path, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) if err != nil { + fmt.Printf("\033[0;32mfailed\033[0m\n") return nil, err } header := &segmentHeader{ @@ -165,8 +213,11 @@ func (segment *queueSegment) getWriter( } err = writeSegmentHeader(file, header) if err != nil { + fmt.Printf("\033[0;32mfailed (header)\033[0m\n") return nil, fmt.Errorf("Couldn't write segment header: %w", err) } + fmt.Printf("\033[0;32msuccess\033[0m\n") + return file, nil } @@ -179,10 +230,12 @@ func readSegmentHeader(in *os.File) (*segmentHeader, error) { if header.version != 0 { return nil, fmt.Errorf("Unrecognized schema version %d", header.version) } - err = binary.Read(in, binary.LittleEndian, &header.checksumType) + var rawChecksumType uint32 + err = binary.Read(in, binary.LittleEndian, &rawChecksumType) if err != nil { return nil, err } + header.checksumType = ChecksumType(rawChecksumType) return header, nil } diff --git a/libbeat/publisher/queue/diskqueue/serialize.go b/libbeat/publisher/queue/diskqueue/serialize.go index b73cf98e1e9..ff4c6a5ee5d 100644 --- a/libbeat/publisher/queue/diskqueue/serialize.go +++ b/libbeat/publisher/queue/diskqueue/serialize.go @@ -34,26 +34,13 @@ import ( "github.com/elastic/go-structform/json" ) -// ChecksumType specifies what checksum algorithm the queue should use to -// verify its data frames. -type ChecksumType int - -// ChecksumTypeNone: Don't compute or verify checksums. -// ChecksumTypeCRC32: Compute the checksum with the Go standard library's -// "hash/crc32" package. -const ( - ChecksumTypeNone = iota - - ChecksumTypeCRC32 -) - -type frameEncoder struct { +type eventEncoder struct { buf bytes.Buffer folder *gotype.Iterator checksumType ChecksumType } -type decoder struct { +type eventDecoder struct { buf []byte parser *json.Parser @@ -67,13 +54,13 @@ type entry struct { Fields common.MapStr } -func newFrameEncoder(checksumType ChecksumType) *frameEncoder { - e := &frameEncoder{checksumType: checksumType} +func newFrameEncoder() *eventEncoder { + e := &eventEncoder{} e.reset() return e } -func (e *frameEncoder) reset() { +func (e *eventEncoder) reset() { e.folder = nil visitor := json.NewVisitor(&e.buf) @@ -90,7 +77,7 @@ func (e *frameEncoder) reset() { e.folder = folder } -func (e *frameEncoder) encode(event *publisher.Event) ([]byte, error) { +func (e *eventEncoder) encode(event *publisher.Event) ([]byte, error) { e.buf.Reset() var flags uint8 @@ -113,13 +100,13 @@ func (e *frameEncoder) encode(event *publisher.Event) ([]byte, error) { return e.buf.Bytes(), nil } -func newDecoder() *decoder { - d := &decoder{} +func newDecoder() *eventDecoder { + d := &eventDecoder{} d.reset() return d } -func (d *decoder) reset() { +func (d *eventDecoder) reset() { unfolder, err := gotype.NewUnfolder(nil) if err != nil { panic(err) // can not happen @@ -130,7 +117,7 @@ func (d *decoder) reset() { } // Buffer prepares the read buffer to hold the next event of n bytes. -func (d *decoder) Buffer(n int) []byte { +func (d *eventDecoder) Buffer(n int) []byte { if cap(d.buf) > n { d.buf = d.buf[:n] } else { @@ -139,7 +126,7 @@ func (d *decoder) Buffer(n int) []byte { return d.buf } -func (d *decoder) Decode() (publisher.Event, error) { +func (d *eventDecoder) Decode() (publisher.Event, error) { var ( to entry err error diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 18ca406119c..657ce037cdf 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -121,7 +121,7 @@ func (wl *writerLoop) processRequest(request writerLoopRequest) []writerSegmentM wl.currentSegment = frameRequest.segment file, err := wl.currentSegment.getWriter(wl.settings) if err != nil { - wl.logger.Errorf("Couldn't open new segment file: %w", err) + wl.logger.Errorf("Couldn't open new segment file: %v", err) // TODO: retry, etc } wl.outputFile = file From 4a22ccd51fb2efb07bfb90d6730bf3b1a89f280e Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 8 Sep 2020 15:17:57 -0400 Subject: [PATCH 40/91] move frame-wrapping responsibility to the writer loop --- libbeat/publisher/queue/diskqueue/core_loop.go | 6 +++--- libbeat/publisher/queue/diskqueue/frames.go | 4 ++++ libbeat/publisher/queue/diskqueue/segments.go | 13 ------------- libbeat/publisher/queue/diskqueue/writer_loop.go | 9 +++++++++ 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index c2757269c13..7680c522f7e 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -124,7 +124,7 @@ func (cl *coreLoop) handleProducerWriteRequest(request producerWriteRequest) { // Pathological case checking: make sure the incoming frame isn't bigger // than an entire segment all by itself (as long as it isn't, it is // guaranteed to eventually enter the queue assuming no disk errors). - frameSize := uint64(len(request.frame.serialized)) + frameSize := request.frame.sizeOnDisk() if cl.queue.settings.MaxSegmentSize < frameSize { cl.queue.logger.Warnf( "Rejecting event with size %v because the maximum segment size is %v", @@ -138,7 +138,7 @@ func (cl *coreLoop) handleProducerWriteRequest(request producerWriteRequest) { // already accepted). pendingBytes := uint64(0) for _, request := range cl.pendingFrames { - pendingBytes += uint64(len(request.frame.serialized)) + pendingBytes += request.frame.sizeOnDisk() } currentSize := pendingBytes + cl.queue.segments.sizeOnDisk() // cl.queue.logger.Debugf( @@ -386,7 +386,7 @@ func (cl *coreLoop) enqueueProducerFrame(frame *writeFrame) { if len(dq.segments.writing) > 0 { segment = dq.segments.writing[len(dq.segments.writing)-1] } - frameLen := segmentOffset(len(frame.serialized)) + frameLen := segmentOffset(frame.sizeOnDisk()) // If segment is nil, or the new segment exceeds its bounds, // we need to create a new writing segment. if segment == nil || diff --git a/libbeat/publisher/queue/diskqueue/frames.go b/libbeat/publisher/queue/diskqueue/frames.go index 93671eaa77f..d2b03ea416b 100644 --- a/libbeat/publisher/queue/diskqueue/frames.go +++ b/libbeat/publisher/queue/diskqueue/frames.go @@ -66,3 +66,7 @@ type readFrame struct { const frameHeaderSize = 4 const frameFooterSize = 8 const frameMetadataSize = frameHeaderSize + frameFooterSize + +func (frame writeFrame) sizeOnDisk() uint64 { + return uint64(len(frame.serialized) + frameMetadataSize) +} diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 4bf88af7e95..6b0d02aea85 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -64,13 +64,6 @@ type diskQueueSegments struct { nextID segmentID } -// Every data frame read from the queue is assigned a unique sequential -// integer, which is used to keep track of which frames have been -// acknowledged. -// This id is not stable between restarts; the value 0 is always assigned -// to the oldest remaining frame on startup. -type frameID uint64 - // segmentOffset is a byte index into the segment's data region. // An offset of 0 means the first byte after the segment file header. type segmentOffset uint64 @@ -118,12 +111,6 @@ const ( ChecksumTypeCRC32 ) -// Each data frame has a 32-bit length in the header, and a 32-bit checksum -// and a duplicate 32-bit length in the footer. -const frameHeaderSize = 4 -const frameFooterSize = 8 -const frameMetadataSize = frameHeaderSize + frameFooterSize - // Each segment header has a 32-bit version and a 32-bit checksum type. const segmentHeaderSize = 8 diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 657ce037cdf..aa9aa1d6f84 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -19,6 +19,7 @@ package diskqueue import ( "bytes" + "encoding/binary" "os" "github.com/elastic/beats/v7/libbeat/logp" @@ -129,6 +130,9 @@ func (wl *writerLoop) processRequest(request writerLoopRequest) []writerSegmentM // We have the data and a file to write it to. We are now committed // to writing this block unless the queue is closed in the meantime. + frameSize := uint32(frameRequest.frame.sizeOnDisk()) + binary.Write(wl.outputFile, binary.LittleEndian, frameSize) + bytesWritten += 4 n, err := wl.outputFile.Write(frameRequest.frame.serialized) bytesWritten += int64(n) // TODO: retry forever if there is an error or n isn't the right @@ -136,6 +140,11 @@ func (wl *writerLoop) processRequest(request writerLoopRequest) []writerSegmentM if err != nil { wl.logger.Errorf("Couldn't write pending data to disk: %w", err) } + // empty crc + binary.Write(wl.outputFile, binary.LittleEndian, uint32(0)) + // duplicate length + binary.Write(wl.outputFile, binary.LittleEndian, frameSize) + bytesWritten += 8 } if bytesWritten > 0 { segments = append(segments, writerSegmentMetadata{ From d4e1dcdf6d214840f421ee456101c57ee02d1010 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 8 Sep 2020 15:50:08 -0400 Subject: [PATCH 41/91] add event deserialization --- libbeat/publisher/queue/diskqueue/checksum.go | 51 +++++++++++++++++++ libbeat/publisher/queue/diskqueue/queue.go | 3 +- .../publisher/queue/diskqueue/reader_loop.go | 37 +++++++++----- libbeat/publisher/queue/diskqueue/segments.go | 13 ----- .../publisher/queue/diskqueue/serialize.go | 29 +++-------- 5 files changed, 84 insertions(+), 49 deletions(-) create mode 100644 libbeat/publisher/queue/diskqueue/checksum.go diff --git a/libbeat/publisher/queue/diskqueue/checksum.go b/libbeat/publisher/queue/diskqueue/checksum.go new file mode 100644 index 00000000000..80f067582c4 --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/checksum.go @@ -0,0 +1,51 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +import ( + "encoding/binary" + "hash/crc32" +) + +// ChecksumType specifies what checksum algorithm the queue should use to +// verify its data frames. +type ChecksumType uint32 + +// ChecksumTypeNone: Don't compute or verify checksums. +// ChecksumTypeCRC32: Compute the checksum with the Go standard library's +// "hash/crc32" package. +const ( + ChecksumTypeNone = iota + + ChecksumTypeCRC32 +) + +func computeChecksum(data []byte, checksumType ChecksumType) uint32 { + switch checksumType { + case ChecksumTypeNone: + return 0 + case ChecksumTypeCRC32: + hash := crc32.NewIEEE() + frameLength := uint32(len(data) + frameMetadataSize) + binary.Write(hash, binary.LittleEndian, &frameLength) + hash.Write(data) + return hash.Sum32() + default: + panic("segmentReader: invalid checksum type") + } +} diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index d7b8fff70ad..2038b81cbf6 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -187,6 +187,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { requestChan: make(chan readerLoopRequest, 1), responseChan: make(chan readerLoopResponse), output: make(chan *readFrame, 20), // TODO: customize this buffer size + decoder: newEventDecoder(), } go func() { readerLoop.run() @@ -314,7 +315,7 @@ func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { return &diskQueueProducer{ queue: dq, config: cfg, - encoder: newFrameEncoder(), + encoder: newEventEncoder(), } } diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 861c89348df..5f84db340d8 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -62,6 +62,10 @@ type readerLoop struct { // the reader to read ahead and cache pending frames before a consumer // explicitly requests them. output chan *readFrame + + // The helper object to deserialize binary blobs from the queue into + // publisher.Event objects that can be returned in a readFrame. + decoder *eventDecoder } func (rl *readerLoop) run() { @@ -103,19 +107,13 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon targetLength := int64(request.endOffset - request.startOffset) for { remainingLength := targetLength - byteCount - /*if byteCount+frame.bytesOnDisk > targetLength { - // Something is wrong, read requests must end on a segment boundary. - return readResponse{ - frameCount: frameCount, - byteCount: byteCount, - } - }*/ - // Try to read the next, clipping to the length we were told to read. + // Try to read the next frame, clipping to the given bound. // If the next frame extends past this boundary, nextFrame will return // an error. frame, err := nextFrame( - handle, remainingLength, request.segment.header.checksumType) + handle, remainingLength, rl.decoder, + request.segment.header.checksumType) if frame != nil { // We've read the frame, try sending it to the output channel. select { @@ -163,7 +161,10 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon } func nextFrame( - handle *os.File, maxLength int64, checksumType ChecksumType, + handle *os.File, + maxLength int64, + decoder *eventDecoder, + checksumType ChecksumType, ) (*readFrame, error) { // Ensure we are allowed to read the frame header. if maxLength < frameHeaderSize { @@ -195,7 +196,8 @@ func nextFrame( // Read the actual frame data dataLength := frameLength - frameMetadataSize - bytes := make([]byte, dataLength) + bytes := decoder.Buffer(int(dataLength)) + //bytes := make([]byte, dataLength) _, err = reader.Read(bytes) if err != nil { return nil, fmt.Errorf("Couldn't read data frame: %w", err) @@ -224,8 +226,19 @@ func nextFrame( frameLength, duplicateLength) } - // TODO: deserialize + event, err := decoder.Decode() + if err != nil { + // TODO: Unlike errors in the segment or frame metadata, this is entirely + // a problem in the event [de]serialization which may be isolated. + // Rather than pass this error back to the read request, which discards + // the rest of the segment, we should just log the error and advance to + // the next frame, which is likely still valid. + return nil, err + } + fmt.Printf("Decoded event from frame: %v\n", event) + frame := &readFrame{ + event: event, bytesOnDisk: int64(frameLength), } diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 6b0d02aea85..a9f3ce2c8bc 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -98,19 +98,6 @@ type segmentHeader struct { checksumType ChecksumType } -// ChecksumType specifies what checksum algorithm the queue should use to -// verify its data frames. -type ChecksumType uint32 - -// ChecksumTypeNone: Don't compute or verify checksums. -// ChecksumTypeCRC32: Compute the checksum with the Go standard library's -// "hash/crc32" package. -const ( - ChecksumTypeNone = iota - - ChecksumTypeCRC32 -) - // Each segment header has a 32-bit version and a 32-bit checksum type. const segmentHeaderSize = 8 diff --git a/libbeat/publisher/queue/diskqueue/serialize.go b/libbeat/publisher/queue/diskqueue/serialize.go index ff4c6a5ee5d..434ad9e1b53 100644 --- a/libbeat/publisher/queue/diskqueue/serialize.go +++ b/libbeat/publisher/queue/diskqueue/serialize.go @@ -22,8 +22,6 @@ package diskqueue import ( "bytes" - "encoding/binary" - "hash/crc32" "time" "github.com/elastic/beats/v7/libbeat/beat" @@ -54,7 +52,7 @@ type entry struct { Fields common.MapStr } -func newFrameEncoder() *eventEncoder { +func newEventEncoder() *eventEncoder { e := &eventEncoder{} e.reset() return e @@ -100,7 +98,7 @@ func (e *eventEncoder) encode(event *publisher.Event) ([]byte, error) { return e.buf.Bytes(), nil } -func newDecoder() *eventDecoder { +func newEventDecoder() *eventDecoder { d := &eventDecoder{} d.reset() return d @@ -128,15 +126,14 @@ func (d *eventDecoder) Buffer(n int) []byte { func (d *eventDecoder) Decode() (publisher.Event, error) { var ( - to entry - err error - contents = d.buf[1:] + to entry + err error ) d.unfolder.SetTarget(&to) defer d.unfolder.Reset() - err = d.parser.Parse(contents) + err = d.parser.Parse(d.buf) if err != nil { d.reset() // reset parser just in case @@ -144,6 +141,7 @@ func (d *eventDecoder) Decode() (publisher.Event, error) { } var flags publisher.EventFlags + // TODO: handle guaranteed send? /*if (to.Flags & flagGuaranteed) != 0 { flags |= publisher.GuaranteedSend }*/ @@ -157,18 +155,3 @@ func (d *eventDecoder) Decode() (publisher.Event, error) { }, }, nil } - -func computeChecksum(data []byte, checksumType ChecksumType) uint32 { - switch checksumType { - case ChecksumTypeNone: - return 0 - case ChecksumTypeCRC32: - hash := crc32.NewIEEE() - frameLength := uint32(len(data) + frameMetadataSize) - binary.Write(hash, binary.LittleEndian, &frameLength) - hash.Write(data) - return hash.Sum32() - default: - panic("segmentReader: invalid checksum type") - } -} From 0e1249558315a630aee688be95f1d5d590cdef85 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 8 Sep 2020 16:06:28 -0400 Subject: [PATCH 42/91] properly assign frame ids after reading --- .../publisher/queue/diskqueue/reader_loop.go | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 5f84db340d8..0d02c92d389 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -66,6 +66,11 @@ type readerLoop struct { // The helper object to deserialize binary blobs from the queue into // publisher.Event objects that can be returned in a readFrame. decoder *eventDecoder + + // The id that will be assigned to the next successfully-read frame. + // Always starts from 0; this is just to track which frames have been + // acknowledged, and doesn't need any consistency between runs. + nextFrameID frameID } func (rl *readerLoop) run() { @@ -111,9 +116,7 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon // Try to read the next frame, clipping to the given bound. // If the next frame extends past this boundary, nextFrame will return // an error. - frame, err := nextFrame( - handle, remainingLength, rl.decoder, - request.segment.header.checksumType) + frame, err := rl.nextFrame(handle, remainingLength) if frame != nil { // We've read the frame, try sending it to the output channel. select { @@ -160,11 +163,8 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon } } -func nextFrame( - handle *os.File, - maxLength int64, - decoder *eventDecoder, - checksumType ChecksumType, +func (rl *readerLoop) nextFrame( + handle *os.File, maxLength int64, ) (*readFrame, error) { // Ensure we are allowed to read the frame header. if maxLength < frameHeaderSize { @@ -196,7 +196,7 @@ func nextFrame( // Read the actual frame data dataLength := frameLength - frameMetadataSize - bytes := decoder.Buffer(int(dataLength)) + bytes := rl.decoder.Buffer(int(dataLength)) //bytes := make([]byte, dataLength) _, err = reader.Read(bytes) if err != nil { @@ -226,7 +226,7 @@ func nextFrame( frameLength, duplicateLength) } - event, err := decoder.Decode() + event, err := rl.decoder.Decode() if err != nil { // TODO: Unlike errors in the segment or frame metadata, this is entirely // a problem in the event [de]serialization which may be isolated. @@ -239,8 +239,10 @@ func nextFrame( frame := &readFrame{ event: event, + id: rl.nextFrameID, bytesOnDisk: int64(frameLength), } + rl.nextFrameID++ return frame, nil } From 56fe5ba3c5d6697600f11f4639b044fe68b5429e Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Wed, 9 Sep 2020 14:25:08 -0400 Subject: [PATCH 43/91] remove obsolete code --- .../publisher/queue/diskqueue/writer_loop.go | 92 ------------------- 1 file changed, 92 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index aa9aa1d6f84..66b71dd0731 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -18,7 +18,6 @@ package diskqueue import ( - "bytes" "encoding/binary" "os" @@ -155,97 +154,6 @@ func (wl *writerLoop) processRequest(request writerLoopRequest) []writerSegmentM return segments } -// frameForContent wraps the given content buffer in a -// frame header / footer and returns the resulting larger buffer. -func frameForContent( - frameContent []byte, checksumType ChecksumType, -) bytes.Buffer { - buf := bytes.Buffer{} - //checksum := computeChecksum(frameContent, checksumType) - /*buf - frameLength := len(frameContent) + frameMetadataSize; - frameBytes := make([]byte, frameLength) - frameWriter := - binary.Write(reader.raw, binary.LittleEndian, &frameLength)*/ - return buf -} - -/*type writerState struct { - // The open file handle for the segment currently being written. - // This should be non-nil if and only if diskQueue.segments.writing is. - file *os.File - filePosition int64 -}*/ - -/*func handleFrame(dq *diskQueue, state *writerState, frame bytes.Buffer) { - dq.segments.Lock() - defer dq.segments.Unlock() - - frameLen := uint64(frame.Len()) - // If there isn't enough space left in the current segment, close the - // segment's file handle and move it to the reading list. - if dq.segments.writing != nil && - dq.segments.writing.size+frameLen > dq.settings.MaxSegmentSize { - // TODO: make reasonable efforts to sync to disk. - state.file.Close() - dq.segments.reading = append(dq.segments.reading, dq.segments.writing) - dq.segments.writing = nil - } - - if dq.segments.writing == nil { - // There is no active writing segment, create one. - // TODO: (actually create one) - } - - // TODO: try to delete dq.segments.acked - - currentSize := dq.segments.sizeOnDiskWithLock() - // Block (releasing the dq.segments lock) until - // currentSize + frameLen <= dq.settings.MaxBufferSize - for currentSize+frameLen > dq.settings.MaxBufferSize { - // Wait for some space to be freed. - dq.segments.segmentDeletedCond.Wait() - if dq.closedForWrite.Load() { - // The queue is closed, abort - } - } - - // We now have a frame we want to write to disk, and enough free capacity - // to write it. - writeAll(state.file, frame.Bytes()) -}*/ - -// The writer loop's job is to continuously read a data frame from the -// queue's intake channel, if there is one, and write it to disk. -// It continues until the intake channel is empty or -/*func (dq *diskQueue) writerLoop() { - defer dq.waitGroup.Done() - //logger := dq.settings.Logger.Named("writerLoop") - state := &writerState{} - - for { - if dq.abort.Load() { - // We are aborting, ignore any remaining buffered frames. - return - } - select { - case frameContent := <-dq.inChan: - if frameContent == nil { - // The channel has been drained, the writer loop should shut down. - return - } - frameBuffer := frameForContent(frameContent, dq.settings.ChecksumType) - handleFrame(dq, state, frameBuffer) - if !dq.abort.Load() { - // As long as we aren't aborting, continue processing any pending - // frames. - continue - } - case <-dq.done: - } - // We've processed - } -}*/ /* func writeAll(writer io.Writer, p []byte) (int, error) { var N int From eaf6e2bbba4d849c5728e365f88d54410453cf6e Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Wed, 9 Sep 2020 14:44:16 -0400 Subject: [PATCH 44/91] compute / verify real checksums --- .../publisher/queue/diskqueue/reader_loop.go | 67 ++----------------- .../publisher/queue/diskqueue/writer_loop.go | 14 ++-- 2 files changed, 14 insertions(+), 67 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 0d02c92d389..617ba1f8382 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -197,7 +197,6 @@ func (rl *readerLoop) nextFrame( // Read the actual frame data dataLength := frameLength - frameMetadataSize bytes := rl.decoder.Buffer(int(dataLength)) - //bytes := make([]byte, dataLength) _, err = reader.Read(bytes) if err != nil { return nil, fmt.Errorf("Couldn't read data frame: %w", err) @@ -209,10 +208,13 @@ func (rl *readerLoop) nextFrame( if err != nil { return nil, fmt.Errorf("Couldn't read data frame checksum: %w", err) } - // TODO: validate checksum - /*if computeChecksum(data, reader.checksumType) != checksum { - return nil, fmt.Errorf("Disk queue: bad data frame checksum") - }*/ + if rl.settings.ChecksumType != ChecksumTypeNone { + expected := computeChecksum(bytes, rl.settings.ChecksumType) + if checksum != expected { + return nil, fmt.Errorf( + "Data frame checksum mismatch (%x != %x)", checksum, expected) + } + } var duplicateLength uint32 err = binary.Read(reader, binary.LittleEndian, &duplicateLength) @@ -274,58 +276,3 @@ func (r autoRetryReader) Read(p []byte) (int, error) { func readErrorIsRetriable(err error) bool { return errors.Is(err, syscall.EINTR) || errors.Is(err, syscall.EAGAIN) } - -/*func (dq *diskQueue) readerLoop() { - curFrameID := frameID(0) - logger := dq.settings.Logger.Named("readerLoop") - for { - dq.frameWrittenCond.Wait() - reader, errs := dq.nextSegmentReader() - for _, err := range errs { - // Errors encountered while reading should be logged. - logger.Error(err) - } - if reader == nil { - // We couldn't find a readable segment, wait for a new - // data frame to be written. - dq.frameWrittenCond.Wait() - if dq.closedForRead.Load() { - // The queue has been closed, shut down. - // TODO: cleanup (write the final read position) - return - } - continue - } - // If we made it here, we have a nonempty reader and we want - // to send all its frames to dq.outChan. - framesRead := int64(0) - for { - bytes, err := reader.nextDataFrame() - if err != nil { - // An error at this stage generally means there has been - // data corruption. For now, in this case we log the - // error, then discard any remaining frames. When all - // successfully read frames have been acknowledged, we - // delete the underlying file. - break - } - if bytes == nil { - // If bytes is nil with no error, we've reached the end - // of this segmentReader. Update the segment's frame count. - break - } - framesRead++ - output := diskQueueOutput{ - data: bytes, - segment: reader.segment, - frame: curFrameID, - } - select { - case dq.outChan <- output: - curFrameID++ - case <-dq.done: - } - } - reader.segment.framesRead += framesRead - } -}*/ diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 66b71dd0731..bd711b1e27a 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -131,19 +131,19 @@ func (wl *writerLoop) processRequest(request writerLoopRequest) []writerSegmentM // to writing this block unless the queue is closed in the meantime. frameSize := uint32(frameRequest.frame.sizeOnDisk()) binary.Write(wl.outputFile, binary.LittleEndian, frameSize) - bytesWritten += 4 - n, err := wl.outputFile.Write(frameRequest.frame.serialized) - bytesWritten += int64(n) // TODO: retry forever if there is an error or n isn't the right // length. + n, err := wl.outputFile.Write(frameRequest.frame.serialized) if err != nil { wl.logger.Errorf("Couldn't write pending data to disk: %w", err) } - // empty crc - binary.Write(wl.outputFile, binary.LittleEndian, uint32(0)) - // duplicate length + // Compute / write the frame's checksum + checksum := computeChecksum( + frameRequest.frame.serialized, wl.settings.ChecksumType) + binary.Write(wl.outputFile, binary.LittleEndian, checksum) + // Write the frame footer's (duplicate) length binary.Write(wl.outputFile, binary.LittleEndian, frameSize) - bytesWritten += 8 + bytesWritten += int64(n) + frameMetadataSize } if bytesWritten > 0 { segments = append(segments, writerSegmentMetadata{ From 5f0376d8e81409117fb392dc84e442d69f0021bc Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Wed, 9 Sep 2020 14:55:59 -0400 Subject: [PATCH 45/91] don't read everything twice --- .../publisher/queue/diskqueue/reader_loop.go | 37 +++++++++---------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 617ba1f8382..7b4476150da 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -82,16 +82,16 @@ func (rl *readerLoop) run() { return } response := rl.processRequest(request) - fmt.Printf("\033[0;32mread response: read %d frames and %d bytes\033[0m\n", response.frameCount, response.byteCount) - if response.err != nil { - fmt.Printf("\033[0;32mresponse had err: %v\033[0m\n", response.err) - } - rl.responseChan <- rl.processRequest(request) + //fmt.Printf("\033[0;32mread response: read %d frames and %d bytes\033[0m\n", response.frameCount, response.byteCount) + // if response.err != nil { + // fmt.Printf("\033[0;32mresponse had err: %v\033[0m\n", response.err) + // } + rl.responseChan <- response } } func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopResponse { - fmt.Printf("\033[0;32mprocessRequest(segment %d from %d to %d)\033[0m\n", request.segment.id, request.startOffset, request.endOffset) + fmt.Printf("\033[0;32mreaderLoop.processRequest(segment %d from %d to %d)\033[0m\n", request.segment.id, request.startOffset, request.endOffset) defer time.Sleep(time.Second) @@ -177,8 +177,7 @@ func (rl *readerLoop) nextFrame( var frameLength uint32 err := binary.Read(reader, binary.LittleEndian, &frameLength) if err != nil { - return nil, fmt.Errorf( - "Couldn't read next frame length: %w", err) + return nil, fmt.Errorf("Couldn't read data frame header: %w", err) } // If the frame extends past the area we were told to read, return an error. @@ -199,7 +198,7 @@ func (rl *readerLoop) nextFrame( bytes := rl.decoder.Buffer(int(dataLength)) _, err = reader.Read(bytes) if err != nil { - return nil, fmt.Errorf("Couldn't read data frame: %w", err) + return nil, fmt.Errorf("Couldn't read data frame content: %w", err) } // Read the footer (checksum + duplicate length) @@ -219,25 +218,25 @@ func (rl *readerLoop) nextFrame( var duplicateLength uint32 err = binary.Read(reader, binary.LittleEndian, &duplicateLength) if err != nil { - return nil, fmt.Errorf( - "Disk queue couldn't read trailing frame length: %w", err) + return nil, fmt.Errorf("Couldn't read data frame footer: %w", err) } if duplicateLength != frameLength { return nil, fmt.Errorf( - "Disk queue: inconsistent frame length (%d vs %d)", + "Inconsistent data frame length (%d vs %d)", frameLength, duplicateLength) } event, err := rl.decoder.Decode() if err != nil { - // TODO: Unlike errors in the segment or frame metadata, this is entirely - // a problem in the event [de]serialization which may be isolated. - // Rather than pass this error back to the read request, which discards - // the rest of the segment, we should just log the error and advance to - // the next frame, which is likely still valid. - return nil, err + // Unlike errors in the segment or frame metadata, this is entirely + // a problem in the event [de]serialization which may be isolated (i.e. + // may not indicate data corruption in the segment). + // TODO: Rather than pass this error back to the read request, which + // discards the rest of the segment, we should just log the error and + // advance to the next frame, which is likely still valid (especially + // if checksums are enabled). + return nil, fmt.Errorf("Couldn't decode data frame: %w", err) } - fmt.Printf("Decoded event from frame: %v\n", event) frame := &readFrame{ event: event, From d63cbf0bb0a7a6176f9c382775db31c7d49914de Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Wed, 9 Sep 2020 16:05:23 -0400 Subject: [PATCH 46/91] Fix ACKing / deletion --- libbeat/publisher/queue/diskqueue/consumer.go | 18 +++++++++++------- libbeat/publisher/queue/diskqueue/core_loop.go | 6 ++++-- .../publisher/queue/diskqueue/deleter_loop.go | 16 ++++++++++++++-- libbeat/publisher/queue/diskqueue/queue.go | 7 +++++-- .../publisher/queue/diskqueue/reader_loop.go | 8 +++++--- libbeat/publisher/queue/diskqueue/segments.go | 9 --------- .../publisher/queue/diskqueue/writer_loop.go | 3 ++- 7 files changed, 41 insertions(+), 26 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/consumer.go b/libbeat/publisher/queue/diskqueue/consumer.go index 46df08551b7..3fe742a3cb2 100644 --- a/libbeat/publisher/queue/diskqueue/consumer.go +++ b/libbeat/publisher/queue/diskqueue/consumer.go @@ -99,6 +99,8 @@ func (batch *diskQueueBatch) Events() []publisher.Event { // and we don't want the core loop to bottleneck on manipulating // a potentially large dictionary, so we use a lock and let // consumer threads handle most of the processing themselves. +// TODO: this shouldn't really be a dictionary, use a bitfield or +// something more efficient. func (batch *diskQueueBatch) ACK() { dq := batch.queue dq.ackLock.Lock() @@ -106,13 +108,15 @@ func (batch *diskQueueBatch) ACK() { for _, frameID := range batch.ackIDs { dq.acked[frameID] = true } - ackedCount := 0 - for ; dq.acked[dq.ackedUpTo]; dq.ackedUpTo++ { - delete(dq.acked, dq.ackedUpTo) - ackedCount++ - } - if ackedCount > 0 { - dq.ackedUpTo += frameID(ackedCount) + if dq.acked[dq.ackedUpTo] { + for ; dq.acked[dq.ackedUpTo]; dq.ackedUpTo++ { + delete(dq.acked, dq.ackedUpTo) + } + // It would be considerate to send this less frequently, so + // as not to bother the core loop with messages that are + // usually no-ops. + // TODO: only inform the core loop when we cross a segment + // boundary. dq.consumerAckChan <- dq.ackedUpTo } } diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 7680c522f7e..45232bfabec 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -263,11 +263,12 @@ func (cl *coreLoop) handleConsumerAck(ackedUpTo frameID) { ackedSegmentCount := 0 for ; ackedSegmentCount < len(acking); ackedSegmentCount++ { segment := acking[ackedSegmentCount] - endFrame += frameID(segment.framesRead) - if endFrame > ackedUpTo { + if endFrame+frameID(segment.framesRead) > ackedUpTo { // This segment is still waiting for acks, we're done. break } + // Otherwise, advance the ending frame ID. + endFrame += frameID(segment.framesRead) } if ackedSegmentCount > 0 { // Move fully acked segments to the acked list and remove them @@ -275,6 +276,7 @@ func (cl *coreLoop) handleConsumerAck(ackedUpTo frameID) { cl.queue.segments.acked = append(cl.queue.segments.acked, acking[:ackedSegmentCount]...) cl.queue.segments.acking = acking[ackedSegmentCount:] + // Advance oldestFrameID past the segments we just removed. cl.oldestFrameID = endFrame cl.maybeDeleteAcked() } diff --git a/libbeat/publisher/queue/diskqueue/deleter_loop.go b/libbeat/publisher/queue/diskqueue/deleter_loop.go index 9813bc70416..1c22ee628f0 100644 --- a/libbeat/publisher/queue/diskqueue/deleter_loop.go +++ b/libbeat/publisher/queue/diskqueue/deleter_loop.go @@ -20,6 +20,7 @@ package diskqueue import ( "errors" "os" + "time" ) type deleterLoop struct { @@ -65,14 +66,25 @@ func (dl *deleterLoop) run() { // We ignore errors caused by the file not existing: this shouldn't // happen, but it is still safe to report it as successfully removed. if err == nil || errors.Is(err, os.ErrNotExist) { - errorList = append(errorList, err) - } else { deleted[segment] = true + } else { + errorList = append(errorList, err) } } dl.responseChan <- deleterLoopResponse{ deleted: deleted, errors: errorList, } + if len(request.segments) > 0 && len(deleted) == 0 { + // If we were asked to delete segments but could not delete + // _any_ of them, we haven't made progress. Returning an error + // will log the issue and retry, but in this situation we + // want to delay before retrying. The core loop itself can't + // delay (it can never sleep or block), so we handle the + // delay here, by ignoring the next request until the retry + // interval has passed. + // TODO: make the retry interval configurable. + time.Sleep(time.Second) + } } } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 2038b81cbf6..2294e7f3823 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -209,7 +209,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { deleterLoop := &deleterLoop{ settings: &settings, - requestChan: make(chan deleterLoopRequest), + requestChan: make(chan deleterLoopRequest, 1), responseChan: make(chan deleterLoopResponse), } go func() { @@ -320,5 +320,8 @@ func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { } func (dq *diskQueue) Consumer() queue.Consumer { - panic("TODO: not implemented") + return &diskQueueConsumer{ + queue: dq, + closed: false, + } } diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 7b4476150da..0718280ab62 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -24,7 +24,6 @@ import ( "io" "os" "syscall" - "time" ) type readerLoopRequest struct { @@ -91,12 +90,15 @@ func (rl *readerLoop) run() { } func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopResponse { - fmt.Printf("\033[0;32mreaderLoop.processRequest(segment %d from %d to %d)\033[0m\n", request.segment.id, request.startOffset, request.endOffset) + //fmt.Printf("\033[0;32mreaderLoop.processRequest(segment %d from %d to %d)\033[0m\n", request.segment.id, request.startOffset, request.endOffset) - defer time.Sleep(time.Second) + //defer time.Sleep(time.Second) frameCount := int64(0) byteCount := int64(0) + // defer func() { + // fmt.Printf(" \033[0;32mread %d bytes in %d frames\033[0m\n", byteCount, frameCount) + // }() // Open the file and seek to the starting position. handle, err := request.segment.getReader(rl.settings) diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index a9f3ce2c8bc..6b08c75d38f 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -149,24 +149,19 @@ func (segment *queueSegment) sizeOnDisk() uint64 { func (segment *queueSegment) getReader( queueSettings *Settings, ) (*os.File, error) { - fmt.Printf("\033[94mgetReader(%v)\033[0m\n", segment.id) - path := queueSettings.segmentPath(segment.id) file, err := os.Open(path) if err != nil { - fmt.Printf("\033[94mfailed: %v\033[0m\n", err) return nil, fmt.Errorf( "Couldn't open segment %d: %w", segment.id, err) } header, err := readSegmentHeader(file) if err != nil { - fmt.Printf("\033[94mfailed (header): %v\033[0m\n", err) file.Close() return nil, fmt.Errorf("Couldn't read segment header: %w", err) } segment.header = header - fmt.Printf("\033[94msuccess\033[0m\n") return file, nil } @@ -174,11 +169,9 @@ func (segment *queueSegment) getReader( func (segment *queueSegment) getWriter( queueSettings *Settings, ) (*os.File, error) { - fmt.Printf("\033[0;32mgetWriter(%v)\033[0m\n", segment.id) path := queueSettings.segmentPath(segment.id) file, err := os.OpenFile(path, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) if err != nil { - fmt.Printf("\033[0;32mfailed\033[0m\n") return nil, err } header := &segmentHeader{ @@ -187,10 +180,8 @@ func (segment *queueSegment) getWriter( } err = writeSegmentHeader(file, header) if err != nil { - fmt.Printf("\033[0;32mfailed (header)\033[0m\n") return nil, fmt.Errorf("Couldn't write segment header: %w", err) } - fmt.Printf("\033[0;32msuccess\033[0m\n") return file, nil } diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index bd711b1e27a..35851c7270b 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -106,7 +106,8 @@ func (wl *writerLoop) processRequest(request writerLoopRequest) []writerSegmentM // If the new segment doesn't match the last one, we need to open a new // file handle and possibly clean up the old one. if wl.currentSegment != frameRequest.segment { - wl.logger.Debugf("") + wl.logger.Debugf( + "Creating new segment file with id %v\n", frameRequest.segment.id) if wl.outputFile != nil { wl.outputFile.Close() wl.outputFile = nil From 98e61f0fd5f94454f87ef04f49a32051fb3a559e Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Wed, 9 Sep 2020 16:26:47 -0400 Subject: [PATCH 47/91] revert testing wrapper around filebeat --- filebeat/main.go | 17 +++-------------- 1 file changed, 3 insertions(+), 14 deletions(-) diff --git a/filebeat/main.go b/filebeat/main.go index 57fc5bc6703..e8f36acde01 100644 --- a/filebeat/main.go +++ b/filebeat/main.go @@ -18,15 +18,12 @@ package main import ( - "fmt" "os" - "github.com/elastic/beats/v7/libbeat/publisher/queue/diskqueue" + "github.com/elastic/beats/v7/filebeat/cmd" + inputs "github.com/elastic/beats/v7/filebeat/input/default-inputs" ) -//"github.com/elastic/beats/v7/filebeat/cmd" -//inputs "github.com/elastic/beats/v7/filebeat/input/default-inputs" - // The basic model of execution: // - input: finds files in paths/globs to harvest, starts harvesters // - harvester: reads a file, sends events to the spooler @@ -36,15 +33,7 @@ import ( // Finally, input uses the registrar information, on restart, to // determine where in each file to restart a harvester. func main() { - //if err := cmd.Filebeat(inputs.Init).Execute(); err != nil { - // os.Exit(1) - //} - settings := diskqueue.Settings{} - queue, err := diskqueue.NewQueue(settings) - if err != nil { - fmt.Printf("queue creation error: %v", err) + if err := cmd.Filebeat(inputs.Init).Execute(); err != nil { os.Exit(1) } - queue.Close() - fmt.Printf("Hello") } From e48c815d8b4f232721aefe817fb8dfa6a05b772e Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 10 Sep 2020 11:23:24 -0400 Subject: [PATCH 48/91] filebeat ingestion is working! --- libbeat/publisher/includes/includes.go | 1 + libbeat/publisher/queue/diskqueue/config.go | 3 ++- libbeat/publisher/queue/diskqueue/core_loop.go | 7 +++++++ libbeat/publisher/queue/diskqueue/queue.go | 4 ++++ libbeat/publisher/queue/diskqueue/reader_loop.go | 5 +++-- libbeat/publisher/queue/diskqueue/serialize.go | 7 ++++++- 6 files changed, 23 insertions(+), 4 deletions(-) diff --git a/libbeat/publisher/includes/includes.go b/libbeat/publisher/includes/includes.go index e6f3ded0bee..a14dd16d3ba 100644 --- a/libbeat/publisher/includes/includes.go +++ b/libbeat/publisher/includes/includes.go @@ -27,6 +27,7 @@ import ( _ "github.com/elastic/beats/v7/libbeat/outputs/kafka" _ "github.com/elastic/beats/v7/libbeat/outputs/logstash" _ "github.com/elastic/beats/v7/libbeat/outputs/redis" + _ "github.com/elastic/beats/v7/libbeat/publisher/queue/diskqueue" _ "github.com/elastic/beats/v7/libbeat/publisher/queue/memqueue" _ "github.com/elastic/beats/v7/libbeat/publisher/queue/spool" ) diff --git a/libbeat/publisher/queue/diskqueue/config.go b/libbeat/publisher/queue/diskqueue/config.go index 7463e868b2f..e9f54717043 100644 --- a/libbeat/publisher/queue/diskqueue/config.go +++ b/libbeat/publisher/queue/diskqueue/config.go @@ -52,10 +52,11 @@ func DefaultSettings() Settings { func SettingsForUserConfig(config *common.Config) (Settings, error) { userConfig := userConfig{} if err := config.Unpack(&userConfig); err != nil { + // TODO: report error / decide what to do return Settings{}, err } settings := DefaultSettings() settings.Path = userConfig.Path - return Settings{}, nil + return settings, nil } diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 45232bfabec..3f8bec7bc18 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -223,6 +223,13 @@ func (cl *coreLoop) handleReaderLoopResponse(response readerLoopResponse) { } segment.framesRead += response.frameCount + // If there was an error, report it. + if response.err != nil { + cl.queue.logger.Errorf( + "Error reading segment file %s: %v", + cl.queue.settings.segmentPath(segment.id), response.err) + } + // If there is more data to read, start a new read request. cl.maybeReadPending() } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 2294e7f3823..adb972cd2fe 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -21,6 +21,7 @@ import ( "fmt" "os" "path/filepath" + "runtime/debug" "sync" "github.com/elastic/beats/v7/libbeat/common" @@ -140,6 +141,7 @@ func queueFactory( // NewQueue returns a disk-based queue configured with the given logger // and settings, creating it if it doesn't exist. func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { + logger = logger.Named("diskqueue") logger.Debugf( "Initializing disk queue at path %v", settings.directoryPath()) @@ -320,6 +322,8 @@ func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { } func (dq *diskQueue) Consumer() queue.Consumer { + fmt.Printf("diskQueue.Consumer()\n") + debug.PrintStack() return &diskQueueConsumer{ queue: dq, closed: false, diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 0718280ab62..a1f10804eed 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -24,6 +24,7 @@ import ( "io" "os" "syscall" + "time" ) type readerLoopRequest struct { @@ -92,7 +93,7 @@ func (rl *readerLoop) run() { func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopResponse { //fmt.Printf("\033[0;32mreaderLoop.processRequest(segment %d from %d to %d)\033[0m\n", request.segment.id, request.startOffset, request.endOffset) - //defer time.Sleep(time.Second) + defer time.Sleep(time.Second) frameCount := int64(0) byteCount := int64(0) @@ -123,7 +124,7 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon // We've read the frame, try sending it to the output channel. select { case rl.output <- frame: - // Success! Increment the total for this request. + // Successfully sent! Increment the total for this request. frameCount++ byteCount += frame.bytesOnDisk case <-rl.requestChan: diff --git a/libbeat/publisher/queue/diskqueue/serialize.go b/libbeat/publisher/queue/diskqueue/serialize.go index 434ad9e1b53..6bd3f3e97c1 100644 --- a/libbeat/publisher/queue/diskqueue/serialize.go +++ b/libbeat/publisher/queue/diskqueue/serialize.go @@ -95,7 +95,12 @@ func (e *eventEncoder) encode(event *publisher.Event) ([]byte, error) { return nil, err } - return e.buf.Bytes(), nil + // Copy the encoded bytes to a new array owned by the caller. + bytes := e.buf.Bytes() + result := make([]byte, len(bytes)) + copy(result, bytes) + + return result, nil } func newEventDecoder() *eventDecoder { From 157564073b1c30e9e46bdff5464053f08d9e15cf Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 10 Sep 2020 12:35:15 -0400 Subject: [PATCH 49/91] clean up capacity checking / add maybeUnblockProducers --- .../publisher/queue/diskqueue/core_loop.go | 113 +++++++++++------- 1 file changed, 67 insertions(+), 46 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 3f8bec7bc18..ad1e63c87c8 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -107,20 +107,6 @@ func (cl *coreLoop) run() { } func (cl *coreLoop) handleProducerWriteRequest(request producerWriteRequest) { - if len(cl.blockedProducers) > 0 { - // If other requests are still waiting for space, then there - // definitely isn't enough for this one. - if request.shouldBlock { - // Blocked writes don't get a response until there is enough free - // space and they are moved to pendingWrites. - cl.blockedProducers = append(cl.blockedProducers, request) - } else { - // If the request is non-blocking, send immediate failure and discard it. - request.responseChan <- false - } - return - } - // Pathological case checking: make sure the incoming frame isn't bigger // than an entire segment all by itself (as long as it isn't, it is // guaranteed to eventually enter the queue assuming no disk errors). @@ -133,38 +119,26 @@ func (cl *coreLoop) handleProducerWriteRequest(request producerWriteRequest) { return } - // We will accept this request if there is enough capacity left in - // the queue (after accounting for the pending writes that were - // already accepted). - pendingBytes := uint64(0) - for _, request := range cl.pendingFrames { - pendingBytes += request.frame.sizeOnDisk() - } - currentSize := pendingBytes + cl.queue.segments.sizeOnDisk() - // cl.queue.logger.Debugf( - // "currentSize: %v frameSize: %v MaxBufferSize: %v", - // currentSize, frameSize, cl.queue.settings.MaxBufferSize) - if cl.queue.settings.MaxBufferSize > 0 && - currentSize+frameSize > cl.queue.settings.MaxBufferSize { - // The queue is too full. Either add the request to blockedWrites, + // If no one else is blocked waiting for queue capacity, and there is + // enough space, then we add the new frame and report success. + // Otherwise, we either add to the end of blockedProducers to wait for + // the requested space or report immediate failure, depending on the + // producer settings. + if len(cl.blockedProducers) == 0 && cl.canAcceptFrameOfSize(frameSize) { + // There is enough space for the new frame! Add it to the + // pending list and report success, then dispatch it to the + // writer loop if no other requests are outstanding. + cl.enqueueWriteFrame(request.frame) + request.responseChan <- true + cl.maybeWritePending() + } else { + // The queue is too full. Either add the request to blockedProducers, // or send an immediate reject. if request.shouldBlock { cl.blockedProducers = append(cl.blockedProducers, request) } else { request.responseChan <- false } - } else { - // There is enough space for the new frame! Add it to the - // pending list and report success, then dispatch it to the - // writer loop if no other requests are outstanding. - // Right now we accept any request if there is enough space for it - // on disk. High-throughput inputs may produce events faster than - // they can be written to disk, so it would make sense to - // additionally bound the amount of data in pendingWrites to some - // configurable limit to avoid out-of-memory errors. - cl.enqueueProducerFrame(request.frame) - request.responseChan <- true - cl.maybeWritePending() } } @@ -323,14 +297,14 @@ func (cl *coreLoop) handleShutdown() { // TODO: write final queue state to the metadata file. } -// If the pendingWrites list is nonempty, and there are no outstanding +// If the pendingFrames list is nonempty, and there are no outstanding // requests to the writer loop, send the next batch of frames. func (cl *coreLoop) maybeWritePending() { if cl.writing || len(cl.pendingFrames) == 0 { // Nothing to do right now return } - // Remove everything from pendingWrites and forward it to the writer loop. + // Remove everything from pendingFrames and forward it to the writer loop. requests := cl.pendingFrames cl.pendingFrames = nil @@ -381,13 +355,29 @@ func (cl *coreLoop) maybeDeleteAcked() { } } +// maybeUnblockProducers checks whether the queue has enough free space +// to accept any of the requests in the blockedProducers list, and if so +// accepts them in order and updates the list. func (cl *coreLoop) maybeUnblockProducers() { - // TODO: implement me + unblockedCount := 0 + for _, request := range cl.blockedProducers { + if !cl.canAcceptFrameOfSize(request.frame.sizeOnDisk()) { + // Not enough space for this frame, we're done. + break + } + // Add the frame to pendingFrames and report success. + cl.enqueueWriteFrame(request.frame) + request.responseChan <- true + unblockedCount++ + } + if unblockedCount > 0 { + cl.blockedProducers = cl.blockedProducers[unblockedCount:] + } } -// enqueueProducerFrame determines which segment an incoming frame should be -// written to and adds the result to pendingWrites. -func (cl *coreLoop) enqueueProducerFrame(frame *writeFrame) { +// enqueueWriteFrame determines which segment an incoming frame should be +// written to and adds the resulting segmentedFrame to pendingFrames. +func (cl *coreLoop) enqueueWriteFrame(frame *writeFrame) { dq := cl.queue // Start with the most recent writing segment if there is one. @@ -412,3 +402,34 @@ func (cl *coreLoop) enqueueProducerFrame(frame *writeFrame) { segment: segment, }) } + +// canAcceptFrameOfSize checks whether there is enough free space in the +// queue (subject to settings.MaxBufferSize) to accept a new frame with +// the given size. Size includes both the serialized data and the frame +// header / footer; the easy way to do this for a writeFrame is to pass +// in frame.sizeOnDisk(). +// Capacity calculations do not include requests in the blockedProducers +// list (that data is owned by its callers and we can't touch it until +// we are ready to respond). That allows this helper to be used both while +// handling producer requests and while deciding whether to unblock +// producers after free capacity increases. +// If we decide to add limits on how many events / bytes can be stored +// in pendingFrames (to avoid unbounded memory use if the input is faster +// than the disk), this is the function to modify. +func (cl *coreLoop) canAcceptFrameOfSize(frameSize uint64) bool { + if cl.queue.settings.MaxBufferSize == 0 { + // Currently we impose no limitations if the queue size is unbounded. + return true + } + + // Compute the current queue size. We accept if there is enough capacity + // left in the queue after accounting for the existing segments and the + // pending writes that were already accepted. + pendingBytes := uint64(0) + for _, request := range cl.pendingFrames { + pendingBytes += request.frame.sizeOnDisk() + } + currentSize := pendingBytes + cl.queue.segments.sizeOnDisk() + + return currentSize+frameSize <= cl.queue.settings.MaxBufferSize +} From d495aa8fff3536f8addbb1b8df37a1286d4cc95d Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 10 Sep 2020 12:53:46 -0400 Subject: [PATCH 50/91] move all dq.maybe... calls out of the handle... helpers --- .../publisher/queue/diskqueue/core_loop.go | 44 ++++++++++++------- 1 file changed, 27 insertions(+), 17 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index ad1e63c87c8..4ab2bfedeac 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -72,7 +72,8 @@ func (cl *coreLoop) run() { cl.queue.logger.Debug("Core loop starting up...") dq := cl.queue - // Wake up the reader loop if there are segments available to read. + // Wake up the reader loop if there are segments available to read + // (from a previous instantiation of the queue). cl.maybeReadPending() for { @@ -81,12 +82,20 @@ func (cl *coreLoop) run() { case producerWriteRequest := <-dq.producerWriteRequestChan: cl.handleProducerWriteRequest(producerWriteRequest) + // After a write request, there may be data ready to send to the + // writer loop. + cl.maybeWritePending() + case cancelRequest := <-dq.producerCancelRequestChan: + // TODO: this isn't really handled yet. cl.handleProducerCancelRequest(cancelRequest) case ackedUpTo := <-dq.consumerAckChan: cl.handleConsumerAck(ackedUpTo) + // After receiving new ACKs, a segment might be ready to delete. + cl.maybeDeleteAcked() + case <-dq.done: cl.handleShutdown() return @@ -95,13 +104,30 @@ func (cl *coreLoop) run() { case writerLoopResponse := <-dq.writerLoop.responseChan: cl.handleWriterLoopResponse(writerLoopResponse) + // The writer loop completed a request, so check if there is more + // data to be sent. + cl.maybeWritePending() + // We also check whether the reader loop is waiting for the data + // that was just written. + cl.maybeReadPending() + // Reader loop handling case readerLoopResponse := <-dq.readerLoop.responseChan: cl.handleReaderLoopResponse(readerLoopResponse) + // If there is more data to read, start a new read request. + cl.maybeReadPending() + // Deleter loop handling case deleterLoopResponse := <-dq.deleterLoop.responseChan: cl.handleDeleterLoopResponse(deleterLoopResponse) + + // If there are still files waiting to be deleted, send another request. + cl.maybeDeleteAcked() + + // If there were blocked producers waiting for more queue space, + // we might be able to unblock them now. + cl.maybeUnblockProducers() } } } @@ -130,7 +156,6 @@ func (cl *coreLoop) handleProducerWriteRequest(request producerWriteRequest) { // writer loop if no other requests are outstanding. cl.enqueueWriteFrame(request.frame) request.responseChan <- true - cl.maybeWritePending() } else { // The queue is too full. Either add the request to blockedProducers, // or send an immediate reject. @@ -163,11 +188,6 @@ func (cl *coreLoop) handleWriterLoopResponse(response writerLoopResponse) { append(cl.queue.segments.reading, metadata.segment) } } - - // New data is available, so we also check if we should send a new - // read request. - cl.maybeReadPending() - cl.maybeWritePending() } func (cl *coreLoop) handleReaderLoopResponse(response readerLoopResponse) { @@ -203,9 +223,6 @@ func (cl *coreLoop) handleReaderLoopResponse(response readerLoopResponse) { "Error reading segment file %s: %v", cl.queue.settings.segmentPath(segment.id), response.err) } - - // If there is more data to read, start a new read request. - cl.maybeReadPending() } func (cl *coreLoop) handleDeleterLoopResponse(response deleterLoopResponse) { @@ -226,12 +243,6 @@ func (cl *coreLoop) handleDeleterLoopResponse(response deleterLoopResponse) { dq.logger.Errorw("Couldn't delete old segment files", "errors", response.errors) } - // If there are still files to delete, send the next request. - cl.maybeDeleteAcked() - - // If there are blocked producers waiting for more queue space, this - // deletion might have unblocked them. - cl.maybeUnblockProducers() } func (cl *coreLoop) handleConsumerAck(ackedUpTo frameID) { @@ -259,7 +270,6 @@ func (cl *coreLoop) handleConsumerAck(ackedUpTo frameID) { cl.queue.segments.acking = acking[ackedSegmentCount:] // Advance oldestFrameID past the segments we just removed. cl.oldestFrameID = endFrame - cl.maybeDeleteAcked() } } From dd08e2cba1115e818c3cc22ba569ea2005781d58 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 10 Sep 2020 13:48:36 -0400 Subject: [PATCH 51/91] simplify / clarify the writer loop response logic --- .../publisher/queue/diskqueue/core_loop.go | 35 +++++---- .../publisher/queue/diskqueue/writer_loop.go | 71 ++++++++----------- 2 files changed, 52 insertions(+), 54 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 4ab2bfedeac..065e96ecb9f 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -56,6 +56,9 @@ type coreLoop struct { // pendingFrames is a list of all incoming data frames that have been // accepted by the queue and are waiting to be sent to the writer loop. + // Segment ids in this list always appear in sorted order, even between + // requests (that is, a frame added to this list always has segment id + // at least as high as every previous frame that has ever been added). pendingFrames []segmentedFrame // blockedProducers is a list of all producer write requests that are @@ -168,25 +171,29 @@ func (cl *coreLoop) handleProducerWriteRequest(request producerWriteRequest) { } func (cl *coreLoop) handleProducerCancelRequest(request producerCancelRequest) { + // TODO: implement me } func (cl *coreLoop) handleWriterLoopResponse(response writerLoopResponse) { cl.writing = false - for _, metadata := range response.segments { - // Update the segments with their new size and, if the writer - // closed them, move them to the reading list. - // TODO: i don't like this. It is redundant, and in brittle ways: - // segments are always written and closed in strict order, and the - // core loop knows what that order is, but we let the writer loop - // report them as independent parameters and then depend on those - // instead? It works for the moment but needs to be fixed soon. - metadata.segment.endOffset += segmentOffset(metadata.bytesWritten) - if metadata.closed { - cl.queue.segments.writing = cl.queue.segments.writing[1:] - cl.queue.segments.reading = - append(cl.queue.segments.reading, metadata.segment) - } + // The writer loop response contains the number of bytes written to + // each segment that appeared in the request. Entries always appear in + // the same sequence as (the beginning of) segments.writing. + for index, bytesWritten := range response.bytesWritten { + // Update the segment with its new size. + cl.queue.segments.writing[index].endOffset += segmentOffset(bytesWritten) + } + + // If there is more than one segment in the response, then all but the + // last have been closed and are ready to move to the reading list. + closedCount := len(response.bytesWritten) - 1 + if closedCount > 0 { + // Remove the prefix of the writing array and append to to reading. + closedSegments := cl.queue.segments.writing[:closedCount] + cl.queue.segments.writing = cl.queue.segments.writing[closedCount:] + cl.queue.segments.reading = + append(cl.queue.segments.reading, closedSegments...) } } diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 35851c7270b..2eda803265d 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -25,7 +25,7 @@ import ( ) // A segmentedFrame is a data frame waiting to be written to disk along with -// the containing segment it has been assigned to. +// the segment it has been assigned to. type segmentedFrame struct { // The frame to be written to disk. frame *writeFrame @@ -34,29 +34,26 @@ type segmentedFrame struct { segment *queueSegment } -// One block for the writer loop consists of a write request and the -// segment it should be written to. +// A writer loop request contains a list of writeFrames with the +// segment each should be written to. +// +// Input invariant: If a frame f is included in a writerLoopRequest, then +// every subsequent frame in this and future requests must have +// frame id at least f.segment.id. +// +// That is: we must write all frames for segment 0 before we start writing +// to frame 1, etc. This assumption allows all file operations to happen +// safely in the writer loop without any knowledge of the broader queue state. type writerLoopRequest struct { frames []segmentedFrame } -// writerSegmentMetadata returns the actions taken by the writer loop -// on a single segment: the number of bytes written, and whether or -// not the segment was closed while handling the request (which signals -// to the core loop that the segment can be moved to segments.reading). -type writerSegmentMetadata struct { - segment *queueSegment - bytesWritten int64 - closed bool -} - -// A writerLoopResponse reports the list of segments that have been -// completely written and can be moved to segments.reading. -// A segment is determined to have been completely written -// (and is then closed by the writer loop) when a frameWriteRequest -// targets a different segment than the previous ones. +// A writerLoopResponse reports the number of bytes written to each +// segment in the request. There is guaranteed to be one entry for each +// segment that appeared in the request, in the same order. If there is +// more than one entry, then all but the last segment have been closed. type writerLoopResponse struct { - segments []writerSegmentMetadata + bytesWritten []int64 } type writerLoop struct { @@ -92,16 +89,17 @@ func (wl *writerLoop) run() { // The input channel is closed, we are done return } - segments := wl.processRequest(block) - wl.responseChan <- writerLoopResponse{segments: segments} + bytesWritten := wl.processRequest(block) + wl.responseChan <- writerLoopResponse{bytesWritten: bytesWritten} } } -// Write the given data to disk, returns the list of segments that were -// completed in the process. -func (wl *writerLoop) processRequest(request writerLoopRequest) []writerSegmentMetadata { - var segments []writerSegmentMetadata - bytesWritten := int64(0) // Bytes written to the current segment. +// processRequest writes the frames in the given request to disk and returns +// the number of bytes written to each segment, in the order they were +// encountered. +func (wl *writerLoop) processRequest(request writerLoopRequest) []int64 { + var bytesWritten []int64 // Bytes written to all segments. + curBytesWritten := int64(0) // Bytes written to the current segment. for _, frameRequest := range request.frames { // If the new segment doesn't match the last one, we need to open a new // file handle and possibly clean up the old one. @@ -112,12 +110,10 @@ func (wl *writerLoop) processRequest(request writerLoopRequest) []writerSegmentM wl.outputFile.Close() wl.outputFile = nil // TODO: try to sync? - segments = append(segments, writerSegmentMetadata{ - segment: wl.currentSegment, - bytesWritten: bytesWritten, - closed: true, - }) - bytesWritten = 0 + // We are done with this segment, add the byte count to the list and + // reset the current counter. + bytesWritten = append(bytesWritten, curBytesWritten) + curBytesWritten = 0 } wl.currentSegment = frameRequest.segment file, err := wl.currentSegment.getWriter(wl.settings) @@ -144,15 +140,10 @@ func (wl *writerLoop) processRequest(request writerLoopRequest) []writerSegmentM binary.Write(wl.outputFile, binary.LittleEndian, checksum) // Write the frame footer's (duplicate) length binary.Write(wl.outputFile, binary.LittleEndian, frameSize) - bytesWritten += int64(n) + frameMetadataSize - } - if bytesWritten > 0 { - segments = append(segments, writerSegmentMetadata{ - segment: wl.currentSegment, - bytesWritten: bytesWritten, - }) + curBytesWritten += int64(n) + frameMetadataSize } - return segments + // Return the total byte counts, including the final segment. + return append(bytesWritten, curBytesWritten) } /* From 8db0acf4725bb35524509c47d1e6131200c4f977 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 10 Sep 2020 16:53:17 -0400 Subject: [PATCH 52/91] add retrying to file creation / writing --- libbeat/publisher/queue/diskqueue/segments.go | 81 ++--------- .../publisher/queue/diskqueue/writer_loop.go | 134 ++++++++++++++---- 2 files changed, 121 insertions(+), 94 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 6b08c75d38f..41d0af8afe4 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -186,6 +186,21 @@ func (segment *queueSegment) getWriter( return file, nil } +// getWriterWithRetry tries to create a file handle for writing via +// queueSegment.getWriter. On error, it retries as long as the given +// retry callback returns true. This is used for timed retries when +// creating a queue segment from the writer loop. +func (segment *queueSegment) getWriterWithRetry( + queueSettings *Settings, retry func(error) bool, +) (*os.File, error) { + file, err := segment.getWriter(queueSettings) + for err != nil && retry(err) { + // Try again + file, err = segment.getWriter(queueSettings) + } + return file, err +} + func readSegmentHeader(in *os.File) (*segmentHeader, error) { header := &segmentHeader{} err := binary.Read(in, binary.LittleEndian, &header.version) @@ -230,69 +245,3 @@ func (segments *diskQueueSegments) sizeOnDisk() uint64 { } return total } - -// nextDataFrame returns the bytes of the next data frame, or an error if the -// frame couldn't be read. If an error is returned, the caller should log it -// and drop the containing segment. A nil return value with no error means -// there are no frames to read. -/*func (reader *segmentReader) nextDataFrame() ([]byte, error) { - if reader.curPosition >= reader.endPosition { - return nil, nil - } - var frameLength uint32 - err := binary.Read(reader.raw, binary.LittleEndian, &frameLength) - if err != nil { - return nil, fmt.Errorf( - "Disk queue couldn't read next frame length: %w", err) - } - - // Bounds checking to make sure we can read this frame. - if reader.curPosition+segmentOffset(frameLength) > reader.endPosition { - // This frame extends past the end of our data region, which - // should never happen unless there is data corruption. - return nil, fmt.Errorf( - "Data frame length (%d) exceeds remaining data (%d)", - frameLength, reader.endPosition-reader.curPosition) - } - if frameLength <= frameMetadataSize { - // Actual enqueued data must have positive length - return nil, fmt.Errorf( - "Data frame with no data (length %d)", frameLength) - } - - // Read the actual frame data - dataLength := frameLength - frameMetadataSize - data := make([]byte, dataLength) - _, err = io.ReadFull(reader.raw, data) - if err != nil { - return nil, fmt.Errorf( - "Couldn't read data frame from disk: %w", err) - } - - // Read the footer (length + checksum) - var duplicateLength uint32 - err = binary.Read(reader.raw, binary.LittleEndian, &duplicateLength) - if err != nil { - return nil, fmt.Errorf( - "Disk queue couldn't read trailing frame length: %w", err) - } - if duplicateLength != frameLength { - return nil, fmt.Errorf( - "Disk queue: inconsistent frame length (%d vs %d)", - frameLength, duplicateLength) - } - - // Validate the checksum - var checksum uint32 - err = binary.Read(reader.raw, binary.LittleEndian, &checksum) - if err != nil { - return nil, fmt.Errorf( - "Disk queue couldn't read data frame's checksum: %w", err) - } - if computeChecksum(data, reader.checksumType) != checksum { - return nil, fmt.Errorf("Disk queue: bad data frame checksum") - } - - reader.curPosition += segmentOffset(frameLength) - return data, nil -}*/ diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 2eda803265d..9eb4d0cfb7f 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -19,7 +19,11 @@ package diskqueue import ( "encoding/binary" + "errors" + "io" "os" + "syscall" + "time" "github.com/elastic/beats/v7/libbeat/logp" ) @@ -63,9 +67,11 @@ type writerLoop struct { // The logger for the writer loop, assigned when the queue creates it. logger *logp.Logger - // The writer loop listens on requestChan for write blocks, and + // The writer loop listens on requestChan for frames to write, and // writes them to disk immediately (all queue capacity checking etc. is // done by the core loop before sending it to the writer). + // When this channel is closed, any in-progress writes are aborted and + // the run loop terminates. requestChan chan writerLoopRequest // The writer loop sends to responseChan when it has finished handling a @@ -82,11 +88,10 @@ type writerLoop struct { } func (wl *writerLoop) run() { - wl.logger.Debug("Writer loop starting up...") for { block, ok := <-wl.requestChan if !ok { - // The input channel is closed, we are done + // The requst channel is closed, we are done return } bytesWritten := wl.processRequest(block) @@ -98,8 +103,14 @@ func (wl *writerLoop) run() { // the number of bytes written to each segment, in the order they were // encountered. func (wl *writerLoop) processRequest(request writerLoopRequest) []int64 { + // A wrapper around the file handle that enables timed retries. After + // a call to retryWriter.Write, it is guaranteed that either the write + // completely succeeded or the queue is being closed. + retryWriter := callbackRetryWriter{retry: wl.retryCallback} + var bytesWritten []int64 // Bytes written to all segments. curBytesWritten := int64(0) // Bytes written to the current segment. +outerLoop: for _, frameRequest := range request.frames { // If the new segment doesn't match the last one, we need to open a new // file handle and possibly clean up the old one. @@ -107,59 +118,126 @@ func (wl *writerLoop) processRequest(request writerLoopRequest) []int64 { wl.logger.Debugf( "Creating new segment file with id %v\n", frameRequest.segment.id) if wl.outputFile != nil { + // TODO: try to sync? wl.outputFile.Close() wl.outputFile = nil - // TODO: try to sync? // We are done with this segment, add the byte count to the list and // reset the current counter. bytesWritten = append(bytesWritten, curBytesWritten) curBytesWritten = 0 } wl.currentSegment = frameRequest.segment - file, err := wl.currentSegment.getWriter(wl.settings) + file, err := wl.currentSegment.getWriterWithRetry( + wl.settings, wl.retryCallback) if err != nil { - wl.logger.Errorf("Couldn't open new segment file: %v", err) - // TODO: retry, etc + // This can only happen if the queue is being closed; abort. + break } wl.outputFile = file } + // Make sure our writer points to the current file handle. + retryWriter.wrapped = wl.outputFile // We have the data and a file to write it to. We are now committed // to writing this block unless the queue is closed in the meantime. frameSize := uint32(frameRequest.frame.sizeOnDisk()) - binary.Write(wl.outputFile, binary.LittleEndian, frameSize) - // TODO: retry forever if there is an error or n isn't the right - // length. - n, err := wl.outputFile.Write(frameRequest.frame.serialized) + + // The Write calls below all pass through retryWriter, so they can + // only return an error if the write should be aborted. Thus, all we + // need to do when we see an error is break out of the request loop. + err := binary.Write(retryWriter, binary.LittleEndian, frameSize) + if err != nil { + break + } + _, err = retryWriter.Write(frameRequest.frame.serialized) if err != nil { - wl.logger.Errorf("Couldn't write pending data to disk: %w", err) + break } // Compute / write the frame's checksum checksum := computeChecksum( frameRequest.frame.serialized, wl.settings.ChecksumType) - binary.Write(wl.outputFile, binary.LittleEndian, checksum) + err = binary.Write(wl.outputFile, binary.LittleEndian, checksum) + if err != nil { + break + } // Write the frame footer's (duplicate) length - binary.Write(wl.outputFile, binary.LittleEndian, frameSize) - curBytesWritten += int64(n) + frameMetadataSize + err = binary.Write(wl.outputFile, binary.LittleEndian, frameSize) + if err != nil { + break + } + // Update the byte count as the last step: that way if we abort while + // a frame is partially written, we only report up to the last + // complete frame. (This almost never matters, but it allows for + // more controlled recovery after a bad shutdown.) + curBytesWritten += int64(frameSize) + + // Explicitly check if we should abort before starting the next frame. + select { + case <-wl.requestChan: + break outerLoop + default: + } } // Return the total byte counts, including the final segment. return append(bytesWritten, curBytesWritten) } -/* -func writeAll(writer io.Writer, p []byte) (int, error) { - var N int - for len(p) > 0 { - n, err := writer.Write(p) - N, p = N+n, p[n:] - if err != nil && isRetryErr(err) { - return N, err - } +// retryCallback is called (by way of retryCallbackWriter) when there is +// an error writing to a segment file. It pauses for a configurable +// interval and returns true if the operation should be retried (which +// it always should, unless the queue is being closed). +func (wl *writerLoop) retryCallback(err error) bool { + if writeErrorIsRetriable(err) { + return true + } + // If the error is not immediately retriable, log the error + // and wait for the retry interval before trying again, but + // abort if the queue is closed (indicated by the request channel + // becoming unblocked). + wl.logger.Errorf("Writing to segment %v: %v", + wl.currentSegment.id, err) + select { + case <-time.After(time.Second): + // TODO: use a configurable interval here + return true + case <-wl.requestChan: + return false } - return N, nil } -func isRetryErr(err error) bool { - return err == syscall.EINTR || err == syscall.EAGAIN +// writeErrorIsRetriable returns true if the given IO error can be +// immediately retried. +func writeErrorIsRetriable(err error) bool { + return errors.Is(err, syscall.EINTR) || errors.Is(err, syscall.EAGAIN) +} + +// callbackRetryWriter is an io.Writer that wraps another writer and enables +// write-with-retry. When a Write encounters an error, it is passed to the +// retry callback. If the callback returns true, the the writer retries +// any unwritten portion of the input, otherwise it passes the error back +// to the caller. +// This helper is specifically for working with the writer loop, which needs +// to be able to retry forever at configurable intervals, but also cancel +// immediately if the queue is closed. +// This writer is unbuffered. In particular, it is safe to modify +// "wrapped" in-place as long as it isn't captured by the callback. +type callbackRetryWriter struct { + wrapped io.Writer + retry func(error) bool +} + +func (w callbackRetryWriter) Write(p []byte) (int, error) { + bytesWritten := 0 + writer := w.wrapped + n, err := writer.Write(p) + for n < len(p) { + if err != nil && !w.retry(err) { + return bytesWritten + n, err + } + // Advance p and try again. + bytesWritten += n + p = p[n:] + n, err = writer.Write(p) + } + return bytesWritten + n, nil } -*/ From 26d226d9a87306f1d87e1a72c4b5377611c3779c Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 10 Sep 2020 16:55:39 -0400 Subject: [PATCH 53/91] correct / clarify comments --- libbeat/publisher/queue/diskqueue/writer_loop.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 9eb4d0cfb7f..b4c9db4e8b8 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -41,9 +41,9 @@ type segmentedFrame struct { // A writer loop request contains a list of writeFrames with the // segment each should be written to. // -// Input invariant: If a frame f is included in a writerLoopRequest, then -// every subsequent frame in this and future requests must have -// frame id at least f.segment.id. +// Input invariant (segment ids are sorted): If a frame f is included in a +// writerLoopRequest, then every subsequent frame in this and future +// requests must have segment id at least f.segment.id. // // That is: we must write all frames for segment 0 before we start writing // to frame 1, etc. This assumption allows all file operations to happen From 442a513af862655bb6f7e954288fc7d349622150 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 10 Sep 2020 17:16:56 -0400 Subject: [PATCH 54/91] move segment read / write positions to the segments struct --- .../publisher/queue/diskqueue/core_loop.go | 35 ++++++------------- libbeat/publisher/queue/diskqueue/queue.go | 3 +- libbeat/publisher/queue/diskqueue/segments.go | 14 ++++++++ .../publisher/queue/diskqueue/writer_loop.go | 4 +-- 4 files changed, 28 insertions(+), 28 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 065e96ecb9f..0920aeca804 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -39,21 +39,6 @@ type coreLoop struct { // request, false otherwise. deleting bool - // nextReadOffset is the segment offset to start reading at during - // the next read request. This offset always refers to the first read - // segment: either segments.reading[0], if that list is nonempty, or - // segments.writing[0] (if all segments have been read except the one - // currently being written). - nextReadOffset segmentOffset - - // nextWriteOffset is the segment offset at which the next new frame - // should be written. This offset always refers to the last entry of - // segments.writing. This is distinct from the endOffset field - // within a segment: endOffset tracks how much data _has_ been - // written to disk, while nextWriteOffset also includes all pending - // frames that are scheduled to be written to disk. - nextWriteOffset segmentOffset - // pendingFrames is a list of all incoming data frames that have been // accepted by the queue and are waiting to be sent to the writer loop. // Segment ids in this list always appear in sorted order, even between @@ -202,7 +187,7 @@ func (cl *coreLoop) handleReaderLoopResponse(response readerLoopResponse) { segments := cl.queue.segments // Advance the read offset based on what was just completed. - cl.nextReadOffset += segmentOffset(response.byteCount) + segments.nextReadOffset += segmentOffset(response.byteCount) var segment *queueSegment if len(segments.reading) > 0 { @@ -212,10 +197,10 @@ func (cl *coreLoop) handleReaderLoopResponse(response readerLoopResponse) { // so we can rely on their endOffset field to determine the // size of the data. segment = segments.reading[0] - if cl.nextReadOffset >= segment.endOffset || response.err != nil { + if segments.nextReadOffset >= segment.endOffset || response.err != nil { segments.reading = segments.reading[1:] segments.acking = append(segments.acking, segment) - cl.nextReadOffset = 0 + segments.nextReadOffset = 0 } } else { // A segment in the writing list can't be finished writing, @@ -349,14 +334,16 @@ func (cl *coreLoop) maybeReadPending() { // A read request is already pending return } - segment := cl.queue.segments.readingSegment() - if segment == nil || cl.nextReadOffset >= segmentOffset(segment.endOffset) { + segments := cl.queue.segments + segment := segments.readingSegment() + if segment == nil || + segments.nextReadOffset >= segmentOffset(segment.endOffset) { // Nothing to read return } request := readerLoopRequest{ segment: segment, - startOffset: cl.nextReadOffset, + startOffset: segments.nextReadOffset, endOffset: segment.endOffset, } cl.queue.readerLoop.requestChan <- request @@ -406,14 +393,14 @@ func (cl *coreLoop) enqueueWriteFrame(frame *writeFrame) { // If segment is nil, or the new segment exceeds its bounds, // we need to create a new writing segment. if segment == nil || - cl.nextWriteOffset+frameLen > dq.settings.maxSegmentOffset() { + dq.segments.nextWriteOffset+frameLen > dq.settings.maxSegmentOffset() { segment = &queueSegment{id: dq.segments.nextID} dq.segments.writing = append(dq.segments.writing, segment) dq.segments.nextID++ - cl.nextWriteOffset = 0 + dq.segments.nextWriteOffset = 0 } - cl.nextWriteOffset += frameLen + dq.segments.nextWriteOffset += frameLen cl.pendingFrames = append(cl.pendingFrames, segmentedFrame{ frame: frame, segment: segment, diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index adb972cd2fe..ef5919abd93 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -259,8 +259,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { // to refer back to the queue. (TODO: just merge the core loop fields // and logic into the queue itself.) queue.coreLoop = &coreLoop{ - queue: queue, - nextReadOffset: 0, // TODO: initialize this if we're opening an existing queue + queue: queue, } go func() { queue.coreLoop.run() diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 41d0af8afe4..7414a0ee7a3 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -29,6 +29,7 @@ import ( // diskQueueSegments encapsulates segment-related queue metadata. type diskQueueSegments struct { + // The segments that are currently being written. The writer loop // writes these segments in order. When a segment has been // completely written, the writer loop notifies the core loop @@ -62,6 +63,19 @@ type diskQueueSegments struct { // The next sequential unused segment ID. This is what will be assigned // to the next queueSegment we create. nextID segmentID + + // nextWriteOffset is the segment offset at which the next new frame + // should be written. This offset always applies to the last entry of + // writing[]. This is distinct from the endOffset field within a segment: + // endOffset tracks how much data _has_ been written to a segment, while + // nextWriteOffset also includes everything that is _scheduled_ to be + // written. + nextWriteOffset segmentOffset + + // nextReadOffset is the position to start reading during the next + // read request. This offset always applies to the first reading + // segment: either reading[0], or writing[0] if reading is empty. + nextReadOffset segmentOffset } // segmentOffset is a byte index into the segment's data region. diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index b4c9db4e8b8..57ed864f618 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -103,8 +103,8 @@ func (wl *writerLoop) run() { // the number of bytes written to each segment, in the order they were // encountered. func (wl *writerLoop) processRequest(request writerLoopRequest) []int64 { - // A wrapper around the file handle that enables timed retries. After - // a call to retryWriter.Write, it is guaranteed that either the write + // retryWriter wraps the file handle with timed retries. + // retryWriter.Write is guaranteed to return only if the write // completely succeeded or the queue is being closed. retryWriter := callbackRetryWriter{retry: wl.retryCallback} From 1a1742e754cf276242b7b8e08ad2288a563114c2 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 10 Sep 2020 17:39:08 -0400 Subject: [PATCH 55/91] remove the separate 'core loop' struct and coalesce its helpers in diskQueue --- .../publisher/queue/diskqueue/core_loop.go | 241 ++++++++---------- libbeat/publisher/queue/diskqueue/queue.go | 46 +++- 2 files changed, 136 insertions(+), 151 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 0920aeca804..479a5f9a464 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -17,118 +17,84 @@ package diskqueue -type coreLoop struct { - // The queue that created this coreLoop. The core loop is the only one of - // the main goroutines for the queue that has a pointer to the queue and - // understands its logic / structure. - // Possible TODO: the split between fields in coreLoop and fields in - // diskQueue seems artificial. Maybe the values here should be promoted - // to diskQueue fields, and the core loop should just be a function on - // diskQueue. - queue *diskQueue - - // writing is true if a writeRequest is currently being processed by the - // writer loop, false otherwise. - writing bool - - // reading is true if the reader loop is processing a readBlock, false - // otherwise. - reading bool - - // deleting is true if the segment-deletion loop is processing a deletion - // request, false otherwise. - deleting bool - - // pendingFrames is a list of all incoming data frames that have been - // accepted by the queue and are waiting to be sent to the writer loop. - // Segment ids in this list always appear in sorted order, even between - // requests (that is, a frame added to this list always has segment id - // at least as high as every previous frame that has ever been added). - pendingFrames []segmentedFrame - - // blockedProducers is a list of all producer write requests that are - // waiting for free space in the queue. - blockedProducers []producerWriteRequest - - // This value represents the oldest frame ID for a segment that has not - // yet been moved to the acked list. It is used to detect when the oldest - // outstanding segment has been fully acknowledged by the consumer. - oldestFrameID frameID -} +// This file contains the queue's "core loop" -- the central goroutine +// that owns all queue state that is not encapsulated in one of the +// self-contained helper loops. This is the only file that is allowed to +// modify the queue state after its creation, and it contains the full +// logical "state transition diagram" for queue operation. -func (cl *coreLoop) run() { - cl.queue.logger.Debug("Core loop starting up...") - dq := cl.queue +func (dq *diskQueue) run() { + dq.logger.Debug("Core loop starting up...") // Wake up the reader loop if there are segments available to read // (from a previous instantiation of the queue). - cl.maybeReadPending() + dq.maybeReadPending() for { select { // Endpoints used by the producer / consumer API implementation. case producerWriteRequest := <-dq.producerWriteRequestChan: - cl.handleProducerWriteRequest(producerWriteRequest) + dq.handleProducerWriteRequest(producerWriteRequest) // After a write request, there may be data ready to send to the // writer loop. - cl.maybeWritePending() + dq.maybeWritePending() case cancelRequest := <-dq.producerCancelRequestChan: // TODO: this isn't really handled yet. - cl.handleProducerCancelRequest(cancelRequest) + dq.handleProducerCancelRequest(cancelRequest) case ackedUpTo := <-dq.consumerAckChan: - cl.handleConsumerAck(ackedUpTo) + dq.handleConsumerAck(ackedUpTo) // After receiving new ACKs, a segment might be ready to delete. - cl.maybeDeleteAcked() + dq.maybeDeleteAcked() case <-dq.done: - cl.handleShutdown() + dq.handleShutdown() return // Writer loop handling case writerLoopResponse := <-dq.writerLoop.responseChan: - cl.handleWriterLoopResponse(writerLoopResponse) + dq.handleWriterLoopResponse(writerLoopResponse) // The writer loop completed a request, so check if there is more // data to be sent. - cl.maybeWritePending() + dq.maybeWritePending() // We also check whether the reader loop is waiting for the data // that was just written. - cl.maybeReadPending() + dq.maybeReadPending() // Reader loop handling case readerLoopResponse := <-dq.readerLoop.responseChan: - cl.handleReaderLoopResponse(readerLoopResponse) + dq.handleReaderLoopResponse(readerLoopResponse) // If there is more data to read, start a new read request. - cl.maybeReadPending() + dq.maybeReadPending() // Deleter loop handling case deleterLoopResponse := <-dq.deleterLoop.responseChan: - cl.handleDeleterLoopResponse(deleterLoopResponse) + dq.handleDeleterLoopResponse(deleterLoopResponse) // If there are still files waiting to be deleted, send another request. - cl.maybeDeleteAcked() + dq.maybeDeleteAcked() // If there were blocked producers waiting for more queue space, // we might be able to unblock them now. - cl.maybeUnblockProducers() + dq.maybeUnblockProducers() } } } -func (cl *coreLoop) handleProducerWriteRequest(request producerWriteRequest) { +func (dq *diskQueue) handleProducerWriteRequest(request producerWriteRequest) { // Pathological case checking: make sure the incoming frame isn't bigger // than an entire segment all by itself (as long as it isn't, it is // guaranteed to eventually enter the queue assuming no disk errors). frameSize := request.frame.sizeOnDisk() - if cl.queue.settings.MaxSegmentSize < frameSize { - cl.queue.logger.Warnf( + if dq.settings.MaxSegmentSize < frameSize { + dq.logger.Warnf( "Rejecting event with size %v because the maximum segment size is %v", - frameSize, cl.queue.settings.MaxSegmentSize) + frameSize, dq.settings.MaxSegmentSize) request.responseChan <- false return } @@ -138,36 +104,38 @@ func (cl *coreLoop) handleProducerWriteRequest(request producerWriteRequest) { // Otherwise, we either add to the end of blockedProducers to wait for // the requested space or report immediate failure, depending on the // producer settings. - if len(cl.blockedProducers) == 0 && cl.canAcceptFrameOfSize(frameSize) { + if len(dq.blockedProducers) == 0 && dq.canAcceptFrameOfSize(frameSize) { // There is enough space for the new frame! Add it to the // pending list and report success, then dispatch it to the // writer loop if no other requests are outstanding. - cl.enqueueWriteFrame(request.frame) + dq.enqueueWriteFrame(request.frame) request.responseChan <- true } else { // The queue is too full. Either add the request to blockedProducers, // or send an immediate reject. if request.shouldBlock { - cl.blockedProducers = append(cl.blockedProducers, request) + dq.blockedProducers = append(dq.blockedProducers, request) } else { request.responseChan <- false } } } -func (cl *coreLoop) handleProducerCancelRequest(request producerCancelRequest) { +func (dq *diskQueue) handleProducerCancelRequest( + request producerCancelRequest, +) { // TODO: implement me } -func (cl *coreLoop) handleWriterLoopResponse(response writerLoopResponse) { - cl.writing = false +func (dq *diskQueue) handleWriterLoopResponse(response writerLoopResponse) { + dq.writing = false // The writer loop response contains the number of bytes written to // each segment that appeared in the request. Entries always appear in // the same sequence as (the beginning of) segments.writing. for index, bytesWritten := range response.bytesWritten { // Update the segment with its new size. - cl.queue.segments.writing[index].endOffset += segmentOffset(bytesWritten) + dq.segments.writing[index].endOffset += segmentOffset(bytesWritten) } // If there is more than one segment in the response, then all but the @@ -175,51 +143,49 @@ func (cl *coreLoop) handleWriterLoopResponse(response writerLoopResponse) { closedCount := len(response.bytesWritten) - 1 if closedCount > 0 { // Remove the prefix of the writing array and append to to reading. - closedSegments := cl.queue.segments.writing[:closedCount] - cl.queue.segments.writing = cl.queue.segments.writing[closedCount:] - cl.queue.segments.reading = - append(cl.queue.segments.reading, closedSegments...) + closedSegments := dq.segments.writing[:closedCount] + dq.segments.writing = dq.segments.writing[closedCount:] + dq.segments.reading = + append(dq.segments.reading, closedSegments...) } } -func (cl *coreLoop) handleReaderLoopResponse(response readerLoopResponse) { - cl.reading = false - segments := cl.queue.segments +func (dq *diskQueue) handleReaderLoopResponse(response readerLoopResponse) { + dq.reading = false // Advance the read offset based on what was just completed. - segments.nextReadOffset += segmentOffset(response.byteCount) + dq.segments.nextReadOffset += segmentOffset(response.byteCount) var segment *queueSegment - if len(segments.reading) > 0 { + if len(dq.segments.reading) > 0 { // A segment is finished if we have read all the data, or // the read response reports an error. // Segments in the reading list have been completely written, // so we can rely on their endOffset field to determine the // size of the data. - segment = segments.reading[0] - if segments.nextReadOffset >= segment.endOffset || response.err != nil { - segments.reading = segments.reading[1:] - segments.acking = append(segments.acking, segment) - segments.nextReadOffset = 0 + segment = dq.segments.reading[0] + if dq.segments.nextReadOffset >= segment.endOffset || response.err != nil { + dq.segments.reading = dq.segments.reading[1:] + dq.segments.acking = append(dq.segments.acking, segment) + dq.segments.nextReadOffset = 0 } } else { // A segment in the writing list can't be finished writing, // so we don't check the endOffset. - segment = segments.writing[0] + segment = dq.segments.writing[0] } segment.framesRead += response.frameCount // If there was an error, report it. if response.err != nil { - cl.queue.logger.Errorf( + dq.logger.Errorf( "Error reading segment file %s: %v", - cl.queue.settings.segmentPath(segment.id), response.err) + dq.settings.segmentPath(segment.id), response.err) } } -func (cl *coreLoop) handleDeleterLoopResponse(response deleterLoopResponse) { - dq := cl.queue - cl.deleting = false +func (dq *diskQueue) handleDeleterLoopResponse(response deleterLoopResponse) { + dq.deleting = false if len(response.deleted) > 0 { // One or more segments were deleted, recompute the outstanding list. newAckedSegments := []*queueSegment{} @@ -237,12 +203,12 @@ func (cl *coreLoop) handleDeleterLoopResponse(response deleterLoopResponse) { } } -func (cl *coreLoop) handleConsumerAck(ackedUpTo frameID) { - acking := cl.queue.segments.acking +func (dq *diskQueue) handleConsumerAck(ackedUpTo frameID) { + acking := dq.segments.acking if len(acking) == 0 { return } - startFrame := cl.oldestFrameID + startFrame := dq.oldestFrameID endFrame := startFrame ackedSegmentCount := 0 for ; ackedSegmentCount < len(acking); ackedSegmentCount++ { @@ -257,39 +223,38 @@ func (cl *coreLoop) handleConsumerAck(ackedUpTo frameID) { if ackedSegmentCount > 0 { // Move fully acked segments to the acked list and remove them // from the acking list. - cl.queue.segments.acked = - append(cl.queue.segments.acked, acking[:ackedSegmentCount]...) - cl.queue.segments.acking = acking[ackedSegmentCount:] + dq.segments.acked = + append(dq.segments.acked, acking[:ackedSegmentCount]...) + dq.segments.acking = acking[ackedSegmentCount:] // Advance oldestFrameID past the segments we just removed. - cl.oldestFrameID = endFrame + dq.oldestFrameID = endFrame } } -func (cl *coreLoop) handleShutdown() { +func (dq *diskQueue) handleShutdown() { // We need to close the input channels for all other goroutines and // wait for any outstanding responses. Order is important: handling // a read response may require the deleter, so the reader must be // shut down first. - close(cl.queue.readerLoop.requestChan) - if cl.reading { - response := <-cl.queue.readerLoop.responseChan - cl.handleReaderLoopResponse(response) + close(dq.readerLoop.requestChan) + if dq.reading { + response := <-dq.readerLoop.responseChan + dq.handleReaderLoopResponse(response) } - close(cl.queue.writerLoop.requestChan) - if cl.writing { - <-cl.queue.writerLoop.responseChan - //cl.queue.segments.writing.writer.Close() + close(dq.writerLoop.requestChan) + if dq.writing { + <-dq.writerLoop.responseChan } - close(cl.queue.deleterLoop.requestChan) - if cl.deleting { - response := <-cl.queue.deleterLoop.responseChan + close(dq.deleterLoop.requestChan) + if dq.deleting { + response := <-dq.deleterLoop.responseChan // We can't retry any more if deletion failed, but we still check the // response so we can log any errors. if len(response.errors) > 0 { - cl.queue.logger.Errorw("Couldn't delete old segment files", + dq.logger.Errorw("Couldn't delete old segment files", "errors", response.errors) } } @@ -301,19 +266,19 @@ func (cl *coreLoop) handleShutdown() { // If the pendingFrames list is nonempty, and there are no outstanding // requests to the writer loop, send the next batch of frames. -func (cl *coreLoop) maybeWritePending() { - if cl.writing || len(cl.pendingFrames) == 0 { +func (dq *diskQueue) maybeWritePending() { + if dq.writing || len(dq.pendingFrames) == 0 { // Nothing to do right now return } // Remove everything from pendingFrames and forward it to the writer loop. - requests := cl.pendingFrames - cl.pendingFrames = nil + requests := dq.pendingFrames + dq.pendingFrames = nil - cl.queue.writerLoop.requestChan <- writerLoopRequest{ + dq.writerLoop.requestChan <- writerLoopRequest{ frames: requests, } - cl.writing = true + dq.writing = true } // Returns the active read segment, or nil if there is none. @@ -329,61 +294,59 @@ func (segments *diskQueueSegments) readingSegment() *queueSegment { // If the reading list is nonempty, and there are no outstanding read // requests, send one. -func (cl *coreLoop) maybeReadPending() { - if cl.reading { +func (dq *diskQueue) maybeReadPending() { + if dq.reading { // A read request is already pending return } - segments := cl.queue.segments - segment := segments.readingSegment() + segment := dq.segments.readingSegment() if segment == nil || - segments.nextReadOffset >= segmentOffset(segment.endOffset) { + dq.segments.nextReadOffset >= segmentOffset(segment.endOffset) { // Nothing to read return } request := readerLoopRequest{ segment: segment, - startOffset: segments.nextReadOffset, + startOffset: dq.segments.nextReadOffset, endOffset: segment.endOffset, } - cl.queue.readerLoop.requestChan <- request - cl.reading = true + dq.readerLoop.requestChan <- request + dq.reading = true } // If the acked list is nonempty, and there are no outstanding deletion // requests, send one. -func (cl *coreLoop) maybeDeleteAcked() { - if !cl.deleting && len(cl.queue.segments.acked) > 0 { - cl.queue.deleterLoop.requestChan <- deleterLoopRequest{segments: cl.queue.segments.acked} - cl.deleting = true +func (dq *diskQueue) maybeDeleteAcked() { + if !dq.deleting && len(dq.segments.acked) > 0 { + dq.deleterLoop.requestChan <- deleterLoopRequest{ + segments: dq.segments.acked} + dq.deleting = true } } // maybeUnblockProducers checks whether the queue has enough free space // to accept any of the requests in the blockedProducers list, and if so // accepts them in order and updates the list. -func (cl *coreLoop) maybeUnblockProducers() { +func (dq *diskQueue) maybeUnblockProducers() { unblockedCount := 0 - for _, request := range cl.blockedProducers { - if !cl.canAcceptFrameOfSize(request.frame.sizeOnDisk()) { + for _, request := range dq.blockedProducers { + if !dq.canAcceptFrameOfSize(request.frame.sizeOnDisk()) { // Not enough space for this frame, we're done. break } // Add the frame to pendingFrames and report success. - cl.enqueueWriteFrame(request.frame) + dq.enqueueWriteFrame(request.frame) request.responseChan <- true unblockedCount++ } if unblockedCount > 0 { - cl.blockedProducers = cl.blockedProducers[unblockedCount:] + dq.blockedProducers = dq.blockedProducers[unblockedCount:] } } // enqueueWriteFrame determines which segment an incoming frame should be // written to and adds the resulting segmentedFrame to pendingFrames. -func (cl *coreLoop) enqueueWriteFrame(frame *writeFrame) { - dq := cl.queue - +func (dq *diskQueue) enqueueWriteFrame(frame *writeFrame) { // Start with the most recent writing segment if there is one. var segment *queueSegment if len(dq.segments.writing) > 0 { @@ -401,7 +364,7 @@ func (cl *coreLoop) enqueueWriteFrame(frame *writeFrame) { } dq.segments.nextWriteOffset += frameLen - cl.pendingFrames = append(cl.pendingFrames, segmentedFrame{ + dq.pendingFrames = append(dq.pendingFrames, segmentedFrame{ frame: frame, segment: segment, }) @@ -420,8 +383,8 @@ func (cl *coreLoop) enqueueWriteFrame(frame *writeFrame) { // If we decide to add limits on how many events / bytes can be stored // in pendingFrames (to avoid unbounded memory use if the input is faster // than the disk), this is the function to modify. -func (cl *coreLoop) canAcceptFrameOfSize(frameSize uint64) bool { - if cl.queue.settings.MaxBufferSize == 0 { +func (dq *diskQueue) canAcceptFrameOfSize(frameSize uint64) bool { + if dq.settings.MaxBufferSize == 0 { // Currently we impose no limitations if the queue size is unbounded. return true } @@ -430,10 +393,10 @@ func (cl *coreLoop) canAcceptFrameOfSize(frameSize uint64) bool { // left in the queue after accounting for the existing segments and the // pending writes that were already accepted. pendingBytes := uint64(0) - for _, request := range cl.pendingFrames { + for _, request := range dq.pendingFrames { pendingBytes += request.frame.sizeOnDisk() } - currentSize := pendingBytes + cl.queue.segments.sizeOnDisk() + currentSize := pendingBytes + dq.segments.sizeOnDisk() - return currentSize+frameSize <= cl.queue.settings.MaxBufferSize + return currentSize+frameSize <= dq.settings.MaxBufferSize } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index ef5919abd93..9cee9f0a230 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -79,11 +79,14 @@ type diskQueue struct { segments *diskQueueSegments // The queue's helper loops, each of which is run in its own goroutine. - coreLoop *coreLoop readerLoop *readerLoop writerLoop *writerLoop deleterLoop *deleterLoop + // Wait group for shutdown of the goroutines associated with this queue: + // reader loop, writer loop, deleter loop, and core loop (diskQueue.run()). + waitGroup *sync.WaitGroup + // The API channels used by diskQueueProducer to send write / cancel calls. producerWriteRequestChan chan producerWriteRequest producerCancelRequestChan chan producerCancelRequest @@ -96,6 +99,34 @@ type diskQueue struct { // waiting on ackLock. consumerAckChan chan frameID + // writing is true if a writeRequest is currently being processed by the + // writer loop, false otherwise. + writing bool + + // reading is true if the reader loop is processing a readBlock, false + // otherwise. + reading bool + + // deleting is true if the segment-deletion loop is processing a deletion + // request, false otherwise. + deleting bool + + // pendingFrames is a list of all incoming data frames that have been + // accepted by the queue and are waiting to be sent to the writer loop. + // Segment ids in this list always appear in sorted order, even between + // requests (that is, a frame added to this list always has segment id + // at least as high as every previous frame that has ever been added). + pendingFrames []segmentedFrame + + // blockedProducers is a list of all producer write requests that are + // waiting for free space in the queue. + blockedProducers []producerWriteRequest + + // This value represents the oldest frame ID for a segment that has not + // yet been moved to the acked list. It is used to detect when the oldest + // outstanding segment has been fully acknowledged by the consumer. + oldestFrameID frameID + // This lock must be held to read and write acked and ackedUpTo. ackLock sync.Mutex @@ -107,10 +138,6 @@ type diskQueue struct { // TODO: do this better. acked map[frameID]bool - // Wait group for shutdown of the goroutines associated with this queue: - // core loop, reader loop, writer loop, deleter loop. - waitGroup *sync.WaitGroup - // The channel to signal our goroutines to shut down. done chan struct{} } @@ -255,14 +282,9 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { done: make(chan struct{}), } - // The core loop is created last because it's the only one that needs - // to refer back to the queue. (TODO: just merge the core loop fields - // and logic into the queue itself.) - queue.coreLoop = &coreLoop{ - queue: queue, - } + // Start the queue's main loop. go func() { - queue.coreLoop.run() + queue.run() waitGroup.Done() }() From 5ca039d5d7bd836f37956304cc0c80a95b9ffc93 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Mon, 14 Sep 2020 11:43:06 -0400 Subject: [PATCH 56/91] set output channel buffer to a more reasonable size --- libbeat/publisher/queue/diskqueue/queue.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 9cee9f0a230..9a16d4a08eb 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -215,7 +215,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { requestChan: make(chan readerLoopRequest, 1), responseChan: make(chan readerLoopResponse), - output: make(chan *readFrame, 20), // TODO: customize this buffer size + output: make(chan *readFrame, 100), // TODO: customize this buffer size decoder: newEventDecoder(), } go func() { From fdd4be6bbb6023d7127e956131d4b7c216923faa Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Mon, 14 Sep 2020 17:21:13 -0400 Subject: [PATCH 57/91] Expose some user configuration settings --- libbeat/publisher/queue/diskqueue/config.go | 46 ++++++++++++++++--- libbeat/publisher/queue/diskqueue/queue.go | 51 ++++++++++++--------- 2 files changed, 69 insertions(+), 28 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/config.go b/libbeat/publisher/queue/diskqueue/config.go index e9f54717043..5ea8b055716 100644 --- a/libbeat/publisher/queue/diskqueue/config.go +++ b/libbeat/publisher/queue/diskqueue/config.go @@ -21,17 +21,40 @@ import ( "errors" "github.com/elastic/beats/v7/libbeat/common" + "github.com/elastic/beats/v7/libbeat/common/cfgtype" ) // userConfig holds the parameters for a disk queue that are configurable // by the end user in the beats yml file. type userConfig struct { - Path string `config:"path"` + Path string `config:"path"` + MaxSize cfgtype.ByteSize `config:"max_size" validate:"required"` + SegmentSize *cfgtype.ByteSize `config:"segment_size"` + ReadAheadLimit *int `config:"read_ahead"` + WriteAheadLimit *int `config:"write_ahead"` } func (c *userConfig) Validate() error { - if false { - return errors.New("something is wrong") + // If the segment size is explicitly specified, the total queue size must + // be at least twice as large. + if c.SegmentSize != nil && c.MaxSize != 0 && c.MaxSize < *c.SegmentSize*2 { + return errors.New( + "Disk queue max_size must be at least twice as big as segment_size") + } + + // We require a total queue size of at least 10MB, and a segment size of + // at least 1MB. The queue can support lower thresholds, but it will perform + // terribly, so we give an explicit error in that case. + // These bounds are still extremely low for Beats ingestion, but if all you + // need is for a low-volume stream on a tiny device to persist between + // restarts, it will work fine. + if c.MaxSize != 0 && c.MaxSize < 10*1000*1000 { + return errors.New( + "Disk queue max_size cannot be less than 10MB") + } + if c.SegmentSize != nil && *c.SegmentSize < 1000*1000 { + return errors.New( + "Disk queue segment_size cannot be less than 1MB") } return nil @@ -42,8 +65,11 @@ func (c *userConfig) Validate() error { func DefaultSettings() Settings { return Settings{ ChecksumType: ChecksumTypeCRC32, - MaxSegmentSize: 100 * (1 << 20), // 100MB - MaxBufferSize: (1 << 30), // 1GB + MaxSegmentSize: 100 * (1 << 20), // 100MiB + MaxBufferSize: (1 << 30), // 1GiB + + ReadAheadLimit: 256, + WriteAheadLimit: 1024, } } @@ -52,11 +78,19 @@ func DefaultSettings() Settings { func SettingsForUserConfig(config *common.Config) (Settings, error) { userConfig := userConfig{} if err := config.Unpack(&userConfig); err != nil { - // TODO: report error / decide what to do return Settings{}, err } settings := DefaultSettings() settings.Path = userConfig.Path + settings.MaxBufferSize = uint64(userConfig.MaxSize) + if userConfig.SegmentSize != nil { + settings.MaxSegmentSize = uint64(*userConfig.SegmentSize) + } else { + // If no value is specified, default segment size is total queue size + // divided by 10. + settings.MaxSegmentSize = uint64(userConfig.MaxSize) / 10 + } + return settings, nil } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 9a16d4a08eb..ef781aabe6f 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -52,6 +52,16 @@ type Settings struct { MaxSegmentSize uint64 ChecksumType ChecksumType + + // How many events will be read from disk while waiting for a consumer + // request. + ReadAheadLimit int + + // How many events will be queued in memory waiting to be written to disk. + // This setting should rarely matter in practice, but if data is coming + // in faster than it can be written to disk for an extended period, + // this limit can keep it from overflowing memory. + WriteAheadLimit int } type segmentID uint64 @@ -61,7 +71,7 @@ type queuePosition struct { segment segmentID // The byte offset of this position within its segment. - // This is specified relative to the start of the segment's data region, i.e. + // This is relative to the start of the segment's data region, i.e. // an offset of 0 means the first byte after the end of the segment header. offset segmentOffset } @@ -159,7 +169,7 @@ func queueFactory( ) (queue.Queue, error) { settings, err := SettingsForUserConfig(cfg) if err != nil { - return nil, err + return nil, fmt.Errorf("Disk queue couldn't load user config: %w", err) } //settings.producerAckListener = ackListener return NewQueue(logger, settings) @@ -186,18 +196,23 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { return nil, fmt.Errorf("Couldn't create disk queue directory: %w", err) } + // Index any existing data segments to be placed in segments.reading. + initialSegments, err := scanExistingSegments(settings.directoryPath()) + if err != nil { + return nil, err + } + var nextSegmentID segmentID + if len(initialSegments) > 0 { + lastID := initialSegments[len(initialSegments)-1].id + nextSegmentID = lastID + 1 + } + // Load the file handle for the queue state. stateFile, err := stateFileForPath(settings.stateFilePath()) if err != nil { return nil, fmt.Errorf("Couldn't open disk queue metadata file: %w", err) } - defer func() { - if err != nil { - // If the function is returning because of an error, close the - // file handle. - stateFile.Close() - } - }() + //if stateFile.loadedState. // We wait for four goroutines: core loop, reader loop, writer loop, // deleter loop. @@ -215,7 +230,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { requestChan: make(chan readerLoopRequest, 1), responseChan: make(chan readerLoopResponse), - output: make(chan *readFrame, 100), // TODO: customize this buffer size + output: make(chan *readFrame, settings.ReadAheadLimit), decoder: newEventDecoder(), } go func() { @@ -246,17 +261,6 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { waitGroup.Done() }() - // Index any existing data segments to be placed in segments.reading. - initialSegments, err := scanExistingSegments(settings.directoryPath()) - if err != nil { - return nil, err - } - var nextSegmentID segmentID - if len(initialSegments) > 0 { - lastID := initialSegments[len(initialSegments)-1].id - nextSegmentID = lastID + 1 - } - queue := &diskQueue{ logger: logger, settings: settings, @@ -271,7 +275,10 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { writerLoop: writerLoop, deleterLoop: deleterLoop, - // TODO: customize this channel buffer size + // TODO: is this a reasonable size for this channel buffer? + // Should this be customizable? (Tentatively: no, since we + // expect most producer write requests to be handled near- + // instantly so the requests are unlikely to accumulate). producerWriteRequestChan: make(chan producerWriteRequest, 10), producerCancelRequestChan: make(chan producerCancelRequest), From f7a446d9247f7588dfd4f3cff4dcc34f77a73872 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Wed, 16 Sep 2020 12:49:18 -0400 Subject: [PATCH 58/91] send producer acks --- libbeat/publisher/queue/diskqueue/queue.go | 9 +++++---- libbeat/publisher/queue/diskqueue/writer_loop.go | 12 ++++++++++++ 2 files changed, 17 insertions(+), 4 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index ef781aabe6f..c6523295379 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -21,7 +21,6 @@ import ( "fmt" "os" "path/filepath" - "runtime/debug" "sync" "github.com/elastic/beats/v7/libbeat/common" @@ -62,6 +61,10 @@ type Settings struct { // in faster than it can be written to disk for an extended period, // this limit can keep it from overflowing memory. WriteAheadLimit int + + // A listener that should be sent ACKs when an event is successfully + // written to disk. + WriteToDiskListener queue.ACKListener } type segmentID uint64 @@ -171,7 +174,7 @@ func queueFactory( if err != nil { return nil, fmt.Errorf("Disk queue couldn't load user config: %w", err) } - //settings.producerAckListener = ackListener + settings.WriteToDiskListener = ackListener return NewQueue(logger, settings) } @@ -350,8 +353,6 @@ func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { } func (dq *diskQueue) Consumer() queue.Consumer { - fmt.Printf("diskQueue.Consumer()\n") - debug.PrintStack() return &diskQueueConsumer{ queue: dq, closed: false, diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 57ed864f618..5adec95b865 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -171,6 +171,18 @@ outerLoop: // more controlled recovery after a bad shutdown.) curBytesWritten += int64(frameSize) + // If the producer has an ack listener, notify it the frame was written. + // TODO: it probably makes sense to batch these up and send them at the + // end of a full request. + if frameRequest.frame.producer.config.ACK != nil { + frameRequest.frame.producer.config.ACK(1) + } + + // If the queue has an ack listener, notify it the frame was written. + if wl.settings.WriteToDiskListener != nil { + wl.settings.WriteToDiskListener.OnACK(1) + } + // Explicitly check if we should abort before starting the next frame. select { case <-wl.requestChan: From 79a81de0b9ef5b933a44fa233fb1224bdfe5add2 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Wed, 16 Sep 2020 15:45:35 -0400 Subject: [PATCH 59/91] remove checksum type as a configurable field --- libbeat/publisher/queue/diskqueue/checksum.go | 34 +++++-------------- libbeat/publisher/queue/diskqueue/config.go | 1 - libbeat/publisher/queue/diskqueue/queue.go | 2 -- .../publisher/queue/diskqueue/reader_loop.go | 10 +++--- libbeat/publisher/queue/diskqueue/segments.go | 21 +++--------- .../publisher/queue/diskqueue/serialize.go | 5 ++- .../publisher/queue/diskqueue/writer_loop.go | 3 +- 7 files changed, 19 insertions(+), 57 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/checksum.go b/libbeat/publisher/queue/diskqueue/checksum.go index 80f067582c4..87cdb7b1aef 100644 --- a/libbeat/publisher/queue/diskqueue/checksum.go +++ b/libbeat/publisher/queue/diskqueue/checksum.go @@ -22,30 +22,12 @@ import ( "hash/crc32" ) -// ChecksumType specifies what checksum algorithm the queue should use to -// verify its data frames. -type ChecksumType uint32 - -// ChecksumTypeNone: Don't compute or verify checksums. -// ChecksumTypeCRC32: Compute the checksum with the Go standard library's -// "hash/crc32" package. -const ( - ChecksumTypeNone = iota - - ChecksumTypeCRC32 -) - -func computeChecksum(data []byte, checksumType ChecksumType) uint32 { - switch checksumType { - case ChecksumTypeNone: - return 0 - case ChecksumTypeCRC32: - hash := crc32.NewIEEE() - frameLength := uint32(len(data) + frameMetadataSize) - binary.Write(hash, binary.LittleEndian, &frameLength) - hash.Write(data) - return hash.Sum32() - default: - panic("segmentReader: invalid checksum type") - } +// Computes the checksum that should be written / read in a frame footer +// based on the raw content of that frame (excluding header / footer). +func computeChecksum(data []byte) uint32 { + hash := crc32.NewIEEE() + frameLength := uint32(len(data) + frameMetadataSize) + binary.Write(hash, binary.LittleEndian, &frameLength) + hash.Write(data) + return hash.Sum32() } diff --git a/libbeat/publisher/queue/diskqueue/config.go b/libbeat/publisher/queue/diskqueue/config.go index 5ea8b055716..e6499218073 100644 --- a/libbeat/publisher/queue/diskqueue/config.go +++ b/libbeat/publisher/queue/diskqueue/config.go @@ -64,7 +64,6 @@ func (c *userConfig) Validate() error { // for all important fields. func DefaultSettings() Settings { return Settings{ - ChecksumType: ChecksumTypeCRC32, MaxSegmentSize: 100 * (1 << 20), // 100MiB MaxBufferSize: (1 << 30), // 1GiB diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index c6523295379..496f9564f4f 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -50,8 +50,6 @@ type Settings struct { // to a single segment file before creating a new one. MaxSegmentSize uint64 - ChecksumType ChecksumType - // How many events will be read from disk while waiting for a consumer // request. ReadAheadLimit int diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index a1f10804eed..30728a0cce7 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -210,12 +210,10 @@ func (rl *readerLoop) nextFrame( if err != nil { return nil, fmt.Errorf("Couldn't read data frame checksum: %w", err) } - if rl.settings.ChecksumType != ChecksumTypeNone { - expected := computeChecksum(bytes, rl.settings.ChecksumType) - if checksum != expected { - return nil, fmt.Errorf( - "Data frame checksum mismatch (%x != %x)", checksum, expected) - } + expected := computeChecksum(bytes) + if checksum != expected { + return nil, fmt.Errorf( + "Data frame checksum mismatch (%x != %x)", checksum, expected) } var duplicateLength uint32 diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 7414a0ee7a3..11bb20bd55a 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -108,12 +108,11 @@ type queueSegment struct { } type segmentHeader struct { - version uint32 - checksumType ChecksumType + version uint32 } -// Each segment header has a 32-bit version and a 32-bit checksum type. -const segmentHeaderSize = 8 +// Segment headers are currently just a 32-bit version. +const segmentHeaderSize = 4 // Sort order: we store loaded segments in ascending order by their id. type bySegmentID []*queueSegment @@ -188,10 +187,7 @@ func (segment *queueSegment) getWriter( if err != nil { return nil, err } - header := &segmentHeader{ - version: 0, - checksumType: queueSettings.ChecksumType, - } + header := &segmentHeader{version: 0} err = writeSegmentHeader(file, header) if err != nil { return nil, fmt.Errorf("Couldn't write segment header: %w", err) @@ -224,20 +220,11 @@ func readSegmentHeader(in *os.File) (*segmentHeader, error) { if header.version != 0 { return nil, fmt.Errorf("Unrecognized schema version %d", header.version) } - var rawChecksumType uint32 - err = binary.Read(in, binary.LittleEndian, &rawChecksumType) - if err != nil { - return nil, err - } - header.checksumType = ChecksumType(rawChecksumType) return header, nil } func writeSegmentHeader(out *os.File, header *segmentHeader) error { err := binary.Write(out, binary.LittleEndian, header.version) - if err == nil { - err = binary.Write(out, binary.LittleEndian, uint32(header.checksumType)) - } return err } diff --git a/libbeat/publisher/queue/diskqueue/serialize.go b/libbeat/publisher/queue/diskqueue/serialize.go index 6bd3f3e97c1..10062e3ef1c 100644 --- a/libbeat/publisher/queue/diskqueue/serialize.go +++ b/libbeat/publisher/queue/diskqueue/serialize.go @@ -33,9 +33,8 @@ import ( ) type eventEncoder struct { - buf bytes.Buffer - folder *gotype.Iterator - checksumType ChecksumType + buf bytes.Buffer + folder *gotype.Iterator } type eventDecoder struct { diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 5adec95b865..69bcf6afc93 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -154,8 +154,7 @@ outerLoop: break } // Compute / write the frame's checksum - checksum := computeChecksum( - frameRequest.frame.serialized, wl.settings.ChecksumType) + checksum := computeChecksum(frameRequest.frame.serialized) err = binary.Write(wl.outputFile, binary.LittleEndian, checksum) if err != nil { break From f88481ceef5509b74e07d1cc64aa5203c2c0a4f5 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 17 Sep 2020 09:47:51 -0400 Subject: [PATCH 60/91] Batch producer ACKs / fsync after writes --- .../publisher/queue/diskqueue/writer_loop.go | 35 +++++++++++++------ 1 file changed, 25 insertions(+), 10 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 69bcf6afc93..8f296a67412 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -108,6 +108,13 @@ func (wl *writerLoop) processRequest(request writerLoopRequest) []int64 { // completely succeeded or the queue is being closed. retryWriter := callbackRetryWriter{retry: wl.retryCallback} + // We keep track of how many frames are written during this request, + // and send the associated ACKs to the queue / producer listeners + // in a batch at the end (since each ACK call can involve a round-trip + // to the registry). + totalACKCount := 0 + producerACKCounts := make(map[*diskQueueProducer]int) + var bytesWritten []int64 // Bytes written to all segments. curBytesWritten := int64(0) // Bytes written to the current segment. outerLoop: @@ -118,7 +125,8 @@ outerLoop: wl.logger.Debugf( "Creating new segment file with id %v\n", frameRequest.segment.id) if wl.outputFile != nil { - // TODO: try to sync? + // Try to sync to disk, then close the file. + wl.outputFile.Sync() wl.outputFile.Close() wl.outputFile = nil // We are done with this segment, add the byte count to the list and @@ -170,16 +178,10 @@ outerLoop: // more controlled recovery after a bad shutdown.) curBytesWritten += int64(frameSize) - // If the producer has an ack listener, notify it the frame was written. - // TODO: it probably makes sense to batch these up and send them at the - // end of a full request. + // Update the ACKs that will be sent at the end of the request. + totalACKCount++ if frameRequest.frame.producer.config.ACK != nil { - frameRequest.frame.producer.config.ACK(1) - } - - // If the queue has an ack listener, notify it the frame was written. - if wl.settings.WriteToDiskListener != nil { - wl.settings.WriteToDiskListener.OnACK(1) + producerACKCounts[frameRequest.frame.producer]++ } // Explicitly check if we should abort before starting the next frame. @@ -189,6 +191,19 @@ outerLoop: default: } } + // Try to sync the written data to disk. + wl.outputFile.Sync() + + // If the queue has an ACK listener, notify it the frames were written. + if wl.settings.WriteToDiskListener != nil { + wl.settings.WriteToDiskListener.OnACK(totalACKCount) + } + + // Notify any producers with ACK listeners that their frames were written. + for producer, ackCount := range producerACKCounts { + producer.config.ACK(ackCount) + } + // Return the total byte counts, including the final segment. return append(bytesWritten, curBytesWritten) } From 8b61f5badb419932a54fd6b66f5d4638be6b66b3 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 17 Sep 2020 10:09:02 -0400 Subject: [PATCH 61/91] remove queueSegment.header which is no longer needed --- libbeat/publisher/queue/diskqueue/segments.go | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 11bb20bd55a..2f8d97f9783 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -93,11 +93,6 @@ type queueSegment struct { // on disk is segmentHeaderSize + segment.endOffset. endOffset segmentOffset - // The header metadata for this segment file. This field is nil if the - // segment has not yet been opened for reading. It should only be - // accessed by the reader loop. - header *segmentHeader - // The number of frames read from this segment during this session. This // does not necessarily equal the number of frames in the segment, even // after reading is complete, since the segment may have been partially @@ -168,12 +163,13 @@ func (segment *queueSegment) getReader( return nil, fmt.Errorf( "Couldn't open segment %d: %w", segment.id, err) } - header, err := readSegmentHeader(file) + // Right now there is only one valid header (indicating schema version + // zero) so we don't need the value itself. + _, err = readSegmentHeader(file) if err != nil { file.Close() return nil, fmt.Errorf("Couldn't read segment header: %w", err) } - segment.header = header return file, nil } From 8c32477df21b48a28c604efa8b68c6a4667bd450 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 17 Sep 2020 13:25:23 -0400 Subject: [PATCH 62/91] reorganizing files a bit, starting the final pass of consumer acks --- libbeat/publisher/queue/diskqueue/config.go | 65 +++++++++++ libbeat/publisher/queue/diskqueue/consumer.go | 2 +- .../publisher/queue/diskqueue/core_loop.go | 31 ++--- libbeat/publisher/queue/diskqueue/queue.go | 107 +++++------------- .../publisher/queue/diskqueue/reader_loop.go | 55 ++------- libbeat/publisher/queue/diskqueue/segments.go | 56 +++++---- libbeat/publisher/queue/diskqueue/util.go | 89 +++++++++++++++ .../publisher/queue/diskqueue/writer_loop.go | 40 ------- 8 files changed, 244 insertions(+), 201 deletions(-) create mode 100644 libbeat/publisher/queue/diskqueue/util.go diff --git a/libbeat/publisher/queue/diskqueue/config.go b/libbeat/publisher/queue/diskqueue/config.go index e6499218073..14ecb28cb4f 100644 --- a/libbeat/publisher/queue/diskqueue/config.go +++ b/libbeat/publisher/queue/diskqueue/config.go @@ -19,11 +19,50 @@ package diskqueue import ( "errors" + "fmt" + "path/filepath" "github.com/elastic/beats/v7/libbeat/common" "github.com/elastic/beats/v7/libbeat/common/cfgtype" + "github.com/elastic/beats/v7/libbeat/paths" + "github.com/elastic/beats/v7/libbeat/publisher/queue" ) +// Settings contains the configuration fields to create a new disk queue +// or open an existing one. +type Settings struct { + // The path on disk of the queue's containing directory, which will be + // created if it doesn't exist. Within the directory, the queue's state + // is stored in state.dat and each segment's data is stored in + // {segmentIndex}.seg + // If blank, the default directory is "diskqueue" within the beat's data + // directory. + Path string + + // MaxBufferSize is the maximum number of bytes that the queue should + // ever occupy on disk. A value of 0 means the queue can grow until the + // disk is full (this is not recommended on a primary system disk). + MaxBufferSize uint64 + + // MaxSegmentSize is the maximum number of bytes that should be written + // to a single segment file before creating a new one. + MaxSegmentSize uint64 + + // How many events will be read from disk while waiting for a consumer + // request. + ReadAheadLimit int + + // How many events will be queued in memory waiting to be written to disk. + // This setting should rarely matter in practice, but if data is coming + // in faster than it can be written to disk for an extended period, + // this limit can keep it from overflowing memory. + WriteAheadLimit int + + // A listener that should be sent ACKs when an event is successfully + // written to disk. + WriteToDiskListener queue.ACKListener +} + // userConfig holds the parameters for a disk queue that are configurable // by the end user in the beats yml file. type userConfig struct { @@ -93,3 +132,29 @@ func SettingsForUserConfig(config *common.Config) (Settings, error) { return settings, nil } + +// +// bookkeeping helpers +// + +func (settings Settings) directoryPath() string { + if settings.Path == "" { + return paths.Resolve(paths.Data, "diskqueue") + } + + return settings.Path +} + +func (settings Settings) stateFilePath() string { + return filepath.Join(settings.directoryPath(), "state.dat") +} + +func (settings Settings) segmentPath(segmentID segmentID) string { + return filepath.Join( + settings.directoryPath(), + fmt.Sprintf("%v.seg", segmentID)) +} + +func (settings Settings) maxSegmentOffset() segmentOffset { + return segmentOffset(settings.MaxSegmentSize - segmentHeaderSize) +} diff --git a/libbeat/publisher/queue/diskqueue/consumer.go b/libbeat/publisher/queue/diskqueue/consumer.go index 3fe742a3cb2..fc30ea75341 100644 --- a/libbeat/publisher/queue/diskqueue/consumer.go +++ b/libbeat/publisher/queue/diskqueue/consumer.go @@ -117,6 +117,6 @@ func (batch *diskQueueBatch) ACK() { // usually no-ops. // TODO: only inform the core loop when we cross a segment // boundary. - dq.consumerAckChan <- dq.ackedUpTo + dq.consumerACKChan <- dq.ackedUpTo } } diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 479a5f9a464..39622fadfcb 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -44,11 +44,11 @@ func (dq *diskQueue) run() { // TODO: this isn't really handled yet. dq.handleProducerCancelRequest(cancelRequest) - case ackedUpTo := <-dq.consumerAckChan: - dq.handleConsumerAck(ackedUpTo) + case ackedUpTo := <-dq.consumerACKChan: + dq.handleConsumerACK(ackedUpTo) // After receiving new ACKs, a segment might be ready to delete. - dq.maybeDeleteAcked() + dq.maybeDeleteACKed() case <-dq.done: dq.handleShutdown() @@ -77,7 +77,7 @@ func (dq *diskQueue) run() { dq.handleDeleterLoopResponse(deleterLoopResponse) // If there are still files waiting to be deleted, send another request. - dq.maybeDeleteAcked() + dq.maybeDeleteACKed() // If there were blocked producers waiting for more queue space, // we might be able to unblock them now. @@ -153,7 +153,8 @@ func (dq *diskQueue) handleWriterLoopResponse(response writerLoopResponse) { func (dq *diskQueue) handleReaderLoopResponse(response readerLoopResponse) { dq.reading = false - // Advance the read offset based on what was just completed. + // Advance the frame / offset based on what was just completed. + dq.segments.nextReadFrameID += frameID(response.frameCount) dq.segments.nextReadOffset += segmentOffset(response.byteCount) var segment *queueSegment @@ -161,8 +162,7 @@ func (dq *diskQueue) handleReaderLoopResponse(response readerLoopResponse) { // A segment is finished if we have read all the data, or // the read response reports an error. // Segments in the reading list have been completely written, - // so we can rely on their endOffset field to determine the - // size of the data. + // so we can rely on their endOffset field to determine their size. segment = dq.segments.reading[0] if dq.segments.nextReadOffset >= segment.endOffset || response.err != nil { dq.segments.reading = dq.segments.reading[1:] @@ -174,7 +174,7 @@ func (dq *diskQueue) handleReaderLoopResponse(response readerLoopResponse) { // so we don't check the endOffset. segment = dq.segments.writing[0] } - segment.framesRead += response.frameCount + segment.framesRead = int64(dq.segments.nextReadFrameID - segment.firstFrameID) // If there was an error, report it. if response.err != nil { @@ -203,7 +203,7 @@ func (dq *diskQueue) handleDeleterLoopResponse(response deleterLoopResponse) { } } -func (dq *diskQueue) handleConsumerAck(ackedUpTo frameID) { +func (dq *diskQueue) handleConsumerACK(ackedUpTo frameID) { acking := dq.segments.acking if len(acking) == 0 { return @@ -305,10 +305,15 @@ func (dq *diskQueue) maybeReadPending() { // Nothing to read return } + if dq.segments.nextReadOffset == 0 { + // If we're reading the beginning of this segment, assign its firstFrameID. + segment.firstFrameID = dq.segments.nextReadFrameID + } request := readerLoopRequest{ - segment: segment, - startOffset: dq.segments.nextReadOffset, - endOffset: segment.endOffset, + segment: segment, + startFrameID: dq.segments.nextReadFrameID, + startOffset: dq.segments.nextReadOffset, + endOffset: segment.endOffset, } dq.readerLoop.requestChan <- request dq.reading = true @@ -316,7 +321,7 @@ func (dq *diskQueue) maybeReadPending() { // If the acked list is nonempty, and there are no outstanding deletion // requests, send one. -func (dq *diskQueue) maybeDeleteAcked() { +func (dq *diskQueue) maybeDeleteACKed() { if !dq.deleting && len(dq.segments.acked) > 0 { dq.deleterLoop.requestChan <- deleterLoopRequest{ segments: dq.segments.acked} diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 496f9564f4f..ba1d3c1fc89 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -20,63 +20,14 @@ package diskqueue import ( "fmt" "os" - "path/filepath" "sync" "github.com/elastic/beats/v7/libbeat/common" "github.com/elastic/beats/v7/libbeat/feature" "github.com/elastic/beats/v7/libbeat/logp" - "github.com/elastic/beats/v7/libbeat/paths" "github.com/elastic/beats/v7/libbeat/publisher/queue" ) -// Settings contains the configuration fields to create a new disk queue -// or open an existing one. -type Settings struct { - // The path on disk of the queue's containing directory, which will be - // created if it doesn't exist. Within the directory, the queue's state - // is stored in state.dat and each segment's data is stored in - // {segmentIndex}.seg - // If blank, the default directory is "diskqueue" within the beat's data - // directory. - Path string - - // MaxBufferSize is the maximum number of bytes that the queue should - // ever occupy on disk. A value of 0 means the queue can grow until the - // disk is full (this is not recommended on a primary system disk). - MaxBufferSize uint64 - - // MaxSegmentSize is the maximum number of bytes that should be written - // to a single segment file before creating a new one. - MaxSegmentSize uint64 - - // How many events will be read from disk while waiting for a consumer - // request. - ReadAheadLimit int - - // How many events will be queued in memory waiting to be written to disk. - // This setting should rarely matter in practice, but if data is coming - // in faster than it can be written to disk for an extended period, - // this limit can keep it from overflowing memory. - WriteAheadLimit int - - // A listener that should be sent ACKs when an event is successfully - // written to disk. - WriteToDiskListener queue.ACKListener -} - -type segmentID uint64 - -type queuePosition struct { - // The index of this position's segment within the queue. - segment segmentID - - // The byte offset of this position within its segment. - // This is relative to the start of the segment's data region, i.e. - // an offset of 0 means the first byte after the end of the segment header. - offset segmentOffset -} - // diskQueue is the internal type representing a disk-based implementation // of queue.Queue. type diskQueue struct { @@ -89,6 +40,10 @@ type diskQueue struct { // Metadata related to the segment files. segments *diskQueueSegments + // Metadata related to consumer acks / positions of the oldest remaining + // frame. + acks *diskQueueACKs + // The queue's helper loops, each of which is run in its own goroutine. readerLoop *readerLoop writerLoop *writerLoop @@ -108,7 +63,7 @@ type diskQueue struct { // The value sent on the channel is redundant with the value of ackedUpTo, // but we send it anyway so we don't have to worry about the core loop // waiting on ackLock. - consumerAckChan chan frameID + consumerACKChan chan frameID // writing is true if a writeRequest is currently being processed by the // writer loop, false otherwise. @@ -153,6 +108,26 @@ type diskQueue struct { done chan struct{} } +// queuePosition represents a logical position within the queue buffer. +type queuePosition struct { + segment segmentID + offset segmentOffset +} + +type diskQueueACKs struct { + // This lock must be held to access this structure. + lock sync.Mutex + + // The id and position of the first unacknowledged frame. + nextFrameID frameID + nextPosition queuePosition + + // A map of all acked indices that are above ackedUpTo (and thus + // can't yet be acknowledged as a continuous block). + // TODO: do this better. + acked map[frameID]bool +} + func init() { queue.RegisterQueueType( "disk", @@ -163,8 +138,8 @@ func init() { feature.Beta)) } -// queueFactory matches the queue.Factory type, and is used to add the disk -// queue to the registry. +// queueFactory matches the queue.Factory interface, and is used to add the +// disk queue to the registry. func queueFactory( ackListener queue.ACKListener, logger *logp.Logger, cfg *common.Config, ) (queue.Queue, error) { @@ -283,7 +258,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { producerWriteRequestChan: make(chan producerWriteRequest, 10), producerCancelRequestChan: make(chan producerCancelRequest), - consumerAckChan: make(chan frameID), + consumerACKChan: make(chan frameID), acked: make(map[frameID]bool), waitGroup: &waitGroup, @@ -299,32 +274,6 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { return queue, nil } -// -// bookkeeping helpers -// - -func (settings Settings) directoryPath() string { - if settings.Path == "" { - return paths.Resolve(paths.Data, "diskqueue") - } - - return settings.Path -} - -func (settings Settings) stateFilePath() string { - return filepath.Join(settings.directoryPath(), "state.dat") -} - -func (settings Settings) segmentPath(segmentID segmentID) string { - return filepath.Join( - settings.directoryPath(), - fmt.Sprintf("%v.seg", segmentID)) -} - -func (settings Settings) maxSegmentOffset() segmentOffset { - return segmentOffset(settings.MaxSegmentSize - segmentHeaderSize) -} - // // diskQueue implementation of the queue.Queue interface // diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 30728a0cce7..7c7c0e3e429 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -19,18 +19,16 @@ package diskqueue import ( "encoding/binary" - "errors" "fmt" - "io" "os" - "syscall" "time" ) type readerLoopRequest struct { - segment *queueSegment - startOffset segmentOffset - endOffset segmentOffset + segment *queueSegment + startOffset segmentOffset + startFrameID frameID + endOffset segmentOffset } type readerLoopResponse struct { @@ -66,11 +64,6 @@ type readerLoop struct { // The helper object to deserialize binary blobs from the queue into // publisher.Event objects that can be returned in a readFrame. decoder *eventDecoder - - // The id that will be assigned to the next successfully-read frame. - // Always starts from 0; this is just to track which frames have been - // acknowledged, and doesn't need any consistency between runs. - nextFrameID frameID } func (rl *readerLoop) run() { @@ -97,6 +90,7 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon frameCount := int64(0) byteCount := int64(0) + nextFrameID := request.startFrameID // defer func() { // fmt.Printf(" \033[0;32mread %d bytes in %d frames\033[0m\n", byteCount, frameCount) // }() @@ -119,8 +113,9 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon // Try to read the next frame, clipping to the given bound. // If the next frame extends past this boundary, nextFrame will return // an error. - frame, err := rl.nextFrame(handle, remainingLength) + frame, err := rl.nextFrame(handle, remainingLength, nextFrameID) if frame != nil { + nextFrameID++ // We've read the frame, try sending it to the output channel. select { case rl.output <- frame: @@ -167,7 +162,7 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon } func (rl *readerLoop) nextFrame( - handle *os.File, maxLength int64, + handle *os.File, maxLength int64, id frameID, ) (*readFrame, error) { // Ensure we are allowed to read the frame header. if maxLength < frameHeaderSize { @@ -234,45 +229,15 @@ func (rl *readerLoop) nextFrame( // may not indicate data corruption in the segment). // TODO: Rather than pass this error back to the read request, which // discards the rest of the segment, we should just log the error and - // advance to the next frame, which is likely still valid (especially - // if checksums are enabled). + // advance to the next frame, which is likely still valid. return nil, fmt.Errorf("Couldn't decode data frame: %w", err) } frame := &readFrame{ event: event, - id: rl.nextFrameID, + id: id, bytesOnDisk: int64(frameLength), } - rl.nextFrameID++ return frame, nil } - -// A wrapper for an io.Reader that tries to read the full number of bytes -// requested, retrying on EAGAIN and EINTR, and returns an error if -// and only if the number of bytes read is less than requested. -// This is similar to io.ReadFull but with retrying. -type autoRetryReader struct { - wrapped io.Reader -} - -func (r autoRetryReader) Read(p []byte) (int, error) { - bytesRead := 0 - reader := r.wrapped - n, err := reader.Read(p) - for n < len(p) { - if err != nil && !readErrorIsRetriable(err) { - return bytesRead + n, err - } - // If there is an error, it is retriable, so advance p and try again. - bytesRead += n - p = p[n:] - n, err = reader.Read(p) - } - return bytesRead + n, nil -} - -func readErrorIsRetriable(err error) bool { - return errors.Is(err, syscall.EINTR) || errors.Is(err, syscall.EAGAIN) -} diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 2f8d97f9783..45c9caefaae 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -30,34 +30,30 @@ import ( // diskQueueSegments encapsulates segment-related queue metadata. type diskQueueSegments struct { - // The segments that are currently being written. The writer loop - // writes these segments in order. When a segment has been - // completely written, the writer loop notifies the core loop - // in a writeResponse, and it is moved to the reading list. - // If the reading list is empty, the reader loop may read from - // a segment that is still being written, but it will always - // be writing[0], since later entries have generally not been - // created yet. + // A list of the segments that have not yet been completely written, sorted + // by increasing segment ID. When the first entry has been completely + // written, it is removed from this list and appended to reading. + // + // If the reading list is empty, the queue may read from a segment that is + // still being written, but it will always be writing[0], since later + // entries do not yet exist on disk. writing []*queueSegment // A list of the segments that have been completely written but have - // not yet been completely processed by the reader loop, sorted by increasing - // segment ID. Segments are always read in order. When a segment has - // been read completely, it is removed from the front of this list and - // appended to read. + // not yet been completely read, sorted by increasing segment ID. When the + // first entry has been completely read, it is removed from this list and + // appended to acking. reading []*queueSegment - // A list of the segments that have been read but have not yet been - // completely acknowledged, sorted by increasing segment ID. When the - // first entry of this list is completely acknowledged, it is removed - // from this list and added to acked. + // A list of the segments that have been completely read but have not yet + // been completely acknowledged, sorted by increasing segment ID. When the + // first entry has been completely acknowledged, it is removed from this + // list and appended to acked. acking []*queueSegment - // A list of the segments that have been completely processed and are - // ready to be deleted. The writer loop always tries to delete segments - // in this list before writing new data. When a segment is successfully - // deleted, it is removed from this list and the queue's - // segmentDeletedCond is signalled. + // A list of the segments that have been completely read and acknowledged + // and are ready to be deleted. When a segment is successfully deleted, it + // is removed from this list and discarded. acked []*queueSegment // The next sequential unused segment ID. This is what will be assigned @@ -72,12 +68,20 @@ type diskQueueSegments struct { // written. nextWriteOffset segmentOffset - // nextReadOffset is the position to start reading during the next - // read request. This offset always applies to the first reading + // nextReadFrameID is the first frame ID in the current or pending + // read request. + nextReadFrameID frameID + + // nextReadOffset is the segment offset corresponding to the frame + // nextReadFrameID. This offset always applies to the first reading // segment: either reading[0], or writing[0] if reading is empty. nextReadOffset segmentOffset } +// segmentID is a unique persistent integer id assigned to each created +// segment in ascending order. +type segmentID uint64 + // segmentOffset is a byte index into the segment's data region. // An offset of 0 means the first byte after the segment file header. type segmentOffset uint64 @@ -93,6 +97,12 @@ type queueSegment struct { // on disk is segmentHeaderSize + segment.endOffset. endOffset segmentOffset + // The ID of the first frame that was / will be read from this segment. + // This field is only valid after a read request has been sent for + // this segment. (Currently it is only used to handle consumer ACKs, + // which can only happen after reading has begun on the segment.) + firstFrameID frameID + // The number of frames read from this segment during this session. This // does not necessarily equal the number of frames in the segment, even // after reading is complete, since the segment may have been partially diff --git a/libbeat/publisher/queue/diskqueue/util.go b/libbeat/publisher/queue/diskqueue/util.go new file mode 100644 index 00000000000..a988f88348b --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/util.go @@ -0,0 +1,89 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +import ( + "errors" + "io" + "syscall" +) + +// A wrapper for an io.Reader that tries to read the full number of bytes +// requested, retrying on EAGAIN and EINTR, and returns an error if +// and only if the number of bytes read is less than requested. +// This is similar to io.ReadFull but with retrying. +type autoRetryReader struct { + wrapped io.Reader +} + +func (r autoRetryReader) Read(p []byte) (int, error) { + bytesRead := 0 + reader := r.wrapped + n, err := reader.Read(p) + for n < len(p) { + if err != nil && !readErrorIsRetriable(err) { + return bytesRead + n, err + } + // If there is an error, it is retriable, so advance p and try again. + bytesRead += n + p = p[n:] + n, err = reader.Read(p) + } + return bytesRead + n, nil +} + +func readErrorIsRetriable(err error) bool { + return errors.Is(err, syscall.EINTR) || errors.Is(err, syscall.EAGAIN) +} + +// writeErrorIsRetriable returns true if the given IO error can be +// immediately retried. +func writeErrorIsRetriable(err error) bool { + return errors.Is(err, syscall.EINTR) || errors.Is(err, syscall.EAGAIN) +} + +// callbackRetryWriter is an io.Writer that wraps another writer and enables +// write-with-retry. When a Write encounters an error, it is passed to the +// retry callback. If the callback returns true, the the writer retries +// any unwritten portion of the input, otherwise it passes the error back +// to the caller. +// This helper is specifically for working with the writer loop, which needs +// to be able to retry forever at configurable intervals, but also cancel +// immediately if the queue is closed. +// This writer is unbuffered. In particular, it is safe to modify +// "wrapped" in-place as long as it isn't captured by the callback. +type callbackRetryWriter struct { + wrapped io.Writer + retry func(error) bool +} + +func (w callbackRetryWriter) Write(p []byte) (int, error) { + bytesWritten := 0 + writer := w.wrapped + n, err := writer.Write(p) + for n < len(p) { + if err != nil && !w.retry(err) { + return bytesWritten + n, err + } + // Advance p and try again. + bytesWritten += n + p = p[n:] + n, err = writer.Write(p) + } + return bytesWritten + n, nil +} diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 8f296a67412..6906018b3e6 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -19,10 +19,7 @@ package diskqueue import ( "encoding/binary" - "errors" - "io" "os" - "syscall" "time" "github.com/elastic/beats/v7/libbeat/logp" @@ -230,40 +227,3 @@ func (wl *writerLoop) retryCallback(err error) bool { return false } } - -// writeErrorIsRetriable returns true if the given IO error can be -// immediately retried. -func writeErrorIsRetriable(err error) bool { - return errors.Is(err, syscall.EINTR) || errors.Is(err, syscall.EAGAIN) -} - -// callbackRetryWriter is an io.Writer that wraps another writer and enables -// write-with-retry. When a Write encounters an error, it is passed to the -// retry callback. If the callback returns true, the the writer retries -// any unwritten portion of the input, otherwise it passes the error back -// to the caller. -// This helper is specifically for working with the writer loop, which needs -// to be able to retry forever at configurable intervals, but also cancel -// immediately if the queue is closed. -// This writer is unbuffered. In particular, it is safe to modify -// "wrapped" in-place as long as it isn't captured by the callback. -type callbackRetryWriter struct { - wrapped io.Writer - retry func(error) bool -} - -func (w callbackRetryWriter) Write(p []byte) (int, error) { - bytesWritten := 0 - writer := w.wrapped - n, err := writer.Write(p) - for n < len(p) { - if err != nil && !w.retry(err) { - return bytesWritten + n, err - } - // Advance p and try again. - bytesWritten += n - p = p[n:] - n, err = writer.Write(p) - } - return bytesWritten + n, nil -} From 0a6f9f4aac00542919a3c09670d54f2fea03e35a Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 17 Sep 2020 16:23:11 -0400 Subject: [PATCH 63/91] initialize the read position properly, continue consumer ack revision --- libbeat/publisher/queue/diskqueue/acks.go | 60 ++++++++++++++ libbeat/publisher/queue/diskqueue/consumer.go | 41 +++------- .../publisher/queue/diskqueue/core_loop.go | 10 +-- libbeat/publisher/queue/diskqueue/producer.go | 4 +- libbeat/publisher/queue/diskqueue/queue.go | 80 ++++++++----------- .../publisher/queue/diskqueue/state_file.go | 17 +++- 6 files changed, 127 insertions(+), 85 deletions(-) create mode 100644 libbeat/publisher/queue/diskqueue/acks.go diff --git a/libbeat/publisher/queue/diskqueue/acks.go b/libbeat/publisher/queue/diskqueue/acks.go new file mode 100644 index 00000000000..d765c3b2ba1 --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/acks.go @@ -0,0 +1,60 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +import "sync" + +// queuePosition represents a logical position within the queue buffer. +type queuePosition struct { + segmentID segmentID + offset segmentOffset +} + +type diskQueueACKs struct { + // This lock must be held to access this structure. + lock sync.Mutex + + // The id and position of the first unacknowledged frame. + nextFrameID frameID + nextPosition queuePosition + + // A map of all acked indices that are above ackedUpTo (and thus can't yet + // be acknowledged as a continuous block). + // TODO: do this better. + acked map[frameID]bool + + // When a segment has been completely acknowledged by a consumer, it sends + // the segment ID to this channel, where it is read by the core loop and + // scheduled for deletion. + segmentACKChan chan segmentID +} + +func (dqa *diskQueueACKs) ackFrames(frames []*readFrame) { + dqa.lock.Lock() + defer dqa.lock.Unlock() + for _, frame := range frames { + dqa.acked[frame.id] = true + } + if dqa.acked[dqa.nextFrameID] { + for ; dqa.acked[dqa.nextFrameID]; dqa.nextFrameID++ { + delete(dqa.acked, dqa.nextFrameID) + } + // TODO: we now need to send the segment id, not the frame id. + dqa.segmentACKChan <- dqa.nextFrameID + } +} diff --git a/libbeat/publisher/queue/diskqueue/consumer.go b/libbeat/publisher/queue/diskqueue/consumer.go index fc30ea75341..97e1f2b2e42 100644 --- a/libbeat/publisher/queue/diskqueue/consumer.go +++ b/libbeat/publisher/queue/diskqueue/consumer.go @@ -30,9 +30,10 @@ type diskQueueConsumer struct { } type diskQueueBatch struct { - queue *diskQueue - events []publisher.Event - ackIDs []frameID + queue *diskQueue + //events []publisher.Event + //ackIDs []frameID + frames []*readFrame } // @@ -67,17 +68,9 @@ eventLoop: } } - var ackIDs []frameID - var events []publisher.Event - for _, frame := range frames { - events = append(events, frame.event) - ackIDs = append(ackIDs, frame.id) - } - return &diskQueueBatch{ queue: consumer.queue, - events: events, - ackIDs: ackIDs, + frames: frames, }, nil } @@ -91,7 +84,11 @@ func (consumer *diskQueueConsumer) Close() error { // func (batch *diskQueueBatch) Events() []publisher.Event { - return batch.events + events := make([]publisher.Event, len(batch.frames)) + for i, frame := range batch.frames { + events[i] = frame.event + } + return events } // This is the only place that the queue state is changed from @@ -102,21 +99,5 @@ func (batch *diskQueueBatch) Events() []publisher.Event { // TODO: this shouldn't really be a dictionary, use a bitfield or // something more efficient. func (batch *diskQueueBatch) ACK() { - dq := batch.queue - dq.ackLock.Lock() - defer dq.ackLock.Unlock() - for _, frameID := range batch.ackIDs { - dq.acked[frameID] = true - } - if dq.acked[dq.ackedUpTo] { - for ; dq.acked[dq.ackedUpTo]; dq.ackedUpTo++ { - delete(dq.acked, dq.ackedUpTo) - } - // It would be considerate to send this less frequently, so - // as not to bother the core loop with messages that are - // usually no-ops. - // TODO: only inform the core loop when we cross a segment - // boundary. - dq.consumerACKChan <- dq.ackedUpTo - } + batch.queue.acks.ackFrames(batch.frames) } diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 39622fadfcb..b09c92fa0dd 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -44,8 +44,8 @@ func (dq *diskQueue) run() { // TODO: this isn't really handled yet. dq.handleProducerCancelRequest(cancelRequest) - case ackedUpTo := <-dq.consumerACKChan: - dq.handleConsumerACK(ackedUpTo) + case ackedSegmentID := <-dq.acks.segmentACKChan: + dq.handleSegmentACK(ackedSegmentID) // After receiving new ACKs, a segment might be ready to delete. dq.maybeDeleteACKed() @@ -203,8 +203,8 @@ func (dq *diskQueue) handleDeleterLoopResponse(response deleterLoopResponse) { } } -func (dq *diskQueue) handleConsumerACK(ackedUpTo frameID) { - acking := dq.segments.acking +func (dq *diskQueue) handleSegmentACK(ackedSegmentID segmentID) { + /*acking := dq.segments.acking if len(acking) == 0 { return } @@ -228,7 +228,7 @@ func (dq *diskQueue) handleConsumerACK(ackedUpTo frameID) { dq.segments.acking = acking[ackedSegmentCount:] // Advance oldestFrameID past the segments we just removed. dq.oldestFrameID = endFrame - } + }*/ } func (dq *diskQueue) handleShutdown() { diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index f1aa174c8a2..8d2312f5563 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -75,7 +75,9 @@ func (producer *diskQueueProducer) publish( serialized: serialized, producer: producer, }, - shouldBlock: shouldBlock, + shouldBlock: shouldBlock, + // This response channel will be used by the core loop, so it must have + // buffer size 1 to guarantee that the core loop will not need to block. responseChan: make(chan bool, 1), } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index ba1d3c1fc89..dd058186d58 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -18,6 +18,7 @@ package diskqueue import ( + "errors" "fmt" "os" "sync" @@ -34,9 +35,6 @@ type diskQueue struct { logger *logp.Logger settings Settings - // The persistent queue state (wraps diskQueuePersistentState on disk). - stateFile *stateFile - // Metadata related to the segment files. segments *diskQueueSegments @@ -57,14 +55,6 @@ type diskQueue struct { producerWriteRequestChan chan producerWriteRequest producerCancelRequestChan chan producerCancelRequest - // When a consumer ack increments ackedUpTo, the consumer sends - // its new value to this channel. The core loop then decides whether to - // delete the containing segments. - // The value sent on the channel is redundant with the value of ackedUpTo, - // but we send it anyway so we don't have to worry about the core loop - // waiting on ackLock. - consumerACKChan chan frameID - // writing is true if a writeRequest is currently being processed by the // writer loop, false otherwise. writing bool @@ -108,26 +98,6 @@ type diskQueue struct { done chan struct{} } -// queuePosition represents a logical position within the queue buffer. -type queuePosition struct { - segment segmentID - offset segmentOffset -} - -type diskQueueACKs struct { - // This lock must be held to access this structure. - lock sync.Mutex - - // The id and position of the first unacknowledged frame. - nextFrameID frameID - nextPosition queuePosition - - // A map of all acked indices that are above ackedUpTo (and thus - // can't yet be acknowledged as a continuous block). - // TODO: do this better. - acked map[frameID]bool -} - func init() { queue.RegisterQueueType( "disk", @@ -172,6 +142,14 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { return nil, fmt.Errorf("Couldn't create disk queue directory: %w", err) } + // Load the previous queue position, if any. + nextReadPosition, err := nextReadPositionFromPath(settings.stateFilePath()) + if err != nil && !errors.Is(err, os.ErrNotExist) { + // Errors reading / writing the position are non-fatal -- we just log a + // warning and fall back on the oldest existing segment, if any. + logger.Warnf("Couldn't load most recent queue position: %v", err) + } + // Index any existing data segments to be placed in segments.reading. initialSegments, err := scanExistingSegments(settings.directoryPath()) if err != nil { @@ -179,16 +157,26 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { } var nextSegmentID segmentID if len(initialSegments) > 0 { + // Initialize nextSegmentID to the first ID after the existing segments. lastID := initialSegments[len(initialSegments)-1].id nextSegmentID = lastID + 1 } - // Load the file handle for the queue state. - stateFile, err := stateFileForPath(settings.stateFilePath()) - if err != nil { - return nil, fmt.Errorf("Couldn't open disk queue metadata file: %w", err) + // If any of the initial segments are older than the read position from + // the state file, move them directly to the acked list where they can be + // deleted. + ackedSegments := []*queueSegment{} + readSegmentID := nextReadPosition.segmentID + for len(initialSegments) > 0 && initialSegments[0].id < readSegmentID { + ackedSegments = append(ackedSegments, initialSegments[0]) + initialSegments = initialSegments[1:] + } + + // If the next read position is older than all existing segments, advance + // it to the beginning of the first one. + if len(initialSegments) > 0 && readSegmentID < initialSegments[0].id { + nextReadPosition = queuePosition{segmentID: initialSegments[0].id} } - //if stateFile.loadedState. // We wait for four goroutines: core loop, reader loop, writer loop, // deleter loop. @@ -241,26 +229,24 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { logger: logger, settings: settings, - stateFile: stateFile, segments: &diskQueueSegments{ - reading: initialSegments, - nextID: nextSegmentID, + reading: initialSegments, + nextID: nextSegmentID, + nextReadOffset: nextReadPosition.offset, + }, + + acks: &diskQueueACKs{ + nextPosition: nextReadPosition, + segmentACKChan: make(chan segmentID), }, readerLoop: readerLoop, writerLoop: writerLoop, deleterLoop: deleterLoop, - // TODO: is this a reasonable size for this channel buffer? - // Should this be customizable? (Tentatively: no, since we - // expect most producer write requests to be handled near- - // instantly so the requests are unlikely to accumulate). - producerWriteRequestChan: make(chan producerWriteRequest, 10), + producerWriteRequestChan: make(chan producerWriteRequest), producerCancelRequestChan: make(chan producerCancelRequest), - consumerACKChan: make(chan frameID), - acked: make(map[frameID]bool), - waitGroup: &waitGroup, done: make(chan struct{}), } diff --git a/libbeat/publisher/queue/diskqueue/state_file.go b/libbeat/publisher/queue/diskqueue/state_file.go index c29e1a528cb..f3d2c55d106 100644 --- a/libbeat/publisher/queue/diskqueue/state_file.go +++ b/libbeat/publisher/queue/diskqueue/state_file.go @@ -68,7 +68,7 @@ func persistentStateFromHandle( return nil, err } - err = binary.Read(reader, binary.LittleEndian, &state.firstPosition.segment) + err = binary.Read(reader, binary.LittleEndian, &state.firstPosition.segmentID) if err != nil { return nil, err } @@ -100,7 +100,7 @@ func writePersistentStateToHandle( return err } - err = binary.Write(file, binary.LittleEndian, &firstPosition.segment) + err = binary.Write(file, binary.LittleEndian, &firstPosition.segmentID) if err != nil { return err } @@ -117,6 +117,19 @@ func (stateFile *stateFile) Close() error { return stateFile.file.Close() } +func nextReadPositionFromPath(path string) (queuePosition, error) { + // Try to open an existing state file. + file, err := os.OpenFile(path, os.O_RDONLY, 0600) + if err != nil { + return queuePosition{}, err + } + state, err := persistentStateFromHandle(file) + if err != nil { + return queuePosition{}, err + } + return state.firstPosition, nil +} + func stateFileForPath(path string) (*stateFile, error) { var state *diskQueuePersistentState var stateErr error From 91bc3b2dc44cf407651531b9d62147dab3ef5281 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Fri, 18 Sep 2020 11:33:16 -0400 Subject: [PATCH 64/91] still rewriting consumer acks --- libbeat/publisher/queue/diskqueue/acks.go | 20 ++++++++++++---- libbeat/publisher/queue/diskqueue/consumer.go | 6 ++--- .../publisher/queue/diskqueue/core_loop.go | 5 ++-- libbeat/publisher/queue/diskqueue/frames.go | 8 ++++++- libbeat/publisher/queue/diskqueue/queue.go | 2 ++ .../publisher/queue/diskqueue/reader_loop.go | 23 +++++++------------ 6 files changed, 38 insertions(+), 26 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/acks.go b/libbeat/publisher/queue/diskqueue/acks.go index d765c3b2ba1..7b83e18bc10 100644 --- a/libbeat/publisher/queue/diskqueue/acks.go +++ b/libbeat/publisher/queue/diskqueue/acks.go @@ -36,7 +36,9 @@ type diskQueueACKs struct { // A map of all acked indices that are above ackedUpTo (and thus can't yet // be acknowledged as a continuous block). // TODO: do this better. - acked map[frameID]bool + //acked map[frameID]bool + + segments map[segmentID]segmentACKs // When a segment has been completely acknowledged by a consumer, it sends // the segment ID to this channel, where it is read by the core loop and @@ -44,17 +46,27 @@ type diskQueueACKs struct { segmentACKChan chan segmentID } -func (dqa *diskQueueACKs) ackFrames(frames []*readFrame) { +// segmentACKs stores the ACKs for a single segment. If a frame has been +// ACKed, then segmentACKs[frameID] contains its size on disk. The size is +// used to track the queuePosition of the oldest remaining frame, which is +// written to disk as ACKs are received. (We do this to avoid duplicating +// events if the beat terminates without a clean shutdown.) +type segmentACKs map[frameID]int64 + +func (dqa *diskQueueACKs) addFrames(frames []*readFrame) { dqa.lock.Lock() defer dqa.lock.Unlock() for _, frame := range frames { - dqa.acked[frame.id] = true + segmentID := frame.segmentID + segment := dqa.segments[segmentID] + segment[frame.id] = frame.bytesOnDisk + //dqa.acked[frame.id] = true } if dqa.acked[dqa.nextFrameID] { for ; dqa.acked[dqa.nextFrameID]; dqa.nextFrameID++ { delete(dqa.acked, dqa.nextFrameID) } // TODO: we now need to send the segment id, not the frame id. - dqa.segmentACKChan <- dqa.nextFrameID + //dqa.segmentACKChan <- dqa.nextFrameID } } diff --git a/libbeat/publisher/queue/diskqueue/consumer.go b/libbeat/publisher/queue/diskqueue/consumer.go index 97e1f2b2e42..3e8c67feaa4 100644 --- a/libbeat/publisher/queue/diskqueue/consumer.go +++ b/libbeat/publisher/queue/diskqueue/consumer.go @@ -30,9 +30,7 @@ type diskQueueConsumer struct { } type diskQueueBatch struct { - queue *diskQueue - //events []publisher.Event - //ackIDs []frameID + queue *diskQueue frames []*readFrame } @@ -99,5 +97,5 @@ func (batch *diskQueueBatch) Events() []publisher.Event { // TODO: this shouldn't really be a dictionary, use a bitfield or // something more efficient. func (batch *diskQueueBatch) ACK() { - batch.queue.acks.ackFrames(batch.frames) + batch.queue.acks.addFrames(batch.frames) } diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index b09c92fa0dd..75627b060fc 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -26,9 +26,10 @@ package diskqueue func (dq *diskQueue) run() { dq.logger.Debug("Core loop starting up...") - // Wake up the reader loop if there are segments available to read - // (from a previous instantiation of the queue). + // Wake up the reader and deleter loops if there are segments to process + // from a previous instantiation of the queue. dq.maybeReadPending() + dq.maybeDeleteACKed() for { select { diff --git a/libbeat/publisher/queue/diskqueue/frames.go b/libbeat/publisher/queue/diskqueue/frames.go index d2b03ea416b..97e2bdc179b 100644 --- a/libbeat/publisher/queue/diskqueue/frames.go +++ b/libbeat/publisher/queue/diskqueue/frames.go @@ -53,8 +53,14 @@ type writeFrame struct { // A frame that has been read from disk and is waiting to be read / // acknowledged through the consumer API. type readFrame struct { + // The segment containing this frame. + segment *queueSegment + + // The id of this frame. + id frameID + + // The event decoded from the data frame. event publisher.Event - id frameID // How much space this frame occupied on disk (before deserialization), // including the frame header / footer. diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index dd058186d58..d0b425c0a2b 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -236,7 +236,9 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { }, acks: &diskQueueACKs{ + nextFrameID: 0, nextPosition: nextReadPosition, + segments: make(map[segmentID]segmentACKs), segmentACKChan: make(chan segmentID), }, diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 7c7c0e3e429..446e5627c42 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -21,7 +21,6 @@ import ( "encoding/binary" "fmt" "os" - "time" ) type readerLoopRequest struct { @@ -75,25 +74,14 @@ func (rl *readerLoop) run() { return } response := rl.processRequest(request) - //fmt.Printf("\033[0;32mread response: read %d frames and %d bytes\033[0m\n", response.frameCount, response.byteCount) - // if response.err != nil { - // fmt.Printf("\033[0;32mresponse had err: %v\033[0m\n", response.err) - // } rl.responseChan <- response } } func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopResponse { - //fmt.Printf("\033[0;32mreaderLoop.processRequest(segment %d from %d to %d)\033[0m\n", request.segment.id, request.startOffset, request.endOffset) - - defer time.Sleep(time.Second) - frameCount := int64(0) byteCount := int64(0) nextFrameID := request.startFrameID - // defer func() { - // fmt.Printf(" \033[0;32mread %d bytes in %d frames\033[0m\n", byteCount, frameCount) - // }() // Open the file and seek to the starting position. handle, err := request.segment.getReader(rl.settings) @@ -113,8 +101,11 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon // Try to read the next frame, clipping to the given bound. // If the next frame extends past this boundary, nextFrame will return // an error. - frame, err := rl.nextFrame(handle, remainingLength, nextFrameID) + frame, err := rl.nextFrame(handle, remainingLength) if frame != nil { + // Add the segment / frame ID, which nextFrame leaves blank. + frame.segment = request.segment + frame.id = nextFrameID nextFrameID++ // We've read the frame, try sending it to the output channel. select { @@ -161,8 +152,11 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon } } +// nextFrame reads and decodes one frame from the given file handle, as long +// it does not exceed the given length bound. The returned frame leaves the +// segment and frame IDs unset. func (rl *readerLoop) nextFrame( - handle *os.File, maxLength int64, id frameID, + handle *os.File, maxLength int64, ) (*readFrame, error) { // Ensure we are allowed to read the frame header. if maxLength < frameHeaderSize { @@ -235,7 +229,6 @@ func (rl *readerLoop) nextFrame( frame := &readFrame{ event: event, - id: id, bytesOnDisk: int64(frameLength), } From aea4cadf29b98b96b08252e706592a906b5abeb5 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Fri, 18 Sep 2020 11:53:03 -0400 Subject: [PATCH 65/91] plug consumer acks back in --- libbeat/publisher/queue/diskqueue/acks.go | 39 +++++++++++++++---- .../publisher/queue/diskqueue/core_loop.go | 15 ++----- libbeat/publisher/queue/diskqueue/queue.go | 9 +++-- 3 files changed, 41 insertions(+), 22 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/acks.go b/libbeat/publisher/queue/diskqueue/acks.go index 7b83e18bc10..0212306e283 100644 --- a/libbeat/publisher/queue/diskqueue/acks.go +++ b/libbeat/publisher/queue/diskqueue/acks.go @@ -38,7 +38,17 @@ type diskQueueACKs struct { // TODO: do this better. //acked map[frameID]bool - segments map[segmentID]segmentACKs + // If a frame has been ACKed, then frames[frameID] contains its size on + // disk. The size is used to track the queuePosition of the oldest + // remaining frame, which is written to disk as ACKs are received. (We do + // this to avoid duplicating events if the beat terminates without a clean + // shutdown.) + frames map[frameID]int64 + //segments map[segmentID]segmentACKs + + // segmentBoundaries maps the first frameID of each segment to its + // corresponding segment ID. + segmentBoundaries map[frameID]segmentID // When a segment has been completely acknowledged by a consumer, it sends // the segment ID to this channel, where it is read by the core loop and @@ -57,14 +67,29 @@ func (dqa *diskQueueACKs) addFrames(frames []*readFrame) { dqa.lock.Lock() defer dqa.lock.Unlock() for _, frame := range frames { - segmentID := frame.segmentID - segment := dqa.segments[segmentID] - segment[frame.id] = frame.bytesOnDisk + //segmentID := frame.segmentID + segment := frame.segment + if frame.id == segment.firstFrameID { + // This is the first frame in its segment, mark it so we know when + // we're starting a new segment. + dqa.segmentBoundaries[frame.id] = segment.id + } + //segment := dqa.segments[segmentID] + //segment[frame.id] = frame.bytesOnDisk //dqa.acked[frame.id] = true + dqa.frames[frame.id] = frame.bytesOnDisk } - if dqa.acked[dqa.nextFrameID] { - for ; dqa.acked[dqa.nextFrameID]; dqa.nextFrameID++ { - delete(dqa.acked, dqa.nextFrameID) + if dqa.frames[dqa.nextFrameID] != 0 { + //for ; dqa.frames[dqa.nextFrameID] != 0; dqa.nextFrameID++ { + for dqa.frames[dqa.nextFrameID] != 0 { + segmentID, ok := dqa.segmentBoundaries[dqa.nextFrameID] + if ok { + // This is the start of a new segment, inform the ACK channel that + // earlier segments are completely acknowledged. + dqa.segmentACKChan <- segmentID - 1 + delete(dqa.segmentBoundaries, dqa.nextFrameID) + } + delete(dqa.frames, dqa.nextFrameID) } // TODO: we now need to send the segment id, not the frame id. //dqa.segmentACKChan <- dqa.nextFrameID diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 75627b060fc..f30d0c7f784 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -205,21 +205,16 @@ func (dq *diskQueue) handleDeleterLoopResponse(response deleterLoopResponse) { } func (dq *diskQueue) handleSegmentACK(ackedSegmentID segmentID) { - /*acking := dq.segments.acking + acking := dq.segments.acking if len(acking) == 0 { return } - startFrame := dq.oldestFrameID - endFrame := startFrame ackedSegmentCount := 0 for ; ackedSegmentCount < len(acking); ackedSegmentCount++ { - segment := acking[ackedSegmentCount] - if endFrame+frameID(segment.framesRead) > ackedUpTo { - // This segment is still waiting for acks, we're done. + if acking[ackedSegmentCount].id > ackedSegmentID { + // This segment has not been acked yet, we're done. break } - // Otherwise, advance the ending frame ID. - endFrame += frameID(segment.framesRead) } if ackedSegmentCount > 0 { // Move fully acked segments to the acked list and remove them @@ -227,9 +222,7 @@ func (dq *diskQueue) handleSegmentACK(ackedSegmentID segmentID) { dq.segments.acked = append(dq.segments.acked, acking[:ackedSegmentCount]...) dq.segments.acking = acking[ackedSegmentCount:] - // Advance oldestFrameID past the segments we just removed. - dq.oldestFrameID = endFrame - }*/ + } } func (dq *diskQueue) handleShutdown() { diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index d0b425c0a2b..cc6f5594075 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -236,10 +236,11 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { }, acks: &diskQueueACKs{ - nextFrameID: 0, - nextPosition: nextReadPosition, - segments: make(map[segmentID]segmentACKs), - segmentACKChan: make(chan segmentID), + nextFrameID: 0, + nextPosition: nextReadPosition, + frames: make(map[frameID]int64), + segmentBoundaries: make(map[frameID]segmentID), + segmentACKChan: make(chan segmentID), }, readerLoop: readerLoop, From a8ca56a67a0f7659668696c3a51ce04fb3f89953 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Fri, 18 Sep 2020 12:14:07 -0400 Subject: [PATCH 66/91] ...increment the loop variable -.- --- libbeat/publisher/queue/diskqueue/acks.go | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/acks.go b/libbeat/publisher/queue/diskqueue/acks.go index 0212306e283..4fd7b9eeb45 100644 --- a/libbeat/publisher/queue/diskqueue/acks.go +++ b/libbeat/publisher/queue/diskqueue/acks.go @@ -17,7 +17,9 @@ package diskqueue -import "sync" +import ( + "sync" +) // queuePosition represents a logical position within the queue buffer. type queuePosition struct { @@ -67,23 +69,18 @@ func (dqa *diskQueueACKs) addFrames(frames []*readFrame) { dqa.lock.Lock() defer dqa.lock.Unlock() for _, frame := range frames { - //segmentID := frame.segmentID segment := frame.segment if frame.id == segment.firstFrameID { // This is the first frame in its segment, mark it so we know when // we're starting a new segment. dqa.segmentBoundaries[frame.id] = segment.id } - //segment := dqa.segments[segmentID] - //segment[frame.id] = frame.bytesOnDisk - //dqa.acked[frame.id] = true dqa.frames[frame.id] = frame.bytesOnDisk } if dqa.frames[dqa.nextFrameID] != 0 { - //for ; dqa.frames[dqa.nextFrameID] != 0; dqa.nextFrameID++ { - for dqa.frames[dqa.nextFrameID] != 0 { - segmentID, ok := dqa.segmentBoundaries[dqa.nextFrameID] - if ok { + for ; dqa.frames[dqa.nextFrameID] != 0; dqa.nextFrameID++ { + segmentID := dqa.segmentBoundaries[dqa.nextFrameID] + if segmentID > 0 { // This is the start of a new segment, inform the ACK channel that // earlier segments are completely acknowledged. dqa.segmentACKChan <- segmentID - 1 @@ -91,7 +88,5 @@ func (dqa *diskQueueACKs) addFrames(frames []*readFrame) { } delete(dqa.frames, dqa.nextFrameID) } - // TODO: we now need to send the segment id, not the frame id. - //dqa.segmentACKChan <- dqa.nextFrameID } } From 6d91ab50cd25f8ce27532b9517fa0b390808e43b Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Fri, 18 Sep 2020 14:01:38 -0400 Subject: [PATCH 67/91] cleanups --- libbeat/publisher/queue/diskqueue/acks.go | 46 +++++++++---------- .../publisher/queue/diskqueue/core_loop.go | 4 +- libbeat/publisher/queue/diskqueue/queue.go | 2 +- .../publisher/queue/diskqueue/writer_loop.go | 2 +- 4 files changed, 27 insertions(+), 27 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/acks.go b/libbeat/publisher/queue/diskqueue/acks.go index 4fd7b9eeb45..342f335bda6 100644 --- a/libbeat/publisher/queue/diskqueue/acks.go +++ b/libbeat/publisher/queue/diskqueue/acks.go @@ -35,18 +35,12 @@ type diskQueueACKs struct { nextFrameID frameID nextPosition queuePosition - // A map of all acked indices that are above ackedUpTo (and thus can't yet - // be acknowledged as a continuous block). - // TODO: do this better. - //acked map[frameID]bool - - // If a frame has been ACKed, then frames[frameID] contains its size on + // If a frame has been ACKed, then frameSize[frameID] contains its size on // disk. The size is used to track the queuePosition of the oldest // remaining frame, which is written to disk as ACKs are received. (We do // this to avoid duplicating events if the beat terminates without a clean // shutdown.) - frames map[frameID]int64 - //segments map[segmentID]segmentACKs + frameSize map[frameID]int64 // segmentBoundaries maps the first frameID of each segment to its // corresponding segment ID. @@ -58,35 +52,41 @@ type diskQueueACKs struct { segmentACKChan chan segmentID } -// segmentACKs stores the ACKs for a single segment. If a frame has been -// ACKed, then segmentACKs[frameID] contains its size on disk. The size is -// used to track the queuePosition of the oldest remaining frame, which is -// written to disk as ACKs are received. (We do this to avoid duplicating -// events if the beat terminates without a clean shutdown.) -type segmentACKs map[frameID]int64 - func (dqa *diskQueueACKs) addFrames(frames []*readFrame) { dqa.lock.Lock() defer dqa.lock.Unlock() for _, frame := range frames { segment := frame.segment - if frame.id == segment.firstFrameID { + if frame.id != 0 && frame.id == segment.firstFrameID { // This is the first frame in its segment, mark it so we know when // we're starting a new segment. + // + // Subtlety: we don't count the very first frame as a "boundary" even + // though it is the first frame we read from its segment. This prevents + // us from resetting our segment offset to zero, in case the initial + // offset was restored from a previous session instead of starting at + // the beginning of the first segment. dqa.segmentBoundaries[frame.id] = segment.id } - dqa.frames[frame.id] = frame.bytesOnDisk + dqa.frameSize[frame.id] = frame.bytesOnDisk } - if dqa.frames[dqa.nextFrameID] != 0 { - for ; dqa.frames[dqa.nextFrameID] != 0; dqa.nextFrameID++ { - segmentID := dqa.segmentBoundaries[dqa.nextFrameID] - if segmentID > 0 { + if dqa.frameSize[dqa.nextFrameID] != 0 { + for ; dqa.frameSize[dqa.nextFrameID] != 0; dqa.nextFrameID++ { + newSegment, ok := dqa.segmentBoundaries[dqa.nextFrameID] + if ok { // This is the start of a new segment, inform the ACK channel that // earlier segments are completely acknowledged. - dqa.segmentACKChan <- segmentID - 1 + dqa.segmentACKChan <- newSegment - 1 delete(dqa.segmentBoundaries, dqa.nextFrameID) + + // Update the position to the start of the new segment. + dqa.nextPosition = queuePosition{ + segmentID: newSegment, + offset: 0, + } } - delete(dqa.frames, dqa.nextFrameID) + dqa.nextPosition.offset += segmentOffset(dqa.frameSize[dqa.nextFrameID]) + delete(dqa.frameSize, dqa.nextFrameID) } } } diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index f30d0c7f784..8cbc7fc5817 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -266,11 +266,11 @@ func (dq *diskQueue) maybeWritePending() { return } // Remove everything from pendingFrames and forward it to the writer loop. - requests := dq.pendingFrames + frames := dq.pendingFrames dq.pendingFrames = nil dq.writerLoop.requestChan <- writerLoopRequest{ - frames: requests, + frames: frames, } dq.writing = true } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index cc6f5594075..2bdcc3f0e5d 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -238,7 +238,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { acks: &diskQueueACKs{ nextFrameID: 0, nextPosition: nextReadPosition, - frames: make(map[frameID]int64), + frameSize: make(map[frameID]int64), segmentBoundaries: make(map[frameID]segmentID), segmentACKChan: make(chan segmentID), }, diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 6906018b3e6..790659806f1 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -88,7 +88,7 @@ func (wl *writerLoop) run() { for { block, ok := <-wl.requestChan if !ok { - // The requst channel is closed, we are done + // The request channel is closed, we are done return } bytesWritten := wl.processRequest(block) From 767aabbaa60a00690b6318c689b3d41359eadaa2 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Mon, 21 Sep 2020 11:30:07 -0400 Subject: [PATCH 68/91] remove unused variables --- libbeat/publisher/queue/diskqueue/queue.go | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 2bdcc3f0e5d..cf409b6ecd7 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -78,22 +78,6 @@ type diskQueue struct { // waiting for free space in the queue. blockedProducers []producerWriteRequest - // This value represents the oldest frame ID for a segment that has not - // yet been moved to the acked list. It is used to detect when the oldest - // outstanding segment has been fully acknowledged by the consumer. - oldestFrameID frameID - - // This lock must be held to read and write acked and ackedUpTo. - ackLock sync.Mutex - - // The lowest frame id that has not yet been acknowledged. - ackedUpTo frameID - - // A map of all acked indices that are above ackedUpTo (and thus - // can't yet be acknowledged as a continuous block). - // TODO: do this better. - acked map[frameID]bool - // The channel to signal our goroutines to shut down. done chan struct{} } From 0bf668e4371399132eebe680a2363773bc06d41a Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Mon, 21 Sep 2020 16:33:27 -0400 Subject: [PATCH 69/91] shutdown cleanly --- libbeat/publisher/queue/diskqueue/acks.go | 35 +++++++-- libbeat/publisher/queue/diskqueue/consumer.go | 31 +++++--- .../publisher/queue/diskqueue/core_loop.go | 75 ++++++++++++++----- .../publisher/queue/diskqueue/deleter_loop.go | 18 +++-- libbeat/publisher/queue/diskqueue/queue.go | 6 +- 5 files changed, 121 insertions(+), 44 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/acks.go b/libbeat/publisher/queue/diskqueue/acks.go index 342f335bda6..1dcaa5df1f5 100644 --- a/libbeat/publisher/queue/diskqueue/acks.go +++ b/libbeat/publisher/queue/diskqueue/acks.go @@ -28,7 +28,8 @@ type queuePosition struct { } type diskQueueACKs struct { - // This lock must be held to access this structure. + // This lock must be held to access diskQueueACKs fields (except for + // diskQueueACKs.done, which is always safe). lock sync.Mutex // The id and position of the first unacknowledged frame. @@ -50,11 +51,22 @@ type diskQueueACKs struct { // the segment ID to this channel, where it is read by the core loop and // scheduled for deletion. segmentACKChan chan segmentID + + // When the queue is closed, diskQueueACKs.done is closed to signal that + // the core loop will not accept any more acked segments and any future + // ACKs should be ignored. + done chan struct{} } func (dqa *diskQueueACKs) addFrames(frames []*readFrame) { dqa.lock.Lock() defer dqa.lock.Unlock() + select { + case <-dqa.done: + // We are already done and should ignore any leftover ACKs we receive. + return + default: + } for _, frame := range frames { segment := frame.segment if frame.id != 0 && frame.id == segment.firstFrameID { @@ -65,21 +77,20 @@ func (dqa *diskQueueACKs) addFrames(frames []*readFrame) { // though it is the first frame we read from its segment. This prevents // us from resetting our segment offset to zero, in case the initial // offset was restored from a previous session instead of starting at - // the beginning of the first segment. + // the beginning of the first file. dqa.segmentBoundaries[frame.id] = segment.id } dqa.frameSize[frame.id] = frame.bytesOnDisk } + oldSegmentID := dqa.nextPosition.segmentID if dqa.frameSize[dqa.nextFrameID] != 0 { for ; dqa.frameSize[dqa.nextFrameID] != 0; dqa.nextFrameID++ { newSegment, ok := dqa.segmentBoundaries[dqa.nextFrameID] if ok { - // This is the start of a new segment, inform the ACK channel that - // earlier segments are completely acknowledged. - dqa.segmentACKChan <- newSegment - 1 + // This is the start of a new segment. Remove this frame from the + // segment boundary list and set the position to the start of the + // new segment. delete(dqa.segmentBoundaries, dqa.nextFrameID) - - // Update the position to the start of the new segment. dqa.nextPosition = queuePosition{ segmentID: newSegment, offset: 0, @@ -89,4 +100,14 @@ func (dqa *diskQueueACKs) addFrames(frames []*readFrame) { delete(dqa.frameSize, dqa.nextFrameID) } } + if oldSegmentID != dqa.nextPosition.segmentID { + // We crossed at least one segment boundary, inform the listener that + // everything before the current segment has been acknowledged (but bail + // out if our done channel has been closed, since that means there is no + // listener on the other end.) + select { + case dqa.segmentACKChan <- dqa.nextPosition.segmentID - 1: + case <-dqa.done: + } + } } diff --git a/libbeat/publisher/queue/diskqueue/consumer.go b/libbeat/publisher/queue/diskqueue/consumer.go index 3e8c67feaa4..b2922778ea5 100644 --- a/libbeat/publisher/queue/diskqueue/consumer.go +++ b/libbeat/publisher/queue/diskqueue/consumer.go @@ -40,8 +40,9 @@ type diskQueueBatch struct { func (consumer *diskQueueConsumer) Get(eventCount int) (queue.Batch, error) { if consumer.closed { - return nil, fmt.Errorf("Tried to read from closed disk queue consumer") + return nil, fmt.Errorf("Tried to read from a closed disk queue consumer") } + // Read at least one frame. This is guaranteed to eventually // succeed unless the queue is closed. frame, ok := <-consumer.queue.readerLoop.output @@ -52,7 +53,7 @@ func (consumer *diskQueueConsumer) Get(eventCount int) (queue.Batch, error) { eventLoop: for eventCount <= 0 || len(frames) < eventCount { select { - case frame, ok = <-consumer.queue.readerLoop.output: + case frame, ok := <-consumer.queue.readerLoop.output: if !ok { // The queue was closed while we were reading it, just send back // what we have so far. @@ -66,6 +67,25 @@ eventLoop: } } + // There is a mild race condition here based on queue closure: events + // written to readerLoop.output may have been buffered before the + // queue was closed, and we may be reading its leftovers afterwards. + // We could try to detect this case here by checking the + // consumer.queue.done channel, and return nothing if it's been closed. + // But this gives rise to another race: maybe the queue was + // closed _after_ we read those frames, and we _ought_ to return them + // to the reader. The queue interface doesn't specify the proper + // behavior in this case. + // + // Lacking formal requirements, we elect to be permissive: if we have + // managed to read frames, then the queue already knows and considers them + // "read," so we lose no consistency by returning them. If someone closes + // the queue while we are draining the channel, nothing changes functionally + // except that any ACKs after that point will be ignored. A well-behaved + // Beats shutdown will always ACK / close its consumers before closing the + // queue itself, so we expect this corner case not to arise in practice, but + // if it does it is innocuous. + return &diskQueueBatch{ queue: consumer.queue, frames: frames, @@ -89,13 +109,6 @@ func (batch *diskQueueBatch) Events() []publisher.Event { return events } -// This is the only place that the queue state is changed from -// outside the core loop. This is because ACKs are messy and bursty -// and we don't want the core loop to bottleneck on manipulating -// a potentially large dictionary, so we use a lock and let -// consumer threads handle most of the processing themselves. -// TODO: this shouldn't really be a dictionary, use a bitfield or -// something more efficient. func (batch *diskQueueBatch) ACK() { batch.queue.acks.addFrames(batch.frames) } diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 8cbc7fc5817..21e757f4322 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -24,8 +24,6 @@ package diskqueue // logical "state transition diagram" for queue operation. func (dq *diskQueue) run() { - dq.logger.Debug("Core loop starting up...") - // Wake up the reader and deleter loops if there are segments to process // from a previous instantiation of the queue. dq.maybeReadPending() @@ -226,36 +224,79 @@ func (dq *diskQueue) handleSegmentACK(ackedSegmentID segmentID) { } func (dq *diskQueue) handleShutdown() { - // We need to close the input channels for all other goroutines and - // wait for any outstanding responses. Order is important: handling - // a read response may require the deleter, so the reader must be - // shut down first. - + // Shutdown: first, we wait for any outstanding requests to complete, to + // make sure the helper loops are idle and all state is finalized, then + // we do final cleanup and write our position to disk. + + // Close the reader loop's request channel to signal an abort in case it's + // still processing a request -- we don't need any more frames. + // We still wait for acknowledgement afterwards: if there is a request in + // progress, it's possible that a consumer already read and acknowledged + // some of its data, so we want the final metadata before we write our + // closing state. close(dq.readerLoop.requestChan) if dq.reading { response := <-dq.readerLoop.responseChan dq.handleReaderLoopResponse(response) } + // We are assured by our callers within Beats that we will not be sent a + // shutdown signal until all our producers have already been finalized / + // shut down -- thus, there should be no writer requests outstanding, and + // writerLop.requestChan should be idle. But just in case (and in particular + // to handle the case where a request is stuck retrying a fatal error), + // we signal abort by closing the request channel, and read the final + // state if there is any. close(dq.writerLoop.requestChan) if dq.writing { - <-dq.writerLoop.responseChan + response := <-dq.writerLoop.responseChan + dq.handleWriterLoopResponse(response) } - close(dq.deleterLoop.requestChan) + // We let the deleter loop finish its current request, but we don't send + // the abort signal yet, since we might want to do one last deletion + // after checking the final consumer ACK state. if dq.deleting { response := <-dq.deleterLoop.responseChan - // We can't retry any more if deletion failed, but we still check the - // response so we can log any errors. - if len(response.errors) > 0 { - dq.logger.Errorw("Couldn't delete old segment files", - "errors", response.errors) - } + dq.handleDeleterLoopResponse(response) } - // TODO: wait (with timeout?) for any outstanding acks? + // If there are any blocked producers still hoping for space to open up + // in the queue, send them the bad news. + for _, request := range dq.blockedProducers { + request.responseChan <- false + } + dq.blockedProducers = nil + + // The reader and writer loops are now shut down, and the deleter loop is + // idle. The remaining cleanup is in finalizing the read position in the + // queue (the first event that hasn't been acknowledged by consumers), and + // in deleting any older segment files that may be left. + // + // Events read by consumers have been accumulating their ACK data in + // dq.acks. During regular operation the core loop is not allowed to use + // this data, since it requires holding a mutex, but during shutdown we're + // allowed to block to acquire it. However, we still must close its done + // channel first, otherwise the lock may be held by a consumer that is + // blocked trying to send us a message we're no longer listening to... + close(dq.acks.done) + dq.acks.lock.Lock() + finalPosition := dq.acks.nextPosition + dq.acks.lock.Unlock() + + if finalPosition.segmentID > 0 { + // All segments before the current one have been fully acknowledged. + dq.handleSegmentACK(finalPosition.segmentID - 1) + } + // Do one last round of deletions, then terminate the deleter loop. + dq.maybeDeleteACKed() + if dq.deleting { + response := <-dq.deleterLoop.responseChan + dq.handleDeleterLoopResponse(response) + } + close(dq.deleterLoop.requestChan) - // TODO: write final queue state to the metadata file. + // TODO: write finalPosition to disk. } // If the pendingFrames list is nonempty, and there are no outstanding diff --git a/libbeat/publisher/queue/diskqueue/deleter_loop.go b/libbeat/publisher/queue/diskqueue/deleter_loop.go index 1c22ee628f0..9674b71f191 100644 --- a/libbeat/publisher/queue/diskqueue/deleter_loop.go +++ b/libbeat/publisher/queue/diskqueue/deleter_loop.go @@ -71,20 +71,24 @@ func (dl *deleterLoop) run() { errorList = append(errorList, err) } } - dl.responseChan <- deleterLoopResponse{ - deleted: deleted, - errors: errorList, - } if len(request.segments) > 0 && len(deleted) == 0 { // If we were asked to delete segments but could not delete // _any_ of them, we haven't made progress. Returning an error // will log the issue and retry, but in this situation we // want to delay before retrying. The core loop itself can't // delay (it can never sleep or block), so we handle the - // delay here, by ignoring the next request until the retry - // interval has passed. + // delay here, by waiting before sending the result. + // The delay can be interrupted if the request channel is closed, + // indicating queue shutdown. + select { // TODO: make the retry interval configurable. - time.Sleep(time.Second) + case <-time.After(time.Second): + case <-dl.requestChan: + } + } + dl.responseChan <- deleterLoopResponse{ + deleted: deleted, + errors: errorList, } } } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index cf409b6ecd7..fd2c6b09e1c 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -225,6 +225,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { frameSize: make(map[frameID]int64), segmentBoundaries: make(map[frameID]segmentID), segmentACKChan: make(chan segmentID), + done: make(chan struct{}), }, readerLoop: readerLoop, @@ -273,8 +274,5 @@ func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { } func (dq *diskQueue) Consumer() queue.Consumer { - return &diskQueueConsumer{ - queue: dq, - closed: false, - } + return &diskQueueConsumer{queue: dq} } From 857077c88a5d2db5f037172aa9cd72c228bf4821 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Mon, 21 Sep 2020 16:43:54 -0400 Subject: [PATCH 70/91] delete the final segment on shutdown if everything is ACKed --- libbeat/publisher/queue/diskqueue/core_loop.go | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 21e757f4322..3a6c0525520 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -284,10 +284,18 @@ func (dq *diskQueue) handleShutdown() { finalPosition := dq.acks.nextPosition dq.acks.lock.Unlock() - if finalPosition.segmentID > 0 { - // All segments before the current one have been fully acknowledged. + // First check for the rare and fortunate case that every single event we + // wrote to the queue was ACKed. In this case it is safe to delete + // everything up to and including the current segment. Otherwise, we only + // delete things before the current segment. + if len(dq.segments.writing) > 0 && + finalPosition.segmentID == dq.segments.writing[0].id && + finalPosition.offset >= dq.segments.writing[0].endOffset { + dq.handleSegmentACK(finalPosition.segmentID) + } else if finalPosition.segmentID > 0 { dq.handleSegmentACK(finalPosition.segmentID - 1) } + // Do one last round of deletions, then terminate the deleter loop. dq.maybeDeleteACKed() if dq.deleting { From d4cd0bbdb0dfac0459be30643dbd2f97541e2558 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Mon, 21 Sep 2020 17:00:55 -0400 Subject: [PATCH 71/91] turn on basic producer canceling --- .../publisher/queue/diskqueue/core_loop.go | 6 ++--- libbeat/publisher/queue/diskqueue/producer.go | 22 ++++++++++++++++++- libbeat/publisher/queue/diskqueue/queue.go | 1 + 3 files changed, 25 insertions(+), 4 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 3a6c0525520..057d9be7dd7 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -229,7 +229,7 @@ func (dq *diskQueue) handleShutdown() { // we do final cleanup and write our position to disk. // Close the reader loop's request channel to signal an abort in case it's - // still processing a request -- we don't need any more frames. + // still processing a request (we don't need any more frames). // We still wait for acknowledgement afterwards: if there is a request in // progress, it's possible that a consumer already read and acknowledged // some of its data, so we want the final metadata before we write our @@ -241,7 +241,7 @@ func (dq *diskQueue) handleShutdown() { } // We are assured by our callers within Beats that we will not be sent a - // shutdown signal until all our producers have already been finalized / + // shutdown signal until all our producers have been finalized / // shut down -- thus, there should be no writer requests outstanding, and // writerLop.requestChan should be idle. But just in case (and in particular // to handle the case where a request is stuck retrying a fatal error), @@ -296,7 +296,7 @@ func (dq *diskQueue) handleShutdown() { dq.handleSegmentACK(finalPosition.segmentID - 1) } - // Do one last round of deletions, then terminate the deleter loop. + // Do one last round of deletions, then shut down the deleter loop. dq.maybeDeleteACKed() if dq.deleting { response := <-dq.deleterLoop.responseChan diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index 8d2312f5563..a0b56f06426 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -30,6 +30,13 @@ type diskQueueProducer struct { config queue.ProducerConfig encoder *eventEncoder + + // When a producer is cancelled, cancelled is set to true and the done + // channel is closed. (We could get by with just a done channel, but we + // need to make sure that calling Cancel repeatedly doesn't close an + // already-closed channel, which would panic.) + cancelled bool + done chan struct{} } // A request sent from a producer to the core loop to add a frame to the queue. @@ -63,6 +70,9 @@ func (producer *diskQueueProducer) TryPublish(event publisher.Event) bool { func (producer *diskQueueProducer) publish( event publisher.Event, shouldBlock bool, ) bool { + if producer.cancelled { + return false + } serialized, err := producer.encoder.encode(&event) if err != nil { producer.queue.logger.Errorf( @@ -91,9 +101,19 @@ func (producer *diskQueueProducer) publish( return response case <-producer.queue.done: return false + case <-producer.done: + return false } } func (producer *diskQueueProducer) Cancel() int { - panic("TODO: not implemented") + if producer.cancelled { + return 0 + } + producer.cancelled = true + close(producer.done) + return 0 + + // TODO (possibly?): message the core loop to remove any pending events that + // were sent through this producer. } diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index fd2c6b09e1c..2277d83d8a3 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -270,6 +270,7 @@ func (dq *diskQueue) Producer(cfg queue.ProducerConfig) queue.Producer { queue: dq, config: cfg, encoder: newEventEncoder(), + done: make(chan struct{}), } } From 91ce56fd0211c981fcd75a8a9a5b82832b44da2f Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 08:56:26 -0400 Subject: [PATCH 72/91] propagate event flags thru the queue --- libbeat/publisher/queue/diskqueue/producer.go | 6 +++--- .../publisher/queue/diskqueue/serialize.go | 20 +++++++++---------- 2 files changed, 12 insertions(+), 14 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index a0b56f06426..6674495bf52 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -96,7 +96,6 @@ func (producer *diskQueueProducer) publish( // The request has been sent, and we are now guaranteed to get a result on // the response channel, so we must read from it immediately to avoid // blocking the core loop. - // TODO: this should be unblocked by a call to Cancel response := <-request.responseChan return response case <-producer.queue.done: @@ -112,8 +111,9 @@ func (producer *diskQueueProducer) Cancel() int { } producer.cancelled = true close(producer.done) - return 0 // TODO (possibly?): message the core loop to remove any pending events that - // were sent through this producer. + // were sent through this producer. If we do, return the number of cancelled + // events here instead of zero. + return 0 } diff --git a/libbeat/publisher/queue/diskqueue/serialize.go b/libbeat/publisher/queue/diskqueue/serialize.go index 10062e3ef1c..ea8f12564e8 100644 --- a/libbeat/publisher/queue/diskqueue/serialize.go +++ b/libbeat/publisher/queue/diskqueue/serialize.go @@ -51,6 +51,11 @@ type entry struct { Fields common.MapStr } +const ( + // If + flagGuaranteed uint8 = 1 << 0 +) + func newEventEncoder() *eventEncoder { e := &eventEncoder{} e.reset() @@ -77,15 +82,14 @@ func (e *eventEncoder) reset() { func (e *eventEncoder) encode(event *publisher.Event) ([]byte, error) { e.buf.Reset() - var flags uint8 - // TODO: handle guaranteed send? - /*if (event.Flags & publisher.GuaranteedSend) == publisher.GuaranteedSend { + /*var flags uint8 + if (event.Flags & publisher.GuaranteedSend) == publisher.GuaranteedSend { flags = flagGuaranteed }*/ err := e.folder.Fold(entry{ Timestamp: event.Content.Timestamp.UTC().UnixNano(), - Flags: flags, + Flags: uint8(event.Flags), Meta: event.Content.Meta, Fields: event.Content.Fields, }) @@ -144,14 +148,8 @@ func (d *eventDecoder) Decode() (publisher.Event, error) { return publisher.Event{}, err } - var flags publisher.EventFlags - // TODO: handle guaranteed send? - /*if (to.Flags & flagGuaranteed) != 0 { - flags |= publisher.GuaranteedSend - }*/ - return publisher.Event{ - Flags: flags, + Flags: publisher.EventFlags(to.Flags), Content: beat.Event{ Timestamp: time.Unix(0, to.Timestamp), Fields: to.Fields, From f9dbfb16922f7ee17d23dc58c4fbd2d8ec257d79 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 09:21:00 -0400 Subject: [PATCH 73/91] clean up state_file / remove unused code --- libbeat/publisher/queue/diskqueue/queue.go | 2 +- .../publisher/queue/diskqueue/state_file.go | 133 +++++------------- 2 files changed, 34 insertions(+), 101 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 2277d83d8a3..ddd8ad42329 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -127,7 +127,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { } // Load the previous queue position, if any. - nextReadPosition, err := nextReadPositionFromPath(settings.stateFilePath()) + nextReadPosition, err := queuePositionFromPath(settings.stateFilePath()) if err != nil && !errors.Is(err, os.ErrNotExist) { // Errors reading / writing the position are non-fatal -- we just log a // warning and fall back on the oldest existing segment, if any. diff --git a/libbeat/publisher/queue/diskqueue/state_file.go b/libbeat/publisher/queue/diskqueue/state_file.go index f3d2c55d106..1869bf26251 100644 --- a/libbeat/publisher/queue/diskqueue/state_file.go +++ b/libbeat/publisher/queue/diskqueue/state_file.go @@ -24,70 +24,57 @@ import ( "os" ) -// This is the queue metadata that is saved to disk. Currently it only -// tracks the read position in the queue; all other data is contained -// in the segment files. -type diskQueuePersistentState struct { - // The schema version for the state file (currently always 0). - version uint32 - - // The oldest position in the queue. This is advanced as we receive ACKs from - // downstream consumers indicating it is safe to remove old events. - firstPosition queuePosition -} - -// A wrapper around os.File that saves and loads the queue state. -type stateFile struct { - // An open file handle to the queue's state file. - file *os.File - - // A pointer to the disk queue state that was read when this queue was - // opened, or nil if there was no preexisting state file. - loadedState *diskQueuePersistentState - - // If there was a non-fatal error loading the queue state, it is stored - // here. In this case, the queue overwrites the existing state file with - // a valid starting state. - stateErr error -} - -// Given an open file handle, decode the file as a diskQueuePersistentState +// Given an open file handle to the queue state, decode the current position // and return the result if successful, otherwise an error. -func persistentStateFromHandle( +func queuePositionFromHandle( file *os.File, -) (*diskQueuePersistentState, error) { +) (queuePosition, error) { _, err := file.Seek(0, 0) if err != nil { - return nil, err + return queuePosition{}, err } - state := diskQueuePersistentState{} reader := bufio.NewReader(file) - err = binary.Read(reader, binary.LittleEndian, &state.version) + var version uint32 + err = binary.Read(reader, binary.LittleEndian, &version) if err != nil { - return nil, err + return queuePosition{}, err + } + if version != 0 { + return queuePosition{}, + fmt.Errorf("Unsupported queue metadata version (%d)", version) } - err = binary.Read(reader, binary.LittleEndian, &state.firstPosition.segmentID) + position := queuePosition{} + err = binary.Read(reader, binary.LittleEndian, &position.segmentID) if err != nil { - return nil, err + return queuePosition{}, err } err = binary.Read( - reader, binary.LittleEndian, &state.firstPosition.offset) + reader, binary.LittleEndian, &position.offset) if err != nil { - return nil, err + return queuePosition{}, err } - return &state, nil + return position, nil +} + +func queuePositionFromPath(path string) (queuePosition, error) { + // Try to open an existing state file. + file, err := os.OpenFile(path, os.O_RDONLY, 0600) + if err != nil { + return queuePosition{}, err + } + defer file.Close() + return queuePositionFromHandle(file) } -// Given an open file handle and the first remaining position of a disk queue, -// binary encode the corresponding diskQueuePersistentState and overwrite the -// file with the result. Returns nil if successful, otherwise an error. -func writePersistentStateToHandle( +// Given the queue position, encode and write it to the given file handle. +// Returns nil if successful, otherwise an error. +func writeQueuePositionToHandle( file *os.File, - firstPosition queuePosition, + position queuePosition, ) error { _, err := file.Seek(0, 0) if err != nil { @@ -100,69 +87,15 @@ func writePersistentStateToHandle( return err } - err = binary.Write(file, binary.LittleEndian, &firstPosition.segmentID) + err = binary.Write(file, binary.LittleEndian, &position.segmentID) if err != nil { return err } - err = binary.Write(file, binary.LittleEndian, &firstPosition.offset) + err = binary.Write(file, binary.LittleEndian, &position.offset) if err != nil { return err } return nil } - -func (stateFile *stateFile) Close() error { - return stateFile.file.Close() -} - -func nextReadPositionFromPath(path string) (queuePosition, error) { - // Try to open an existing state file. - file, err := os.OpenFile(path, os.O_RDONLY, 0600) - if err != nil { - return queuePosition{}, err - } - state, err := persistentStateFromHandle(file) - if err != nil { - return queuePosition{}, err - } - return state.firstPosition, nil -} - -func stateFileForPath(path string) (*stateFile, error) { - var state *diskQueuePersistentState - var stateErr error - // Try to open an existing state file. - file, err := os.OpenFile(path, os.O_RDWR, 0600) - if err != nil { - // If we can't open the file, it's likely a new queue, so try to create it. - file, err = os.OpenFile(path, os.O_RDWR|os.O_CREATE, 0600) - if err != nil { - return nil, fmt.Errorf("Couldn't open disk queue metadata file: %w", err) - } - } else { - // Read the existing state. - state, stateErr = persistentStateFromHandle(file) - // This - if err != nil { - // TODO: this shouldn't be a fatal error. If the state file exists but - // its contents are invalid, then we should log a warning and overwrite - // it with metadata derived from the segment files instead. - return nil, err - } - } - result := &stateFile{ - file: file, - loadedState: state, - stateErr: stateErr, - } - if state == nil { - // Initialize with new zero state. - err = writePersistentStateToHandle(file, queuePosition{0, 0}) - if err != nil { - return nil, fmt.Errorf("Couldn't write queue state to disk: %w", err) - } - } - return result, nil -} From 93d3fff23cf655a19198ed52f3a5d6520576e241 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 09:57:06 -0400 Subject: [PATCH 74/91] save queue position as events are ACKed --- libbeat/publisher/queue/diskqueue/acks.go | 33 +++++++++++++++++++ .../publisher/queue/diskqueue/core_loop.go | 5 +-- libbeat/publisher/queue/diskqueue/queue.go | 20 ++++++----- 3 files changed, 48 insertions(+), 10 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/acks.go b/libbeat/publisher/queue/diskqueue/acks.go index 1dcaa5df1f5..53f5713a7ce 100644 --- a/libbeat/publisher/queue/diskqueue/acks.go +++ b/libbeat/publisher/queue/diskqueue/acks.go @@ -18,7 +18,10 @@ package diskqueue import ( + "os" "sync" + + "github.com/elastic/beats/v7/libbeat/logp" ) // queuePosition represents a logical position within the queue buffer. @@ -28,6 +31,8 @@ type queuePosition struct { } type diskQueueACKs struct { + logger *logp.Logger + // This lock must be held to access diskQueueACKs fields (except for // diskQueueACKs.done, which is always safe). lock sync.Mutex @@ -52,12 +57,33 @@ type diskQueueACKs struct { // scheduled for deletion. segmentACKChan chan segmentID + // An open writable file handle to the file that stores the queue position. + // This position is advanced as we receive ACKs, confirming it is safe + // to move forward, so the acking code is responsible for updating this + // file. + positionFile *os.File + // When the queue is closed, diskQueueACKs.done is closed to signal that // the core loop will not accept any more acked segments and any future // ACKs should be ignored. done chan struct{} } +func makeDiskQueueACKs( + logger *logp.Logger, position queuePosition, positionFile *os.File, +) *diskQueueACKs { + return &diskQueueACKs{ + logger: logger, + nextFrameID: 0, + nextPosition: position, + frameSize: make(map[frameID]int64), + segmentBoundaries: make(map[frameID]segmentID), + segmentACKChan: make(chan segmentID), + positionFile: positionFile, + done: make(chan struct{}), + } +} + func (dqa *diskQueueACKs) addFrames(frames []*readFrame) { dqa.lock.Lock() defer dqa.lock.Unlock() @@ -99,6 +125,13 @@ func (dqa *diskQueueACKs) addFrames(frames []*readFrame) { dqa.nextPosition.offset += segmentOffset(dqa.frameSize[dqa.nextFrameID]) delete(dqa.frameSize, dqa.nextFrameID) } + // We advanced the ACK position at least somewhat, so write its + // new value. + err := writeQueuePositionToHandle(dqa.positionFile, dqa.nextPosition) + if err != nil { + // TODO: Don't spam this warning on every ACK if it's a permanent error. + dqa.logger.Warnf("Couldn't save queue position: %v", err) + } } if oldSegmentID != dqa.nextPosition.segmentID { // We crossed at least one segment boundary, inform the listener that diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 057d9be7dd7..e2976d5031c 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -282,6 +282,9 @@ func (dq *diskQueue) handleShutdown() { close(dq.acks.done) dq.acks.lock.Lock() finalPosition := dq.acks.nextPosition + // We won't be updating the position anymore, so we can close the file. + dq.acks.positionFile.Sync() + dq.acks.positionFile.Close() dq.acks.lock.Unlock() // First check for the rare and fortunate case that every single event we @@ -303,8 +306,6 @@ func (dq *diskQueue) handleShutdown() { dq.handleDeleterLoopResponse(response) } close(dq.deleterLoop.requestChan) - - // TODO: write finalPosition to disk. } // If the pendingFrames list is nonempty, and there are no outstanding diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index ddd8ad42329..a9fc0df67da 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -133,6 +133,17 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { // warning and fall back on the oldest existing segment, if any. logger.Warnf("Couldn't load most recent queue position: %v", err) } + positionFile, err := os.OpenFile( + settings.stateFilePath(), os.O_WRONLY|os.O_CREATE, 0600) + if err != nil { + // This is not the _worst_ error: we could try operating even without a + // position file. But it indicates a problem with the queue permissions on + // disk, which keeps us from tracking our position within the segment files + // and could also prevent us from creating new ones, so we treat this as a + // fatal error on startup rather than quietly providing degraded + // performance. + return nil, fmt.Errorf("Couldn't write to state file: %v", err) + } // Index any existing data segments to be placed in segments.reading. initialSegments, err := scanExistingSegments(settings.directoryPath()) @@ -219,14 +230,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { nextReadOffset: nextReadPosition.offset, }, - acks: &diskQueueACKs{ - nextFrameID: 0, - nextPosition: nextReadPosition, - frameSize: make(map[frameID]int64), - segmentBoundaries: make(map[frameID]segmentID), - segmentACKChan: make(chan segmentID), - done: make(chan struct{}), - }, + acks: makeDiskQueueACKs(logger, nextReadPosition, positionFile), readerLoop: readerLoop, writerLoop: writerLoop, From 0ac493c168fabf81c29a57655a70e443793af621 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 11:51:22 -0400 Subject: [PATCH 75/91] remove producer cancel boilerplate (not needed in v1) --- libbeat/publisher/queue/diskqueue/core_loop.go | 10 ---------- libbeat/publisher/queue/diskqueue/producer.go | 9 --------- libbeat/publisher/queue/diskqueue/queue.go | 8 +++----- 3 files changed, 3 insertions(+), 24 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index e2976d5031c..1724269d748 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -39,10 +39,6 @@ func (dq *diskQueue) run() { // writer loop. dq.maybeWritePending() - case cancelRequest := <-dq.producerCancelRequestChan: - // TODO: this isn't really handled yet. - dq.handleProducerCancelRequest(cancelRequest) - case ackedSegmentID := <-dq.acks.segmentACKChan: dq.handleSegmentACK(ackedSegmentID) @@ -120,12 +116,6 @@ func (dq *diskQueue) handleProducerWriteRequest(request producerWriteRequest) { } } -func (dq *diskQueue) handleProducerCancelRequest( - request producerCancelRequest, -) { - // TODO: implement me -} - func (dq *diskQueue) handleWriterLoopResponse(response writerLoopResponse) { dq.writing = false diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index 6674495bf52..fb0e64689b0 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -46,15 +46,6 @@ type producerWriteRequest struct { responseChan chan bool } -// A request to the core loop to cancel the specified producer. -type producerCancelRequest struct { - producer *diskQueueProducer - // If producer.config.DropOnCancel is true, then the core loop will respond - // on responseChan with the number of dropped events. - // Otherwise, this field may be nil. - responseChan chan int -} - // // diskQueueProducer implementation of the queue.Producer interface // diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index a9fc0df67da..e996d0683f5 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -51,9 +51,8 @@ type diskQueue struct { // reader loop, writer loop, deleter loop, and core loop (diskQueue.run()). waitGroup *sync.WaitGroup - // The API channels used by diskQueueProducer to send write / cancel calls. - producerWriteRequestChan chan producerWriteRequest - producerCancelRequestChan chan producerCancelRequest + // The API channel used by diskQueueProducer to write events. + producerWriteRequestChan chan producerWriteRequest // writing is true if a writeRequest is currently being processed by the // writer loop, false otherwise. @@ -236,8 +235,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { writerLoop: writerLoop, deleterLoop: deleterLoop, - producerWriteRequestChan: make(chan producerWriteRequest), - producerCancelRequestChan: make(chan producerCancelRequest), + producerWriteRequestChan: make(chan producerWriteRequest), waitGroup: &waitGroup, done: make(chan struct{}), From 3af14214a2be44a75bc700f0b8caa6a621e87fb8 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 12:27:13 -0400 Subject: [PATCH 76/91] various review comments --- libbeat/publisher/queue/diskqueue/core_loop.go | 8 ++++---- libbeat/publisher/queue/diskqueue/deleter_loop.go | 2 +- libbeat/publisher/queue/diskqueue/reader_loop.go | 4 ++-- libbeat/publisher/queue/diskqueue/segments.go | 6 +++--- libbeat/publisher/queue/diskqueue/writer_loop.go | 2 +- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 1724269d748..2404425ab3b 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -233,10 +233,10 @@ func (dq *diskQueue) handleShutdown() { // We are assured by our callers within Beats that we will not be sent a // shutdown signal until all our producers have been finalized / // shut down -- thus, there should be no writer requests outstanding, and - // writerLop.requestChan should be idle. But just in case (and in particular - // to handle the case where a request is stuck retrying a fatal error), - // we signal abort by closing the request channel, and read the final - // state if there is any. + // writerLoop.requestChan should be idle. But just in case (and in + // particular to handle the case where a request is stuck retrying a fatal + // error), we signal abort by closing the request channel, and read the + // final state if there is any. close(dq.writerLoop.requestChan) if dq.writing { response := <-dq.writerLoop.responseChan diff --git a/libbeat/publisher/queue/diskqueue/deleter_loop.go b/libbeat/publisher/queue/diskqueue/deleter_loop.go index 9674b71f191..3dc379f3186 100644 --- a/libbeat/publisher/queue/diskqueue/deleter_loop.go +++ b/libbeat/publisher/queue/diskqueue/deleter_loop.go @@ -25,7 +25,7 @@ import ( type deleterLoop struct { // The settings for the queue that created this loop. - settings *Settings + settings Settings // When one or more segments are ready to delete, they are sent to // requestChan. At most one deleteRequest may be outstanding at any time. diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index 446e5627c42..a0b492fe1e8 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -44,7 +44,7 @@ type readerLoopResponse struct { type readerLoop struct { // The settings for the queue that created this loop. - settings *Settings + settings Settings // When there is a block available for reading, it will be sent to // requestChan. When the reader loop has finished processing it, it @@ -69,7 +69,7 @@ func (rl *readerLoop) run() { for { request, ok := <-rl.requestChan if !ok { - // The channel has closed, we are shutting down. + // The channel is closed, we are shutting down. close(rl.output) return } diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index 45c9caefaae..a32ffbe5181 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -165,7 +165,7 @@ func (segment *queueSegment) sizeOnDisk() uint64 { // Should only be called from the reader loop. func (segment *queueSegment) getReader( - queueSettings *Settings, + queueSettings Settings, ) (*os.File, error) { path := queueSettings.segmentPath(segment.id) file, err := os.Open(path) @@ -186,7 +186,7 @@ func (segment *queueSegment) getReader( // Should only be called from the writer loop. func (segment *queueSegment) getWriter( - queueSettings *Settings, + queueSettings Settings, ) (*os.File, error) { path := queueSettings.segmentPath(segment.id) file, err := os.OpenFile(path, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) @@ -207,7 +207,7 @@ func (segment *queueSegment) getWriter( // retry callback returns true. This is used for timed retries when // creating a queue segment from the writer loop. func (segment *queueSegment) getWriterWithRetry( - queueSettings *Settings, retry func(error) bool, + queueSettings Settings, retry func(error) bool, ) (*os.File, error) { file, err := segment.getWriter(queueSettings) for err != nil && retry(err) { diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 790659806f1..3365402ded8 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -59,7 +59,7 @@ type writerLoopResponse struct { type writerLoop struct { // The settings for the queue that created this loop. - settings *Settings + settings Settings // The logger for the writer loop, assigned when the queue creates it. logger *logp.Logger From 333cb766a62be647f04acb12d57da0bf4b05e065 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 12:27:52 -0400 Subject: [PATCH 77/91] ...save everything before commit --- libbeat/publisher/queue/diskqueue/queue.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index e996d0683f5..d7f326e6192 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -184,7 +184,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { // any helper loop). readerLoop := &readerLoop{ - settings: &settings, + settings: settings, requestChan: make(chan readerLoopRequest, 1), responseChan: make(chan readerLoopResponse), @@ -198,7 +198,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { writerLoop := &writerLoop{ logger: logger, - settings: &settings, + settings: settings, requestChan: make(chan writerLoopRequest, 1), responseChan: make(chan writerLoopResponse), @@ -209,7 +209,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { }() deleterLoop := &deleterLoop{ - settings: &settings, + settings: settings, requestChan: make(chan deleterLoopRequest, 1), responseChan: make(chan deleterLoopResponse), From 0dfffe10ecc65c1a30130a2c7cabe320a1bd5890 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 12:39:37 -0400 Subject: [PATCH 78/91] clarify the errors that "can't" fail --- .../publisher/queue/diskqueue/serialize.go | 21 +++++++------------ 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/serialize.go b/libbeat/publisher/queue/diskqueue/serialize.go index ea8f12564e8..9db8e7b1bd9 100644 --- a/libbeat/publisher/queue/diskqueue/serialize.go +++ b/libbeat/publisher/queue/diskqueue/serialize.go @@ -66,15 +66,16 @@ func (e *eventEncoder) reset() { e.folder = nil visitor := json.NewVisitor(&e.buf) - folder, err := gotype.NewIterator(visitor, + // This can't return an error: NewIterator is deterministic based on its + // input, and doesn't return an error when called with valid options. In + // this case the options are hard-coded to fixed values, so they are + // guaranteed to be valid and we can safely proceed. + folder, _ := gotype.NewIterator(visitor, gotype.Folders( codec.MakeTimestampEncoder(), codec.MakeBCTimestampEncoder(), ), ) - if err != nil { - panic(err) - } e.folder = folder } @@ -82,11 +83,6 @@ func (e *eventEncoder) reset() { func (e *eventEncoder) encode(event *publisher.Event) ([]byte, error) { e.buf.Reset() - /*var flags uint8 - if (event.Flags & publisher.GuaranteedSend) == publisher.GuaranteedSend { - flags = flagGuaranteed - }*/ - err := e.folder.Fold(entry{ Timestamp: event.Content.Timestamp.UTC().UnixNano(), Flags: uint8(event.Flags), @@ -113,10 +109,9 @@ func newEventDecoder() *eventDecoder { } func (d *eventDecoder) reset() { - unfolder, err := gotype.NewUnfolder(nil) - if err != nil { - panic(err) // can not happen - } + // When called on nil, NewUnfolder deterministically returns a nil error, + // so it's safe to ignore the error result. + unfolder, _ := gotype.NewUnfolder(nil) d.unfolder = unfolder d.parser = json.NewParser(unfolder) From 43c4aaaddc3563e53b704b8b93d42c09f894ae1d Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 12:56:04 -0400 Subject: [PATCH 79/91] be more consistent about unsignedness --- libbeat/publisher/queue/diskqueue/acks.go | 4 ++-- libbeat/publisher/queue/diskqueue/core_loop.go | 2 +- libbeat/publisher/queue/diskqueue/frames.go | 2 +- .../publisher/queue/diskqueue/reader_loop.go | 18 +++++++++--------- libbeat/publisher/queue/diskqueue/segments.go | 2 +- 5 files changed, 14 insertions(+), 14 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/acks.go b/libbeat/publisher/queue/diskqueue/acks.go index 53f5713a7ce..959950fc16a 100644 --- a/libbeat/publisher/queue/diskqueue/acks.go +++ b/libbeat/publisher/queue/diskqueue/acks.go @@ -46,7 +46,7 @@ type diskQueueACKs struct { // remaining frame, which is written to disk as ACKs are received. (We do // this to avoid duplicating events if the beat terminates without a clean // shutdown.) - frameSize map[frameID]int64 + frameSize map[frameID]uint64 // segmentBoundaries maps the first frameID of each segment to its // corresponding segment ID. @@ -76,7 +76,7 @@ func makeDiskQueueACKs( logger: logger, nextFrameID: 0, nextPosition: position, - frameSize: make(map[frameID]int64), + frameSize: make(map[frameID]uint64), segmentBoundaries: make(map[frameID]segmentID), segmentACKChan: make(chan segmentID), positionFile: positionFile, diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 2404425ab3b..2dc18447109 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -163,7 +163,7 @@ func (dq *diskQueue) handleReaderLoopResponse(response readerLoopResponse) { // so we don't check the endOffset. segment = dq.segments.writing[0] } - segment.framesRead = int64(dq.segments.nextReadFrameID - segment.firstFrameID) + segment.framesRead = uint64(dq.segments.nextReadFrameID - segment.firstFrameID) // If there was an error, report it. if response.err != nil { diff --git a/libbeat/publisher/queue/diskqueue/frames.go b/libbeat/publisher/queue/diskqueue/frames.go index 97e2bdc179b..4a661bee9d8 100644 --- a/libbeat/publisher/queue/diskqueue/frames.go +++ b/libbeat/publisher/queue/diskqueue/frames.go @@ -64,7 +64,7 @@ type readFrame struct { // How much space this frame occupied on disk (before deserialization), // including the frame header / footer. - bytesOnDisk int64 + bytesOnDisk uint64 } // Each data frame has a 32-bit length in the header, and a 32-bit checksum diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index a0b492fe1e8..e18d4ebe654 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -32,10 +32,10 @@ type readerLoopRequest struct { type readerLoopResponse struct { // The number of frames successfully read from the requested segment file. - frameCount int64 + frameCount uint64 // The number of bytes successfully read from the requested segment file. - byteCount int64 + byteCount uint64 // If there was an error in the segment file (i.e. inconsistent data), the // err field is set. @@ -79,8 +79,8 @@ func (rl *readerLoop) run() { } func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopResponse { - frameCount := int64(0) - byteCount := int64(0) + frameCount := uint64(0) + byteCount := uint64(0) nextFrameID := request.startFrameID // Open the file and seek to the starting position. @@ -94,7 +94,7 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon return readerLoopResponse{err: err} } - targetLength := int64(request.endOffset - request.startOffset) + targetLength := uint64(request.endOffset - request.startOffset) for { remainingLength := targetLength - byteCount @@ -156,7 +156,7 @@ func (rl *readerLoop) processRequest(request readerLoopRequest) readerLoopRespon // it does not exceed the given length bound. The returned frame leaves the // segment and frame IDs unset. func (rl *readerLoop) nextFrame( - handle *os.File, maxLength int64, + handle *os.File, maxLength uint64, ) (*readFrame, error) { // Ensure we are allowed to read the frame header. if maxLength < frameHeaderSize { @@ -174,9 +174,9 @@ func (rl *readerLoop) nextFrame( // If the frame extends past the area we were told to read, return an error. // This should never happen unless the segment file is corrupted. - if maxLength < int64(frameLength) { + if maxLength < uint64(frameLength) { return nil, fmt.Errorf( - "Can't read next frame: frame size is %v but remaining data is only %v", + "Can't read next frame: frame size is %d but remaining data is only %d", frameLength, maxLength) } if frameLength <= frameMetadataSize { @@ -229,7 +229,7 @@ func (rl *readerLoop) nextFrame( frame := &readFrame{ event: event, - bytesOnDisk: int64(frameLength), + bytesOnDisk: uint64(frameLength), } return frame, nil diff --git a/libbeat/publisher/queue/diskqueue/segments.go b/libbeat/publisher/queue/diskqueue/segments.go index a32ffbe5181..5ce0dc49962 100644 --- a/libbeat/publisher/queue/diskqueue/segments.go +++ b/libbeat/publisher/queue/diskqueue/segments.go @@ -109,7 +109,7 @@ type queueSegment struct { // read during a previous session. // // Used to count how many frames still need to be acknowledged by consumers. - framesRead int64 + framesRead uint64 } type segmentHeader struct { From 0b33c4b4968031404483db5941a7f78acf3f27d6 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 13:28:35 -0400 Subject: [PATCH 80/91] change the format of the deleter loop response --- .../publisher/queue/diskqueue/core_loop.go | 32 ++++++++++++------- .../publisher/queue/diskqueue/deleter_loop.go | 20 +++++------- 2 files changed, 28 insertions(+), 24 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/core_loop.go b/libbeat/publisher/queue/diskqueue/core_loop.go index 2dc18447109..56a50b5a422 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop.go +++ b/libbeat/publisher/queue/diskqueue/core_loop.go @@ -17,6 +17,8 @@ package diskqueue +import "fmt" + // This file contains the queue's "core loop" -- the central goroutine // that owns all queue state that is not encapsulated in one of the // self-contained helper loops. This is the only file that is allowed to @@ -175,20 +177,26 @@ func (dq *diskQueue) handleReaderLoopResponse(response readerLoopResponse) { func (dq *diskQueue) handleDeleterLoopResponse(response deleterLoopResponse) { dq.deleting = false - if len(response.deleted) > 0 { - // One or more segments were deleted, recompute the outstanding list. - newAckedSegments := []*queueSegment{} - for _, segment := range dq.segments.acked { - if !response.deleted[segment] { - // This segment wasn't deleted, so it goes in the new list. - newAckedSegments = append(newAckedSegments, segment) - } + newAckedSegments := []*queueSegment{} + errors := []error{} + for i, err := range response.results { + if err != nil { + // This segment had an error, so it stays in the acked list. + newAckedSegments = append(newAckedSegments, dq.segments.acked[i]) + errors = append(errors, + fmt.Errorf("Couldn't delete segment %d: %w", + dq.segments.acked[i].id, err)) } - dq.segments.acked = newAckedSegments } - if len(response.errors) > 0 { - dq.logger.Errorw("Couldn't delete old segment files", - "errors", response.errors) + if len(dq.segments.acked) > len(response.results) { + // Preserve any new acked segments that were added during the deletion + // request. + tail := dq.segments.acked[len(response.results):] + newAckedSegments = append(newAckedSegments, tail...) + } + dq.segments.acked = newAckedSegments + if len(errors) > 0 { + dq.logger.Errorw("Deleting segment files", "errors", errors) } } diff --git a/libbeat/publisher/queue/diskqueue/deleter_loop.go b/libbeat/publisher/queue/diskqueue/deleter_loop.go index 3dc379f3186..ef00276ad3b 100644 --- a/libbeat/publisher/queue/diskqueue/deleter_loop.go +++ b/libbeat/publisher/queue/diskqueue/deleter_loop.go @@ -44,11 +44,7 @@ type deleterLoopRequest struct { } type deleterLoopResponse struct { - // The queue segments that were successfully deleted. - deleted map[*queueSegment]bool - - // Errors - errors []error + results []error } func (dl *deleterLoop) run() { @@ -58,20 +54,21 @@ func (dl *deleterLoop) run() { // The channel has been closed, time to shut down. return } - deleted := make(map[*queueSegment]bool, len(request.segments)) - errorList := []error{} + results := []error{} + deletedCount := 0 for _, segment := range request.segments { path := dl.settings.segmentPath(segment.id) err := os.Remove(path) // We ignore errors caused by the file not existing: this shouldn't // happen, but it is still safe to report it as successfully removed. if err == nil || errors.Is(err, os.ErrNotExist) { - deleted[segment] = true + deletedCount++ + results = append(results, nil) } else { - errorList = append(errorList, err) + results = append(results, err) } } - if len(request.segments) > 0 && len(deleted) == 0 { + if len(request.segments) > 0 && deletedCount == 0 { // If we were asked to delete segments but could not delete // _any_ of them, we haven't made progress. Returning an error // will log the issue and retry, but in this situation we @@ -87,8 +84,7 @@ func (dl *deleterLoop) run() { } } dl.responseChan <- deleterLoopResponse{ - deleted: deleted, - errors: errorList, + results: results, } } } From 7b2f4bab9930a8437f8cc72f89f51d8bbd029278 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 14:15:51 -0400 Subject: [PATCH 81/91] make helper functions to initialize the helper loop structures --- libbeat/publisher/queue/diskqueue/acks.go | 2 +- .../publisher/queue/diskqueue/deleter_loop.go | 9 +++ libbeat/publisher/queue/diskqueue/queue.go | 78 ++++++------------- .../publisher/queue/diskqueue/reader_loop.go | 11 +++ .../publisher/queue/diskqueue/writer_loop.go | 10 +++ 5 files changed, 54 insertions(+), 56 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/acks.go b/libbeat/publisher/queue/diskqueue/acks.go index 959950fc16a..ed9d7589db2 100644 --- a/libbeat/publisher/queue/diskqueue/acks.go +++ b/libbeat/publisher/queue/diskqueue/acks.go @@ -69,7 +69,7 @@ type diskQueueACKs struct { done chan struct{} } -func makeDiskQueueACKs( +func newDiskQueueACKs( logger *logp.Logger, position queuePosition, positionFile *os.File, ) *diskQueueACKs { return &diskQueueACKs{ diff --git a/libbeat/publisher/queue/diskqueue/deleter_loop.go b/libbeat/publisher/queue/diskqueue/deleter_loop.go index ef00276ad3b..4e685285948 100644 --- a/libbeat/publisher/queue/diskqueue/deleter_loop.go +++ b/libbeat/publisher/queue/diskqueue/deleter_loop.go @@ -47,6 +47,15 @@ type deleterLoopResponse struct { results []error } +func newDeleterLoop(settings Settings) *deleterLoop { + return &deleterLoop{ + settings: settings, + + requestChan: make(chan deleterLoopRequest, 1), + responseChan: make(chan deleterLoopResponse), + } +} + func (dl *deleterLoop) run() { for { request, ok := <-dl.requestChan diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index d7f326e6192..11933a593b7 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -172,53 +172,6 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { nextReadPosition = queuePosition{segmentID: initialSegments[0].id} } - // We wait for four goroutines: core loop, reader loop, writer loop, - // deleter loop. - var waitGroup sync.WaitGroup - waitGroup.Add(4) - - // The helper loops all have an input channel with buffer size 1, to ensure - // that the core loop can never block when sending a request (the core - // loop never sends a request until receiving the response from the - // previous one, so there is never more than one outstanding request for - // any helper loop). - - readerLoop := &readerLoop{ - settings: settings, - - requestChan: make(chan readerLoopRequest, 1), - responseChan: make(chan readerLoopResponse), - output: make(chan *readFrame, settings.ReadAheadLimit), - decoder: newEventDecoder(), - } - go func() { - readerLoop.run() - waitGroup.Done() - }() - - writerLoop := &writerLoop{ - logger: logger, - settings: settings, - - requestChan: make(chan writerLoopRequest, 1), - responseChan: make(chan writerLoopResponse), - } - go func() { - writerLoop.run() - waitGroup.Done() - }() - - deleterLoop := &deleterLoop{ - settings: settings, - - requestChan: make(chan deleterLoopRequest, 1), - responseChan: make(chan deleterLoopResponse), - } - go func() { - deleterLoop.run() - waitGroup.Done() - }() - queue := &diskQueue{ logger: logger, settings: settings, @@ -229,22 +182,37 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { nextReadOffset: nextReadPosition.offset, }, - acks: makeDiskQueueACKs(logger, nextReadPosition, positionFile), + acks: newDiskQueueACKs(logger, nextReadPosition, positionFile), - readerLoop: readerLoop, - writerLoop: writerLoop, - deleterLoop: deleterLoop, + readerLoop: newReaderLoop(settings), + writerLoop: newWriterLoop(logger, settings), + deleterLoop: newDeleterLoop(settings), producerWriteRequestChan: make(chan producerWriteRequest), - waitGroup: &waitGroup, - done: make(chan struct{}), + done: make(chan struct{}), } - // Start the queue's main loop. + // We wait for four goroutines on shutdown: core loop, reader loop, + // writer loop, deleter loop. + queue.waitGroup.Add(4) + + // Start the goroutines and return the queue! + go func() { + queue.readerLoop.run() + queue.waitGroup.Done() + }() + go func() { + queue.writerLoop.run() + queue.waitGroup.Done() + }() + go func() { + queue.deleterLoop.run() + queue.waitGroup.Done() + }() go func() { queue.run() - waitGroup.Done() + queue.waitGroup.Done() }() return queue, nil diff --git a/libbeat/publisher/queue/diskqueue/reader_loop.go b/libbeat/publisher/queue/diskqueue/reader_loop.go index e18d4ebe654..dc2bb95777f 100644 --- a/libbeat/publisher/queue/diskqueue/reader_loop.go +++ b/libbeat/publisher/queue/diskqueue/reader_loop.go @@ -65,6 +65,17 @@ type readerLoop struct { decoder *eventDecoder } +func newReaderLoop(settings Settings) *readerLoop { + return &readerLoop{ + settings: settings, + + requestChan: make(chan readerLoopRequest, 1), + responseChan: make(chan readerLoopResponse), + output: make(chan *readFrame, settings.ReadAheadLimit), + decoder: newEventDecoder(), + } +} + func (rl *readerLoop) run() { for { request, ok := <-rl.requestChan diff --git a/libbeat/publisher/queue/diskqueue/writer_loop.go b/libbeat/publisher/queue/diskqueue/writer_loop.go index 3365402ded8..b42e4573cab 100644 --- a/libbeat/publisher/queue/diskqueue/writer_loop.go +++ b/libbeat/publisher/queue/diskqueue/writer_loop.go @@ -84,6 +84,16 @@ type writerLoop struct { outputFile *os.File } +func newWriterLoop(logger *logp.Logger, settings Settings) *writerLoop { + return &writerLoop{ + logger: logger, + settings: settings, + + requestChan: make(chan writerLoopRequest, 1), + responseChan: make(chan writerLoopResponse), + } +} + func (wl *writerLoop) run() { for { block, ok := <-wl.requestChan From a626cec80a622212c3cd7e8a103d621476ff21bf Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 14:29:42 -0400 Subject: [PATCH 82/91] clarify comments --- libbeat/publisher/queue/diskqueue/queue.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 11933a593b7..fc01575b7df 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -156,8 +156,8 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { nextSegmentID = lastID + 1 } - // If any of the initial segments are older than the read position from - // the state file, move them directly to the acked list where they can be + // If any of the initial segments are older than the current queue + // position, move them directly to the acked list where they can be // deleted. ackedSegments := []*queueSegment{} readSegmentID := nextReadPosition.segmentID @@ -166,7 +166,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { initialSegments = initialSegments[1:] } - // If the next read position is older than all existing segments, advance + // If the queue position is older than all existing segments, advance // it to the beginning of the first one. if len(initialSegments) > 0 && readSegmentID < initialSegments[0].id { nextReadPosition = queuePosition{segmentID: initialSegments[0].id} From 53da09a1cf5d49e3c859d75baeab937e64be08c9 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 14:34:30 -0400 Subject: [PATCH 83/91] minor tweaks --- libbeat/publisher/queue/diskqueue/queue.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index fc01575b7df..cef84836bab 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -51,21 +51,21 @@ type diskQueue struct { // reader loop, writer loop, deleter loop, and core loop (diskQueue.run()). waitGroup *sync.WaitGroup - // The API channel used by diskQueueProducer to write events. - producerWriteRequestChan chan producerWriteRequest - - // writing is true if a writeRequest is currently being processed by the - // writer loop, false otherwise. + // writing is true if the writer loop is processing a request, false + // otherwise. writing bool - // reading is true if the reader loop is processing a readBlock, false + // reading is true if the reader loop is processing a request, false // otherwise. reading bool - // deleting is true if the segment-deletion loop is processing a deletion - // request, false otherwise. + // deleting is true if the deleter loop is processing a request, false + // otherwise. deleting bool + // The API channel used by diskQueueProducer to write events. + producerWriteRequestChan chan producerWriteRequest + // pendingFrames is a list of all incoming data frames that have been // accepted by the queue and are waiting to be sent to the writer loop. // Segment ids in this list always appear in sorted order, even between From 565a99f58441aca6c600274bca9dd3859088dff3 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 14:36:26 -0400 Subject: [PATCH 84/91] remove unused event field in writeFrame --- libbeat/publisher/queue/diskqueue/producer.go | 1 - 1 file changed, 1 deletion(-) diff --git a/libbeat/publisher/queue/diskqueue/producer.go b/libbeat/publisher/queue/diskqueue/producer.go index fb0e64689b0..f4ff4ef2706 100644 --- a/libbeat/publisher/queue/diskqueue/producer.go +++ b/libbeat/publisher/queue/diskqueue/producer.go @@ -72,7 +72,6 @@ func (producer *diskQueueProducer) publish( } request := producerWriteRequest{ frame: &writeFrame{ - event: event, serialized: serialized, producer: producer, }, From 008fbe7a959c40906e4d9349b3e5070c07375f70 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 14:36:58 -0400 Subject: [PATCH 85/91] ...save all files before commit... --- libbeat/publisher/queue/diskqueue/frames.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/frames.go b/libbeat/publisher/queue/diskqueue/frames.go index 4a661bee9d8..02571a65ce9 100644 --- a/libbeat/publisher/queue/diskqueue/frames.go +++ b/libbeat/publisher/queue/diskqueue/frames.go @@ -29,12 +29,6 @@ type frameID uint64 // A data frame created through the producer API and waiting to be // written to disk. type writeFrame struct { - // The original event provided by the client to diskQueueProducer. - // We keep this as well as the serialized form until we are done - // writing, because we may need to send this value back to the producer - // callback if it is cancelled. - event publisher.Event - // The event, serialized for writing to disk and wrapped in a frame // header / footer. serialized []byte From b5ec92600dce8277fa7b579adbe0c9f05ccffcc7 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Tue, 22 Sep 2020 14:41:36 -0400 Subject: [PATCH 86/91] don't initialize waitgroup to a nil pointer --- libbeat/publisher/queue/diskqueue/queue.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index cef84836bab..ed999e248f9 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -49,7 +49,7 @@ type diskQueue struct { // Wait group for shutdown of the goroutines associated with this queue: // reader loop, writer loop, deleter loop, and core loop (diskQueue.run()). - waitGroup *sync.WaitGroup + waitGroup sync.WaitGroup // writing is true if the writer loop is processing a request, false // otherwise. From 11e55a847a3430973ea203ee57a80ed9943fd719 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Wed, 23 Sep 2020 10:14:27 -0400 Subject: [PATCH 87/91] error message cleanups --- libbeat/publisher/queue/diskqueue/queue.go | 8 ++++---- libbeat/publisher/queue/diskqueue/util.go | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index ed999e248f9..c0a80e1edd2 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -98,7 +98,7 @@ func queueFactory( ) (queue.Queue, error) { settings, err := SettingsForUserConfig(cfg) if err != nil { - return nil, fmt.Errorf("Disk queue couldn't load user config: %w", err) + return nil, fmt.Errorf("disk queue couldn't load user config: %w", err) } settings.WriteToDiskListener = ackListener return NewQueue(logger, settings) @@ -114,7 +114,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { if settings.MaxBufferSize > 0 && settings.MaxBufferSize < settings.MaxSegmentSize*2 { return nil, fmt.Errorf( - "Disk queue buffer size (%v) must be at least "+ + "disk queue buffer size (%v) must be at least "+ "twice the segment size (%v)", settings.MaxBufferSize, settings.MaxSegmentSize) } @@ -122,7 +122,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { // Create the given directory path if it doesn't exist. err := os.MkdirAll(settings.directoryPath(), os.ModePerm) if err != nil { - return nil, fmt.Errorf("Couldn't create disk queue directory: %w", err) + return nil, fmt.Errorf("couldn't create disk queue directory: %w", err) } // Load the previous queue position, if any. @@ -141,7 +141,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { // and could also prevent us from creating new ones, so we treat this as a // fatal error on startup rather than quietly providing degraded // performance. - return nil, fmt.Errorf("Couldn't write to state file: %v", err) + return nil, fmt.Errorf("couldn't write to state file: %v", err) } // Index any existing data segments to be placed in segments.reading. diff --git a/libbeat/publisher/queue/diskqueue/util.go b/libbeat/publisher/queue/diskqueue/util.go index a988f88348b..60c529a9992 100644 --- a/libbeat/publisher/queue/diskqueue/util.go +++ b/libbeat/publisher/queue/diskqueue/util.go @@ -65,8 +65,8 @@ func writeErrorIsRetriable(err error) bool { // This helper is specifically for working with the writer loop, which needs // to be able to retry forever at configurable intervals, but also cancel // immediately if the queue is closed. -// This writer is unbuffered. In particular, it is safe to modify -// "wrapped" in-place as long as it isn't captured by the callback. +// This writer is unbuffered. In particular, it is safe to modify the +// "wrapped" field in-place as long as it isn't captured by the callback. type callbackRetryWriter struct { wrapped io.Writer retry func(error) bool From ff04bb5fcf253e00103fb93736eb8da9931fd180 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Wed, 23 Sep 2020 10:19:01 -0400 Subject: [PATCH 88/91] simplify position write-to-disk --- .../publisher/queue/diskqueue/state_file.go | 22 ++++++------------- 1 file changed, 7 insertions(+), 15 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/state_file.go b/libbeat/publisher/queue/diskqueue/state_file.go index 1869bf26251..2ff14e3e5e2 100644 --- a/libbeat/publisher/queue/diskqueue/state_file.go +++ b/libbeat/publisher/queue/diskqueue/state_file.go @@ -81,21 +81,13 @@ func writeQueuePositionToHandle( return err } - var version uint32 = 0 - err = binary.Write(file, binary.LittleEndian, &version) - if err != nil { - return err - } - - err = binary.Write(file, binary.LittleEndian, &position.segmentID) - if err != nil { - return err + // Want to write: version (0), segment id, segment offset. + elems := []interface{}{uint32(0), position.segmentID, position.offset} + for _, elem := range elems { + err = binary.Write(file, binary.LittleEndian, &elem) + if err != nil { + return err + } } - - err = binary.Write(file, binary.LittleEndian, &position.offset) - if err != nil { - return err - } - return nil } From 2baf1d4125f0e10c600418226a58d510c26c182a Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 24 Sep 2020 13:46:05 -0400 Subject: [PATCH 89/91] add some unit tests --- libbeat/publisher/queue/diskqueue/acks.go | 4 +- .../queue/diskqueue/core_loop_test.go | 94 +++++++++++++++++++ libbeat/publisher/queue/diskqueue/queue.go | 6 +- 3 files changed, 99 insertions(+), 5 deletions(-) create mode 100644 libbeat/publisher/queue/diskqueue/core_loop_test.go diff --git a/libbeat/publisher/queue/diskqueue/acks.go b/libbeat/publisher/queue/diskqueue/acks.go index ed9d7589db2..24b980d1178 100644 --- a/libbeat/publisher/queue/diskqueue/acks.go +++ b/libbeat/publisher/queue/diskqueue/acks.go @@ -71,8 +71,8 @@ type diskQueueACKs struct { func newDiskQueueACKs( logger *logp.Logger, position queuePosition, positionFile *os.File, -) *diskQueueACKs { - return &diskQueueACKs{ +) diskQueueACKs { + return diskQueueACKs{ logger: logger, nextFrameID: 0, nextPosition: position, diff --git a/libbeat/publisher/queue/diskqueue/core_loop_test.go b/libbeat/publisher/queue/diskqueue/core_loop_test.go new file mode 100644 index 00000000000..245c091ba21 --- /dev/null +++ b/libbeat/publisher/queue/diskqueue/core_loop_test.go @@ -0,0 +1,94 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package diskqueue + +import "testing" + +func TestProducerWriteRequest(t *testing.T) { + dq := &diskQueue{settings: DefaultSettings()} + frame := &writeFrame{ + serialized: make([]byte, 100), + } + request := producerWriteRequest{ + frame: frame, + shouldBlock: true, + responseChan: make(chan bool, 1), + } + dq.handleProducerWriteRequest(request) + + // The request inserts 100 bytes into an empty queue, so it should succeed. + // We expect: + // - the response channel should contain the value true + // - the frame should be added to pendingFrames and assigned to + // segment 0. + success, ok := <-request.responseChan + if !ok { + t.Error("Expected a response from the producer write request.") + } + if !success { + t.Error("Expected write request to succeed") + } + + if len(dq.pendingFrames) != 1 { + t.Error("Expected 1 pending frame after a write request.") + } + if dq.pendingFrames[0].frame != frame { + t.Error("Expected pendingFrames to contain the new frame.") + } + if dq.pendingFrames[0].segment.id != 0 { + t.Error("Expected new frame to be assigned to segment 0.") + } +} + +func TestHandleWriterLoopResponse(t *testing.T) { + // Initialize the queue with two writing segments only. + dq := &diskQueue{ + settings: DefaultSettings(), + segments: diskQueueSegments{ + writing: []*queueSegment{ + &queueSegment{id: 1}, + &queueSegment{id: 2}, + }, + }, + } + // This response says that the writer loop wrote 200 bytes to the first + // segment and 100 bytes to the second. + dq.handleWriterLoopResponse(writerLoopResponse{ + bytesWritten: []int64{200, 100}, + }) + + // After the response is handled, we expect: + // - Each segment's endOffset should be incremented by the bytes written + // - Segment 1 should be moved to the reading list (because all but the + // last segment in a writer loop response has been closed) + // - Segment 2 should remain in the writing list + if len(dq.segments.reading) != 1 || dq.segments.reading[0].id != 1 { + t.Error("Expected segment 1 to move to the reading list") + } + if len(dq.segments.writing) != 1 || dq.segments.writing[0].id != 2 { + t.Error("Expected segment 2 to remain in the writing list") + } + if dq.segments.reading[0].endOffset != 200 { + t.Errorf("Expected segment 1 endOffset 200, got %d", + dq.segments.reading[0].endOffset) + } + if dq.segments.writing[0].endOffset != 100 { + t.Errorf("Expected segment 2 endOffset 100, got %d", + dq.segments.writing[0].endOffset) + } +} diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index c0a80e1edd2..9ba875cb7df 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -36,11 +36,11 @@ type diskQueue struct { settings Settings // Metadata related to the segment files. - segments *diskQueueSegments + segments diskQueueSegments // Metadata related to consumer acks / positions of the oldest remaining // frame. - acks *diskQueueACKs + acks diskQueueACKs // The queue's helper loops, each of which is run in its own goroutine. readerLoop *readerLoop @@ -176,7 +176,7 @@ func NewQueue(logger *logp.Logger, settings Settings) (queue.Queue, error) { logger: logger, settings: settings, - segments: &diskQueueSegments{ + segments: diskQueueSegments{ reading: initialSegments, nextID: nextSegmentID, nextReadOffset: nextReadPosition.offset, From 30cf56c0f0ae385bd1a817522ddedea163136278 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 24 Sep 2020 14:00:45 -0400 Subject: [PATCH 90/91] more verbose config errors --- libbeat/publisher/queue/diskqueue/config.go | 12 +++++------- libbeat/publisher/queue/diskqueue/core_loop_test.go | 4 ++-- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/config.go b/libbeat/publisher/queue/diskqueue/config.go index 14ecb28cb4f..f39f608361d 100644 --- a/libbeat/publisher/queue/diskqueue/config.go +++ b/libbeat/publisher/queue/diskqueue/config.go @@ -88,12 +88,12 @@ func (c *userConfig) Validate() error { // need is for a low-volume stream on a tiny device to persist between // restarts, it will work fine. if c.MaxSize != 0 && c.MaxSize < 10*1000*1000 { - return errors.New( - "Disk queue max_size cannot be less than 10MB") + return fmt.Errorf( + "Disk queue max_size (%d) cannot be less than 10MB", c.MaxSize) } if c.SegmentSize != nil && *c.SegmentSize < 1000*1000 { - return errors.New( - "Disk queue segment_size cannot be less than 1MB") + return fmt.Errorf( + "Disk queue segment_size (%d) cannot be less than 1MB", *c.SegmentSize) } return nil @@ -116,7 +116,7 @@ func DefaultSettings() Settings { func SettingsForUserConfig(config *common.Config) (Settings, error) { userConfig := userConfig{} if err := config.Unpack(&userConfig); err != nil { - return Settings{}, err + return Settings{}, fmt.Errorf("parsing user config: %w", err) } settings := DefaultSettings() settings.Path = userConfig.Path @@ -129,7 +129,6 @@ func SettingsForUserConfig(config *common.Config) (Settings, error) { // divided by 10. settings.MaxSegmentSize = uint64(userConfig.MaxSize) / 10 } - return settings, nil } @@ -141,7 +140,6 @@ func (settings Settings) directoryPath() string { if settings.Path == "" { return paths.Resolve(paths.Data, "diskqueue") } - return settings.Path } diff --git a/libbeat/publisher/queue/diskqueue/core_loop_test.go b/libbeat/publisher/queue/diskqueue/core_loop_test.go index 245c091ba21..b5f0d301d15 100644 --- a/libbeat/publisher/queue/diskqueue/core_loop_test.go +++ b/libbeat/publisher/queue/diskqueue/core_loop_test.go @@ -61,8 +61,8 @@ func TestHandleWriterLoopResponse(t *testing.T) { settings: DefaultSettings(), segments: diskQueueSegments{ writing: []*queueSegment{ - &queueSegment{id: 1}, - &queueSegment{id: 2}, + {id: 1}, + {id: 2}, }, }, } From 5b41351d4e6a63baa5531ce3bfd5bd4e139b9052 Mon Sep 17 00:00:00 2001 From: Fae Charlton Date: Thu, 24 Sep 2020 14:37:55 -0400 Subject: [PATCH 91/91] change queue acks back to a pointer since it can't be used w/o initialization --- libbeat/publisher/queue/diskqueue/acks.go | 4 ++-- libbeat/publisher/queue/diskqueue/queue.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/libbeat/publisher/queue/diskqueue/acks.go b/libbeat/publisher/queue/diskqueue/acks.go index 24b980d1178..ed9d7589db2 100644 --- a/libbeat/publisher/queue/diskqueue/acks.go +++ b/libbeat/publisher/queue/diskqueue/acks.go @@ -71,8 +71,8 @@ type diskQueueACKs struct { func newDiskQueueACKs( logger *logp.Logger, position queuePosition, positionFile *os.File, -) diskQueueACKs { - return diskQueueACKs{ +) *diskQueueACKs { + return &diskQueueACKs{ logger: logger, nextFrameID: 0, nextPosition: position, diff --git a/libbeat/publisher/queue/diskqueue/queue.go b/libbeat/publisher/queue/diskqueue/queue.go index 9ba875cb7df..5f756996e5f 100644 --- a/libbeat/publisher/queue/diskqueue/queue.go +++ b/libbeat/publisher/queue/diskqueue/queue.go @@ -40,7 +40,7 @@ type diskQueue struct { // Metadata related to consumer acks / positions of the oldest remaining // frame. - acks diskQueueACKs + acks *diskQueueACKs // The queue's helper loops, each of which is run in its own goroutine. readerLoop *readerLoop