diff --git a/.travis.yml b/.travis.yml index 575dccb0074..5748ca2ce80 100644 --- a/.travis.yml +++ b/.travis.yml @@ -68,7 +68,7 @@ jobs: go: $GO_VERSION stage: test - os: linux - env: TARGETS="-C libbeat stress-tests" + env: STRESS_TEST_OPTIONS="-timeout=20m -race -v -parallel 1" TARGETS="-C libbeat stress-tests" go: $GO_VERSION stage: test diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc index 5257a31b9e9..690cc985643 100644 --- a/CHANGELOG.asciidoc +++ b/CHANGELOG.asciidoc @@ -207,6 +207,7 @@ https://github.com/elastic/beats/compare/v6.0.0-beta2...master[Check the HEAD di - Add appender support to autodiscover {pull}6469[6469] - Add add_host_metadata processor {pull}5968[5968] - Retry configuration to load dashboards if Kibana is not reachable when the beat starts. {pull}6560[6560] +- Add support for spooling to disk to the beats event publishing pipeline. {pull}6581[6581] *Auditbeat* diff --git a/NOTICE.txt b/NOTICE.txt index 93037436bfa..3118d690145 100644 --- a/NOTICE.txt +++ b/NOTICE.txt @@ -372,6 +372,16 @@ License type (autodetected): Apache-2.0 Apache License 2.0 +-------------------------------------------------------------------- +Dependency: github.com/elastic/go-txfile +Version: v0.0.1 +Revision: 7e7e33cc236f30fff545f3ee2c35ada5b70b6b13 +License type (autodetected): Apache-2.0 +./vendor/github.com/elastic/go-txfile/LICENSE: +-------------------------------------------------------------------- +Apache License 2.0 + + -------------------------------------------------------------------- Dependency: github.com/elastic/go-ucfg Version: v0.5.1 @@ -2052,6 +2062,41 @@ DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. +-------------------------------------------------------------------- +Dependency: github.com/theckman/go-flock +Version: v0.4.0 +Revision: b139a2487364247d91814e4a7c7b8fdc69e342b2 +License type (autodetected): BSD-3-Clause +./vendor/github.com/theckman/go-flock/LICENSE: +-------------------------------------------------------------------- +Copyright (c) 2015, Tim Heckman +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +* Neither the name of linode-netint nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + -------------------------------------------------------------------- Dependency: github.com/tsg/gopacket Revision: f289b3ea3e41a01b2822be9caf5f40c01fdda05c @@ -2087,6 +2132,24 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +-------------------------------------------------------------------- +Dependency: github.com/urso/go-bin +Revision: 781c575c9f0eb3cb9dca94521bd7ad7d5aec7fd4 +License type (autodetected): Apache-2.0 +./vendor/github.com/urso/go-bin/LICENSE: +-------------------------------------------------------------------- +Apache License 2.0 + + +-------------------------------------------------------------------- +Dependency: github.com/urso/qcgen +Revision: 0b059e7db4f40a062ca3d975b7500c6a0a968d87 +License type (autodetected): Apache-2.0 +./vendor/github.com/urso/qcgen/LICENSE: +-------------------------------------------------------------------- +Apache License 2.0 + + -------------------------------------------------------------------- Dependency: github.com/vmware/govmomi Revision: 2cad15190b417804d82edb4981e7b3e62907c4ee diff --git a/auditbeat/auditbeat.reference.yml b/auditbeat/auditbeat.reference.yml index beb10bff54a..f86238c4e6d 100644 --- a/auditbeat/auditbeat.reference.yml +++ b/auditbeat/auditbeat.reference.yml @@ -148,6 +148,66 @@ auditbeat.modules: # if the number of events stored in the queue is < min_flush_events. #flush.timeout: 1s + # The spool queue will store events in a local spool file, before + # forwarding the events to the outputs. + # + # Beta: spooling to disk is currently a beta feature. Use with care. + # + # The spool file is a circular buffer, which blocks once the file/buffer is full. + # Events are put into a write buffer and flushed once the write buffer + # is full or the flush_timeout is triggered. + # Once ACKed by the output, events are removed immediately from the queue, + # making space for new events to be persisted. + #spool: + # The file namespace configures the file path and the file creation settings. + # Once the file exists, the `size`, `page_size` and `prealloc` settings + # will have no more effect. + #file: + # Location of spool file. The default value is ${path.data}/spool.dat. + #path: "${path.data}/spool.dat" + + # Configure file permissions if file is created. The default value is 0600. + #permissions: 0600 + + # File size hint. The spool blocks, once this limit is reached. The default value is 100 MiB. + #size: 100MiB + + # The files page size. A file is split into multiple pages of the same size. The default value is 4KiB. + #page_size: 4KiB + + # If prealloc is set, the required space for the file is reserved using + # truncate. The default value is true. + #prealloc: true + + # Spool writer settings + # Events are serialized into a write buffer. The write buffer is flushed if: + # - The buffer limit has been reached. + # - The configured limit of buffered events is reached. + # - The flush timeout is triggered. + #write: + # Sets the write buffer size. + #buffer_size: 1MiB + + # Maximum duration after which events are flushed, if the write buffer + # is not full yet. The default value is 1s. + #flush.timeout: 1s + + # Number of maximum buffered events. The write buffer is flushed once the + # limit is reached. + #flush.events: 16384 + + # Configure the on-disk event encoding. The encoding can be changed + # between restarts. + # Valid encodings are: json, ubjson, and cbor. + #codec: cbor + #read: + # Reader flush timeout, waiting for more events to become available, so + # to fill a complete batch, as required by the outputs. + # If flush_timeout is 0, all available events are forwarded to the + # outputs immediately. + # The default value is 0s. + #flush.timeout: 0s + # Sets the maximum number of CPUs that can be executing simultaneously. The # default is the number of logical CPUs available in the system. #max_procs: diff --git a/filebeat/filebeat.reference.yml b/filebeat/filebeat.reference.yml index 664dcea51be..df6418d0262 100644 --- a/filebeat/filebeat.reference.yml +++ b/filebeat/filebeat.reference.yml @@ -629,6 +629,66 @@ filebeat.inputs: # if the number of events stored in the queue is < min_flush_events. #flush.timeout: 1s + # The spool queue will store events in a local spool file, before + # forwarding the events to the outputs. + # + # Beta: spooling to disk is currently a beta feature. Use with care. + # + # The spool file is a circular buffer, which blocks once the file/buffer is full. + # Events are put into a write buffer and flushed once the write buffer + # is full or the flush_timeout is triggered. + # Once ACKed by the output, events are removed immediately from the queue, + # making space for new events to be persisted. + #spool: + # The file namespace configures the file path and the file creation settings. + # Once the file exists, the `size`, `page_size` and `prealloc` settings + # will have no more effect. + #file: + # Location of spool file. The default value is ${path.data}/spool.dat. + #path: "${path.data}/spool.dat" + + # Configure file permissions if file is created. The default value is 0600. + #permissions: 0600 + + # File size hint. The spool blocks, once this limit is reached. The default value is 100 MiB. + #size: 100MiB + + # The files page size. A file is split into multiple pages of the same size. The default value is 4KiB. + #page_size: 4KiB + + # If prealloc is set, the required space for the file is reserved using + # truncate. The default value is true. + #prealloc: true + + # Spool writer settings + # Events are serialized into a write buffer. The write buffer is flushed if: + # - The buffer limit has been reached. + # - The configured limit of buffered events is reached. + # - The flush timeout is triggered. + #write: + # Sets the write buffer size. + #buffer_size: 1MiB + + # Maximum duration after which events are flushed, if the write buffer + # is not full yet. The default value is 1s. + #flush.timeout: 1s + + # Number of maximum buffered events. The write buffer is flushed once the + # limit is reached. + #flush.events: 16384 + + # Configure the on-disk event encoding. The encoding can be changed + # between restarts. + # Valid encodings are: json, ubjson, and cbor. + #codec: cbor + #read: + # Reader flush timeout, waiting for more events to become available, so + # to fill a complete batch, as required by the outputs. + # If flush_timeout is 0, all available events are forwarded to the + # outputs immediately. + # The default value is 0s. + #flush.timeout: 0s + # Sets the maximum number of CPUs that can be executing simultaneously. The # default is the number of logical CPUs available in the system. #max_procs: diff --git a/heartbeat/heartbeat.reference.yml b/heartbeat/heartbeat.reference.yml index 1fcedeae1e1..2a5a20d11b6 100644 --- a/heartbeat/heartbeat.reference.yml +++ b/heartbeat/heartbeat.reference.yml @@ -257,6 +257,66 @@ heartbeat.scheduler: # if the number of events stored in the queue is < min_flush_events. #flush.timeout: 1s + # The spool queue will store events in a local spool file, before + # forwarding the events to the outputs. + # + # Beta: spooling to disk is currently a beta feature. Use with care. + # + # The spool file is a circular buffer, which blocks once the file/buffer is full. + # Events are put into a write buffer and flushed once the write buffer + # is full or the flush_timeout is triggered. + # Once ACKed by the output, events are removed immediately from the queue, + # making space for new events to be persisted. + #spool: + # The file namespace configures the file path and the file creation settings. + # Once the file exists, the `size`, `page_size` and `prealloc` settings + # will have no more effect. + #file: + # Location of spool file. The default value is ${path.data}/spool.dat. + #path: "${path.data}/spool.dat" + + # Configure file permissions if file is created. The default value is 0600. + #permissions: 0600 + + # File size hint. The spool blocks, once this limit is reached. The default value is 100 MiB. + #size: 100MiB + + # The files page size. A file is split into multiple pages of the same size. The default value is 4KiB. + #page_size: 4KiB + + # If prealloc is set, the required space for the file is reserved using + # truncate. The default value is true. + #prealloc: true + + # Spool writer settings + # Events are serialized into a write buffer. The write buffer is flushed if: + # - The buffer limit has been reached. + # - The configured limit of buffered events is reached. + # - The flush timeout is triggered. + #write: + # Sets the write buffer size. + #buffer_size: 1MiB + + # Maximum duration after which events are flushed, if the write buffer + # is not full yet. The default value is 1s. + #flush.timeout: 1s + + # Number of maximum buffered events. The write buffer is flushed once the + # limit is reached. + #flush.events: 16384 + + # Configure the on-disk event encoding. The encoding can be changed + # between restarts. + # Valid encodings are: json, ubjson, and cbor. + #codec: cbor + #read: + # Reader flush timeout, waiting for more events to become available, so + # to fill a complete batch, as required by the outputs. + # If flush_timeout is 0, all available events are forwarded to the + # outputs immediately. + # The default value is 0s. + #flush.timeout: 0s + # Sets the maximum number of CPUs that can be executing simultaneously. The # default is the number of logical CPUs available in the system. #max_procs: diff --git a/libbeat/_meta/config.reference.yml b/libbeat/_meta/config.reference.yml index 7eeefa2c261..d50765800bc 100644 --- a/libbeat/_meta/config.reference.yml +++ b/libbeat/_meta/config.reference.yml @@ -43,6 +43,66 @@ # if the number of events stored in the queue is < min_flush_events. #flush.timeout: 1s + # The spool queue will store events in a local spool file, before + # forwarding the events to the outputs. + # + # Beta: spooling to disk is currently a beta feature. Use with care. + # + # The spool file is a circular buffer, which blocks once the file/buffer is full. + # Events are put into a write buffer and flushed once the write buffer + # is full or the flush_timeout is triggered. + # Once ACKed by the output, events are removed immediately from the queue, + # making space for new events to be persisted. + #spool: + # The file namespace configures the file path and the file creation settings. + # Once the file exists, the `size`, `page_size` and `prealloc` settings + # will have no more effect. + #file: + # Location of spool file. The default value is ${path.data}/spool.dat. + #path: "${path.data}/spool.dat" + + # Configure file permissions if file is created. The default value is 0600. + #permissions: 0600 + + # File size hint. The spool blocks, once this limit is reached. The default value is 100 MiB. + #size: 100MiB + + # The files page size. A file is split into multiple pages of the same size. The default value is 4KiB. + #page_size: 4KiB + + # If prealloc is set, the required space for the file is reserved using + # truncate. The default value is true. + #prealloc: true + + # Spool writer settings + # Events are serialized into a write buffer. The write buffer is flushed if: + # - The buffer limit has been reached. + # - The configured limit of buffered events is reached. + # - The flush timeout is triggered. + #write: + # Sets the write buffer size. + #buffer_size: 1MiB + + # Maximum duration after which events are flushed, if the write buffer + # is not full yet. The default value is 1s. + #flush.timeout: 1s + + # Number of maximum buffered events. The write buffer is flushed once the + # limit is reached. + #flush.events: 16384 + + # Configure the on-disk event encoding. The encoding can be changed + # between restarts. + # Valid encodings are: json, ubjson, and cbor. + #codec: cbor + #read: + # Reader flush timeout, waiting for more events to become available, so + # to fill a complete batch, as required by the outputs. + # If flush_timeout is 0, all available events are forwarded to the + # outputs immediately. + # The default value is 0s. + #flush.timeout: 0s + # Sets the maximum number of CPUs that can be executing simultaneously. The # default is the number of logical CPUs available in the system. #max_procs: diff --git a/libbeat/publisher/includes/includes.go b/libbeat/publisher/includes/includes.go index e36aed5258e..99b65e0984c 100644 --- a/libbeat/publisher/includes/includes.go +++ b/libbeat/publisher/includes/includes.go @@ -1,6 +1,10 @@ package includes import ( + // import queue types + _ "github.com/elastic/beats/libbeat/publisher/queue/memqueue" + _ "github.com/elastic/beats/libbeat/publisher/queue/spool" + // load supported output plugins _ "github.com/elastic/beats/libbeat/outputs/console" _ "github.com/elastic/beats/libbeat/outputs/elasticsearch" diff --git a/libbeat/publisher/pipeline/client_ack.go b/libbeat/publisher/pipeline/client_ack.go index f5d3749e721..a9dfd6e7e33 100644 --- a/libbeat/publisher/pipeline/client_ack.go +++ b/libbeat/publisher/pipeline/client_ack.go @@ -13,7 +13,7 @@ type clientACKer struct { } func (p *Pipeline) makeACKer( - withProcessors bool, + canDrop bool, cfg *beat.ClientConfig, waitClose time.Duration, ) acker { @@ -25,17 +25,17 @@ func (p *Pipeline) makeACKer( sema := p.eventSema switch { case cfg.ACKCount != nil: - acker = bld.createCountACKer(withProcessors, sema, cfg.ACKCount) + acker = bld.createCountACKer(canDrop, sema, cfg.ACKCount) case cfg.ACKEvents != nil: - acker = bld.createEventACKer(withProcessors, sema, cfg.ACKEvents) + acker = bld.createEventACKer(canDrop, sema, cfg.ACKEvents) case cfg.ACKLastEvent != nil: cb := lastEventACK(cfg.ACKLastEvent) - acker = bld.createEventACKer(withProcessors, sema, cb) + acker = bld.createEventACKer(canDrop, sema, cb) default: if waitClose <= 0 { - return bld.createPipelineACKer(withProcessors, sema) + return bld.createPipelineACKer(canDrop, sema) } - acker = bld.createCountACKer(withProcessors, sema, func(_ int) {}) + acker = bld.createCountACKer(canDrop, sema, func(_ int) {}) } if waitClose <= 0 { diff --git a/libbeat/publisher/pipeline/pipeline.go b/libbeat/publisher/pipeline/pipeline.go index bb060324ba0..a5f3f4a1e95 100644 --- a/libbeat/publisher/pipeline/pipeline.go +++ b/libbeat/publisher/pipeline/pipeline.go @@ -173,7 +173,18 @@ func New( if err != nil { return nil, err } - p.eventSema = newSema(p.queue.BufferConfig().Events) + + if count := p.queue.BufferConfig().Events; count > 0 { + p.eventSema = newSema(count) + } + + maxEvents := p.queue.BufferConfig().Events + if maxEvents <= 0 { + // Maximum number of events until acker starts blocking. + // Only active if pipeline can drop events. + maxEvents = 64000 + } + p.eventSema = newSema(maxEvents) p.output = newOutputController(log, p.observer, p.queue) p.output.Set(out) diff --git a/libbeat/publisher/pipeline/stress/configs/pipeline/default.yml b/libbeat/publisher/pipeline/stress/configs/pipeline/default_mem.yml similarity index 75% rename from libbeat/publisher/pipeline/stress/configs/pipeline/default.yml rename to libbeat/publisher/pipeline/stress/configs/pipeline/default_mem.yml index f6dc45cc7ef..ead9e17ecc4 100644 --- a/libbeat/publisher/pipeline/stress/configs/pipeline/default.yml +++ b/libbeat/publisher/pipeline/stress/configs/pipeline/default_mem.yml @@ -1,4 +1,4 @@ -queue.mem: +pipeline.queue.mem: events: 4096 flush: min_events: 2048 diff --git a/libbeat/publisher/pipeline/stress/configs/pipeline/direct.yml b/libbeat/publisher/pipeline/stress/configs/pipeline/direct_mem.yml similarity index 74% rename from libbeat/publisher/pipeline/stress/configs/pipeline/direct.yml rename to libbeat/publisher/pipeline/stress/configs/pipeline/direct_mem.yml index 7ab3b82dcc8..cf8fdc2f9d2 100644 --- a/libbeat/publisher/pipeline/stress/configs/pipeline/direct.yml +++ b/libbeat/publisher/pipeline/stress/configs/pipeline/direct_mem.yml @@ -1,4 +1,4 @@ -queue.mem: +pipeline.queue.mem: events: 4096 flush: min_events: 0 diff --git a/libbeat/publisher/pipeline/stress/configs/pipeline/small_spool.yml b/libbeat/publisher/pipeline/stress/configs/pipeline/small_spool.yml new file mode 100644 index 00000000000..d5f999440e3 --- /dev/null +++ b/libbeat/publisher/pipeline/stress/configs/pipeline/small_spool.yml @@ -0,0 +1,11 @@ +pipeline.queue.spool: + file: + path: ${test.tmpdir}/${test.name}-spool.dat + size: 1MiB + page_size: 4KiB + prealloc: true + write: + buffer_size: 16KiB + flush_timeout: 100ms + read: + flush_timeout: 0 diff --git a/libbeat/publisher/pipeline/stress/gen.go b/libbeat/publisher/pipeline/stress/gen.go index e0afdfd2f0a..e3c7bd30775 100644 --- a/libbeat/publisher/pipeline/stress/gen.go +++ b/libbeat/publisher/pipeline/stress/gen.go @@ -1,7 +1,9 @@ package stress import ( + "bytes" "fmt" + "runtime/pprof" "sync" "time" @@ -25,7 +27,7 @@ var defaultGenerateConfig = generateConfig{ ACK: false, MaxEvents: 0, WaitClose: 0, - Watchdog: 1 * time.Second, + Watchdog: 2 * time.Second, } var publishModes = map[string]beat.PublishMode{ @@ -92,7 +94,7 @@ func generate( // start generator watchdog withWG(&wg, func() { last := uint64(0) - ticker := time.NewTicker(config.Watchdog) // todo: make ticker interval configurable + ticker := time.NewTicker(config.Watchdog) defer ticker.Stop() for { select { @@ -105,7 +107,11 @@ func generate( current := count.Load() if last == current { - err := fmt.Errorf("no progress in generators (last=%v, current=%v)", last, current) + // collect all active go-routines stack-traces: + var buf bytes.Buffer + pprof.Lookup("goroutine").WriteTo(&buf, 2) + + err := fmt.Errorf("no progress in generator %v (last=%v, current=%v):\n%s", id, last, current, buf.Bytes()) errors(err) } last = current diff --git a/libbeat/publisher/pipeline/stress/run.go b/libbeat/publisher/pipeline/stress/run.go index ce60355de36..27125a9dc83 100644 --- a/libbeat/publisher/pipeline/stress/run.go +++ b/libbeat/publisher/pipeline/stress/run.go @@ -1,6 +1,7 @@ package stress import ( + "fmt" "sync" "time" @@ -36,13 +37,13 @@ func RunTests( ) error { config := defaultConfig if err := cfg.Unpack(&config); err != nil { - return err + return fmt.Errorf("unpacking config failed: %v", err) } // reg := monitoring.NewRegistry() pipeline, err := pipeline.Load(info, nil, config.Pipeline, config.Output) if err != nil { - return err + return fmt.Errorf("loading pipeline failed: %v", err) } defer func() { logp.Info("Stop pipeline") diff --git a/libbeat/publisher/pipeline/stress/stress_test.go b/libbeat/publisher/pipeline/stress/stress_test.go index c019096a276..93f3021420f 100644 --- a/libbeat/publisher/pipeline/stress/stress_test.go +++ b/libbeat/publisher/pipeline/stress/stress_test.go @@ -5,7 +5,10 @@ package stress_test import ( "flag" "fmt" + "io/ioutil" + "os" "path/filepath" + "strings" "testing" "time" @@ -15,6 +18,7 @@ import ( // import queue types "github.com/elastic/beats/libbeat/publisher/pipeline/stress" _ "github.com/elastic/beats/libbeat/publisher/queue/memqueue" + _ "github.com/elastic/beats/libbeat/publisher/queue/spool" ) // additional flags @@ -39,19 +43,46 @@ func TestPipeline(t *testing.T) { } if duration == 0 { - duration = 10 * time.Second + duration = 15 * time.Second } - // TODO: if verbose, enable logging - configTest(t, "gen", genConfigs, func(t *testing.T, gen string) { configTest(t, "pipeline", pipelineConfigs, func(t *testing.T, pipeline string) { configTest(t, "out", outConfigs, func(t *testing.T, out string) { + + if testing.Verbose() { + start := time.Now() + fmt.Printf("%v Start stress test %v\n", start.Format(time.RFC3339), t.Name()) + defer func() { + end := time.Now() + fmt.Printf("%v Finished stress test %v. Duration=%v\n", end.Format(time.RFC3339), t.Name(), end.Sub(start)) + }() + } + config, err := common.LoadFiles(gen, pipeline, out) if err != nil { t.Fatal(err) } + name := t.Name() + name = strings.Replace(name, "/", "-", -1) + name = strings.Replace(name, "\\", "-", -1) + + dir, err := ioutil.TempDir("", "") + if err != nil { + t.Fatal(err) + } + defer os.RemoveAll(dir) + + // Merge test info into config object + config.Merge(map[string]interface{}{ + "test": map[string]interface{}{ + "tmpdir": dir, + "name": name, + }, + }) + + // check if the pipeline configuration allows for parallel tests onErr := func(err error) { t.Error(err) } diff --git a/libbeat/publisher/queue/queuetest/queuetest.go b/libbeat/publisher/queue/queuetest/queuetest.go index b4e2b2b86e7..835da001b9f 100644 --- a/libbeat/publisher/queue/queuetest/queuetest.go +++ b/libbeat/publisher/queue/queuetest/queuetest.go @@ -8,6 +8,7 @@ import ( "github.com/elastic/beats/libbeat/publisher/queue" ) +// QueueFactory is used to create a per test queue instance. type QueueFactory func(t *testing.T) queue.Queue type workerFactory func(*sync.WaitGroup, interface{}, *TestLogger, queue.Queue) func() diff --git a/libbeat/publisher/queue/spool/codec.go b/libbeat/publisher/queue/spool/codec.go new file mode 100644 index 00000000000..051ae656ca2 --- /dev/null +++ b/libbeat/publisher/queue/spool/codec.go @@ -0,0 +1,179 @@ +package spool + +import ( + "bytes" + "fmt" + "time" + + "github.com/elastic/go-structform" + "github.com/elastic/go-structform/cborl" + "github.com/elastic/go-structform/gotype" + "github.com/elastic/go-structform/json" + "github.com/elastic/go-structform/ubjson" + + "github.com/elastic/beats/libbeat/beat" + "github.com/elastic/beats/libbeat/common" + "github.com/elastic/beats/libbeat/publisher" +) + +type encoder struct { + buf bytes.Buffer + folder *gotype.Iterator + codec codecID +} + +type decoder struct { + buf []byte + + json *json.Parser + cborl *cborl.Parser + ubjson *ubjson.Parser + unfolder *gotype.Unfolder +} + +type codecID uint8 + +type entry struct { + Timestamp int64 + Flags uint8 + Meta common.MapStr + Fields common.MapStr +} + +const ( + // Note: Never change order. Codec IDs must be not change in the future. Only + // adding new IDs is allowed. + codecUnknown codecID = iota + codecJSON + codecUBJSON + codecCBORL + + flagGuaranteed uint8 = 1 << 0 +) + +func newEncoder(codec codecID) (*encoder, error) { + switch codec { + case codecJSON, codecCBORL, codecUBJSON: + break + default: + return nil, fmt.Errorf("unknown codec type '%v'", codec) + } + + e := &encoder{codec: codec} + e.reset() + return e, nil +} + +func (e *encoder) reset() { + e.folder = nil + + var visitor structform.Visitor + switch e.codec { + case codecJSON: + visitor = json.NewVisitor(&e.buf) + case codecCBORL: + visitor = cborl.NewVisitor(&e.buf) + case codecUBJSON: + visitor = ubjson.NewVisitor(&e.buf) + default: + panic("no codec configured") + } + + folder, err := gotype.NewIterator(visitor) + if err != nil { + panic(err) + } + + e.folder = folder +} + +func (e *encoder) encode(event *publisher.Event) ([]byte, error) { + e.buf.Reset() + e.buf.WriteByte(byte(e.codec)) + + 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.json = json.NewParser(unfolder) + d.cborl = cborl.NewParser(unfolder) + d.ubjson = ubjson.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 + codec = codecID(d.buf[0]) + contents = d.buf[1:] + ) + + d.unfolder.SetTarget(&to) + switch codec { + case codecJSON: + err = d.json.Parse(contents) + case codecUBJSON: + err = d.ubjson.Parse(contents) + case codecCBORL: + err = d.cborl.Parse(contents) + default: + return publisher.Event{}, fmt.Errorf("unknown codec type '%v'", codec) + } + + 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 +} diff --git a/libbeat/publisher/queue/spool/config.go b/libbeat/publisher/queue/spool/config.go new file mode 100644 index 00000000000..aaf9a5b93a6 --- /dev/null +++ b/libbeat/publisher/queue/spool/config.go @@ -0,0 +1,122 @@ +package spool + +import ( + "errors" + "fmt" + "os" + "strings" + "time" + + "github.com/dustin/go-humanize" + "github.com/joeshaw/multierror" +) + +type config struct { + File pathConfig `config:"file"` + Write writeConfig `config:"write"` + Read readConfig `config:"read"` +} + +type pathConfig struct { + Path string `config:"path"` + Permissions os.FileMode `config:"permissions"` + MaxSize size `config:"size"` + PageSize size `config:"page_size"` + Prealloc bool `config:"prealloc"` +} + +type writeConfig struct { + BufferSize size `config:"buffer_size"` + FlushEvents time.Duration `config:"flush.events"` + FlushTimeout time.Duration `config:"flush.timeout"` + Codec codecID `config:"codec"` +} + +type readConfig struct { + FlushTimeout time.Duration `config:"flush.timeout"` +} + +func defaultConfig() config { + return config{ + File: pathConfig{ + Path: "", + Permissions: 0600, + MaxSize: 100 * humanize.MiByte, + PageSize: 4 * humanize.KiByte, + Prealloc: true, + }, + Write: writeConfig{ + BufferSize: 1 * humanize.MiByte, + FlushTimeout: 1 * time.Second, + FlushEvents: 16 * 1024, + Codec: codecCBORL, + }, + Read: readConfig{ + FlushTimeout: 0, + }, + } +} + +type size uint64 + +func (c *pathConfig) Validate() error { + var errs multierror.Errors + + if c.MaxSize < humanize.MiByte { + errs = append(errs, errors.New("max size must be larger 1MiB")) + } + + if !c.Permissions.IsRegular() { + errs = append(errs, fmt.Errorf("permissions %v are not regular file permissions", c.Permissions.String())) + } else { + m := c.Permissions.Perm() + if (m & 0400) == 0 { + errs = append(errs, errors.New("file must be readable by current user")) + } + if (m & 0200) == 0 { + errs = append(errs, errors.New("file must be writable by current user")) + } + } + + // TODO: good 'limit' on pageSize? + + if c.PageSize >= c.MaxSize { + errs = append(errs, fmt.Errorf("page_size (%v) must be less then size (%v)", c.PageSize, c.MaxSize)) + } + + return errs.Err() +} + +func (c *writeConfig) Validate() error { + return nil +} + +func (c *readConfig) Validate() error { + return nil +} + +func (s *size) Unpack(value string) error { + sz, err := humanize.ParseBytes(value) + if err != nil { + return err + } + + *s = size(sz) + return nil +} + +func (c *codecID) Unpack(value string) error { + ids := map[string]codecID{ + "json": codecJSON, + "ubjson": codecUBJSON, + "cbor": codecCBORL, + } + + id, exists := ids[strings.ToLower(value)] + if !exists { + return fmt.Errorf("codec '%v' not available", value) + } + + *c = id + return nil +} diff --git a/libbeat/publisher/queue/spool/consume.go b/libbeat/publisher/queue/spool/consume.go new file mode 100644 index 00000000000..5e543fc2f71 --- /dev/null +++ b/libbeat/publisher/queue/spool/consume.go @@ -0,0 +1,122 @@ +package spool + +import ( + "errors" + "io" + + "github.com/elastic/beats/libbeat/common/atomic" + "github.com/elastic/beats/libbeat/publisher" + "github.com/elastic/beats/libbeat/publisher/queue" +) + +type consumer struct { + ctx *spoolCtx + closed atomic.Bool + done chan struct{} + + resp chan getResponse + requ chan getRequest +} + +type batch struct { + events []publisher.Event + state ackState + ack chan batchAckMsg +} + +type ackState uint8 + +const ( + batchActive ackState = iota + batchACK +) + +func newConsumer(ctx *spoolCtx, requ chan getRequest) *consumer { + return &consumer{ + ctx: ctx, + closed: atomic.MakeBool(false), + done: make(chan struct{}), + + // internal API + resp: make(chan getResponse), + requ: requ, + } +} + +func (c *consumer) Close() error { + if c.closed.Swap(true) { + return errors.New("already closed") + } + + close(c.done) + return nil +} + +func (c *consumer) Closed() bool { + return c.closed.Load() || c.ctx.Closed() +} + +func (c *consumer) Get(sz int) (queue.Batch, error) { + log := c.ctx.logger + + if c.Closed() { + return nil, io.EOF + } + + var resp getResponse + for { + select { + case <-c.ctx.Done(): + return nil, io.EOF + + case <-c.done: + return nil, io.EOF + + case c.requ <- getRequest{sz: sz, resp: c.resp}: + } + + resp = <-c.resp + err := resp.err + if err == nil { + break + } + + if err != errRetry { + log.Debug("consumer: error response:", err) + return nil, err + } + } + + log.Debug("consumer: received batch:", len(resp.buf)) + return &batch{ + events: resp.buf, + state: batchActive, + ack: resp.ack, + }, nil +} + +func (b *batch) Events() []publisher.Event { + if b.state != batchActive { + panic("Get Events from inactive batch") + } + return b.events +} + +func (b *batch) ACK() { + if b.state != batchActive { + switch b.state { + case batchACK: + panic("Can not acknowledge already acknowledged batch") + default: + panic("inactive batch") + } + } + + b.report() +} + +func (b *batch) report() { + if b.ack != nil { + b.ack <- batchAckMsg{} + } +} diff --git a/libbeat/publisher/queue/spool/inbroker.go b/libbeat/publisher/queue/spool/inbroker.go new file mode 100644 index 00000000000..a3fcc8d3b7c --- /dev/null +++ b/libbeat/publisher/queue/spool/inbroker.go @@ -0,0 +1,522 @@ +package spool + +import ( + "fmt" + "math" + "time" + + "github.com/elastic/beats/libbeat/publisher/queue" + "github.com/elastic/go-txfile/pq" +) + +type inBroker struct { + ctx *spoolCtx + eventer queue.Eventer + + // active state handler + state func(*inBroker) bool + + // api channels + events chan pushRequest + pubCancel chan producerCancelRequest + + // queue signaling + sigACK chan struct{} + sigFlush chan uint + ackDone chan struct{} + + // queue state + queue *pq.Queue + clientStates clientStates + + // Event contents, that still needs to be send to the queue. An event is + // pending if it has been serialized, but not added to the write buffer in + // full, as some I/O operation on the write buffer failed. + // => + // - keep pointer to yet unwritten event contents + // - do not accept any events if pending is not nil + // - wait for signal from reader/queue-gc to retry writing the pending + // events contents + pending []byte + + bufferedEvents uint // number of buffered events + + // flush settings + timer *timer + flushEvents uint + + enc *encoder +} + +const ( + inSigChannelSize = 3 + inEventChannelSize = 20 +) + +func newInBroker( + ctx *spoolCtx, + eventer queue.Eventer, + qu *pq.Queue, + codec codecID, + flushTimeout time.Duration, + flushEvents uint, +) (*inBroker, error) { + enc, err := newEncoder(codec) + if err != nil { + return nil, err + } + + b := &inBroker{ + ctx: ctx, + eventer: eventer, + state: (*inBroker).stateEmpty, + + // API + events: make(chan pushRequest, inEventChannelSize), + pubCancel: make(chan producerCancelRequest), + sigACK: make(chan struct{}, inSigChannelSize), + sigFlush: make(chan uint, inSigChannelSize), + ackDone: make(chan struct{}), + + // queue state + queue: qu, + clientStates: clientStates{}, + pending: nil, + bufferedEvents: 0, + + // internal + timer: newTimer(flushTimeout), + flushEvents: flushEvents, + enc: enc, + } + + ctx.Go(b.eventLoop) + ctx.Go(b.ackLoop) + return b, nil +} + +func (b *inBroker) Producer(cfg queue.ProducerConfig) queue.Producer { + return newProducer(b.ctx, b.pubCancel, b.events, cfg.ACK, cfg.OnDrop, cfg.DropOnCancel) +} + +// onFlush is run whenever the queue flushes it's write buffer. The callback is +// run in the same go-routine as the Flush was executed from. +// Only the (*inBroker).eventLoop triggers a flush. +func (b *inBroker) onFlush(n uint) { + if n == 0 { + return + } + + if b.eventer != nil { + b.eventer.OnACK(int(n)) + } + b.ctx.logger.Debug("inbroker: flushed events:", n) + b.bufferedEvents -= n + b.sigFlush <- n +} + +// onACK is run whenever the queue releases ACKed events. The number of acked +// events and freed pages will is reported. +// Flush events are forward to the brokers eventloop, so to give the broker a +// chance to retry writing in case it has been blocked on a full queue. +func (b *inBroker) onACK(events, pages uint) { + if pages > 0 { + b.sigACK <- struct{}{} + } +} + +func (b *inBroker) ackLoop() { + log := b.ctx.logger + + log.Debug("start flush ack loop") + defer log.Debug("stop flush ack loop") + + for { + var n uint + select { + case <-b.ackDone: + return + + case n = <-b.sigFlush: + log.Debug("inbroker: receive flush", n) + states := b.clientStates.Pop(int(n)) + b.sendACKs(states) + } + } +} + +// sendACKs returns the range of ACKed/Flushed events to the individual +// producers ACK handlers. +func (b *inBroker) sendACKs(states []clientState) { + log := b.ctx.logger + + // reverse iteration on client states, so to report ranges of ACKed events + // only once. + N := len(states) + total := 0 + for i := N - 1; i != -1; i-- { + st := &states[i] + if st.state == nil { + continue + } + + count := (st.seq - st.state.lastACK) + if count == 0 || count > math.MaxUint32/2 { + // seq number comparison did underflow. This happens only if st.seq has + // already been acknowledged + // log.Debug("seq number already acked: ", st.seq) + + st.state = nil + continue + } + + log.Debugf("broker ACK events: count=%v, start-seq=%v, end-seq=%v\n", + count, + st.state.lastACK+1, + st.seq, + ) + + total += int(count) + if total > N { + panic(fmt.Sprintf("Too many events acked (expected=%v, total=%v)", + N, total, + )) + } + + // report range of ACKed events + st.state.ackCB(int(count)) + st.state.lastACK = st.seq + st.state = nil + } +} + +func (b *inBroker) eventLoop() { + log := b.ctx.logger + log.Info("spool input eventloop start") + defer log.Info("spool input eventloop stop") + + // notify ackLoop to stop only after eventLoop has finished (after last flush) + defer close(b.ackDone) + + for { + ok := b.state(b) + if !ok { + break + } + } + + // try to flush events/buffers on shutdown. + if b.bufferedEvents == 0 { + return + } + + // try to append pending events + for len(b.pending) > 0 { + n, err := b.queue.Writer().Write(b.pending) + b.pending = b.pending[n:] + if err != nil { + return + } + } + + // final flush + b.queue.Writer().Flush() +} + +// stateEmpty is the brokers active state if the write buffer is empty and the +// queue did not block on write or flush operations. +// ACKs from the output are ignored, as events can still be added to the write +// buffer. +// +// stateEmpty transitions: +// -> stateEmpty if serializing the event failed +// -> stateWithTimer if event is written to buffer without flush +// => start timer +// -> stateBlocked if queue did return an error on write (Flush failed) +func (b *inBroker) stateEmpty() bool { + log := b.ctx.logger + + select { + case <-b.ctx.Done(): + return false + + case req := <-b.events: + log.Debug("inbroker (stateEmpty): new event") + + buf, st, err := b.encodeEvent(&req) + if err != nil { + log.Debug(" inbroker (stateEmpty): encode failed") + b.respondDrop(&req) + break + } + + // write/flush failed -> block until space in file becomes available + err = b.addEvent(buf, st) + if err != nil { + log.Debug(" inbroker: append failed, blocking") + b.state = (*inBroker).stateBlocked + break + } + + // start flush timer + if b.flushEvents > 0 && b.bufferedEvents == b.flushEvents { + log.Debug(" inbroker (stateEmpty): flush events") + err := b.flushBuffer() + if err != nil { + log.Debug(" inbroker (stateEmpty): flush failed, blocking") + b.state = (*inBroker).stateBlocked + } + break + + } else if b.bufferedEvents > 0 { + log.Debug(" inbroker (stateEmpty): start flush timer") + b.timer.Start() + b.state = (*inBroker).stateWithTimer + } + + case req := <-b.pubCancel: + b.handleCancel(&req) + + case <-b.sigACK: + // ignore ACKs as long as we can write without blocking + } + + return true +} + +// stateWithTimer is the brokers active state, if the write buffer is not empty. +// The flush timer is enabled as long as the broker is in this state. +// ACKs from the output are ignored, as events can still be added to the write +// buffer. +// +// stateWithTimer transitions: +// -> stateWithTimer +// - if serializing failed +// - if event is added to buffer, without flush +// - flush, but more events are available in the buffer (might reset timer) +// -> stateEmpty if all events have been flushed +// -> stateBlocked if queue did return an error on write/flush (Flush failed) +func (b *inBroker) stateWithTimer() bool { + log := b.ctx.logger + + select { + case <-b.ctx.Done(): + return false + + case req := <-b.events: + log.Debug("inbroker (stateWithTimer): new event") + + buf, st, err := b.encodeEvent(&req) + if err != nil { + log.Debug(" inbroker (stateWithTimer): encode failed") + b.respondDrop(&req) + break + } + + count := b.bufferedEvents + err = b.addEvent(buf, st) + if err != nil { + log.Debug(" inbroker (stateWithTimer): append failed, blocking") + b.state = (*inBroker).stateBlocked + break + } + + flushed := b.bufferedEvents < count + if !flushed && b.flushEvents > 0 && b.bufferedEvents == b.flushEvents { + err := b.flushBuffer() + if err != nil { + log.Debug(" inbroker (stateWithTimer): flush failed, blocking") + b.state = (*inBroker).stateBlocked + break + } + + flushed = true + } + + if !flushed { + break + } + + // write buffer has been flushed, reset timer and broker state + log.Debug(" inbroker (stateWithTimer): buffer flushed") + if b.bufferedEvents == 0 { + b.timer.Stop(false) + b.state = (*inBroker).stateEmpty + } else { + // restart timer, as new event is most likely the only event buffered + // -> reduce IO + log.Debug(" inbroker (stateWithTimer): start flush timer") + b.timer.Restart() + } + + case req := <-b.pubCancel: + b.handleCancel(&req) + + case <-b.timer.C: + log.Debug("inbroker (stateWithTimer): flush timeout") + + b.timer.Stop(true) + + err := b.flushBuffer() + if err != nil { + log.Debug(" inbroker (stateWithTimer): flush failed, blocking") + b.state = (*inBroker).stateBlocked + break + } + + log.Debug(" inbroker (stateWithTimer): flush succeeded") + + if b.bufferedEvents > 0 { + // flush did not push all events? Restart timer. + log.Debug(" inbroker (stateWithTimer): start flush timer") + b.timer.Start() + break + } + + b.state = (*inBroker).stateEmpty + + case <-b.sigACK: + // ignore ACKs as long as we can write without blocking + } + + return true +} + +// stateBlocked is the brokers active state if the write buffer can not accept +// any new events. +// The broker will wait for an ACK signal from the outputs and retry flushing, +// in the hope of enough memory being available to flush the buffers. +// If flush did succeed, we try to add the pending event. +// For the time the broker is in this state, no events from any producers will +// be accepted. Thusly all producers will block. Closing a producer, unblocks +// the producer. The producers event (after close) might be processed or +// ignored in the future. +// +// stateBlocked transitions: +// -> stateEmpty if flush was successfull and write buffer is empty +// -> stateWithTimer if flush was successfull, but we still have some pending events +// -> stateBlocked if flush failed (still not enough space) +func (b *inBroker) stateBlocked() bool { + log := b.ctx.logger + + select { + case <-b.ctx.Done(): + return false + + case req := <-b.pubCancel: + b.handleCancel(&req) + + case <-b.sigACK: + // TODO: + // Have write buffer report number of unallocated pages and take number + // of freed pages into account before retrying. This way no transaction + // must be created if it's already clear the flush will not succeed. + + log.Debug("inbroker (stateBlocked): ACK event from queue -> try to unblock") + + err := b.flushBuffer() + if err != nil { + log.Debug(" inbroker (stateBlocked): flush failed, blocking") + break + } + + if len(b.pending) > 0 { + tmp := b.pending + b.pending = nil + err := b.writeEvent(tmp) + if err != nil || len(b.pending) > 0 { + log.Debug("writing pending event failed: ", err) + break + } + } + + if b.bufferedEvents == 0 { + b.state = (*inBroker).stateEmpty + break + } + + b.timer.Start() + log.Debug(" inbroker (stateBlocked): start flush timer") + b.state = (*inBroker).stateWithTimer + } + + return true +} + +func (b *inBroker) handleCancel(req *producerCancelRequest) { + // mark state as cancelled, so to not accept any new events + // from the state object. + if st := req.state; st != nil { + st.cancelled = true + } + + if req.resp != nil { + req.resp <- producerCancelResponse{removed: 0} + } +} + +func (b *inBroker) encodeEvent(req *pushRequest) ([]byte, clientState, error) { + buf, err := b.enc.encode(&req.event) + if err != nil { + return nil, clientState{}, err + } + + if req.state == nil { + return buf, clientState{}, nil + } + + return buf, clientState{seq: req.seq, state: req.state}, nil +} + +func (b *inBroker) respondDrop(req *pushRequest) { + if req.state != nil { + if cb := req.state.dropCB; cb != nil { + cb(req.event.Content) + } + } +} + +func (b *inBroker) addEvent(buf []byte, st clientState) error { + log := b.ctx.logger + + b.bufferedEvents++ + log.Debug(" inbroker: add event of size", len(buf), b.bufferedEvents) + + count := b.clientStates.Add(st) + log.Debug(" add event -> active:", count) + + err := b.writeEvent(buf) + log.Debug(" inbroker write ->", err, b.bufferedEvents) + + return err +} + +func (b *inBroker) writeEvent(buf []byte) error { + log := b.ctx.logger + + // append event to queue + queueWriter := b.queue.Writer() + n, err := queueWriter.Write(buf) + buf = buf[n:] + if len(buf) > 0 { + b.pending = buf + } else if err == nil { + log.Debug("writer: finalize event in buffer") + err = queueWriter.Next() + } + + if err != nil { + log := b.ctx.logger + log.Debugf("appending event content to write buffer failed with %v", err) + } + return err +} + +func (b *inBroker) flushBuffer() error { + err := b.queue.Writer().Flush() + if err != nil { + log := b.ctx.logger + log.Debugf("spool flush failed with: %v", err) + } + return err +} diff --git a/libbeat/publisher/queue/spool/internal_api.go b/libbeat/publisher/queue/spool/internal_api.go new file mode 100644 index 00000000000..28a1a99a58c --- /dev/null +++ b/libbeat/publisher/queue/spool/internal_api.go @@ -0,0 +1,44 @@ +package spool + +import ( + "github.com/elastic/beats/libbeat/publisher" +) + +// producer -> broker API +type ( + pushRequest struct { + event publisher.Event + seq uint32 + state *produceState + } + + producerCancelRequest struct { + state *produceState + resp chan producerCancelResponse + } + + producerCancelResponse struct { + removed int + } +) + +// consumer -> broker API + +type ( + getRequest struct { + sz int // request sz events from the broker + resp chan getResponse // channel to send response to + } + + getResponse struct { + ack chan batchAckMsg + err error + buf []publisher.Event + } + + batchAckMsg struct{} + + batchCancelRequest struct { + // ack *ackChan + } +) diff --git a/libbeat/publisher/queue/spool/log.go b/libbeat/publisher/queue/spool/log.go new file mode 100644 index 00000000000..5dd17048061 --- /dev/null +++ b/libbeat/publisher/queue/spool/log.go @@ -0,0 +1,54 @@ +package spool + +import ( + "fmt" + "sync" + + "github.com/elastic/beats/libbeat/logp" +) + +type logger interface { + Debug(...interface{}) + Debugf(string, ...interface{}) + + Info(...interface{}) + Infof(string, ...interface{}) + + Error(...interface{}) + Errorf(string, ...interface{}) +} + +var _defaultLogger struct { + singleton logger + init sync.Once +} + +func defaultLogger() logger { + _defaultLogger.init.Do(func() { + _defaultLogger.singleton = logp.NewLogger("spool") + }) + return _defaultLogger.singleton +} + +// func defaultLogger() logger { return (*outLogger)(nil) } + +type outLogger struct{} + +func (l *outLogger) Debug(vs ...interface{}) { l.report("Debug", vs) } +func (l *outLogger) Debugf(fmt string, vs ...interface{}) { l.reportf("Debug: ", fmt, vs) } + +func (l *outLogger) Info(vs ...interface{}) { l.report("Info", vs) } +func (l *outLogger) Infof(fmt string, vs ...interface{}) { l.reportf("Info", fmt, vs) } + +func (l *outLogger) Error(vs ...interface{}) { l.report("Error", vs) } +func (l *outLogger) Errorf(fmt string, vs ...interface{}) { l.reportf("Error", fmt, vs) } + +func (l *outLogger) report(level string, vs []interface{}) { + args := append([]interface{}{level, ":"}, vs...) + fmt.Println(args...) +} + +func (*outLogger) reportf(level string, str string, vs []interface{}) { + str = level + ": " + str + fmt.Printf(str, vs...) +} diff --git a/libbeat/publisher/queue/spool/module.go b/libbeat/publisher/queue/spool/module.go new file mode 100644 index 00000000000..78930f8b1c3 --- /dev/null +++ b/libbeat/publisher/queue/spool/module.go @@ -0,0 +1,48 @@ +package spool + +import ( + "github.com/elastic/beats/libbeat/common" + "github.com/elastic/beats/libbeat/common/cfgwarn" + "github.com/elastic/beats/libbeat/paths" + "github.com/elastic/beats/libbeat/publisher/queue" + "github.com/elastic/go-txfile" +) + +func init() { + queue.RegisterType("spool", create) +} + +func create(eventer queue.Eventer, cfg *common.Config) (queue.Queue, error) { + cfgwarn.Beta("Spooling to disk is beta") + + config := defaultConfig() + if err := cfg.Unpack(&config); err != nil { + return nil, err + } + + path := config.File.Path + if path == "" { + path = paths.Resolve(paths.Data, "spool.dat") + } + + flushEvents := uint(0) + if count := config.Write.FlushEvents; count > 0 { + flushEvents = uint(count) + } + + return NewSpool(defaultLogger(), path, Settings{ + Eventer: eventer, + Mode: config.File.Permissions, + WriteBuffer: uint(config.Write.BufferSize), + WriteFlushTimeout: config.Write.FlushTimeout, + WriteFlushEvents: flushEvents, + ReadFlushTimeout: config.Read.FlushTimeout, + Codec: config.Write.Codec, + File: txfile.Options{ + MaxSize: uint64(config.File.MaxSize), + PageSize: uint32(config.File.PageSize), + Prealloc: config.File.Prealloc, + Readonly: false, + }, + }) +} diff --git a/libbeat/publisher/queue/spool/outbroker.go b/libbeat/publisher/queue/spool/outbroker.go new file mode 100644 index 00000000000..cc4d74feaab --- /dev/null +++ b/libbeat/publisher/queue/spool/outbroker.go @@ -0,0 +1,493 @@ +package spool + +import ( + "errors" + "sync" + "time" + + "github.com/elastic/beats/libbeat/publisher" + "github.com/elastic/go-txfile/pq" +) + +type outBroker struct { + ctx *spoolCtx + state func(*outBroker) bool + + // internal API + sigFlushed chan uint + get chan getRequest + + // ack signaling + pendingACKs chanList // list of pending batches to be forwarded to the ackLoop + scheduledACKs chan chanList // shared channel for forwarding batches to ackLoop + schedACKs chan chanList // active ack forwarding channel, as used by broker (nil if pendingACKs is empty) + + // queue state + queue *pq.Queue + available uint // number of available events. getRequests are only accepted if available > 0 + events []publisher.Event + required int + total int + active getRequest + + // internal + timer *timer + dec *decoder +} + +type chanList struct { + head *ackChan + tail *ackChan +} + +type ackChan struct { + next *ackChan + ch chan batchAckMsg + total int // total number of events to ACK with this batch +} + +const ( + // maximum number of events if getRequest size is <0 + maxEvents = 2048 + + outSigChannelSize = 3 +) + +var ackChanPool = sync.Pool{ + New: func() interface{} { + return &ackChan{ + ch: make(chan batchAckMsg, 1), + } + }, +} + +var errRetry = errors.New("retry") + +func newOutBroker(ctx *spoolCtx, qu *pq.Queue, flushTimeout time.Duration) (*outBroker, error) { + b := &outBroker{ + ctx: ctx, + state: nil, + + // API + sigFlushed: make(chan uint, outSigChannelSize), + get: make(chan getRequest), + + // ack signaling + pendingACKs: chanList{}, + scheduledACKs: make(chan chanList), + schedACKs: nil, + + // queue state + queue: qu, + available: qu.Reader().Available(), + events: nil, + required: 0, + total: 0, + active: getRequest{}, + + // internal + timer: newTimer(flushTimeout), + dec: newDecoder(), + } + + b.initState() + ctx.Go(b.eventLoop) + ctx.Go(b.ackLoop) + return b, nil +} + +func (b *outBroker) Consumer() *consumer { + return newConsumer(b.ctx, b.get) +} + +// onFlush is run whenever the queue flushes it's write buffer. The callback is +// run in the same go-routine as the Flush was executed from. +func (b *outBroker) onFlush(n uint) { + if n > 0 { + select { + case <-b.ctx.Done(): // ignore flush messages on shutdown + + case b.sigFlushed <- n: + + } + } +} + +// onACK is run whenever the queue releases ACKed events. The number of acked +// events and freed pages will is reported. +func (b *outBroker) onACK(events, pages uint) { +} + +func (b *outBroker) ackLoop() { + log := b.ctx.logger + + log.Debug("start output ack loop") + defer log.Debug("stop output ack loop") + + var ackList chanList // list of pending acks + for { + select { + case <-b.ctx.Done(): + return + + case lst := <-b.scheduledACKs: + ackList.concat(&lst) + + case <-ackList.channel(): + ackCh := ackList.pop() + + for { + log.Debugf("receive ACK of %v events\n", ackCh.total) + err := b.queue.ACK(uint(ackCh.total)) + if err != nil { + log.Debug("ack failed with:", err) + time.Sleep(1 * time.Second) + continue + } + + log.Debug("ACK succeeded") + break + } + + releaseACKChan(ackCh) + } + } +} + +func (b *outBroker) eventLoop() { + for { + ok := b.state(b) + if !ok { + break + } + } +} + +// initState resets the brokers state to the initial state and clears +// buffers/points from last state updates. +func (b *outBroker) initState() { + b.events = nil + b.required = 0 + b.total = 0 + b.active = getRequest{} + if b.available == 0 { + b.state = (*outBroker).stateWaitEvents + } else { + b.state = (*outBroker).stateActive + } +} + +// stateWaitEvents is the brokers state if the queue is empty. +// The broker waits for new events and does not accept and consumer requests. +// +// stateWaitEvents transitions: +// -> stateActive: if a queue flush signal has been received +func (b *outBroker) stateWaitEvents() bool { + log := b.ctx.logger + log.Debug("outbroker (stateWaitEvents): waiting for new events") + + select { + case <-b.ctx.Done(): + return false + + case n := <-b.sigFlushed: + log.Debug("outbroker (stateWaitEvents): flush event", n) + b.available += n + b.state = (*outBroker).stateActive + + case b.schedACKs <- b.pendingACKs: + b.handleACKsScheduled() + } + + return true +} + +// stateActive is the brokers initial state, waiting for consumer to request +// new events. +// Flush signals from the input are ignored. +// +// stateActive transitions: +// -> stateActive: if consumer event get request has been fulfilled (N events +// copied or 0 timeout) +// -> stateWaitEvents: if queue is empty after read +// -> stateWithTimer: if only small number of events are available and flush +// timeout is configured. +func (b *outBroker) stateActive() bool { + log := b.ctx.logger + + select { + case <-b.ctx.Done(): + return false + + case n := <-b.sigFlushed: + b.available += n + + case b.schedACKs <- b.pendingACKs: + b.handleACKsScheduled() + + case req := <-b.get: + var events []publisher.Event + required := maxEvents + if req.sz > 0 { + events = make([]publisher.Event, 0, req.sz) + required = req.sz + } + + log.Debug("outbroker (stateActive): get request", required) + + var err error + var total int + events, total, err = b.collectEvents(events, required) + required -= len(events) + b.available -= uint(total) + + log.Debug(" outbroker (stateActive): events collected", len(events), total, err) + + // forward error to consumer and continue with current state + if err != nil { + log.Debug(" outbroker (stateActive): return error") + b.returnError(req, events, total, err) + b.initState() + break + } + + // enough events? Return + if required == 0 || (len(events) > 0 && b.timer.Zero()) { + log.Debug(" outbroker (stateActive): return events") + b.returnEvents(req, events, total) + b.initState() // prepare for next request + break + } + + // If no events have been decoded, signal an error to the consumer to retry. + // Meanwhile reinitialize state, waiting for more events. + if len(events) == 0 { + b.returnError(req, nil, total, errRetry) + b.initState() + break + } + + // not enough events -> start timer and try to collect more + b.events = events + b.required = required + b.active = req + b.total = total + b.timer.Start() + log.Debug(" outbroker (stateActive): switch to stateWithTimer") + b.state = (*outBroker).stateWithTimer + } + + return true +} + +// stateWithTimer is the brokers active state, if the events read is less then +// the minimal number of requested events. +// Once the timer triggers or more events have been consumed, the get response +// will be send to the consumer. +// +// stateWithTimer transitions: +// -> stateWithTimer: if some, but not enough events have been read from the +// queue +// -> stateActive: if the timer triggers or enough events have been returned +// to the consumer +func (b *outBroker) stateWithTimer() bool { + log := b.ctx.logger + + select { + case <-b.ctx.Done(): + return false + + case b.schedACKs <- b.pendingACKs: + b.handleACKsScheduled() + + case <-b.timer.C: + b.timer.Stop(true) + log.Debug("outbroker (stateWithTimer): flush timer") + b.returnEvents(b.active, b.events, b.total) + + log.Debug("outbroker (stateWithTimer): switch to stateActive") + b.initState() + + case n := <-b.sigFlushed: + // yay, more events \o/ + + b.available += n + + L := len(b.events) + required := b.required + events, total, err := b.collectEvents(b.events, required) + b.available -= uint(total) + collected := len(events) - L + required -= collected + total += b.total + + log.Debug(" outbroker (stateWithTimer): events collected", len(events), total, err) + + // continue with stateWithTimer? + if err == nil && required > 0 { + b.events = events + b.total = total + b.required = required + log.Debug(" outbroker (stateWithTimer): switch to stateWithTimer") + break + } + + // done serving consumer request + b.timer.Stop(false) + if err != nil { + log.Debug(" outbroker (stateWithTimer): return error") + b.returnError(b.active, events, total, err) + } else { + log.Debug(" outbroker (stateWithTimer): return events") + b.returnEvents(b.active, events, total) + } + + log.Debug("outbroker (stateWithTimer): switch to stateActive") + b.initState() + } + + return true +} + +func (b *outBroker) handleACKsScheduled() { + b.schedACKs = nil + b.pendingACKs = chanList{} +} + +func (b *outBroker) newACKChan(total int) *ackChan { + ackCh := newACKChan(total) + b.pendingACKs.append(ackCh) + b.schedACKs = b.scheduledACKs + return ackCh +} + +// signalDrop forwards an ACK of total events to the ackloop. +// The batch is marked as ACKed by the output. +// signalDrop is used to free space in the queue, in case +// a continuous set of events has been dropped due to decoding errors. +func (b *outBroker) signalDrop(total int) { + ackCh := b.newACKChan(total) + ackCh.ch <- batchAckMsg{} +} + +func (b *outBroker) returnEvents(req getRequest, events []publisher.Event, total int) { + ackCh := b.newACKChan(total) + req.resp <- getResponse{ + ack: ackCh.ch, + err: nil, + buf: events, + } +} + +func (b *outBroker) returnError( + req getRequest, + events []publisher.Event, + total int, + err error, +) { + var ch chan batchAckMsg + + if len(events) == 0 && total > 0 { + b.signalDrop(total) + } + if len(events) > 0 { + ackCh := b.newACKChan(total) + ch = ackCh.ch + } + + req.resp <- getResponse{ + ack: ch, + err: err, + buf: events, + } +} + +func (b *outBroker) collectEvents( + events []publisher.Event, + N int, +) ([]publisher.Event, int, error) { + log := b.ctx.logger + reader := b.queue.Reader() + + count := 0 + for N > 0 { + sz, err := reader.Next() + if sz <= 0 || err != nil { + return events, count, err + } + + count++ + + buf := b.dec.Buffer(sz) + _, err = reader.Read(buf) + if err != nil { + return events, count, err + } + + event, err := b.dec.Decode() + if err != nil { + log.Debug("Failed to decode event from spool: %v", err) + continue + } + + events = append(events, event) + N-- + } + + return events, count, nil +} + +func newACKChan(total int) *ackChan { + c := ackChanPool.Get().(*ackChan) + c.next = nil + c.total = total + return c +} + +func releaseACKChan(c *ackChan) { + c.next = nil + ackChanPool.Put(c) +} + +func (l *chanList) append(ch *ackChan) { + if l.head == nil { + l.head = ch + } else { + l.tail.next = ch + } + l.tail = ch +} + +func (l *chanList) concat(other *chanList) { + if other.head == nil { + return + } + + if l.head == nil { + *l = *other + return + } + + l.tail.next = other.head + l.tail = other.tail +} + +func (l *chanList) channel() chan batchAckMsg { + if l.head == nil { + return nil + } + return l.head.ch +} + +func (l *chanList) pop() *ackChan { + ch := l.head + if ch != nil { + l.head = ch.next + if l.head == nil { + l.tail = nil + } + } + + ch.next = nil + return ch +} diff --git a/libbeat/publisher/queue/spool/produce.go b/libbeat/publisher/queue/spool/produce.go new file mode 100644 index 00000000000..839dac06d79 --- /dev/null +++ b/libbeat/publisher/queue/spool/produce.go @@ -0,0 +1,190 @@ +package spool + +import ( + "sync" + + "github.com/elastic/beats/libbeat/beat" + "github.com/elastic/beats/libbeat/common/atomic" + "github.com/elastic/beats/libbeat/publisher" + "github.com/elastic/beats/libbeat/publisher/queue" +) + +// forgetfullProducer forwards event to the inBroker. The forgetfullProducer +// provides no event ACK handling and no callbacks. +type forgetfullProducer struct { + openState openState +} + +// ackProducer forwards events to the inBroker. The ackBroker provides +// functionality for ACK/Drop callbacks. +type ackProducer struct { + cancel bool + seq uint32 + state produceState + openState openState + pubCancel chan producerCancelRequest +} + +// openState tracks the producer->inBroker connection state. +type openState struct { + ctx *spoolCtx + isOpen atomic.Bool + done chan struct{} + events chan pushRequest +} + +// produceState holds the ackProducer internal callback and event ACK state +// shared between ackProducer instances and inBroker instances. +// The state is used to compute the number of per producer ACKed events and +// executing locally configured callbacks. +type produceState struct { + ackCB ackHandler + dropCB func(beat.Event) + cancelled bool + lastACK uint32 +} + +type ackHandler func(count int) + +type clientStates struct { + mux sync.Mutex + clients []clientState +} + +type clientState struct { + seq uint32 // event sequence number + state *produceState // the producer it's state used to compute and signal the ACK count +} + +func newProducer( + ctx *spoolCtx, + pubCancel chan producerCancelRequest, + events chan pushRequest, + ackCB ackHandler, + dropCB func(beat.Event), + dropOnCancel bool, +) queue.Producer { + openState := openState{ + ctx: ctx, + isOpen: atomic.MakeBool(true), + done: make(chan struct{}), + events: events, + } + + if ackCB == nil { + return &forgetfullProducer{openState: openState} + } + + p := &ackProducer{ + seq: 1, + cancel: dropOnCancel, + openState: openState, + pubCancel: pubCancel, + } + p.state.ackCB = ackCB + p.state.dropCB = dropCB + return p +} + +func (p *forgetfullProducer) Publish(event publisher.Event) bool { + return p.openState.publish(p.makeRequest(event)) +} + +func (p *forgetfullProducer) TryPublish(event publisher.Event) bool { + return p.openState.tryPublish(p.makeRequest(event)) +} + +func (p *forgetfullProducer) makeRequest(event publisher.Event) pushRequest { + return pushRequest{event: event} +} + +func (p *forgetfullProducer) Cancel() int { + p.openState.Close() + return 0 +} + +func (p *ackProducer) Publish(event publisher.Event) bool { + return p.updSeq(p.openState.publish(p.makeRequest(event))) +} + +func (p *ackProducer) TryPublish(event publisher.Event) bool { + return p.updSeq(p.openState.tryPublish(p.makeRequest(event))) +} + +func (p *ackProducer) Cancel() int { + p.openState.Close() + + if p.cancel { + ch := make(chan producerCancelResponse) + p.pubCancel <- producerCancelRequest{ + state: &p.state, + resp: ch, + } + + // wait for cancel to being processed + resp := <-ch + return resp.removed + } + return 0 +} + +func (p *ackProducer) updSeq(ok bool) bool { + if ok { + p.seq++ + } + return ok +} + +func (p *ackProducer) makeRequest(event publisher.Event) pushRequest { + return pushRequest{event: event, seq: p.seq, state: &p.state} +} + +func (st *openState) Close() { + st.isOpen.Store(false) + close(st.done) +} + +func (st *openState) publish(req pushRequest) bool { + select { + case st.events <- req: + return true + case <-st.done: + st.events = nil + return false + } +} + +func (st *openState) tryPublish(req pushRequest) bool { + select { + case st.events <- req: + return true + case <-st.done: + st.events = nil + return false + default: + log := st.ctx.logger + log.Debugf("Dropping event, queue is blocked (seq=%v) ", req.seq) + return false + } +} + +func (s *clientStates) Add(st clientState) int { + s.mux.Lock() + s.clients = append(s.clients, st) + l := len(s.clients) + s.mux.Unlock() + return l +} + +func (s *clientStates) RemoveLast() { + s.mux.Lock() + s.clients = s.clients[:len(s.clients)-1] + s.mux.Unlock() +} + +func (s *clientStates) Pop(n int) (states []clientState) { + s.mux.Lock() + states, s.clients = s.clients[:n], s.clients[n:] + s.mux.Unlock() + return states +} diff --git a/libbeat/publisher/queue/spool/spool.go b/libbeat/publisher/queue/spool/spool.go new file mode 100644 index 00000000000..26fecff27fb --- /dev/null +++ b/libbeat/publisher/queue/spool/spool.go @@ -0,0 +1,229 @@ +package spool + +import ( + "fmt" + "os" + "sync" + "time" + + "github.com/elastic/beats/libbeat/common/atomic" + "github.com/elastic/beats/libbeat/publisher/queue" + "github.com/elastic/go-txfile" + "github.com/elastic/go-txfile/pq" +) + +// Spool implements an on-disk queue.Queue. +type Spool struct { + // producer/input support + inCtx *spoolCtx + inBroker *inBroker + + // consumer/output support + outCtx *spoolCtx + outBroker *outBroker + + queue *pq.Queue + file *txfile.File +} + +type spoolCtx struct { + logger logger + wg sync.WaitGroup + active atomic.Bool + done chan struct{} +} + +// Settings configure a new spool to be created. +type Settings struct { + Mode os.FileMode + + File txfile.Options + + // Queue write buffer size. If a single event is bigger then the + // write-buffer, the write-buffer will grow. In this case will the write + // buffer be flushed and reset to its original size. + WriteBuffer uint + + Eventer queue.Eventer + + WriteFlushTimeout time.Duration + WriteFlushEvents uint + ReadFlushTimeout time.Duration + + Codec codecID +} + +const minInFlushTimeout = 100 * time.Millisecond +const minOutFlushTimeout = 0 * time.Millisecond + +// NewSpool creates and initializes a new file based queue. +func NewSpool(logger logger, path string, settings Settings) (*Spool, error) { + mode := settings.Mode + if mode == 0 { + mode = os.ModePerm + } + + ok := false + inCtx := newSpoolCtx(logger) + outCtx := newSpoolCtx(logger) + defer ifNotOK(&ok, inCtx.Close) + defer ifNotOK(&ok, outCtx.Close) + + if info, err := os.Lstat(path); err != nil { + if !os.IsNotExist(err) { + return nil, err + } + } else { + perm := info.Mode().Perm() + cfgPerm := settings.Mode.Perm() + + // check if file has permissions set, that must not be set via config + if (perm | cfgPerm) != cfgPerm { + return nil, fmt.Errorf("file permissions must be more strict (required permissions: %v, actual permissions: %v)", + cfgPerm, perm) + } + } + + f, err := txfile.Open(path, mode, settings.File) + if err != nil { + return nil, err + } + defer ifNotOK(&ok, ignoreErr(f.Close)) + + queueDelegate, err := pq.NewStandaloneDelegate(f) + if err != nil { + return nil, err + } + + spool := &Spool{ + inCtx: inCtx, + outCtx: outCtx, + } + + queue, err := pq.New(queueDelegate, pq.Settings{ + WriteBuffer: settings.WriteBuffer, + Flushed: spool.onFlush, + ACKed: spool.onACK, + }) + if err != nil { + return nil, err + } + defer ifNotOK(&ok, ignoreErr(queue.Close)) + + inFlushTimeout := settings.WriteFlushTimeout + if inFlushTimeout < minInFlushTimeout { + inFlushTimeout = minInFlushTimeout + } + inBroker, err := newInBroker(inCtx, settings.Eventer, queue, settings.Codec, + inFlushTimeout, settings.WriteFlushEvents) + if err != nil { + return nil, err + } + + outFlushTimeout := settings.ReadFlushTimeout + if outFlushTimeout < minOutFlushTimeout { + outFlushTimeout = minOutFlushTimeout + } + outBroker, err := newOutBroker(outCtx, queue, outFlushTimeout) + if err != nil { + return nil, err + } + + ok = true + spool.queue = queue + spool.inBroker = inBroker + spool.outBroker = outBroker + spool.file = f + return spool, nil +} + +// Close shuts down the queue and closes the used file. +func (s *Spool) Close() error { + // stop all workers (waits for all workers to be finished) + s.outCtx.Close() + s.inCtx.Close() + + // close queue (potentially flushing write buffer) + err := s.queue.Close() + + // finally unmap and close file + s.file.Close() + + return err +} + +// BufferConfig returns the queue initial buffer settings. +func (s *Spool) BufferConfig() queue.BufferConfig { + return queue.BufferConfig{Events: -1} +} + +// Producer creates a new queue producer for publishing events. +func (s *Spool) Producer(cfg queue.ProducerConfig) queue.Producer { + return s.inBroker.Producer(cfg) +} + +// Consumer creates a new queue consumer for consuming and acking events. +func (s *Spool) Consumer() queue.Consumer { + return s.outBroker.Consumer() +} + +// onFlush is run whenever the queue signals it's write buffer being flushed. +// Flush events are forwarded to all workers. +// The onFlush callback is directly called by the queue writer (same go-routine) +// on Write or Flush operations. +func (s *Spool) onFlush(n uint) { + s.inBroker.onFlush(n) + s.outBroker.onFlush(n) +} + +// onACK is run whenever the queue signals events being acked and removed from +// the queue. +// ACK events are forwarded to all workers. +func (s *Spool) onACK(events, pages uint) { + s.inBroker.onACK(events, pages) +} + +func newSpoolCtx(logger logger) *spoolCtx { + return &spoolCtx{ + logger: logger, + active: atomic.MakeBool(true), + done: make(chan struct{}), + } +} + +func (ctx *spoolCtx) Close() { + if ctx.active.CAS(true, false) { + close(ctx.done) + ctx.wg.Wait() + } +} + +func (ctx *spoolCtx) Done() <-chan struct{} { + return ctx.done +} + +func (ctx *spoolCtx) Open() bool { + return ctx.active.Load() +} + +func (ctx *spoolCtx) Closed() bool { + return !ctx.Open() +} + +func (ctx *spoolCtx) Go(fn func()) { + ctx.wg.Add(1) + go func() { + defer ctx.wg.Done() + fn() + }() +} + +func ifNotOK(b *bool, fn func()) { + if !(*b) { + fn() + } +} + +func ignoreErr(fn func() error) func() { + return func() { fn() } +} diff --git a/libbeat/publisher/queue/spool/spool_test.go b/libbeat/publisher/queue/spool/spool_test.go new file mode 100644 index 00000000000..c661e0e81a4 --- /dev/null +++ b/libbeat/publisher/queue/spool/spool_test.go @@ -0,0 +1,130 @@ +package spool + +import ( + "flag" + "fmt" + "math/rand" + "testing" + "time" + + humanize "github.com/dustin/go-humanize" + + "github.com/elastic/beats/libbeat/publisher/queue" + "github.com/elastic/beats/libbeat/publisher/queue/queuetest" + "github.com/elastic/go-txfile" + "github.com/elastic/go-txfile/txfiletest" +) + +var seed int64 +var debug bool + +type testQueue struct { + *Spool + teardown func() +} + +type testLogger struct { + t *testing.T +} + +func init() { + flag.Int64Var(&seed, "seed", time.Now().UnixNano(), "test random seed") + flag.BoolVar(&debug, "noisy", false, "print test logs to console") +} + +func TestProduceConsumer(t *testing.T) { + maxEvents := 4096 + minEvents := 32 + + rand.Seed(seed) + events := rand.Intn(maxEvents-minEvents) + minEvents + batchSize := rand.Intn(events-8) + 4 + + t.Log("seed: ", seed) + t.Log("events: ", events) + t.Log("batchSize: ", batchSize) + + testWith := func(factory queuetest.QueueFactory) func(t *testing.T) { + return func(test *testing.T) { + t.Run("single", func(t *testing.T) { + queuetest.TestSingleProducerConsumer(t, events, batchSize, factory) + }) + t.Run("multi", func(t *testing.T) { + queuetest.TestMultiProducerConsumer(t, events, batchSize, factory) + }) + } + } + + testWith(makeTestQueue( + 128*humanize.KiByte, 4*humanize.KiByte, 16*humanize.KiByte, + 100*time.Millisecond, + ))(t) +} + +func makeTestQueue( + maxSize, pageSize, writeBuffer uint, + flushTimeout time.Duration, +) func(*testing.T) queue.Queue { + return func(t *testing.T) queue.Queue { + if debug { + fmt.Println("Test:", t.Name()) + } + + ok := false + path, cleanPath := txfiletest.SetupPath(t, "") + defer func() { + if !ok { + cleanPath() + } + }() + + spool, err := NewSpool(&testLogger{t}, path, Settings{ + WriteBuffer: writeBuffer, + WriteFlushTimeout: flushTimeout, + Codec: codecCBORL, + File: txfile.Options{ + MaxSize: uint64(maxSize), + PageSize: uint32(pageSize), + Prealloc: true, + Readonly: false, + }, + }) + if err != nil { + t.Fatal(err) + } + + tq := &testQueue{Spool: spool, teardown: cleanPath} + return tq + } +} + +func (t *testQueue) Close() error { + err := t.Spool.Close() + t.teardown() + return err +} + +func (l *testLogger) Debug(vs ...interface{}) { l.report("Debug", vs) } +func (l *testLogger) Debugf(fmt string, vs ...interface{}) { l.reportf("Debug: ", fmt, vs) } + +func (l *testLogger) Info(vs ...interface{}) { l.report("Info", vs) } +func (l *testLogger) Infof(fmt string, vs ...interface{}) { l.reportf("Info", fmt, vs) } + +func (l *testLogger) Error(vs ...interface{}) { l.report("Error", vs) } +func (l *testLogger) Errorf(fmt string, vs ...interface{}) { l.reportf("Error", fmt, vs) } + +func (l *testLogger) report(level string, vs []interface{}) { + args := append([]interface{}{level, ":"}, vs...) + l.t.Log(args...) + if debug { + fmt.Println(args...) + } +} + +func (l *testLogger) reportf(level string, str string, vs []interface{}) { + str = level + ": " + str + l.t.Logf(str, vs...) + if debug { + fmt.Printf(str, vs...) + } +} diff --git a/libbeat/publisher/queue/spool/timer.go b/libbeat/publisher/queue/spool/timer.go new file mode 100644 index 00000000000..8c4650b5961 --- /dev/null +++ b/libbeat/publisher/queue/spool/timer.go @@ -0,0 +1,55 @@ +package spool + +import ( + "time" +) + +type timer struct { + // flush timer + timer *time.Timer + C <-chan time.Time + duration time.Duration +} + +func newTimer(duration time.Duration) *timer { + stdtimer := time.NewTimer(duration) + if !stdtimer.Stop() { + <-stdtimer.C + } + + return &timer{ + timer: stdtimer, + C: nil, + duration: duration, + } +} + +func (t *timer) Zero() bool { + return t.duration == 0 +} + +func (t *timer) Restart() { + t.Stop(false) + t.Start() +} + +func (t *timer) Start() { + if t.C != nil { + return + } + + t.timer.Reset(t.duration) + t.C = t.timer.C +} + +func (t *timer) Stop(triggered bool) { + if t.C == nil { + return + } + + if !triggered && !t.timer.Stop() { + <-t.C + } + + t.C = nil +} diff --git a/libbeat/scripts/Makefile b/libbeat/scripts/Makefile index 3e174d23cff..4e83dbad8e0 100755 --- a/libbeat/scripts/Makefile +++ b/libbeat/scripts/Makefile @@ -52,6 +52,7 @@ NOSETESTS_OPTIONS?=--process-timeout=$(TIMEOUT) --with-timer -v --with-xunit --x TEST_ENVIRONMENT?=false ## @testing if true, "make testsuite" runs integration tests and system tests in a dockerized test environment SYSTEM_TESTS?=false ## @testing if true, "make test" and "make testsuite" run unit tests and system tests STRESS_TESTS?=false ## @testing if true, "make test" and "make testsuite" run also run the stress tests +STRESS_TEST_OPTIONS?=-timeout=20m -race -v GOX_OS?=linux darwin windows freebsd netbsd openbsd ## @Building List of all OS to be supported by "make crosscompile". GOX_OSARCH?=!darwin/arm !darwin/arm64 !darwin/386 ## @building Space separated list of GOOS/GOARCH pairs to build by "make crosscompile". GOX_FLAGS?= ## @building Additional flags to append to the gox command used by "make crosscompile". @@ -206,7 +207,7 @@ fast-system-tests: ${BEAT_NAME}.test python-env stress-tests: ## @testing Runs the stress tests with race detector enabled stress-tests: if [ -n '${GOPACKAGES_STRESSTESTS}' ]; then \ - go test -race --tags=stresstest -v ${GOPACKAGES_STRESSTESTS}; \ + go test --tags=stresstest ${STRESS_TEST_OPTIONS} ${GOPACKAGES_STRESSTESTS}; \ fi # Run benchmark tests diff --git a/libbeat/scripts/cmd/stress_pipeline/main.go b/libbeat/scripts/cmd/stress_pipeline/main.go index 79e35fbb544..fb65c3273df 100644 --- a/libbeat/scripts/cmd/stress_pipeline/main.go +++ b/libbeat/scripts/cmd/stress_pipeline/main.go @@ -2,6 +2,7 @@ package main import ( "flag" + "fmt" "log" "net/http" _ "net/http/pprof" @@ -9,13 +10,14 @@ import ( "github.com/elastic/beats/libbeat/beat" "github.com/elastic/beats/libbeat/common" - "github.com/elastic/beats/libbeat/logp" + logpcfg "github.com/elastic/beats/libbeat/logp/configure" "github.com/elastic/beats/libbeat/paths" "github.com/elastic/beats/libbeat/publisher/pipeline/stress" "github.com/elastic/beats/libbeat/service" // import queue types _ "github.com/elastic/beats/libbeat/publisher/queue/memqueue" + _ "github.com/elastic/beats/libbeat/publisher/queue/spool" // import outputs _ "github.com/elastic/beats/libbeat/outputs/console" @@ -31,7 +33,7 @@ var ( type config struct { Path paths.Path - Logging logp.Logging + Logging *common.Config } func main() { @@ -52,6 +54,8 @@ func run() error { flag.Parse() files := flag.Args() + fmt.Println("load config files:", files) + cfg, err := common.LoadFiles(files...) if err != nil { return err @@ -72,10 +76,11 @@ func run() error { if err := paths.InitPaths(&config.Path); err != nil { return err } - if err = logp.Init("test", time.Now(), &config.Logging); err != nil { + if err = logpcfg.Logging("test", config.Logging); err != nil { return err } - logp.SetStderr() + + cfg.PrintDebugf("input config:") return stress.RunTests(info, duration, cfg, nil) } diff --git a/metricbeat/metricbeat.reference.yml b/metricbeat/metricbeat.reference.yml index b90f3875cf5..91ea630f71d 100644 --- a/metricbeat/metricbeat.reference.yml +++ b/metricbeat/metricbeat.reference.yml @@ -603,6 +603,66 @@ metricbeat.modules: # if the number of events stored in the queue is < min_flush_events. #flush.timeout: 1s + # The spool queue will store events in a local spool file, before + # forwarding the events to the outputs. + # + # Beta: spooling to disk is currently a beta feature. Use with care. + # + # The spool file is a circular buffer, which blocks once the file/buffer is full. + # Events are put into a write buffer and flushed once the write buffer + # is full or the flush_timeout is triggered. + # Once ACKed by the output, events are removed immediately from the queue, + # making space for new events to be persisted. + #spool: + # The file namespace configures the file path and the file creation settings. + # Once the file exists, the `size`, `page_size` and `prealloc` settings + # will have no more effect. + #file: + # Location of spool file. The default value is ${path.data}/spool.dat. + #path: "${path.data}/spool.dat" + + # Configure file permissions if file is created. The default value is 0600. + #permissions: 0600 + + # File size hint. The spool blocks, once this limit is reached. The default value is 100 MiB. + #size: 100MiB + + # The files page size. A file is split into multiple pages of the same size. The default value is 4KiB. + #page_size: 4KiB + + # If prealloc is set, the required space for the file is reserved using + # truncate. The default value is true. + #prealloc: true + + # Spool writer settings + # Events are serialized into a write buffer. The write buffer is flushed if: + # - The buffer limit has been reached. + # - The configured limit of buffered events is reached. + # - The flush timeout is triggered. + #write: + # Sets the write buffer size. + #buffer_size: 1MiB + + # Maximum duration after which events are flushed, if the write buffer + # is not full yet. The default value is 1s. + #flush.timeout: 1s + + # Number of maximum buffered events. The write buffer is flushed once the + # limit is reached. + #flush.events: 16384 + + # Configure the on-disk event encoding. The encoding can be changed + # between restarts. + # Valid encodings are: json, ubjson, and cbor. + #codec: cbor + #read: + # Reader flush timeout, waiting for more events to become available, so + # to fill a complete batch, as required by the outputs. + # If flush_timeout is 0, all available events are forwarded to the + # outputs immediately. + # The default value is 0s. + #flush.timeout: 0s + # Sets the maximum number of CPUs that can be executing simultaneously. The # default is the number of logical CPUs available in the system. #max_procs: diff --git a/packetbeat/packetbeat.reference.yml b/packetbeat/packetbeat.reference.yml index 349432834eb..02793c47626 100644 --- a/packetbeat/packetbeat.reference.yml +++ b/packetbeat/packetbeat.reference.yml @@ -511,6 +511,66 @@ packetbeat.protocols: # if the number of events stored in the queue is < min_flush_events. #flush.timeout: 1s + # The spool queue will store events in a local spool file, before + # forwarding the events to the outputs. + # + # Beta: spooling to disk is currently a beta feature. Use with care. + # + # The spool file is a circular buffer, which blocks once the file/buffer is full. + # Events are put into a write buffer and flushed once the write buffer + # is full or the flush_timeout is triggered. + # Once ACKed by the output, events are removed immediately from the queue, + # making space for new events to be persisted. + #spool: + # The file namespace configures the file path and the file creation settings. + # Once the file exists, the `size`, `page_size` and `prealloc` settings + # will have no more effect. + #file: + # Location of spool file. The default value is ${path.data}/spool.dat. + #path: "${path.data}/spool.dat" + + # Configure file permissions if file is created. The default value is 0600. + #permissions: 0600 + + # File size hint. The spool blocks, once this limit is reached. The default value is 100 MiB. + #size: 100MiB + + # The files page size. A file is split into multiple pages of the same size. The default value is 4KiB. + #page_size: 4KiB + + # If prealloc is set, the required space for the file is reserved using + # truncate. The default value is true. + #prealloc: true + + # Spool writer settings + # Events are serialized into a write buffer. The write buffer is flushed if: + # - The buffer limit has been reached. + # - The configured limit of buffered events is reached. + # - The flush timeout is triggered. + #write: + # Sets the write buffer size. + #buffer_size: 1MiB + + # Maximum duration after which events are flushed, if the write buffer + # is not full yet. The default value is 1s. + #flush.timeout: 1s + + # Number of maximum buffered events. The write buffer is flushed once the + # limit is reached. + #flush.events: 16384 + + # Configure the on-disk event encoding. The encoding can be changed + # between restarts. + # Valid encodings are: json, ubjson, and cbor. + #codec: cbor + #read: + # Reader flush timeout, waiting for more events to become available, so + # to fill a complete batch, as required by the outputs. + # If flush_timeout is 0, all available events are forwarded to the + # outputs immediately. + # The default value is 0s. + #flush.timeout: 0s + # Sets the maximum number of CPUs that can be executing simultaneously. The # default is the number of logical CPUs available in the system. #max_procs: diff --git a/vendor/github.com/elastic/go-txfile/LICENSE b/vendor/github.com/elastic/go-txfile/LICENSE new file mode 100644 index 00000000000..261eeb9e9f8 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/vendor/github.com/elastic/go-txfile/README.md b/vendor/github.com/elastic/go-txfile/README.md new file mode 100644 index 00000000000..94a6a4b6d15 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/README.md @@ -0,0 +1 @@ +# txfile \ No newline at end of file diff --git a/vendor/github.com/elastic/go-txfile/alloc.go b/vendor/github.com/elastic/go-txfile/alloc.go new file mode 100644 index 00000000000..eb74fa189eb --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/alloc.go @@ -0,0 +1,765 @@ +package txfile + +import ( + "math" + + "github.com/elastic/go-txfile/internal/invariant" +) + +// file global allocator state +type ( + + // allocator manages the on-disk page allocation. Pages in the allocator can + // be either part of the meta-area or data-area. Users allocate pages from + // the data-area only. The meta-area keeps pages available for in file + // meta-data like overwrite pages and freelists. The meta-area allocates + // pages from the data-area, if required. The meta-area grows by always doubling + // the amount of pages in the meta-area. + // For allocations one must get an instance to the dataAllocator, + // walAllocator or metaAllocator respectively. Each allocator provides + // slightly different allocation strategies. + // The walAllocator is used for contents overwrite pages, while the + // metaAllocator is used to allocate pages for for serializing the overwrite + // mapping and freelispages for for serializing the overwrite mapping and + // freelist. + allocator struct { + // configuration + maxPages uint + maxSize uint + pageSize uint + + // meta area + meta allocArea + metaTotal uint // total number of pages reserved by meta area + + // data area + data allocArea + + // allocator file metadata + freelistRoot PageID + freelistPages regionList // page ids used to store the free list + } + + allocArea struct { + endMarker PageID + freelist freelist + } + + // custom allocator implementations, sharing the global allocator state + dataAllocator allocator // allocate from data area + walAllocator allocator // allocate WAL overwrite pages from beginning of meta area + metaAllocator allocator // allocate meta pages from end of meta area + + // metaManager manages the data and meta regions, by moving regions + // between those areas. The manager is used by walAllocator and metaAllocator + // only. + metaManager allocator +) + +//transaction local allocation state +type ( + // txAllocState is used by write transactions, to record changes to the file + // allocation state. The file global allocator state is modified within the + // write transaction. txAllocState acts as undo/redo-log for the in-memory + // allocation state. + // Page frees are only recorded within the transaction. No pages are returned + // to the allocator, so to ensure a page freed can not be allocated. This + // guarantees freed pages can not be overwritten in the current transaction + // (keep most recent transaction intact). + txAllocState struct { + manager txAreaManageState + data txAllocArea + meta txAllocArea + options txAllocOptions // per transaction allocation options + } + + txAllocArea struct { + endMarker PageID + allocated pageSet // allocated pages from freelist + new pageSet // allocated pages from end of file + freed pageSet // set of pages freed within transaction + } + + txAreaManageState struct { + moveToMeta regionList // list regions moved from data area to meta area + } + + // txAllocOptions keeps track of user options passed to the transaction. + txAllocOptions struct { + overflowAreaEnabled bool // enable allocating pages with ID > maxPages for metadata + metaGrowPercentage int // limit of meta area in use, so to allocate new pages into the meta area + } +) + +// allocCommitState keeps track of the new allocator state during the commit. +// These changes must be recorded for now, as the final allocator state must +// not be updated in memory until after the transaction has been commited to +// the file. +type allocCommitState struct { + tx *txAllocState + updated bool // set if updates to allocator within current transaction + allocRegions regionList // meta pages allocated to write new freelist too + metaList regionList // new meta area freelist + dataList regionList // new data area freelist + overflowFreed uint // number of pages in overflow region to be returned +} + +// noLimit indicates the data/meta-area can grow without any limits. +const noLimit uint = maxUint + +const defaultMetaGrowPercentage = 80 + +// allocator +// --------- + +func (a *allocator) DataAllocator() *dataAllocator { return (*dataAllocator)(a) } +func (a *allocator) WALPageAllocator() *walAllocator { return (*walAllocator)(a) } +func (a *allocator) MetaAllocator() *metaAllocator { return (*metaAllocator)(a) } +func (a *allocator) metaManager() *metaManager { return (*metaManager)(a) } + +func (a *allocator) makeTxAllocState(withOverflow bool, growPercentage int) txAllocState { + if growPercentage <= 0 { + growPercentage = defaultMetaGrowPercentage + } + + return txAllocState{ + data: txAllocArea{ + endMarker: a.data.endMarker, + }, + meta: txAllocArea{ + endMarker: a.meta.endMarker, + }, + options: txAllocOptions{ + overflowAreaEnabled: withOverflow, + metaGrowPercentage: growPercentage, + }, + } +} + +func (a *allocator) fileCommitPrepare(st *allocCommitState, tx *txAllocState) { + st.tx = tx + st.updated = tx.Updated() + if st.updated { + a.MetaAllocator().FreeRegions(tx, a.freelistPages) + } +} + +func (a *allocator) fileCommitAlloc(st *allocCommitState) error { + if !st.updated { + return nil + } + + dataFreed := st.tx.data.freed.Regions() + metaFreed := st.tx.meta.freed.Regions() + + // Predict number of meta pages required to store new freelist, + // by iterating all region entries and take the potential encoding size + // into account. As allocation might force a region from the data area + // being moved (or split) into the meta area, we add more dummy region + // with enforced max size. So the allocator can move pages between + // meta and data if required. + // This method over-estimates the number of required pages, as + // we will have to allocate pages from the metaFree lists end + // after the estimator finishes. + prediction := prepareFreelistEncPagePrediction(freePageHeaderSize, a.pageSize) + prediction.AddRegions(dataFreed) + prediction.AddRegions(metaFreed) + prediction.AddRegions(a.data.freelist.regions) + prediction.AddRegions(a.meta.freelist.regions) + if prediction.count > 0 { + // only add extra pages if we need to write the meta page + prediction.AddRegion(region{id: 1, count: math.MaxUint32}) + prediction.AddRegion(region{id: 1, count: math.MaxUint32}) + } + + // alloc regions for writing the new freelist + var allocRegions regionList + if n := prediction.count; n > 0 { + allocRegions = a.MetaAllocator().AllocRegions(st.tx, n) + if allocRegions == nil { + return errOutOfMemory + } + } + + // Compute new freelist. As consecutive regions are merged the + // resulting list might require less pages + newDataList := mergeRegionLists(a.data.freelist.regions, dataFreed) + newMetaList := mergeRegionLists(a.meta.freelist.regions, metaFreed) + + st.allocRegions = allocRegions + st.dataList = newDataList + + // remove pages from end of overflow area from meta freelist + adjust end marker + st.metaList, st.overflowFreed = releaseOverflowPages(newMetaList, a.maxPages, a.meta.endMarker) + return nil +} + +// releaseOverflowPages removes pages at the end of a region list as long as +// the current end marker is bigger then the maximum number of allowed pages +// and the freelist contains some continuous regions up to endMarker. +func releaseOverflowPages( + list regionList, + maxPages uint, endMarker PageID, +) (regionList, uint) { + overflowStart, overflowEnd := PageID(maxPages), endMarker + if maxPages == 0 || overflowStart >= overflowEnd { + return list, 0 + } + + var freed uint + for i := len(list) - 1; i != -1; i-- { + start, end := list[i].Range() + if end < overflowEnd { + break + } + + if start < overflowStart { + // split + list[i].count = uint32(overflowStart - start) + freed += uint(end - overflowStart) + overflowEnd = overflowStart + } else { + // remove range + overflowEnd = start + freed += uint(list[i].count) + list = list[:i] + } + } + + return list, freed +} + +func (a *allocator) fileCommitSerialize( + st *allocCommitState, + onPage func(id PageID, buf []byte) error, +) error { + if !st.updated || len(st.allocRegions) == 0 { + return nil + } + return writeFreeLists(st.allocRegions, a.pageSize, st.metaList, st.dataList, onPage) +} + +func (a *allocator) fileCommitMeta(meta *metaPage, st *allocCommitState) { + if st.updated { + var freelistRoot PageID + if len(st.allocRegions) > 0 { + freelistRoot = st.allocRegions[0].id + } + meta.freelist.Set(freelistRoot) + + dataEndMarker := a.data.endMarker + metaEndMarker := a.meta.endMarker + if st.overflowFreed > 0 { + metaEndMarker -= PageID(st.overflowFreed) + if metaEndMarker > dataEndMarker { + dataEndMarker = metaEndMarker + } + } + + meta.dataEndMarker.Set(dataEndMarker) + meta.metaEndMarker.Set(metaEndMarker) + meta.metaTotal.Set(uint64(a.metaTotal - st.overflowFreed)) + } +} + +func (a *allocator) Commit(st *allocCommitState) { + if st.updated { + a.freelistPages = st.allocRegions + if len(st.allocRegions) > 0 { + a.freelistRoot = st.allocRegions[0].id + } else { + a.freelistRoot = 0 + } + + a.data.commit(st.dataList) + a.meta.commit(st.metaList) + a.metaTotal -= st.overflowFreed + } +} + +func (a *allocator) Rollback(st *txAllocState) { + // restore meta area + a.meta.rollback(&st.meta) + for _, reg := range st.manager.moveToMeta { + a.meta.freelist.RemoveRegion(reg) + a.metaTotal -= uint(reg.count) + + if reg.id < st.data.endMarker { + reg.EachPage(st.data.allocated.Add) + } + } + + // restore data area + a.data.rollback(&st.data) +} + +func (a *allocArea) commit(regions regionList) { + a.freelist.regions = regions + a.freelist.avail = regions.CountPages() +} + +func (a *allocArea) rollback(st *txAllocArea) { + for id := range st.allocated { + if id >= st.endMarker { + delete(st.allocated, id) + } + } + a.freelist.AddRegions(st.allocated.Regions()) + a.endMarker = st.endMarker +} + +// metaManager +// ----------- + +func (mm *metaManager) DataAllocator() *dataAllocator { + return (*dataAllocator)(mm) +} + +func (mm *metaManager) Avail(st *txAllocState) uint { + dataAvail := mm.DataAllocator().Avail(st) + if dataAvail == noLimit || st.options.overflowAreaEnabled { + return noLimit + } + + return mm.meta.freelist.Avail() + dataAvail +} + +func (mm *metaManager) Ensure(st *txAllocState, n uint) bool { + total := mm.metaTotal + avail := mm.meta.freelist.Avail() + used := total - avail + targetUsed := used + n + + invariant.Check(total >= avail, "invalid meta total page count") + + tracef("ensure(%v): total=%v, avail=%v, used=%v, targetUsed=%v\n", + n, total, avail, used, targetUsed) + + pctGrow := st.options.metaGrowPercentage + pctShrink := pctGrow / 2 + + szMinMeta, szMaxMeta := metaAreaTargetQuota(total, targetUsed, pctShrink, pctGrow) + traceln(" target quota: ", szMinMeta, szMaxMeta) + + invariant.Check(szMaxMeta >= szMinMeta, "required page count must grow") + + if szMaxMeta == total { + // we still have enough memory in the meta area -> return success + + // TODO: allow 'ensure' to shrink the meta area + return true + } + + invariant.Check(szMaxMeta > total, "expected new page count exceeding allocated pages") + + // try to move regions from data area into the meta area: + requiredMax := szMaxMeta - total + if mm.tryGrow(st, requiredMax, false) { + return true + } + + // Can not grow until 'requiredMax' -> try to grow up to requiredMin, + // potentially allocating pages from the overflow area + requiredMin := szMinMeta - total + if mm.tryGrow(st, requiredMin, st.options.overflowAreaEnabled) { + return true + } + + // out of memory + return false +} + +func (mm *metaManager) tryGrow( + st *txAllocState, + count uint, + withOverflow bool, +) bool { + da := mm.DataAllocator() + avail := da.Avail(st) + + tracef("try grow meta area pages=%v, avail=%v\n", count, avail) + + if count == 0 { + return true + } + + if avail < count { + if !withOverflow { + traceln("can not grow meta area yet") + return false + } + + da.AllocRegionsWith(st, avail, func(reg region) { + st.manager.moveToMeta.Add(reg) + mm.metaTotal += uint(reg.count) + mm.meta.freelist.AddRegion(reg) + }) + + // allocate from overflow area + required := count - avail + if required > 0 { + traceln("try to grow overflow area") + } + allocFromArea(&st.meta, &mm.meta.endMarker, required, func(reg region) { + // st.manager.fromOverflow.Add(reg) + mm.metaTotal += uint(reg.count) + mm.meta.freelist.AddRegion(reg) + }) + if mm.maxPages == 0 && mm.data.endMarker < mm.meta.endMarker { + mm.data.endMarker = mm.meta.endMarker + } + + return true + } + + // Enough memory available in data area. Try to allocate continuous region first + reg := da.AllocContinuousRegion(st, count) + if reg.id != 0 { + st.manager.moveToMeta.Add(reg) + mm.metaTotal += uint(reg.count) + mm.meta.freelist.AddRegion(reg) + return true + } + + // no continuous memory block -> allocate single regions + n := da.AllocRegionsWith(st, count, func(reg region) { + st.manager.moveToMeta.Add(reg) + mm.metaTotal += uint(reg.count) + mm.meta.freelist.AddRegion(reg) + }) + return n == count +} + +func (mm *metaManager) Free(st *txAllocState, id PageID) { + // mark page as freed for now + st.meta.freed.Add(id) +} + +func metaAreaTargetQuota( + total, used uint, + shrinkPercentage, growPercentage int, +) (min, max uint) { + min = used + max = uint(nextPowerOf2(uint64(used))) + if max < total { + max = total + } + + usage := 100 * float64(used) / float64(max) + + // grow 'max' by next power of 2, if used area would exceed growPercentage + needsGrow := usage > float64(growPercentage) + + // If memory is to be freed (max < total), still grow 'max' by next power of + // 2 (so not to free too much memory at once), if used area in new meta area + // would exceed shrinkPercentage. + // => percentage of used area in new meta area will be shrinkPercentage/2 + needsGrow = needsGrow || (max < total && usage > float64(shrinkPercentage)) + + if min < total { + min = total + } + + if needsGrow { + max = max * 2 + } + return min, max +} + +// dataAllocator +// ------------- + +func (a *dataAllocator) Avail(_ *txAllocState) uint { + if a.maxPages == 0 { + return noLimit + } + return a.maxPages - uint(a.data.endMarker) + a.data.freelist.Avail() +} + +func (a *dataAllocator) AllocContinuousRegion( + st *txAllocState, + n uint, +) region { + avail := a.Avail(st) + if avail < n { + return region{} + } + + reg := allocContFromFreelist(&a.data.freelist, &st.data, allocFromBeginning, n) + if reg.id != 0 { + return reg + } + + avail = a.maxPages - uint(a.data.endMarker) + if avail < n { + // out of memory + return region{} + } + + allocFromArea(&st.data, &a.data.endMarker, n, func(r region) { reg = r }) + if a.meta.endMarker < a.data.endMarker { + a.meta.endMarker = a.data.endMarker + } + return reg +} + +func (a *dataAllocator) AllocRegionsWith( + st *txAllocState, + n uint, + fn func(region), +) uint { + avail := a.Avail(st) + if avail < n { + return 0 + } + + // Enough space available -> allocate all pages. + count := n + + // 1. allocate subset of regions from freelist + n -= allocFromFreelist(&a.data.freelist, &st.data, allocFromBeginning, n, fn) + if n > 0 { + // 2. allocate from yet unused data area + allocFromArea(&st.data, &a.data.endMarker, n, fn) + if a.meta.endMarker < a.data.endMarker { + a.meta.endMarker = a.data.endMarker + } + } + return count +} + +func (a *dataAllocator) Free(st *txAllocState, id PageID) { + traceln("free page:", id) + + if id < 2 || id >= a.data.endMarker { + panic(errOutOfBounds) + } + + if !st.data.new.Has(id) { + // fast-path, page has not been allocated in current transaction + st.data.freed.Add(id) + return + } + + // page has been allocated in current transaction -> return to allocator for immediate re-use + a.data.freelist.AddRegion(region{id: id, count: 1}) + + if st.data.endMarker >= id { + // allocation from within old data region + return + } + + // allocation was from past the old end-marker. Check if we can shrink the + // end marker again + regions := a.data.freelist.regions + last := len(regions) - 1 + start, end := regions[last].Range() + if end < a.data.endMarker { + // in middle of new data region -> can not adjust end marker -> keep update to freelist + return + } + + if st.data.endMarker > start { + start = st.data.endMarker + count := uint(end - start) + regions[last].count -= uint32(count) + a.data.freelist.avail -= count + } else { + a.data.freelist.avail -= uint(regions[last].count) + a.data.freelist.regions = regions[:last] + } + a.data.endMarker = start +} + +// walAllocator +// ------------ + +func (a *walAllocator) metaManager() *metaManager { return (*metaManager)(a) } + +func (a *walAllocator) Avail(st *txAllocState) uint { + return a.metaManager().Avail(st) +} + +func (a *walAllocator) Alloc(st *txAllocState) PageID { + mm := a.metaManager() + if !mm.Ensure(st, 1) { + return 0 + } + + // Use AllocContinuousRegion to find smallest fitting region + // to allocate from. + reg := a.meta.freelist.AllocContinuousRegion(allocFromBeginning, 1) + if reg.id == 0 { + return 0 + } + st.meta.allocated.Add(reg.id) + return reg.id +} + +func (a *walAllocator) AllocRegionsWith(st *txAllocState, n uint, fn func(region)) uint { + mm := a.metaManager() + if !mm.Ensure(st, n) { + return 0 + } + + return allocFromFreelist(&a.meta.freelist, &st.meta, allocFromBeginning, n, fn) +} + +func (a *walAllocator) Free(st *txAllocState, id PageID) { + a.metaManager().Free(st, id) +} + +// metaAllocator +// ------------ + +func (a *metaAllocator) metaManager() *metaManager { return (*metaManager)(a) } + +func (a *metaAllocator) Avail(st *txAllocState) uint { + return a.metaManager().Avail(st) +} + +func (a *metaAllocator) AllocRegionsWith( + st *txAllocState, + n uint, + fn func(region), +) uint { + mm := a.metaManager() + if !mm.Ensure(st, n) { + return 0 + } + + return allocFromFreelist(&a.meta.freelist, &st.meta, allocFromEnd, n, fn) +} + +func (a *metaAllocator) AllocRegions(st *txAllocState, n uint) regionList { + reg := make(regionList, 0, n) + if n := a.AllocRegionsWith(st, n, reg.Add); n == 0 { + return nil + } + return reg +} + +func (a *metaAllocator) Free(st *txAllocState, id PageID) { + a.metaManager().Free(st, id) +} + +func (a *metaAllocator) FreeAll(st *txAllocState, ids idList) { + for _, id := range ids { + a.Free(st, id) + } +} + +func (a *metaAllocator) FreeRegions(st *txAllocState, regions regionList) { + regions.EachPage(func(id PageID) { + a.Free(st, id) + }) +} + +// tx allocation state methods +// --------------------------- + +func (s *txAllocState) Updated() bool { + return s.meta.Updated() || s.data.Updated() +} + +func (s *txAllocArea) Updated() bool { + return !s.allocated.Empty() || !s.new.Empty() || !s.freed.Empty() +} + +// allocator state (de-)serialization +// ---------------------------------- + +func readAllocatorState(a *allocator, f *File, meta *metaPage, opts Options) error { + if a.maxSize > 0 { + a.maxPages = a.maxSize / a.pageSize + } + + a.data.endMarker = meta.dataEndMarker.Get() + a.meta.endMarker = meta.metaEndMarker.Get() + a.metaTotal = uint(meta.metaTotal.Get()) + + a.freelistRoot = meta.freelist.Get() + if a.freelistRoot == 0 { + return nil + } + + var metaList, dataList freelist + ids, err := readFreeList(f.mmapedPage, a.freelistRoot, func(isMeta bool, region region) { + lst := &dataList + if isMeta { + lst = &metaList + } + + lst.avail += uint(region.count) + lst.regions.Add(region) + }) + if err != nil { + return err + } + + dataList.regions.Sort() + dataList.regions.MergeAdjacent() + metaList.regions.Sort() + metaList.regions.MergeAdjacent() + + a.data.freelist = dataList + a.meta.freelist = metaList + a.freelistPages = ids.Regions() + return nil +} + +// allocator helpers/utilities +// --------------------------- + +// allocFromFreelist allocates up to 'max' pages from the free list. +// The number of allocated pages is returned +func allocFromFreelist( + f *freelist, + area *txAllocArea, + order *allocOrder, + max uint, + fn func(region), +) uint { + count := max + if f.avail < count { + count = f.avail + } + + f.AllocRegionsWith(order, count, func(region region) { + region.EachPage(area.allocated.Add) + fn(region) + }) + return count +} + +func allocContFromFreelist( + f *freelist, + area *txAllocArea, + order *allocOrder, + n uint, +) region { + region := f.AllocContinuousRegion(order, n) + if region.id != 0 { + region.EachPage(area.new.Add) + } + return region +} + +func allocFromArea(area *txAllocArea, marker *PageID, count uint, fn func(region)) { + // region can be max 2<<32 -> allocate in loop + id := *marker + for count > 0 { + n := count + if n > math.MaxUint32 { + n = math.MaxUint32 + } + + region := region{id: id, count: uint32(n)} + region.EachPage(area.new.Add) + fn(region) + + id += PageID(n) + count -= n + } + *marker = id +} diff --git a/vendor/github.com/elastic/go-txfile/errors.go b/vendor/github.com/elastic/go-txfile/errors.go new file mode 100644 index 00000000000..49ced4a7c1b --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/errors.go @@ -0,0 +1,32 @@ +package txfile + +import "errors" + +var ( + // file meta page validation errors + + errMagic = errors.New("invalid magic number") + errVersion = errors.New("invalid version number") + errChecksum = errors.New("checksum mismatch") + + // file sizing errors + + errMmapTooLarge = errors.New("mmap too large") + errFileSizeTooLage = errors.New("max file size to large for this system") + errInvalidFileSize = errors.New("invalid file size") + + // page access/allocation errors + + errOutOfBounds = errors.New("out of bounds page id") + errOutOfMemory = errors.New("out of memory") + errFreedPage = errors.New("trying to access an already freed page") + errPageFlushed = errors.New("page is already flushed") + errTooManyBytes = errors.New("contents exceeds page size") + errNoPageData = errors.New("accessing page without contents") + errFreeDirtyPage = errors.New("freeing dirty page") + + // transaction errors + + errTxFinished = errors.New("transaction has already been closed") + errTxReadonly = errors.New("readonly transaction") +) diff --git a/vendor/github.com/elastic/go-txfile/file.go b/vendor/github.com/elastic/go-txfile/file.go new file mode 100644 index 00000000000..4c18696dfcc --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/file.go @@ -0,0 +1,478 @@ +package txfile + +import ( + "fmt" + "math" + "math/bits" + "os" + "sync" + "unsafe" + + "github.com/elastic/go-txfile/internal/cleanup" + "github.com/elastic/go-txfile/internal/invariant" +) + +// File provides transactional support to pages of a file. A file is split into +// pages of type PageSize. Pages within the file are only accessible by page IDs +// from within active transactions. +type File struct { + path string + file vfsFile + locks lock + wg sync.WaitGroup // local async workers wait group + writer writer + allocator allocator + wal waLog + + // mmap info + mapped []byte + + // meta pages + meta [2]*metaPage + metaActive int +} + +// Options provides common file options used when opening or creating a file. +type Options struct { + // MaxSize sets the maximum file size in bytes. This should be a multiple of PageSize. + // If it's not a multiple of PageSize, the actual files maximum size is rounded downwards + // to the next multiple of PageSize. + // A value of 0 indicates the file can grow without limits. + MaxSize uint64 + + // PageSize sets the files page size on file creation. PageSize is ignored if + // the file already exists. + // If PageSize is not configured, the OSes main memory page size is selected. + PageSize uint32 + + // Prealloc disk space if MaxSize is set. + Prealloc bool + + // Open file in readonly mode. + Readonly bool +} + +// Open opens or creates a new transactional file. +// Open tries to create the file, if the file does not exist yet. Returns an +// error if file access fails, file can not be locked or file meta pages are +// found to be invalid. +func Open(path string, mode os.FileMode, opts Options) (*File, error) { + file, err := openOSFile(path, mode) + if err != nil { + return nil, err + } + + initOK := false + defer cleanup.IfNot(&initOK, cleanup.IgnoreError(file.Close)) + + // Create exclusive lock on the file and initialize the file state. + var f *File + if err = file.Lock(true, true); err == nil { + // initialize the file + f, err = openWith(file, opts) + } + if err != nil { + return nil, err + } + + initOK = true + + tracef("open file: %p (%v)\n", f, path) + traceMetaPage(f.getMetaPage()) + return f, nil +} + +// openWith implements the actual opening sequence, including file +// initialization and validation. +func openWith(file vfsFile, opts Options) (*File, error) { + sz, err := file.Size() + if err != nil { + return nil, err + } + + fileExists := sz > 0 + if !fileExists { + if err := initNewFile(file, opts); err != nil { + return nil, err + } + } + + meta, err := readValidMeta(file) + if err != nil { + return nil, err + } + + pageSize := meta.pageSize.Get() + maxSize := meta.maxSize.Get() + if maxSize == 0 && opts.MaxSize > 0 { + maxSize = opts.MaxSize + } + + if maxSize > uint64(maxUint) { + return nil, errFileSizeTooLage + } + + return newFile(file, opts, uint(maxSize), uint(pageSize)) +} + +// newFile creates and initializes a new File. File state is initialized +// from file and internal workers will be started. +func newFile(file vfsFile, opts Options, maxSize, pageSize uint) (*File, error) { + + f := &File{ + file: file, + path: file.Name(), + allocator: allocator{ + maxSize: maxSize, + pageSize: pageSize, + }, + } + f.locks.init() + + if err := f.mmap(); err != nil { + return nil, err + } + initOK := false + defer cleanup.IfNot(&initOK, cleanup.IgnoreError(f.munmap)) + + if err := f.init(opts); err != nil { + return nil, err + } + + invariant.CheckNot(f.allocator.maxSize != 0 && f.allocator.maxPages == 0, + "page limit not configured on allocator") + + // create asynchronous writer + f.writer.Init(file, f.allocator.pageSize) + f.wg.Add(1) + go func() { + defer f.wg.Done() + f.writer.Run() + }() + + initOK = true + return f, nil +} + +// init initializes the File state from most recent valid meta-page. +func (f *File) init(opts Options) error { + // validate meta pages and set active meta page id + var metaErr [2]error + metaErr[0] = f.meta[0].Validate() + metaErr[1] = f.meta[1].Validate() + switch { + case metaErr[0] != nil && metaErr[1] != nil: + return metaErr[0] + case metaErr[0] == nil && metaErr[1] != nil: + f.metaActive = 1 + case metaErr[0] != nil && metaErr[1] == nil: + f.metaActive = 1 + default: + // both meta pages valid, choose page with highest transaction number + tx0 := f.meta[0].txid.Get() + tx1 := f.meta[1].txid.Get() + if tx0 == tx1 { + panic("meta pages with same transaction id") + } + + if int64(tx0-tx1) > 0 { // if tx0 > tx1 + f.metaActive = 0 + } else { + f.metaActive = 1 + } + } + + // reference active meta page for initializing internal structures + meta := f.meta[f.metaActive] + + if err := readWALMapping(&f.wal, f.mmapedPage, meta.wal.Get()); err != nil { + return err + } + + return readAllocatorState(&f.allocator, f, meta, opts) +} + +// Close closes the file, after all transactions have been quit. After closing +// a file, no more transactions can be started. +func (f *File) Close() error { + // zero out f on exit -> using f after close should generate a panic + defer func() { *f = File{} }() + + tracef("start file shutdown: %p\n", f) + defer tracef("file closed: %p\n", f) + + // get reserved lock, such that no write transactions can be started + f.locks.Reserved().Lock() + defer f.locks.Reserved().Unlock() + + // get pending lock, such that no new read transaction can be started + f.locks.Pending().Lock() + defer f.locks.Pending().Unlock() + + // get exclusive lock, waiting for active read transactions to be finished + f.locks.Exclusive().Lock() + defer f.locks.Exclusive().Unlock() + + // no other active transactions -> close file + f.munmap() + f.writer.Stop() + + err := f.file.Close() + + // wait for workers to stop + f.wg.Wait() + + return err +} + +// Begin creates a new read-write transaction. The transaction returned +// does hold the Reserved Lock on the file. Use Close, Rollback, or Commit to +// release the lock. +func (f *File) Begin() *Tx { + return f.BeginWith(TxOptions{Readonly: false}) +} + +// BeginReadonly creates a new readonly transaction. The transaction returned +// does hold the Shared Lock on the file. Use Close() to release the lock. +func (f *File) BeginReadonly() *Tx { + return f.BeginWith(TxOptions{Readonly: true}) +} + +// BeginWith creates a new readonly or read-write transaction, with additional +// transaction settings. +func (f *File) BeginWith(settings TxOptions) *Tx { + tracef("request new transaction (readonly: %v)\n", settings.Readonly) + lock := f.locks.TxLock(settings.Readonly) + lock.Lock() + tracef("init new transaction (readonly: %v)\n", settings.Readonly) + tx := newTx(f, lock, settings) + tracef("begin transaction: %p (readonly: %v)\n", tx, settings.Readonly) + return tx +} + +// PageSize returns the files page size in bytes +func (f *File) PageSize() int { + return int(f.allocator.pageSize) +} + +// Offset computes a file offset from PageID and offset within the current +// page. +func (f *File) Offset(id PageID, offset uintptr) uintptr { + sz := uintptr(f.allocator.pageSize) + if offset >= sz { + panic("offset not within page boundary") + } + return offset + uintptr(id)*uintptr(f.allocator.pageSize) +} + +// SplitOffset splits a file offset into a page ID for accessing the page and +// and offset within the page. +func (f *File) SplitOffset(offset uintptr) (PageID, uintptr) { + sz := uintptr(f.allocator.pageSize) + id := PageID(offset / sz) + off := offset - ((offset / sz) * sz) + return id, off +} + +// mmapUpdate updates the mmaped states. +// A go-routine updating the mmaped aread, must hold all locks on the file. +func (f *File) mmapUpdate() (err error) { + if err = f.munmap(); err == nil { + err = f.mmap() + } + return +} + +// mmap maps the files contents and updates internal pointers into the mmaped memory area. +func (f *File) mmap() error { + fileSize, err := f.file.Size() + if err != nil { + return err + } + + if fileSize < 0 { + return errInvalidFileSize + } + + maxSize := f.allocator.maxSize + if em := uint(f.allocator.meta.endMarker); maxSize > 0 && em > f.allocator.maxPages { + maxSize = em * f.allocator.pageSize + } + pageSize := f.allocator.pageSize + sz, err := computeMmapSize(uint(fileSize), maxSize, uint(pageSize)) + if err != nil { + return err + } + + // map file + buf, err := f.file.MMap(int(sz)) + if err != nil { + return err + } + + f.mapped = buf + f.meta[0] = castMetaPage(buf[0:]) + f.meta[1] = castMetaPage(buf[pageSize:]) + + return nil +} + +// munmap unmaps the file and sets internal mapping to nil. +func (f *File) munmap() error { + err := f.file.MUnmap(f.mapped) + f.mapped = nil + return err +} + +// mmapedPage finds the mmaped page contents by the given pageID. +// The byte buffer can only be used for reading. +func (f *File) mmapedPage(id PageID) []byte { + pageSize := uint64(f.allocator.pageSize) + start := uint64(id) * pageSize + end := start + pageSize + if uint64(len(f.mapped)) < end { + return nil + } + + return f.mapped[start:end] +} + +// initNewFile initializes a new, yet empty Files metapages. +func initNewFile(file vfsFile, opts Options) error { + var flags uint32 + if opts.MaxSize > 0 && opts.Prealloc { + flags |= metaFlagPrealloc + if err := file.Truncate(int64(opts.MaxSize)); err != nil { + return fmt.Errorf("truncation failed with %v", err) + } + } + + pageSize := opts.PageSize + if opts.PageSize == 0 { + pageSize = uint32(os.Getpagesize()) + if pageSize < minPageSize { + pageSize = minPageSize + } + } + if !isPowerOf2(uint64(pageSize)) { + return fmt.Errorf("pageSize %v is no power of 2", pageSize) + } + if pageSize < minPageSize { + return fmt.Errorf("pageSize must be > %v", minPageSize) + } + + // create buffer to hold contents for the four initial pages: + // 1. meta page 0 + // 2. meta page 1 + // 3. free list page + buf := make([]byte, pageSize*3) + + // write meta pages + for i := 0; i < 2; i++ { + pg := castMetaPage(buf[int(pageSize)*i:]) + pg.Init(flags, pageSize, opts.MaxSize) + pg.txid.Set(uint64(1 - i)) + pg.dataEndMarker.Set(2) // endMarker is index of next to be allocated page at end of file + pg.Finalize() + } + + // write initial pages to disk + err := writeAt(file, buf, 0) + if err == nil { + err = file.Sync() + } + + if err != nil { + return fmt.Errorf("initializing data file failed with %v", err) + } + return nil +} + +// readValidMeta tries to read a valid meta page from the file. +// The first valid meta page encountered is returned. +func readValidMeta(f vfsFile) (metaPage, error) { + meta, err := readMeta(f, 0) + if err != nil { + return meta, err + } + + if err := meta.Validate(); err != nil { + // try next metapage + offset := meta.pageSize.Get() + if meta, err = readMeta(f, int64(offset)); err != nil { + return meta, err + } + return meta, meta.Validate() + } + return meta, nil +} + +func readMeta(f vfsFile, off int64) (metaPage, error) { + var buf [unsafe.Sizeof(metaPage{})]byte + _, err := f.ReadAt(buf[:], off) + return *castMetaPage(buf[:]), err +} + +// computeMmapSize determines the page count in multiple of pages. +// Up to 1GB, the mmaped file area is double (starting at 64KB) on every grows. +// That is, exponential grows with values of 64KB, 128KB, 512KB, 1024KB, and so on. +// Once 1GB is reached, the mmaped area is always a multiple of 1GB. +func computeMmapSize(minSize, maxSize, pageSize uint) (uint, error) { + const ( + initBits uint = 16 // 2 ^ 16 Bytes + initSize = 1 << initBits // 64KB + sz1GB = 1 << 30 + doubleLimit = sz1GB // upper limit when to stop doubling the mmaped area + ) + + var maxMapSize uint + if math.MaxUint32 == maxUint { + maxMapSize = 2 * sz1GB + } else { + tmp := uint64(0x1FFFFFFFFFFF) + maxMapSize = uint(tmp) + } + + if maxSize != 0 { + // return maxSize as multiple of pages. Round downwards in case maxSize + // is not multiple of pages + + if minSize > maxSize { + maxSize = minSize + } + + sz := ((maxSize + pageSize - 1) / pageSize) * pageSize + if sz < initSize { + return 0, fmt.Errorf("max size of %v bytes is too small", maxSize) + } + + return sz, nil + } + + if minSize < doubleLimit { + // grow by next power of 2, starting at 64KB + initBits := uint(16) // 64KB min + power2Bits := uint(64 - bits.LeadingZeros64(uint64(minSize))) + if power2Bits < initBits { + power2Bits = initBits + } + return 1 << power2Bits, nil + } + + // allocate number of 1GB blocks to fulfill minSize + sz := ((minSize + (sz1GB - 1)) / sz1GB) * sz1GB + if sz > maxMapSize { + return 0, errMmapTooLarge + } + + // ensure we have a multiple of pageSize + sz = ((sz + pageSize - 1) / pageSize) * pageSize + + return sz, nil +} + +// getMetaPage returns a pointer to the meta-page of the last valid transaction +// found. +func (f *File) getMetaPage() *metaPage { + return f.meta[f.metaActive] +} diff --git a/vendor/github.com/elastic/go-txfile/freelist.go b/vendor/github.com/elastic/go-txfile/freelist.go new file mode 100644 index 00000000000..e6628127287 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/freelist.go @@ -0,0 +1,457 @@ +package txfile + +import ( + "math" + "sort" + + "github.com/elastic/go-txfile/internal/invariant" + "github.com/elastic/go-txfile/internal/iter" +) + +// freelist manages freed pages within an area. The freelist uses +// run-length-encoding, compining multiple pages into one region, so to reduce +// memory usage in memory, as well as when serializing the freelist. The +// freelist guarantees pages are sorted by PageID. Depending on allocOrder, +// pages with smallest/biggest PageID will be allocated first. +type freelist struct { + avail uint + regions regionList +} + +// freelistEncPagePrediction is used to predict the number of meta-pages required +// to serialize the freelist. +// The prediction might over-estimate the number of pages required, which is +// perfectly fine, as long as we don't under-estimate (which would break +// serialization -> transaction fail). +type freelistEncPagePrediction struct { + count uint + payloadSize, avail uint +} + +// allocOrder provides freelist access strategies. +type allocOrder struct { + // freelist iteration order + iter iter.Fn + + // reportRange provides iteration limits for reporting allocated regions in + // order (by PageID). + reportRange func(last, len int) (int, int) + + // allocFromRegion split the region into allocated and leftover region. + allocFromRegion func(reg region, N uint32) (region, region) + + // keepRange determines which pages to keep/remove from the freelist, after + // allocation. + keepRange func(last, len int, partial bool) (int, int) +} + +var ( + allocFromBeginning = &allocOrder{ + iter: iter.Forward, + reportRange: func(last, len int) (int, int) { + return 0, last + 1 + }, + allocFromRegion: func(reg region, N uint32) (region, region) { + return region{id: reg.id, count: N}, region{id: reg.id + PageID(N), count: reg.count - N} + }, + keepRange: func(last, len int, partial bool) (int, int) { + if partial { + return last, len + } + return last + 1, len + }, + } + + allocFromEnd = &allocOrder{ + iter: iter.Reversed, + reportRange: func(last, len int) (int, int) { + return last, len + }, + allocFromRegion: func(reg region, N uint32) (region, region) { + return region{id: reg.id + PageID(reg.count-N), count: N}, region{id: reg.id, count: reg.count - N} + }, + keepRange: func(last, len int, partial bool) (int, int) { + if partial { + return 0, last + 1 + } + return 0, last + }, + } +) + +// Avail returns number of pages available in the current freelist. +func (f *freelist) Avail() uint { + return f.avail +} + +// AllocAllRegionsWith allocates all regions in the freelist. +// The list will be empty afterwards. +func (f *freelist) AllocAllRegionsWith(fn func(region)) { + for _, r := range f.AllocAllRegions() { + fn(r) + } +} + +// AllocAllRegions allocates all regions in the freelist. +// The list will be empty afterwards. +func (f *freelist) AllocAllRegions() regionList { + regions := f.regions + f.avail = 0 + f.regions = nil + return regions +} + +// AllocContinuousRegion tries to find a contiuous set of pages in the freelist. +// The best-fitting (or smallest) region having at least n pages will be used +// for allocation. +// Returns an empty region, if no continuous space could be found. +func (f *freelist) AllocContinuousRegion(order *allocOrder, n uint) region { + if f.avail < n || (f.avail == n && len(f.regions) > 1) { + return region{} + } + + if n > math.MaxUint32 { // continuous regions max out at 4GB + return region{} + } + + bestFit := -1 + bestSz := uint(math.MaxUint32) + for i, end, next := order.iter(len(f.regions)); i != end; i = next(i) { + count := uint(f.regions[i].count) + if n <= count && count < bestSz { + bestFit = i + bestSz = count + if bestSz == n { + break + } + } + } + + if bestFit < 0 { + // no continuous region found + return region{} + } + + // allocate best fitting region from list + i := bestFit + selected := f.regions[i] + allocated, rest := order.allocFromRegion(selected, uint32(n)) + + invariant.Check(allocated.count == uint32(n), "allocation mismatch") + invariant.Check(allocated.count+rest.count == selected.count, "region split page count mismatch") + + if rest.count == 0 { + // remove entry + copy(f.regions[i:], f.regions[i+1:]) + f.regions = f.regions[:len(f.regions)-1] + } else { + f.regions[i] = rest + } + + f.avail -= uint(allocated.count) + return allocated +} + +// AllocRegionsWith allocates up n potentially non-continuous pages from the +// freelist. No page will be allocated, if n succeeds the number of available +// pages. +func (f *freelist) AllocRegionsWith(order *allocOrder, n uint, fn func(region)) { + if n == 0 { + return + } + + var ( + last int // last region to allocate from + L = len(f.regions) + N = n // number of pages to be allocated from 'last' region + ) + + if N > f.avail { + // not enough space -> return early + return + } + + // Collect indices of regions to be allocated from. + for i, end, next := order.iter(L); i != end; i = next(i) { + count := uint(f.regions[i].count) + if count >= N { + last = i + break + } + N -= count + } + + // Compute region split on last region to be allocated from. + selected := f.regions[last] + allocated, leftover := order.allocFromRegion(selected, uint32(N)) + + invariant.Check(allocated.count == uint32(N), "allocation mismatch") + invariant.Check(allocated.count+leftover.count == selected.count, "region split page count mismatch") + + // Implicitely update last allocated region to match the allocation size + // and report all regions allocated. + f.regions[last] = allocated + for i, end := order.reportRange(last, L); i != end; i++ { + fn(f.regions[i]) + } + + // update free regions + f.regions[last] = leftover + start, end := order.keepRange(last, L, leftover.count != 0) + f.regions = f.regions[start:end] + f.avail -= n +} + +// AddRegions merges a new list of regions with the freelist. The regions +// in the list must be sorted. +func (f *freelist) AddRegions(list regionList) { + count := list.CountPages() + if count > 0 { + f.regions = mergeRegionLists(f.regions, list) + f.avail += count + } +} + +// AddRegion inserts a new region into the freelist. AddRegion ensures the new +// region is sorted within the freelist, potentially merging the new region +// with existing regions. +// Note: The region to be added MUST NOT overlap with existing regions. +func (f *freelist) AddRegion(reg region) { + if len(f.regions) == 0 { + f.regions = regionList{reg} + f.avail += uint(reg.count) + return + } + + i := sort.Search(len(f.regions), func(i int) bool { + _, end := f.regions[i].Range() + return reg.id < end + }) + + total := uint(reg.count) + switch { + case len(f.regions) <= i: // add to end of region list? + last := len(f.regions) - 1 + if regionsMergable(f.regions[last], reg) { + f.regions[last] = mergeRegions(f.regions[last], reg) + } else { + f.regions.Add(reg) + } + case i == 0: // add to start of region list? + if regionsMergable(reg, f.regions[0]) { + f.regions[0] = mergeRegions(reg, f.regions[0]) + } else { + f.regions = append(f.regions, region{}) + copy(f.regions[1:], f.regions) + f.regions[0] = reg + } + default: // insert in middle of region list + // try to merge region with already existing regions + mergeBefore := regionsMergable(f.regions[i-1], reg) + if mergeBefore { + reg = mergeRegions(f.regions[i-1], reg) + } + mergeAfter := regionsMergable(reg, f.regions[i]) + if mergeAfter { + reg = mergeRegions(reg, f.regions[i]) + } + + // update region list + switch { + case mergeBefore && mergeAfter: // combine adjacent regions -> shrink list + f.regions[i-1] = reg + copy(f.regions[i:], f.regions[i+1:]) + f.regions = f.regions[:len(f.regions)-1] + case mergeBefore: + f.regions[i-1] = reg + case mergeAfter: + f.regions[i] = reg + default: // no adjacent entries -> grow list + f.regions = append(f.regions, region{}) + copy(f.regions[i+1:], f.regions[i:]) + f.regions[i] = reg + } + } + + f.avail += total +} + +// RemoveRegion removes all pages from the freelist, that are found within +// the input region. +func (f *freelist) RemoveRegion(removed region) { + i := sort.Search(len(f.regions), func(i int) bool { + _, end := f.regions[i].Range() + return removed.id <= end + }) + if i < 0 || i >= len(f.regions) { + return + } + + current := &f.regions[i] + if current.id == removed.id && current.count == removed.count { + // fast path: entry can be completely removed + f.regions = append(f.regions[:i], f.regions[i+1:]...) + f.avail -= uint(removed.count) + return + } + + var total uint + removedStart, removedEnd := removed.Range() + for removedStart < removedEnd && i < len(f.regions) { + current := &f.regions[i] + + if removedStart < current.id { + // Gap: advance removedStart, so to deal with holes when removing the all regions + // matching the input region + removedStart = current.id + continue + } + + count := uint32(removedEnd - removedStart) + if removedStart == current.id { + if current.count < count { + count = current.count + } + + // remove entry: + current.id = current.id + PageID(count) + current.count -= count + if current.count == 0 { + // remove region from freelist -> i will point to next region if + // `removed` overlaps 2 non-merged regions + f.regions = append(f.regions[:i], f.regions[i+1:]...) + } else { + // overlapping region, but old region must be preserved: + i++ + } + + removedStart += PageID(count) + total += uint(count) + } else { + // split current region in removedStart + keep := uint32(removedStart - current.id) + leftover := region{ + id: removedStart, + count: current.count - keep, + } + current.count = keep + + // remove sub-region from leftover + if leftover.count < count { + count = leftover.count + } + leftover.id += PageID(count) + leftover.count -= count + + total += uint(count) + removedStart += PageID(count) + i++ // advance to next region + + // insert new entry into regionList if removed did remove region in + // middle of old region + if leftover.count > 0 { + f.regions = append(f.regions, region{}) + copy(f.regions[i+1:], f.regions[i:]) + f.regions[i] = leftover + break // no more region to split from + } + } + } + + f.avail -= total +} + +// (de-)serialization + +func readFreeList( + access func(PageID) []byte, + root PageID, + fn func(bool, region), +) (idList, error) { + if root == 0 { + return nil, nil + } + + rootPage := access(root) + if rootPage == nil { + return nil, errOutOfBounds + } + + var metaPages idList + for pageID := root; pageID != 0; { + metaPages.Add(pageID) + node, payload := castFreePage(access(pageID)) + if node == nil { + return nil, errOutOfBounds + } + + pageID = node.next.Get() + entries := node.count.Get() + tracef("free list node: (next: %v, entries: %v)", pageID, entries) + + for ; entries > 0; entries-- { + isMeta, reg, n := decodeRegion(payload) + payload = payload[n:] + fn(isMeta, reg) + } + } + + return metaPages, nil +} + +func writeFreeLists( + to regionList, + pageSize uint, + metaList, dataList regionList, + onPage func(id PageID, buf []byte) error, +) error { + allocPages := to.PageIDs() + writer := newPagingWriter(allocPages, pageSize, 0, onPage) + + var writeErr error + writeList := func(isMeta bool, lst regionList) { + if writeErr != nil { + return + } + + for _, reg := range lst { + var buf [maxRegionEncSz]byte + n := encodeRegion(buf[:], isMeta, reg) + if err := writer.Write(buf[:n]); err != nil { + writeErr = err + return + } + } + } + + writeList(true, metaList) + writeList(false, dataList) + if writeErr != nil { + return writeErr + } + + return writer.Flush() +} + +func prepareFreelistEncPagePrediction(header int, pageSize uint) freelistEncPagePrediction { + return freelistEncPagePrediction{payloadSize: pageSize - uint(header)} +} + +func (f *freelistEncPagePrediction) Estimate() uint { + return f.count +} + +func (f *freelistEncPagePrediction) AddRegion(reg region) { + sz := uint(regionEncodingSize(reg)) + if f.avail < sz { + f.count++ + f.avail = f.payloadSize + } + f.avail -= sz +} + +func (f *freelistEncPagePrediction) AddRegions(lst regionList) { + for _, reg := range lst { + f.AddRegion(reg) + } +} diff --git a/vendor/github.com/elastic/go-txfile/idlist.go b/vendor/github.com/elastic/go-txfile/idlist.go new file mode 100644 index 00000000000..af18dbead1d --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/idlist.go @@ -0,0 +1,41 @@ +package txfile + +import "sort" + +type idList []PageID + +func (l *idList) Add(id PageID) { + *l = append(*l, id) +} + +func (l idList) ToSet() pageSet { + L := len(l) + if L == 0 { + return nil + } + + s := make(pageSet, L) + for _, id := range l { + s.Add(id) + } + return s +} + +func (l idList) Sort() { + sort.Slice(l, func(i, j int) bool { + return l[i] < l[j] + }) +} + +func (l idList) Regions() regionList { + if len(l) == 0 { + return nil + } + + regions := make(regionList, len(l)) + for i, id := range l { + regions[i] = region{id: id, count: 1} + } + optimizeRegionList(®ions) + return regions +} diff --git a/vendor/github.com/elastic/go-txfile/internal/cleanup/cleanup.go b/vendor/github.com/elastic/go-txfile/internal/cleanup/cleanup.go new file mode 100644 index 00000000000..7cea6ca0683 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/internal/cleanup/cleanup.go @@ -0,0 +1,58 @@ +// Package cleanup provides common helpers for common cleanup patterns on defer +// +// Use the helpers with `defer`. For example use IfNot with `defer`, such that +// cleanup functions will be executed if `check` is false, no matter if an +// error has been returned or an panic has occured. +// +// initOK := false +// defer cleanup.IfNot(&initOK, func() { +// cleanup +// }) +// +// ... // init structures... +// +// initOK = true // notify handler cleanup code must not be executed +// +package cleanup + +// If will run the cleanup function if the bool value is true. +func If(check *bool, cleanup func()) { + if *check { + cleanup() + } +} + +// IfNot will run the cleanup function if the bool value is false. +func IfNot(check *bool, cleanup func()) { + if !(*check) { + cleanup() + } +} + +// IfPred will run the cleanup function if pred returns true. +func IfPred(pred func() bool, cleanup func()) { + if pred() { + cleanup() + } +} + +// IfNotPred will run the cleanup function if pred returns false. +func IfNotPred(pred func() bool, cleanup func()) { + if !pred() { + cleanup() + } +} + +// WithError returns a cleanup function calling a custom handler if an error occured. +func WithError(fn func(error), cleanup func() error) func() { + return func() { + if err := cleanup(); err != nil { + fn(err) + } + } +} + +// IgnoreError silently ignores errors in the cleanup function. +func IgnoreError(cleanup func() error) func() { + return func() { _ = cleanup() } +} diff --git a/vendor/github.com/elastic/go-txfile/internal/cleanup/multi.go b/vendor/github.com/elastic/go-txfile/internal/cleanup/multi.go new file mode 100644 index 00000000000..280010b9c66 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/internal/cleanup/multi.go @@ -0,0 +1,29 @@ +package cleanup + +// FailClean keeps track of functions to be executed of FailClean did +// not receive a success signal. +type FailClean struct { + success bool + fns []func() +} + +// Signal sends a success or fail signal to FailClean. +func (f *FailClean) Signal(success bool) { + f.success = success +} + +// Add adds another cleanup handler. The last added handler will be run first. +func (f *FailClean) Add(fn func()) { + f.fns = append(f.fns, fn) +} + +// Cleanup runs all cleanup handlers in reverse order. +func (f *FailClean) Cleanup() { + if f.success { + return + } + + for i := len(f.fns) - 1; i >= 0; i-- { + f.fns[i]() + } +} diff --git a/vendor/github.com/elastic/go-txfile/internal/invariant/invariant.go b/vendor/github.com/elastic/go-txfile/internal/invariant/invariant.go new file mode 100644 index 00000000000..fed16395c4a --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/internal/invariant/invariant.go @@ -0,0 +1,54 @@ +// Package invariant provides helpers for checking and panicing on faulty invariants. +package invariant + +import "fmt" + +// Check will raise an error with the provided message in case b is false. +func Check(b bool, msg string) { + if b { + return + } + + if msg == "" { + panic("failing invariant") + } + panic(msg) +} + +// Checkf will raise an error in case b is false. Checkf accept a fmt.Sprintf +// compatible format string with parameters. +func Checkf(b bool, msgAndArgs ...interface{}) { + if b { + return + } + + switch len(msgAndArgs) { + case 0: + panic("failing invariant") + case 1: + panic(msgAndArgs[0].(string)) + default: + panic(fmt.Sprintf(msgAndArgs[0].(string), msgAndArgs[1:]...)) + } +} + +// CheckNot will raise an error with the provided message in case b is true. +func CheckNot(b bool, msg string) { + Check(!b, msg) +} + +// CheckNotf will raise an error with the provided message in case b is true. +// CheckNotf accept a fmt.Sprintf compatible format string with parameters. +func CheckNotf(b bool, msgAndArgs ...interface{}) { + Checkf(!b, msgAndArgs...) +} + +// Unreachable marks some code sequence that must never be executed. +func Unreachable(msg string) { + panic(msg) +} + +// Unreachablef marks some code sequence that must never be executed. +func Unreachablef(f string, vs ...interface{}) { + panic(fmt.Sprintf(f, vs...)) +} diff --git a/vendor/github.com/elastic/go-txfile/internal/iter/iter.go b/vendor/github.com/elastic/go-txfile/internal/iter/iter.go new file mode 100644 index 00000000000..0c54a211701 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/internal/iter/iter.go @@ -0,0 +1,15 @@ +// Package iter provides functions for common array iteration strategies. +package iter + +// Fn type for range based iterators. +type Fn func(len int) (begin, end int, next func(int) int) + +// Forward returns limits and next function for forward iteration. +func Forward(l int) (begin, end int, next func(int) int) { + return 0, l, func(i int) int { return i + 1 } +} + +// Reversed returns limits and next function for reverse iteration. +func Reversed(l int) (begin, end int, next func(int) int) { + return l - 1, -1, func(i int) int { return i - 1 } +} diff --git a/vendor/github.com/elastic/go-txfile/internal/tracelog/tracelog.go b/vendor/github.com/elastic/go-txfile/internal/tracelog/tracelog.go new file mode 100644 index 00000000000..c023cf05afb --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/internal/tracelog/tracelog.go @@ -0,0 +1,44 @@ +package tracelog + +import ( + "fmt" + "os" + "strings" +) + +type Logger interface { + Println(...interface{}) + Printf(string, ...interface{}) +} + +type stderrLogger struct{} + +type nilLogger struct{} + +func Get(selector string) Logger { + if isEnabled(selector) { + return (*stderrLogger)(nil) + } + return (*nilLogger)(nil) +} + +func isEnabled(selector string) bool { + v := os.Getenv("TRACE_SELECTOR") + if v == "" { + return true + } + + selectors := strings.Split(v, ",") + for _, sel := range selectors { + if selector == strings.TrimSpace(sel) { + return true + } + } + return false +} + +func (*nilLogger) Println(...interface{}) {} +func (*nilLogger) Printf(string, ...interface{}) {} + +func (*stderrLogger) Println(vs ...interface{}) { fmt.Fprintln(os.Stderr, vs...) } +func (*stderrLogger) Printf(s string, vs ...interface{}) { fmt.Fprintf(os.Stderr, s, vs...) } diff --git a/vendor/github.com/elastic/go-txfile/layout.go b/vendor/github.com/elastic/go-txfile/layout.go new file mode 100644 index 00000000000..7b02b95e022 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/layout.go @@ -0,0 +1,184 @@ +package txfile + +import ( + "fmt" + "hash/fnv" + "reflect" + "unsafe" + + bin "github.com/urso/go-bin" +) + +// on disk page layout for writing and parsing + +// primitive types: +type ( + u8 = bin.U8le + u16 = bin.U16le + u32 = bin.U32le + u64 = bin.U64le + i8 = bin.I8le + i16 = bin.I16le + i32 = bin.I32le + i64 = bin.I64le + + pgID u64 +) + +// Special page at beginning of file. +// A file holds to meta pages at the beginning of the file. A metaPage is +// updated after a write transaction has been completed. On error during +// transactions or when updating the metaPage, the old metaPage will still be +// valid, technically ignoring all contents written by the transactions active +// while the program/id did crash/fail. +type metaPage struct { + magic u32 + version u32 + pageSize u32 + maxSize u64 // maximum file size + flags u32 + root pgID // ID of first page to look for data. + txid u64 // page transaction ID + freelist pgID // pointer to user area freelist + wal pgID // write-ahead-log root + dataEndMarker pgID // end marker of user-area page + metaEndMarker pgID // file end marker + metaTotal u64 // total number of pages in meta area + checksum u32 +} + +type metaBuf [unsafe.Sizeof(metaPage{})]byte + +const ( + metaFlagPrealloc = 1 << 0 // indicates the complete file has been preallocated +) + +type listPage struct { + next pgID // pointer to next entry + count u32 // number of entries in current page +} + +type freePage = listPage +type walPage = listPage + +const ( + metaPageHeaderSize = int(unsafe.Sizeof(metaPage{})) + listPageHeaderSize = int(unsafe.Sizeof(listPage{})) + walPageHeaderSize = int(unsafe.Sizeof(walPage{})) + freePageHeaderSize = int(unsafe.Sizeof(freePage{})) +) + +const magic uint32 = 0xBEA77AEB +const version uint32 = 1 + +func init() { + checkPacked := func(t reflect.Type) error { + off := uintptr(0) + for i := 0; i < t.NumField(); i++ { + f := t.Field(i) + if f.Offset != off { + return fmt.Errorf("field %v offset mismatch (expected=%v, actual=%v)", + f.Name, off, f.Offset) + } + off += f.Type.Size() + } + return nil + } + + // check compiler really generates packed structes. Required, so file can be + // accesed from within different architectures + checksum based on raw bytes + // contents are correct. + checkPacked(reflect.TypeOf(metaPage{})) + checkPacked(reflect.TypeOf(freePage{})) + checkPacked(reflect.TypeOf(walPage{})) +} + +func castMetaPage(b []byte) (p *metaPage) { castPageTo(&p, b); return } + +func (m *metaPage) Init(flags uint32, pageSize uint32, maxSize uint64) { + m.magic.Set(magic) + m.version.Set(version) + m.pageSize.Set(pageSize) + m.maxSize.Set(maxSize) + m.flags.Set(flags) + m.root.Set(0) + m.freelist.Set(0) + m.wal.Set(0) + m.dataEndMarker.Set(0) +} + +func (m *metaPage) Finalize() { + m.checksum.Set(m.computeChecksum()) +} + +func (m *metaPage) Validate() error { + if m.magic.Get() != magic { + return errMagic + } + if m.version.Get() != version { + return errVersion + } + if m.checksum.Get() != m.computeChecksum() { + return errChecksum + } + + return nil +} + +func (b *metaBuf) cast() *metaPage { return castMetaPage((*b)[:]) } + +func (m *metaPage) computeChecksum() uint32 { + h := fnv.New32a() + type metaHashContent [unsafe.Offsetof(metaPage{}.checksum)]byte + contents := *(*metaHashContent)(unsafe.Pointer(m)) + _, _ = h.Write(contents[:]) + return h.Sum32() +} + +func (id *pgID) Len() int { return id.access().Len() } +func (id *pgID) Get() PageID { return PageID(id.access().Get()) } +func (id *pgID) Set(v PageID) { id.access().Set(uint64(v)) } +func (id *pgID) access() *u64 { return (*u64)(id) } + +func castU8(b []byte) (u *u8) { mapMem(&u, b); return } +func castU16(b []byte) (u *u16) { mapMem(&u, b); return } +func castU32(b []byte) (u *u32) { mapMem(&u, b); return } +func castU64(b []byte) (u *u64) { mapMem(&u, b); return } + +func castListPage(b []byte) (node *listPage, data []byte) { + if castPageTo(&node, b); node != nil { + data = b[unsafe.Sizeof(listPage{}):] + } + return +} + +func castFreePage(b []byte) (node *freePage, data []byte) { + return castListPage(b) +} + +func castWalPage(b []byte) (node *walPage, data []byte) { + return castListPage(b) +} + +func mapMem(to interface{}, b []byte) { + bin.UnsafeCastStruct(to, b) +} + +func castPageTo(to interface{}, b []byte) { + mapMem(to, b) +} + +func traceMetaPage(meta *metaPage) { + traceln("meta page:") + traceln(" version:", meta.version.Get()) + traceln(" pagesize:", meta.pageSize.Get()) + traceln(" maxsize:", meta.maxSize.Get()) + traceln(" root:", meta.root.Get()) + traceln(" txid:", meta.txid.Get()) + traceln(" freelist:", meta.freelist.Get()) + traceln(" wal:", meta.wal.Get()) + traceln(" data end:", meta.dataEndMarker.Get()) + traceln(" meta end:", meta.metaEndMarker.Get()) + traceln(" meta total:", meta.metaTotal.Get()) + traceln(" checksum:", meta.checksum.Get()) +} diff --git a/vendor/github.com/elastic/go-txfile/lock.go b/vendor/github.com/elastic/go-txfile/lock.go new file mode 100644 index 00000000000..b54cf2eec17 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/lock.go @@ -0,0 +1,119 @@ +package txfile + +import "sync" + +// lock provides the file locking primitives for use within the current +// process. File locking as provided by lock, is not aware of other processes +// accessing the file. +// +// Lock types: +// - Shared: Shared locks are used by readonly transactions. Multiple readonly +// transactions can co-exist with one active write transaction. +// - Reserved: Write transactions take the 'Reserved' lock on a file, +// such that no other concurrent write transaction can exist. +// The Shared lock can still be locked by concurrent readers. +// - Pending: The Pending lock is used by write transactions to signal +// a write transaction is currently being committed. +// The Shared lock can still be used by readonly transactions, +// but no new readonly transaction can be started after +// the Pending lock has been acquired. +// - Exclusive: Once the exclusive lock is acquired by a write transaction, +// No other active transactions/locks exist on the file. +// +// Each Locktype can be accessed using `(*lock).()`. Each lock type +// implements a `Lock` and `Unlock` method. +// +// Note: Shared file access should be protected using `flock`. +type lock struct { + mu sync.Mutex + + // conditions + mutexes + shared *sync.Cond + exclusive *sync.Cond + reserved sync.Mutex + + // state + sharedCount uint + pendingSet bool +} + +type sharedLock lock +type reservedLock lock +type pendingLock lock +type exclusiveLock lock + +func newLock() *lock { + l := &lock{} + l.init() + return l +} + +func (l *lock) init() { + l.shared = sync.NewCond(&l.mu) + l.exclusive = sync.NewCond(&l.mu) +} + +// TxLock returns the standard Locker for the given transaction type. +func (l *lock) TxLock(readonly bool) sync.Locker { + if readonly { + return l.Shared() + } + return l.Reserved() +} + +// Shared returns the files shared locker. +func (l *lock) Shared() *sharedLock { return (*sharedLock)(l) } + +// Reserved returns the files reserved locker. +func (l *lock) Reserved() *reservedLock { return (*reservedLock)(l) } + +// Pending returns the files pending locker. +func (l *lock) Pending() *pendingLock { return (*pendingLock)(l) } + +// Pending returns the files exclusive locker. +func (l *lock) Exclusive() *exclusiveLock { return (*exclusiveLock)(l) } + +func (l *sharedLock) Lock() { waitCond(l.shared, l.check, l.inc) } +func (l *sharedLock) Unlock() { withLocker(&l.mu, l.dec) } +func (l *sharedLock) check() bool { return !l.pendingSet } +func (l *sharedLock) inc() { l.sharedCount++ } +func (l *sharedLock) dec() { + l.sharedCount-- + if l.sharedCount == 0 { + l.exclusive.Signal() + } +} + +func (l *reservedLock) Lock() { l.reserved.Lock() } +func (l *reservedLock) Unlock() { l.reserved.Unlock() } + +func (l *pendingLock) Lock() { + l.mu.Lock() + l.pendingSet = true + l.mu.Unlock() +} +func (l *pendingLock) Unlock() { + l.mu.Lock() + l.pendingSet = false + l.mu.Unlock() + l.shared.Broadcast() +} + +func (l *exclusiveLock) Lock() { waitCond(l.exclusive, l.check, func() {}) } +func (l *exclusiveLock) Unlock() {} +func (l *exclusiveLock) check() bool { return l.sharedCount == 0 } + +func waitCond(c *sync.Cond, check func() bool, upd func()) { + withLocker(c.L, func() { + for !check() { + c.Wait() + } + upd() + }) +} + +func withLocker(l sync.Locker, fn func()) { + l.Lock() + defer l.Unlock() + fn() +} diff --git a/vendor/github.com/elastic/go-txfile/meta_sizing.py b/vendor/github.com/elastic/go-txfile/meta_sizing.py new file mode 100644 index 00000000000..4f561d3cbc4 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/meta_sizing.py @@ -0,0 +1,97 @@ +#!/usr/bin/env python + +import argparse +import pprint + + +list_header = 8 + 4 # next pointer + page entry count + + +def main(): + parser = argparse.ArgumentParser() + parser.add_argument('-p', '--pagesize', dest='pagesize', type=long, default=4096) + parser.add_argument('-s', '--maxsize', dest='maxsize', type=long, default=1 << 30) + parser.add_argument('-w', '--wal', dest='wal', type=long, default=1000) + args = parser.parse_args() + + stats = compute_stats(args.pagesize, args.maxsize, args.wal) + pprint.pprint(stats, indent=2) + + +def compute_stats(page_size, max_size, wal_entries): + # multiply by 2, as next transaction might require same amount + # of pages + + max_pages = max_size / page_size + + stats = { + "pagesize": page_size, + "max_size": max_size, + "max_pages": max_pages, + "wal_entries": wal_entries, + } + + wal_meta = wal_mapping_pages(page_size, wal_entries) + stats['wal_meta'] = 2 * wal_meta + stats['wal_meta_bytes'] = 2 * wal_meta * page_size + stats['wal_meta_bytes_io_per_tx'] = wal_meta * page_size + + freelist = freelist_pages(page_size, max_pages) + stats['freelist_pages'] = 2 * freelist + stats['freelist_bytes'] = 2 * freelist * page_size + stats['freelist_bytes_io_per_tx'] = freelist * page_size + + file_header = 2 + stats['file header'] = file_header + + count = wal_meta + wal_entries + 2 * freelist + file_header + stats['min_meta_pages'] = count + + # meta allocator grows in power of 2 + meta_pages = next_power_of_2(count) + internal_frag = meta_pages - count + data_pages = max_pages - meta_pages + + stats['meta_pages'] = meta_pages + stats['data_pages'] = data_pages + stats['meta_bytes'] = meta_pages * page_size + stats['data_bytes'] = data_pages * page_size + stats['internal_fragmentation'] = internal_frag + stats['meta_percentage'] = 100.0 * float(meta_pages) / float(max_pages) + stats['data_percentage'] = 100.0 * float(data_pages) / float(max_pages) + stats['frag_percentage'] = 100.0 * float(internal_frag) / float(max_pages) + + return stats + + +def pages(entries, entries_per_page): + return (entries + (entries_per_page - 1)) / entries_per_page + + +def freelist_pages(page_size, max_pages): + """Compute max number of freelist pages required. + Assumes full fragmentation, such that every second page is free. + Due to run-length-encoding of freelist entries, this assumption gets us + the max number of freelist entries.""" + + # estimate of max number of free pages with full fragmentation + entries = (max_pages + 1) / 2 + + avail = page_size - list_header + entries_per_page = avail / 8 # 8 byte per entry + + return pages(entries, entries_per_page) + + +def wal_mapping_pages(page_size, entries): + """Compute number of required pages for the wal id mapping""" + entries_per_page = (page_size - list_header) / 14 # 14 byte per entry + return pages(entries, entries_per_page) + + +def next_power_of_2(x): + return 1 << (x-1).bit_length() + + +if __name__ == "__main__": + main() diff --git a/vendor/github.com/elastic/go-txfile/page.go b/vendor/github.com/elastic/go-txfile/page.go new file mode 100644 index 00000000000..8b26781f25f --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/page.go @@ -0,0 +1,238 @@ +package txfile + +// Page provides access to an on disk page. +// Pages can only be overwritten from within a read-write Transaction. +// Writes are be buffered until transaction commit, such that other but the +// current transaction will not be able to see file changes. +type Page struct { + id PageID // Original PageID for user access. + ondiskID PageID // On disk PageID. If contents is loaded from overwrite page, ondiskID != id + + tx *Tx // Parent transaction. + bytes []byte // Page contents. + flags pageFlags +} + +// PageID used to reference a file pages. +type PageID uint64 + +type pageFlags struct { + new bool // page has been allocated. No on-disk contents. + freed bool // page has been freed within current transaction. + flushed bool // page has already been flushed. No more writing possible. + cached bool // original page contents is copied in memory and can be overwritten. + dirty bool // page is marked as dirty and will be written on commit +} + +const minPageSize = 1024 + +// newPage creates a new page context within the current transaction. +func newPage(tx *Tx, id PageID) *Page { + return &Page{id: id, ondiskID: id, tx: tx} +} + +// ID returns the pages PageID. The ID can be used to store a reference +// to this page, for use within another transaction. +func (p *Page) ID() PageID { return p.id } + +// Readonly checks if the page is accessed in readonly mode. +func (p *Page) Readonly() bool { return p.tx.Readonly() } + +// Writable checks if the page can be written to. +func (p *Page) Writable() bool { return !p.Readonly() } + +// Dirty reports if the page is marked as dirty and needs to be flushed on +// commit. +func (p *Page) Dirty() bool { return p.flags.dirty } + +// MarkDirty marks a page as dirty. MarkDirty should only be used if +// in-place modification to the pages buffer have been made, after use of Load(). +func (p *Page) MarkDirty() error { + if err := p.canWrite(); err != nil { + return err + } + + p.flags.dirty = true + return nil +} + +// Free marks a page as free. Freeing a dirty page will return an error. +// The page will be returned to the allocator when the transaction commits. +func (p *Page) Free() error { + if err := p.canWrite(); err != nil { + return err + } + if p.flags.dirty { + return errFreeDirtyPage + } + + p.tx.freePage(p.id) + if p.id != p.ondiskID { + p.tx.freeWALID(p.id, p.ondiskID) + } + + p.flags.freed = true + return nil +} + +// Bytes returns the page its contents. +// One can only modify the buffer in write transaction, if Load() or SetBytes() +// have been called before Bytes(). Otherwise a non-recoverable BUS panic might +// be triggerd (program will be killed by OS). +// Bytes returns an error if the page has just been allocated (no backing buffer) +// or the transaction is already been closed. +// Use SetBytes() or Load(), to initialize the buffer of a newly allocated page. +func (p *Page) Bytes() ([]byte, error) { + if err := p.canRead(); err != nil { + return nil, err + } + if p.bytes == nil && p.flags.new { + return nil, errNoPageData + } + + return p.getBytes() +} + +func (p *Page) getBytes() ([]byte, error) { + if p.bytes == nil { + bytes := p.tx.access(p.ondiskID) + if bytes == nil { + return nil, errOutOfBounds + } + + p.bytes = bytes + } + + return p.bytes, nil +} + +// Load reads the pages original contents into a cached memory buffer, allowing +// for in-place modifications to the page. Load returns and error, if used from +// within a readonly transaction. +// If the page has been allocated from within the current transaction, a new +// temporary buffer will be allocated. +// After load, the write-buffer can be accessed via Bytes(). After modifications to the buffer, +// one must use MarkDirty(), so the page will be flushed on commit. +func (p *Page) Load() error { + if err := p.canWrite(); err != nil { + return err + } + + return p.loadBytes() +} + +func (p *Page) loadBytes() error { + if p.flags.cached { + return nil + } + + if p.flags.new { + p.flags.cached = true + p.bytes = make([]byte, p.tx.PageSize()) + return nil + } + + if p.flags.dirty { + p.flags.cached = true + return nil + } + + // copy original contents into writable buffer (page needs to be marked dirty if contents is overwritten) + orig, err := p.getBytes() + if err != nil { + return err + } + tmp := make([]byte, len(orig)) + copy(tmp, orig) + p.bytes = tmp + p.flags.cached = true + + return nil +} + +// SetBytes sets the new contents of a page. If the size of contents is less +// then the files page size, the original contents must be read. If the length +// of contents matches the page size, a reference to the contents buffer will +// be held. To enforce a copy, use Load(), Bytes(), copy() and MarkDirty(). +func (p *Page) SetBytes(contents []byte) error { + if err := p.canWrite(); err != nil { + return err + } + + pageSize := p.tx.PageSize() + if len(contents) > pageSize { + return errTooManyBytes + } + + if len(contents) < pageSize { + if err := p.loadBytes(); err != nil { + return err + } + copy(p.bytes, contents) + } else { + p.bytes = contents + } + + p.flags.dirty = true + return nil +} + +// Flush flushes the page write buffer, if the page is marked as dirty. +// The page its contents must not be changed after calling Flush, as the flush +// is executed asynchronously in the background. +// Dirty pages will be automatically flushed on commit. +func (p *Page) Flush() error { + if err := p.canWrite(); err != nil { + return err + } + + return p.doFlush() +} + +func (p *Page) doFlush() error { + if !p.flags.dirty || p.flags.flushed { + return nil + } + + if !p.flags.new { + if p.id == p.ondiskID { + walID := p.tx.allocWALID(p.id) + if walID == 0 { + return errOutOfMemory + } + p.ondiskID = walID + } else { + // page already in WAL -> free WAL page and write into original page + p.tx.freeWALID(p.id, p.ondiskID) + p.ondiskID = p.id + } + } + + p.flags.flushed = true + p.tx.scheduleWrite(p.ondiskID, p.bytes) + return nil +} + +func (p *Page) canRead() error { + if !p.tx.Active() { + return errTxFinished + } + if p.flags.freed { + return errFreedPage + } + return nil +} + +func (p *Page) canWrite() error { + if err := p.tx.canWrite(); err != nil { + return err + } + + if p.flags.freed { + return errFreedPage + } + if p.flags.flushed { + return errPageFlushed + } + return nil +} diff --git a/vendor/github.com/elastic/go-txfile/pageset.go b/vendor/github.com/elastic/go-txfile/pageset.go new file mode 100644 index 00000000000..33e26cf251f --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pageset.go @@ -0,0 +1,49 @@ +package txfile + +type pageSet map[PageID]struct{} + +func (s *pageSet) Add(id PageID) { + if *s == nil { + *s = pageSet{} + } + (*s)[id] = struct{}{} +} + +func (s pageSet) Has(id PageID) bool { + if s != nil { + _, exists := s[id] + return exists + } + return false +} + +func (s pageSet) Empty() bool { return s.Count() == 0 } + +func (s pageSet) Count() int { return len(s) } + +func (s pageSet) IDs() idList { + L := len(s) + if L == 0 { + return nil + } + + l, i := make(idList, L), 0 + for id := range s { + l[i], i = id, i+1 + } + return l +} + +func (s pageSet) Regions() regionList { + if len(s) == 0 { + return nil + } + + regions, i := make(regionList, len(s)), 0 + for id := range s { + regions[i], i = region{id: id, count: 1}, i+1 + } + optimizeRegionList(®ions) + + return regions +} diff --git a/vendor/github.com/elastic/go-txfile/pq/access.go b/vendor/github.com/elastic/go-txfile/pq/access.go new file mode 100644 index 00000000000..4459a31b4dc --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/access.go @@ -0,0 +1,103 @@ +package pq + +import "github.com/elastic/go-txfile" + +// Access provides transaction support and access to pages and queue header. +// It wraps the Delegate for providing a common interface for working with +// transactions and files. +type access struct { + Delegate + rootID txfile.PageID + rootOff int +} + +func makeAccess(delegate Delegate) (access, error) { + rootID, rootOff := delegate.Root() + if rootID == 0 { + return access{}, errNoQueueRoot + } + + return access{ + Delegate: delegate, + rootID: rootID, + rootOff: int(rootOff), + }, nil +} + +// ReadRoot reads the root page into an array. +// ReadRoot create a short lived read transaction for accessing and copying the +// queue root. +func (a *access) ReadRoot() ([SzRoot]byte, error) { + var buf [SzRoot]byte + + tx := a.BeginRead() + defer tx.Close() + + return buf, withPage(tx, a.rootID, func(page []byte) error { + n := copy(buf[:], page[a.rootOff:]) + if n < SzRoot { + return errIncompleteQueueRoot + } + return nil + }) +} + +// RootPage accesses the queue root page from within the passed transaction. +func (a *access) RootPage(tx *txfile.Tx) (*txfile.Page, error) { + return tx.Page(a.rootID) +} + +// LoadRootPage accesses the queue root page from within the passed write +// transaction. +// The Root page it's content is loaded into the write buffer for manipulations. +// The page returned is not marked as dirty yet. +func (a *access) LoadRootPage(tx *txfile.Tx) (*txfile.Page, *queuePage, error) { + var hdr *queuePage + page, err := a.RootPage(tx) + if err == nil { + err = page.Load() + if err == nil { + buf, _ := page.Bytes() + hdr = castQueueRootPage(buf[a.rootOff:]) + } + } + + return page, hdr, err +} + +// RootHdr returns a pointer to the queue root header. The pointer to the +// header is only valid as long as the transaction is still active. +func (a *access) RootHdr(tx *txfile.Tx) (hdr *queuePage, err error) { + err = withPage(tx, a.rootID, func(buf []byte) error { + hdr = castQueueRootPage(buf[a.rootOff:]) + return nil + }) + return +} + +// ParsePosition parses an on disk position, providing page id, page offset and +// event id in a more accessible format. +func (a *access) ParsePosition(p *pos) position { + page, off := a.SplitOffset(uintptr(p.offset.Get())) + if page != 0 && off == 0 { + off = uintptr(a.PageSize()) + } + + return position{ + page: page, + off: int(off), + id: p.id.Get(), + } +} + +// WritePosition serializes a position into it's on-disk representation. +func (a *access) WritePosition(to *pos, pos position) { + pageOff := pos.off + if pageOff == a.PageSize() { + pageOff = 0 // use 0 to mark page offset as end-of-page + } + + off := a.Offset(pos.page, uintptr(pageOff)) + to.offset.Set(uint64(off)) + to.id.Set(pos.id) +} diff --git a/vendor/github.com/elastic/go-txfile/pq/ack.go b/vendor/github.com/elastic/go-txfile/pq/ack.go new file mode 100644 index 00000000000..704d5f4c206 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/ack.go @@ -0,0 +1,297 @@ +package pq + +import ( + "github.com/elastic/go-txfile" + "github.com/elastic/go-txfile/internal/invariant" +) + +// acker is used to asynchronously ack and remove events from the queue. +type acker struct { + accessor *access + active bool + + totalEventCount uint + totalFreedPages uint + + ackCB func(events, pages uint) +} + +// ackState records the changes required to finish the ACK step. +type ackState struct { + free []txfile.PageID // Collect page ids to be freed. + head position // New queue head, pointing to first event in first available page + read position // New on-disk read pointer, pointing to first not-yet ACKed event. +} + +func newAcker( + accessor *access, + cb func(uint, uint), +) (*acker, error) { + return &acker{ + active: true, + accessor: accessor, + ackCB: cb, + }, nil +} + +func (a *acker) close() { + a.active = false +} + +// handle processes an ACK by freeing pages and +// updating the head and read positions in the queue root. +// So to not interfere with concurrent readers potentially updating pointers +// or adding new contents to a page, the last event page in the queue will never +// be freed. Still the read pointer might point past the last page. +func (a *acker) handle(n uint) error { + if n == 0 { + return nil + } + + if !a.active { + return errClosed + } + + traceln("acker: pq ack events:", n) + + state, err := a.initACK(n) + if err != nil { + return err + } + + // start write transaction to free pages and update the next read offset in + // the queue root + tx := a.accessor.BeginCleanup() + defer tx.Close() + + traceln("acker: free data pages:", len(state.free)) + for _, id := range state.free { + page, err := tx.Page(id) + if err != nil { + return err + } + + traceln("free page", id) + if err := page.Free(); err != nil { + return err + } + } + + // update queue header + hdrPage, hdr, err := a.accessor.LoadRootPage(tx) + if err != nil { + return err + } + a.accessor.WritePosition(&hdr.head, state.head) + a.accessor.WritePosition(&hdr.read, state.read) + hdr.inuse.Set(hdr.inuse.Get() - uint64(len(state.free))) + hdrPage.MarkDirty() + + traceQueueHeader(hdr) + + if err := tx.Commit(); err != nil { + return err + } + + a.totalEventCount += n + a.totalFreedPages += uint(len(state.free)) + tracef("Acked events. total events acked: %v, total pages freed: %v \n", a.totalEventCount, a.totalFreedPages) + + if a.ackCB != nil { + a.ackCB(n, uint(len(state.free))) + } + + return nil +} + +// initACK uses a read-transaction to collect pages to be removed from list and +// find offset of next read required to start reading the next un-acked event. +func (a *acker) initACK(n uint) (ackState, error) { + tx := a.accessor.BeginRead() + defer tx.Close() + + hdr, err := a.accessor.RootHdr(tx) + if err != nil { + return ackState{}, err + } + + headPos, startPos, endPos := a.queueRange(hdr) + startID := startPos.id + endID := startID + uint64(n) + if startPos.page == 0 { + return ackState{}, errACKEmptyQueue + } + if !idLessEq(endID, endPos.id) { + return ackState{}, errACKTooManyEvents + } + + c := makeTxCursor(tx, a.accessor, &cursor{ + page: headPos.page, + off: headPos.off, + pageSize: a.accessor.PageSize(), + }) + + // Advance through pages and collect ids of all pages to be freed. + // Free all pages, but the very last data page, so to not interfere with + // concurrent writes. + ids, cleanAll, err := a.collectFreePages(&c, endID) + if err != nil { + return ackState{}, err + } + + // find offset of next event to start reading from + var head, read position + if !cleanAll { + head, read, err = a.findNewStartPositions(&c, endID) + if err != nil { + return ackState{}, err + } + } else { + head = endPos + read = endPos + } + + return ackState{ + free: ids, + head: head, + read: read, + }, nil +} + +// queueRange finds the start and end positions of not yet acked events in the +// queue. +func (a *acker) queueRange(hdr *queuePage) (head, start, end position) { + start = a.accessor.ParsePosition(&hdr.read) + head = a.accessor.ParsePosition(&hdr.head) + if start.page == 0 { + start = head + } + + end = a.accessor.ParsePosition(&hdr.tail) + return +} + +// collectFreePages collects all pages to be freed. A page can be freed if all +// events within the page have been acked. We want to free all pages, but the +// very last data page, so to not interfere with concurrent writes. +// All pages up to endID will be collected. +func (a *acker) collectFreePages(c *txCursor, endID uint64) ([]txfile.PageID, bool, error) { + var ( + ids []txfile.PageID + firstID, lastID uint64 + cleanAll = false + ) + + for { + hdr, err := c.PageHeader() + if err != nil { + return nil, false, err + } + + // stop searching if endID is in the current page + dataOnlyPage := hdr.off.Get() == 0 // no event starts within this page + if !dataOnlyPage { + firstID, lastID = hdr.first.Get(), hdr.last.Get() + + // inc 'lastID', so to hold on current page if endID would point to next + // the page. This helps the reader, potentially pointing to the current + // page, if next page has not been committed when reading events. + lastID++ + + if idLessEq(firstID, endID) && idLessEq(endID, lastID) { + break + } + } + + // stop searching if current page is the last page. The last page must + // be active for the writer to add more events and link new pages. + lastPage := hdr.next.Get() == 0 + if lastPage { + cleanAll = true + invariant.Check(lastID+1 == endID, "last event ID and ack event id missmatch") + break + } + + // found intermediate page with ACKed events/contents + // -> add page id to freelist and advance to next page + ids = append(ids, c.cursor.page) + ok, err := c.AdvancePage() + if err != nil { + return nil, false, err + } + invariant.Check(ok, "page list linkage broken") + } + + return ids, cleanAll, nil +} + +// findNewStartPositions skips acked events, so to find the new head and read pointers to be set +// in the updated queue header. +func (a *acker) findNewStartPositions(c *txCursor, id uint64) (head, read position, err error) { + var hdr *eventPage + + hdr, err = c.PageHeader() + if err != nil { + return + } + + head = position{ + page: c.cursor.page, + off: int(hdr.off.Get()), + id: hdr.first.Get(), + } + + if id == head.id { + read = head + return + } + + // skip contents in current page until we did reach start of next event. + c.cursor.off = head.off + for currentID := head.id; currentID != id; currentID++ { + var evtHdr *eventHeader + evtHdr, err = c.ReadEventHeader() + if err != nil { + return + } + + err = c.Skip(int(evtHdr.sz.Get())) + if err != nil { + return + } + } + + read = position{ + page: c.cursor.page, + off: c.cursor.off, + id: id, + } + return +} + +// Active returns the total number of active, not yet ACKed events. +func (a *acker) Active() (uint, error) { + tx := a.accessor.BeginRead() + defer tx.Close() + + hdr, err := a.accessor.RootHdr(tx) + if err != nil { + return 0, err + } + + // Empty queue? + if hdr.tail.offset.Get() == 0 { + return 0, nil + } + + var start, end uint64 + + end = hdr.tail.id.Get() + if hdr.read.offset.Get() != 0 { + start = hdr.read.id.Get() + } else { + start = hdr.head.id.Get() + } + + return uint(end - start), nil +} diff --git a/vendor/github.com/elastic/go-txfile/pq/buffer.go b/vendor/github.com/elastic/go-txfile/pq/buffer.go new file mode 100644 index 00000000000..c63ed6ade76 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/buffer.go @@ -0,0 +1,244 @@ +package pq + +import "github.com/elastic/go-txfile/internal/invariant" + +// buffer holds allocated and yet unallocated in-memory pages, for appending +// events to. +type buffer struct { + // in-memory pages + head, tail *page + + pool *pagePool + + // settings (values don't change after init) + pageSize int + hdrSize int + payloadSize int // effective page contents + + // page write state + avail int // available space before forcing flush + payload []byte // byte slice of available payload/bytes in the current page + page *page // current page + + // Event write state. Stores reference to start of current events, so we can + // put in the event header once the current event is finished. + eventHdrPage *page + eventHdrOffset int + eventHdrSize int +} + +func newBuffer(pool *pagePool, page *page, pages, pageSize, hdrSz int) *buffer { + payloadSz := pageSize - hdrSz + avail := payloadSz * pages + + b := &buffer{ + head: nil, + tail: nil, + pool: pool, + pageSize: pageSize, + hdrSize: hdrSz, + payloadSize: payloadSz, + avail: avail, + payload: nil, + page: nil, + eventHdrPage: nil, + eventHdrOffset: -1, + eventHdrSize: -1, + } + + if page != nil { + // init with end of on-disk list from former writes + b.head = page + b.tail = page + + contentsLength := int(page.Meta.EndOff) - b.hdrSize + b.avail -= contentsLength + b.payload = page.Data[page.Meta.EndOff:] + b.page = page + } + + return b +} + +// Avail returns amount of bytes available. Returns a value <0, if contents in +// buffer exceeds the high-water-marks. +func (b *buffer) Avail() int { + return b.avail +} + +// Append adds more bytes to the current event. Use `CommitEvent` to finalize the +// writing of the current event. +// If required append adds new unallocated pages to the write buffer. +func (b *buffer) Append(data []byte) { + for len(data) > 0 { + if len(b.payload) == 0 { + b.advancePage() + } + + n := copy(b.payload, data) + b.payload = b.payload[n:] + data = data[n:] + b.avail -= n + + tracef("writer: append %v bytes to (page: %v, off: %v)\n", n, b.page.Meta.ID, b.page.Meta.EndOff) + + b.page.Meta.EndOff += uint32(n) + } +} + +func (b *buffer) advancePage() { + // link new page into list + page := b.newPage() + if b.tail == nil { + b.head = page + b.tail = page + } else { + b.tail.Next = page + b.tail = page + } + + b.page = page + b.payload = page.Payload() + page.Meta.EndOff = uint32(szEventPageHeader) +} + +func (b *buffer) newPage() *page { + return b.pool.NewPage() +} + +func (b *buffer) releasePage(p *page) { + b.pool.Release(p) +} + +// ReserveHdr reserves space for the next event header in the write buffer. +// The start position in the buffer is tracked by the buffer, until the event is +// finished via CommitEvent. +func (b *buffer) ReserveHdr(n int) []byte { + if n > b.payloadSize { + return nil + } + + invariant.Check(b.eventHdrPage == nil, "can not reserve a new event header if recent event is not finished yet") + + // reserve n bytes in payload + if len(b.payload) < n { + b.advancePage() + } + + payloadWritten := b.payloadSize - len(b.payload) + b.eventHdrPage = b.page + b.eventHdrPage.Meta.EndOff += uint32(n) + b.eventHdrOffset = b.hdrSize + payloadWritten + b.eventHdrSize = n + b.payload = b.payload[n:] + b.avail -= n + + return b.ActiveEventHdr() +} + +// ActiveEventHdr returns the current event header bytes content for writing/reading. +func (b *buffer) ActiveEventHdr() []byte { + if b.eventHdrPage == nil { + return nil + } + + off := b.eventHdrOffset + return b.eventHdrPage.Data[off : off+b.eventHdrSize] +} + +// CommitEvent marks the current event being finished. Finalize pages +// and prepare for next event. +func (b *buffer) CommitEvent(id uint64) { + invariant.Check(b.eventHdrPage != nil, "no active event") + + page := b.eventHdrPage + meta := &page.Meta + if meta.FirstOff == 0 { + meta.FirstOff = uint32(b.eventHdrOffset) + meta.FirstID = id + } + meta.LastID = id + page.MarkDirty() + + // mark all event pages as dirty + for current := b.eventHdrPage; current != nil; current = current.Next { + current.MarkDirty() + } + // mark head as dirty if yet unlinked + if b.head != b.eventHdrPage && b.head.Next == b.eventHdrPage { + b.head.MarkDirty() + } + + b.eventHdrPage = nil + b.eventHdrOffset = -1 + b.eventHdrSize = -1 +} + +// Pages returns start and end page to be serialized. +// The `end` page must not be serialized +func (b *buffer) Pages() (start, end *page) { + if b.head == nil || !b.head.Dirty() { + return nil, nil + } + + if b.eventHdrPage == nil { + if b.tail.Dirty() { + return b.head, nil + } + for current := b.head; current != nil; current = current.Next { + if !current.Dirty() { + return b.head, current + } + } + + invariant.Unreachable("tail if list dirty and not dirty?") + } + + end = b.eventHdrPage + if end.Dirty() { + end = end.Next + } + return b.head, end +} + +// Reset removes all but the last page non-dirty page from the buffer. +// The last written page is still required for writing/linking new events/pages. +func (b *buffer) Reset(last *page) { + if b.head == nil { + return + } + + // Find last page not to be removed. A non-dirty page must not be removed + // if the next page is dirty, so to update the on-disk link. + // If no page is dirty, keep last page for linking. + pages := 0 + end := b.head + for current := b.head; current.Next != nil && current != b.eventHdrPage; current = current.Next { + if current.Next.Dirty() || current == last { + end = current + break + } + end = current.Next + pages++ + } + + tracef("reset pages (%v)\n", pages) + + invariant.Check(end != nil, "must not empty page list on reset") + + // release pages + spaceFreed := 0 + for page := b.head; page != end; { + freed := int(page.Meta.EndOff) - szEventPageHeader + tracef("writer: release page %v (%v)\n", page.Meta.ID, freed) + + next := page.Next + spaceFreed += freed + b.releasePage(page) + page = next + } + b.head = end + + // update memory usage counters + b.avail += spaceFreed +} diff --git a/vendor/github.com/elastic/go-txfile/pq/cursor.go b/vendor/github.com/elastic/go-txfile/pq/cursor.go new file mode 100644 index 00000000000..f0564f93511 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/cursor.go @@ -0,0 +1,172 @@ +package pq + +import "github.com/elastic/go-txfile" + +// cursor holds state for iterating events in the queue. +type cursor struct { + page txfile.PageID + off int + pageSize int +} + +// txCursor is used to advance a cursor within a transaction. +type txCursor struct { + *cursor + accessor *access + tx *txfile.Tx + page *txfile.Page +} + +// Nil checks if the cursor is pointing to a page. Returns true, if cursor is +// not pointing to any page in the queue. +func (c *cursor) Nil() bool { + return c.page == 0 +} + +func makeTxCursor(tx *txfile.Tx, accessor *access, cursor *cursor) txCursor { + return txCursor{ + tx: tx, + accessor: accessor, + page: nil, + cursor: cursor, + } +} + +func (c *txCursor) init() error { + if c.page != nil { + return nil + } + page, err := c.tx.Page(c.cursor.page) + if err != nil { + return err + } + + c.page = page + return nil +} + +// Read reads more bytes from the current event into b. If the end of the +// current event has reached, no bytes will be read. +func (c *txCursor) Read(b []byte) (int, error) { + if err := c.init(); err != nil { + return 0, err + } + + if c.Nil() { + return 0, nil + } + + to, err := c.readInto(b) + return len(b) - len(to), err +} + +// Skip skips the next n bytes. +func (c *txCursor) Skip(n int) error { + for n > 0 { + if c.PageBytes() == 0 { + ok, err := c.AdvancePage() + if err != nil { + return err + } + if !ok { + return errSeekPageFailed + } + } + + max := n + if L := c.PageBytes(); L < max { + max = L + } + c.cursor.off += max + n -= max + } + + return nil +} + +func (c *txCursor) readInto(to []byte) ([]byte, error) { + for len(to) > 0 { + // try to advance cursor to next page if last read did end at end of page + if c.PageBytes() == 0 { + ok, err := c.AdvancePage() + if !ok || err != nil { + return to, err + } + } + + var n int + err := c.WithBytes(func(b []byte) { n = copy(to, b) }) + to = to[n:] + c.cursor.off += n + if err != nil { + return to, err + } + } + + return to, nil +} + +func (c *txCursor) ReadEventHeader() (hdr *eventHeader, err error) { + err = c.WithBytes(func(b []byte) { + hdr = castEventHeader(b) + c.off += szEventHeader + }) + return +} + +func (c *txCursor) PageHeader() (hdr *eventPage, err error) { + err = c.WithHdr(func(h *eventPage) { + hdr = h + }) + return +} + +func (c *txCursor) AdvancePage() (ok bool, err error) { + err = c.WithHdr(func(hdr *eventPage) { + nextID := txfile.PageID(hdr.next.Get()) + tracef("advance page from %v -> %v\n", c.cursor.page, nextID) + ok = nextID != 0 + + if ok { + c.cursor.page = nextID + c.cursor.off = szEventPageHeader + c.page = nil + } + }) + return +} + +func (c *txCursor) WithPage(fn func([]byte)) error { + if err := c.init(); err != nil { + return err + } + + buf, err := c.page.Bytes() + if err != nil { + return err + } + + fn(buf) + return nil +} + +func (c *txCursor) WithHdr(fn func(*eventPage)) error { + return c.WithPage(func(b []byte) { + fn(castEventPageHeader(b)) + }) +} + +func (c *txCursor) WithBytes(fn func([]byte)) error { + return c.WithPage(func(b []byte) { + fn(b[c.off:]) + }) +} + +// PageBytes reports the amount of bytes still available in current page +func (c *cursor) PageBytes() int { + return c.pageSize - c.off +} + +func (c *cursor) Reset() { + *c = cursor{} +} diff --git a/vendor/github.com/elastic/go-txfile/pq/delegate.go b/vendor/github.com/elastic/go-txfile/pq/delegate.go new file mode 100644 index 00000000000..aa9197993dc --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/delegate.go @@ -0,0 +1,110 @@ +package pq + +import "github.com/elastic/go-txfile" + +// Delegate is used by the persistent queue to query common parameters and +// start transactions when required. +type Delegate interface { + // PageSize reports the page size to be used by the backing file. + PageSize() int + + // Root returns the queues root on file. + Root() (txfile.PageID, uintptr) + + Offset(id txfile.PageID, offset uintptr) uintptr + + SplitOffset(uintptr) (txfile.PageID, uintptr) + + // BeginWrite must create a read-write transaction for use by the writer. + // The transaction will be used to allocate pages and flush the current write + // buffer. + BeginWrite() *txfile.Tx + + // BeginRead must return a readonly transaction. + BeginRead() *txfile.Tx + + // BeginCleanup must return a read-write transaction for the ACK handling to + // remove events. No new contents will be written, but pages will be freed + // and the queue root page being updated. + BeginCleanup() *txfile.Tx +} + +// standaloneDelegate wraps a txfile.File into a standalone queue only file. +// The delegate sets the files root to the queue header. +type standaloneDelegate struct { + file *txfile.File + root txfile.PageID +} + +// NewStandaloneDelegate creates a standaonle Delegate from an txfile.File +// instance. This function will allocate and initialize the queue root page. +func NewStandaloneDelegate(f *txfile.File) (Delegate, error) { + tx := f.Begin() + defer tx.Close() + + root := tx.Root() + if root == 0 { + var err error + + root, err = initQueueRoot(tx) + if err != nil { + return nil, err + } + } + + return &standaloneDelegate{file: f, root: root}, nil +} + +func initQueueRoot(tx *txfile.Tx) (txfile.PageID, error) { + page, err := tx.Alloc() + if err != nil { + return 0, err + } + + buf := MakeRoot() + if err := page.SetBytes(buf[:]); err != nil { + return 0, err + } + + tx.SetRoot(page.ID()) + return page.ID(), tx.Commit() +} + +// PageSize returns the files page size. +func (d *standaloneDelegate) PageSize() int { + return d.file.PageSize() +} + +// Root finds the queue root page and offset. +func (d *standaloneDelegate) Root() (txfile.PageID, uintptr) { + return d.root, 0 +} + +func (d *standaloneDelegate) Offset(id txfile.PageID, offset uintptr) uintptr { + return d.file.Offset(id, offset) +} + +func (d *standaloneDelegate) SplitOffset(offset uintptr) (txfile.PageID, uintptr) { + return d.file.SplitOffset(offset) +} + +// BeginWrite creates a new transaction for flushing the write buffers to disk. +func (d *standaloneDelegate) BeginWrite() *txfile.Tx { + return d.file.BeginWith(txfile.TxOptions{ + WALLimit: 3, + }) +} + +// BeginRead returns a readonly transaction. +func (d *standaloneDelegate) BeginRead() *txfile.Tx { + return d.file.BeginReadonly() +} + +// BeginCleanup creates a new write transaction configured for cleaning up used +// events/pages only. +func (d *standaloneDelegate) BeginCleanup() *txfile.Tx { + return d.file.BeginWith(txfile.TxOptions{ + EnableOverflowArea: true, + WALLimit: 3, + }) +} diff --git a/vendor/github.com/elastic/go-txfile/pq/error.go b/vendor/github.com/elastic/go-txfile/pq/error.go new file mode 100644 index 00000000000..2d525a3dcd7 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/error.go @@ -0,0 +1,15 @@ +package pq + +import "errors" + +var ( + errNODelegate = errors.New("delegate must not be nil") + errInvalidPagesize = errors.New("invalid page size") + errClosed = errors.New("queue closed") + errNoQueueRoot = errors.New("no queue root") + errIncompleteQueueRoot = errors.New("incomplete queue root") + errInvalidVersion = errors.New("invalid queue version") + errACKEmptyQueue = errors.New("ack on empty queue") + errACKTooManyEvents = errors.New("too many events have been acked") + errSeekPageFailed = errors.New("failed to seek to next page") +) diff --git a/vendor/github.com/elastic/go-txfile/pq/layout.go b/vendor/github.com/elastic/go-txfile/pq/layout.go new file mode 100644 index 00000000000..85e6ed69e70 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/layout.go @@ -0,0 +1,81 @@ +package pq + +import ( + "unsafe" + + bin "github.com/urso/go-bin" + + "github.com/elastic/go-txfile" +) + +// primitive types +type ( + u32 = bin.U32le + u64 = bin.U64le +) + +// queuePage is the root structure into a persisted Queue instance. +type queuePage struct { + version u32 + + // start/end of single linked events list and event ids + head pos // head points to first event in list + tail pos // tail points to next event to be written + + // read points to next event to continue reading from + // if read == tail, all events have been read + read pos + + inuse u64 // number of actively used data pages +} + +type pos struct { + offset u64 // file offset of event + id u64 // id of event +} + +// eventPage create a single list of event pages, storing a number +// of events per page. +// If off == 0, the page does contain data only. +type eventPage struct { + next u64 // PageID of next eventPage + first u64 // event id of first event in current page + last u64 // event id of last even in current page + off u32 // offset of first event in current page +} + +// eventHeader is keeps track of the event size in bytes. +// The event ID can be 'computed' by iterating the events in a page. +type eventHeader struct { + sz u32 +} + +const ( + queueVersion = 1 + + // SzRoot is the size of the queue header in bytes. + SzRoot = int(unsafe.Sizeof(queuePage{})) + + szEventPageHeader = int(unsafe.Sizeof(eventPage{})) + szEventHeader = int(unsafe.Sizeof(eventHeader{})) +) + +func castQueueRootPage(b []byte) (hdr *queuePage) { bin.UnsafeCastStruct(&hdr, b); return } + +func castEventPageHeader(b []byte) (hdr *eventPage) { bin.UnsafeCastStruct(&hdr, b); return } + +func castEventHeader(b []byte) (hdr *eventHeader) { bin.UnsafeCastStruct(&hdr, b); return } + +func traceQueueHeader(hdr *queuePage) { + traceln("queue header:") + traceln(" version:", hdr.version.Get()) + tracef(" head(%v, %v)\n", hdr.head.id.Get(), hdr.head.offset.Get()) + tracef(" tail(%v, %v)\n", hdr.tail.id.Get(), hdr.tail.offset.Get()) + tracef(" read(%v, %v)\n", hdr.read.id.Get(), hdr.read.offset.Get()) + traceln(" data pages", hdr.inuse.Get()) +} + +func tracePageHeader(id txfile.PageID, hdr *eventPage) { + tracef("event page %v (next=%v, first=%v, last=%v, off=%v)\n", + id, hdr.next.Get(), hdr.first.Get(), hdr.last.Get(), hdr.off.Get()) +} diff --git a/vendor/github.com/elastic/go-txfile/pq/page.go b/vendor/github.com/elastic/go-txfile/pq/page.go new file mode 100644 index 00000000000..03b6b7cca22 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/page.go @@ -0,0 +1,117 @@ +package pq + +import ( + "sync" + + "github.com/elastic/go-txfile" +) + +// page is used by the write buffer to keep page content and on-disk +// assignment. Pages with Meta.ID == 0 are not allocated on disk yet. +type page struct { + Next *page + + Meta pageMeta + Data []byte +} + +type pageMeta struct { + ID txfile.PageID + FirstID, LastID uint64 + FirstOff uint32 + EndOff uint32 + Flags pageFlags +} + +type pageFlags struct { + Dirty bool // indicates new event contents being written to this page +} + +type pagePool struct { + sync.Pool +} + +func newPagePool(pageSize int) *pagePool { + return &pagePool{sync.Pool{ + New: func() interface{} { + return &page{ + Data: make([]byte, pageSize), + } + }, + }} +} + +func (pp *pagePool) NewPage() *page { + return pp.get() +} + +func (pp *pagePool) NewPageWith(id txfile.PageID, contents []byte) *page { + p := pp.NewPage() + copy(p.Data, contents) + hdr := castEventPageHeader(contents) + p.Meta = pageMeta{ + ID: id, + FirstID: hdr.first.Get(), + LastID: hdr.last.Get(), + FirstOff: hdr.off.Get(), + } + return p +} + +func (pp *pagePool) get() *page { return pp.Pool.Get().(*page) } + +func (pp *pagePool) Release(p *page) { + p.Clear() + pp.Pool.Put(p) +} + +// Clear zeroes out a page object and the buffer page header, preparing the +// page object for being reused. +func (p *page) Clear() { + p.Meta = pageMeta{} + p.Next = nil + + // clear page header + for i := 0; i < szEventPageHeader; i++ { + p.Data[i] = 0 + } +} + +// Assigned checks if the page is represented by on on-disk page. +func (p *page) Assigned() bool { + return p.Meta.ID != 0 +} + +// Dirty checks if the page is dirty and must be flushed. +func (p *page) Dirty() bool { + return p.Meta.Flags.Dirty +} + +// MarkDirty marks a page as dirty. +func (p *page) MarkDirty() { + p.Meta.Flags.Dirty = true +} + +// UnmarkDirty marks a page as being in sync with the on-disk page. +func (p *page) UnmarkDirty() { + p.Meta.Flags.Dirty = false +} + +// SetNext write the next page ID into the page header. +func (p *page) SetNext(id txfile.PageID) { + hdr := castEventPageHeader(p.Data) + hdr.next.Set(uint64(id)) +} + +// Payload returns the slice of the page it's complete payload. +func (p *page) Payload() []byte { + return p.Data[szEventPageHeader:] +} + +// UpdateHeader updates the page header to reflect the page meta-data pages. +func (p *page) UpdateHeader() { + hdr := castEventPageHeader(p.Data) + hdr.first.Set(p.Meta.FirstID) + hdr.last.Set(p.Meta.LastID) + hdr.off.Set(p.Meta.FirstOff) +} diff --git a/vendor/github.com/elastic/go-txfile/pq/pq.go b/vendor/github.com/elastic/go-txfile/pq/pq.go new file mode 100644 index 00000000000..3f0076bfc68 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/pq.go @@ -0,0 +1,161 @@ +package pq + +import ( + "github.com/elastic/go-txfile" +) + +// Queue implements the on-disk queue data structure. The queue requires a +// Delegate, so to start transactions at any time. The Queue provides a reader +// and writer. While it is safe to use the Reader and Writer concurrently, the +// Reader and Writer themselves are not thread-safe. +type Queue struct { + accessor access + + // TODO: add support for multiple named readers with separate ACK handling. + + reader *Reader + writer *Writer + acker *acker +} + +type position struct { + page txfile.PageID + off int + id uint64 +} + +// Settings configures a queue when being instantiated with `New`. +type Settings struct { + // Queue write buffer size. If a single event is bigger then the + // write-buffer, the write-buffer will grow. In this case will the write + // buffer be flushed and reset to its original size. + WriteBuffer uint + + // Optional Flushed callback. Will be used to notify n events being + // successfully committed. + Flushed func(n uint) + + // Optional ACK callback. Will be use to notify number of events being successfully + // ACKed and pages being freed. + ACKed func(event, pages uint) +} + +// MakeRoot prepares the queue header (empty queue). +// When creating a queue with `New`, the queue header must be available. +// Still, a delegate is allowed to create the queue header lazily. +func MakeRoot() [SzRoot]byte { + var buf [SzRoot]byte + qu := castQueueRootPage(buf[:]) + qu.version.Set(queueVersion) + return buf +} + +// New creates a new Queue. The delegate is required to access the file and +// start transactions. An error is returned if the delegate is nil, the queue +// header is invalid, some settings are invalid, or if some IO error occurred. +func New(delegate Delegate, settings Settings) (*Queue, error) { + if delegate == nil { + return nil, errNODelegate + } + + accessor, err := makeAccess(delegate) + if err != nil { + return nil, err + } + + q := &Queue{accessor: accessor} + + pageSize := delegate.PageSize() + pagePool := newPagePool(pageSize) + + rootBuf, err := q.accessor.ReadRoot() + if err != nil { + return nil, err + } + + root := castQueueRootPage(rootBuf[:]) + if root.version.Get() != queueVersion { + return nil, errInvalidVersion + } + + tracef("open queue: %p (pageSize: %v)\n", q, pageSize) + traceQueueHeader(root) + + tail := q.accessor.ParsePosition(&root.tail) + writer, err := newWriter(&q.accessor, pagePool, + settings.WriteBuffer, tail, settings.Flushed) + if err != nil { + return nil, err + } + + reader, err := newReader(&q.accessor) + if err != nil { + return nil, err + } + + acker, err := newAcker(&q.accessor, settings.ACKed) + if err != nil { + return nil, err + } + + q.reader = reader + q.writer = writer + q.acker = acker + return q, nil +} + +// Close will try to flush the current write buffer, +// but after closing the queue, no more reads or writes can be executed +func (q *Queue) Close() error { + tracef("close queue %p\n", q) + defer tracef("queue %p closed\n", q) + + q.reader.close() + q.acker.close() + return q.writer.close() +} + +// Pending returns the total number of enqueued, but unacked events. +func (q *Queue) Pending() int { + tx := q.accessor.BeginRead() + defer tx.Close() + + hdr, err := q.accessor.RootHdr(tx) + if err != nil { + return -1 + } + + head := q.accessor.ParsePosition(&hdr.read) + if head.page == 0 { + head = q.accessor.ParsePosition(&hdr.head) + } + tail := q.accessor.ParsePosition(&hdr.tail) + + return int(tail.id - head.id) +} + +// Writer returns the queue writer for inserting new events into the queue. +// A queue has only one single writer instance, which is returned by GetWriter. +// The writer is is not thread safe. +func (q *Queue) Writer() *Writer { + return q.writer +} + +// Reader returns the queue reader for reading a new events from the queue. +// A queue has only one single reader instance. +// The reader is not thread safe. +func (q *Queue) Reader() *Reader { + return q.reader +} + +// ACK signals the queue, the most n events at the front of the queue have been +// processed. +// The queue will try to remove these asynchronously. +func (q *Queue) ACK(n uint) error { + return q.acker.handle(n) +} + +// Active returns the number of active, not yet ACKed events. +func (q *Queue) Active() (uint, error) { + return q.acker.Active() +} diff --git a/vendor/github.com/elastic/go-txfile/pq/reader.go b/vendor/github.com/elastic/go-txfile/pq/reader.go new file mode 100644 index 00000000000..762e242d95c --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/reader.go @@ -0,0 +1,210 @@ +package pq + +import ( + "github.com/elastic/go-txfile" + "github.com/elastic/go-txfile/internal/invariant" +) + +// Reader is used to iterate events stored in the queue. +type Reader struct { + accessor *access + state readState + active bool +} + +type readState struct { + id uint64 + endID uint64 // id of next, yet unwritten event. + eventBytes int // number of unread bytes in current event + + cursor cursor +} + +func newReader(accessor *access) (*Reader, error) { + return &Reader{ + active: true, + accessor: accessor, + state: readState{ + eventBytes: -1, + cursor: cursor{ + pageSize: accessor.PageSize(), + }, + }, + }, nil +} + +func (r *Reader) close() { + r.active = false +} + +// Available returns the number of unread events that can be read. +func (r *Reader) Available() uint { + if !r.active { + return 0 + } + + func() { + tx := r.accessor.BeginRead() + defer tx.Close() + r.updateQueueState(tx) + }() + + if r.state.cursor.Nil() { + return 0 + } + + return uint(r.state.endID - r.state.id) +} + +// Read reads the contents of the current event into the buffer. +// Returns 0 without reading if end of the current event has been reached. +// Use `Next` to skip/continue reading the next event. +func (r *Reader) Read(b []byte) (int, error) { + if !r.active { + return -1, errClosed + } + + if r.state.eventBytes <= 0 { + return 0, nil + } + + to, err := r.readInto(b) + return len(b) - len(to), err +} + +func (r *Reader) readInto(to []byte) ([]byte, error) { + tx := r.accessor.BeginRead() + defer tx.Close() + + n := r.state.eventBytes + if L := len(to); L < n { + n = L + } + + cursor := makeTxCursor(tx, r.accessor, &r.state.cursor) + for n > 0 { + consumed, err := cursor.Read(to[:n]) + to = to[consumed:] + n -= consumed + r.state.eventBytes -= consumed + + if err != nil { + return to, err + } + } + + // end of event -> advance to next event + var err error + if r.state.eventBytes == 0 { + r.state.eventBytes = -1 + r.state.id++ + + // As page is already in memory, use current transaction to try to skip to + // next page if no more new event fits into current page. + if cursor.PageBytes() < szEventHeader { + cursor.AdvancePage() + } + } + + return to, err +} + +// Next advances to the next event to be read. The event size in bytes is +// returned. A size of 0 is reported if no more event is available in the +// queue. +func (r *Reader) Next() (int, error) { + if !r.active { + return -1, errClosed + } + + tx := r.accessor.BeginRead() + defer tx.Close() + + cursor := makeTxCursor(tx, r.accessor, &r.state.cursor) + + // in event? Skip contents + if r.state.eventBytes > 0 { + err := cursor.Skip(r.state.eventBytes) + if err != nil { + return 0, err + } + + r.state.eventBytes = -1 + r.state.id++ + } + + // end of buffered queue state. Update state and check if we did indeed reach + // the end of the queue. + if cursor.Nil() || !idLess(r.state.id, r.state.endID) { + err := r.updateQueueState(tx) + if err != nil { + return 0, err + } + + // end of queue + if cursor.Nil() || !idLess(r.state.id, r.state.endID) { + return 0, nil + } + } + + // Advance page and initialize cursor if event header does not fit into + // current page. + if cursor.PageBytes() < szEventHeader { + // cursor was not advanced by last read. The acker will not have deleted + // the current page -> try to advance now. + ok, err := cursor.AdvancePage() + if err != nil { + return 0, err + } + invariant.Check(ok, "page list linkage broken") + + hdr, err := cursor.PageHeader() + if err != nil { + return 0, err + } + + id := hdr.first.Get() + off := int(hdr.off.Get()) + invariant.Check(r.state.id == id, "page start event id mismatch") + invariant.CheckNot(off == 0, "page event offset missing") + r.state.cursor.off = off + } + + // Initialize next event read by determining event size. + hdr, err := cursor.ReadEventHeader() + if err != nil { + return 0, err + } + L := int(hdr.sz.Get()) + r.state.eventBytes = L + return L, nil +} + +func (r *Reader) updateQueueState(tx *txfile.Tx) error { + root, err := r.accessor.RootHdr(tx) + if err != nil { + return err + } + + // Initialize cursor, if queue was empty on previous (without any pages). + if r.state.cursor.Nil() { + head := r.findReadStart(root) + tail := r.accessor.ParsePosition(&root.tail) + + r.state.id = head.id + r.state.cursor.page = head.page + r.state.cursor.off = head.off + r.state.endID = tail.id + } else { + r.state.endID = root.tail.id.Get() + } + return nil +} + +func (r *Reader) findReadStart(root *queuePage) position { + head := r.accessor.ParsePosition(&root.read) + if head.page != 0 { + return head + } + return r.accessor.ParsePosition(&root.head) +} diff --git a/vendor/github.com/elastic/go-txfile/pq/trace.go b/vendor/github.com/elastic/go-txfile/pq/trace.go new file mode 100644 index 00000000000..2acef0de01a --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/trace.go @@ -0,0 +1,10 @@ +package pq + +type tracer interface { + Println(...interface{}) + Printf(string, ...interface{}) +} + +var ( + logTracer tracer +) diff --git a/vendor/github.com/elastic/go-txfile/pq/trace_disabled.go b/vendor/github.com/elastic/go-txfile/pq/trace_disabled.go new file mode 100644 index 00000000000..4bdc19a8a49 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/trace_disabled.go @@ -0,0 +1,9 @@ +// +build !tracing + +package pq + +func pushTracer(t tracer) {} +func popTracer() {} + +func traceln(vs ...interface{}) {} +func tracef(fmt string, vs ...interface{}) {} diff --git a/vendor/github.com/elastic/go-txfile/pq/trace_enabled.go b/vendor/github.com/elastic/go-txfile/pq/trace_enabled.go new file mode 100644 index 00000000000..75340aacb0c --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/trace_enabled.go @@ -0,0 +1,36 @@ +// +build tracing + +package pq + +import ( + "github.com/elastic/go-txfile/internal/tracelog" +) + +var ( + tracers []tracer + activeTracer tracer +) + +func init() { + logTracer = tracelog.Get("pq") + activeTracer = logTracer +} + +func pushTracer(t tracer) { + tracers = append(tracers, activeTracer) + activeTracer = t +} + +func popTracer() { + i := len(tracers) - 1 + activeTracer = tracers[i] + tracers = tracers[:i] +} + +func traceln(vs ...interface{}) { + activeTracer.Println(vs...) +} + +func tracef(s string, vs ...interface{}) { + activeTracer.Printf(s, vs...) +} diff --git a/vendor/github.com/elastic/go-txfile/pq/util.go b/vendor/github.com/elastic/go-txfile/pq/util.go new file mode 100644 index 00000000000..5d7351e357d --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/util.go @@ -0,0 +1,39 @@ +package pq + +import "github.com/elastic/go-txfile" + +func getPage(tx *txfile.Tx, id txfile.PageID) ([]byte, error) { + page, err := tx.Page(id) + if err != nil { + return nil, err + } + + return page.Bytes() +} + +func withPage(tx *txfile.Tx, id txfile.PageID, fn func([]byte) error) error { + page, err := getPage(tx, id) + if err != nil { + return err + } + return fn(page) +} + +func readPageByID(accessor *access, pool *pagePool, id txfile.PageID) (*page, error) { + tx := accessor.BeginRead() + defer tx.Close() + + var page *page + return page, withPage(tx, id, func(buf []byte) error { + page = pool.NewPageWith(id, buf) + return nil + }) +} + +func idLess(a, b uint64) bool { + return int64(a-b) < 0 +} + +func idLessEq(a, b uint64) bool { + return a == b || idLess(a, b) +} diff --git a/vendor/github.com/elastic/go-txfile/pq/writer.go b/vendor/github.com/elastic/go-txfile/pq/writer.go new file mode 100644 index 00000000000..27498891197 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/pq/writer.go @@ -0,0 +1,327 @@ +package pq + +import ( + "github.com/elastic/go-txfile" + "github.com/elastic/go-txfile/internal/cleanup" +) + +// Writer is used to push new events onto the queue. +// The writer uses a write buffer, which is flushed once the buffer is full +// or if Flush is called. +// Only complete events are flushed. If an event is bigger then the configured write buffer, +// the write buffer will grow with the event size. +type Writer struct { + active bool + + accessor *access + flushCB func(uint) + + state writeState +} + +type writeState struct { + buf *buffer + + activeEventCount uint // count number of finished events since last flush + totalEventCount uint + totalAllocPages uint + + eventID uint64 + eventBytes int +} + +const defaultMinPages = 5 + +func newWriter( + accessor *access, + pagePool *pagePool, + writeBuffer uint, + end position, + flushCB func(uint), +) (*Writer, error) { + pageSize := accessor.PageSize() + if pageSize <= 0 { + return nil, errInvalidPagesize + } + + pages := int(writeBuffer) / pageSize + if pages <= defaultMinPages { + pages = defaultMinPages + } + + var tail *page + if end.page != 0 { + traceln("writer load endpage: ", end) + + page := end.page + off := end.off + + var err error + tail, err = readPageByID(accessor, pagePool, page) + if err != nil { + return nil, err + } + + tail.Meta.EndOff = uint32(off) + } + + w := &Writer{ + active: true, + accessor: accessor, + state: writeState{ + buf: newBuffer(pagePool, tail, pages, pageSize, szEventPageHeader), + eventID: end.id, + }, + flushCB: flushCB, + } + + // init buffer with 'first' event to be written + w.state.buf.ReserveHdr(szEventHeader) + return w, nil +} + +func (w *Writer) close() error { + if !w.active { + return nil + } + + err := w.doFlush() + if err != nil { + return err + } + + w.active = false + w.state.buf = nil + return err +} + +func (w *Writer) Write(p []byte) (int, error) { + if !w.active { + return 0, errClosed + } + + if w.state.buf.Avail() <= len(p) { + if err := w.doFlush(); err != nil { + return 0, err + } + } + + w.state.buf.Append(p) + w.state.eventBytes += len(p) + + return len(p), nil +} + +// Next is used to indicate the end of the current event. +// If write is used with a streaming encoder, the buffers +// of the actual writer must be flushed before calling Next on this writer. +// Upon next, the queue writer will add the event framing header and footer. +func (w *Writer) Next() error { + if !w.active { + return errClosed + } + + // finalize current event in buffer and prepare next event + hdr := castEventHeader(w.state.buf.ActiveEventHdr()) + hdr.sz.Set(uint32(w.state.eventBytes)) + w.state.buf.CommitEvent(w.state.eventID) + w.state.buf.ReserveHdr(szEventHeader) + w.state.eventBytes = 0 + w.state.eventID++ + w.state.activeEventCount++ + + // check if we need to flush + if w.state.buf.Avail() <= szEventHeader { + if err := w.doFlush(); err != nil { + return err + } + } + + return nil +} + +// Flush flushes the write buffer. Returns an error if the queue is closed, +// some error occurred or no more space is available in the file. +func (w *Writer) Flush() error { + if !w.active { + return errClosed + } + return w.doFlush() +} + +func (w *Writer) doFlush() error { + start, end := w.state.buf.Pages() + if start == nil || start == end { + return nil + } + + traceln("writer flush", w.state.activeEventCount) + + // unallocated points to first page in list that must be allocated. All + // pages between unallocated and end require a new page to be allocated. + var unallocated *page + for current := start; current != end; current = current.Next { + if !current.Assigned() { + unallocated = current + break + } + } + + tx := w.accessor.BeginWrite() + defer tx.Close() + + rootPage, queueHdr, err := w.accessor.LoadRootPage(tx) + if err != nil { + return err + } + + traceQueueHeader(queueHdr) + + ok := false + allocN, err := allocatePages(tx, unallocated, end) + if err != nil { + return err + } + linkPages(start, end) + defer cleanup.IfNot(&ok, func() { unassignPages(unallocated, end) }) + + traceln("write queue pages") + last, err := flushPages(tx, start, end) + if err != nil { + return err + } + + // update queue root + w.updateRootHdr(queueHdr, start, last, allocN) + rootPage.MarkDirty() + + err = tx.Commit() + if err != nil { + return err + } + + // mark write as success -> no error-cleanup required + ok = true + + // remove dirty flag from all published pages + for current := start; current != end; current = current.Next { + current.UnmarkDirty() + } + + w.state.buf.Reset(last) + + activeEventCount := w.state.activeEventCount + w.state.totalEventCount += activeEventCount + w.state.totalAllocPages += uint(allocN) + + traceln("Write buffer flushed. Total events: %v, total pages allocated: %v", + w.state.totalEventCount, + w.state.totalAllocPages) + + w.state.activeEventCount = 0 + if w.flushCB != nil { + w.flushCB(activeEventCount) + } + + return nil +} + +func (w *Writer) updateRootHdr(hdr *queuePage, start, last *page, allocated int) { + if hdr.head.offset.Get() == 0 { + w.accessor.WritePosition(&hdr.head, position{ + page: start.Meta.ID, + off: int(start.Meta.FirstOff), + id: start.Meta.FirstID, + }) + } + + hdr.inuse.Set(hdr.inuse.Get() + uint64(allocated)) + + endOff := int(last.Meta.EndOff) + if last == w.state.buf.eventHdrPage { + endOff = w.state.buf.eventHdrOffset + } + + w.accessor.WritePosition(&hdr.tail, position{ + page: last.Meta.ID, + off: endOff, + id: w.state.eventID, + }) + + traceln("writer: update queue header") + traceQueueHeader(hdr) +} + +func allocatePages(tx *txfile.Tx, start, end *page) (int, error) { + if start == nil { + return 0, nil + } + + allocN := 0 + for current := start; current != end; current = current.Next { + allocN++ + } + + tracef("allocate %v queue pages\n", allocN) + + txPages, err := tx.AllocN(allocN) + if err != nil { + return 0, err + } + + // assign new page IDs + for current, i := start, 0; current != end; current, i = current.Next, i+1 { + current.Meta.ID = txPages[i].ID() + } + + return allocN, nil +} + +// unassignPages removes page assignments from all pages between start and end, +// so to mark these pages as 'not allocated'. +func unassignPages(start, end *page) { + for current := start; current != end; current = current.Next { + current.Meta.ID = 0 + } +} + +// Update page headers to point to next page in the list. +func linkPages(start, end *page) { + for current := start; current.Next != end; current = current.Next { + tracef("link page %v -> %v\n", current.Meta.ID, current.Next.Meta.ID) + current.SetNext(current.Next.Meta.ID) + } +} + +// flushPages flushes all pages in the list of pages and returns the last page +// being flushed. +func flushPages(tx *txfile.Tx, start, end *page) (*page, error) { + last := start + for current := start; current != end; current = current.Next { + last = current + + err := flushPage(tx, current) + if err != nil { + return nil, err + } + } + + return last, nil +} + +func flushPage(tx *txfile.Tx, page *page) error { + page.UpdateHeader() + tracePageHeader(page.Meta.ID, castEventPageHeader(page.Data)) + + diskPage, err := tx.Page(page.Meta.ID) + if err != nil { + return err + } + + err = diskPage.SetBytes(page.Data) + if err != nil { + return err + } + + return diskPage.Flush() +} diff --git a/vendor/github.com/elastic/go-txfile/region.go b/vendor/github.com/elastic/go-txfile/region.go new file mode 100644 index 00000000000..a20a2f143c0 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/region.go @@ -0,0 +1,234 @@ +package txfile + +import ( + "sort" +) + +// region values represent a continuous set of pages. +type region struct { + id PageID + count uint32 +} + +type regionList []region + +// PageIDs represent pages. The minimal page size is 512Bytes + +// all contents in a file must be addressable by offset. This gives us +// 9 bytes to store additional flags or value in the entry. + +const ( + maxRegionEncSz = 8 + 4 + entryBits = 9 // region entry header size in bits + entryOverflow = (1 << 8) - 1 // overflow marker == all counter bits set + entryOverflowMarker = uint64(entryOverflow) << (64 - entryBits) + entryCounterMask = uint32(((1 << (entryBits - 1)) - 1)) + entryMetaFlag = 1 << 63 // indicates the region holding pages used by the meta-area +) + +func (l regionList) Len() int { + return len(l) +} + +func (l *regionList) Add(reg region) { + *l = append(*l, reg) +} + +func (l regionList) Sort() { + sort.Slice(l, func(i, j int) bool { + return l[i].Before(l[j]) + }) +} + +func (l *regionList) MergeAdjacent() { + if len(*l) <= 1 { + return + } + + tmp := (*l)[:1] + i := 0 + for _, r := range (*l)[1:] { + if regionsMergable(tmp[i], r) { + tmp[i] = mergeRegions(tmp[i], r) + } else { + tmp = append(tmp, r) + i = i + 1 + } + } + *l = tmp +} + +func (l regionList) CountPagesUpTo(id PageID) (count uint) { + for _, reg := range l { + if reg.id >= id { + break + } + + start, end := reg.Range() + if end > id { + end = id + } + count += uint(end - start) + } + return +} + +func (l regionList) CountPages() (count uint) { + for _, reg := range l { + count += uint(reg.count) + } + return +} + +func (l regionList) EachPage(fn func(PageID)) { + for _, reg := range l { + reg.EachPage(fn) + } +} + +func (l regionList) EachRegion(fn func(region)) { + for _, reg := range l { + fn(reg) + } +} + +func (l regionList) PageIDs() (ids idList) { + l.EachPage(ids.Add) + return +} + +func (r region) Start() PageID { return r.id } +func (r region) End() PageID { return r.id + PageID(r.count) } +func (r region) Range() (PageID, PageID) { return r.Start(), r.End() } +func (r region) InRange(id PageID) bool { return r.Start() <= id && id < r.End() } + +func (r region) SplitAt(id PageID) region { + start, end := r.Range() + if id <= start || end < id { + return region{} + } + + if end > id { + end = id + } + + return region{id: start, count: uint32(end - start)} +} + +func (r region) EachPage(fn func(PageID)) { + for id, end := r.Range(); id != end; id++ { + fn(id) + } +} + +func (r region) Before(other region) bool { + return r.id < other.id +} + +func (r region) Precedes(other region) bool { + return r.id+PageID(r.count) == other.id +} + +func mergeRegions(a, b region) region { + return region{id: a.id, count: a.count + b.count} +} + +// mergeRegionLists merges 2 sorter regionLists into a new sorted region list. +// Adjacent regions will be merged into a single region as well. +func mergeRegionLists(a, b regionList) regionList { + L := len(a) + len(b) + if L == 0 { + return nil + } + + final := make(regionList, 0, L) + for len(a) > 0 && len(b) > 0 { + if a[0].Before(b[0]) { + final, a = append(final, a[0]), a[1:] + } else { + final, b = append(final, b[0]), b[1:] + } + } + + // copy leftover elements + final = append(final, a...) + final = append(final, b...) + + final.MergeAdjacent() + + return final +} + +// regionsMergable checks region a directly precedes regions b and +// the region counter will not overflow. +func regionsMergable(a, b region) bool { + if !a.Before(b) { + a, b = b, a + } + return a.Precedes(b) && (a.count+b.count) > a.count +} + +// optimizeRegionList sorts and merges adjacent regions. +func optimizeRegionList(reg *regionList) { + initLen := reg.Len() + reg.Sort() + reg.MergeAdjacent() + if l := reg.Len(); initLen > l { + tmp := make(regionList, l, l) + copy(tmp, *reg) + *reg = tmp + } +} + +// (de-)serialization +// ------------------ + +func regionEncodingSize(r region) int { + if r.count < entryOverflow { + return (&u64{}).Len() + } + return (&u64{}).Len() + (&u32{}).Len() +} + +func encodeRegion(buf []byte, isMeta bool, reg region) int { + flag := uint64(0) + if isMeta { + flag = entryMetaFlag + } + + payload := buf + entry := castU64(payload) + payload = payload[entry.Len():] + + if reg.count < entryOverflow { + count := uint64(reg.count) << (64 - entryBits) + entry.Set(flag | count | uint64(reg.id)) + } else { + count := castU32(payload) + payload = payload[count.Len():] + + entry.Set(flag | entryOverflowMarker | uint64(reg.id)) + count.Set(reg.count) + } + + return len(buf) - len(payload) +} + +func decodeRegion(buf []byte) (bool, region, int) { + payload := buf + entry := castU64(payload) + value := entry.Get() + payload = payload[entry.Len():] + + id := PageID((value << entryBits) >> entryBits) + isMeta := (entryMetaFlag & value) == entryMetaFlag + count := uint32(value>>(64-entryBits)) & entryCounterMask + switch count { + case 0: + count = 1 + case entryOverflow: + extra := castU32(payload) + count, payload = extra.Get(), payload[extra.Len():] + } + + return isMeta, region{id: id, count: count}, len(buf) - len(payload) +} diff --git a/vendor/github.com/elastic/go-txfile/trace.go b/vendor/github.com/elastic/go-txfile/trace.go new file mode 100644 index 00000000000..c6c062a6edb --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/trace.go @@ -0,0 +1,10 @@ +package txfile + +type tracer interface { + Println(...interface{}) + Printf(string, ...interface{}) +} + +var ( + logTracer tracer +) diff --git a/vendor/github.com/elastic/go-txfile/trace_disabled.go b/vendor/github.com/elastic/go-txfile/trace_disabled.go new file mode 100644 index 00000000000..00bb45c55c0 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/trace_disabled.go @@ -0,0 +1,9 @@ +// +build !tracing + +package txfile + +func pushTracer(t tracer) {} +func popTracer() {} + +func traceln(vs ...interface{}) {} +func tracef(fmt string, vs ...interface{}) {} diff --git a/vendor/github.com/elastic/go-txfile/trace_enabled.go b/vendor/github.com/elastic/go-txfile/trace_enabled.go new file mode 100644 index 00000000000..04c1e292047 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/trace_enabled.go @@ -0,0 +1,36 @@ +// +build tracing + +package txfile + +import ( + "github.com/elastic/go-txfile/internal/tracelog" +) + +var ( + tracers []tracer + activeTracer tracer +) + +func init() { + logTracer = tracelog.Get("txfile") + activeTracer = logTracer +} + +func pushTracer(t tracer) { + tracers = append(tracers, activeTracer) + activeTracer = t +} + +func popTracer() { + i := len(tracers) - 1 + activeTracer = tracers[i] + tracers = tracers[:i] +} + +func traceln(vs ...interface{}) { + activeTracer.Println(vs...) +} + +func tracef(s string, vs ...interface{}) { + activeTracer.Printf(s, vs...) +} diff --git a/vendor/github.com/elastic/go-txfile/tx.go b/vendor/github.com/elastic/go-txfile/tx.go new file mode 100644 index 00000000000..69eded22e20 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/tx.go @@ -0,0 +1,624 @@ +package txfile + +import ( + "fmt" + "sync" + + "github.com/elastic/go-txfile/internal/cleanup" +) + +// Tx provides access to pages in a File. +// A transaction MUST always be closed, so to guarantee locks being released as +// well. +type Tx struct { + flags txFlags + file *File + lock sync.Locker + writeSync *txWriteSync + rootID PageID + dataEndID PageID + + // pages accessed by the transaction + pages map[PageID]*Page + + // allocation/free state + alloc txAllocState + + // scheduled WAL updates + wal txWalState +} + +// TxOptions adds some per transaction options user can set. +type TxOptions struct { + // Readonly transaction. + Readonly bool + + // Allow write transaction to allocate meta pages from overflow area. + // Potentially increasing the file size past the configured max size. + // This setting should only be used to guarantee progress when having a + // transaction only freeing pages. + // Later transactions will try to release pages from the overflow area and + // truncate the file, such that we have a chance to operate within max-size + // limits again. + EnableOverflowArea bool + + // MetaAreaGrowPercentage sets the percentage of meta pages in use, until + // the meta-area grows again. The value must be between 0 and 100. + // The default value is 80%. + MetaAreaGrowPercentage int + + // Number of pages in wal overwrite log to automatically trigger + // CheckpointWAL on commit. + WALLimit uint +} + +type txFlags struct { + readonly bool + active bool + checkpoint bool // mark wal checkpoint has been applied +} + +func newTx(file *File, lock sync.Locker, settings TxOptions) *Tx { + meta := file.getMetaPage() + + rootID := meta.root.Get() + dataEndMarker := meta.dataEndMarker.Get() + + tx := &Tx{ + flags: txFlags{ + readonly: settings.Readonly, + active: true, + }, + file: file, + lock: lock, + rootID: rootID, + dataEndID: dataEndMarker, + + pages: map[PageID]*Page{}, + } + + if !settings.Readonly { + tx.writeSync = newTxWriteSync() + tx.alloc = file.allocator.makeTxAllocState( + settings.EnableOverflowArea, + settings.MetaAreaGrowPercentage, + ) + tx.wal = file.wal.makeTxWALState(settings.WALLimit) + } + + return tx +} + +// Writable returns true if the transaction supports file modifications. +func (tx *Tx) Writable() bool { + return !tx.flags.readonly +} + +// Readonly returns true if no modifications to the page are allowed. Trying to +// write to a readonly page might result in a non-recoverable panic. +func (tx *Tx) Readonly() bool { + return tx.flags.readonly +} + +// Active returns true if the transaction can still be used to access pages. +// A transaction becomes inactive after Close, Commit or Rollback. +// Errors within a transaction might inactivate the transaction as well. +// When encountering errors, one should check if the transaction still can be used. +func (tx *Tx) Active() bool { + return tx.flags.active +} + +// PageSize returns the file page size. +func (tx *Tx) PageSize() int { + return int(tx.file.allocator.pageSize) +} + +// Root returns the data root page id. This ID must be set via SetRoot +// to indicate the start of application data to later transactions. +// On new files, the default root is 0, as no application data are stored yet. +func (tx *Tx) Root() PageID { + return tx.rootID +} + +// SetRoot sets the new root page id, indicating the new start of application +// data. SetRoot should be set by the first write transaction, when the file is +// generated first. +func (tx *Tx) SetRoot(id PageID) { + tx.rootID = id +} + +// RootPage returns the application data root page, if the root id has been set +// in the past. Returns nil, if no root page is set. +func (tx *Tx) RootPage() (*Page, error) { + if tx.rootID < 2 { + return nil, nil + } + return tx.Page(tx.rootID) +} + +// Rollback rolls back and closes the current transaction. Rollback returns an +// error if the transaction has already been closed by Close, Rollback or +// Commit. +func (tx *Tx) Rollback() error { + tracef("rollback transaction: %p\n", tx) + return tx.finishWith(tx.rollbackChanges) +} + +// Commit commits the current transaction to file. The commit step needs to +// take the Exclusive Lock, waiting for readonly transactions to be Closed. +// Returns an error if the transaction has already been closed by Close, +// Rollback or Commit. +func (tx *Tx) Commit() error { + tracef("commit transaction: %p\n", tx) + return tx.finishWith(tx.commitChanges) +} + +// Close closes the transaction, releasing any locks held by the transaction. +// It is safe to call Close multiple times. Close on an inactive transaction +// will be ignored. +// A non-committed read-write transaction will be rolled back on close. +// To guaranteed the File and Locking state being valid, even on panic or early return on error, +// one should also defer the Close operation on new transactions. +// For example: +// +// tx := f.Begin() +// defer tx.Close() +// +// err := some operation +// if err != nil { +// return err +// } +// +// return tx.Commit() +// +func (tx *Tx) Close() error { + tracef("close transaction: %p\n", tx) + if !tx.flags.active { + return nil + } + return tx.finishWith(tx.rollbackChanges) +} + +// CheckpointWAL copies all overwrite pages contents into the original pages. +// Only already committed pages from older transactions will be overwritten. +// Checkpointing only copies the contents and marks the overwrite pages as +// freed. The final transaction Commit is required, to propage the WAL mapping changes +// to all other transactions. +// Dirty pages are not overwritten. Manual checkpointing should be executed at +// the end of a transaction, right before committing, so to reduce writes if +// contents is to be overwritten anyways. +func (tx *Tx) CheckpointWAL() error { + if err := tx.canWrite(); err != nil { + return err + } + return tx.doCheckpointWAL() +} + +func (tx *Tx) doCheckpointWAL() error { + if tx.flags.checkpoint { + return nil + } + + // collect page ids that would have an old WAL page + // entry still alive after this transaction. + ids := make([]PageID, 0, len(tx.file.wal.mapping)) + walIDS := make([]PageID, 0, len(tx.file.wal.mapping)) + for id, walID := range tx.file.wal.mapping { + page := tx.pages[id] + if page != nil { + if page.flags.dirty { + // wal pages of dirty pages will be freed on flush -> do not copy + continue + } + } + + ids = append(ids, id) + walIDS = append(walIDS, walID) + } + + if len(ids) == 0 { + return nil + } + + // XXX: Some OS/filesystems might lock up when writing to file + // from mmapped area. + // -> Copy contents into temporary buffer, such that + // write operations are not backed by mmapped pages from same file. + pageSize := int(tx.PageSize()) + writeBuffer := make([]byte, pageSize*len(ids)) + for i := range ids { + id, walID := ids[i], walIDS[i] + + contents := tx.file.mmapedPage(walID) + if contents == nil { + panic("invalid WAL mapping") + } + + tracef("checkpoint copy from WAL page %v -> %v\n", walID, id) + + n := copy(writeBuffer, contents) + buf := writeBuffer[:n] + writeBuffer = writeBuffer[n:] + + tx.file.writer.Schedule(tx.writeSync, id, buf) + tx.freeWALID(id, walID) + } + + tx.flags.checkpoint = true + return nil +} + +func (tx *Tx) finishWith(fn func() error) error { + if !tx.flags.active { + return errTxFinished + } + defer tx.close() + + if !tx.flags.readonly { + return fn() + } + return nil +} + +func (tx *Tx) close() { + tx.flags.active = false + tx.pages = nil + tx.alloc = txAllocState{} + tx.wal = txWalState{} + tx.writeSync = nil + tx.file = nil + tx.lock.Unlock() +} + +func (tx *Tx) commitChanges() error { + commitOK := false + defer cleanup.IfNot(&commitOK, cleanup.IgnoreError(tx.rollbackChanges)) + + err := tx.tryCommitChanges() + if commitOK = err == nil; !commitOK { + return err + } + + traceMetaPage(tx.file.getMetaPage()) + return err +} + +// tryCommitChanges attempts to write flush all pages written and update the +// files state by writing the new meta data and finally the meta page. +// So to keep the most recent transaction successfully committed usable/consistent, +// tryCommitChanges is not allowed to re-use any pages freed within this transaction. +// +// rough commit sequence: +// 1. get pending lock, so no new readers can be started +// 2. flush all dirty pages. +// - dirty pages overwriting existing contents will, will allocate +// a new WAL page to be written to +// - If dirty page already has an WAL page, overwrite the original page and +// return WAL page to allocator +// 3. if WAL was updated (pages added/removed): +// - free pages holding the old WAL mapping +// - write new WAL mapping +// 4. if pages have been freed/allocated: +// - free pages holding the old free list entries +// - write new free list +// 5. fsync, to ensure all updates have been executed before updating the meta page +// 6. acquire esclusive lock -> no more readers/writers accessing the file +// 6. update the meta page +// 7. fsync +// 8. update internal structures +// 9. release locks +func (tx *Tx) tryCommitChanges() error { + pending, exclusive := tx.file.locks.Pending(), tx.file.locks.Exclusive() + + var newMetaBuf metaBuf + newMeta := newMetaBuf.cast() + *newMeta = *tx.file.getMetaPage() // init new meta header from current active meta header + newMeta.txid.Set(1 + newMeta.txid.Get()) // inc txid + newMeta.root.Set(tx.rootID) // update data root + + // give concurrent read transactions a chance to complete, but don't allow + // for new read transactions to start while executing the commit + pending.Lock() + defer pending.Unlock() + + // On function exit wait on writer to finish outstanding operations, in case + // we have to return early on error. On success, this is basically a no-op. + defer tx.writeSync.Wait() + + // Flush pages. + if err := tx.Flush(); err != nil { + return fmt.Errorf("dirty pages flushing failed with %v", err) + } + + // 1. finish Tx state updates and free file pages used to hold meta pages + csWAL, err := tx.commitPrepareWAL() + if err != nil { + return err + } + + var csAlloc allocCommitState + tx.file.allocator.fileCommitPrepare(&csAlloc, &tx.alloc) + + // 2. allocate new file pages for new meta data to be written + if err := tx.file.wal.fileCommitAlloc(tx, &csWAL); err != nil { + return err + } + csAlloc.updated = csAlloc.updated || len(csWAL.allocRegions) > 0 + + if err := tx.file.allocator.fileCommitAlloc(&csAlloc); err != nil { + return err + } + + // 3. serialize page mappings and new freelist + err = tx.file.wal.fileCommitSerialize(&csWAL, uint(tx.PageSize()), tx.scheduleWrite) + if err != nil { + return err + } + + err = tx.file.allocator.fileCommitSerialize(&csAlloc, tx.scheduleWrite) + if err != nil { + return err + } + + // 4. sync all new contents and metadata before updating the ondisk meta page. + tx.file.writer.Sync(tx.writeSync) + + // 5. finalize on-disk transaction be writing new meta page. + tx.file.wal.fileCommitMeta(newMeta, &csWAL) + tx.file.allocator.fileCommitMeta(newMeta, &csAlloc) + newMeta.Finalize() + metaID := 1 - tx.file.metaActive + tx.file.writer.Schedule(tx.writeSync, PageID(metaID), newMetaBuf[:]) + tx.file.writer.Sync(tx.writeSync) + + // 6. wait for all pages beeing written and synced, + // before updating in memory state. + if err := tx.writeSync.Wait(); err != nil { + return err + } + + // At this point the transaction has been completed on file level. + // Update internal structures as well, so future transactions + // will use the new serialized transaction state. + + // We have only one active write transaction + freelist is not shared with read transactions + // -> update freelist state before waiting for the exclusive lock to be available + tx.file.allocator.Commit(&csAlloc) + + // Wait for all read transactions to finish before updating global references + // to new contents. + exclusive.Lock() + defer exclusive.Unlock() + + // Update the WAL mapping. + tx.file.wal.Commit(&csWAL) + + // Switch the files active meta page to meta page being written. + tx.file.metaActive = metaID + + // check + apply mmap update. If we fail here, the file and internal + // state is already updated + valid. + // But mmap failed on us -> fatal error + endMarker := tx.file.allocator.data.endMarker + if metaEnd := tx.file.allocator.meta.endMarker; metaEnd > endMarker { + endMarker = metaEnd + } + fileSize := uint(endMarker) * tx.file.allocator.pageSize + if int(fileSize) > len(tx.file.mapped) { + err = tx.file.mmapUpdate() + } + + traceln("tx stats:") + traceln(" available data pages:", tx.file.allocator.DataAllocator().Avail(nil)) + traceln(" available meta pages:", tx.file.allocator.meta.freelist.Avail()) + traceln(" total meta pages:", tx.file.allocator.metaTotal) + traceln(" freelist pages:", len(tx.file.allocator.freelistPages)) + traceln(" wal mapping pages:", len(tx.file.wal.metaPages)) + traceln(" max pages:", tx.file.allocator.maxPages) + traceln(" wal mapped pages:", len(tx.file.wal.mapping)) + + return nil +} + +func (tx *Tx) commitPrepareWAL() (walCommitState, error) { + var st walCommitState + + tx.file.wal.fileCommitPrepare(&st, &tx.wal) + if st.checkpoint { + if err := tx.doCheckpointWAL(); err != nil { + return st, err + } + } + + if st.updated { + tx.metaAllocator().FreeRegions(&tx.alloc, tx.file.wal.metaPages) + } + return st, nil +} + +func (tx *Tx) access(id PageID) []byte { + return tx.file.mmapedPage(id) +} + +func (tx *Tx) scheduleWrite(id PageID, buf []byte) error { + tx.file.writer.Schedule(tx.writeSync, id, buf) + return nil +} + +// rollbackChanges undoes all changes scheduled. +// Potentially changes to be undone: +// 1. WAL: +// - mapping is only updated after ACK. +// - pages have been allocated from meta area -> only restore freelists +// 2. Allocations: +// - restore freelists, by returning allocated page +// ids < old endmarker to freelists +// - restore old end markers. +// - move pages allocated into meta area back into data area +// 3. File: +// - With page flushing or transaction failing late during commit, +// file might have been grown. +// => +// - Truncate file only if pages in overflow area have been allocated. +// - If maxSize == 0, truncate file to old end marker. +func (tx *Tx) rollbackChanges() error { + tx.file.allocator.Rollback(&tx.alloc) + + maxPages := tx.file.allocator.maxPages + if maxPages == 0 { + return nil + } + + // compute endmarker from before running the last transaction + endMarker := tx.file.allocator.meta.endMarker + if dataEnd := tx.file.allocator.data.endMarker; dataEnd > endMarker { + endMarker = dataEnd + } + + sz, err := tx.file.file.Size() + if err != nil { + // getting file size failed. State is valid, but we can not truncate :/ + return err + } + + truncateSz := uint(endMarker) * tx.file.allocator.pageSize + if uint(sz) > uint(truncateSz) { + return tx.file.file.Truncate(int64(truncateSz)) + } + + return nil +} + +// Page accesses a page by ID. Accessed pages are cached. Retrieving a page +// that has already been accessed, will return a pointer to the same Page object. +// Returns an error if the id is known to be invalid or the page has already +// been freed. +func (tx *Tx) Page(id PageID) (*Page, error) { + inBounds := id >= 2 + if tx.flags.readonly { + inBounds = inBounds && id < tx.dataEndID + } else { + inBounds = inBounds && id < tx.file.allocator.data.endMarker + } + if !inBounds { + return nil, errOutOfBounds + } + + if tx.alloc.data.freed.Has(id) || tx.alloc.meta.freed.Has(id) { + return nil, errFreedPage + } + + if p := tx.pages[id]; p != nil { + return p, nil + } + + page := newPage(tx, id) + if walID := tx.file.wal.Get(id); walID != 0 { + page.ondiskID = walID + } + + tx.pages[id] = page + return page, nil +} + +// Alloc allocates a new writable page with yet empty contents. +// Use Load(), Bytes and MarkDirty(), or SetBytes() to fill the page with +// new contents. +// Returns an error if the transaction is readonly or no more space is available. +func (tx *Tx) Alloc() (page *Page, err error) { + if err := tx.canWrite(); err != nil { + return nil, err + } + + err = tx.allocPagesWith(1, func(p *Page) { page = p }) + return +} + +// AllocN allocates n potentially non-contious, yet empty pages. +// Returns an error if the transaction is readonly or no more space is available. +func (tx *Tx) AllocN(n int) (pages []*Page, err error) { + if err := tx.canWrite(); err != nil { + return nil, err + } + + if n <= 0 { + return nil, nil + } + + pages, i := make([]*Page, n), 0 + err = tx.allocPagesWith(n, func(page *Page) { + pages[i], i = page, i+1 + }) + if err != nil { + return nil, err + } + return pages, nil +} + +func (tx *Tx) dataAllocator() *dataAllocator { + return tx.file.allocator.DataAllocator() +} + +func (tx *Tx) metaAllocator() *metaAllocator { + return tx.file.allocator.MetaAllocator() +} + +func (tx *Tx) walAllocator() *walAllocator { + return tx.file.allocator.WALPageAllocator() +} + +func (tx *Tx) allocPagesWith(n int, fn func(*Page)) error { + count := tx.dataAllocator().AllocRegionsWith(&tx.alloc, uint(n), func(reg region) { + reg.EachPage(func(id PageID) { + page := newPage(tx, id) + page.flags.new = true + tx.pages[id] = page + fn(page) + }) + }) + if count == 0 { + return errOutOfMemory + } + return nil +} + +func (tx *Tx) freePage(id PageID) { + tx.dataAllocator().Free(&tx.alloc, id) +} + +func (tx *Tx) allocWALID(orig PageID) PageID { + id := tx.walAllocator().Alloc(&tx.alloc) + if id != 0 { + tx.wal.Set(orig, id) + } + return id +} + +func (tx *Tx) freeWALID(id, walID PageID) { + tx.walAllocator().Free(&tx.alloc, walID) + tx.wal.Release(id) +} + +// Flush flushes all dirty pages within the transaction. +func (tx *Tx) Flush() error { + if err := tx.canWrite(); err != nil { + return err + } + + for _, page := range tx.pages { + if err := page.doFlush(); err != nil { + return err + } + } + return nil +} + +func (tx *Tx) canWrite() error { + if !tx.flags.active { + return errTxFinished + } + if tx.flags.readonly { + return errTxReadonly + } + return nil +} diff --git a/vendor/github.com/elastic/go-txfile/txfiletest/txfiletest.go b/vendor/github.com/elastic/go-txfile/txfiletest/txfiletest.go new file mode 100644 index 00000000000..aa1b78b22b7 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/txfiletest/txfiletest.go @@ -0,0 +1,91 @@ +// Package txfiletest provides utilities for testing on top of txfile. +package txfiletest + +import ( + "io/ioutil" + "os" + "path" + + "github.com/elastic/go-txfile" + "github.com/elastic/go-txfile/internal/cleanup" +) + +// TestFile wraps a txfile.File structure for testing. +type TestFile struct { + *txfile.File + t testT + Path string + opts txfile.Options +} + +type testT interface { + Error(...interface{}) + Fatal(...interface{}) +} + +// SetupTestFile creates a new testfile in a temporary directory. +// The teardown function will remove the directory and the temporary file. +func SetupTestFile(t testT, opts txfile.Options) (tf *TestFile, teardown func()) { + if opts.PageSize == 0 { + opts.PageSize = 4096 + } + + ok := false + path, cleanPath := SetupPath(t, "") + defer cleanup.IfNot(&ok, cleanPath) + + tf = &TestFile{Path: path, t: t, opts: opts} + tf.Open() + + ok = true + return tf, func() { + tf.Close() + cleanPath() + } +} + +// Reopen tries to close and open the file again. +func (f *TestFile) Reopen() { + f.Close() + f.Open() +} + +// Close the test file. +func (f *TestFile) Close() { + if f.File != nil { + if err := f.File.Close(); err != nil { + f.t.Fatal("close failed on reopen") + } + f.File = nil + } +} + +// Open opens the file if it has been closed. +// The File pointer will be changed. +func (f *TestFile) Open() { + if f.File != nil { + return + } + + tmp, err := txfile.Open(f.Path, os.ModePerm, f.opts) + if err != nil { + f.t.Fatal("reopen failed") + } + f.File = tmp +} + +// SetupPath creates a temporary directory for testing. +// Use the teardown function to remove the directory again. +func SetupPath(t testT, file string) (dir string, teardown func()) { + dir, err := ioutil.TempDir("", "") + if err != nil { + t.Fatal(err) + } + + if file == "" { + file = "test.dat" + } + return path.Join(dir, file), func() { + os.RemoveAll(dir) + } +} diff --git a/vendor/github.com/elastic/go-txfile/util.go b/vendor/github.com/elastic/go-txfile/util.go new file mode 100644 index 00000000000..61f044dcee9 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/util.go @@ -0,0 +1,141 @@ +package txfile + +import ( + "math/bits" +) + +// pagingWriter supports writing entries into a linked (pre-allocated) list of +// pages. +type pagingWriter struct { + ids idList + buf []byte + pageSize uint + extraHeader uint + + onPage func(id PageID, buf []byte) error + + // current page state + i int + off uint + hdr *listPage + page []byte + payload []byte + count uint32 +} + +const maxUint uint = ^uint(0) + +func newPagingWriter( + ids idList, + pageSize uint, + extraHeader uint, + onPage func(id PageID, buf []byte) error, +) *pagingWriter { + if len(ids) == 0 { + return nil + } + + buf := make([]byte, len(ids)*int(pageSize)) + + // prelink all pages, in case some are not written to + off := 0 + for _, id := range ids[1:] { + hdr, _ := castListPage(buf[off:]) + hdr.next.Set(id) + off += int(pageSize) + } + + w := &pagingWriter{ + ids: ids, + buf: buf, + pageSize: pageSize, + extraHeader: extraHeader, + onPage: onPage, + } + w.prepareNext() + return w +} + +func (w *pagingWriter) Write(entry []byte) error { + if w == nil { + return nil + } + + if len(w.payload) < len(entry) { + if err := w.flushCurrent(); err != nil { + return err + } + } + + n := copy(w.payload, entry) + w.payload = w.payload[n:] + w.count++ + return nil +} + +func (w *pagingWriter) Flush() error { + if w == nil { + return nil + } + + if err := w.finalizePage(); err != nil { + return err + } + + for w.i < len(w.ids) { + // update to next page + if err := w.prepareNext(); err != nil { + return err + } + + if err := w.finalizePage(); err != nil { + return err + } + } + + return nil +} + +func (w *pagingWriter) flushCurrent() (err error) { + if err = w.finalizePage(); err == nil { + err = w.prepareNext() + } + return +} + +func (w *pagingWriter) finalizePage() error { + w.hdr.count.Set(w.count) + if w.onPage != nil { + if err := w.onPage(w.ids[w.i], w.page); err != nil { + return err + } + } + + w.count = 0 + w.off += w.pageSize + w.i++ + return nil +} + +func (w *pagingWriter) prepareNext() error { + if w.i >= len(w.ids) { + return errOutOfMemory + } + w.page = w.buf[w.off : w.off+w.pageSize] + w.hdr, w.payload = castListPage(w.page) + w.payload = w.payload[w.extraHeader:] + return nil +} + +func isPowerOf2(v uint64) bool { + // an uint is a power of two if exactly one bit is set -> + return v > 0 && (v&(v-1)) == 0 +} + +// nextPowerOf2 computes the next power of two value of `u`, such that +// nextPowerOf2(u) > u +// The input value must not have the highest bit being set. +func nextPowerOf2(u uint64) uint64 { + b := uint64(bits.LeadingZeros64(u)) + return uint64(1) << (64 - b) +} diff --git a/vendor/github.com/elastic/go-txfile/vfs.go b/vendor/github.com/elastic/go-txfile/vfs.go new file mode 100644 index 00000000000..3fbb9ed5c91 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/vfs.go @@ -0,0 +1,42 @@ +package txfile + +import ( + "io" + "os" +) + +type vfsFile interface { + io.Closer + io.WriterAt + io.ReaderAt + + Name() string + Size() (int64, error) + Sync() error + Truncate(int64) error + + Lock(exclusive, blocking bool) error + Unlock() error + + MMap(sz int) ([]byte, error) + MUnmap([]byte) error +} + +type osFile struct { + *os.File + state osFileState +} + +func openOSFile(path string, mode os.FileMode) (*osFile, error) { + flags := os.O_RDWR | os.O_CREATE + f, err := os.OpenFile(path, flags, mode) + return &osFile{File: f}, err +} + +func (o *osFile) Size() (int64, error) { + stat, err := o.File.Stat() + if err != nil { + return -1, err + } + return stat.Size(), nil +} diff --git a/vendor/github.com/elastic/go-txfile/vfs_unix.go b/vendor/github.com/elastic/go-txfile/vfs_unix.go new file mode 100644 index 00000000000..ecea5ab094b --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/vfs_unix.go @@ -0,0 +1,33 @@ +// +build darwin dragonfly freebsd linux netbsd openbsd solaris + +package txfile + +import ( + "golang.org/x/sys/unix" +) + +type osFileState struct{} + +func (f *osFile) MMap(sz int) ([]byte, error) { + return unix.Mmap(int(f.Fd()), 0, int(sz), unix.PROT_READ, unix.MAP_SHARED) +} + +func (f *osFile) MUnmap(b []byte) error { + return unix.Munmap(b) +} + +func (f *osFile) Lock(exclusive, blocking bool) error { + flags := unix.LOCK_SH + if exclusive { + flags = unix.LOCK_EX + } + if !blocking { + flags |= unix.LOCK_NB + } + + return unix.Flock(int(f.Fd()), flags) +} + +func (f *osFile) Unlock() error { + return unix.Flock(int(f.Fd()), unix.LOCK_UN) +} diff --git a/vendor/github.com/elastic/go-txfile/vfs_windows.go b/vendor/github.com/elastic/go-txfile/vfs_windows.go new file mode 100644 index 00000000000..e5369657b94 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/vfs_windows.go @@ -0,0 +1,98 @@ +package txfile + +import ( + "fmt" + "os" + "reflect" + "unsafe" + + "golang.org/x/sys/windows" + + "github.com/theckman/go-flock" +) + +type osFileState struct { + mmapHandle windows.Handle + lock *flock.Flock +} + +const ( + lockExt = ".lock" +) + +func (f *osFile) MMap(sz int) ([]byte, error) { + szhi, szlo := uint32(sz>>32), uint32(sz) + hdl, err := windows.CreateFileMapping(windows.Handle(f.Fd()), nil, windows.PAGE_READONLY, szhi, szlo, nil) + if hdl == 0 { + return nil, os.NewSyscallError("CreateFileMapping", err) + } + + // map memory + addr, err := windows.MapViewOfFile(hdl, windows.FILE_MAP_READ, 0, 0, uintptr(sz)) + if addr == 0 { + windows.CloseHandle(hdl) + return nil, os.NewSyscallError("MapViewOfFile", err) + } + + f.state.mmapHandle = hdl + + slice := *(*[]byte)(unsafe.Pointer(&reflect.SliceHeader{ + Data: uintptr(addr), + Len: sz, + Cap: sz})) + return slice, nil +} + +func (f *osFile) MUnmap(b []byte) error { + err1 := windows.UnmapViewOfFile(uintptr(unsafe.Pointer(&b[0]))) + b = nil + + err2 := windows.CloseHandle(f.state.mmapHandle) + f.state.mmapHandle = 0 + + if err1 != nil { + return os.NewSyscallError("UnmapViewOfFile", err1) + } else if err2 != nil { + return os.NewSyscallError("CloseHandle", err2) + } + return nil +} + +func (f *osFile) Lock(exclusive, blocking bool) error { + if f.state.lock != nil { + return fmt.Errorf("file %v is already locked", f.Name()) + } + + var ok bool + var err error + lock := flock.NewFlock(f.Name() + lockExt) + if blocking { + err = lock.Lock() + ok = err != nil + } else { + ok, err = lock.TryLock() + } + + if err != nil { + return err + } + if !ok { + return fmt.Errorf("file %v can not be locked right now", f.Name()) + } + + f.state.lock = lock + return nil +} + +func (f *osFile) Unlock() error { + if f.state.lock == nil { + return fmt.Errorf("file %v is not locked", f.Name()) + } + + err := f.state.lock.Unlock() + if err == nil { + f.state.lock = nil + } + + return err +} diff --git a/vendor/github.com/elastic/go-txfile/wal.go b/vendor/github.com/elastic/go-txfile/wal.go new file mode 100644 index 00000000000..123b0c8a9d5 --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/wal.go @@ -0,0 +1,240 @@ +package txfile + +import "unsafe" + +// waLog (write-ahead-log) mapping page ids to overwrite page ids in +// the write-ahead-log. +type waLog struct { + mapping walMapping + metaPages regionList +} + +type txWalState struct { + free pageSet // ids being freed + new walMapping // all wal pages used for overwrites in a transaction + walLimit uint // transaction wal page count -> execute checkpoint when reached +} + +// walCommitState keeps track of changes applied to the wal log during the +// commit. These changes must be recorded for now, as the new wal state must +// not be updated in memory until after the transaction has been commit to disk. +type walCommitState struct { + tx *txWalState + updated bool + checkpoint bool + mapping walMapping // new wal mapping + allocRegions regionList // pre-allocate meta pages for serializing new mapping +} + +type walMapping map[PageID]PageID + +const ( + walHeaderSize = uint(unsafe.Sizeof(walPage{})) + walEntrySize = 14 + + defaultWALLimit = 1000 +) + +func makeWALog() waLog { + return waLog{ + mapping: walMapping{}, + metaPages: nil, + } +} + +func (l *waLog) makeTxWALState(limit uint) txWalState { + if limit == 0 { + // TODO: init wal limit on init, based on max file size + limit = defaultWALLimit + } + + return txWalState{ + walLimit: limit, + } +} + +func (l *waLog) Get(id PageID) PageID { + return l.mapping[id] +} + +func (l *waLog) fileCommitPrepare(st *walCommitState, tx *txWalState) { + st.tx = tx + newWal := createMappingUpdate(l.mapping, tx) + st.checkpoint = tx.walLimit > 0 && uint(len(newWal)) >= tx.walLimit + st.updated = st.checkpoint || tx.Updated() + + if st.checkpoint { + newWal = tx.new + } + st.mapping = newWal +} + +func (l *waLog) fileCommitAlloc(tx *Tx, st *walCommitState) error { + if !st.updated { + return nil + } + + pages := predictWALMappingPages(st.mapping, uint(tx.PageSize())) + if pages > 0 { + st.allocRegions = tx.metaAllocator().AllocRegions(&tx.alloc, pages) + if st.allocRegions == nil { + return errOutOfMemory + } + } + return nil +} + +func (l *waLog) fileCommitSerialize( + st *walCommitState, + pageSize uint, + onPage func(id PageID, buf []byte) error, +) error { + if !st.updated { + return nil + } + return writeWAL(st.allocRegions, pageSize, st.mapping, onPage) +} + +func (l *waLog) fileCommitMeta(meta *metaPage, st *walCommitState) { + if st.updated { + var rootPage PageID + if len(st.allocRegions) > 0 { + rootPage = st.allocRegions[0].id + } + meta.wal.Set(rootPage) + } +} + +func (l *waLog) Commit(st *walCommitState) { + if st.updated { + l.mapping = st.mapping + l.metaPages = st.allocRegions + } +} + +func (l walMapping) empty() bool { + return len(l) == 0 +} + +func (s *txWalState) Release(id PageID) { + s.free.Add(id) + if s.new != nil { + delete(s.new, id) + } +} + +func (s *txWalState) Updated() bool { + return !s.free.Empty() || !s.new.empty() +} + +func (s *txWalState) Set(orig, overwrite PageID) { + if s.new == nil { + s.new = walMapping{} + } + s.new[orig] = overwrite +} + +func createMappingUpdate(old walMapping, tx *txWalState) walMapping { + if !tx.Updated() { + return nil + } + + new := walMapping{} + for id, walID := range old { + if tx.free.Has(id) { + continue + } + if _, exists := tx.new[id]; exists { + continue + } + + new[id] = walID + } + for id, walID := range tx.new { + new[id] = walID + } + + return new +} + +func predictWALMappingPages(m walMapping, pageSize uint) uint { + perPage := walEntriesPerPage(pageSize) + return (uint(len(m)) + perPage - 1) / perPage +} + +func walEntriesPerPage(pageSize uint) uint { + payload := pageSize - walHeaderSize + return payload / walEntrySize +} + +func readWALMapping( + wal *waLog, + access func(PageID) []byte, + root PageID, +) error { + mapping, ids, err := readWAL(access, root) + if err != nil { + return nil + } + + wal.mapping = mapping + wal.metaPages = ids.Regions() + return nil +} + +func readWAL( + access func(PageID) []byte, + root PageID, +) (walMapping, idList, error) { + if root == 0 { + return walMapping{}, nil, nil + } + + mapping := walMapping{} + var metaPages idList + for pageID := root; pageID != 0; { + metaPages.Add(pageID) + node, data := castWalPage(access(pageID)) + if node == nil { + return nil, nil, errOutOfBounds + } + + count := int(node.count.Get()) + pageID = node.next.Get() + + for i := 0; i < count; i++ { + // read node mapping. Only 7 bytes are used per pageID + var k, v pgID + copy(k[0:7], data[0:7]) + copy(v[0:7], data[7:14]) + data = data[14:] + + mapping[k.Get()] = v.Get() + } + } + + return mapping, metaPages, nil +} + +func writeWAL( + to regionList, + pageSize uint, + mapping walMapping, + onPage func(id PageID, buf []byte) error, +) error { + allocPages := to.PageIDs() + writer := newPagingWriter(allocPages, pageSize, 0, onPage) + for id, walID := range mapping { + var k, v pgID + k.Set(id) + v.Set(walID) + + var payload [walEntrySize]byte + copy(payload[0:7], k[0:7]) + copy(payload[7:14], v[0:7]) + if err := writer.Write(payload[:]); err != nil { + return err + } + } + return writer.Flush() +} diff --git a/vendor/github.com/elastic/go-txfile/write.go b/vendor/github.com/elastic/go-txfile/write.go new file mode 100644 index 00000000000..4a56a3b54bf --- /dev/null +++ b/vendor/github.com/elastic/go-txfile/write.go @@ -0,0 +1,277 @@ +package txfile + +import ( + "io" + "sort" + "sync" +) + +type writer struct { + target writable + pageSize uint + + mux sync.Mutex + cond *sync.Cond + done bool + scheduled []writeMsg + scheduled0 [64]writeMsg + fsync []syncMsg + fsync0 [8]syncMsg + + pending int // number of scheduled writes since last sync + published int // number of writes executed since last sync +} + +type writeMsg struct { + sync *txWriteSync + id PageID + buf []byte + fsync bool +} + +type syncMsg struct { + sync *txWriteSync + count int // number of pages to process, before fsyncing +} + +type txWriteSync struct { + err error + wg sync.WaitGroup +} + +type writable interface { + io.WriterAt + Sync() error +} + +func (w *writer) Init(target writable, pageSize uint) { + w.target = target + w.pageSize = pageSize + w.cond = sync.NewCond(&w.mux) + w.scheduled = w.scheduled0[:0] + w.fsync = w.fsync[:0] +} + +func (w *writer) Stop() { + w.mux.Lock() + w.done = true + w.mux.Unlock() + w.cond.Signal() +} + +func (w *writer) Schedule(sync *txWriteSync, id PageID, buf []byte) { + sync.Retain() + traceln("schedule write") + + w.mux.Lock() + defer w.mux.Unlock() + w.scheduled = append(w.scheduled, writeMsg{ + sync: sync, + id: id, + buf: buf, + }) + w.pending++ + + w.cond.Signal() +} + +func (w *writer) Sync(sync *txWriteSync) { + sync.Retain() + traceln("schedule sync") + + w.mux.Lock() + defer w.mux.Unlock() + w.fsync = append(w.fsync, syncMsg{ + sync: sync, + count: w.pending, + }) + w.pending = 0 + + w.cond.Signal() +} + +func (w *writer) Run() error { + var ( + buf [1024]writeMsg + n int + err error + fsync *txWriteSync + done bool + ) + + traceln("start async writer") + defer traceln("stop async writer") + + for { + n, fsync, done = w.nextCommand(buf[:]) + if done { + break + } + + traceln("writer message: ", n, fsync != nil, done) + + // TODO: use vector IO if possible + msgs := buf[:n] + sort.Slice(msgs, func(i, j int) bool { + return msgs[i].id < msgs[j].id + }) + + for _, msg := range msgs { + if err != nil { + traceln("done error") + + msg.sync.err = err + msg.sync.wg.Done() + continue + } + + off := uint64(msg.id) * uint64(w.pageSize) + tracef("write at(id=%v, off=%v, len=%v)\n", msg.id, off, len(msg.buf)) + + err = writeAt(w.target, msg.buf, int64(off)) + if err != nil { + msg.sync.err = err + } + + traceln("done send") + msg.sync.Release() + } + + if fsync != nil { + if err == nil { + if err = w.target.Sync(); err != nil { + fsync.err = err + } + } + + traceln("done fsync") + fsync.Release() + } + + if err != nil { + break + } + } + + if done { + return err + } + + // file still active, but we're facing errors -> stop writing and propagate + // last error to all transactions. + for { + n, fsync, done = w.nextCommand(buf[:]) + if done { + break + } + + traceln("ignoring writer message: ", n, fsync != nil, done) + + for _, msg := range buf[:n] { + msg.sync.err = err + msg.sync.Release() + } + if fsync != nil { + fsync.err = err + fsync.Release() + } + } + + return err +} + +func (w *writer) nextCommand(buf []writeMsg) (int, *txWriteSync, bool) { + w.mux.Lock() + defer w.mux.Unlock() + + traceln("async writer: wait next command") + defer traceln("async writer: received next command") + + for { + if w.done { + return 0, nil, true + } + + max := len(w.scheduled) + if max == 0 && len(w.fsync) == 0 { // no messages + w.cond.Wait() + continue + } + + if l := len(buf); l < max { + max = l + } + + // Check if we need to fsync and adjust `max` number of pages of required. + var sync *txWriteSync + traceln("check fsync: ", len(w.fsync)) + + if len(w.fsync) > 0 { + msg := w.fsync[0] + + // number of outstanding scheduled writes before fsync + outstanding := msg.count - w.published + traceln("outstanding:", outstanding) + + if outstanding <= max { // -> fsync + max, sync = outstanding, msg.sync + + // advance fsync state + w.fsync[0] = syncMsg{} // clear entry, so to potentially clean references from w.fsync0 + w.fsync = w.fsync[1:] + if len(w.fsync) == 0 { + w.fsync = w.fsync0[:0] + } + } + } + + // return buffers to be processed + var n int + scheduled := w.scheduled[:max] + if len(scheduled) > 0 { + n = copy(buf, scheduled) + w.scheduled = w.scheduled[n:] + if len(w.scheduled) == 0 { + w.scheduled = w.scheduled0[:0] + } + } + + if sync == nil { + w.published += n + } else { + w.published = 0 + } + + return n, sync, false + } +} + +func newTxWriteSync() *txWriteSync { + return &txWriteSync{} +} + +func (s *txWriteSync) Retain() { + s.wg.Add(1) +} + +func (s *txWriteSync) Release() { + s.wg.Done() +} + +func (s *txWriteSync) Wait() error { + s.wg.Wait() + return s.err +} + +func writeAt(out io.WriterAt, buf []byte, off int64) error { + for len(buf) > 0 { + n, err := out.WriteAt(buf, off) + if err != nil { + return err + } + + off += int64(n) + buf = buf[n:] + } + return nil +} diff --git a/vendor/github.com/theckman/go-flock/LICENSE b/vendor/github.com/theckman/go-flock/LICENSE new file mode 100644 index 00000000000..aff7d358e24 --- /dev/null +++ b/vendor/github.com/theckman/go-flock/LICENSE @@ -0,0 +1,27 @@ +Copyright (c) 2015, Tim Heckman +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +* Neither the name of linode-netint nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/github.com/theckman/go-flock/README.md b/vendor/github.com/theckman/go-flock/README.md new file mode 100644 index 00000000000..38c794c8122 --- /dev/null +++ b/vendor/github.com/theckman/go-flock/README.md @@ -0,0 +1,40 @@ +# go-flock +[![TravisCI Build Status](https://img.shields.io/travis/theckman/go-flock/master.svg?style=flat)](https://travis-ci.org/theckman/go-flock) +[![GoDoc](https://img.shields.io/badge/godoc-go--flock-blue.svg?style=flat)](https://godoc.org/github.com/theckman/go-flock) +[![License](https://img.shields.io/badge/license-BSD_3--Clause-brightgreen.svg?style=flat)](https://github.com/theckman/go-flock/blob/master/LICENSE) + +`flock` implements a thread-safe sync.Locker interface for file locking. It also +includes a non-blocking TryLock() function to allow locking without blocking execution. + +## License +`flock` is released under the BSD 3-Clause License. See the `LICENSE` file for more details. + +## Go Compatibility +This package makes use of the `context` package that was introduced in Go 1.7. As such, this +package has an implicit dependency on Go 1.7+. + +## Installation +``` +go get -u github.com/theckman/go-flock +``` + +## Usage +```Go +import "github.com/theckman/go-flock" + +fileLock := flock.NewFlock("/var/lock/go-lock.lock") + +locked, err := fileLock.TryLock() + +if err != nil { + // handle locking error +} + +if locked { + // do work + fileLock.Unlock() +} +``` + +For more detailed usage information take a look at the package API docs on +[GoDoc](https://godoc.org/github.com/theckman/go-flock). diff --git a/vendor/github.com/theckman/go-flock/appveyor.yml b/vendor/github.com/theckman/go-flock/appveyor.yml new file mode 100644 index 00000000000..2b2d603fe40 --- /dev/null +++ b/vendor/github.com/theckman/go-flock/appveyor.yml @@ -0,0 +1,25 @@ +version: '{build}' + +build: false +deploy: false + +clone_folder: 'c:\gopath\src\github.com\theckman\go-flock' + +environment: + GOPATH: 'c:\gopath' + GOVERSION: '1.9.2' + +init: + - git config --global core.autocrlf input + +install: + - rmdir c:\go /s /q + - appveyor DownloadFile https://storage.googleapis.com/golang/go%GOVERSION%.windows-amd64.msi + - msiexec /i go%GOVERSION%.windows-amd64.msi /q + - set Path=c:\go\bin;c:\gopath\bin;%Path% + - go version + - go env + +test_script: + - go get -t ./... + - go test -v ./... diff --git a/vendor/github.com/theckman/go-flock/flock.go b/vendor/github.com/theckman/go-flock/flock.go new file mode 100644 index 00000000000..867c765f141 --- /dev/null +++ b/vendor/github.com/theckman/go-flock/flock.go @@ -0,0 +1,107 @@ +// Copyright 2015 Tim Heckman. All rights reserved. +// Use of this source code is governed by the BSD 3-Clause +// license that can be found in the LICENSE file. + +// Package flock implements a thread-safe sync.Locker interface for file locking. +// It also includes a non-blocking TryLock() function to allow locking +// without blocking execution. +// +// Package flock is released under the BSD 3-Clause License. See the LICENSE file +// for more details. +// +// While using this library, remember that the locking behaviors are not +// guaranteed to be the same on each platform. For example, some UNIX-like +// operating systems will transparently convert a shared lock to an exclusive +// lock. If you Unlock() the flock from a location where you believe that you +// have the shared lock, you may accidently drop the exclusive lock. +package flock + +import ( + "context" + "os" + "sync" + "time" +) + +// Flock is the struct type to handle file locking. All fields are unexported, +// with access to some of the fields provided by getter methods (Path() and Locked()). +type Flock struct { + path string + m sync.RWMutex + fh *os.File + l bool + r bool +} + +// NewFlock is a function to return a new instance of *Flock. The only parameter +// it takes is the path to the desired lockfile. +func NewFlock(path string) *Flock { + return &Flock{path: path} +} + +// Path is a function to return the path as provided in NewFlock(). +func (f *Flock) Path() string { + return f.path +} + +// Locked is a function to return the current lock state (locked: true, unlocked: false). +func (f *Flock) Locked() bool { + f.m.RLock() + defer f.m.RUnlock() + return f.l +} + +// RLocked is a function to return the current read lock state (locked: true, unlocked: false). +func (f *Flock) RLocked() bool { + f.m.RLock() + defer f.m.RUnlock() + return f.r +} + +func (f *Flock) String() string { + return f.path +} + +// TryLockContext repeatedly tries to take an exclusive lock until one of the +// conditions is met: TryLock succeeds, TryLock fails with error, or Context +// Done channel is closed. +func (f *Flock) TryLockContext(ctx context.Context, retryDelay time.Duration) (bool, error) { + return tryCtx(f.TryLock, ctx, retryDelay) +} + +// TryRLockContext repeatedly tries to take a shared lock until one of the +// conditions is met: TryRLock succeeds, TryRLock fails with error, or Context +// Done channel is closed. +func (f *Flock) TryRLockContext(ctx context.Context, retryDelay time.Duration) (bool, error) { + return tryCtx(f.TryRLock, ctx, retryDelay) +} + +func tryCtx(fn func() (bool, error), ctx context.Context, retryDelay time.Duration) (bool, error) { + if ctx.Err() != nil { + return false, ctx.Err() + } + for { + if ok, err := fn(); ok || err != nil { + return ok, err + } + select { + case <-ctx.Done(): + return false, ctx.Err() + case <-time.After(retryDelay): + // try again + } + } +} + +func (f *Flock) setFh() error { + // open a new os.File instance + // create it if it doesn't exist, and open the file read-only. + fh, err := os.OpenFile(f.path, os.O_CREATE|os.O_RDONLY, os.FileMode(0600)) + if err != nil { + return err + } + + // set the filehandle on the struct + f.fh = fh + return nil +} diff --git a/vendor/github.com/theckman/go-flock/flock_unix.go b/vendor/github.com/theckman/go-flock/flock_unix.go new file mode 100644 index 00000000000..a9ae0a89607 --- /dev/null +++ b/vendor/github.com/theckman/go-flock/flock_unix.go @@ -0,0 +1,146 @@ +// Copyright 2015 Tim Heckman. All rights reserved. +// Use of this source code is governed by the BSD 3-Clause +// license that can be found in the LICENSE file. + +// +build !windows + +package flock + +import ( + "syscall" +) + +// Lock is a blocking call to try and take an exclusive file lock. It will wait +// until it is able to obtain the exclusive file lock. It's recommended that +// TryLock() be used over this function. This function may block the ability to +// query the current Locked() or RLocked() status due to a RW-mutex lock. +// +// If we are already exclusive-locked, this function short-circuits and returns +// immediately assuming it can take the mutex lock. +// +// If the *Flock has a shared lock (RLock), this may transparently replace the +// shared lock with an exclusive lock on some UNIX-like operating systems. Be +// careful when using exclusive locks in conjunction with shared locks +// (RLock()), because calling Unlock() may accidentally release the exclusive +// lock that was once a shared lock. +func (f *Flock) Lock() error { + return f.lock(&f.l, syscall.LOCK_EX) +} + +// RLock is a blocking call to try and take a ahred file lock. It will wait +// until it is able to obtain the shared file lock. It's recommended that +// TryRLock() be used over this function. This function may block the ability to +// query the current Locked() or RLocked() status due to a RW-mutex lock. +// +// If we are already shared-locked, this function short-circuits and returns +// immediately assuming it can take the mutex lock. +func (f *Flock) RLock() error { + return f.lock(&f.r, syscall.LOCK_SH) +} + +func (f *Flock) lock(locked *bool, flag int) error { + f.m.Lock() + defer f.m.Unlock() + + if *locked { + return nil + } + + if f.fh == nil { + if err := f.setFh(); err != nil { + return err + } + } + + if err := syscall.Flock(int(f.fh.Fd()), flag); err != nil { + return err + } + + *locked = true + return nil +} + +// Unlock is a function to unlock the file. This file takes a RW-mutex lock, so +// while it is running the Locked() and RLocked() functions will be blocked. +// +// This function short-circuits if we are unlocked already. If not, it calls +// syscall.LOCK_UN on the file and closes the file descriptor. It does not +// remove the file from disk. It's up to your application to do. +// +// Please note, if your shared lock became an exclusive lock this may +// unintentionally drop the exclusive lock if called by the consumer that +// believes they have a shared lock. Please see Lock() for more details. +func (f *Flock) Unlock() error { + f.m.Lock() + defer f.m.Unlock() + + // if we aren't locked or if the lockfile instance is nil + // just return a nil error because we are unlocked + if (!f.l && !f.r) || f.fh == nil { + return nil + } + + // mark the file as unlocked + if err := syscall.Flock(int(f.fh.Fd()), syscall.LOCK_UN); err != nil { + return err + } + + f.fh.Close() + + f.l = false + f.r = false + f.fh = nil + + return nil +} + +// TryLock is the preferred function for taking an exclusive file lock. This +// function takes an RW-mutex lock before it tries to lock the file, so there is +// the possibility that this function may block for a short time if another +// goroutine is trying to take any action. +// +// The actual file lock is non-blocking. If we are unable to get the exclusive +// file lock, the function will return false instead of waiting for the lock. If +// we get the lock, we also set the *Flock instance as being exclusive-locked. +func (f *Flock) TryLock() (bool, error) { + return f.try(&f.l, syscall.LOCK_EX) +} + +// TryRLock is the preferred function for taking a shared file lock. This +// function takes an RW-mutex lock before it tries to lock the file, so there is +// the possibility that this function may block for a short time if another +// goroutine is trying to take any action. +// +// The actual file lock is non-blocking. If we are unable to get the shared file +// lock, the function will return false instead of waiting for the lock. If we +// get the lock, we also set the *Flock instance as being share-locked. +func (f *Flock) TryRLock() (bool, error) { + return f.try(&f.r, syscall.LOCK_SH) +} + +func (f *Flock) try(locked *bool, flag int) (bool, error) { + f.m.Lock() + defer f.m.Unlock() + + if *locked { + return true, nil + } + + if f.fh == nil { + if err := f.setFh(); err != nil { + return false, err + } + } + + err := syscall.Flock(int(f.fh.Fd()), flag|syscall.LOCK_NB) + + switch err { + case syscall.EWOULDBLOCK: + return false, nil + case nil: + *locked = true + return true, nil + } + + return false, err +} diff --git a/vendor/github.com/theckman/go-flock/flock_winapi.go b/vendor/github.com/theckman/go-flock/flock_winapi.go new file mode 100644 index 00000000000..fe405a255ae --- /dev/null +++ b/vendor/github.com/theckman/go-flock/flock_winapi.go @@ -0,0 +1,76 @@ +// Copyright 2015 Tim Heckman. All rights reserved. +// Use of this source code is governed by the BSD 3-Clause +// license that can be found in the LICENSE file. + +// +build windows + +package flock + +import ( + "syscall" + "unsafe" +) + +var ( + kernel32, _ = syscall.LoadLibrary("kernel32.dll") + procLockFileEx, _ = syscall.GetProcAddress(kernel32, "LockFileEx") + procUnlockFileEx, _ = syscall.GetProcAddress(kernel32, "UnlockFileEx") +) + +const ( + winLockfileFailImmediately = 0x00000001 + winLockfileExclusiveLock = 0x00000002 + winLockfileSharedLock = 0x00000000 +) + +// Use of 0x00000000 for the shared lock is a guess based on some the MS Windows +// `LockFileEX` docs, which document the `LOCKFILE_EXCLUSIVE_LOCK` flag as: +// +// > The function requests an exclusive lock. Otherwise, it requests a shared +// > lock. +// +// https://msdn.microsoft.com/en-us/library/windows/desktop/aa365203(v=vs.85).aspx + +func lockFileEx(handle syscall.Handle, flags uint32, reserved uint32, numberOfBytesToLockLow uint32, numberOfBytesToLockHigh uint32, offset *syscall.Overlapped) (bool, syscall.Errno) { + r1, _, errNo := syscall.Syscall6( + uintptr(procLockFileEx), + 6, + uintptr(handle), + uintptr(flags), + uintptr(reserved), + uintptr(numberOfBytesToLockLow), + uintptr(numberOfBytesToLockHigh), + uintptr(unsafe.Pointer(offset))) + + if r1 != 1 { + if errNo == 0 { + return false, syscall.EINVAL + } + + return false, errNo + } + + return true, 0 +} + +func unlockFileEx(handle syscall.Handle, reserved uint32, numberOfBytesToLockLow uint32, numberOfBytesToLockHigh uint32, offset *syscall.Overlapped) (bool, syscall.Errno) { + r1, _, errNo := syscall.Syscall6( + uintptr(procUnlockFileEx), + 5, + uintptr(handle), + uintptr(reserved), + uintptr(numberOfBytesToLockLow), + uintptr(numberOfBytesToLockHigh), + uintptr(unsafe.Pointer(offset)), + 0) + + if r1 != 1 { + if errNo == 0 { + return false, syscall.EINVAL + } + + return false, errNo + } + + return true, 0 +} diff --git a/vendor/github.com/theckman/go-flock/flock_windows.go b/vendor/github.com/theckman/go-flock/flock_windows.go new file mode 100644 index 00000000000..a0103f6daaf --- /dev/null +++ b/vendor/github.com/theckman/go-flock/flock_windows.go @@ -0,0 +1,140 @@ +// Copyright 2015 Tim Heckman. All rights reserved. +// Use of this source code is governed by the BSD 3-Clause +// license that can be found in the LICENSE file. + +package flock + +import ( + "syscall" +) + +// ErrorLockViolation is the error code returned from the Windows syscall when a +// lock would block and you ask to fail immediately. +const ErrorLockViolation syscall.Errno = 0x21 // 33 + +// Lock is a blocking call to try and take an exclusive file lock. It will wait +// until it is able to obtain the exclusive file lock. It's recommended that +// TryLock() be used over this function. This function may block the ability to +// query the current Locked() or RLocked() status due to a RW-mutex lock. +// +// If we are already locked, this function short-circuits and returns +// immediately assuming it can take the mutex lock. +func (f *Flock) Lock() error { + return f.lock(&f.l, winLockfileExclusiveLock) +} + +// RLock is a blocking call to try and take a sahred file lock. It will wait +// until it is able to obtain the shared file lock. It's recommended that +// TryRLock() be used over this function. This function may block the ability to +// query the current Locked() or RLocked() status due to a RW-mutex lock. +// +// If we are already locked, this function short-circuits and returns +// immediately assuming it can take the mutex lock. +func (f *Flock) RLock() error { + return f.lock(&f.r, winLockfileSharedLock) +} + +func (f *Flock) lock(locked *bool, flag uint32) error { + f.m.Lock() + defer f.m.Unlock() + + if *locked { + return nil + } + + if f.fh == nil { + if err := f.setFh(); err != nil { + return err + } + } + + if _, errNo := lockFileEx(syscall.Handle(f.fh.Fd()), flag, 0, 1, 0, &syscall.Overlapped{}); errNo > 0 { + return errNo + } + + *locked = true + return nil +} + +// Unlock is a function to unlock the file. This file takes a RW-mutex lock, so +// while it is running the Locked() and RLocked() functions will be blocked. +// +// This function short-circuits if we are unlocked already. If not, it calls +// UnlockFileEx() on the file and closes the file descriptor. It does not remove +// the file from disk. It's up to your application to do. +func (f *Flock) Unlock() error { + f.m.Lock() + defer f.m.Unlock() + + // if we aren't locked or if the lockfile instance is nil + // just return a nil error because we are unlocked + if (!f.l && !f.r) || f.fh == nil { + return nil + } + + // mark the file as unlocked + if _, errNo := unlockFileEx(syscall.Handle(f.fh.Fd()), 0, 1, 0, &syscall.Overlapped{}); errNo > 0 { + return errNo + } + + f.fh.Close() + + f.l = false + f.r = false + f.fh = nil + + return nil +} + +// TryLock is the preferred function for taking an exlusive file lock. This +// function does take a RW-mutex lock before it tries to lock the file, so there +// is the possibility that this function may block for a short time if another +// goroutine is trying to take any action. +// +// The actual file lock is non-blocking. If we are unable to get the exclusive +// file lock, the function will return false instead of waiting for the lock. If +// we get the lock, we also set the *Flock instance as being exclusive-locked. +func (f *Flock) TryLock() (bool, error) { + return f.try(&f.l, winLockfileExclusiveLock) +} + +// TryRLock is the preferred function for taking a shared file lock. This +// function does take a RW-mutex lock before it tries to lock the file, so there +// is the possibility that this function may block for a short time if another +// goroutine is trying to take any action. +// +// The actual file lock is non-blocking. If we are unable to get the shared file +// lock, the function will return false instead of waiting for the lock. If we +// get the lock, we also set the *Flock instance as being shared-locked. +func (f *Flock) TryRLock() (bool, error) { + return f.try(&f.r, winLockfileSharedLock) +} + +func (f *Flock) try(locked *bool, flag uint32) (bool, error) { + f.m.Lock() + defer f.m.Unlock() + + if *locked { + return true, nil + } + + if f.fh == nil { + if err := f.setFh(); err != nil { + return false, err + } + } + + _, errNo := lockFileEx(syscall.Handle(f.fh.Fd()), flag|winLockfileFailImmediately, 0, 1, 0, &syscall.Overlapped{}) + + if errNo > 0 { + if errNo == ErrorLockViolation || errNo == syscall.ERROR_IO_PENDING { + return false, nil + } + + return false, errNo + } + + *locked = true + + return true, nil +} diff --git a/vendor/github.com/urso/go-bin/0gen.go b/vendor/github.com/urso/go-bin/0gen.go new file mode 100644 index 00000000000..da238013882 --- /dev/null +++ b/vendor/github.com/urso/go-bin/0gen.go @@ -0,0 +1,4 @@ +package bin + +//go:generate mktmpl -f -o bin.generated.go bin.yml +//go:generate mktmpl -f -o bin.generated_test.go bin_test.yml diff --git a/vendor/github.com/urso/go-bin/LICENSE b/vendor/github.com/urso/go-bin/LICENSE new file mode 100644 index 00000000000..261eeb9e9f8 --- /dev/null +++ b/vendor/github.com/urso/go-bin/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/vendor/github.com/urso/go-bin/README.md b/vendor/github.com/urso/go-bin/README.md new file mode 100644 index 00000000000..45ba5c3516f --- /dev/null +++ b/vendor/github.com/urso/go-bin/README.md @@ -0,0 +1,2 @@ +# go-bin +Support for encoding/decoding buffers using casts into go structures diff --git a/vendor/github.com/urso/go-bin/bin.generated.go b/vendor/github.com/urso/go-bin/bin.generated.go new file mode 100644 index 00000000000..d0cc28545c6 --- /dev/null +++ b/vendor/github.com/urso/go-bin/bin.generated.go @@ -0,0 +1,260 @@ +// This file has been generated from 'bin.yml', do not edit +package bin + +import "encoding/binary" + +// I8be wraps a byte array into a big endian encoded 8bit signed integer. +type I8be [1]byte + +// Len returns the number of bytes required to store the value. +func (b *I8be) Len() int { return 1 } + +// Get returns the decoded value. +func (b *I8be) Get() int8 { + return int8(b[0]) +} + +// Set encodes a new value into the backing buffer: +func (b *I8be) Set(v int8) { + b[0] = byte(v) +} + +// I16be wraps a byte array into a big endian encoded 16bit signed integer. +type I16be [2]byte + +// Len returns the number of bytes required to store the value. +func (b *I16be) Len() int { return 2 } + +// Get returns the decoded value. +func (b *I16be) Get() int16 { + return int16(binary.BigEndian.Uint16(b[:])) +} + +// Set encodes a new value into the backing buffer: +func (b *I16be) Set(v int16) { + binary.BigEndian.PutUint16(b[:], uint16(v)) +} + +// I32be wraps a byte array into a big endian encoded 32bit signed integer. +type I32be [4]byte + +// Len returns the number of bytes required to store the value. +func (b *I32be) Len() int { return 4 } + +// Get returns the decoded value. +func (b *I32be) Get() int32 { + return int32(binary.BigEndian.Uint32(b[:])) +} + +// Set encodes a new value into the backing buffer: +func (b *I32be) Set(v int32) { + binary.BigEndian.PutUint32(b[:], uint32(v)) +} + +// I64be wraps a byte array into a big endian encoded 64bit signed integer. +type I64be [8]byte + +// Len returns the number of bytes required to store the value. +func (b *I64be) Len() int { return 8 } + +// Get returns the decoded value. +func (b *I64be) Get() int64 { + return int64(binary.BigEndian.Uint64(b[:])) +} + +// Set encodes a new value into the backing buffer: +func (b *I64be) Set(v int64) { + binary.BigEndian.PutUint64(b[:], uint64(v)) +} + +// U8be wraps a byte array into a big endian encoded 8bit unsigned integer. +type U8be [1]byte + +// Len returns the number of bytes required to store the value. +func (b *U8be) Len() int { return 1 } + +// Get returns the decoded value. +func (b *U8be) Get() uint8 { + return uint8(b[0]) +} + +// Set encodes a new value into the backing buffer: +func (b *U8be) Set(v uint8) { + b[0] = byte(v) +} + +// U16be wraps a byte array into a big endian encoded 16bit unsigned integer. +type U16be [2]byte + +// Len returns the number of bytes required to store the value. +func (b *U16be) Len() int { return 2 } + +// Get returns the decoded value. +func (b *U16be) Get() uint16 { + return uint16(binary.BigEndian.Uint16(b[:])) +} + +// Set encodes a new value into the backing buffer: +func (b *U16be) Set(v uint16) { + binary.BigEndian.PutUint16(b[:], uint16(v)) +} + +// U32be wraps a byte array into a big endian encoded 32bit unsigned integer. +type U32be [4]byte + +// Len returns the number of bytes required to store the value. +func (b *U32be) Len() int { return 4 } + +// Get returns the decoded value. +func (b *U32be) Get() uint32 { + return uint32(binary.BigEndian.Uint32(b[:])) +} + +// Set encodes a new value into the backing buffer: +func (b *U32be) Set(v uint32) { + binary.BigEndian.PutUint32(b[:], uint32(v)) +} + +// U64be wraps a byte array into a big endian encoded 64bit unsigned integer. +type U64be [8]byte + +// Len returns the number of bytes required to store the value. +func (b *U64be) Len() int { return 8 } + +// Get returns the decoded value. +func (b *U64be) Get() uint64 { + return uint64(binary.BigEndian.Uint64(b[:])) +} + +// Set encodes a new value into the backing buffer: +func (b *U64be) Set(v uint64) { + binary.BigEndian.PutUint64(b[:], uint64(v)) +} + +// I8le wraps a byte array into a little endian encoded 8bit signed integer. +type I8le [1]byte + +// Len returns the number of bytes required to store the value. +func (b *I8le) Len() int { return 1 } + +// Get returns the decoded value. +func (b *I8le) Get() int8 { + return int8(b[0]) +} + +// Set encodes a new value into the backing buffer: +func (b *I8le) Set(v int8) { + b[0] = byte(v) +} + +// I16le wraps a byte array into a little endian encoded 16bit signed integer. +type I16le [2]byte + +// Len returns the number of bytes required to store the value. +func (b *I16le) Len() int { return 2 } + +// Get returns the decoded value. +func (b *I16le) Get() int16 { + return int16(binary.LittleEndian.Uint16(b[:])) +} + +// Set encodes a new value into the backing buffer: +func (b *I16le) Set(v int16) { + binary.LittleEndian.PutUint16(b[:], uint16(v)) +} + +// I32le wraps a byte array into a little endian encoded 32bit signed integer. +type I32le [4]byte + +// Len returns the number of bytes required to store the value. +func (b *I32le) Len() int { return 4 } + +// Get returns the decoded value. +func (b *I32le) Get() int32 { + return int32(binary.LittleEndian.Uint32(b[:])) +} + +// Set encodes a new value into the backing buffer: +func (b *I32le) Set(v int32) { + binary.LittleEndian.PutUint32(b[:], uint32(v)) +} + +// I64le wraps a byte array into a little endian encoded 64bit signed integer. +type I64le [8]byte + +// Len returns the number of bytes required to store the value. +func (b *I64le) Len() int { return 8 } + +// Get returns the decoded value. +func (b *I64le) Get() int64 { + return int64(binary.LittleEndian.Uint64(b[:])) +} + +// Set encodes a new value into the backing buffer: +func (b *I64le) Set(v int64) { + binary.LittleEndian.PutUint64(b[:], uint64(v)) +} + +// U8le wraps a byte array into a little endian encoded 8bit unsigned integer. +type U8le [1]byte + +// Len returns the number of bytes required to store the value. +func (b *U8le) Len() int { return 1 } + +// Get returns the decoded value. +func (b *U8le) Get() uint8 { + return uint8(b[0]) +} + +// Set encodes a new value into the backing buffer: +func (b *U8le) Set(v uint8) { + b[0] = byte(v) +} + +// U16le wraps a byte array into a little endian encoded 16bit unsigned integer. +type U16le [2]byte + +// Len returns the number of bytes required to store the value. +func (b *U16le) Len() int { return 2 } + +// Get returns the decoded value. +func (b *U16le) Get() uint16 { + return uint16(binary.LittleEndian.Uint16(b[:])) +} + +// Set encodes a new value into the backing buffer: +func (b *U16le) Set(v uint16) { + binary.LittleEndian.PutUint16(b[:], uint16(v)) +} + +// U32le wraps a byte array into a little endian encoded 32bit unsigned integer. +type U32le [4]byte + +// Len returns the number of bytes required to store the value. +func (b *U32le) Len() int { return 4 } + +// Get returns the decoded value. +func (b *U32le) Get() uint32 { + return uint32(binary.LittleEndian.Uint32(b[:])) +} + +// Set encodes a new value into the backing buffer: +func (b *U32le) Set(v uint32) { + binary.LittleEndian.PutUint32(b[:], uint32(v)) +} + +// U64le wraps a byte array into a little endian encoded 64bit unsigned integer. +type U64le [8]byte + +// Len returns the number of bytes required to store the value. +func (b *U64le) Len() int { return 8 } + +// Get returns the decoded value. +func (b *U64le) Get() uint64 { + return uint64(binary.LittleEndian.Uint64(b[:])) +} + +// Set encodes a new value into the backing buffer: +func (b *U64le) Set(v uint64) { + binary.LittleEndian.PutUint64(b[:], uint64(v)) +} diff --git a/vendor/github.com/urso/go-bin/bin.yml b/vendor/github.com/urso/go-bin/bin.yml new file mode 100644 index 00000000000..416b225dab8 --- /dev/null +++ b/vendor/github.com/urso/go-bin/bin.yml @@ -0,0 +1,53 @@ +import: + - 'types.yml' + +main: | + package bin + + import "encoding/binary" + + {{ range $enc, $long := data.endianess }} + {{ range $signdness, $long := data.signdness }} + {{ range $k, $bits := data.bits }} + {{ invoke "makeBinType" "bits" $bits "name" $signdness "enc" $enc }} + {{ end }} + {{ end }} + {{ end }} + +# makeBinType(bits, name, enc) +templates.makeBinType: | + {{ $bits := .bits }} + {{ $len := div $bits 8 }} + {{ $name := .name }} + {{ $enc := .enc }} + {{ $endianess := index data.endianess $enc }} + {{ $inst := capitalize $endianess | printf "%vEndian" }} + {{ $signdness := index data.signdness $name }} + {{ $gotype := printf "%v%v" (index data.baseType $name) $bits }} + {{ $accessor := printf "Uint%v" $bits }} + + {{ $type := printf "%v%v%v" (capitalize $name) $bits $enc }} + + // {{ $type }} wraps a byte array into a {{ $endianess }} endian encoded {{ $bits }}bit {{ $signdness }} integer. + type {{ $type }} [{{ $len }}]byte + + // Len returns the number of bytes required to store the value. + func (b *{{ $type }}) Len() int { return {{ $len }} } + + // Get returns the decoded value. + func (b *{{ $type }}) Get() {{ $gotype }} { + {{- if (eq $bits 8) }} + return {{ $gotype }}(b[0]) + {{ else }} + return {{ $gotype }}(binary.{{ $inst }}.{{ $accessor }}(b[:])) + {{ end -}} + } + + // Set encodes a new value into the backing buffer: + func (b *{{ $type }}) Set(v {{ $gotype }}) { + {{- if (eq $bits 8) }} + b[0] = byte(v) + {{ else }} + binary.{{ $inst }}.Put{{ $accessor }}(b[:], uint{{ $bits }}(v)) + {{ end -}} + } diff --git a/vendor/github.com/urso/go-bin/bin_test.yml b/vendor/github.com/urso/go-bin/bin_test.yml new file mode 100644 index 00000000000..4da772af803 --- /dev/null +++ b/vendor/github.com/urso/go-bin/bin_test.yml @@ -0,0 +1,53 @@ +import: + - 'types.yml' + +main: | + package bin + + func TestPrimitives(t *testing.T) { + {{ range $enc, $long := data.endianess }} + {{ range $signdness, $long := data.signdness }} + {{ range $k, $bits := data.bits }} + {{ invoke "makeBinType" "bits" $bits "name" $signdness "enc" $enc }} + {{ end }} + {{ end }} + {{ end }} + } + +templates.makeBinType: | + {{ $bits := .bits }} + {{ $len := div $bits 8 }} + {{ $name := .name }} + {{ $enc := .enc }} + {{ $endianess := index data.endianess $enc }} + {{ $inst := capitalize $endianess | printf "%vEndian" }} + {{ $signdness := index data.signdness $name }} + {{ $gotype := printf "%v%v" (index data.baseType $name) $bits }} + {{ $accessor := printf "Uint%v" $bits }} + + {{ $type := printf "%v%v%v" (capitalize $name) $bits $enc }} + + t.Run("{{ $gotype }} {{ $endianess }} endian", func(t *testing.T) { + var v {{ $type }} + err := quick.Check(func(in {{ $gotype }}) bool { + v.Set(in) + + // check raw contents correct encoding + tmp := make([]byte, v.Len()) + {{ if (eq $bits 8) }} + tmp[0] = byte(in) + {{ else }} + binary.{{ $inst }}.Put{{ $accessor }}(tmp, uint{{ $bits }}(in)) + {{ end }} + if !bytes.Equal(v[:], tmp) { + t.Error("encoding mismatch") + return false + } + + // check extracted value matches original value + return v.Get() == in + }, nil) + if err != nil { + t.Error(err) + } + }) diff --git a/vendor/github.com/urso/go-bin/cast.go b/vendor/github.com/urso/go-bin/cast.go new file mode 100644 index 00000000000..ac444ba5bbd --- /dev/null +++ b/vendor/github.com/urso/go-bin/cast.go @@ -0,0 +1,53 @@ +package bin + +import ( + "errors" + "fmt" + "reflect" + "unsafe" +) + +var ( + errPtrPtrStructRequired = errors.New("pointer to pointer of go structure required") +) + +type emptyIfc struct { + typ, ptr unsafe.Pointer +} + +// UnsafeCastStruct casts a byte slice its contents into an arbitrary go-structure. +// The structure passed must be a pointer to a pointer of a struct to be casted too. +// +// If the input buffers length is 0, `to` will be set to nil. +// +// The operation is unsafe, as it does not validate the input value to be a +// pointer of a pointer, plus no length check is executed. +func UnsafeCastStruct(to interface{}, b []byte) { + ifc := (*emptyIfc)(unsafe.Pointer(&to)) + + if len(b) != 0 { + *(*uintptr)(ifc.ptr) = uintptr(unsafe.Pointer(&b[0])) + } else { + *(*uintptr)(ifc.ptr) = 0 + } +} + +// CastStruct casts a byte slice its contents into an arbitrary go-structure. +// The structure passed must be a pointer to a pointer of a structed to be casted too. +// An error is returned if the input type is invalid or the buffer is not big +// enough to hold the structure. +// If the input buffers length is 0, `to` will be set to nil. +func CastStruct(to interface{}, b []byte) error { + v := reflect.ValueOf(to) + if v.Kind() != reflect.Ptr || v.Elem().Kind() != reflect.Ptr { + return errPtrPtrStructRequired + } + + if bl, tl := len(b), int(v.Type().Size()); 0 < bl && bl < tl { + return fmt.Errorf("buffer of %v byte(s) can not be casted into structure requiring %v byte(s)", + bl, tl) + } + + UnsafeCastStruct(to, b) + return nil +} diff --git a/vendor/github.com/urso/go-bin/native_be.go b/vendor/github.com/urso/go-bin/native_be.go new file mode 100644 index 00000000000..9516267ed0d --- /dev/null +++ b/vendor/github.com/urso/go-bin/native_be.go @@ -0,0 +1,18 @@ +// +build ppc64 mips64 mips s390x + +package bin + +import "encoding/binary" + +// Architecture native encoding +var NativeEndian = binary.BigEndian + +type I8 = I8be +type I16 = I16be +type I32 = I32be +type I64 = I64be + +type U8 = U8be +type U16 = U16be +type U32 = U32be +type U64 = U64be diff --git a/vendor/github.com/urso/go-bin/native_le.go b/vendor/github.com/urso/go-bin/native_le.go new file mode 100644 index 00000000000..914cfced312 --- /dev/null +++ b/vendor/github.com/urso/go-bin/native_le.go @@ -0,0 +1,18 @@ +// +build amd64 386 arm arm64 ppc64le mips64le mipsle + +package bin + +import "encoding/binary" + +// Architecture native encoding +var NativeEndian = binary.LittleEndian + +type I8 = I8le +type I16 = I16le +type I32 = I32le +type I64 = I64le + +type U8 = U8le +type U16 = U16le +type U32 = U32le +type U64 = U64le diff --git a/vendor/github.com/urso/go-bin/types.yml b/vendor/github.com/urso/go-bin/types.yml new file mode 100644 index 00000000000..0ef2ce20fbf --- /dev/null +++ b/vendor/github.com/urso/go-bin/types.yml @@ -0,0 +1,6 @@ +data: + bits: [8, 16, 32, 64] + endianess: {le: little, be: big} + signdness: {i: signed, u: unsigned} + baseType: {i: int, u: uint} + diff --git a/vendor/github.com/urso/qcgen/LICENSE b/vendor/github.com/urso/qcgen/LICENSE new file mode 100644 index 00000000000..261eeb9e9f8 --- /dev/null +++ b/vendor/github.com/urso/qcgen/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/vendor/github.com/urso/qcgen/README.md b/vendor/github.com/urso/qcgen/README.md new file mode 100644 index 00000000000..882062dd429 --- /dev/null +++ b/vendor/github.com/urso/qcgen/README.md @@ -0,0 +1,2 @@ +# qcgen +helpers for building go quick check generators from custom functions diff --git a/vendor/github.com/urso/qcgen/helpers.go b/vendor/github.com/urso/qcgen/helpers.go new file mode 100644 index 00000000000..a98782115d4 --- /dev/null +++ b/vendor/github.com/urso/qcgen/helpers.go @@ -0,0 +1,32 @@ +package qcgen + +import "math/rand" + +func GenBool(rng *rand.Rand) bool { + return (rng.Uint64() & 1) == 1 +} + +func GenUint64(rng *rand.Rand) uint64 { + return rng.Uint64() +} + +func MakeGenUint64Range(min, max uint64) func(rng *rand.Rand) uint64 { + return func(rng *rand.Rand) uint64 { + return GenUint64Range(rng, min, max) + } +} + +func GenUint64Range(rng *rand.Rand, min, max uint64) uint64 { + delta := max - min + return min + (rng.Uint64() % delta) +} + +func MakeGenUintRange(min, max uint) func(rng *rand.Rand) uint { + return func(rng *rand.Rand) uint { + return GenUintRange(rng, min, max) + } +} + +func GenUintRange(rng *rand.Rand, min, max uint) uint { + return uint(GenUint64Range(rng, uint64(min), uint64(max))) +} diff --git a/vendor/github.com/urso/qcgen/qcgen.go b/vendor/github.com/urso/qcgen/qcgen.go new file mode 100644 index 00000000000..f7ec12ff28d --- /dev/null +++ b/vendor/github.com/urso/qcgen/qcgen.go @@ -0,0 +1,80 @@ +package qcgen + +import ( + "fmt" + "math/rand" + "reflect" + "testing/quick" +) + +type Generator struct { + arguments []userGen +} + +type userGen func(rng *rand.Rand, params []reflect.Value) reflect.Value + +var tRand = reflect.TypeOf((*rand.Rand)(nil)) + +// NewGenerator creates a new generator. Each function must implement +// `func(*rand.Rand) T`, with T being the custom type to be generated. +// The generators Gen methods selects the function to execute on +// matching return type. +func NewGenerator(testFn interface{}, fns ...interface{}) *Generator { + mapping := map[reflect.Type]reflect.Value{} + + for i, fn := range fns { + v := reflect.ValueOf(fn) + t := v.Type() + if t.Kind() != reflect.Func { + panic(fmt.Errorf("argument %v is no function", i)) + } + + if t.NumIn() != 1 || t.NumOut() != 1 { + panic(fmt.Errorf("argument %v must accept one argument and return one value", i)) + } + + tIn := t.In(0) + if tIn != tRand { + panic(fmt.Errorf("argument %v must accept *rand.Rand as input only", i)) + } + + mapping[t.Out(0)] = v + } + + fn := reflect.TypeOf(testFn) + argGen := make([]userGen, fn.NumIn()) + for i := range argGen { + tIn := fn.In(i) + if v, exists := mapping[tIn]; exists { + argGen[i] = makeUserGen(v) + } else { + argGen[i] = makeDefaultGen(tIn) + } + } + + return &Generator{argGen} +} + +func makeUserGen(fn reflect.Value) userGen { + return func(_ *rand.Rand, params []reflect.Value) reflect.Value { + out := fn.Call(params) + return out[0] + } +} + +func makeDefaultGen(t reflect.Type) userGen { + return func(rng *rand.Rand, _ []reflect.Value) reflect.Value { + out, ok := quick.Value(t, rng) + if !ok { + panic(fmt.Errorf("cannot create arbitrary value of type %s", t)) + } + return out + } +} + +func (g *Generator) Gen(args []reflect.Value, rng *rand.Rand) { + rngParam := []reflect.Value{reflect.ValueOf(rng)} + for i := range args { + args[i] = g.arguments[i](rng, rngParam) + } +} diff --git a/vendor/vendor.json b/vendor/vendor.json index 3fd6cc6b416..04e2f9f49b1 100644 --- a/vendor/vendor.json +++ b/vendor/vendor.json @@ -542,6 +542,62 @@ "revision": "9c842019df6fc236a6f1ef5537109ac36357dd63", "revisionTime": "2018-03-16T07:12:44Z" }, + { + "checksumSHA1": "IMU9C/FvHiOI52jKElb62Gokv0s=", + "path": "github.com/elastic/go-txfile", + "revision": "7e7e33cc236f30fff545f3ee2c35ada5b70b6b13", + "revisionTime": "2018-03-29T16:52:21Z", + "version": "v0.0.1", + "versionExact": "v0.0.1" + }, + { + "checksumSHA1": "cXIYcUhYjLs+sUW42/yIE9ellS0=", + "path": "github.com/elastic/go-txfile/internal/cleanup", + "revision": "7e7e33cc236f30fff545f3ee2c35ada5b70b6b13", + "revisionTime": "2018-03-29T16:52:21Z", + "version": "v0.0.1", + "versionExact": "v0.0.1" + }, + { + "checksumSHA1": "ERbGKs6IgC062SkwB1jI0EHfAM8=", + "path": "github.com/elastic/go-txfile/internal/invariant", + "revision": "7e7e33cc236f30fff545f3ee2c35ada5b70b6b13", + "revisionTime": "2018-03-29T16:52:21Z", + "version": "v0.0.1", + "versionExact": "v0.0.1" + }, + { + "checksumSHA1": "xaTyh6jlrYI94Kok8+QT2bpRkHM=", + "path": "github.com/elastic/go-txfile/internal/iter", + "revision": "7e7e33cc236f30fff545f3ee2c35ada5b70b6b13", + "revisionTime": "2018-03-29T16:52:21Z", + "version": "v0.0.1", + "versionExact": "v0.0.1" + }, + { + "checksumSHA1": "5tZTXzITZkrmyx4tamhtW9ilmhE=", + "path": "github.com/elastic/go-txfile/internal/tracelog", + "revision": "7e7e33cc236f30fff545f3ee2c35ada5b70b6b13", + "revisionTime": "2018-03-29T16:52:21Z", + "version": "v0.0.1", + "versionExact": "v0.0.1" + }, + { + "checksumSHA1": "cdu99vXUoUWPJww+ndmzlfh/Rsk=", + "path": "github.com/elastic/go-txfile/pq", + "revision": "7e7e33cc236f30fff545f3ee2c35ada5b70b6b13", + "revisionTime": "2018-03-29T16:52:21Z", + "version": "v0.0.1", + "versionExact": "v0.0.1" + }, + { + "checksumSHA1": "1ltkQVfITTz9qKLzO2XvgG+K7Zs=", + "path": "github.com/elastic/go-txfile/txfiletest", + "revision": "7e7e33cc236f30fff545f3ee2c35ada5b70b6b13", + "revisionTime": "2018-03-29T16:52:21Z", + "version": "v0.0.1", + "versionExact": "v0.0.1" + }, { "checksumSHA1": "61XUpyQ3zWnJ7Tlj0xLsHtnzwJY=", "path": "github.com/elastic/go-ucfg", @@ -1193,6 +1249,14 @@ "version": "v1.2.0", "versionExact": "v1.2.0" }, + { + "checksumSHA1": "CpcG17Q/0k1g2uy8AL26Uu7TouU=", + "path": "github.com/theckman/go-flock", + "revision": "b139a2487364247d91814e4a7c7b8fdc69e342b2", + "revisionTime": "2018-01-24T01:19:07Z", + "version": "v0.4.0", + "versionExact": "v0.4.0" + }, { "checksumSHA1": "M0S9278lG9Fztu+ZUsLUi40GDJU=", "path": "github.com/tsg/gopacket", @@ -1223,6 +1287,18 @@ "revision": "8e703b9968693c15f25cabb6ba8be4370cf431d0", "revisionTime": "2016-08-17T18:24:57Z" }, + { + "checksumSHA1": "H7tCgNt2ajKK4FBJIDNlevu9MAc=", + "path": "github.com/urso/go-bin", + "revision": "781c575c9f0eb3cb9dca94521bd7ad7d5aec7fd4", + "revisionTime": "2018-02-20T13:58:11Z" + }, + { + "checksumSHA1": "eNXPqquHSqrsae/NydhZJSqOP5c=", + "path": "github.com/urso/qcgen", + "revision": "0b059e7db4f40a062ca3d975b7500c6a0a968d87", + "revisionTime": "2018-01-31T10:30:24Z" + }, { "checksumSHA1": "Und+nhgN1YsNWvd0aDYO+0cMcAo=", "path": "github.com/yuin/gopher-lua", diff --git a/winlogbeat/winlogbeat.reference.yml b/winlogbeat/winlogbeat.reference.yml index 6dff01e3bf8..628537eb765 100644 --- a/winlogbeat/winlogbeat.reference.yml +++ b/winlogbeat/winlogbeat.reference.yml @@ -72,6 +72,66 @@ winlogbeat.event_logs: # if the number of events stored in the queue is < min_flush_events. #flush.timeout: 1s + # The spool queue will store events in a local spool file, before + # forwarding the events to the outputs. + # + # Beta: spooling to disk is currently a beta feature. Use with care. + # + # The spool file is a circular buffer, which blocks once the file/buffer is full. + # Events are put into a write buffer and flushed once the write buffer + # is full or the flush_timeout is triggered. + # Once ACKed by the output, events are removed immediately from the queue, + # making space for new events to be persisted. + #spool: + # The file namespace configures the file path and the file creation settings. + # Once the file exists, the `size`, `page_size` and `prealloc` settings + # will have no more effect. + #file: + # Location of spool file. The default value is ${path.data}/spool.dat. + #path: "${path.data}/spool.dat" + + # Configure file permissions if file is created. The default value is 0600. + #permissions: 0600 + + # File size hint. The spool blocks, once this limit is reached. The default value is 100 MiB. + #size: 100MiB + + # The files page size. A file is split into multiple pages of the same size. The default value is 4KiB. + #page_size: 4KiB + + # If prealloc is set, the required space for the file is reserved using + # truncate. The default value is true. + #prealloc: true + + # Spool writer settings + # Events are serialized into a write buffer. The write buffer is flushed if: + # - The buffer limit has been reached. + # - The configured limit of buffered events is reached. + # - The flush timeout is triggered. + #write: + # Sets the write buffer size. + #buffer_size: 1MiB + + # Maximum duration after which events are flushed, if the write buffer + # is not full yet. The default value is 1s. + #flush.timeout: 1s + + # Number of maximum buffered events. The write buffer is flushed once the + # limit is reached. + #flush.events: 16384 + + # Configure the on-disk event encoding. The encoding can be changed + # between restarts. + # Valid encodings are: json, ubjson, and cbor. + #codec: cbor + #read: + # Reader flush timeout, waiting for more events to become available, so + # to fill a complete batch, as required by the outputs. + # If flush_timeout is 0, all available events are forwarded to the + # outputs immediately. + # The default value is 0s. + #flush.timeout: 0s + # Sets the maximum number of CPUs that can be executing simultaneously. The # default is the number of logical CPUs available in the system. #max_procs: