From 1426fc10f049dd3e32d57dcffee9ace9ab51caed Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Thu, 12 May 2016 17:19:09 +0900 Subject: [PATCH 01/29] Fix to do commit/rollback for 2 or more chunks at once * without this fix, some events will be written paritally if any chunks raise error in #append/#concat * it makes duplicated events Buffer#write will do: (method name was changed: #emit is method names for event routers) * this method receives pairs of metadata and data to be written at once * append/concat these data to chunks (not committed) * commit first chunk * if succeeded, then commit all (even if any following chunk raises error) * if failed, rollback all In memory/file buffer, #commit is very lightweight operation and will NOT fail in most cases. This change requires some additional internal APIs for buffers/chunks * chunk status in general: this is required to make #write thread safe * keyword argument of #write: bulk and enqueue * #write method becomes much more complex, so bulk operation should be merged into an implementation (we can't maintain two different/similar methods) * #write method enqueues chunks if needed, so :immediate mode should be implemented in same level * chunk_full_threshold configuration parameter to control "chunk_size_full?" * bulk and non-bulk #write were merged * In non-bulk mode, it's too rare that written chunk content bytesize is same with limitation --- lib/fluent/plugin/buffer.rb | 252 ++++++++++++++--------- lib/fluent/plugin/buffer/chunk.rb | 26 ++- lib/fluent/plugin/buffer/file_chunk.rb | 50 +++-- lib/fluent/plugin/buffer/memory_chunk.rb | 9 +- 4 files changed, 199 insertions(+), 138 deletions(-) diff --git a/lib/fluent/plugin/buffer.rb b/lib/fluent/plugin/buffer.rb index 6868c5b3c7..47c0c11106 100644 --- a/lib/fluent/plugin/buffer.rb +++ b/lib/fluent/plugin/buffer.rb @@ -36,6 +36,8 @@ class BufferChunkOverflowError < BufferError; end # A record size is larger than DEFAULT_CHUNK_BYTES_LIMIT = 8 * 1024 * 1024 # 8MB DEFAULT_TOTAL_BYTES_LIMIT = 512 * 1024 * 1024 # 512MB, same with v0.12 (BufferedOutput + buf_memory: 64 x 8MB) + DEFAULT_CHUNK_FULL_THRESHOLD = 0.95 + configured_in :buffer # TODO: system total buffer bytes limit by SystemConfig @@ -50,6 +52,9 @@ class BufferChunkOverflowError < BufferError; end # A record size is larger than # optional new limitations config_param :chunk_records_limit, :integer, default: nil + # if chunk size (or records) is 95% or more after #write, then that chunk will be enqueued + config_param :chunk_full_threshold, :float, default: DEFAULT_CHUNK_FULL_THRESHOLD + Metadata = Struct.new(:timekey, :tag, :variables) # for tests @@ -168,77 +173,66 @@ def metadata(timekey: nil, tag: nil, variables: nil) # metadata MUST have consistent object_id for each variation # data MUST be Array of serialized events - def emit(metadata, data, force: false) - return if data.size < 1 + # metadata_and_data MUST be a hash of { metadata => data } + def write(metadata_and_data, bulk: false, enqueue: false) + return if metadata_and_data.size < 1 raise BufferOverflowError unless storable? - stored = false + staged_bytesize = 0 + operated_chunks = [] - # the case whole data can be stored in staged chunk: almost all emits will success - chunk = synchronize { @stage[metadata] ||= generate_chunk(metadata) } - original_bytesize = chunk.bytesize - chunk.synchronize do - begin - chunk.append(data) - if !chunk_size_over?(chunk) || force - chunk.commit - stored = true - @stage_size += (chunk.bytesize - original_bytesize) - else - chunk.rollback + begin + metadata_and_data.each do |metadata, data| + write_once(metadata, data, bulk: bulk) do |chunk, adding_bytesize| + chunk.mon_enter # add lock to prevent to be committed/rollbacked from other threads + operated_chunks << chunk + staged_bytesize += adding_bytesize end - rescue - chunk.rollback - raise end - end - return if stored - # try step-by-step appending if data can't be stored into existing a chunk - emit_step_by_step(metadata, data) - end + return if operated_chunks.empty? - def emit_bulk(metadata, bulk, size) - return if bulk.nil? || bulk.empty? - raise BufferOverflowError unless storable? + first_chunk = operated_chunks.shift + # Following commits for other chunks also can finish successfully if the first commit operation + # finishes without any exceptions. + # In most cases, #commit just requires very small disk spaces, so major failure reason are + # permission errors, disk failures and other permanent(fatal) errors. + begin + first_chunk.commit + enqueue_chunk(first_chunk.metadata) if enqueue || chunk_size_full?(first_chunk) + first_chunk.mon_exit + rescue + operated_chunks.unshift(first_chunk) + raise + end - stored = false - synchronize do # critical section for buffer (stage/queue) - until stored - chunk = @stage[metadata] - unless chunk - chunk = @stage[metadata] = generate_chunk(metadata) + errors = [] + # Buffer plugin estimates there's no serious error cause: will commit for all chunks eigher way + operated_chunks.each do |chunk| + begin + chunk.commit + enqueue_chunk(chunk.metadata) if enqueue || chunk_size_full?(chunk) + chunk.mon_exit + rescue => e + chunk.rollback + chunk.mon_exit + errors << e end + end - chunk.synchronize do # critical section for chunk (chunk append/commit/rollback) - begin - empty_chunk = chunk.empty? - chunk.concat(bulk, size) - - if chunk_size_over?(chunk) - if empty_chunk - log.warn "chunk bytes limit exceeds for a bulk event stream: #{bulk.bytesize}bytes" - else - chunk.rollback - enqueue_chunk(metadata) - next - end - end + @stage_size += staged_bytesize - chunk.commit - stored = true - @stage_size += bulk.bytesize - if chunk_size_full?(chunk) - enqueue_chunk(metadata) - end - rescue - chunk.rollback - raise - end - end + if errors.size > 0 + log.warn "error occurs in committing chunks: only first one raised", errors: errors.map(&:class) + raise errors.first + end + rescue + operated_chunks.each do |chunk| + chunk.rollback rescue nil # nothing possible to do for #rollback failure + chunk.mon_exit rescue nil # this may raise ThreadError for chunks already committed end + raise end - nil end def queued_records @@ -358,64 +352,118 @@ def chunk_size_over?(chunk) end def chunk_size_full?(chunk) - chunk.bytesize >= @chunk_bytes_limit || (@chunk_records_limit && chunk.size >= @chunk_records_limit) + chunk.bytesize >= @chunk_bytes_limit * @chunk_full_threshold || (@chunk_records_limit && chunk.size >= @chunk_records_limit * @chunk_full_threshold) end - def emit_step_by_step(metadata, data) - attempt_records = data.size / 3 + class ShouldRetry < StandardError; end + + def write_once(metadata, data, bulk: false, &block) + return if !bulk && (data.nil? || data.empty?) + return if bulk && (data.empty? || data.first.nil? || data.first.empty?) - synchronize do # critical section for buffer (stage/queue) - while data.size > 0 - if attempt_records < MINIMUM_APPEND_ATTEMPT_RECORDS - attempt_records = MINIMUM_APPEND_ATTEMPT_RECORDS + stored = false + adding_bytesize = nil + + chunk = synchronize { @stage[metadata] ||= generate_chunk(metadata) } + + chunk.synchronize do + # retry this method if chunk is already queued (between getting chunk and entering critical section) + raise ShouldRetry unless chunk.staged? + + empty_chunk = chunk.empty? + + original_bytesize = chunk.bytesize + begin + if bulk + content, size = data + chunk.concat(content, size) + else + chunk.append(data) end + adding_bytesize = chunk.bytesize - original_bytesize - chunk = @stage[metadata] - unless chunk - chunk = @stage[metadata] = generate_chunk(metadata) + if chunk_size_over?(chunk) + if empty_chunk && bulk + log.warn "chunk bytes limit exceeds for a bulk event stream: #{bulk.bytesize}bytes" + stored = true + else + chunk.rollback + end + else + stored = true end + rescue + chunk.rollback + raise + end - chunk.synchronize do # critical section for chunk (chunk append/commit/rollback) - begin - empty_chunk = chunk.empty? - original_bytesize = chunk.bytesize - - attempt = data.slice(0, attempt_records) - chunk.append(attempt) - - if chunk_size_over?(chunk) - chunk.rollback - - if attempt_records <= MINIMUM_APPEND_ATTEMPT_RECORDS - if empty_chunk # record is too large even for empty chunk - raise BufferChunkOverflowError, "minimum append butch exceeds chunk bytes limit" - end - # no more records for this chunk -> enqueue -> to be flushed - enqueue_chunk(metadata) # `chunk` will be removed from stage - attempt_records = data.size # fresh chunk may have enough space - else - # whole data can be processed by twice operation - # ( by using apttempt /= 2, 3 operations required for odd numbers of data) - attempt_records = (attempt_records / 2) + 1 - end + if stored + block.call(chunk, adding_bytesize) + elsif bulk + # this metadata might be enqueued already by other threads + # but #enqueue_chunk does nothing in such case + enqueue_chunk(metadata) + raise ShouldRetry + end + end - next - end + unless stored + # try step-by-step appending if data can't be stored into existing a chunk in non-bulk mode + write_step_by_step(metadata, data, data.size / 3, &block) + end + rescue ShouldRetry + retry + end + + def write_step_by_step(metadata, data, attempt_records, &block) + while data.size > 0 + if attempt_records < MINIMUM_APPEND_ATTEMPT_RECORDS + attempt_records = MINIMUM_APPEND_ATTEMPT_RECORDS + end + + chunk = synchronize{ @stage[metadata] ||= generate_chunk(metadata) } + chunk.synchronize do # critical section for chunk (chunk append/commit/rollback) + raise ShouldRetry unless chunk.staged? + begin + empty_chunk = chunk.empty? + original_bytesize = chunk.bytesize + + attempt = data.slice(0, attempt_records) + chunk.append(attempt) + adding_bytesize = (chunk.bytesize - original_bytesize) - chunk.commit - @stage_size += (chunk.bytesize - original_bytesize) - data.slice!(0, attempt_records) - # same attempt size - nil # discard return value of data.slice!() immediately - rescue + if chunk_size_over?(chunk) chunk.rollback - raise + + if attempt_records <= MINIMUM_APPEND_ATTEMPT_RECORDS + if empty_chunk # record is too large even for empty chunk + raise BufferChunkOverflowError, "minimum append butch exceeds chunk bytes limit" + end + # no more records for this chunk -> enqueue -> to be flushed + enqueue_chunk(metadata) # `chunk` will be removed from stage + attempt_records = data.size # fresh chunk may have enough space + else + # whole data can be processed by twice operation + # ( by using apttempt /= 2, 3 operations required for odd numbers of data) + attempt_records = (attempt_records / 2) + 1 + end + + next end + + block.call(chunk, adding_bytesize) + data.slice!(0, attempt_records) + # same attempt size + nil # discard return value of data.slice!() immediately + rescue + chunk.rollback + raise end end end - nil - end # emit_step_by_step + rescue ShouldRetry + retry + end # write_step_by_step end end end diff --git a/lib/fluent/plugin/buffer/chunk.rb b/lib/fluent/plugin/buffer/chunk.rb index b915e64c10..9c81331371 100644 --- a/lib/fluent/plugin/buffer/chunk.rb +++ b/lib/fluent/plugin/buffer/chunk.rb @@ -51,12 +51,15 @@ def initialize(metadata) @unique_id = generate_unique_id @metadata = metadata + # state: staged/queued/closed + @state = :staged + @size = 0 @created_at = Time.now @modified_at = Time.now end - attr_reader :unique_id, :metadata, :created_at, :modified_at + attr_reader :unique_id, :metadata, :created_at, :modified_at, :state # data is array of formatted record string def append(data) @@ -89,15 +92,28 @@ def empty? size == 0 end - ## method for post-process of enqueue (e.g., renaming file for file chunks) - # def enqueued! + def staged? + @state == :staged + end + + def queued? + @state == :queued + end + + def closed? + @state == :closed + end + + def enqueued! + @state = :queued + end def close - raise NotImplementedError, "Implement this method in child class" + @state = :closed end def purge - raise NotImplementedError, "Implement this method in child class" + @state = :closed end def read diff --git a/lib/fluent/plugin/buffer/file_chunk.rb b/lib/fluent/plugin/buffer/file_chunk.rb index 0dc64a8ba2..2c8fc59257 100644 --- a/lib/fluent/plugin/buffer/file_chunk.rb +++ b/lib/fluent/plugin/buffer/file_chunk.rb @@ -38,16 +38,12 @@ class FileChunk < Chunk FILE_PERMISSION = 0644 - attr_reader :path, :state, :permission + attr_reader :path, :permission def initialize(metadata, path, mode, perm: system_config.file_permission || FILE_PERMISSION) super(metadata) - # state: staged/queued/closed - @state = nil @meta = nil - @permission = perm - @bytesize = @size = @adding_bytes = @adding_size = 0 case mode @@ -60,7 +56,7 @@ def initialize(metadata, path, mode, perm: system_config.file_permission || FILE end def append(data) - raise "BUG: appending to non-staged chunk, now '#{@state}'" unless @state == :staged + raise "BUG: appending to non-staged chunk, now '#{self.state}'" unless self.staged? bytes = 0 adding = ''.force_encoding(Encoding::ASCII_8BIT) @@ -78,7 +74,7 @@ def append(data) end def concat(bulk, bulk_size) - raise "BUG: appending to non-staged chunk, now '#{@state}'" unless @state == :staged + raise "BUG: appending to non-staged chunk, now '#{self.state}'" unless self.staged? bulk.force_encoding(Encoding::ASCII_8BIT) @chunk.write bulk @@ -120,8 +116,25 @@ def empty? @bytesize == 0 end + def enqueued! + return unless self.staged? + + new_chunk_path = self.class.generate_queued_chunk_path(@path, @unique_id) + new_meta_path = new_chunk_path + '.meta' + + write_metadata(update: false) # re-write metadata w/ finalized records + + file_rename(@chunk, @path, new_chunk_path, ->(new_io){ @chunk = new_io }) + @path = new_chunk_path + + file_rename(@meta, @meta_path, new_meta_path, ->(new_io){ @meta = new_io }) + @meta_path = new_meta_path + + super + end + def close - @state = :closed + super size = @chunk.size @chunk.close @meta.close if @meta # meta may be missing if chunk is queued at first @@ -131,7 +144,7 @@ def close end def purge - @state = :closed + super @chunk.close @meta.close if @meta @bytesize = @size = @adding_bytes = @adding_size = 0 @@ -146,7 +159,7 @@ def read def open(&block) @chunk.seek(0, IO::SEEK_SET) val = yield @chunk - @chunk.seek(0, IO::SEEK_END) if @state == :staged + @chunk.seek(0, IO::SEEK_END) if self.staged? val end @@ -225,23 +238,6 @@ def write_metadata(update: true) @meta.write(msgpack_packer.pack(data)) end - def enqueued! - return unless @state == :staged - - new_chunk_path = self.class.generate_queued_chunk_path(@path, @unique_id) - new_meta_path = new_chunk_path + '.meta' - - write_metadata(update: false) # re-write metadata w/ finalized records - - file_rename(@chunk, @path, new_chunk_path, ->(new_io){ @chunk = new_io }) - @path = new_chunk_path - - file_rename(@meta, @meta_path, new_meta_path, ->(new_io){ @meta = new_io }) - @meta_path = new_meta_path - - @state = :queued - end - def file_rename(file, old_path, new_path, callback=nil) pos = file.pos if Fluent.windows? diff --git a/lib/fluent/plugin/buffer/memory_chunk.rb b/lib/fluent/plugin/buffer/memory_chunk.rb index 2f6653b24f..84e5222c34 100644 --- a/lib/fluent/plugin/buffer/memory_chunk.rb +++ b/lib/fluent/plugin/buffer/memory_chunk.rb @@ -29,6 +29,8 @@ def initialize(metadata) end def append(data) + raise "BUG: appending to non-staged chunk, now '#{self.state}'" unless self.staged? + adding = data.join.force_encoding(Encoding::ASCII_8BIT) @chunk << adding @adding_bytes += adding.bytesize @@ -37,6 +39,8 @@ def append(data) end def concat(bulk, bulk_size) + raise "BUG: appending to non-staged chunk, now '#{self.state}'" unless self.staged? + bulk.force_encoding(Encoding::ASCII_8BIT) @chunk << bulk @adding_bytes += bulk.bytesize @@ -71,11 +75,8 @@ def empty? @chunk.empty? end - def close - true - end - def purge + super @chunk = ''.force_encoding("ASCII-8BIT") @chunk_bytes = @size = @adding_bytes = @adding_size = 0 true From a82614160ba876ae15f881c6374c63b461706f99 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Thu, 12 May 2016 17:34:05 +0900 Subject: [PATCH 02/29] fix to use Buffer#write --- lib/fluent/plugin/output.rb | 43 ++++++++++++++++--------------------- 1 file changed, 19 insertions(+), 24 deletions(-) diff --git a/lib/fluent/plugin/output.rb b/lib/fluent/plugin/output.rb index 0b4eb559e9..739ac0c3b6 100644 --- a/lib/fluent/plugin/output.rb +++ b/lib/fluent/plugin/output.rb @@ -493,12 +493,7 @@ def emit_sync(tag, es) def emit_buffered(tag, es) @counters_monitor.synchronize{ @emit_count += 1 } begin - metalist = execute_chunking(tag, es) - if @flush_mode == :immediate - metalist.each do |meta| - @buffer.enqueue_chunk(meta) - end - end + execute_chunking(tag, es, enqueue: (@flush_mode == :immediate)) if !@retry && @buffer.queued? submit_flush_once end @@ -532,12 +527,12 @@ def metadata(tag, time, record) elsif @chunk_key_time && @chunk_key_tag timekey_range = @buffer_config.timekey_range time_int = time.to_i - timekey = time_int - (time_int % timekey_range) + timekey = (time_int - (time_int % timekey_range)).to_i @buffer.metadata(timekey: timekey, tag: tag) elsif @chunk_key_time timekey_range = @buffer_config.timekey_range time_int = time.to_i - timekey = time_int - (time_int % timekey_range) + timekey = (time_int - (time_int % timekey_range)).to_i @buffer.metadata(timekey: timekey) else @buffer.metadata(tag: tag) @@ -546,7 +541,7 @@ def metadata(tag, time, record) timekey_range = @buffer_config.timekey_range timekey = if @chunk_key_time time_int = time.to_i - time_int - (time_int % timekey_range) + (time_int - (time_int % timekey_range)).to_i else nil end @@ -555,17 +550,17 @@ def metadata(tag, time, record) end end - def execute_chunking(tag, es) + def execute_chunking(tag, es, enqueue: false) if @simple_chunking - handle_stream_simple(tag, es) + handle_stream_simple(tag, es, enqueue: enqueue) elsif @custom_format - handle_stream_with_custom_format(tag, es) + handle_stream_with_custom_format(tag, es, enqueue: enqueue) else - handle_stream_with_standard_format(tag, es) + handle_stream_with_standard_format(tag, es, enqueue: enqueue) end end - def handle_stream_with_custom_format(tag, es) + def handle_stream_with_custom_format(tag, es, enqueue: false) meta_and_data = {} records = 0 es.each do |time, record| @@ -574,14 +569,12 @@ def handle_stream_with_custom_format(tag, es) meta_and_data[meta] << format(tag, time, record) records += 1 end - meta_and_data.each_pair do |meta, data| - @buffer.emit(meta, data) - end + @buffer.write(meta_and_data, bulk: false, enqueue: enqueue) @counters_monitor.synchronize{ @emit_records += records } - meta_and_data.keys + true end - def handle_stream_with_standard_format(tag, es) + def handle_stream_with_standard_format(tag, es, enqueue: false) meta_and_data = {} records = 0 es.each do |time, record| @@ -590,14 +583,16 @@ def handle_stream_with_standard_format(tag, es) meta_and_data[meta].add(time, record) records += 1 end + meta_and_data_bulk = {} meta_and_data.each_pair do |meta, es| - @buffer.emit_bulk(meta, es.to_msgpack_stream(time_int: @time_as_integer), es.size) + meta_and_data_bulk[meta] = [es.to_msgpack_stream(time_int: @time_as_integer), es.size] end + @buffer.write(meta_and_data_bulk, bulk: true, enqueue: enqueue) @counters_monitor.synchronize{ @emit_records += records } - meta_and_data.keys + true end - def handle_stream_simple(tag, es) + def handle_stream_simple(tag, es, enqueue: false) meta = metadata((@chunk_key_tag ? tag : nil), nil, nil) records = es.size if @custom_format @@ -613,9 +608,9 @@ def handle_stream_simple(tag, es) es_size = es.size es_bulk = es.to_msgpack_stream(time_int: @time_as_integer) end - @buffer.emit_bulk(meta, es_bulk, es_size) + @buffer.write({meta => [es_bulk, es_size]}, bulk: true, enqueue: enqueue) @counters_monitor.synchronize{ @emit_records += records } - [meta] + true end def commit_write(chunk_id, delayed: @delayed_commit, secondary: false) From 9b3e3ba6aa4c2ce7de6931f30fb0508f5da818e3 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Thu, 12 May 2016 17:34:35 +0900 Subject: [PATCH 03/29] fix tests with Buffer#write --- test/plugin/test_buffer.rb | 127 ++++++++++++++++++------- test/plugin/test_buffer_chunk.rb | 26 ++++- test/plugin/test_output_as_buffered.rb | 33 ++++--- test/plugin/test_output_as_standard.rb | 78 ++++++++------- 4 files changed, 176 insertions(+), 88 deletions(-) diff --git a/test/plugin/test_buffer.rb b/test/plugin/test_buffer.rb index 3ac16f1cb7..a198acf9ec 100644 --- a/test/plugin/test_buffer.rb +++ b/test/plugin/test_buffer.rb @@ -13,21 +13,26 @@ class DummyOutputPlugin < Fluent::Plugin::Base include Fluent::PluginId include Fluent::PluginLoggerMixin end + class DummyMemoryChunkError < StandardError; end class DummyMemoryChunk < Fluent::Plugin::Buffer::MemoryChunk attr_reader :append_count, :rollbacked, :closed, :purged + attr_accessor :failing def initialize(metadata) super @append_count = 0 @rollbacked = false @closed = false @purged = false + @failing = false end def append(data) @append_count += 1 + raise DummyMemoryChunkError if @failing super end def concat(data, size) @append_count += 1 + raise DummyMemoryChunkError if @failing super end def rollback @@ -495,30 +500,30 @@ def create_chunk(metadata, data) assert{ qchunks.all?{ |c| c.purged } } end - test '#emit returns immediately if argument data is empty array' do + test '#write returns immediately if argument data is empty array' do assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3], @p.stage.keys m = @p.metadata(timekey: Time.parse('2016-04-11 16:40:00 +0000').to_i) - @p.emit(m, []) + @p.write({m => []}) assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3], @p.stage.keys end - test '#emit raises BufferOverflowError if buffer is not storable' do + test '#write raises BufferOverflowError if buffer is not storable' do @p.stage_size = 256 * 1024 * 1024 @p.queue_size = 256 * 1024 * 1024 m = @p.metadata(timekey: Time.parse('2016-04-11 16:40:00 +0000').to_i) assert_raise Fluent::Plugin::Buffer::BufferOverflowError do - @p.emit(m, ["x" * 256]) + @p.write({m => ["x" * 256]}) end end - test '#emit stores data into an existing chunk with metadata specified' do + test '#write stores data into an existing chunk with metadata specified' do assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3], @p.stage.keys @@ -527,7 +532,7 @@ def create_chunk(metadata, data) assert_equal 1, @p.stage[@dm3].append_count - @p.emit(@dm3, ["x" * 256, "y" * 256, "z" * 256]) + @p.write({@dm3 => ["x" * 256, "y" * 256, "z" * 256]}) assert_equal 2, @p.stage[@dm3].append_count assert_equal (dm3data + ("x" * 256) + ("y" * 256) + ("z" * 256)), @p.stage[@dm3].read @@ -537,7 +542,7 @@ def create_chunk(metadata, data) assert_equal [@dm2,@dm3], @p.stage.keys end - test '#emit creates new chunk and store data into it if there are no chunks for specified metadata' do + test '#write creates new chunk and store data into it if there are no chunks for specified metadata' do assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3], @p.stage.keys @@ -545,7 +550,7 @@ def create_chunk(metadata, data) m = @p.metadata(timekey: Time.parse('2016-04-11 16:40:00 +0000').to_i) - @p.emit(m, ["x" * 256, "y" * 256, "z" * 256]) + @p.write({m => ["x" * 256, "y" * 256, "z" * 256]}) assert_equal 1, @p.stage[m].append_count assert_equal ("x" * 256 + "y" * 256 + "z" * 256), @p.stage[m].read @@ -555,8 +560,9 @@ def create_chunk(metadata, data) assert_equal [@dm2,@dm3,m], @p.stage.keys end - test '#emit tries to enqueue and store data into a new chunk if existing chunk is full' do + test '#write tries to enqueue and store data into a new chunk if existing chunk is full' do assert_equal 8 * 1024 * 1024, @p.chunk_bytes_limit + assert_equal 0.95, @p.chunk_full_threshold assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3], @p.stage.keys @@ -564,30 +570,31 @@ def create_chunk(metadata, data) m = @p.metadata(timekey: Time.parse('2016-04-11 16:40:00 +0000').to_i) row = "x" * 1024 * 1024 - @p.emit(m, [row] * 8) + small_row = "x" * 1024 * 512 + @p.write({m => [row] * 7 + [small_row]}) assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3,m], @p.stage.keys assert_equal 1, @p.stage[m].append_count - @p.emit(m, [row]) + @p.write({m => [row]}) assert_equal [@dm0,@dm1,@dm1,m], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3,m], @p.stage.keys assert_equal 1, @p.stage[m].append_count assert_equal 1024*1024, @p.stage[m].bytesize - assert_equal 3, @p.queue.last.append_count # 1 -> emit (2) -> emit_step_by_step (3) + assert_equal 3, @p.queue.last.append_count # 1 -> write (2) -> write_step_by_step (3) assert @p.queue.last.rollbacked end - test '#emit rollbacks if commit raises errors' do + test '#write rollbacks if commit raises errors' do assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3], @p.stage.keys m = @p.metadata(timekey: Time.parse('2016-04-11 16:40:00 +0000').to_i) row = "x" * 1024 - @p.emit(m, [row] * 8) + @p.write({m => [row] * 8}) assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3,m], @p.stage.keys @@ -602,7 +609,7 @@ def create_chunk(metadata, data) end assert_raise "yay" do - @p.emit(m, [row]) + @p.write({m => [row]}) end assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) @@ -613,30 +620,31 @@ def create_chunk(metadata, data) assert_equal row * 8, target_chunk.read end - test '#emit_bulk returns immediately if argument data is nil or empty string' do + test '#write w/ bulk returns immediately if argument data is nil or empty string' do assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3], @p.stage.keys m = @p.metadata(timekey: Time.parse('2016-04-11 16:40:00 +0000').to_i) - @p.emit_bulk(m, '', 0) + @p.write({}, bulk: true) + @p.write({m => ['', 0]}, bulk: true) assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3], @p.stage.keys end - test '#emit_bulk raises BufferOverflowError if buffer is not storable' do + test '#write w/ bulk raises BufferOverflowError if buffer is not storable' do @p.stage_size = 256 * 1024 * 1024 @p.queue_size = 256 * 1024 * 1024 m = @p.metadata(timekey: Time.parse('2016-04-11 16:40:00 +0000').to_i) assert_raise Fluent::Plugin::Buffer::BufferOverflowError do - @p.emit_bulk(m, "x" * 256, 1) + @p.write({m => ["x" * 256, 1]}, bulk: true) end end - test '#emit_bulk stores data into an existing chunk with metadata specified' do + test '#write w/ bulk stores data into an existing chunk with metadata specified' do assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3], @p.stage.keys @@ -645,7 +653,7 @@ def create_chunk(metadata, data) assert_equal 1, @p.stage[@dm3].append_count - @p.emit_bulk(@dm3, ("x"*256 + "y"*256 + "z"*256), 3) + @p.write({@dm3 => [("x"*256 + "y"*256 + "z"*256), 3]}, bulk: true) assert_equal 2, @p.stage[@dm3].append_count assert_equal (dm3data + ("x" * 256) + ("y" * 256) + ("z" * 256)), @p.stage[@dm3].read @@ -655,7 +663,7 @@ def create_chunk(metadata, data) assert_equal [@dm2,@dm3], @p.stage.keys end - test '#emit_bulk creates new chunk and store data into it if there are not chunks for specified metadata' do + test '#write w/ bulk creates new chunk and store data into it if there are not chunks for specified metadata' do assert_equal 8 * 1024 * 1024, @p.chunk_bytes_limit assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) @@ -665,14 +673,14 @@ def create_chunk(metadata, data) row = "x" * 1024 * 1024 row_half = "x" * 1024 * 512 - @p.emit_bulk(m, row*7 + row_half, 8) + @p.write({m => [row*7 + row_half, 8]}, bulk: true) assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3,m], @p.stage.keys assert_equal 1, @p.stage[m].append_count end - test '#emit_bulk tries to enqueue and store data into a new chunk if existing chunk does not have space for bulk' do + test '#write w/ bulk tries to enqueue and store data into a new chunk if existing chunk does not have space for bulk' do assert_equal 8 * 1024 * 1024, @p.chunk_bytes_limit assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) @@ -682,23 +690,23 @@ def create_chunk(metadata, data) row = "x" * 1024 * 1024 row_half = "x" * 1024 * 512 - @p.emit_bulk(m, row*7 + row_half, 8) + @p.write({m => [row*7 + row_half, 8]}, bulk: true) assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3,m], @p.stage.keys assert_equal 1, @p.stage[m].append_count - @p.emit_bulk(m, row, 1) + @p.write({m => [row, 1]}, bulk: true) assert_equal [@dm0,@dm1,@dm1,m], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3,m], @p.stage.keys assert_equal 1, @p.stage[m].append_count assert_equal 1024*1024, @p.stage[m].bytesize - assert_equal 2, @p.queue.last.append_count # 1 -> emit (2) -> rollback&enqueue + assert_equal 2, @p.queue.last.append_count # 1 -> write (2) -> rollback&enqueue assert @p.queue.last.rollbacked end - test '#emit_bulk enqueues chunk if it is already full after adding bulk data' do + test '#write w/ bulk enqueues chunk if it is already full after adding bulk data' do assert_equal 8 * 1024 * 1024, @p.chunk_bytes_limit assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) @@ -707,14 +715,14 @@ def create_chunk(metadata, data) m = @p.metadata(timekey: Time.parse('2016-04-11 16:40:00 +0000').to_i) row = "x" * 1024 * 1024 - @p.emit_bulk(m, row * 8, 8) + @p.write({m => [row * 8, 8]}, bulk: true) assert_equal [@dm0,@dm1,@dm1,m], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3], @p.stage.keys assert_equal 1, @p.queue.last.append_count end - test '#emit_bulk rollbacks if commit raises errors' do + test '#write w/ bulk rollbacks if commit raises errors' do assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3], @p.stage.keys @@ -722,7 +730,7 @@ def create_chunk(metadata, data) row = "x" * 1024 row_half = "x" * 512 - @p.emit_bulk(m, row * 7 + row_half, 8) + @p.write({m => [row * 7 + row_half, 8]}, bulk: true) assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3,m], @p.stage.keys @@ -737,7 +745,7 @@ def create_chunk(metadata, data) end assert_raise "yay" do - @p.emit_bulk(m, row, 1) + @p.write({m => [row, 1]}, bulk: true) end assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) @@ -747,6 +755,52 @@ def create_chunk(metadata, data) assert target_chunk.rollbacked assert_equal row * 7 + row_half, target_chunk.read end + + test '#write writes many metadata and data pairs at once' do + assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) + assert_equal [@dm2,@dm3], @p.stage.keys + + row = "x" * 1024 + @p.write({ @dm0 => [row, row, row], @dm1 => [row, row] }, bulk: false) + + assert_equal [@dm2,@dm3,@dm0,@dm1], @p.stage.keys + end + + test '#write does not commit on any chunks if any append operation on chunk fails' do + assert_equal [@dm0,@dm1,@dm1], @p.queue.map(&:metadata) + assert_equal [@dm2,@dm3], @p.stage.keys + + row = "x" * 1024 + @p.write({ @dm0 => [row, row, row], @dm1 => [row, row] }, bulk: false) + + assert_equal [@dm2,@dm3,@dm0,@dm1], @p.stage.keys + + dm2_size = @p.stage[@dm2].size + assert !@p.stage[@dm2].rollbacked + dm3_size = @p.stage[@dm3].size + assert !@p.stage[@dm3].rollbacked + + assert{ @p.stage[@dm0].size == 3 } + assert !@p.stage[@dm0].rollbacked + assert{ @p.stage[@dm1].size == 2 } + assert !@p.stage[@dm1].rollbacked + + @p.stage[@dm1].failing = true + + assert_raise(FluentPluginBufferTest::DummyMemoryChunkError) do + @p.write({ @dm2 => [row], @dm3 => [row], @dm0 => [row, row, row], @dm1 => [row, row] }, bulk: false) + end + + assert{ @p.stage[@dm2].size == dm2_size } + assert @p.stage[@dm2].rollbacked + assert{ @p.stage[@dm3].size == dm3_size } + assert @p.stage[@dm3].rollbacked + + assert{ @p.stage[@dm0].size == 3 } + assert @p.stage[@dm0].rollbacked + assert{ @p.stage[@dm1].size == 2 } + assert @p.stage[@dm1].rollbacked + end end sub_test_case 'with configuration for test with lower limits' do @@ -789,7 +843,7 @@ def create_chunk(metadata, data) assert @p.storable? dm3 = @p.metadata(timekey: @dm3.timekey) - @p.emit(dm3, ["c" * 128]) + @p.write({dm3 => ["c" * 128]}) assert_equal 10240, (@p.stage_size + @p.queue_size) assert !@p.storable? @@ -817,18 +871,19 @@ def create_chunk(metadata, data) c2 = create_chunk(m, ["a" * 128] * 8) assert @p.chunk_size_full?(c2) - c3 = create_chunk(m, ["a" * 128] * 7 + ["a" * 127]) + assert_equal 0.95, @p.chunk_full_threshold + c3 = create_chunk(m, ["a" * 128] * 6 + ["a" * 64]) assert !@p.chunk_size_full?(c3) end - test '#emit raises BufferChunkOverflowError if incoming data is bigger than chunk bytes limit' do + test '#write raises BufferChunkOverflowError if incoming data is bigger than chunk bytes limit' do assert_equal [@dm0,@dm0,@dm0,@dm0,@dm0,@dm1,@dm1,@dm1,@dm1], @p.queue.map(&:metadata) assert_equal [@dm2,@dm3], @p.stage.keys m = create_metadata(Time.parse('2016-04-11 16:40:00 +0000').to_i) assert_raise Fluent::Plugin::Buffer::BufferChunkOverflowError do - @p.emit(m, ["a" * 128] * 9) + @p.write({m => ["a" * 128] * 9}) end end end diff --git a/test/plugin/test_buffer_chunk.rb b/test/plugin/test_buffer_chunk.rb index 7cd2bd38b0..6fd2f78412 100644 --- a/test/plugin/test_buffer_chunk.rb +++ b/test/plugin/test_buffer_chunk.rb @@ -10,6 +10,9 @@ class BufferChunkTest < Test::Unit::TestCase assert{ chunk.metadata.object_id == meta.object_id } assert{ chunk.created_at.is_a? Time } assert{ chunk.modified_at.is_a? Time } + assert chunk.staged? + assert !chunk.queued? + assert !chunk.closed? end test 'has many methods for chunks, but not implemented' do @@ -23,8 +26,6 @@ class BufferChunkTest < Test::Unit::TestCase assert chunk.respond_to?(:size) assert chunk.respond_to?(:length) assert chunk.respond_to?(:empty?) - assert chunk.respond_to?(:close) - assert chunk.respond_to?(:purge) assert chunk.respond_to?(:read) assert chunk.respond_to?(:open) assert chunk.respond_to?(:write_to) @@ -36,8 +37,6 @@ class BufferChunkTest < Test::Unit::TestCase assert_raise(NotImplementedError){ chunk.size } assert_raise(NotImplementedError){ chunk.length } assert_raise(NotImplementedError){ chunk.empty? } - assert_raise(NotImplementedError){ chunk.close } - assert_raise(NotImplementedError){ chunk.purge } assert_raise(NotImplementedError){ chunk.read } assert_raise(NotImplementedError){ chunk.open(){} } assert_raise(NotImplementedError){ chunk.write_to(nil) } @@ -62,6 +61,25 @@ def open end sub_test_case 'minimum chunk implements #size and #open' do + test 'chunk lifecycle' do + c = TestChunk.new(Object.new) + assert c.staged? + assert !c.queued? + assert !c.closed? + + c.enqueued! + + assert !c.staged? + assert c.queued? + assert !c.closed? + + c.close + + assert !c.staged? + assert !c.queued? + assert c.closed? + end + test 'can respond to #empty? correctly' do c = TestChunk.new(Object.new) assert_equal 0, c.size diff --git a/test/plugin/test_output_as_buffered.rb b/test/plugin/test_output_as_buffered.rb index 4d78b9d347..0212aaa65d 100644 --- a/test/plugin/test_output_as_buffered.rb +++ b/test/plugin/test_output_as_buffered.rb @@ -152,12 +152,12 @@ def waiting(seconds) t = event_time() es = Fluent::ArrayEventStream.new([ [t, {"key" => "value1"}], [t, {"key" => "value2"}] ]) - 5.times do + 4.times do @i.emit_events('tag.test', es) end - assert_equal 10, ary.size - 5.times do |i| + assert_equal 8, ary.size + 4.times do |i| assert_equal ["tag.test", t, {"key" => "value1"}], ary[i*2] assert_equal ["tag.test", t, {"key" => "value2"}], ary[i*2+1] end @@ -175,7 +175,7 @@ def waiting(seconds) end event_size = [tag, t, r].to_json.size # 195 - (1024 / event_size).times do |i| + (1024 * 0.9 / event_size).to_i.times do |i| @i.emit_events("test.tag", Fluent::ArrayEventStream.new([ [t, r] ])) end assert{ @i.buffer.queue.size == 0 && ary.size == 0 } @@ -208,7 +208,7 @@ def waiting(seconds) end event_size = [tag, t, r].to_json.size # 195 - (1024 / event_size).times do |i| + (1024 * 0.9 / event_size).to_i.times do |i| @i.emit_events("test.tag", Fluent::ArrayEventStream.new([ [t, r] ])) end assert{ @i.buffer.queue.size == 0 && ary.size == 0 } @@ -221,7 +221,7 @@ def waiting(seconds) waiting(10) do Thread.pass until ary.size == 1 end - assert_equal [tag,t,r].to_json * (1024 / event_size), ary.first + assert_equal [tag,t,r].to_json * (1024 * 0.9 / event_size), ary.first end end @@ -252,12 +252,12 @@ def waiting(seconds) t = event_time() es = Fluent::ArrayEventStream.new([ [t, {"key" => "value1"}], [t, {"key" => "value2"}] ]) - 5.times do + 4.times do @i.emit_events('tag.test', es) end - assert_equal 10, ary.size - 5.times do |i| + assert_equal 8, ary.size + 4.times do |i| assert_equal ["tag.test", t, {"key" => "value1"}], ary[i*2] assert_equal ["tag.test", t, {"key" => "value2"}], ary[i*2+1] end @@ -278,14 +278,17 @@ def waiting(seconds) end 3.times do |i| - rand_records = rand(1..5) + rand_records = rand(1..4) es = Fluent::ArrayEventStream.new([ [t, r] ] * rand_records) assert_equal rand_records, es.size @i.interrupt_flushes + assert{ @i.buffer.queue.size == 0 } + @i.emit_events("test.tag", es) + assert{ @i.buffer.queue.size == 0 } assert{ @i.buffer.stage.size == 1 } staged_chunk = @i.instance_eval{ @buffer.stage[@buffer.stage.keys.first] } @@ -315,7 +318,7 @@ def waiting(seconds) end event_size = [tag, t, r].to_json.size # 195 - (1024 / event_size).times do |i| + (1024 * 0.9 / event_size).to_i.times do |i| @i.emit_events("test.tag", Fluent::ArrayEventStream.new([ [t, r] ])) end assert{ @i.buffer.queue.size == 0 && ary.size == 0 } @@ -328,7 +331,7 @@ def waiting(seconds) waiting(10) do Thread.pass until ary.size == 1 end - assert_equal [tag,t,r].to_json * (1024 / event_size), ary.first + assert_equal [tag,t,r].to_json * (1024 * 0.9 / event_size), ary.first end end @@ -358,12 +361,12 @@ def waiting(seconds) t = event_time() es = Fluent::ArrayEventStream.new([ [t, {"key" => "value1"}], [t, {"key" => "value2"}] ]) - 5.times do + 4.times do @i.emit_events('tag.test', es) end - assert_equal 10, ary.size - 5.times do |i| + assert_equal 8, ary.size + 4.times do |i| assert_equal ["tag.test", t, {"key" => "value1"}], ary[i*2] assert_equal ["tag.test", t, {"key" => "value2"}], ary[i*2+1] end diff --git a/test/plugin/test_output_as_standard.rb b/test/plugin/test_output_as_standard.rb index 05b8021a93..37de54d548 100644 --- a/test/plugin/test_output_as_standard.rb +++ b/test/plugin/test_output_as_standard.rb @@ -81,7 +81,7 @@ def test_event_stream end sub_test_case 'standard buffered without any chunk keys' do - test '#execute_chunking calls @buffer.emit_bulk just once with predefined msgpack format' do + test '#execute_chunking calls @buffer.write(bulk: true) just once with predefined msgpack format' do @i = create_output(:standard) @i.configure(config_element()) @i.start @@ -90,12 +90,12 @@ def test_event_stream es = test_event_stream buffer_mock = flexmock(@i.buffer) - buffer_mock.should_receive(:emit_bulk).once.with(m, es.to_msgpack_stream, es.size) + buffer_mock.should_receive(:write).once.with({m => [es.to_msgpack_stream, es.size]}, bulk: true, enqueue: false) @i.execute_chunking("mytag.test", es) end - test '#execute_chunking calls @buffer.emit_bulk just once with predefined msgpack format, but time will be int if time_as_integer specified' do + test '#execute_chunking calls @buffer.write(bulk: true) just once with predefined msgpack format, but time will be int if time_as_integer specified' do @i = create_output(:standard) @i.configure(config_element('ROOT','',{"time_as_integer"=>"true"})) @i.start @@ -104,14 +104,14 @@ def test_event_stream es = test_event_stream buffer_mock = flexmock(@i.buffer) - buffer_mock.should_receive(:emit_bulk).once.with(m, es.to_msgpack_stream(time_int: true), es.size) + buffer_mock.should_receive(:write).once.with({m => [es.to_msgpack_stream(time_int: true), es.size]}, bulk: true, enqueue: false) @i.execute_chunking("mytag.test", es) end end sub_test_case 'standard buffered with tag chunk key' do - test '#execute_chunking calls @buffer.emit_bulk just once with predefined msgpack format' do + test '#execute_chunking calls @buffer.write(bulk: true) just once with predefined msgpack format' do @i = create_output(:standard) @i.configure(config_element('ROOT','',{},[config_element('buffer','tag')])) @i.start @@ -120,12 +120,12 @@ def test_event_stream es = test_event_stream buffer_mock = flexmock(@i.buffer) - buffer_mock.should_receive(:emit_bulk).once.with(m, es.to_msgpack_stream, es.size) + buffer_mock.should_receive(:write).once.with({m => [es.to_msgpack_stream, es.size]}, bulk: true, enqueue: false) @i.execute_chunking("mytag.test", es) end - test '#execute_chunking calls @buffer.emit_bulk just once with predefined msgpack format, but time will be int if time_as_integer specified' do + test '#execute_chunking calls @buffer.write(bulk: true) just once with predefined msgpack format, but time will be int if time_as_integer specified' do @i = create_output(:standard) @i.configure(config_element('ROOT','',{"time_as_integer"=>"true"},[config_element('buffer','tag')])) @i.start @@ -134,14 +134,14 @@ def test_event_stream es = test_event_stream buffer_mock = flexmock(@i.buffer) - buffer_mock.should_receive(:emit_bulk).once.with(m, es.to_msgpack_stream(time_int: true), es.size) + buffer_mock.should_receive(:write).once.with({m => [es.to_msgpack_stream(time_int: true), es.size]}, bulk: true, enqueue: false) @i.execute_chunking("mytag.test", es) end end sub_test_case 'standard buffered with time chunk key' do - test '#execute_chunking calls @buffer.emit_bulk in times of # of time ranges with predefined msgpack format' do + test '#execute_chunking calls @buffer.write(bulk: true) with predefined msgpack format' do @i = create_output(:standard) @i.configure(config_element('ROOT','',{},[config_element('buffer','time',{"timekey_range" => "60"})])) @i.start @@ -163,15 +163,17 @@ def test_event_stream es3.add(event_time('2016-04-21 17:21:32 -0700'), {"key" => "my value", "name" => "moris1", "message" => "hello!"}) buffer_mock = flexmock(@i.buffer) - buffer_mock.should_receive(:emit_bulk).with(m1, es1.to_msgpack_stream, 3).once - buffer_mock.should_receive(:emit_bulk).with(m2, es2.to_msgpack_stream, 2).once - buffer_mock.should_receive(:emit_bulk).with(m3, es3.to_msgpack_stream, 1).once + buffer_mock.should_receive(:write).once.with({ + m1 => [es1.to_msgpack_stream, 3], + m2 => [es2.to_msgpack_stream, 2], + m3 => [es3.to_msgpack_stream, 1], + }, bulk: true, enqueue: false) es = test_event_stream @i.execute_chunking("mytag.test", es) end - test '#execute_chunking calls @buffer.emit_bulk in times of # of time ranges with predefined msgpack format, but time will be int if time_as_integer specified' do + test '#execute_chunking calls @buffer.write(bulk: true) with predefined msgpack format, but time will be int if time_as_integer specified' do @i = create_output(:standard) @i.configure(config_element('ROOT','',{"time_as_integer" => "true"},[config_element('buffer','time',{"timekey_range" => "60"})])) @i.start @@ -193,9 +195,11 @@ def test_event_stream es3.add(event_time('2016-04-21 17:21:32 -0700'), {"key" => "my value", "name" => "moris1", "message" => "hello!"}) buffer_mock = flexmock(@i.buffer) - buffer_mock.should_receive(:emit_bulk).with(m1, es1.to_msgpack_stream(time_int: true), 3).once - buffer_mock.should_receive(:emit_bulk).with(m2, es2.to_msgpack_stream(time_int: true), 2).once - buffer_mock.should_receive(:emit_bulk).with(m3, es3.to_msgpack_stream(time_int: true), 1).once + buffer_mock.should_receive(:write).with({ + m1 => [es1.to_msgpack_stream(time_int: true), 3], + m2 => [es2.to_msgpack_stream(time_int: true), 2], + m3 => [es3.to_msgpack_stream(time_int: true), 1], + }, bulk: true, enqueue: false) es = test_event_stream @i.execute_chunking("mytag.test", es) @@ -203,7 +207,7 @@ def test_event_stream end sub_test_case 'standard buffered with variable chunk keys' do - test '#execute_chunking calls @buffer.emit_bulk in times of # of variable variations with predefined msgpack format' do + test '#execute_chunking calls @buffer.write(bulk: true) with predefined msgpack format' do @i = create_output(:standard) @i.configure(config_element('ROOT','',{},[config_element('buffer','key,name')])) @i.start @@ -221,14 +225,16 @@ def test_event_stream es2.add(event_time('2016-04-21 17:19:13 -0700'), {"key" => "my value", "name" => "moris2", "message" => "hello!"}) buffer_mock = flexmock(@i.buffer) - buffer_mock.should_receive(:emit_bulk).with(m1, es1.to_msgpack_stream, 5).once - buffer_mock.should_receive(:emit_bulk).with(m2, es2.to_msgpack_stream, 1).once + buffer_mock.should_receive(:write).with({ + m1 => [es1.to_msgpack_stream, 5], + m2 => [es2.to_msgpack_stream, 1], + }, bulk: true, enqueue: false).once es = test_event_stream @i.execute_chunking("mytag.test", es) end - test '#execute_chunking calls @buffer.emit_bulk in times of # of variable variations with predefined msgpack format, but time will be int if time_as_integer specified' do + test '#execute_chunking calls @buffer.write(bulk: true) in times of # of variable variations with predefined msgpack format, but time will be int if time_as_integer specified' do @i = create_output(:standard) @i.configure(config_element('ROOT','',{"time_as_integer" => "true"},[config_element('buffer','key,name')])) @i.start @@ -246,8 +252,10 @@ def test_event_stream es2.add(event_time('2016-04-21 17:19:13 -0700'), {"key" => "my value", "name" => "moris2", "message" => "hello!"}) buffer_mock = flexmock(@i.buffer) - buffer_mock.should_receive(:emit_bulk).with(m1, es1.to_msgpack_stream(time_int: true), 5).once - buffer_mock.should_receive(:emit_bulk).with(m2, es2.to_msgpack_stream(time_int: true), 1).once + buffer_mock.should_receive(:write).with({ + m1 => [es1.to_msgpack_stream(time_int: true), 5], + m2 => [es2.to_msgpack_stream(time_int: true), 1], + }, bulk: true, enqueue: false).once es = test_event_stream @i.execute_chunking("mytag.test", es) @@ -255,7 +263,7 @@ def test_event_stream end sub_test_case 'custom format buffered without any chunk keys' do - test '#execute_chunking calls @buffer.emit_bulk just once with customized format' do + test '#execute_chunking calls @buffer.write(bulk: true) just once with customized format' do @i = create_output(:buffered) @i.register(:format){|tag, time, record| [time, record].to_json } @i.configure(config_element()) @@ -265,14 +273,14 @@ def test_event_stream es = test_event_stream buffer_mock = flexmock(@i.buffer) - buffer_mock.should_receive(:emit_bulk).once.with(m, es.map{|t,r| [t,r].to_json }.join, es.size) + buffer_mock.should_receive(:write).once.with({m => [es.map{|t,r| [t,r].to_json }.join, es.size]}, bulk: true, enqueue: false) @i.execute_chunking("mytag.test", es) end end sub_test_case 'custom format buffered with tag chunk key' do - test '#execute_chunking calls @buffer.emit_bulk just once with customized format' do + test '#execute_chunking calls @buffer.write(bulk: true) just once with customized format' do @i = create_output(:buffered) @i.register(:format){|tag, time, record| [time, record].to_json } @i.configure(config_element('ROOT','',{},[config_element('buffer','tag')])) @@ -282,13 +290,13 @@ def test_event_stream es = test_event_stream buffer_mock = flexmock(@i.buffer) - buffer_mock.should_receive(:emit_bulk).once.with(m, es.map{|t,r| [t,r].to_json }.join, es.size) + buffer_mock.should_receive(:write).once.with({m => [es.map{|t,r| [t,r].to_json }.join, es.size]}, bulk: true, enqueue: false) @i.execute_chunking("mytag.test", es) end end sub_test_case 'custom format buffered with time chunk key' do - test '#execute_chunking calls @buffer.emit in times of # of time ranges with customized format' do + test '#execute_chunking calls @buffer.write with customized format' do @i = create_output(:buffered) @i.register(:format){|tag, time, record| [time, record].to_json } @i.configure(config_element('ROOT','',{},[config_element('buffer','time',{"timekey_range" => "60"})])) @@ -311,9 +319,11 @@ def test_event_stream es3.add(event_time('2016-04-21 17:21:32 -0700'), {"key" => "my value", "name" => "moris1", "message" => "hello!"}) buffer_mock = flexmock(@i.buffer) - buffer_mock.should_receive(:emit).with(m1, es1.map{|t,r| [t,r].to_json }).once - buffer_mock.should_receive(:emit).with(m2, es2.map{|t,r| [t,r].to_json }).once - buffer_mock.should_receive(:emit).with(m3, es3.map{|t,r| [t,r].to_json }).once + buffer_mock.should_receive(:write).with({ + m1 => es1.map{|t,r| [t,r].to_json }, + m2 => es2.map{|t,r| [t,r].to_json }, + m3 => es3.map{|t,r| [t,r].to_json }, + }, bulk: false, enqueue: false).once es = test_event_stream @i.execute_chunking("mytag.test", es) @@ -321,7 +331,7 @@ def test_event_stream end sub_test_case 'custom format buffered with variable chunk keys' do - test '#execute_chunking calls @buffer.emit in times of # of variable variations with customized format' do + test '#execute_chunking calls @buffer.write in times of # of variable variations with customized format' do @i = create_output(:buffered) @i.register(:format){|tag, time, record| [time, record].to_json } @i.configure(config_element('ROOT','',{},[config_element('buffer','key,name')])) @@ -340,8 +350,10 @@ def test_event_stream es2.add(event_time('2016-04-21 17:19:13 -0700'), {"key" => "my value", "name" => "moris2", "message" => "hello!"}) buffer_mock = flexmock(@i.buffer) - buffer_mock.should_receive(:emit).with(m1, es1.map{|t,r| [t,r].to_json }).once - buffer_mock.should_receive(:emit).with(m2, es2.map{|t,r| [t,r].to_json }).once + buffer_mock.should_receive(:write).with({ + m1 => es1.map{|t,r| [t,r].to_json }, + m2 => es2.map{|t,r| [t,r].to_json }, + }, bulk: false, enqueue: false).once es = test_event_stream @i.execute_chunking("mytag.test", es) From cd65e0e8e4de5f12e41561dceff25811966aafe4 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Thu, 12 May 2016 19:26:41 +0900 Subject: [PATCH 04/29] Implement overflow_action (buffer_queue_full_action in v0.12) --- lib/fluent/plugin/output.rb | 49 +++- .../test_output_as_buffered_overflow.rb | 242 ++++++++++++++++++ 2 files changed, 288 insertions(+), 3 deletions(-) create mode 100644 test/plugin/test_output_as_buffered_overflow.rb diff --git a/lib/fluent/plugin/output.rb b/lib/fluent/plugin/output.rb index 739ac0c3b6..e3926e0cb0 100644 --- a/lib/fluent/plugin/output.rb +++ b/lib/fluent/plugin/output.rb @@ -64,6 +64,8 @@ class Output < Base config_param :delayed_commit_timeout, :time, default: 60, desc: 'Seconds of timeout for buffer chunks to be committed by plugins later.' + config_param :overflow_action, :enum, list: [:exception, :block, :drop_oldest_chunk], default: :exception, desc: 'The action when the size of buffer exceeds the limit.' + config_param :retry_forever, :bool, default: false, desc: 'If true, plugin will ignore retry_timeout and retry_max_times options and retry flushing forever.' config_param :retry_timeout, :time, default: 72 * 60 * 60, desc: 'The maximum seconds to retry to flush while failing, until plugin discards buffer chunks.' # 72hours == 17 times with exponential backoff (not to change default behavior) @@ -560,6 +562,41 @@ def execute_chunking(tag, es, enqueue: false) end end + def write_guard(&block) + begin + block.call + rescue Fluent::Plugin::Buffer::BufferOverflowError + log.warn "failed to write data into buffer by buffer overflow" + case @buffer_config.overflow_action + when :exception + raise + when :block + log.debug "buffer.write is now blocking" + until @buffer.storable? + sleep 1 + end + log.debug "retrying buffer.write after blocked operation" + retry + when :drop_oldest_chunk + begin + oldest = @buffer.dequeue_chunk + if oldest + log.warn "dropping oldest chunk to make space after buffer overflow", chunk_id: oldest.unique_id + @buffer.purge_chunk(oldest.unique_id) + else + log.error "no queued chunks to be dropped for drop_oldest_chunk" + end + rescue + # ignore any errors + end + raise unless @buffer.storable? + retry + else + raise "BUG: unknown overflow_action '#{@buffer_config.overflow_action}'" + end + end + end + def handle_stream_with_custom_format(tag, es, enqueue: false) meta_and_data = {} records = 0 @@ -569,7 +606,9 @@ def handle_stream_with_custom_format(tag, es, enqueue: false) meta_and_data[meta] << format(tag, time, record) records += 1 end - @buffer.write(meta_and_data, bulk: false, enqueue: enqueue) + write_guard do + @buffer.write(meta_and_data, bulk: false, enqueue: enqueue) + end @counters_monitor.synchronize{ @emit_records += records } true end @@ -587,7 +626,9 @@ def handle_stream_with_standard_format(tag, es, enqueue: false) meta_and_data.each_pair do |meta, es| meta_and_data_bulk[meta] = [es.to_msgpack_stream(time_int: @time_as_integer), es.size] end - @buffer.write(meta_and_data_bulk, bulk: true, enqueue: enqueue) + write_guard do + @buffer.write(meta_and_data_bulk, bulk: true, enqueue: enqueue) + end @counters_monitor.synchronize{ @emit_records += records } true end @@ -608,7 +649,9 @@ def handle_stream_simple(tag, es, enqueue: false) es_size = es.size es_bulk = es.to_msgpack_stream(time_int: @time_as_integer) end - @buffer.write({meta => [es_bulk, es_size]}, bulk: true, enqueue: enqueue) + write_guard do + @buffer.write({meta => [es_bulk, es_size]}, bulk: true, enqueue: enqueue) + end @counters_monitor.synchronize{ @emit_records += records } true end diff --git a/test/plugin/test_output_as_buffered_overflow.rb b/test/plugin/test_output_as_buffered_overflow.rb new file mode 100644 index 0000000000..a60d8d7143 --- /dev/null +++ b/test/plugin/test_output_as_buffered_overflow.rb @@ -0,0 +1,242 @@ +require_relative '../helper' +require 'fluent/plugin/output' +require 'fluent/plugin/buffer' +require 'fluent/event' + +require 'json' +require 'time' +require 'timeout' +require 'timecop' + +module FluentPluginOutputAsBufferedOverflowTest + class DummyBareOutput < Fluent::Plugin::Output + def register(name, &block) + instance_variable_set("@#{name}", block) + end + end + class DummyAsyncOutput < DummyBareOutput + def format(tag, time, record) + @format ? @format.call(tag, time, record) : [tag, time, record].to_json + end + def write(chunk) + @write ? @write.call(chunk) : nil + end + end +end + +class BufferedOutputOverflowTest < Test::Unit::TestCase + def create_output + FluentPluginOutputAsBufferedOverflowTest::DummyAsyncOutput.new + end + def create_metadata(timekey: nil, tag: nil, variables: nil) + Fluent::Plugin::Buffer::Metadata.new(timekey, tag, variables) + end + def waiting(seconds) + begin + Timeout.timeout(seconds) do + yield + end + rescue Timeout::Error + STDERR.print *(@i.log.out.logs) + raise + end + end + + teardown do + if @i + @i.stop unless @i.stopped? + @i.before_shutdown unless @i.before_shutdown? + @i.shutdown unless @i.shutdown? + @i.after_shutdown unless @i.after_shutdown? + @i.close unless @i.closed? + @i.terminate unless @i.terminated? + end + Timecop.return + end + + sub_test_case 'buffered output with default configuration (throws exception for buffer overflow)' do + setup do + hash = { + 'flush_mode' => 'none', + 'flush_burst_interval' => 0.01, + 'chunk_bytes_limit' => 1024, + 'total_bytes_limit' => 4096, + } + @i = create_output() + @i.configure(config_element('ROOT','',{},[config_element('buffer','tag',hash)])) + @i.start + end + + test '#emit_events raises error when buffer is full' do + @i.register(:format){|tag, time, record| "x" * 128 } # 128bytes per record (x4 -> 512bytes) + + es = Fluent::ArrayEventStream.new([ + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + ]) + + 8.times do |i| + @i.emit_events("tag#{i}", es) + end + + assert !@i.buffer.storable? + + assert_raise(Fluent::Plugin::Buffer::BufferOverflowError) do + @i.emit_events("tag9", es) + end + logs = @i.log.out.logs + assert{ logs.any?{|line| line.include?("failed to write data into buffer by buffer overflow") } } + end + end + + sub_test_case 'buffered output configured with "overflow_action block"' do + setup do + hash = { + 'flush_mode' => 'none', + 'flush_burst_interval' => 0.01, + 'chunk_bytes_limit' => 1024, + 'total_bytes_limit' => 4096, + 'overflow_action' => "block", + } + @i = create_output() + @i.configure(config_element('ROOT','',{'log_level' => 'debug'},[config_element('buffer','tag',hash)])) + @i.start + end + + test '#emit_events blocks until any queues are flushed' do + failing = true + flushed_chunks = [] + @i.register(:format){|tag, time, record| "x" * 128 } # 128bytes per record (x4 -> 512bytes) + @i.register(:write) do |chunk| + if failing + raise "blocking" + end + flushed_chunks << chunk + end + + es = Fluent::ArrayEventStream.new([ + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + ]) + + 4.times do |i| + @i.emit_events("tag#{i}", es) + end + + assert !@i.buffer.storable? + + thread = Thread.new do + sleep 3 + failing = false + end + + assert_nothing_raised do + @i.emit_events("tag9", es) + end + + assert !failing + assert{ flushed_chunks.size > 0 } + + logs = @i.log.out.logs + assert{ logs.any?{|line| line.include?("failed to write data into buffer by buffer overflow") } } + assert{ logs.any?{|line| line.include?("buffer.write is now blocking") } } + assert{ logs.any?{|line| line.include?("retrying buffer.write after blocked operation") } } + end + end + + sub_test_case 'buffered output configured with "overflow_action drop_oldest_chunk"' do + setup do + hash = { + 'flush_mode' => 'none', + 'flush_burst_interval' => 0.01, + 'chunk_bytes_limit' => 1024, + 'total_bytes_limit' => 4096, + 'overflow_action' => "drop_oldest_chunk", + } + @i = create_output() + @i.configure(config_element('ROOT','',{'log_level' => 'debug'},[config_element('buffer','tag',hash)])) + @i.start + end + + test '#emit_events will success by dropping oldest chunk' do + failing = true + flushed_chunks = [] + @i.register(:format){|tag, time, record| "x" * 128 } # 128bytes per record (x4 -> 512bytes) + @i.register(:write) do |chunk| + if failing + raise "blocking" + end + flushed_chunks << chunk + end + + es = Fluent::ArrayEventStream.new([ + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + ]) + + 4.times do |i| + @i.emit_events("tag#{i}", es) + end + + assert !@i.buffer.storable? + + assert{ @i.buffer.queue[0].metadata.tag == "tag0" } + assert{ @i.buffer.queue[1].metadata.tag == "tag1" } + + assert_nothing_raised do + @i.emit_events("tag9", es) + end + + assert failing + assert{ flushed_chunks.size == 0 } + + assert{ @i.buffer.queue[0].metadata.tag == "tag1" } + + logs = @i.log.out.logs + assert{ logs.any?{|line| line.include?("failed to write data into buffer by buffer overflow") } } + assert{ logs.any?{|line| line.include?("dropping oldest chunk to make space after buffer overflow") } } + end + + test '#emit_events raises OverflowError if all buffer spaces are used by staged chunks' do + @i.register(:format){|tag, time, record| "x" * 128 } # 128bytes per record (x4 -> 512bytes) + + es = Fluent::ArrayEventStream.new([ + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + [event_time(), {"message" => "test"}], + ]) + + 8.times do |i| + @i.emit_events("tag#{i}", es) + end + + assert !@i.buffer.storable? + + assert{ @i.buffer.queue.size == 0 } + assert{ @i.buffer.stage.size == 8 } + + assert_raise Fluent::Plugin::Buffer::BufferOverflowError do + @i.emit_events("tag9", es) + end + + logs = @i.log.out.logs + assert{ logs.any?{|line| line.include?("failed to write data into buffer by buffer overflow") } } + assert{ logs.any?{|line| line.include?("no queued chunks to be dropped for drop_oldest_chunk") } } + end + end +end From 76c82643ef72b0994f999ee13241134d7a001738 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Thu, 12 May 2016 19:30:32 +0900 Subject: [PATCH 05/29] set shorter flush_burst_interval to make teardown shorter --- test/plugin/test_output_as_buffered.rb | 4 ++++ test/plugin/test_output_as_standard.rb | 18 +++++++++--------- 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/test/plugin/test_output_as_buffered.rb b/test/plugin/test_output_as_buffered.rb index 0212aaa65d..b7bf1977d8 100644 --- a/test/plugin/test_output_as_buffered.rb +++ b/test/plugin/test_output_as_buffered.rb @@ -129,6 +129,7 @@ def waiting(seconds) setup do hash = { 'flush_mode' => 'none', + 'flush_burst_interval' => 0.01, 'flush_threads' => 2, 'chunk_bytes_limit' => 1024, } @@ -231,6 +232,7 @@ def waiting(seconds) 'flush_mode' => 'fast', 'flush_interval' => 1, 'flush_threads' => 1, + 'flush_burst_interval' => 0.01, 'chunk_bytes_limit' => 1024, } @i = create_output(:buffered) @@ -340,6 +342,7 @@ def waiting(seconds) hash = { 'flush_mode' => 'immediate', 'flush_threads' => 1, + 'flush_burst_interval' => 0.01, 'chunk_bytes_limit' => 1024, } @i = create_output(:buffered) @@ -434,6 +437,7 @@ def waiting(seconds) 'timekey_range' => 30, # per 30seconds 'timekey_wait' => 5, # 5 second delay for flush 'flush_threads' => 1, + 'flush_burst_interval' => 0.01, } @i = create_output(:buffered) @i.configure(config_element('ROOT','',{},[config_element('buffer',chunk_key,hash)])) diff --git a/test/plugin/test_output_as_standard.rb b/test/plugin/test_output_as_standard.rb index 37de54d548..dd605d3ada 100644 --- a/test/plugin/test_output_as_standard.rb +++ b/test/plugin/test_output_as_standard.rb @@ -113,7 +113,7 @@ def test_event_stream sub_test_case 'standard buffered with tag chunk key' do test '#execute_chunking calls @buffer.write(bulk: true) just once with predefined msgpack format' do @i = create_output(:standard) - @i.configure(config_element('ROOT','',{},[config_element('buffer','tag')])) + @i.configure(config_element('ROOT','',{},[config_element('buffer','tag',{'flush_burst_interval' => 0.01})])) @i.start m = create_metadata(tag: "mytag.test") @@ -127,7 +127,7 @@ def test_event_stream test '#execute_chunking calls @buffer.write(bulk: true) just once with predefined msgpack format, but time will be int if time_as_integer specified' do @i = create_output(:standard) - @i.configure(config_element('ROOT','',{"time_as_integer"=>"true"},[config_element('buffer','tag')])) + @i.configure(config_element('ROOT','',{"time_as_integer"=>"true"},[config_element('buffer','tag',{'flush_burst_interval' => 0.01})])) @i.start m = create_metadata(tag: "mytag.test") @@ -143,7 +143,7 @@ def test_event_stream sub_test_case 'standard buffered with time chunk key' do test '#execute_chunking calls @buffer.write(bulk: true) with predefined msgpack format' do @i = create_output(:standard) - @i.configure(config_element('ROOT','',{},[config_element('buffer','time',{"timekey_range" => "60"})])) + @i.configure(config_element('ROOT','',{},[config_element('buffer','time',{"timekey_range" => "60",'flush_burst_interval' => 0.01})])) @i.start m1 = create_metadata(timekey: Time.parse('2016-04-21 17:19:00 -0700').to_i) @@ -175,7 +175,7 @@ def test_event_stream test '#execute_chunking calls @buffer.write(bulk: true) with predefined msgpack format, but time will be int if time_as_integer specified' do @i = create_output(:standard) - @i.configure(config_element('ROOT','',{"time_as_integer" => "true"},[config_element('buffer','time',{"timekey_range" => "60"})])) + @i.configure(config_element('ROOT','',{"time_as_integer" => "true"},[config_element('buffer','time',{"timekey_range" => "60",'flush_burst_interval' => 0.01})])) @i.start m1 = create_metadata(timekey: Time.parse('2016-04-21 17:19:00 -0700').to_i) @@ -209,7 +209,7 @@ def test_event_stream sub_test_case 'standard buffered with variable chunk keys' do test '#execute_chunking calls @buffer.write(bulk: true) with predefined msgpack format' do @i = create_output(:standard) - @i.configure(config_element('ROOT','',{},[config_element('buffer','key,name')])) + @i.configure(config_element('ROOT','',{},[config_element('buffer','key,name',{'flush_burst_interval' => 0.01})])) @i.start m1 = create_metadata(variables: {key: "my value", name: "moris1"}) @@ -236,7 +236,7 @@ def test_event_stream test '#execute_chunking calls @buffer.write(bulk: true) in times of # of variable variations with predefined msgpack format, but time will be int if time_as_integer specified' do @i = create_output(:standard) - @i.configure(config_element('ROOT','',{"time_as_integer" => "true"},[config_element('buffer','key,name')])) + @i.configure(config_element('ROOT','',{"time_as_integer" => "true"},[config_element('buffer','key,name',{'flush_burst_interval' => 0.01})])) @i.start m1 = create_metadata(variables: {key: "my value", name: "moris1"}) @@ -283,7 +283,7 @@ def test_event_stream test '#execute_chunking calls @buffer.write(bulk: true) just once with customized format' do @i = create_output(:buffered) @i.register(:format){|tag, time, record| [time, record].to_json } - @i.configure(config_element('ROOT','',{},[config_element('buffer','tag')])) + @i.configure(config_element('ROOT','',{},[config_element('buffer','tag',{'flush_burst_interval' => 0.01})])) @i.start m = create_metadata(tag: "mytag.test") @@ -299,7 +299,7 @@ def test_event_stream test '#execute_chunking calls @buffer.write with customized format' do @i = create_output(:buffered) @i.register(:format){|tag, time, record| [time, record].to_json } - @i.configure(config_element('ROOT','',{},[config_element('buffer','time',{"timekey_range" => "60"})])) + @i.configure(config_element('ROOT','',{},[config_element('buffer','time',{"timekey_range" => "60",'flush_burst_interval' => 0.01})])) @i.start m1 = create_metadata(timekey: Time.parse('2016-04-21 17:19:00 -0700').to_i) @@ -334,7 +334,7 @@ def test_event_stream test '#execute_chunking calls @buffer.write in times of # of variable variations with customized format' do @i = create_output(:buffered) @i.register(:format){|tag, time, record| [time, record].to_json } - @i.configure(config_element('ROOT','',{},[config_element('buffer','key,name')])) + @i.configure(config_element('ROOT','',{},[config_element('buffer','key,name',{'flush_burst_interval' => 0.01})])) @i.start m1 = create_metadata(variables: {key: "my value", name: "moris1"}) From 6de12b06b1b43ef933cd8f24ce53dba2ec3791a0 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Thu, 12 May 2016 20:06:06 +0900 Subject: [PATCH 06/29] add v0.12 plugin compatibility --- lib/fluent/compat/output.rb | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/fluent/compat/output.rb b/lib/fluent/compat/output.rb index 362deeefd2..af44d8d6a1 100644 --- a/lib/fluent/compat/output.rb +++ b/lib/fluent/compat/output.rb @@ -189,7 +189,7 @@ def support_in_v12_style?(feature) desc 'The length limit of the chunk queue.' config_param :buffer_queue_limit, :integer, default: 256 desc 'The action when the size of buffer queue exceeds the buffer_queue_limit.' - config_param :buffer_queue_full_action, :enum, list: [:exception, :block], default: :exception + config_param :buffer_queue_full_action, :enum, list: [:exception, :block, :drop_oldest_chunk], default: :exception config_param :flush_at_shutdown, :bool, default: true @@ -204,7 +204,7 @@ def support_in_v12_style?(feature) "max_retry_wait" => "retry_max_interval", "buffer_chunk_limit" => "chunk_bytes_limit", "buffer_queue_limit" => "queue_length_limit", - "buffer_queue_full_action" => nil, # TODO: implement this on fluent/plugin/buffer + "buffer_queue_full_action" => "overflow_action", "flush_at_shutdown" => "flush_at_shutdown", } From 36a37c49026ada186ce57d3f757cbe0d8458075d Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Mon, 9 May 2016 15:47:37 +0900 Subject: [PATCH 07/29] for readability --- lib/fluent/compat/output.rb | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/lib/fluent/compat/output.rb b/lib/fluent/compat/output.rb index af44d8d6a1..189b684859 100644 --- a/lib/fluent/compat/output.rb +++ b/lib/fluent/compat/output.rb @@ -313,7 +313,7 @@ def support_in_v12_style?(feature) desc 'The length limit of the chunk queue.' config_param :buffer_queue_limit, :integer, default: 256 desc 'The action when the size of buffer queue exceeds the buffer_queue_limit.' - config_param :buffer_queue_full_action, :enum, list: [:exception, :block], default: :exception + config_param :buffer_queue_full_action, :enum, list: [:exception, :block, :drop_oldest_chunk], default: :exception config_param :flush_at_shutdown, :bool, default: true @@ -330,7 +330,7 @@ def support_in_v12_style?(feature) "max_retry_wait" => "retry_max_interval", "buffer_chunk_limit" => "chunk_bytes_limit", "buffer_queue_limit" => "queue_length_limit", - "buffer_queue_full_action" => nil, # TODO: implement this on fluent/plugin/buffer + "buffer_queue_full_action" => "overflow_action", "flush_at_shutdown" => "flush_at_shutdown", } @@ -426,7 +426,7 @@ def support_in_v12_style?(feature) desc 'The length limit of the chunk queue.' config_param :buffer_queue_limit, :integer, default: 256 desc 'The action when the size of buffer queue exceeds the buffer_queue_limit.' - config_param :buffer_queue_full_action, :enum, list: [:exception, :block], default: :exception + config_param :buffer_queue_full_action, :enum, list: [:exception, :block, :drop_oldest_chunk], default: :exception config_param :flush_at_shutdown, :bool, default: false @@ -448,7 +448,7 @@ def support_in_v12_style?(feature) "max_retry_wait" => "retry_max_interval", "buffer_chunk_limit" => "chunk_bytes_limit", "buffer_queue_limit" => "queue_length_limit", - "buffer_queue_full_action" => nil, # TODO: implement this on fluent/plugin/buffer + "buffer_queue_full_action" => "overflow_action", "flush_at_shutdown" => "flush_at_shutdown", "time_slice_wait" => "timekey_wait", } From e02204ab6fe5d1debbba22b9d4bd53a9e9ab4869 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Thu, 12 May 2016 17:19:09 +0900 Subject: [PATCH 08/29] Fix to do commit/rollback for 2 or more chunks at once * without this fix, some events will be written paritally if any chunks raise error in #append/#concat * it makes duplicated events Buffer#write will do: (method name was changed: #emit is method names for event routers) * this method receives pairs of metadata and data to be written at once * append/concat these data to chunks (not committed) * commit first chunk * if succeeded, then commit all (even if any following chunk raises error) * if failed, rollback all In memory/file buffer, #commit is very lightweight operation and will NOT fail in most cases. This change requires some additional internal APIs for buffers/chunks * chunk status in general: this is required to make #write thread safe * keyword argument of #write: bulk and enqueue * #write method becomes much more complex, so bulk operation should be merged into an implementation (we can't maintain two different/similar methods) * #write method enqueues chunks if needed, so :immediate mode should be implemented in same level * chunk_full_threshold configuration parameter to control "chunk_size_full?" * bulk and non-bulk #write were merged * In non-bulk mode, it's too rare that written chunk content bytesize is same with limitation --- lib/fluent/plugin/buffer/file_chunk.rb | 1 - 1 file changed, 1 deletion(-) diff --git a/lib/fluent/plugin/buffer/file_chunk.rb b/lib/fluent/plugin/buffer/file_chunk.rb index 2c8fc59257..5cc097a9fd 100644 --- a/lib/fluent/plugin/buffer/file_chunk.rb +++ b/lib/fluent/plugin/buffer/file_chunk.rb @@ -42,7 +42,6 @@ class FileChunk < Chunk def initialize(metadata, path, mode, perm: system_config.file_permission || FILE_PERMISSION) super(metadata) - @meta = nil @permission = perm @bytesize = @size = @adding_bytes = @adding_size = 0 From f2ec3b58719a0c59431fd7d0688a3a04b68e452c Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Thu, 28 Apr 2016 19:36:29 +0900 Subject: [PATCH 09/29] add hack to call super surely (even if v0.12 style plugin does not call it) --- lib/fluent/compat/call_super_mixin.rb | 67 +++++++++++ lib/fluent/compat/filter.rb | 24 ++++ lib/fluent/compat/input.rb | 24 ++++ lib/fluent/compat/output.rb | 116 +++++++++++++++++- test/compat/test_calls_super.rb | 164 ++++++++++++++++++++++++++ 5 files changed, 391 insertions(+), 4 deletions(-) create mode 100644 lib/fluent/compat/call_super_mixin.rb create mode 100644 test/compat/test_calls_super.rb diff --git a/lib/fluent/compat/call_super_mixin.rb b/lib/fluent/compat/call_super_mixin.rb new file mode 100644 index 0000000000..c8099a82d1 --- /dev/null +++ b/lib/fluent/compat/call_super_mixin.rb @@ -0,0 +1,67 @@ +# +# Fluentd +# +# 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. +# + +module Fluent + module Compat + module CallSuperMixin + # This mixin is to prepend to 3rd party plugins of v0.12 APIs. + # In past, there were not strong rule to call super in #start, #before_shutdown and #shutdown. + # But v0.14 API requires to call super in these methods to setup/teardown plugin helpers and others. + # This mixin prepends method calls to call super forcedly if checker returns false (it shows Fluent::Plugin::Base#methods wasn't called) + + def self.prepended(klass) + @@_super_start ||= {} + @@_super_before_shutdown ||= {} + @@_super_shutdown ||= {} + + # ancestors[0]: this module + # ancestors[1]: prepended class (plugin itself) + method_search = ->(ancestors, method){ + closest = ancestors[2, ancestors.size - 2].index{|m| m.method_defined?(method) } + ancestors[2 + closest].instance_method(method) + } + @@_super_start[klass] = method_search.call(klass.ancestors, :start) # this returns Fluent::Compat::*#start (or helpers on it) + @@_super_before_shutdown[klass] = method_search.call(klass.ancestors, :before_shutdown) + @@_super_shutdown[klass] = method_search.call(klass.ancestors, :shutdown) + end + + def start + super + unless self.started? + @@_super_start[self.class].bind(self).call + # #super will reset logdev (especially in test), so this warn should be after calling it + log.warn "super was not called in #start: called it forcedly", plugin: self.class + end + end + + def before_shutdown + super + unless self.before_shutdown? + log.warn "super was not called in #before_shutdown: calling it forcedly", plugin: self.class + @@_super_before_shutdown[self.class].bind(self).call + end + end + + def shutdown + super + unless self.shutdown? + log.warn "super was not called in #shutdown: calling it forcedly", plugin: self.class + @@_super_shutdown[self.class].bind(self).call + end + end + end + end +end diff --git a/lib/fluent/compat/filter.rb b/lib/fluent/compat/filter.rb index 352c830671..d55abf723c 100644 --- a/lib/fluent/compat/filter.rb +++ b/lib/fluent/compat/filter.rb @@ -16,11 +16,35 @@ require 'fluent/plugin' require 'fluent/plugin/filter' +require 'fluent/compat/call_super_mixin' module Fluent module Compat class Filter < Fluent::Plugin::Filter # TODO: warn when deprecated + + def initialize + super + unless self.class.ancestors.include?(Fluent::Compat::CallSuperMixin) + self.class.module_eval do + prepend Fluent::Compat::CallSuperMixin + end + end + end + + # These definitions are to get instance methods of superclass of 3rd party plugins + # to make it sure to call super + def start + super + end + + def before_shutdown + super + end + + def shutdown + super + end end end end diff --git a/lib/fluent/compat/input.rb b/lib/fluent/compat/input.rb index fe6ed576f0..19017d539b 100644 --- a/lib/fluent/compat/input.rb +++ b/lib/fluent/compat/input.rb @@ -16,11 +16,35 @@ require 'fluent/plugin' require 'fluent/plugin/input' +require 'fluent/compat/call_super_mixin' module Fluent module Compat class Input < Fluent::Plugin::Input # TODO: warn when deprecated + + def initialize + super + unless self.class.ancestors.include?(Fluent::Compat::CallSuperMixin) + self.class.module_eval do + prepend Fluent::Compat::CallSuperMixin + end + end + end + + # These definitions are to get instance methods of superclass of 3rd party plugins + # to make it sure to call super + def start + super + end + + def before_shutdown + super + end + + def shutdown + super + end end end end diff --git a/lib/fluent/compat/output.rb b/lib/fluent/compat/output.rb index 189b684859..8df50e6d5c 100644 --- a/lib/fluent/compat/output.rb +++ b/lib/fluent/compat/output.rb @@ -16,6 +16,7 @@ require 'fluent/plugin' require 'fluent/plugin/output' +require 'fluent/compat/call_super_mixin' require 'fluent/compat/output_chain' require 'fluent/timezone' @@ -140,16 +141,57 @@ def support_in_v12_style?(feature) end end - ## emit must be implemented in plugin - # def emit(tag, es, chain) - # end - def process(tag, es) emit(tag, es, NULL_OUTPUT_CHAIN) end + + def initialize + super + unless self.class.ancestors.include?(Fluent::Compat::CallSuperMixin) + self.class.module_eval do + prepend Fluent::Compat::CallSuperMixin + end + end + end + + # These definitions are to get instance methods of superclass of 3rd party plugins + # to make it sure to call super + def start + super + end + + def before_shutdown + super + end + + def shutdown + super + end end class MultiOutput < Output + def initialize + super + unless self.class.ancestors.include?(Fluent::Compat::CallSuperMixin) + self.class.module_eval do + prepend Fluent::Compat::CallSuperMixin + end + end + end + + # These definitions are to get instance methods of superclass of 3rd party plugins + # to make it sure to call super + def start + super + end + + def before_shutdown + super + end + + def shutdown + super + end end class BufferedOutput < Fluent::Plugin::Output @@ -270,6 +312,29 @@ def handle_stream_simple(tag, es) def extract_placeholders(str, metadata) raise "BUG: compat plugin does not support extract_placeholders: use newer plugin API" end + + def initialize + super + unless self.class.ancestors.include?(Fluent::Compat::CallSuperMixin) + self.class.module_eval do + prepend Fluent::Compat::CallSuperMixin + end + end + end + + # These definitions are to get instance methods of superclass of 3rd party plugins + # to make it sure to call super + def start + super + end + + def before_shutdown + super + end + + def shutdown + super + end end class ObjectBufferedOutput < Fluent::Plugin::Output @@ -380,6 +445,29 @@ def write(chunk) def extract_placeholders(str, metadata) raise "BUG: compat plugin does not support extract_placeholders: use newer plugin API" end + + def initialize + super + unless self.class.ancestors.include?(Fluent::Compat::CallSuperMixin) + self.class.module_eval do + prepend Fluent::Compat::CallSuperMixin + end + end + end + + # These definitions are to get instance methods of superclass of 3rd party plugins + # to make it sure to call super + def start + super + end + + def before_shutdown + super + end + + def shutdown + super + end end class TimeSlicedOutput < Fluent::Plugin::Output @@ -456,6 +544,12 @@ def support_in_v12_style?(feature) def initialize super @localtime = true + + unless self.class.ancestors.include?(Fluent::Compat::CallSuperMixin) + self.class.module_eval do + prepend Fluent::Compat::CallSuperMixin + end + end end def configure(conf) @@ -530,6 +624,20 @@ def configure(conf) def extract_placeholders(str, metadata) raise "BUG: compat plugin does not support extract_placeholders: use newer plugin API" end + + # These definitions are to get instance methods of superclass of 3rd party plugins + # to make it sure to call super + def start + super + end + + def before_shutdown + super + end + + def shutdown + super + end end end end diff --git a/test/compat/test_calls_super.rb b/test/compat/test_calls_super.rb new file mode 100644 index 0000000000..be4dea4625 --- /dev/null +++ b/test/compat/test_calls_super.rb @@ -0,0 +1,164 @@ +require_relative '../helper' + +# these are Fluent::Compat::* in fact +require 'fluent/input' +require 'fluent/output' +require 'fluent/filter' + +class CompatCallsSuperTest < Test::Unit::TestCase + class DummyGoodInput < Fluent::Input + def configure(conf); super; end + def start; super; end + def before_shutdown; super; end + def shutdown; super; end + end + class DummyBadInput < Fluent::Input + def configure(conf); super; end + def start; end + def before_shutdown; end + def shutdown; end + end + class DummyGoodOutput < Fluent::Output + def configure(conf); super; end + def start; super; end + def before_shutdown; super; end + def shutdown; super; end + end + class DummyBadOutput < Fluent::Output + def configure(conf); super; end + def start; end + def before_shutdown; end + def shutdown; end + end + class DummyGoodFilter < Fluent::Filter + def configure(conf); super; end + def start; super; end + def before_shutdown; super; end + def shutdown; super; end + end + class DummyBadFilter < Fluent::Filter + def configure(conf); super; end + def start; end + def before_shutdown; end + def shutdown; end + end + + setup do + Fluent::Test.setup + end + + sub_test_case 'old API plugin which calls super properly' do + test 'Input#start, #before_shutdown and #shutdown calls all superclass methods properly' do + i = DummyGoodInput.new + i.configure(config_element()) + assert i.configured? + + i.start + assert i.started? + + i.before_shutdown + assert i.before_shutdown? + + i.shutdown + assert i.shutdown? + + assert i.log.out.logs.empty? + end + + test 'Output#start, #before_shutdown and #shutdown calls all superclass methods properly' do + i = DummyGoodOutput.new + i.configure(config_element()) + assert i.configured? + + i.start + assert i.started? + + i.before_shutdown + assert i.before_shutdown? + + i.shutdown + assert i.shutdown? + + assert i.log.out.logs.empty? + end + + test 'Filter#start, #before_shutdown and #shutdown calls all superclass methods properly' do + i = DummyGoodFilter.new + i.configure(config_element()) + assert i.configured? + + i.start + assert i.started? + + i.before_shutdown + assert i.before_shutdown? + + i.shutdown + assert i.shutdown? + + assert i.log.out.logs.empty? + end + end + + sub_test_case 'old API plugin which does not call super' do + test 'Input#start, #before_shutdown and #shutdown calls superclass methods forcedly with logs' do + i = DummyBadInput.new + i.configure(config_element()) + assert i.configured? + + i.start + assert i.started? + + i.before_shutdown + assert i.before_shutdown? + + i.shutdown + assert i.shutdown? + + logs = i.log.out.logs + assert{ logs.any?{|l| l.include?("[warn]: super was not called in #start: called it forcedly plugin=CompatCallsSuperTest::DummyBadInput") } } + assert{ logs.any?{|l| l.include?("[warn]: super was not called in #before_shutdown: calling it forcedly plugin=CompatCallsSuperTest::DummyBadInput") } } + assert{ logs.any?{|l| l.include?("[warn]: super was not called in #shutdown: calling it forcedly plugin=CompatCallsSuperTest::DummyBadInput") } } + end + + test 'Output#start, #before_shutdown and #shutdown calls superclass methods forcedly with logs' do + i = DummyBadOutput.new + i.configure(config_element()) + assert i.configured? + + i.start + assert i.started? + + i.before_shutdown + assert i.before_shutdown? + + i.shutdown + assert i.shutdown? + + logs = i.log.out.logs + assert{ logs.any?{|l| l.include?("[warn]: super was not called in #start: called it forcedly plugin=CompatCallsSuperTest::DummyBadOutput") } } + assert{ logs.any?{|l| l.include?("[warn]: super was not called in #before_shutdown: calling it forcedly plugin=CompatCallsSuperTest::DummyBadOutput") } } + assert{ logs.any?{|l| l.include?("[warn]: super was not called in #shutdown: calling it forcedly plugin=CompatCallsSuperTest::DummyBadOutput") } } + end + + test 'Filter#start, #before_shutdown and #shutdown calls superclass methods forcedly with logs' do + i = DummyBadFilter.new + i.configure(config_element()) + assert i.configured? + + i.start + assert i.started? + + i.before_shutdown + assert i.before_shutdown? + + i.shutdown + assert i.shutdown? + + logs = i.log.out.logs + assert{ logs.any?{|l| l.include?("[warn]: super was not called in #start: called it forcedly plugin=CompatCallsSuperTest::DummyBadFilter") } } + assert{ logs.any?{|l| l.include?("[warn]: super was not called in #before_shutdown: calling it forcedly plugin=CompatCallsSuperTest::DummyBadFilter") } } + assert{ logs.any?{|l| l.include?("[warn]: super was not called in #shutdown: calling it forcedly plugin=CompatCallsSuperTest::DummyBadFilter") } } + end + end +end From 783a393b8150c7695a73468734734051bdb59130 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Fri, 13 May 2016 13:14:05 +0900 Subject: [PATCH 10/29] new plugin uses #emit_events instead of #emit --- lib/fluent/plugin/output.rb | 4 ++-- test/test_output.rb | 2 ++ 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/lib/fluent/plugin/output.rb b/lib/fluent/plugin/output.rb index e3926e0cb0..c69218354c 100644 --- a/lib/fluent/plugin/output.rb +++ b/lib/fluent/plugin/output.rb @@ -301,7 +301,7 @@ def start if @buffering m = method(:emit_buffered) (class << self; self; end).module_eval do - define_method(:emit, m) + define_method(:emit_events, m) end @custom_format = implement?(:custom_format) @@ -314,7 +314,7 @@ def start else # !@buffering m = method(:emit_sync) (class << self; self; end).module_eval do - define_method(:emit, m) + define_method(:emit_events, m) end end diff --git a/test/test_output.rb b/test/test_output.rb index 4718553944..e40bc2e39e 100644 --- a/test/test_output.rb +++ b/test/test_output.rb @@ -300,6 +300,8 @@ def create_driver(conf=CONFIG) end test "emit with valid event" do + omit "there's no #emit method anymore in output plugins" + d = create_driver d.instance.start if d.instance.method(:emit).arity == 3 From b27cdd1c76488c8bae9acb3dff497443283896e6 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Fri, 13 May 2016 13:15:07 +0900 Subject: [PATCH 11/29] remove useless method definitions (CallSuperMixin now searches class with methods) --- lib/fluent/compat/output.rb | 78 ------------------------------------- 1 file changed, 78 deletions(-) diff --git a/lib/fluent/compat/output.rb b/lib/fluent/compat/output.rb index 8df50e6d5c..b4a2aaa646 100644 --- a/lib/fluent/compat/output.rb +++ b/lib/fluent/compat/output.rb @@ -153,45 +153,9 @@ def initialize end end end - - # These definitions are to get instance methods of superclass of 3rd party plugins - # to make it sure to call super - def start - super - end - - def before_shutdown - super - end - - def shutdown - super - end end class MultiOutput < Output - def initialize - super - unless self.class.ancestors.include?(Fluent::Compat::CallSuperMixin) - self.class.module_eval do - prepend Fluent::Compat::CallSuperMixin - end - end - end - - # These definitions are to get instance methods of superclass of 3rd party plugins - # to make it sure to call super - def start - super - end - - def before_shutdown - super - end - - def shutdown - super - end end class BufferedOutput < Fluent::Plugin::Output @@ -321,20 +285,6 @@ def initialize end end end - - # These definitions are to get instance methods of superclass of 3rd party plugins - # to make it sure to call super - def start - super - end - - def before_shutdown - super - end - - def shutdown - super - end end class ObjectBufferedOutput < Fluent::Plugin::Output @@ -454,20 +404,6 @@ def initialize end end end - - # These definitions are to get instance methods of superclass of 3rd party plugins - # to make it sure to call super - def start - super - end - - def before_shutdown - super - end - - def shutdown - super - end end class TimeSlicedOutput < Fluent::Plugin::Output @@ -624,20 +560,6 @@ def configure(conf) def extract_placeholders(str, metadata) raise "BUG: compat plugin does not support extract_placeholders: use newer plugin API" end - - # These definitions are to get instance methods of superclass of 3rd party plugins - # to make it sure to call super - def start - super - end - - def before_shutdown - super - end - - def shutdown - super - end end end end From 3aa3dbe0dc721d0381f0b88b85adb1f2fb72df79 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Fri, 13 May 2016 18:58:23 +0900 Subject: [PATCH 12/29] fix to have corresponding config element on section objects --- lib/fluent/config/section.rb | 9 +++++++-- test/config/test_section.rb | 14 ++++++++++++++ 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/lib/fluent/config/section.rb b/lib/fluent/config/section.rb index 89b1274fbe..94d4ac43da 100644 --- a/lib/fluent/config/section.rb +++ b/lib/fluent/config/section.rb @@ -26,13 +26,18 @@ def self.name 'Fluent::Config::Section' end - def initialize(params = {}) + def initialize(params = {}, config_element = nil) @klass = 'Fluent::Config::Section' @params = params + @corresponding_config_element = config_element end alias :object_id :__id__ + def corresponding_config_element + @corresponding_config_element + end + def to_s inspect end @@ -165,7 +170,7 @@ def self.generate(proxy, conf, logger, plugin_class, stack = []) end end - Section.new(section_params) + Section.new(section_params, conf) end def self.check_unused_section(proxy, conf, plugin_class) diff --git a/test/config/test_section.rb b/test/config/test_section.rb index 6cd86431b3..e8a9927a93 100644 --- a/test/config/test_section.rb +++ b/test/config/test_section.rb @@ -59,6 +59,20 @@ class TestSection < ::Test::Unit::TestCase assert_equal('myid', s1['@id'.to_sym]) assert_equal('myid', s1.__send__('@id'.to_sym)) end + + test 'creates object and config element which corresponds to section object itself' do + hash = { + name: 'tagomoris', + age: 34, + send: 'email', + class: 'normal', + keys: 5, + } + hash['@id'.to_sym] = 'myid' + conf = config_element('section', '', {'name' => 'tagomoris', 'age' => 34, 'send' => 'email', 'class' => 'normal', 'keys' => 5}) + s2 = Fluent::Config::Section.new(hash, conf) + assert s2.corresponding_config_element.is_a?(Fluent::Config::Element) + end end sub_test_case '#object_id' do From 397facf491ee7df775e290a8c121de264deeee3e Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Fri, 13 May 2016 18:58:51 +0900 Subject: [PATCH 13/29] add MultiOutput as a kind of Output plugin --- lib/fluent/compat/output.rb | 15 ++++- lib/fluent/plugin/multi_output.rb | 95 +++++++++++++++++++++++++++++++ 2 files changed, 107 insertions(+), 3 deletions(-) create mode 100644 lib/fluent/plugin/multi_output.rb diff --git a/lib/fluent/compat/output.rb b/lib/fluent/compat/output.rb index b4a2aaa646..ff8fba9e7d 100644 --- a/lib/fluent/compat/output.rb +++ b/lib/fluent/compat/output.rb @@ -16,6 +16,7 @@ require 'fluent/plugin' require 'fluent/plugin/output' +require 'fluent/plugin/multi_output' require 'fluent/compat/call_super_mixin' require 'fluent/compat/output_chain' require 'fluent/timezone' @@ -28,11 +29,11 @@ module Compat module CompatOutputUtils def self.buffer_section(conf) - conf.elements.select{|e| e.name == 'buffer'}.first + conf.elements(name: 'buffer').first end def self.secondary_section(conf) - conf.elements.select{|e| e.name == 'secondary'}.first + conf.elements(name: 'secondary').first end def self.inject_type_from_obsoleted_name(secconf, log) @@ -155,7 +156,15 @@ def initialize end end - class MultiOutput < Output + class MultiOutput < Fluent::Plugin::MultiOutput + def initialize + super + @compat = true + end + + def process(tag, es) + emit(tag, es, NULL_OUTPUT_CHAIN) + end end class BufferedOutput < Fluent::Plugin::Output diff --git a/lib/fluent/plugin/multi_output.rb b/lib/fluent/plugin/multi_output.rb new file mode 100644 index 0000000000..b0ee342fd3 --- /dev/null +++ b/lib/fluent/plugin/multi_output.rb @@ -0,0 +1,95 @@ +# +# Fluentd +# +# 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. +# + +require 'fluent/plugin/base' +require 'fluent/log' +require 'fluent/plugin_id' +require 'fluent/plugin_helper' + +module Fluent + module Plugin + class MultiOutput < Base + include PluginId + include PluginLoggerMixin + include PluginHelper::Mixin # for event_emitter + + helpers :event_emitter # to get router from agent, which will be supplied to child plugins + + config_section :store, param_name: :stores, multi: true, required: true do + config_param :@type, :string, default: nil + end + + attr_reader :outputs + + def initialize + super + @outputs = [] + + @compat = false + + @counters_monitor = Monitor.new + # TODO: well organized counters + @num_errors = 0 + @emit_count = 0 + @emit_records = 0 + @write_count = 0 + @rollback_count = 0 + end + + def configure(conf) + super + + # v0.12 MultiOutput does nothing about initializing stores, and plugin implementation did it. + return if @compat + + @stores.each do |store| + store_conf = store.corresponding_config_element + type = store[:@type] + if !type && store_conf['type'] + type = store_conf['type'] + log.warn "'type' is deprecated, and will be ignored in v1: use '@type' instead." + end + unless type + raise Fluent::ConfigError, "Missing '@type' parameter in section" + end + + log.debug "adding store", type: type + + output = Fluent::Plugin.new_output(type) + if output.has_router? + output.router = router + end + output.configure(store_conf) + @outputs << output + end + end + + def emit_sync(tag, es) + @counters_monitor.synchronize{ @emit_count += 1 } + begin + process(tag, es) + @counters_monitor.synchronize{ @emit_records += es.size } + rescue + @counters_monitor.synchronize{ @num_errors += 1 } + raise + end + end + alias :emit_events :emit_sync + + # def process(tag, es) + end + end +end From 6c8a5afd18cecf6dfcd3a44d6dfb8ba78154471f Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Fri, 13 May 2016 19:25:53 +0900 Subject: [PATCH 14/29] switching plugin lifecycle --- lib/fluent/agent.rb | 102 ++++++++++++++++----------------- lib/fluent/plugin/output.rb | 6 +- lib/fluent/root_agent.rb | 109 +++++++++++++++++++++++++++++------- test/test_root_agent.rb | 6 -- 4 files changed, 144 insertions(+), 79 deletions(-) diff --git a/lib/fluent/agent.rb b/lib/fluent/agent.rb index d99381ee26..e38101b791 100644 --- a/lib/fluent/agent.rb +++ b/lib/fluent/agent.rb @@ -34,8 +34,15 @@ def initialize(log:) @context = nil @outputs = [] @filters = [] - @started_outputs = [] - @started_filters = [] + + @lifecycle_control_list = nil + # lifecycle_control_list is the list of plugins in this agent, and ordered + # from plugins which DOES emit, then DOESN'T emit + # (input -> output w/ router -> filter -> output w/o router) + # for start: use this order DESC + # (because plugins which appears later in configurations will receive events from plugins which appears ealier) + # for stop/before_shutdown/shutdown/after_shutdown/close/terminate: use this order ASC + @lifecycle_cache = nil @log = log @event_router = EventRouter.new(NoMatchMatch.new(log), self) @@ -64,63 +71,58 @@ def configure(conf) } end - def start - @outputs.each { |o| - o.start - @started_outputs << o - } + def lifecycle_control_list + return @lifecycle_control_list if @lifecycle_control_list - @filters.each { |f| - f.start - @started_filters << f + lifecycle_control_list = { + input: [], + output_with_router: [], + filter: [], + output: [], } - end - - def shutdown - @started_filters.map { |f| - Thread.new do - begin - log.info "shutting down filter#{@context.nil? ? '' : " in #{@context}"}", type: Plugin.lookup_type_from_class(f.class), plugin_id: f.plugin_id - f.shutdown - rescue => e - log.warn "unexpected error while shutting down filter plugins", plugin: f.class, plugin_id: f.plugin_id, error: e - log.warn_backtrace - end + if self.respond_to?(:inputs) + inputs.each do |i| + lifecycle_control_list[:input] << i end - }.each { |t| t.join } - - # Output plugin as filter emits records at shutdown so emit problem still exist. - # This problem will be resolved after actual filter mechanizm. - @started_outputs.map { |o| - Thread.new do - begin - log.info "shutting down output#{@context.nil? ? '' : " in #{@context}"}", type: Plugin.lookup_type_from_class(o.class), plugin_id: o.plugin_id - o.shutdown - rescue => e - log.warn "unexpected error while shutting down output plugins", plugin: o.class, plugin_id: o.plugin_id, error: e - log.warn_backtrace + end + recursive_output_traverse = ->(o) { + if o.respond_to?(:outputs) + o.outputs.each do |store| + recursive_output_traverse.call(store) end + elsif o.has_router? + lifecycle_control_list[:output_with_router] << o + else + lifecycle_control_list[:output] << o end - }.each { |t| t.join } - end + } + outputs.each do |o| + recursive_output_traverse.call(o) + end + filters.each do |f| + lifecycle_control_list[:filter] << f + end - def flush! - flush_recursive(@outputs) + @lifecycle_control_list = lifecycle_control_list end - def flush_recursive(array) - array.each { |o| - begin - if o.is_a?(BufferedOutput) - o.force_flush - elsif o.is_a?(MultiOutput) - flush_recursive(o.outputs) - end - rescue => e - log.debug "error while force flushing", error: e - log.debug_backtrace + def lifecycle(desc: false) + kind_list = if desc + [:output, :filter, :output_with_router] + else + [:output_with_router, :filter, :output] + end + kind_list.each do |kind| + list = if desc + lifecycle_control_list[kind].reverse + else + lifecycle_control_list[kind] + end + display_kind = (kind == :output_with_router ? :output : kind) + list.each do |instance| + yield instance, display_kind end - } + end end def add_match(type, pattern, conf) diff --git a/lib/fluent/plugin/output.rb b/lib/fluent/plugin/output.rb index c69218354c..687396f449 100644 --- a/lib/fluent/plugin/output.rb +++ b/lib/fluent/plugin/output.rb @@ -820,8 +820,10 @@ def submit_flush_once end def force_flush - @buffer.enqueue_all - submit_flush_all + if @buffering + @buffer.enqueue_all + submit_flush_all + end end def submit_flush_all diff --git a/lib/fluent/root_agent.rb b/lib/fluent/root_agent.rb index fcdf160810..00d3a85b7f 100644 --- a/lib/fluent/root_agent.rb +++ b/lib/fluent/root_agent.rb @@ -52,7 +52,6 @@ def initialize(log:, system_config: SystemConfig.new) @labels = {} @inputs = [] - @started_inputs = [] @suppress_emit_error_log_interval = 0 @next_emit_error_log_time = nil @without_source = false @@ -105,38 +104,106 @@ def setup_error_label(e) @error_collector = error_label.event_router end - def start - super - - @labels.each { |n, l| - l.start - } + def lifecycle(desc: false) + kind_or_label_list = if desc + [:output, :filter, @labels.values.reverse, :output_with_router, :input].flatten + else + [:input, :output_with_router, @labels.values, :filter, :output].flatten + end + kind_or_label_list.each do |kind| + if kind.respond_to?(:lifecycle) + label = kind + label.lifecycle(desc: desc) do |plugin, display_kind| + yield plugin, display_kind + end + else + list = if desc + lifecycle_control_list[kind].reverse + else + lifecycle_control_list[kind] + end + display_kind = (kind == :output_with_router ? :output : kind) + list.each do |instance| + yield instance, display_kind + end + end + end + end - @inputs.each { |i| + def start + lifecycle(desc: true) do |i| # instance i.start - @started_inputs << i - } + end end - def shutdown - # Shutdown Input plugin first to prevent emitting to terminated Output plugin - @started_inputs.map { |i| - Thread.new do + def flush! + log.info "flushing all buffer forcedly" + flushing_threads = [] + lifecycle(desc: true) do |instance| + if instance.respond_to?(:force_flush) + t = Thread.new do + Thread.current.abort_on_exception = true + begin + instance.force_flush + rescue => e + log.warn "unexpected error while flushing buffer", plugin: instance.class, plugin_id: instance.plugin_id, error: e + log.warn_backtrace + end + end + flushing_threads = [] + end + end + flushing_threads.each{|t| t.join } + end + + def shutdown # Fluentd's shutdown sequence is stop, before_shutdown, shutdown, after_shutdown, close, terminate for plugins + lifecycle_safe_sequence = ->(method) { + lifecycle do |instance, kind| begin - log.info "shutting down input", type: Plugin.lookup_type_from_class(i.class), plugin_id: i.plugin_id - i.shutdown + log.debug "calling #{method} on #{kind} plugin", type: Plugin.lookup_type_from_class(instance.class), plugin_id: instance.plugin_id + instance.send(method) rescue => e - log.warn "unexpected error while shutting down input plugin", plugin: i.class, plugin_id: i.plugin_id, error: e + log.warn "unexpected error while calling #{method} on #{kind} plugin", pluguin: instance.class, plugin_id: instance.plugin_id, error: e log.warn_backtrace end end - }.each { |t| t.join } + } - @labels.each { |n, l| - l.shutdown + lifecycle_unsafe_sequence = ->(method) { + operation = case method + when :shutdown then "shutting down" + when :close then "closing" + else + raise "BUG: unknown method name '#{method}'" + end + operation_threads = [] + lifecycle do |instance, kind| + t = Thread.new do + Thread.current.abort_on_exception = true + begin + log.info "#{operation} #{kind} plugin", type: Plugin.lookup_type_from_class(instance.class), plugin_id: instance.plugin_id + instance.send(method) + rescue => e + log.warn "unexpected error while #{operation} #{kind} plugin", plugin: instance.class, plugin_id: instance.plugin_id, error: e + log.warn_backtrace + end + end + operation_threads << t + end + operation_threads.each{|t| t.join } } - super + lifecycle_safe_sequence.call(:stop) + + lifecycle_safe_sequence.call(:before_shutdown) + + lifecycle_unsafe_sequence.call(:shutdown) + + lifecycle_safe_sequence.call(:after_shutdown) + + lifecycle_unsafe_sequence.call(:close) + + lifecycle_safe_sequence.call(:terminate) end def suppress_interval(interval_time) diff --git a/test/test_root_agent.rb b/test/test_root_agent.rb index dfc0cdfad3..fcd1845890 100644 --- a/test/test_root_agent.rb +++ b/test/test_root_agent.rb @@ -43,9 +43,6 @@ def configure_ra(conf_str) assert_empty ra.labels assert_empty ra.outputs assert_empty ra.filters - [:@started_inputs, :@started_outputs, :@started_filters].each { |k| - assert_empty ra.instance_variable_get(k) - } assert_nil ra.context assert_nil ra.error_collector end @@ -82,9 +79,6 @@ def configure_ra(conf_str) assert_kind_of FluentTestInput, ra.inputs.first assert_kind_of RelabelOutput, ra.outputs.first assert_kind_of FluentTestFilter, ra.filters.first - [:@started_inputs, :@started_outputs, :@started_filters].each { |k| - assert_empty ra.instance_variable_get(k) - } assert ra.error_collector %W(@test @ERROR).each { |label_symbol| From 8a1223ec277a23a6324c243803592c20506dcef8 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Fri, 13 May 2016 20:05:32 +0900 Subject: [PATCH 15/29] fix to load OutputChain always --- lib/fluent/output.rb | 1 + 1 file changed, 1 insertion(+) diff --git a/lib/fluent/output.rb b/lib/fluent/output.rb index 9e3949cc82..c043771751 100644 --- a/lib/fluent/output.rb +++ b/lib/fluent/output.rb @@ -15,6 +15,7 @@ # require 'fluent/compat/output' +require 'fluent/output_chain' module Fluent Output = Fluent::Compat::Output From 9650d10f6258da7860f5f231c64656d8924bb004 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Fri, 13 May 2016 20:06:06 +0900 Subject: [PATCH 16/29] fix test not to raise ConfigError for missing store sections --- test/plugin/test_out_copy.rb | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/test/plugin/test_out_copy.rb b/test/plugin/test_out_copy.rb index 52c947d4b0..2241ba07b0 100644 --- a/test/plugin/test_out_copy.rb +++ b/test/plugin/test_out_copy.rb @@ -108,7 +108,19 @@ def test_msgpack_es_emit_bug def create_event_test_driver(is_deep_copy = false) deep_copy_config = %[ -deep_copy true +deep_copy #{is_deep_copy} + + type test + name c0 + + + type test + name c1 + + + type test + name c2 + ] output1 = Fluent::Plugin.new_output('test') @@ -131,7 +143,7 @@ def create_event_test_driver(is_deep_copy = false) outputs = [output1, output2] d = Fluent::Test::OutputTestDriver.new(Fluent::CopyOutput) - d = d.configure(deep_copy_config) if is_deep_copy + d = d.configure(deep_copy_config) d.instance.instance_eval { @outputs = outputs } d end From ce28b962f071ea07bd5c0991b6069b27cf1cb787 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Mon, 16 May 2016 13:50:58 +0900 Subject: [PATCH 17/29] fix agent to call lifecycle methods for MultiOutput plugins --- lib/fluent/agent.rb | 10 ++++++---- lib/fluent/plugin/multi_output.rb | 9 +++++++-- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/lib/fluent/agent.rb b/lib/fluent/agent.rb index e38101b791..497049acfc 100644 --- a/lib/fluent/agent.rb +++ b/lib/fluent/agent.rb @@ -86,14 +86,16 @@ def lifecycle_control_list end end recursive_output_traverse = ->(o) { + if o.has_router? + lifecycle_control_list[:output_with_router] << o + else + lifecycle_control_list[:output] << o + end + if o.respond_to?(:outputs) o.outputs.each do |store| recursive_output_traverse.call(store) end - elsif o.has_router? - lifecycle_control_list[:output_with_router] << o - else - lifecycle_control_list[:output] << o end } outputs.each do |o| diff --git a/lib/fluent/plugin/multi_output.rb b/lib/fluent/plugin/multi_output.rb index b0ee342fd3..4c8d8f4556 100644 --- a/lib/fluent/plugin/multi_output.rb +++ b/lib/fluent/plugin/multi_output.rb @@ -45,8 +45,8 @@ def initialize @num_errors = 0 @emit_count = 0 @emit_records = 0 - @write_count = 0 - @rollback_count = 0 + # @write_count = 0 + # @rollback_count = 0 end def configure(conf) @@ -77,6 +77,11 @@ def configure(conf) end end + # Child plugin's lifecycles are controlled by agent automatically. + # It calls `outputs` to traverse plugins, and invoke start/stop/*shutdown/close/terminate on these directly. + # * `start` of this plugin will be called after child plugins + # * `stop`, `*shutdown`, `close` and `terminate` of this plugin will be called before child plugins + def emit_sync(tag, es) @counters_monitor.synchronize{ @emit_count += 1 } begin From 5236e16bb5824b104e0d07e0c5de4d2dcfac05e0 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Mon, 16 May 2016 13:51:40 +0900 Subject: [PATCH 18/29] add tests for Fluent::Plugin::MultiOutput --- test/plugin/test_multi_output.rb | 214 +++++++++++++++++++++++++++++++ 1 file changed, 214 insertions(+) create mode 100644 test/plugin/test_multi_output.rb diff --git a/test/plugin/test_multi_output.rb b/test/plugin/test_multi_output.rb new file mode 100644 index 0000000000..30c1b114e2 --- /dev/null +++ b/test/plugin/test_multi_output.rb @@ -0,0 +1,214 @@ +require_relative '../helper' +require 'fluent/plugin/output' +require 'fluent/event' + +require 'json' +require 'time' +require 'timeout' + +module FluentPluginMultiOutputTest + class DummyMultiOutput < Fluent::Plugin::MultiOutput + attr_reader :events + def initialize + super + @events = [] + end + def configure(conf) + super + end + def process(tag, es) + es.each do |time, record| + @events << [tag, time, record] + end + end + end + class DummyCompatMultiOutput < Fluent::Plugin::MultiOutput + def initialize + super + @compat = true + end + def configure(conf) + super + end + def process(tag, es) + # ... + end + end + + class Dummy1Output < Fluent::Plugin::Output + Fluent::Plugin.register_output('dummy_test_multi_output_1', self) + attr_reader :configured + def configure(conf) + super + @configured = true + end + def process(tag, es) + end + end + class Dummy2Output < Fluent::Plugin::Output + Fluent::Plugin.register_output('dummy_test_multi_output_2', self) + attr_reader :configured + def configure(conf) + super + @configured = true + end + def process(tag, es) + end + end + class Dummy3Output < Fluent::Plugin::Output + Fluent::Plugin.register_output('dummy_test_multi_output_3', self) + attr_reader :configured + def configure(conf) + super + @configured = true + end + def process(tag, es) + end + end + class Dummy4Output < Fluent::Plugin::Output + Fluent::Plugin.register_output('dummy_test_multi_output_4', self) + attr_reader :configured + def configure(conf) + super + @configured = true + end + def process(tag, es) + end + end +end + +class MultiOutputTest < Test::Unit::TestCase + def create_output(type=:multi) + case type + when :compat_multi + FluentPluginMultiOutputTest::DummyCompatMultiOutput.new + else + FluentPluginMultiOutputTest::DummyMultiOutput.new + end + end + + sub_test_case 'basic multi output plugin' do + setup do + Fluent::Test.setup + @i = create_output() + end + + teardown do + @i.log.out.reset + end + + test '#configure raises error if sections are missing' do + conf = config_element('ROOT', '', { '@type' => 'dummy_test_multi_output' }, []) + assert_raise Fluent::ConfigError do + @i.configure(conf) + end + end + + test '#configure initialize child plugins and call these #configure' do + assert_equal [], @i.outputs + + conf = config_element('ROOT', '', { '@type' => 'dummy_test_multi_output' }, + [ + config_element('store', '', { '@type' => 'dummy_test_multi_output_1' }), + config_element('store', '', { '@type' => 'dummy_test_multi_output_2' }), + config_element('store', '', { '@type' => 'dummy_test_multi_output_3' }), + config_element('store', '', { '@type' => 'dummy_test_multi_output_4' }), + ] + ) + @i.configure(conf) + + assert_equal 4, @i.outputs.size + + assert @i.outputs[0].is_a? FluentPluginMultiOutputTest::Dummy1Output + assert @i.outputs[0].configured + + assert @i.outputs[1].is_a? FluentPluginMultiOutputTest::Dummy2Output + assert @i.outputs[1].configured + + assert @i.outputs[2].is_a? FluentPluginMultiOutputTest::Dummy3Output + assert @i.outputs[2].configured + + assert @i.outputs[3].is_a? FluentPluginMultiOutputTest::Dummy4Output + assert @i.outputs[3].configured + end + + test '#configure warns if "type" is used in sections instead of "@type"' do + assert_equal [], @i.log.out.logs + + conf = config_element('ROOT', '', { '@type' => 'dummy_test_multi_output' }, + [ + config_element('store', '', { 'type' => 'dummy_test_multi_output_1' }), + config_element('store', '', { 'type' => 'dummy_test_multi_output_2' }), + config_element('store', '', { 'type' => 'dummy_test_multi_output_3' }), + config_element('store', '', { 'type' => 'dummy_test_multi_output_4' }), + ] + ) + @i.configure(conf) + assert_equal 4, @i.outputs.size + + logs = @i.log.out.logs + assert{ logs.select{|log| log.include?('[warn]') && log.include?("'type' is deprecated, and will be ignored in v1: use '@type' instead.") }.size == 4 } + end + + test '#emit_events calls #process always' do + conf = config_element('ROOT', '', { '@type' => 'dummy_test_multi_output' }, + [ + config_element('store', '', { '@type' => 'dummy_test_multi_output_1' }), + config_element('store', '', { '@type' => 'dummy_test_multi_output_2' }), + config_element('store', '', { '@type' => 'dummy_test_multi_output_3' }), + config_element('store', '', { '@type' => 'dummy_test_multi_output_4' }), + ] + ) + @i.configure(conf) + @i.start + + assert @i.events.empty? + + @i.emit_events( + 'test.tag', + Fluent::ArrayEventStream.new( + [ + [event_time(), {"message" => "multi test 1"}], + [event_time(), {"message" => "multi test 1"}], + ] + ) + ) + + assert_equal 2, @i.events.size + end + end + + sub_test_case 'compat multi output plugin' do + setup do + Fluent::Test.setup + @i = create_output(:compat_multi) + end + + teardown do + @i.log.out.reset + end + + test '#configure raises error if sections are missing' do + conf = config_element('ROOT', '', { '@type' => 'dummy_test_multi_output' }, []) + assert_raise Fluent::ConfigError do + @i.configure(conf) + end + end + + test '#configure does NOT initialize child plugins' do + assert_equal [], @i.outputs + + conf = config_element('ROOT', '', { '@type' => 'dummy_test_multi_output' }, + [ + config_element('store', '', { '@type' => 'dummy_test_multi_output_1' }), + config_element('store', '', { '@type' => 'dummy_test_multi_output_2' }), + config_element('store', '', { '@type' => 'dummy_test_multi_output_3' }), + config_element('store', '', { '@type' => 'dummy_test_multi_output_4' }), + ] + ) + @i.configure(conf) + + assert_equal [], @i.outputs + end + end +end From 3047691f5a497c12f325d15ab30518227a3e83d6 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Mon, 16 May 2016 14:17:48 +0900 Subject: [PATCH 19/29] remove buffer plugins with old API --- lib/fluent/buffer.rb | 363 ----------------- lib/fluent/plugin/buf_file.rb | 257 ------------ lib/fluent/plugin/buf_memory.rb | 117 ------ test/plugin/test_buf_file.rb | 690 -------------------------------- test/plugin/test_buf_memory.rb | 204 ---------- test/test_buffer.rb | 606 ---------------------------- 6 files changed, 2237 deletions(-) delete mode 100644 lib/fluent/buffer.rb delete mode 100644 lib/fluent/plugin/buf_file.rb delete mode 100644 lib/fluent/plugin/buf_memory.rb delete mode 100644 test/plugin/test_buf_file.rb delete mode 100644 test/plugin/test_buf_memory.rb delete mode 100644 test/test_buffer.rb diff --git a/lib/fluent/buffer.rb b/lib/fluent/buffer.rb deleted file mode 100644 index d8901f9d36..0000000000 --- a/lib/fluent/buffer.rb +++ /dev/null @@ -1,363 +0,0 @@ -# -# Fluentd -# -# 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. -# - -require 'monitor' -require 'fileutils' - -require 'fluent/configurable' -require 'fluent/plugin' # to register itself to registry - -module Fluent - class BufferError < StandardError - end - - class BufferChunkLimitError < BufferError - end - - class BufferQueueLimitError < BufferError - end - - - class Buffer - include Configurable - - def initialize - super - end - - def configure(conf) - super - end - - def start - end - - def shutdown - end - - def before_shutdown(out) - end - - def emit(key, data, chain) - raise NotImplementedError, "Implement this method in child class" - end - - def keys - raise NotImplementedError, "Implement this method in child class" - end - - def push(key) - raise NotImplementedError, "Implement this method in child class" - end - - def pop(out) - raise NotImplementedError, "Implement this method in child class" - end - - def clear! - raise NotImplementedError, "Implement this method in child class" - end - end - - - class BufferChunk - include MonitorMixin - - def initialize(key) - super() - @key = key - end - - attr_reader :key - - def <<(data) - raise NotImplementedError, "Implement this method in child class" - end - - def size - raise NotImplementedError, "Implement this method in child class" - end - - def empty? - size == 0 - end - - def close - raise NotImplementedError, "Implement this method in child class" - end - - def purge - raise NotImplementedError, "Implement this method in child class" - end - - def read - raise NotImplementedError, "Implement this method in child class" - end - - def open - raise NotImplementedError, "Implement this method in child class" - end - - def write_to(io) - open {|i| - FileUtils.copy_stream(i, io) - } - end - - def msgpack_each(&block) - open {|io| - u = Fluent::Engine.msgpack_factory.unpacker(io) - begin - u.each(&block) - rescue EOFError - end - } - end - end - - - class BasicBuffer < Buffer - include MonitorMixin - - def initialize - super - @map = nil # chunks to store data - @queue = nil # chunks to be flushed - @parallel_pop = true - end - - def enable_parallel(b=true) - @parallel_pop = b - end - - # This configuration assumes plugins to send records to a remote server. - # Local file based plugins which should provide more reliability and efficiency - # should override buffer_chunk_limit with a larger size. - desc 'The size of each buffer chunk.' - config_param :buffer_chunk_limit, :size, default: 8*1024*1024 - desc 'The length limit of the chunk queue.' - config_param :buffer_queue_limit, :integer, default: 256 - desc 'The action when the size of buffer queue exceeds the buffer_queue_limit.' - config_param :buffer_queue_full_action, :enum, list: [:exception, :block], default: :exception - - alias chunk_limit buffer_chunk_limit - alias chunk_limit= buffer_chunk_limit= - alias queue_limit buffer_queue_limit - alias queue_limit= buffer_queue_limit= - - def configure(conf) - super - - if @buffer_queue_full_action == :block - $log.warn "'block' action stops input process until the buffer full is resolved. Check your pipeline this action is fit or not" - end - end - - def start - @queue, @map = resume - @queue.extend(MonitorMixin) - end - - def shutdown - synchronize do - @queue.synchronize do - until @queue.empty? - @queue.shift.close - end - end - @map.each_pair {|key,chunk| - chunk.close - } - end - end - - def storable?(chunk, data) - chunk.size + data.bytesize <= @buffer_chunk_limit - end - - def emit(key, data, chain) - key = key.to_s - - synchronize do - begin - # chunk unique id is generated in #new_chunk - chunk = (@map[key] ||= new_chunk(key)) - - if storable?(chunk, data) - chain.next - chunk << data - return false - - elsif @queue.size >= @buffer_queue_limit - raise BufferQueueLimitError, "queue size exceeds limit" - end - rescue BufferQueueLimitError => e - case @buffer_queue_full_action - when :exception - raise e - when :block - # This is rough implementation. New Buffer API should improve this routine by using wait/signal - $log.debug "buffer queue is full. Wait 1 second to re-emit events" - sleep 1 - retry - end - end - - if data.bytesize > @buffer_chunk_limit - $log.warn "Size of the emitted data exceeds buffer_chunk_limit." - $log.warn "This may occur problems in the output plugins ``at this server.``" - $log.warn "To avoid problems, set a smaller number to the buffer_chunk_limit" - $log.warn "in the forward output ``at the log forwarding server.``" - ### TODO - # raise BufferChunkLimitError, "received data too large" - end - - # chunk unique id is generated in #new_chunk - nc = new_chunk(key) - ok = false - - begin - nc << data - chain.next - - flush_trigger = false - @queue.synchronize { - enqueue(chunk) # this is buffer enqueue *hook* - flush_trigger = @queue.empty? - @queue << chunk # actual enqueue - @map[key] = nc - } - - ok = true - # false: queue have 1 or more chunks before this emit - # so this enqueue is not a trigger to flush - # true: queue have no chunks before this emit - # so this enqueue is a trigger to flush this buffer ASAP - return flush_trigger - ensure - nc.purge unless ok - end - - end # synchronize - end - - def keys - @map.keys - end - - def queue_size - @queue.size - end - - def total_queued_chunk_size - total = 0 - synchronize { - @map.each_value {|c| - total += c.size - } - @queue.synchronize { - @queue.each {|c| - total += c.size - } - } - } - total - end - - def new_chunk(key) - raise NotImplementedError, "Implement this method in child class" - end - - def resume - raise NotImplementedError, "Implement this method in child class" - end - - # enqueueing is done by #push - # this method is actually 'enqueue_hook' - def enqueue(chunk) - raise NotImplementedError, "Implement this method in child class" - end - - # get the chunk specified by key, and push it into queue - def push(key) - synchronize do - chunk = @map[key] - if !chunk || chunk.empty? - return false - end - - @queue.synchronize do - enqueue(chunk) - @queue << chunk - @map.delete(key) - end - - return true - end # synchronize - end - - # shift a chunk from queue, write and purge it - # returns boolean to indicate whether this buffer have more chunk to be flushed or not - def pop(out) - chunk = nil - @queue.synchronize do - if @parallel_pop - chunk = @queue.find {|c| c.try_mon_enter } - return false unless chunk - else - chunk = @queue.first - return false unless chunk - return false unless chunk.try_mon_enter - end - end - - begin - # #push(key) does not push empty chunks into queue. - # so this check is nonsense... - if !chunk.empty? - write_chunk(chunk, out) - end - - queue_empty = false - @queue.synchronize do - @queue.delete_if {|c| - c.object_id == chunk.object_id - } - queue_empty = @queue.empty? - end - - chunk.purge - - # return to be flushed once more immediately, or not - return !queue_empty - ensure - chunk.mon_exit - end - end - - def write_chunk(chunk, out) - out.write(chunk) - end - - def clear! - @queue.delete_if {|chunk| - chunk.purge - true - } - end - end -end - diff --git a/lib/fluent/plugin/buf_file.rb b/lib/fluent/plugin/buf_file.rb deleted file mode 100644 index 05222355d9..0000000000 --- a/lib/fluent/plugin/buf_file.rb +++ /dev/null @@ -1,257 +0,0 @@ -# -# Fluentd -# -# 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. -# - -require 'fileutils' -require 'uri' - -require 'fluent/buffer' -require 'fluent/env' -require 'fluent/plugin' -require 'fluent/system_config' - -module Fluent - class FileBufferChunk < BufferChunk - include SystemConfig::Mixin - - FILE_PERMISSION = 0644 - - def initialize(key, path, unique_id, mode="a+", symlink_path = nil) - super(key) - @path = path - @unique_id = unique_id - @file_permission = system_config.file_permission || FILE_PERMISSION - @file = File.open(@path, mode, @file_permission) - @file.binmode - @file.sync = true - @size = @file.stat.size - FileUtils.ln_sf(@path, symlink_path) if symlink_path - end - - attr_reader :unique_id, :path - - def <<(data) - @file.write(data) - @size += data.bytesize - end - - def size - @size - end - - def empty? - @size == 0 - end - - def close - stat = @file.stat - @file.close - if stat.size == 0 - File.unlink(@path) - end - end - - def purge - @file.close - File.unlink(@path) rescue nil # TODO rescue? - end - - def read - @file.pos = 0 - @file.read - end - - def open(&block) - @file.pos = 0 - yield @file - end - - def mv(path) - if Fluent.windows? - pos = @file.pos - @file.close - File.rename(@path, path) - @path = path - @file = File.open(@path, 'rb', @file_permission) - @file.sync = true - @size = @file.size - @file.pos = pos - @size = @file.stat.size - else - File.rename(@path, path) - @path = path - end - end - end - - class FileBuffer < BasicBuffer - include SystemConfig::Mixin - - Plugin.register_buffer('file', self) - - DIR_PERMISSION = 0755 - - @@buffer_paths = {} - - def initialize - require 'uri' - super - - @uri_parser = URI::Parser.new - @symlink_path = nil - end - - desc 'The path where buffer chunks are stored.' - config_param :buffer_path, :string - desc 'If true, queued chunks are flushed at shutdown process.' - config_param :flush_at_shutdown, :bool, default: false - - # 'symlink_path' is currently only for out_file. - # That is the reason why this is not config_param, but attr_accessor. - # See: https://github.com/fluent/fluentd/pull/181 - attr_accessor :symlink_path - - def configure(conf) - super - - if @@buffer_paths.has_key?(@buffer_path) - raise ConfigError, "Other '#{@@buffer_paths[@buffer_path]}' plugin already use same buffer_path: type = #{conf['@type'] || conf['type']}, buffer_path = #{@buffer_path}" - else - @@buffer_paths[@buffer_path] = conf['@type'] || conf['type'] - end - - if pos = @buffer_path.index('*') - @buffer_path_prefix = @buffer_path[0, pos] - @buffer_path_suffix = @buffer_path[(pos + 1)..-1] - else - @buffer_path_prefix = @buffer_path + "." - @buffer_path_suffix = ".log" - end - - @dir_perm = system_config.dir_permission || DIR_PERMISSION - end - - def start - FileUtils.mkdir_p File.dirname(@buffer_path_prefix + "path"), mode: @dir_perm - super - end - - # Dots are separator for many cases: - # we should have to escape dots in keys... - PATH_MATCH = /^([-_.%0-9a-zA-Z]*)\.(b|q)([0-9a-fA-F]{1,32})$/ - - def new_chunk(key) - encoded_key = encode_key(key) - path, tsuffix = make_path(encoded_key, "b") - unique_id = tsuffix_to_unique_id(tsuffix) - FileBufferChunk.new(key, path, unique_id, "a+", @symlink_path) - end - - def resume - maps = [] - queues = [] - - Dir.glob("#{@buffer_path_prefix}*#{@buffer_path_suffix}") {|path| - identifier_part = chunk_identifier_in_path(path) - if m = PATH_MATCH.match(identifier_part) - key = decode_key(m[1]) - bq = m[2] - tsuffix = m[3] - timestamp = m[3].to_i(16) - unique_id = tsuffix_to_unique_id(tsuffix) - - if bq == 'b' - chunk = FileBufferChunk.new(key, path, unique_id, "a+") - maps << [timestamp, chunk] - elsif bq == 'q' - chunk = FileBufferChunk.new(key, path, unique_id, "r") - queues << [timestamp, chunk] - end - end - } - - map = {} - maps.sort_by {|(timestamp,_chunk)| - timestamp - }.each {|(_timestamp,chunk)| - map[chunk.key] = chunk - } - - queue = queues.sort_by {|(timestamp,_chunk)| - timestamp - }.map {|(_timestamp,chunk)| - chunk - } - - return queue, map - end - - def chunk_identifier_in_path(path) - pos_after_prefix = @buffer_path_prefix.length - pos_before_suffix = @buffer_path_suffix.length + 1 # from tail of path - - path.slice(pos_after_prefix..-pos_before_suffix) - end - - def enqueue(chunk) - path = chunk.path - identifier_part = chunk_identifier_in_path(path) - - m = PATH_MATCH.match(identifier_part) - encoded_key = m ? m[1] : "" - tsuffix = m[3] - npath = "#{@buffer_path_prefix}#{encoded_key}.q#{tsuffix}#{@buffer_path_suffix}" - - chunk.mv(npath) - end - - def before_shutdown(out) - if @flush_at_shutdown - synchronize do - @map.each_key {|key| - push(key) - } - while pop(out) - end - end - end - end - - private - - # Dots are separator for many cases: - # we should have to escape dots in keys... - def encode_key(key) - @uri_parser.escape(key, /[^-_.a-zA-Z0-9]/n) # //n switch means explicit 'ASCII-8BIT' pattern - end - - def decode_key(encoded_key) - @uri_parser.unescape(encoded_key) - end - - def make_path(encoded_key, bq) - now = Time.now.utc - timestamp = ((now.to_i * 1000 * 1000 + now.usec) << 12 | rand(0xfff)) - tsuffix = timestamp.to_s(16) - path = "#{@buffer_path_prefix}#{encoded_key}.#{bq}#{tsuffix}#{@buffer_path_suffix}" - return path, tsuffix - end - - def tsuffix_to_unique_id(tsuffix) - # why *2 ? frsyuki said that I forgot why completely. - tsuffix.scan(/../).map {|x| x.to_i(16) }.pack('C*') * 2 - end - end -end diff --git a/lib/fluent/plugin/buf_memory.rb b/lib/fluent/plugin/buf_memory.rb deleted file mode 100644 index 00b3b6f8a0..0000000000 --- a/lib/fluent/plugin/buf_memory.rb +++ /dev/null @@ -1,117 +0,0 @@ -# -# Fluentd -# -# 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. -# - -require 'stringio' - -require 'fluent/engine' -require 'fluent/plugin' -require 'fluent/buffer' - -module Fluent - class MemoryBufferChunk < BufferChunk - def initialize(key, data='') - @data = data - @data.force_encoding('ASCII-8BIT') - now = Time.now.utc - u1 = ((now.to_i*1000*1000+now.usec) << 12 | rand(0xfff)) - @unique_id = [u1 >> 32, u1 & 0xffffffff, rand(0xffffffff), rand(0xffffffff)].pack('NNNN') - super(key) - end - - attr_reader :unique_id - - def <<(data) - data.force_encoding('ASCII-8BIT') - @data << data - end - - def size - @data.bytesize - end - - def close - end - - def purge - end - - def read - @data - end - - def open(&block) - StringIO.open(@data, &block) - end - - # optimize - def write_to(io) - io.write @data - end - - # optimize - def msgpack_each(&block) - u = Fluent::Engine.msgpack_factory.unpacker - u.feed_each(@data, &block) - end - end - - - class MemoryBuffer < BasicBuffer - Plugin.register_buffer('memory', self) - - def initialize - super - end - - desc 'If true, queued chunks are flushed at shutdown process. Otherwise queued chunks are discarded' - config_param :flush_at_shutdown, :bool, default: true - # Overwrite default BasicBuffer#buffer_queue_limit - # to limit total memory usage upto 512MB. - config_set_default :buffer_queue_limit, 64 - - def configure(conf) - super - - unless @flush_at_shutdown - $log.warn "When flush_at_shutdown is false, buf_memory discards buffered chunks at shutdown." - $log.warn "Please confirm 'flush_at_shutdown false' configuration is correct or not." - end - end - - def before_shutdown(out) - if @flush_at_shutdown - synchronize do - @map.each_key {|key| - push(key) - } - while pop(out) - end - end - end - end - - def new_chunk(key) - MemoryBufferChunk.new(key) - end - - def resume - return [], {} - end - - def enqueue(chunk) - end - end -end diff --git a/test/plugin/test_buf_file.rb b/test/plugin/test_buf_file.rb deleted file mode 100644 index 2f3860177c..0000000000 --- a/test/plugin/test_buf_file.rb +++ /dev/null @@ -1,690 +0,0 @@ -# -*- coding: utf-8 -*- -require_relative '../helper' -require 'fluent/test' -require 'fluent/plugin/buf_file' -require 'fluent/system_config' - -require 'fileutils' - -require 'stringio' -require 'msgpack' - -module FluentFileBufferTest - class FileBufferChunkTest < Test::Unit::TestCase - BUF_FILE_TMPDIR = File.expand_path(File.join(File.dirname(__FILE__), '..', 'tmp', 'buf_file_chunk')) - - def setup - FileUtils.rm_rf(BUF_FILE_TMPDIR, secure: true) - FileUtils.mkdir_p BUF_FILE_TMPDIR - end - - def bufpath(unique, link=false) - File.join(BUF_FILE_TMPDIR, unique + '.log' + (link ? '.link' : '')) - end - - def filebufferchunk(key, unique, opts={}) - Fluent::FileBufferChunk.new(key, bufpath(unique), unique, opts[:mode] || "a+", opts[:symlink]) - end - - def test_init - omit "Windows doesn't support symlink" if Fluent.windows? - chunk = filebufferchunk('key', 'init1') - assert_equal 'key', chunk.key - assert_equal 'init1', chunk.unique_id - assert_equal bufpath('init1'), chunk.path - - chunk.close # size==0, then, unlinked - - symlink_path = bufpath('init2', true) - - chunk = filebufferchunk('key2', 'init2', symlink: symlink_path) - assert_equal 'key2', chunk.key - assert_equal 'init2', chunk.unique_id - assert File.exist?(symlink_path) && File.symlink?(symlink_path) - - chunk.close # unlink - - assert File.symlink?(symlink_path) - File.unlink(symlink_path) - end - - class TestWithSystem < self - include Fluent::SystemConfig::Mixin - - OVERRIDE_FILE_PERMISSION = 0620 - CONFIG_SYSTEM = %[ - - file_permission #{OVERRIDE_FILE_PERMISSION} - - ] - - def parse_system(text) - basepath = File.expand_path(File.dirname(__FILE__) + '/../../') - Fluent::Config.parse(text, '(test)', basepath, true).elements.find { |e| e.name == 'system' } - end - - def setup - omit "NTFS doesn't support UNIX like permissions" if Fluent.windows? - # Store default permission - @default_permission = system_config.instance_variable_get(:@file_permission) - end - - def teardown - # Restore default permission - system_config.instance_variable_set(:@file_permission, @default_permission) - end - - def test_init_with_system - system_conf = parse_system(CONFIG_SYSTEM) - sc = Fluent::SystemConfig.new(system_conf) - Fluent::Engine.init(sc) - chunk = filebufferchunk('key', 'init3') - assert_equal 'key', chunk.key - assert_equal 'init3', chunk.unique_id - assert_equal bufpath('init3'), chunk.path - mode = "%o" % File.stat(chunk.path).mode - assert_equal OVERRIDE_FILE_PERMISSION, mode[-3, 3].to_i - - chunk.close # size==0, then, unlinked - end - end - - def test_buffer_chunk_interface - chunk = filebufferchunk('key', 'interface1') - - assert chunk.respond_to?(:empty?) - assert chunk.respond_to?(:<<) - assert chunk.respond_to?(:size) - assert chunk.respond_to?(:close) - assert chunk.respond_to?(:purge) - assert chunk.respond_to?(:read) - assert chunk.respond_to?(:open) - assert chunk.respond_to?(:write_to) - assert chunk.respond_to?(:msgpack_each) - - chunk.close - end - - def test_empty? - chunk = filebufferchunk('e1', 'empty1') - assert chunk.empty? - chunk.close - - open(bufpath('empty2'), 'w') do |file| - file.write "data1\ndata2\n" - end - chunk = filebufferchunk('e2', 'empty2') - assert !(chunk.empty?) - chunk.close - end - - def test_append_close_purge - chunk = filebufferchunk('a1', 'append1') - assert chunk.empty? - - test_data1 = ("1" * 9 + "\n" + "2" * 9 + "\n").force_encoding('ASCII-8BIT') - test_data2 = "日本語Japanese\n".force_encoding('UTF-8') - chunk << test_data1 - chunk << test_data2 - assert_equal 38, chunk.size - chunk.close - - assert File.exist?(bufpath('append1')) - - chunk = filebufferchunk('a1', 'append1', mode: 'r') - test_data = test_data1.force_encoding('ASCII-8BIT') + test_data2.force_encoding('ASCII-8BIT') - - #### TODO: This assertion currently fails. Oops. - # FileBuffer#read does NOT do force_encoding('ASCII-8BIT'). So encoding of output string instance are 'UTF-8'. - # I think it is a kind of bug, but fixing it may break some behavior of buf_file. So I cannot be sure to fix it just now. - # - # assert_equal test_data, chunk.read - - chunk.purge - - assert !(File.exist?(bufpath('append1'))) - end - - def test_empty_chunk_key # for BufferedOutput#emit - chunk = filebufferchunk('', 'append1') - assert chunk.empty? - - test_data1 = ("1" * 9 + "\n" + "2" * 9 + "\n").force_encoding('ASCII-8BIT') - test_data2 = "日本語Japanese\n".force_encoding('UTF-8') - chunk << test_data1 - chunk << test_data2 - assert_equal 38, chunk.size - chunk.close - end - - def test_read - chunk = filebufferchunk('r1', 'read1') - assert chunk.empty? - - d1 = "abcde" * 200 + "\n" - chunk << d1 - d2 = "12345" * 200 + "\n" - chunk << d2 - assert_equal (d1.size + d2.size), chunk.size - - read_data = chunk.read - assert_equal (d1 + d2), read_data - - chunk.purge - end - - def test_open - chunk = filebufferchunk('o1', 'open1') - assert chunk.empty? - - d1 = "abcde" * 200 + "\n" - chunk << d1 - d2 = "12345" * 200 + "\n" - chunk << d2 - assert_equal (d1.size + d2.size), chunk.size - - read_data = chunk.open do |io| - io.read - end - assert_equal (d1 + d2), read_data - - chunk.purge - end - - def test_write_to - chunk = filebufferchunk('w1', 'write1') - assert chunk.empty? - - d1 = "abcde" * 200 + "\n" - chunk << d1 - d2 = "12345" * 200 + "\n" - chunk << d2 - assert_equal (d1.size + d2.size), chunk.size - - dummy_dst = StringIO.new - - chunk.write_to(dummy_dst) - assert_equal (d1 + d2), dummy_dst.string - - chunk.purge - end - - def test_msgpack_each - chunk = filebufferchunk('m1', 'msgpack1') - assert chunk.empty? - - d0 = MessagePack.pack([[1, "foo"], [2, "bar"], [3, "baz"]]) - d1 = MessagePack.pack({"key1" => "value1", "key2" => "value2"}) - d2 = MessagePack.pack("string1") - d3 = MessagePack.pack(1) - d4 = MessagePack.pack(nil) - chunk << d0 - chunk << d1 - chunk << d2 - chunk << d3 - chunk << d4 - - store = [] - chunk.msgpack_each do |data| - store << data - end - - assert_equal 5, store.size - assert_equal [[1, "foo"], [2, "bar"], [3, "baz"]], store[0] - assert_equal({"key1" => "value1", "key2" => "value2"}, store[1]) - assert_equal "string1", store[2] - assert_equal 1, store[3] - assert_equal nil, store[4] - - chunk.purge - end - - def test_mv - chunk = filebufferchunk('m1', 'move1') - assert chunk.empty? - - d1 = "abcde" * 200 + "\n" - chunk << d1 - d2 = "12345" * 200 + "\n" - chunk << d2 - assert_equal (d1.size + d2.size), chunk.size - - assert_equal bufpath('move1'), chunk.path - - assert File.exist?( bufpath( 'move1' ) ) - assert !(File.exist?( bufpath( 'move2' ) )) - - chunk.mv(bufpath('move2')) - - assert !(File.exist?( bufpath( 'move1' ) )) - assert File.exist?( bufpath( 'move2' ) ) - - assert_equal bufpath('move2'), chunk.path - - chunk.purge - end - end - - class FileBufferTest < Test::Unit::TestCase - BUF_FILE_TMPDIR = File.expand_path(File.join(File.dirname(__FILE__), '..', 'tmp', 'buf_file')) - - def setup - FileUtils.rm_rf(BUF_FILE_TMPDIR, secure: true) - FileUtils.mkdir_p BUF_FILE_TMPDIR - end - - def bufpath(basename) - File.join(BUF_FILE_TMPDIR, basename) - end - - def filebuffer(key, unique, opts={}) - Fluent::FileBufferChunk.new(key, bufpath(unique), unique, opts[:mode] || "a+", opts[:symlink]) - end - - def test_init_configure - buf = Fluent::FileBuffer.new - - assert_raise(Fluent::ConfigError){ buf.configure({}) } - - buf.configure({'buffer_path' => bufpath('configure1.*.log')}) - assert_equal bufpath('configure1.*.log'), buf.buffer_path - assert_equal nil, buf.symlink_path - assert_equal false, buf.instance_eval{ @flush_at_shutdown } - - buf2 = Fluent::FileBuffer.new - - # Same buffer_path value is rejected, not to overwrite exisitng buffer file. - assert_raise(Fluent::ConfigError){ buf2.configure({'buffer_path' => bufpath('configure1.*.log')}) } - - buf2.configure({'buffer_path' => bufpath('configure2.*.log'), 'flush_at_shutdown' => ''}) - assert_equal bufpath('configure2.*.log'), buf2.buffer_path - assert_equal true, buf2.instance_eval{ @flush_at_shutdown } - end - - def test_configure_path_prefix_suffix - # With '*' in path, prefix is the part before '*', suffix is the part after '*' - buf = Fluent::FileBuffer.new - - path1 = bufpath('suffpref1.*.log') - prefix1, suffix1 = path1.split('*', 2) - buf.configure({'buffer_path' => path1}) - assert_equal prefix1, buf.instance_eval{ @buffer_path_prefix } - assert_equal suffix1, buf.instance_eval{ @buffer_path_suffix } - - # Without '*', prefix is the string of whole path + '.', suffix is '.log' - path2 = bufpath('suffpref2') - buf.configure({'buffer_path' => path2}) - assert_equal path2 + '.', buf.instance_eval{ @buffer_path_prefix } - assert_equal '.log', buf.instance_eval{ @buffer_path_suffix } - end - - class DummyOutput - attr_accessor :written - - def write(chunk) - @written ||= [] - @written.push chunk - "return value" - end - end - - def test_encode_key - buf = Fluent::FileBuffer.new - safe_chars = '-_.abcdefgxyzABCDEFGXYZ0123456789' - assert_equal safe_chars, buf.send(:encode_key, safe_chars) - unsafe_chars = '-_.abcdefgxyzABCDEFGXYZ0123456789 ~/*()' - assert_equal safe_chars + '%20%7E%2F%2A%28%29', buf.send(:encode_key, unsafe_chars) - end - - def test_decode_key - buf = Fluent::FileBuffer.new - safe_chars = '-_.abcdefgxyzABCDEFGXYZ0123456789' - assert_equal safe_chars, buf.send(:decode_key, safe_chars) - unsafe_chars = '-_.abcdefgxyzABCDEFGXYZ0123456789 ~/*()' - assert_equal unsafe_chars, buf.send(:decode_key, safe_chars + '%20%7E%2F%2A%28%29') - - assert_equal safe_chars, buf.send(:decode_key, buf.send(:encode_key, safe_chars)) - assert_equal unsafe_chars, buf.send(:decode_key, buf.send(:encode_key, unsafe_chars)) - end - - def test_make_path - buf = Fluent::FileBuffer.new - buf.configure({'buffer_path' => bufpath('makepath.*.log')}) - prefix = buf.instance_eval{ @buffer_path_prefix } - suffix = buf.instance_eval{ @buffer_path_suffix } - - path,tsuffix = buf.send(:make_path, buf.send(:encode_key, 'foo bar'), 'b') - assert path =~ /\A#{prefix}[-_.a-zA-Z0-9\%]+\.[bq][0-9a-f]+#{suffix}\Z/, "invalid format:#{path}" - assert tsuffix =~ /\A[0-9a-f]+\Z/, "invalid hexadecimal:#{tsuffix}" - - path,tsuffix = buf.send(:make_path, buf.send(:encode_key, 'baz 123'), 'q') - assert path =~ /\A#{prefix}[-_.a-zA-Z0-9\%]+\.[bq][0-9a-f]+#{suffix}\Z/, "invalid format:#{path}" - assert tsuffix =~ /\A[0-9a-f]+\Z/, "invalid hexadecimal:#{tsuffix}" - end - - def test_tsuffix_to_unique_id - buf = Fluent::FileBuffer.new - # why *2 ? frsyuki said "I forgot why completely." - assert_equal "\xFF\xFF\xFF\xFF".force_encoding('ASCII-8BIT'), buf.send(:tsuffix_to_unique_id, 'ffff') - assert_equal "\x88\x00\xFF\x00\x11\xEE\x88\x00\xFF\x00\x11\xEE".force_encoding('ASCII-8BIT'), buf.send(:tsuffix_to_unique_id, '8800ff0011ee') - end - - def test_start_makes_parent_directories - buf = Fluent::FileBuffer.new - buf.configure({'buffer_path' => bufpath('start/base.*.log')}) - parent_dirname = File.dirname(buf.instance_eval{ @buffer_path_prefix }) - assert !(Dir.exist?(parent_dirname)) - buf.start - assert Dir.exist?(parent_dirname) - end - - def test_new_chunk - buf = Fluent::FileBuffer.new - buf.configure({'buffer_path' => bufpath('new_chunk_1')}) - prefix = buf.instance_eval{ @buffer_path_prefix } - suffix = buf.instance_eval{ @buffer_path_suffix } - - chunk = buf.new_chunk('key1') - assert chunk - assert File.exist?(chunk.path) - assert chunk.path =~ /\A#{prefix}[-_.a-zA-Z0-9\%]+\.b[0-9a-f]+#{suffix}\Z/, "path from new_chunk must be a 'b' buffer chunk" - chunk.close - end - - def test_chunk_identifier_in_path - buf1 = Fluent::FileBuffer.new - buf1.configure({'buffer_path' => bufpath('chunkid1')}) - prefix1 = buf1.instance_eval{ @buffer_path_prefix } - suffix1 = buf1.instance_eval{ @buffer_path_suffix } - - chunk1 = buf1.new_chunk('key1') - assert_equal chunk1.path, prefix1 + buf1.chunk_identifier_in_path(chunk1.path) + suffix1 - - buf2 = Fluent::FileBuffer.new - buf2.configure({'buffer_path' => bufpath('chunkid2')}) - prefix2 = buf2.instance_eval{ @buffer_path_prefix } - suffix2 = buf2.instance_eval{ @buffer_path_suffix } - - chunk2 = buf2.new_chunk('key2') - assert_equal chunk2.path, prefix2 + buf2.chunk_identifier_in_path(chunk2.path) + suffix2 - end - - def test_enqueue_moves_chunk_from_b_to_q - buf = Fluent::FileBuffer.new - buf.configure({'buffer_path' => bufpath('enqueue1')}) - prefix = buf.instance_eval{ @buffer_path_prefix } - suffix = buf.instance_eval{ @buffer_path_suffix } - - chunk = buf.new_chunk('key1') - chunk << "data1\ndata2\n" - - assert chunk - old_path = chunk.path.dup - assert File.exist?(chunk.path) - assert chunk.path =~ /\A#{prefix}[-_.a-zA-Z0-9\%]+\.b[0-9a-f]+#{suffix}\Z/, "path from new_chunk must be a 'b' buffer chunk" - - buf.enqueue(chunk) - - assert chunk - assert File.exist?(chunk.path) - assert !(File.exist?(old_path)) - assert chunk.path =~ /\A#{prefix}[-_.a-zA-Z0-9\%]+\.q[0-9a-f]+#{suffix}\Z/, "enqueued chunk's path must be a 'q' buffer chunk" - - data = chunk.read - assert "data1\ndata2\n", data - end - - # empty chunk keys are used w/ BufferedOutput - # * ObjectBufferedOutput's keys are tag - # * TimeSlicedOutput's keys are time_key - def test_enqueue_chunk_with_empty_key - buf = Fluent::FileBuffer.new - buf.configure({'buffer_path' => bufpath('enqueue2')}) - prefix = buf.instance_eval{ @buffer_path_prefix } - suffix = buf.instance_eval{ @buffer_path_suffix } - - chunk = buf.new_chunk('') - chunk << "data1\ndata2\n" - - assert chunk - old_path = chunk.path.dup - assert File.exist?(chunk.path) - # chunk key is empty - assert chunk.path =~ /\A#{prefix}\.b[0-9a-f]+#{suffix}\Z/, "path from new_chunk must be a 'b' buffer chunk" - - buf.enqueue(chunk) - - assert chunk - assert File.exist?(chunk.path) - assert !(File.exist?(old_path)) - # chunk key is empty - assert chunk.path =~ /\A#{prefix}\.q[0-9a-f]+#{suffix}\Z/, "enqueued chunk's path must be a 'q' buffer chunk" - - data = chunk.read - assert "data1\ndata2\n", data - end - - def test_before_shutdown_without_flush_at_shutdown - buf = Fluent::FileBuffer.new - buf.configure({'buffer_path' => bufpath('before_shutdown1')}) - buf.start - - # before_shutdown does nothing - - c1 = [ buf.new_chunk('k0'), buf.new_chunk('k1'), buf.new_chunk('k2'), buf.new_chunk('k3') ] - c2 = [ buf.new_chunk('q0'), buf.new_chunk('q1') ] - - buf.instance_eval do - @map = { - 'k0' => c1[0], 'k1' => c1[1], 'k2' => c1[2], 'k3' => c1[3], - 'q0' => c2[0], 'q1' => c2[1] - } - end - c1[0] << "data1\ndata2\n" - c1[1] << "data1\ndata2\n" - c1[2] << "data1\ndata2\n" - # k3 chunk is empty! - - c2[0] << "data1\ndata2\n" - c2[1] << "data1\ndata2\n" - buf.push('q0') - buf.push('q1') - - buf.instance_eval do - @enqueue_hook_times = 0 - def enqueue(chunk) - @enqueue_hook_times += 1 - end - end - assert_equal 0, buf.instance_eval{ @enqueue_hook_times } - - out = DummyOutput.new - assert_equal nil, out.written - - buf.before_shutdown(out) - - assert_equal 0, buf.instance_eval{ @enqueue_hook_times } # k0, k1, k2 - assert_nil out.written - end - - def test_before_shutdown_with_flush_at_shutdown - buf = Fluent::FileBuffer.new - buf.configure({'buffer_path' => bufpath('before_shutdown2'), 'flush_at_shutdown' => 'true'}) - buf.start - - # before_shutdown flushes all chunks in @map and @queue - - c1 = [ buf.new_chunk('k0'), buf.new_chunk('k1'), buf.new_chunk('k2'), buf.new_chunk('k3') ] - c2 = [ buf.new_chunk('q0'), buf.new_chunk('q1') ] - - buf.instance_eval do - @map = { - 'k0' => c1[0], 'k1' => c1[1], 'k2' => c1[2], 'k3' => c1[3], - 'q0' => c2[0], 'q1' => c2[1] - } - end - c1[0] << "data1\ndata2\n" - c1[1] << "data1\ndata2\n" - c1[2] << "data1\ndata2\n" - # k3 chunk is empty! - - c2[0] << "data1\ndata2\n" - c2[1] << "data1\ndata2\n" - buf.push('q0') - buf.push('q1') - - buf.instance_eval do - @enqueue_hook_times = 0 - def enqueue(chunk) - @enqueue_hook_times += 1 - end - end - assert_equal 0, buf.instance_eval{ @enqueue_hook_times } - - out = DummyOutput.new - assert_equal nil, out.written - - buf.before_shutdown(out) - - assert_equal 3, buf.instance_eval{ @enqueue_hook_times } # k0, k1, k2 - assert_equal 5, out.written.size - assert_equal [c2[0], c2[1], c1[0], c1[1], c1[2]], out.written - end - - def test_resume - buffer_path_for_resume_test = bufpath('resume') - - buf1 = Fluent::FileBuffer.new - buf1.configure({'buffer_path' => buffer_path_for_resume_test}) - prefix = buf1.instance_eval{ @buffer_path_prefix } - suffix = buf1.instance_eval{ @buffer_path_suffix } - - buf1.start - - chunk1 = buf1.new_chunk('key1') - chunk1 << "data1\ndata2\n" - - chunk2 = buf1.new_chunk('key2') - chunk2 << "data3\ndata4\n" - - assert chunk1 - assert chunk1.path =~ /\A#{prefix}[-_.a-zA-Z0-9\%]+\.b[0-9a-f]+#{suffix}\Z/, "path from new_chunk must be a 'b' buffer chunk" - - buf1.enqueue(chunk1) - - assert chunk1 - assert chunk1.path =~ /\A#{prefix}[-_.a-zA-Z0-9\%]+\.q[0-9a-f]+#{suffix}\Z/, "chunk1 must be enqueued" - assert chunk2 - assert chunk2.path =~ /\A#{prefix}[-_.a-zA-Z0-9\%]+\.b[0-9a-f]+#{suffix}\Z/, "chunk2 is not enqueued yet" - - buf1.shutdown - - buf2 = Fluent::FileBuffer.new - Fluent::FileBuffer.send(:class_variable_set, :'@@buffer_paths', {}) - buf2.configure({'buffer_path' => buffer_path_for_resume_test}) - prefix = buf2.instance_eval{ @buffer_path_prefix } - suffix = buf2.instance_eval{ @buffer_path_suffix } - - # buf1.start -> resume is normal operation, but now, we cannot it. - queue, map = buf2.resume - - assert_equal 1, queue.size - assert_equal 1, map.size - - resumed_chunk1 = queue.first - assert_equal chunk1.path, resumed_chunk1.path - resumed_chunk2 = map['key2'] - assert_equal chunk2.path, resumed_chunk2.path - - assert_equal "data1\ndata2\n", resumed_chunk1.read - assert_equal "data3\ndata4\n", resumed_chunk2.read - end - - class DummyChain - def next - true - end - end - - def test_resume_only_for_my_buffer_path - chain = DummyChain.new - - buffer_path_for_resume_test_1 = bufpath('resume_fix.1.*.log') - buffer_path_for_resume_test_2 = bufpath('resume_fix.*.log') - - buf1 = Fluent::FileBuffer.new - buf1.configure({'buffer_path' => buffer_path_for_resume_test_1}) - buf1.start - - buf1.emit('key1', "x1\ty1\tz1\n", chain) - buf1.emit('key1', "x2\ty2\tz2\n", chain) - - assert buf1.instance_eval{ @map['key1'] } - - buf1.shutdown - - buf2 = Fluent::FileBuffer.new - buf2.configure({'buffer_path' => buffer_path_for_resume_test_2}) # other buffer_path - - queue, map = buf2.resume - - assert_equal 0, queue.size - - ### TODO: This map size MUST be 0, but actually, 1 - # This is because 1.XXXXX is misunderstood like chunk key of resume_fix.*.log. - # This may be a kind of bug, but we cannot decide whether 1. is a part of chunk key or not, - # because current version of buffer plugin uses '.'(dot) as a one of chars for chunk encoding. - # I think that this is a mistake of design, but we cannot fix it because updated plugin become - # not to be able to resume existing file buffer chunk. - # We will fix it in next API version of buffer plugin. - assert_equal 1, map.size - end - - class TestWithSystem < self - include Fluent::SystemConfig::Mixin - - OVERRIDE_DIR_PERMISSION = 720 - CONFIG_WITH_SYSTEM = %[ - - dir_permission #{OVERRIDE_DIR_PERMISSION} - - ] - - def setup_system_config - system_conf = parse_system(CONFIG_WITH_SYSTEM) - sc = Fluent::SystemConfig.new(system_conf) - Fluent::Engine.init(sc) - end - - def setup - omit "NTFS doesn't support UNIX like permissions" if Fluent.windows? - setup_system_config - FileUtils.rm_rf(BUF_FILE_TMPDIR, secure: true) - end - - def parse_system(text) - basepath = File.expand_path(File.dirname(__FILE__) + '/../../') - Fluent::Config.parse(text, '(test)', basepath, true).elements.find { |e| e.name == 'system' } - end - - def test_new_chunk - buf = Fluent::FileBuffer.new - buf.configure({'buffer_path' => bufpath('new_chunk_1')}) - prefix = buf.instance_eval{ @buffer_path_prefix } - suffix = buf.instance_eval{ @buffer_path_suffix } - - # To create buffer directory - buf.start - - chunk = buf.new_chunk('key1') - assert chunk - assert File.exist?(chunk.path) - assert chunk.path =~ /\A#{prefix}[-_.a-zA-Z0-9\%]+\.b[0-9a-f]+#{suffix}\Z/, "path from new_chunk must be a 'b' buffer chunk" - chunk.close - - mode = "%o" % File.stat(BUF_FILE_TMPDIR).mode - assert_equal(OVERRIDE_DIR_PERMISSION, mode[-3, 3].to_i) - end - end - end -end diff --git a/test/plugin/test_buf_memory.rb b/test/plugin/test_buf_memory.rb deleted file mode 100644 index 0d6431d0d7..0000000000 --- a/test/plugin/test_buf_memory.rb +++ /dev/null @@ -1,204 +0,0 @@ -# -*- coding: utf-8 -*- -require_relative '../helper' -require 'fluent/test' -require 'fluent/plugin/buf_memory' - -require 'stringio' -require 'msgpack' - -module FluentMemoryBufferTest - class MemoryBufferChunkTest < Test::Unit::TestCase - def test_init - chunk = Fluent::MemoryBufferChunk.new('key') - assert_equal 'key', chunk.key - assert_equal '', chunk.instance_eval{ @data } - assert_equal 'ASCII-8BIT', chunk.instance_eval{ @data }.encoding.to_s - assert chunk.unique_id # non nil - - chunk2 = Fluent::MemoryBufferChunk.new('initdata', 'data') - assert_equal 'initdata', chunk2.key - assert_equal 'data', chunk2.instance_eval{ @data } - end - - def test_buffer_chunk_interface - chunk = Fluent::BufferChunk.new('key') - - assert chunk.respond_to?(:empty?) - assert chunk.respond_to?(:<<) - assert chunk.respond_to?(:size) - assert chunk.respond_to?(:close) - assert chunk.respond_to?(:purge) - assert chunk.respond_to?(:read) - assert chunk.respond_to?(:open) - assert chunk.respond_to?(:write_to) - assert chunk.respond_to?(:msgpack_each) - end - - def test_empty? - chunk = Fluent::MemoryBufferChunk.new('key') - assert chunk.empty? - - chunk.instance_eval{ @data = "non empty" } - assert !(chunk.empty?) - end - - def test_append_data_and_size - chunk = Fluent::MemoryBufferChunk.new('key') - assert_equal '', chunk.instance_eval{ @data } - - chunk << "foo bar baz\n".force_encoding('UTF-8') - assert_equal "foo bar baz\n", chunk.instance_eval{ @data } - assert_equal 'ASCII-8BIT', chunk.instance_eval{ @data }.encoding.to_s - - assert_equal 12, chunk.size # bytesize - - chunk << "日本語Japanese\n".force_encoding('UTF-8') - assert_equal "foo bar baz\n日本語Japanese\n".force_encoding('ASCII-8BIT'), chunk.instance_eval{ @data } - assert_equal 'ASCII-8BIT', chunk.instance_eval{ @data }.encoding.to_s - - assert_equal 30, chunk.size # bytesize - end - - def test_close_and_purge_does_nothing - chunk = Fluent::MemoryBufferChunk.new('key', 'data') - chunk.close - chunk.close - chunk.close - chunk.close - chunk.purge - chunk.purge - chunk.purge - chunk.purge - assert_equal 'data', chunk.instance_eval{ @data } - end - - def test_read_just_returns_data - data = "data1\ndata2\n" - chunk = Fluent::MemoryBufferChunk.new('key', data) - assert_equal data, chunk.read - assert_equal data.object_id, chunk.read.object_id - end - - def test_open - # StringIO.open(@data, &block) - chunk = Fluent::MemoryBufferChunk.new('key', 'foo bar baz') - chunk.open do |io| - assert 'foo bar baz', io.read - end - end - - def test_write_to - chunk = Fluent::MemoryBufferChunk.new('key', 'foo bar baz') - dummy_dst = StringIO.new - chunk.write_to(dummy_dst) - assert_equal 'foo bar baz', dummy_dst.string - end - - def test_msgpack_each - d0 = MessagePack.pack([[1, "foo"], [2, "bar"], [3, "baz"]]) - d1 = MessagePack.pack({"key1" => "value1", "key2" => "value2"}) - d2 = MessagePack.pack("string1") - d3 = MessagePack.pack(1) - d4 = MessagePack.pack(nil) - chunk = Fluent::MemoryBufferChunk.new('key', d0 + d1 + d2 + d3 + d4) - - store = [] - chunk.msgpack_each do |data| - store << data - end - - assert_equal 5, store.size - assert_equal [[1, "foo"], [2, "bar"], [3, "baz"]], store[0] - assert_equal({"key1" => "value1", "key2" => "value2"}, store[1]) - assert_equal "string1", store[2] - assert_equal 1, store[3] - assert_equal nil, store[4] - end - end - - class MemoryBufferTest < Test::Unit::TestCase - def test_init_configure - buf = Fluent::MemoryBuffer.new - - buf.configure({}) - assert buf.flush_at_shutdown - assert_equal 64, buf.buffer_queue_limit - end - - class DummyOutput - attr_accessor :written - - def write(chunk) - @written ||= [] - @written.push chunk - "return value" - end - end - - def test_before_shutdown - buf = Fluent::MemoryBuffer.new - buf.start - - # before_shutdown flushes all chunks in @map and @queue - - c1 = [ buf.new_chunk('k0'), buf.new_chunk('k1'), buf.new_chunk('k2'), buf.new_chunk('k3') ] - c2 = [ buf.new_chunk('q0'), buf.new_chunk('q1') ] - - buf.instance_eval do - @map = { - 'k0' => c1[0], 'k1' => c1[1], 'k2' => c1[2], 'k3' => c1[3], - 'q0' => c2[0], 'q1' => c2[1] - } - end - c1[0] << "data1\ndata2\n" - c1[1] << "data1\ndata2\n" - c1[2] << "data1\ndata2\n" - # k3 chunk is empty! - - c2[0] << "data1\ndata2\n" - c2[1] << "data1\ndata2\n" - buf.push('q0') - buf.push('q1') - - buf.instance_eval do - @enqueue_hook_times = 0 - def enqueue(chunk) - @enqueue_hook_times += 1 - end - end - assert_equal 0, buf.instance_eval{ @enqueue_hook_times } - - out = DummyOutput.new - assert_equal nil, out.written - - buf.before_shutdown(out) - - assert_equal 3, buf.instance_eval{ @enqueue_hook_times } # k0, k1, k2 - assert_equal 5, out.written.size - assert_equal [c2[0], c2[1], c1[0], c1[1], c1[2]], out.written - end - - def test_new_chunk - buf = Fluent::MemoryBuffer.new - chunk = buf.new_chunk('key') - assert_equal Fluent::MemoryBufferChunk, chunk.class - assert_equal 'key', chunk.key - assert chunk.empty? - end - - def test_resume - buf = Fluent::MemoryBuffer.new - resumed_queue, resumed_store = buf.resume - assert resumed_queue.empty? - assert resumed_store.empty? - end - - def test_enqueue_does_nothing # enqueue hook - buf = Fluent::MemoryBuffer.new - chunk = Fluent::MemoryBufferChunk.new('k', "data1\ndata2\n") - assert_equal "data1\ndata2\n", chunk.read - buf.enqueue(chunk) - assert_equal "data1\ndata2\n", chunk.read - end - end -end diff --git a/test/test_buffer.rb b/test/test_buffer.rb deleted file mode 100644 index 8d3bdbf5c3..0000000000 --- a/test/test_buffer.rb +++ /dev/null @@ -1,606 +0,0 @@ -require_relative 'helper' -require 'fluent/test' -require 'fluent/buffer' - -require 'stringio' -require 'msgpack' -require 'timeout' - -module FluentBufferTest - class BufferTest < Test::Unit::TestCase - def test_buffer_interface - buf = Fluent::Buffer.new - - assert buf.respond_to?(:configure) - assert buf.respond_to?(:start) - assert buf.respond_to?(:shutdown) - assert buf.respond_to?(:before_shutdown) - - # virtual methods - assert buf.respond_to?(:emit) - assert_raise(NotImplementedError){ buf.emit('key', 'data', 'chain') } - assert buf.respond_to?(:keys) - assert_raise(NotImplementedError){ buf.keys } - assert buf.respond_to?(:push) - assert_raise(NotImplementedError){ buf.push('key') } - assert buf.respond_to?(:pop) - assert_raise(NotImplementedError){ buf.pop('out') } - assert buf.respond_to?(:clear!) - assert_raise(NotImplementedError){ buf.clear! } - end - - def test_buffer_does_nothing - buf = Fluent::Buffer.new - - buf.start - buf.before_shutdown(nil) # out == nil - buf.shutdown - end - end - - class DummyChunk < Fluent::BufferChunk - attr_accessor :size, :data, :purged, :closed - def initialize(key, size=0) - super(key) - @size = size - end - - def <<(data) - @size += data.bytesize - end - - def open(&block) - StringIO.open(@data, &block) - end - - def purge - @purged = true - end - - def close - @closed = true - end - end - - class BufferChunkTest < Test::Unit::TestCase - def test_has_key - chunk = Fluent::BufferChunk.new('key') - assert_equal 'key', chunk.key - end - - def test_buffer_chunk_interface - chunk = Fluent::BufferChunk.new('key') - - assert chunk.respond_to?(:empty?) - assert chunk.respond_to?(:write_to) - assert chunk.respond_to?(:msgpack_each) - - # virtual methods - assert chunk.respond_to?(:<<) - assert_raise(NotImplementedError){ chunk << 'data' } - assert chunk.respond_to?(:size) - assert_raise(NotImplementedError){ chunk.size } - assert chunk.respond_to?(:close) - assert_raise(NotImplementedError){ chunk.close } - assert chunk.respond_to?(:purge) - assert_raise(NotImplementedError){ chunk.purge } - assert chunk.respond_to?(:read) - assert_raise(NotImplementedError){ chunk.read } - assert chunk.respond_to?(:open) - assert_raise(NotImplementedError){ chunk.open } - end - - def test_empty? - dchunk = DummyChunk.new('key', 1) - - assert !(dchunk.empty?) - - dchunk.size = 0 - assert dchunk.empty? - end - - def test_write_to - dummy_chunk = DummyChunk.new('key') - dummy_chunk.data = 'foo bar baz' - - dummy_dst = StringIO.new - dummy_chunk.write_to(dummy_dst) - assert_equal 'foo bar baz', dummy_dst.string - end - - def test_msgpack_each - dummy_chunk = DummyChunk.new('key') - d0 = MessagePack.pack([[1, "foo"], [2, "bar"], [3, "baz"]]) - d1 = MessagePack.pack({"key1" => "value1", "key2" => "value2"}) - d2 = MessagePack.pack("string1") - d3 = MessagePack.pack(1) - d4 = MessagePack.pack(nil) - - dummy_chunk.data = d0 + d1 + d2 + d3 + d4 - - store = [] - dummy_chunk.msgpack_each do |data| - store << data - end - - assert_equal 5, store.size - assert_equal [[1, "foo"], [2, "bar"], [3, "baz"]], store[0] - assert_equal({"key1" => "value1", "key2" => "value2"}, store[1]) - assert_equal "string1", store[2] - assert_equal 1, store[3] - assert_equal nil, store[4] - end - end - - class DummyBuffer < Fluent::BasicBuffer - attr_accessor :queue, :map, :enqueue_hook_times - - def initialize - super - @queue = nil - @map = nil - @enqueue_hook_times = 0 - end - - def resume - return [], {} - end - - def new_chunk(key) - DummyChunk.new(key) - end - - def enqueue(chunk) - @enqueue_hook_times += 1 - end - end - - class DummyChain - def next - true - end - end - - class BasicBufferTest < Test::Unit::TestCase - def test_parallel_pop_default - bb = Fluent::BasicBuffer.new - - assert bb.instance_eval{ @parallel_pop } - bb.enable_parallel(false) - assert !(bb.instance_eval{ @parallel_pop }) - bb.enable_parallel() - assert bb.instance_eval{ @parallel_pop } - end - - def test_configure - bb1 = Fluent::BasicBuffer.new - bb1.configure({}) - assert_equal 8 * 1024 * 1024, bb1.buffer_chunk_limit - assert_equal 256, bb1.buffer_queue_limit - - bb2 = Fluent::BasicBuffer.new - bb2.configure({ - "buffer_chunk_limit" => 256 * 1024 * 1024, - "buffer_queue_limit" => 16 - }) - assert_equal 256 * 1024 * 1024, bb2.buffer_chunk_limit - assert_equal 16, bb2.buffer_queue_limit - end - - def test_virtual_methods - bb = Fluent::BasicBuffer.new - - assert_raise(NotImplementedError){ bb.new_chunk('key') } - assert_raise(NotImplementedError){ bb.resume } - assert_raise(NotImplementedError){ bb.enqueue('chunk') } - end - - def test_start - db = DummyBuffer.new - db.start - assert_equal([], db.queue) - assert_equal({}, db.map) - end - - def test_shutdown - db1 = DummyBuffer.new - db1.start - db1.shutdown - - db2 = DummyBuffer.new - db2.start - - chunks = [ DummyChunk.new('k1'), DummyChunk.new('k2'), DummyChunk.new('k3'), DummyChunk.new('k4') ] - - db2.queue << chunks[0] - db2.queue << chunks[1] - db2.map = { 'k3' => chunks[2], 'k4' => chunks[3] } - - db2.shutdown - - assert chunks[0].closed - assert chunks[1].closed - assert chunks[2].closed - assert chunks[3].closed - - assert_equal 0, db2.queue.size - end - - def test_storable? - db = DummyBuffer.new - db.configure({}) - assert_equal 8 * 1024 * 1024, db.buffer_chunk_limit - assert_equal 256, db.buffer_queue_limit - - # db.storable?(chunk, data) - chunk0 = DummyChunk.new('k', 0) - chunk1 = DummyChunk.new('k', 7 * 1024 * 1024) - - assert db.storable?(chunk0, 'b' * 1024 * 1024) - assert db.storable?(chunk0, 'b' * 8 * 1024 * 1024) - assert !(db.storable?(chunk0, 'b' * 9 * 1024 * 1024)) - - assert db.storable?(chunk1, 'b' * 1024 * 1024) - assert !(db.storable?(chunk1, 'b' * ( 1024 * 1024 + 1 ) )) - end - - def test_emit - db = DummyBuffer.new - db.configure({}) - db.start - - chain = DummyChain.new - - assert_equal 8 * 1024 * 1024, db.buffer_chunk_limit - assert_equal 256, db.buffer_queue_limit - - assert_equal 0, db.enqueue_hook_times - - s1m = "a" * 1024 * 1024 - - d1 = s1m * 4 - d2 = s1m * 4 #=> 8 - d3 = s1m * 1 #=> 9, 1 - d4 = s1m * 6 #=> 7 - d5 = s1m * 2 #=> 9, 2 - d6 = s1m * 9 #=> 11, 9 - d7 = s1m * 9 #=> 18, 9 - d8 = s1m * 1 #=> 10, 1 - d9 = s1m * 2 #=> 3 - - assert !(db.emit('key', d1, chain)) # stored in new chunk, and queue is empty - assert !(db.map['key'].empty?) - assert_equal 0, db.queue.size - assert_equal 0, db.enqueue_hook_times - - assert !(db.emit('key', d2, chain)) # just storable, not queued yet. - assert_equal 0, db.queue.size - assert_equal 0, db.enqueue_hook_times - - assert db.emit('key', d3, chain) # not storable, so old chunk is enqueued & new chunk size is 1m and to be flushed - assert_equal 1, db.queue.size - assert_equal 1, db.enqueue_hook_times - - assert !(db.emit('key', d4, chain)) # stored in chunk - assert_equal 1, db.queue.size - assert_equal 1, db.enqueue_hook_times - - assert !(db.emit('key', d5, chain)) # not storable, old chunk is enqueued & new chunk size is 2m - # not to be flushed (queue is not empty) - assert_equal 2, db.queue.size - assert_equal 2, db.enqueue_hook_times - - db.queue.reject!{|v| true } # flush - - assert db.emit('key', d6, chain) # not storable, old chunk is enqueued - # new chunk is larger than buffer_chunk_limit - # to be flushed - assert_equal 1, db.queue.size - assert_equal 3, db.enqueue_hook_times - - assert !(db.emit('key', d7, chain)) # chunk before emit is already larger than buffer_chunk_limit, so enqueued - # not to be flushed - assert_equal 2, db.queue.size - assert_equal 4, db.enqueue_hook_times - - db.queue.reject!{|v| true } # flush - - assert db.emit('key', d8, chain) # chunk before emit is already larger than buffer_chunk_limit, so enqueued - # to be flushed because just after flushing - assert_equal 1, db.queue_size - assert_equal 5, db.enqueue_hook_times - - db.queue.reject!{|v| true } # flush - - assert !(db.emit('key', d9, chain)) # stored in chunk - assert_equal 0, db.queue_size - assert_equal 5, db.enqueue_hook_times - end - - def test_keys - db = DummyBuffer.new - db.start - - chunks = [ DummyChunk.new('k1'), DummyChunk.new('k2'), DummyChunk.new('k3'), DummyChunk.new('k4') ] - - db.queue << chunks[0] - db.queue << chunks[1] - db.map = { 'k3' => chunks[2], 'k4' => chunks[3] } - - assert_equal ['k3', 'k4'], db.keys - end - - def test_queue_size - db = DummyBuffer.new - db.start - - chunks = [ DummyChunk.new('k1'), DummyChunk.new('k2'), DummyChunk.new('k3'), DummyChunk.new('k4') ] - - db.queue << chunks[0] - db.queue << chunks[1] - db.map = { 'k3' => chunks[2], 'k4' => chunks[3] } - - assert_equal 2, db.queue_size - end - - def test_total_queued_chunk_size - db = DummyBuffer.new - db.start - - chunks = [ DummyChunk.new('k1', 1000), DummyChunk.new('k2', 2000), DummyChunk.new('k3', 3000), DummyChunk.new('k4', 4000) ] - - db.queue << chunks[0] - db.queue << chunks[1] - db.map = { 'k3' => chunks[2], 'k4' => chunks[3] } - - assert_equal (1000 + 2000 + 3000 + 4000), db.total_queued_chunk_size - end - - def test_push - db = DummyBuffer.new - db.start - - chunks = [ DummyChunk.new('k1', 1000), DummyChunk.new('k2', 2000), DummyChunk.new('k3', 3000), DummyChunk.new('k4', 4000) ] - - db.map = { 'k1' => chunks[0], 'k2' => chunks[1], 'k3' => chunks[2], 'k4' => chunks[3] } - - assert_equal 0, db.queue.size - assert_equal 4, db.map.size - - # if key does not exits, this method doesn't anything, and returns false - assert_nil db.map['k5'] - assert !(db.push('k5')) - assert_equal 0, db.queue.size - - # if empty chunk exists for specified key, this method doesn't anything and returns false - empty_chunk = DummyChunk.new('key') - db.map['k5'] = empty_chunk - assert !(db.push('k5')) - assert_equal empty_chunk, db.map['k5'] - - # if non-empty chunk exists for specified key, that chunk is enqueued, and true returned - assert db.push('k3') - assert_equal 1, db.queue.size - assert_equal 3000, db.queue.first.size - assert_nil db.map['k3'] - assert_equal 1, db.instance_eval{ @enqueue_hook_times } - end - - class DummyOutput - attr_accessor :written - - def write(chunk) - @written = chunk - "return value" - end - end - - def test_pop - ### pop(out) - # 1. find a chunk that not owned (by checking monitor) - # 2. return false if @queue is empty or all chunks are already owned - # 3. call `write_chunk(chunk, out)` if it isn't empty - # 4. remove that chunk from @queue - # 5. call `chunk.purge` - # 6. return @queue is not empty, or not - - db = DummyBuffer.new - db.start - out = DummyOutput.new - - assert !(db.pop(out)) # queue is empty - assert_nil out.written - - c1 = DummyChunk.new('k1', 1) - db.map = { 'k1' => c1 } - db.push('k1') - assert_equal 1, db.queue.size - - pop_return_value = nil - c1.synchronize do - pop_return_value = Thread.new { - db.pop(out) - }.value - end - assert !(pop_return_value) # a chunk is in queue, and it's owned by another thread - assert_equal 1, db.queue.size - assert_nil out.written - assert_nil c1.purged - - c2 = DummyChunk.new('k2', 1) - db.map['k2'] = c2 - db.push('k2') - assert_equal 2, db.queue.size - - pop_return_value = nil - c1.synchronize do - pop_return_value = Thread.new { - c2.synchronize do - Thread.new { - db.pop(out) - }.value - end - }.value - end - assert !(pop_return_value) # two chunks are in queue, and these are owned by another thread - assert_equal 2, db.queue.size - assert_nil out.written - assert_nil c1.purged - assert_nil c2.purged - - c3 = DummyChunk.new('k3', 1) - db.map['k3'] = c3 - db.push('k3') - c4 = DummyChunk.new('k4', 1) - db.map['k4'] = c4 - db.push('k4') - assert_equal 4, db.queue.size - - # all of c[1234] are not empty - queue_to_be_flushed_more = db.pop(out) - assert queue_to_be_flushed_more # queue has more chunks - assert c1.purged # the first chunk is shifted, and purged - assert_equal c1, out.written # empty chunk is not passed to output plugin - assert_equal 3, db.queue.size - - c3.synchronize do - queue_to_be_flushed_more = Thread.new { - db.pop(out) - }.value - end - assert queue_to_be_flushed_more # c3, c4 exists in queue - assert c2.purged - assert_equal c2, out.written - assert_equal 2, db.queue.size - - c3.synchronize do - queue_to_be_flushed_more = Thread.new { - db.pop(out) - }.value - end - assert queue_to_be_flushed_more # c3 exists in queue - assert c4.purged - assert_equal c4, out.written - assert_equal 1, db.queue.size - - queue_to_be_flushed_more = db.pop(out) - assert c3.purged - assert_equal c3, out.written - assert_equal 0, db.queue.size - end - - def test_write_chunk - db = DummyBuffer.new - db.start - - chunk = DummyChunk.new('k1', 1) - out = DummyOutput.new - - assert_equal "return value", db.write_chunk(chunk, out) - assert_equal chunk, out.written - end - - def test_clear! - db = DummyBuffer.new - db.start - - keys = (1..5).map{ |i| "c_#{i}" } - chunks = keys.map{ |k| DummyChunk.new(k, 1) } - db.map = Hash[ [keys,chunks].transpose ] - - assert_equal 5, db.map.size - assert_equal 0, db.queue.size - - db.clear! - assert_equal 5, db.map.size - assert_equal 0, db.queue.size - - keys.each do |k| - db.push(k) - end - assert_equal 0, db.map.size - assert_equal 5, db.queue.size - - db.clear! - assert_equal 0, db.map.size - assert_equal 0, db.queue.size - - assert chunks.reduce(true){|a,b| a && b.purged } - end - - sub_test_case 'buffer_queue_full_action' do - def dummy_buffer(action) - db = DummyBuffer.new - db.configure( - "buffer_chunk_limit" => 1024, - "buffer_queue_limit" => 1, - "buffer_queue_full_action" => action - ) - db.start - db - end - - def data - @data ||= "a" * 1024 - end - - def chain - @chain ||= DummyChain.new - end - - def test_emit_with_exception - db = dummy_buffer(:exception) - - assert !db.emit('key', data, chain) - assert db.emit('key', data, chain) - - assert_raise(Fluent::BufferQueueLimitError) { - assert db.emit('key', data, chain) - } - assert db.queue.size == 1 - - assert !pop_chunk(db) - assert db.queue.size == 0 - - # queue is now empty so can emit data again - assert db.emit('key', data, chain) - end - - def test_emit_with_block - db = dummy_buffer(:block) - - assert !db.emit('key', data, chain) - assert db.emit('key', data, chain) - - begin - # with block, emit events to full queue causes sleep loop - Timeout.timeout(1) { - assert db.emit('key', data, chain) - } - flunk("timeout must happen") - rescue Timeout::Error - end - assert db.queue.size == 1 - - assert !pop_chunk(db) - assert db.queue.size == 0 - - # queue is now empty so can emit data again - assert db.emit('key', data, chain) - end - - def pop_chunk(db) - out = DummyOutput.new - c1 = DummyChunk.new('k1', 1) - - pop_return_value = nil - c1.synchronize do - pop_return_value = Thread.new { - db.pop(out) - }.value - end - pop_return_value - end - end - end -end From 7ee586c60551695e2b6a422addba2c0cca0cb645 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Mon, 16 May 2016 14:18:51 +0900 Subject: [PATCH 20/29] rename new buffer plugin implementations as default one --- lib/fluent/plugin/{buf_file2.rb => buf_file.rb} | 0 lib/fluent/plugin/{buf_memory2.rb => buf_memory.rb} | 0 test/plugin/{test_buf_file2.rb => test_buf_file.rb} | 0 test/plugin/{test_buf_memory2.rb => test_buf_memory.rb} | 0 4 files changed, 0 insertions(+), 0 deletions(-) rename lib/fluent/plugin/{buf_file2.rb => buf_file.rb} (100%) rename lib/fluent/plugin/{buf_memory2.rb => buf_memory.rb} (100%) rename test/plugin/{test_buf_file2.rb => test_buf_file.rb} (100%) rename test/plugin/{test_buf_memory2.rb => test_buf_memory.rb} (100%) diff --git a/lib/fluent/plugin/buf_file2.rb b/lib/fluent/plugin/buf_file.rb similarity index 100% rename from lib/fluent/plugin/buf_file2.rb rename to lib/fluent/plugin/buf_file.rb diff --git a/lib/fluent/plugin/buf_memory2.rb b/lib/fluent/plugin/buf_memory.rb similarity index 100% rename from lib/fluent/plugin/buf_memory2.rb rename to lib/fluent/plugin/buf_memory.rb diff --git a/test/plugin/test_buf_file2.rb b/test/plugin/test_buf_file.rb similarity index 100% rename from test/plugin/test_buf_file2.rb rename to test/plugin/test_buf_file.rb diff --git a/test/plugin/test_buf_memory2.rb b/test/plugin/test_buf_memory.rb similarity index 100% rename from test/plugin/test_buf_memory2.rb rename to test/plugin/test_buf_memory.rb From b6c8d04869e9c21e29a340cd93ba4627c80f8c0c Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Mon, 16 May 2016 15:20:09 +0900 Subject: [PATCH 21/29] rename memory2/file2 to memory/file and fix to use new error class for buffer overflow --- lib/fluent/compat/output.rb | 6 +++--- lib/fluent/load.rb | 1 - lib/fluent/plugin/buf_file.rb | 2 +- lib/fluent/plugin/buf_memory.rb | 2 +- lib/fluent/plugin/buffer.rb | 2 +- lib/fluent/plugin/in_tail.rb | 5 +++-- lib/fluent/plugin/output.rb | 2 +- lib/fluent/test/input_test.rb | 8 -------- test/plugin/test_buf_file.rb | 2 +- test/plugin/test_buf_memory.rb | 2 +- test/plugin/test_in_tail.rb | 3 ++- test/plugin/test_out_file.rb | 1 - 12 files changed, 14 insertions(+), 22 deletions(-) diff --git a/lib/fluent/compat/output.rb b/lib/fluent/compat/output.rb index ff8fba9e7d..5262dde3d1 100644 --- a/lib/fluent/compat/output.rb +++ b/lib/fluent/compat/output.rb @@ -466,7 +466,7 @@ def support_in_v12_style?(feature) attr_accessor :localtime config_section :buffer, param_name: :buffer_config do - config_set_default :@type, 'file2' + config_set_default :@type, 'file' end PARAMS_MAP = { @@ -502,7 +502,7 @@ def configure(conf) config_style = (bufconf ? :v1 : :v0) if config_style == :v0 buf_params = { - "@type" => "file2", + "@type" => "file", "flush_mode" => (conf['flush_interval'] ? "fast" : "none"), "retry_type" => "exponential_backoff", } @@ -510,7 +510,7 @@ def configure(conf) buf_params[newer] = conf[older] if conf.has_key?(older) end unless buf_params.has_key?("@type") - buf_params["@type"] = "file2" + buf_params["@type"] = "file" end if conf['timezone'] diff --git a/lib/fluent/load.rb b/lib/fluent/load.rb index 200f09c8ec..e365a15877 100644 --- a/lib/fluent/load.rb +++ b/lib/fluent/load.rb @@ -30,7 +30,6 @@ require 'fluent/parser' require 'fluent/formatter' require 'fluent/event' -require 'fluent/buffer' require 'fluent/input' require 'fluent/output' require 'fluent/filter' diff --git a/lib/fluent/plugin/buf_file.rb b/lib/fluent/plugin/buf_file.rb index 9de0b90628..4eaa0fb3fa 100644 --- a/lib/fluent/plugin/buf_file.rb +++ b/lib/fluent/plugin/buf_file.rb @@ -23,7 +23,7 @@ module Fluent module Plugin class FileBuffer < Fluent::Plugin::Buffer - Plugin.register_buffer('file2', self) + Plugin.register_buffer('file', self) include SystemConfig::Mixin diff --git a/lib/fluent/plugin/buf_memory.rb b/lib/fluent/plugin/buf_memory.rb index ceda281b9a..59f11c535c 100644 --- a/lib/fluent/plugin/buf_memory.rb +++ b/lib/fluent/plugin/buf_memory.rb @@ -20,7 +20,7 @@ module Fluent module Plugin class MemoryBuffer < Fluent::Plugin::Buffer - Plugin.register_buffer('memory2', self) + Plugin.register_buffer('memory', self) def resume return {}, [] diff --git a/lib/fluent/plugin/buffer.rb b/lib/fluent/plugin/buffer.rb index 47c0c11106..a6bc56cee0 100644 --- a/lib/fluent/plugin/buffer.rb +++ b/lib/fluent/plugin/buffer.rb @@ -176,7 +176,7 @@ def metadata(timekey: nil, tag: nil, variables: nil) # metadata_and_data MUST be a hash of { metadata => data } def write(metadata_and_data, bulk: false, enqueue: false) return if metadata_and_data.size < 1 - raise BufferOverflowError unless storable? + raise BufferOverflowError, "buffer space has too many data" unless storable? staged_bytesize = 0 operated_chunks = [] diff --git a/lib/fluent/plugin/in_tail.rb b/lib/fluent/plugin/in_tail.rb index 92bc3ca478..632a25af47 100644 --- a/lib/fluent/plugin/in_tail.rb +++ b/lib/fluent/plugin/in_tail.rb @@ -20,6 +20,7 @@ require 'fluent/config/error' require 'fluent/event' require 'fluent/system_config' +require 'fluent/plugin/buffer' if Fluent.windows? require_relative 'file_wrapper' @@ -270,7 +271,7 @@ def run log.error_backtrace end - # @return true if no error or unrecoverable error happens in emit action. false if got BufferQueueLimitError + # @return true if no error or unrecoverable error happens in emit action. false if got BufferOverflowError def receive_lines(lines, tail_watcher) es = @receive_handler.call(lines, tail_watcher) unless es.empty? @@ -281,7 +282,7 @@ def receive_lines(lines, tail_watcher) end begin router.emit_stream(tag, es) - rescue BufferQueueLimitError + rescue Fluent::Plugin::Buffer::BufferOverflowError return false rescue # ignore non BufferQueueLimitError errors because in_tail can't recover. Engine shows logs and backtraces. diff --git a/lib/fluent/plugin/output.rb b/lib/fluent/plugin/output.rb index 687396f449..78f193edf2 100644 --- a/lib/fluent/plugin/output.rb +++ b/lib/fluent/plugin/output.rb @@ -42,7 +42,7 @@ class Output < Base # `` and `` sections are available only when '#format' and '#write' are implemented config_section :buffer, param_name: :buffer_config, init: true, required: false, multi: false, final: true do config_argument :chunk_keys, :array, value_type: :string, default: [] - config_param :@type, :string, default: 'memory2' + config_param :@type, :string, default: 'memory' config_param :timekey_range, :time, default: nil # range size to be used: `time.to_i / @timekey_range` config_param :timekey_wait, :time, default: 600 diff --git a/lib/fluent/test/input_test.rb b/lib/fluent/test/input_test.rb index ecbccc6f9f..98420b8a86 100644 --- a/lib/fluent/test/input_test.rb +++ b/lib/fluent/test/input_test.rb @@ -14,22 +14,14 @@ # limitations under the License. # -require 'fluent/buffer' require 'fluent/engine' require 'fluent/time' require 'fluent/test/base' module Fluent - class FileBuffer < BasicBuffer - def self.clear_buffer_paths - @@buffer_paths = {} - end - end - module Test class InputTestDriver < TestDriver def initialize(klass, &block) - FileBuffer.clear_buffer_paths super(klass, &block) @emit_streams = [] @expects = nil diff --git a/test/plugin/test_buf_file.rb b/test/plugin/test_buf_file.rb index 493a18a391..7dccd30238 100644 --- a/test/plugin/test_buf_file.rb +++ b/test/plugin/test_buf_file.rb @@ -1,5 +1,5 @@ require_relative '../helper' -require 'fluent/plugin/buf_file2' +require 'fluent/plugin/buf_file' require 'fluent/plugin/output' require 'fluent/unique_id' require 'fluent/system_config' diff --git a/test/plugin/test_buf_memory.rb b/test/plugin/test_buf_memory.rb index 267c33182a..4e1b2e211b 100644 --- a/test/plugin/test_buf_memory.rb +++ b/test/plugin/test_buf_memory.rb @@ -1,5 +1,5 @@ require_relative '../helper' -require 'fluent/plugin/buf_memory2' +require 'fluent/plugin/buf_memory' require 'fluent/plugin/output' require 'flexmock/test_unit' diff --git a/test/plugin/test_in_tail.rb b/test/plugin/test_in_tail.rb index a0c1b6a651..385ead6f16 100644 --- a/test/plugin/test_in_tail.rb +++ b/test/plugin/test_in_tail.rb @@ -1,6 +1,7 @@ require_relative '../helper' require 'fluent/test' require 'fluent/plugin/in_tail' +require 'fluent/plugin/buffer' require 'fluent/system_config' require 'net/http' require 'flexmock/test_unit' @@ -770,7 +771,7 @@ def test_missing_file sub_test_case 'emit error cases' do def test_emit_error_with_buffer_queue_limit_error - emits = execute_test(::Fluent::BufferQueueLimitError, "queue size exceeds limit") + emits = execute_test(Fluent::Plugin::Buffer::BufferOverflowError, "buffer space has too many data") assert_equal(10, emits.length) 10.times { |i| assert_equal({"message" => "test#{i}"}, emits[i][2]) diff --git a/test/plugin/test_out_file.rb b/test/plugin/test_out_file.rb index 0fe34e8975..326dc18a1b 100644 --- a/test/plugin/test_out_file.rb +++ b/test/plugin/test_out_file.rb @@ -300,7 +300,6 @@ def test_write_with_symlink ] symlink_path = "#{SYMLINK_PATH}" - Fluent::FileBuffer.clear_buffer_paths d = Fluent::Test::TestDriver.new(Fluent::FileOutput).configure(conf) begin From 05decc174289a90fe47c123a64a0288956f12ffa Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Mon, 16 May 2016 18:25:01 +0900 Subject: [PATCH 22/29] delete unused class definition --- lib/fluent/match.rb | 36 ------------------------------------ 1 file changed, 36 deletions(-) diff --git a/lib/fluent/match.rb b/lib/fluent/match.rb index 1f08e5eeb1..e498779d33 100644 --- a/lib/fluent/match.rb +++ b/lib/fluent/match.rb @@ -15,42 +15,6 @@ # module Fluent - class Match - def initialize(pattern_str, output) - patterns = pattern_str.split(/\s+/).map {|str| - MatchPattern.create(str) - } - if patterns.length == 1 - @pattern = patterns[0] - else - @pattern = OrMatchPattern.new(patterns) - end - @output = output - end - - attr_reader :output - - def emit(tag, es) - chain = NullOutputChain.instance - @output.emit(tag, es, chain) - end - - def start - @output.start - end - - def shutdown - @output.shutdown - end - - def match(tag) - if @pattern.match(tag) - return true - end - return false - end - end - class MatchPattern def self.create(str) if str == '**' From d8ecb1026c30f6f1e6a4e4c5b3f0a62d98782e59 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Mon, 16 May 2016 18:25:27 +0900 Subject: [PATCH 23/29] use #emit_events instead of #emit for plugins --- lib/fluent/process.rb | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/lib/fluent/process.rb b/lib/fluent/process.rb index d989a5c074..b28fe26616 100644 --- a/lib/fluent/process.rb +++ b/lib/fluent/process.rb @@ -42,7 +42,7 @@ def initialize def fork(delegate_object) ipr, ipw = IO.pipe # child Engine.emit_stream -> parent Engine.emit_stream - opr, opw = IO.pipe # parent target.emit -> child target.emit + opr, opw = IO.pipe # parent target.emit_events -> child target.emit_events pid = Process.fork if pid @@ -153,7 +153,7 @@ def output_forward_main(opr, target) read_event_stream(opr) {|tag,es| # FIXME error handling begin - target.emit(tag, es, NullOutputChain.instance) + target.emit_events(tag, es) rescue $log.warn "failed to emit", error: $!.to_s, pid: Process.pid $log.warn_backtrace @@ -365,7 +365,7 @@ def detach_process_impl(num, &block) } end - # override target.emit and write event stream to the pipe + # override target.emit_events and write event stream to the pipe forwarders = children.map {|pair| pair[1].forwarder } if forwarders.length > 1 # use roundrobin @@ -373,9 +373,8 @@ def detach_process_impl(num, &block) else fwd = forwarders[0] end - define_singleton_method(:emit) do |tag,es,chain| - chain.next - fwd.emit(tag, es) + define_singleton_method(:emit_events) do |tag,es| + fwd.emit_events(tag, es) end end From 46ae1419fd6ffeece37d4a93b46b8716a60ee6de Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Mon, 16 May 2016 18:26:00 +0900 Subject: [PATCH 24/29] fix but, not to set buffer_path to path in section --- lib/fluent/compat/output.rb | 2 ++ 1 file changed, 2 insertions(+) diff --git a/lib/fluent/compat/output.rb b/lib/fluent/compat/output.rb index 5262dde3d1..397c0ffb4b 100644 --- a/lib/fluent/compat/output.rb +++ b/lib/fluent/compat/output.rb @@ -210,6 +210,7 @@ def support_in_v12_style?(feature) PARAMS_MAP = { "buffer_type" => "@type", + "buffer_path" => "path", "num_threads" => "flush_threads", "flush_interval" => "flush_interval", "try_flush_interval" => "flush_thread_interval", @@ -345,6 +346,7 @@ def support_in_v12_style?(feature) PARAMS_MAP = { "buffer_type" => "@type", + "buffer_path" => "path", "num_threads" => "flush_threads", "flush_interval" => "flush_interval", "try_flush_interval" => "flush_thread_interval", From 18fa283aaa449efd41246c4156903108cb74a3a5 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Mon, 16 May 2016 18:26:43 +0900 Subject: [PATCH 25/29] fix to call self_chain.next recursively to call next plugin instance and return value --- lib/fluent/compat/output_chain.rb | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/lib/fluent/compat/output_chain.rb b/lib/fluent/compat/output_chain.rb index bd090a0aa2..3bdecfcc28 100644 --- a/lib/fluent/compat/output_chain.rb +++ b/lib/fluent/compat/output_chain.rb @@ -40,8 +40,8 @@ def next return @chain.next end @offset += 1 - result = @array[@offset-1].emit(@tag, @es, self) - result + @array[@offset-1].emit_events(@tag, @es) + self.next end end @@ -52,8 +52,8 @@ def next end @offset += 1 es = @array.length > @offset ? @es.dup : @es - result = @array[@offset-1].emit(@tag, es, self) - result + @array[@offset-1].emit_events(@tag, es) + self.next end end end From 0cd28449406c27907e0f14f388b23356d9f1a332 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Mon, 16 May 2016 18:27:00 +0900 Subject: [PATCH 26/29] add configuration example with buf_file --- example/out_forward_buf_file.conf | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) create mode 100644 example/out_forward_buf_file.conf diff --git a/example/out_forward_buf_file.conf b/example/out_forward_buf_file.conf new file mode 100644 index 0000000000..4b5fffdbb8 --- /dev/null +++ b/example/out_forward_buf_file.conf @@ -0,0 +1,18 @@ + + @type dummy + tag test + + + + @type forward + buffer_path /tmp/fluentd.forward + buffer_type file + flush_interval 5 + send_timeout 60 + heartbeat_type tcp + heartbeat_interval 1 + + host 127.0.0.1 + port 24224 + + From 3afb1fd339c9637d5093bb92f8c892ea79957b07 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Mon, 16 May 2016 18:43:03 +0900 Subject: [PATCH 27/29] fix to call super to initialize internal state --- test/scripts/fluent/plugin/out_test.rb | 3 +++ 1 file changed, 3 insertions(+) diff --git a/test/scripts/fluent/plugin/out_test.rb b/test/scripts/fluent/plugin/out_test.rb index ea90b08ebe..7a1cd75298 100644 --- a/test/scripts/fluent/plugin/out_test.rb +++ b/test/scripts/fluent/plugin/out_test.rb @@ -19,6 +19,7 @@ class TestOutput < Output Plugin.register_output('test', self) def initialize + super @emit_streams = [] @name = nil end @@ -60,9 +61,11 @@ def configure(conf) end def start + super end def shutdown + super end def emit(tag, es, chain) From dabce88088b7fbed7abbb92c595e8a85dfdbcd73 Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Tue, 17 May 2016 11:24:43 +0900 Subject: [PATCH 28/29] add configuration example to forward events from remote to remote --- example/in_out_forward.conf | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) create mode 100644 example/in_out_forward.conf diff --git a/example/in_out_forward.conf b/example/in_out_forward.conf new file mode 100644 index 0000000000..f96c2bd5b3 --- /dev/null +++ b/example/in_out_forward.conf @@ -0,0 +1,17 @@ + + @type forward + port 24224 + + + + @type forward + buffer_type file + buffer_path /tmp/fluentd.forward.buffer + num_threads 10 + flush_interval 1s + + host 127.0.0.1 + port 24225 + + + From eefaa15c6dbaf03b996e2be2ad7b2f089f9b1abb Mon Sep 17 00:00:00 2001 From: TAGOMORI Satoshi Date: Tue, 17 May 2016 11:25:41 +0900 Subject: [PATCH 29/29] add kwarg (to be ignored) to have same arguments between #to_msgpack_stream --- lib/fluent/event.rb | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/lib/fluent/event.rb b/lib/fluent/event.rb index 27d7dbec31..f4ac6974dd 100644 --- a/lib/fluent/event.rb +++ b/lib/fluent/event.rb @@ -179,7 +179,8 @@ def each(&block) nil end - def to_msgpack_stream + def to_msgpack_stream(time_int: false) + # time_int is always ignored because @data is always packed binary in this class @data end end @@ -196,7 +197,8 @@ def each(&block) end alias :msgpack_each :each - def to_msgpack_stream + def to_msgpack_stream(time_int: false) + # time_int is always ignored because data is already packed and written in chunk read end end