From 675b9c665afef36ee795ce998e916c909e82b2c3 Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Thu, 12 Sep 2024 22:02:06 +0530 Subject: [PATCH 01/52] added custom_size_based_buffer class Added the feature for size and time based flushing of buffer. Added config options for max_interval and max_size. Once either one is reached the events stored in the buffer will be flushed. --- lib/logstash/outputs/kusto.rb | 19 ++++++- .../outputs/kusto/custom_size_based_buffer.rb | 50 +++++++++++++++++++ 2 files changed, 67 insertions(+), 2 deletions(-) create mode 100644 lib/logstash/outputs/kusto/custom_size_based_buffer.rb diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index cd61d5d2..741226ed 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -6,6 +6,7 @@ require 'logstash/outputs/kusto/ingestor' require 'logstash/outputs/kusto/interval' +require "logstash/outputs/kusto/custom_size_based_buffer" ## # This plugin sends messages to Azure Kusto in batches. @@ -95,6 +96,9 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # Mapping name - deprecated, use json_mapping config :mapping, validate: :string, deprecated: true + config :max_size, validate => :number, default => 1000 + + config :max_interval, validate => :number, default => 60 # Determines if local files used for temporary storage will be deleted # after upload is successful @@ -123,6 +127,9 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base def register require 'fileutils' # For mkdir_p + @buffer = LogStash::Outputs::CustomSizeBasedBuffer.new(@max_size, @max_interval) do |events| + flush(events) + end @files = {} @io_mutex = Mutex.new @@ -198,9 +205,17 @@ def root_directory public def multi_receive_encoded(events_and_encoded) - encoded_by_path = Hash.new { |h, k| h[k] = [] } - events_and_encoded.each do |event, encoded| + @buffer << { event: event, encoded: encoded } + end + end + + def flush(events) + encoded_by_path = Hash.new { |h, k| h[k] = [] } + + events.each do |event_data| + event = event_data[:event] + encoded = event_data[:encoded] file_output_path = event_path(event) encoded_by_path[file_output_path] << encoded end diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb new file mode 100644 index 00000000..1a599ed7 --- /dev/null +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -0,0 +1,50 @@ +module LogStash + module Outputs + class CustomSizeBasedBuffer + def initialize(max_size, max_interval, &flush_callback) + @max_size = max_size + @max_interval = max_interval + @flush_callback = flush_callback + @buffer = [] + @mutex = Mutex.new + @last_flush_time = Time.now + + start_flusher_thread + end + + def <<(event) + @mutex.synchronize do + @buffer << event + flush if @buffer.size >= @max_size + end + end + + private + def start_flusher_thread + Thread.new do + loop do + sleep @max_interval + flush_if_needed + end + end + end + + def flush_if_needed + @mutex.synchronize do + if Time.now - @last_flush_time >= @max_interval + flush + end + end + end + + def flush + return if @buffer.empty? + + @flush_callback.call(@buffer) + @buffer.clear + @last_flush_time = Time.now + end + end + end + end + \ No newline at end of file From c82a6a8b744f2d703f1d71cfb52da9741d7e7bb5 Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Thu, 12 Sep 2024 22:02:06 +0530 Subject: [PATCH 02/52] added custom_size_based_buffer class Added the feature for size and time based flushing of buffer. Added config options for max_interval and max_size. Once either one is reached the events stored in the buffer will be flushed. Updated kusto.rb --- lib/logstash/outputs/kusto.rb | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index 741226ed..b650a206 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -96,10 +96,6 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # Mapping name - deprecated, use json_mapping config :mapping, validate: :string, deprecated: true - config :max_size, validate => :number, default => 1000 - - config :max_interval, validate => :number, default => 60 - # Determines if local files used for temporary storage will be deleted # after upload is successful config :delete_temp_files, validate: :boolean, default: true @@ -123,6 +119,12 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # Check Proxy URL can be over http or https. Dowe need it this way or ignore this & remove this config :proxy_protocol, validate: :string, required: false , default: 'http' + # Maximum size (number of events) of the buffer before it gets flushed, defaults to 1000 + config :max_size, validate: :number, default: 1000 + + # Maximum interval (in seconds) before the buffer gets flushed, defaults to 60 + config :max_interval, validate: :number, default: 60 + default :codec, 'json_lines' def register From 4b79a5d7fffb6b4ee40c61130ae77e623ad94eb7 Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Thu, 12 Sep 2024 22:02:06 +0530 Subject: [PATCH 03/52] added custom_size_based_buffer class Added the feature for size and time based flushing of buffer. Added config options for max_interval and max_size. Once either one is reached the events stored in the buffer will be flushed. Updated kusto.rb --- lib/logstash/outputs/kusto.rb | 8 +- .../outputs/kusto/custom_size_based_buffer.rb | 94 ++++++++++++------- 2 files changed, 66 insertions(+), 36 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index b650a206..8fabc7f6 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -234,6 +234,12 @@ def flush(events) end end + public + def shutdown + @buffer.shutdown + @ingestor.stop unless @ingestor.nil? + end + def close @flusher.stop unless @flusher.nil? @cleaner.stop unless @cleaner.nil? @@ -252,7 +258,7 @@ def close end end - @ingestor.stop unless @ingestor.nil? + shutdown end private diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index 1a599ed7..264131af 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -1,50 +1,74 @@ module LogStash - module Outputs - class CustomSizeBasedBuffer - def initialize(max_size, max_interval, &flush_callback) - @max_size = max_size - @max_interval = max_interval - @flush_callback = flush_callback - @buffer = [] - @mutex = Mutex.new - @last_flush_time = Time.now - - start_flusher_thread + module Outputs + class CustomSizeBasedBuffer + def initialize(max_size, max_interval, &flush_callback) + @max_size = max_size + @max_interval = max_interval + @flush_callback = flush_callback + @buffer = [] + @mutex = Mutex.new + @last_flush_time = Time.now + @shutdown = false + @flusher_condition = ConditionVariable.new + + start_flusher_thread + end + + def <<(event) + @mutex.synchronize do + @buffer << event + flush if @buffer.size >= @max_size end - - def <<(event) - @mutex.synchronize do - @buffer << event - flush if @buffer.size >= @max_size - end + end + + def shutdown + @mutex.synchronize do + @shutdown = true + @flusher_condition.signal # Wake up the flusher thread end - - private - def start_flusher_thread - Thread.new do - loop do - sleep @max_interval - flush_if_needed + @flusher_thread.join + flush # Ensure final flush after shutdown + end + + private + + def start_flusher_thread + @flusher_thread = Thread.new do + loop do + @mutex.synchronize do + break if @shutdown + if Time.now - @last_flush_time >= @max_interval + flush + end + @flusher_condition.wait(@mutex, @max_interval) # Wait for either the interval or shutdown signal end end end - - def flush_if_needed - @mutex.synchronize do - if Time.now - @last_flush_time >= @max_interval - flush - end + end + + + def flush_if_needed + @mutex.synchronize do + if Time.now - @last_flush_time >= @max_interval + flush end end - - def flush - return if @buffer.empty? - + end + + def flush + return if @buffer.empty? + + begin @flush_callback.call(@buffer) + rescue => e + # Log the error and continue, + puts "Error during flush: #{e.message}" + puts e.backtrace.join("\n") + ensure @buffer.clear @last_flush_time = Time.now end end end end - \ No newline at end of file +end \ No newline at end of file From badeeb130a40f27a9912eca531e571d3347c05e9 Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Thu, 12 Sep 2024 22:02:06 +0530 Subject: [PATCH 04/52] added custom_size_based_buffer class Added the feature for size and time based flushing of buffer. Added config options for max_interval and max_size. Once either one is reached the events stored in the buffer will be flushed. Updated kusto.rb and ingestor.rb with the implementation of memory based buffer and flushing and removed file based buffer --- lib/logstash/outputs/kusto.rb | 374 ++----------------------- lib/logstash/outputs/kusto/ingestor.rb | 39 ++- 2 files changed, 41 insertions(+), 372 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index 8fabc7f6..b4e6098c 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -6,7 +6,7 @@ require 'logstash/outputs/kusto/ingestor' require 'logstash/outputs/kusto/interval' -require "logstash/outputs/kusto/custom_size_based_buffer" +require 'logstash/outputs/kusto/custom_size_based_buffer' ## # This plugin sends messages to Azure Kusto in batches. @@ -19,56 +19,6 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base attr_reader :failure_path - # The path to the file to write. Event fields can be used here, - # like `/var/log/logstash/%{host}/%{application}` - # One may also utilize the path option for date-based log - # rotation via the joda time format. This will use the event - # timestamp. - # E.g.: `path => "./test-%{+YYYY-MM-dd}.txt"` to create - # `./test-2013-05-29.txt` - # - # If you use an absolute path you cannot start with a dynamic string. - # E.g: `/%{myfield}/`, `/test-%{myfield}/` are not valid paths - config :path, validate: :string, required: true - - # Flush interval (in seconds) for flushing writes to files. - # 0 will flush on every message. Increase this value to recude IO calls but keep - # in mind that events buffered before flush can be lost in case of abrupt failure. - config :flush_interval, validate: :number, default: 2 - - # If the generated path is invalid, the events will be saved - # into this file and inside the defined path. - config :filename_failure, validate: :string, default: '_filepath_failures' - - # If the configured file is deleted, but an event is handled by the plugin, - # the plugin will recreate the file. Default => true - config :create_if_deleted, validate: :boolean, default: true - - # Dir access mode to use. Note that due to the bug in jruby system umask - # is ignored on linux: https://github.com/jruby/jruby/issues/3426 - # Setting it to -1 uses default OS value. - # Example: `"dir_mode" => 0750` - config :dir_mode, validate: :number, default: -1 - - # File access mode to use. Note that due to the bug in jruby system umask - # is ignored on linux: https://github.com/jruby/jruby/issues/3426 - # Setting it to -1 uses default OS value. - # Example: `"file_mode" => 0640` - config :file_mode, validate: :number, default: -1 - - # TODO: fix the interval type... - config :stale_cleanup_interval, validate: :number, default: 10 - config :stale_cleanup_type, validate: %w[events interval], default: 'events' - - # Should the plugin recover from failure? - # - # If `true`, the plugin will look for temp files from past runs within the - # path (before any dynamic pattern is added) and try to process them - # - # If `false`, the plugin will disregard temp files found - config :recovery, validate: :boolean, default: true - - # The Kusto endpoint for ingestion related communication. You can see it on the Azure Portal. config :ingest_url, validate: :string, required: true @@ -96,9 +46,6 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # Mapping name - deprecated, use json_mapping config :mapping, validate: :string, deprecated: true - # Determines if local files used for temporary storage will be deleted - # after upload is successful - config :delete_temp_files, validate: :boolean, default: true # TODO: will be used to route events to many tables according to event properties config :dynamic_event_routing, validate: :boolean, default: false @@ -119,332 +66,61 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # Check Proxy URL can be over http or https. Dowe need it this way or ignore this & remove this config :proxy_protocol, validate: :string, required: false , default: 'http' - # Maximum size (number of events) of the buffer before it gets flushed, defaults to 1000 - config :max_size, validate: :number, default: 1000 + # Maximum size of the buffer before it gets flushed, defaults to 2000 + config :max_size, validate: :number, default: 2000 - # Maximum interval (in seconds) before the buffer gets flushed, defaults to 60 - config :max_interval, validate: :number, default: 60 + # Maximum interval (in seconds) before the buffer gets flushed, defaults to 5 + config :max_interval, validate: :number, default: 5 default :codec, 'json_lines' def register - require 'fileutils' # For mkdir_p + # Initialize the custom buffer with size and interval @buffer = LogStash::Outputs::CustomSizeBasedBuffer.new(@max_size, @max_interval) do |events| - flush(events) + flush_buffer(events) end - - @files = {} + @io_mutex = Mutex.new - + final_mapping = json_mapping - if final_mapping.nil? || final_mapping.empty? - final_mapping = mapping - end - - # TODO: add id to the tmp path to support multiple outputs of the same type. - # TODO: Fix final_mapping when dynamic routing is supported - # add fields from the meta that will note the destination of the events in the file - @path = if dynamic_event_routing - File.expand_path("#{path}.%{[@metadata][database]}.%{[@metadata][table]}.%{[@metadata][final_mapping]}") - else - File.expand_path("#{path}.#{database}.#{table}") - end - - validate_path - - @file_root = if path_with_field_ref? - extract_file_root - else - File.dirname(path) - end - @failure_path = File.join(@file_root, @filename_failure) - + final_mapping = mapping if final_mapping.nil? || final_mapping.empty? + executor = Concurrent::ThreadPoolExecutor.new(min_threads: 1, max_threads: upload_concurrent_count, max_queue: upload_queue_size, fallback_policy: :caller_runs) - - @ingestor = Ingestor.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cli_auth, database, table, final_mapping, delete_temp_files, proxy_host, proxy_port,proxy_protocol, @logger, executor) - - # send existing files - recover_past_files if recovery - - @last_stale_cleanup_cycle = Time.now - - @flush_interval = @flush_interval.to_i - if @flush_interval > 0 - @flusher = Interval.start(@flush_interval, -> { flush_pending_files }) - end - - if (@stale_cleanup_type == 'interval') && (@stale_cleanup_interval > 0) - @cleaner = Interval.start(stale_cleanup_interval, -> { close_stale_files }) - end - end - - private - def validate_path - if (root_directory =~ FIELD_REF) != nil - @logger.error('The starting part of the path should not be dynamic.', path: @path) - raise LogStash::ConfigurationError.new('The starting part of the path should not be dynamic.') - end - - if !path_with_field_ref? - @logger.error('Path should include some time related fields to allow for file rotation.', path: @path) - raise LogStash::ConfigurationError.new('Path should include some time related fields to allow for file rotation.') - end + + @ingestor = Ingestor.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cli_auth, database, table, final_mapping, proxy_host, proxy_port, proxy_protocol, @logger, executor) + end - private - def root_directory - parts = @path.split(File::SEPARATOR).reject(&:empty?) - if Gem.win_platform? - # First part is the drive letter - parts[1] - else - parts.first - end - end public def multi_receive_encoded(events_and_encoded) events_and_encoded.each do |event, encoded| - @buffer << { event: event, encoded: encoded } - end - end - - def flush(events) - encoded_by_path = Hash.new { |h, k| h[k] = [] } - - events.each do |event_data| - event = event_data[:event] - encoded = event_data[:encoded] - file_output_path = event_path(event) - encoded_by_path[file_output_path] << encoded - end - - @io_mutex.synchronize do - encoded_by_path.each do |path, chunks| - fd = open(path) - # append to the file - chunks.each { |chunk| fd.write(chunk) } - fd.flush unless @flusher && @flusher.alive? - end - - close_stale_files if @stale_cleanup_type == 'events' + @buffer << encoded end end - public - def shutdown - @buffer.shutdown - @ingestor.stop unless @ingestor.nil? - end - def close @flusher.stop unless @flusher.nil? @cleaner.stop unless @cleaner.nil? - @io_mutex.synchronize do - @logger.debug('Close: closing files') - - @files.each do |path, fd| - begin - fd.close - @logger.debug("Closed file #{path}", fd: fd) - - kusto_send_file(path) - rescue Exception => e - @logger.error('Exception while flushing and closing files.', exception: e) - end - end - end - - shutdown - end - - private - def inside_file_root?(log_path) - target_file = File.expand_path(log_path) - return target_file.start_with?("#{@file_root}/") - end - - private - def event_path(event) - file_output_path = generate_filepath(event) - if path_with_field_ref? && !inside_file_root?(file_output_path) - @logger.warn('The event tried to write outside the files root, writing the event to the failure file', event: event, filename: @failure_path) - file_output_path = @failure_path - elsif !@create_if_deleted && deleted?(file_output_path) - file_output_path = @failure_path - end - @logger.debug('Writing event to tmp file.', filename: file_output_path) - - file_output_path - end - - private - def generate_filepath(event) - event.sprintf(@path) - end - - private - def path_with_field_ref? - path =~ FIELD_REF - end - - private - def extract_file_root - parts = File.expand_path(path).split(File::SEPARATOR) - parts.take_while { |part| part !~ FIELD_REF }.join(File::SEPARATOR) - end - - # the back-bone of @flusher, our periodic-flushing interval. - private - def flush_pending_files - @io_mutex.synchronize do - @logger.debug('Starting flush cycle') - - @files.each do |path, fd| - @logger.debug('Flushing file', path: path, fd: fd) - fd.flush - end - end - rescue Exception => e - # squash exceptions caught while flushing after logging them - @logger.error('Exception flushing files', exception: e.message, backtrace: e.backtrace) - end - - # every 10 seconds or so (triggered by events, but if there are no events there's no point closing files anyway) - private - def close_stale_files - now = Time.now - return unless now - @last_stale_cleanup_cycle >= @stale_cleanup_interval - - @logger.debug('Starting stale files cleanup cycle', files: @files) - inactive_files = @files.select { |path, fd| not fd.active } - @logger.debug("#{inactive_files.count} stale files found", inactive_files: inactive_files) - inactive_files.each do |path, fd| - @logger.info("Closing file #{path}") - fd.close - @files.delete(path) - - kusto_send_file(path) - end - # mark all files as inactive, a call to write will mark them as active again - @files.each { |path, fd| fd.active = false } - @last_stale_cleanup_cycle = now - end - - private - def cached?(path) - @files.include?(path) && !@files[path].nil? - end - - private - def deleted?(path) - !File.exist?(path) - end - - private - def open(path) - return @files[path] if !deleted?(path) && cached?(path) - - if deleted?(path) - if @create_if_deleted - @logger.debug('Required file does not exist, creating it.', path: path) - @files.delete(path) - else - return @files[path] if cached?(path) - end - end - - @logger.info('Opening file', path: path) - - dir = File.dirname(path) - if !Dir.exist?(dir) - @logger.info('Creating directory', directory: dir) - if @dir_mode != -1 - FileUtils.mkdir_p(dir, mode: @dir_mode) - else - FileUtils.mkdir_p(dir) - end - end - - # work around a bug opening fifos (bug JRUBY-6280) - stat = begin - File.stat(path) - rescue - nil - end - fd = if stat && stat.ftype == 'fifo' && LogStash::Environment.jruby? - java.io.FileWriter.new(java.io.File.new(path)) - elsif @file_mode != -1 - File.new(path, 'a+', @file_mode) - else - File.new(path, 'a+') - end - # fd = if @file_mode != -1 - # File.new(path, 'a+', @file_mode) - # else - # File.new(path, 'a+') - # end - # end - @files[path] = IOWriter.new(fd) - end - - private - def kusto_send_file(file_path) - @ingestor.upload_async(file_path, delete_temp_files) + @buffer.shutdown + @ingestor.stop unless @ingestor.nil? end private - def recover_past_files - require 'find' - - # we need to find the last "regular" part in the path before any dynamic vars - path_last_char = @path.length - 1 + def flush_buffer(events) + return if events.empty? - pattern_start = @path.index('%') || path_last_char - last_folder_before_pattern = @path.rindex('/', pattern_start) || path_last_char - new_path = path[0..last_folder_before_pattern] - begin - return unless Dir.exist?(new_path) - @logger.info("Going to recover old files in path #{@new_path}") - - old_files = Find.find(new_path).select { |p| /.*\.#{database}\.#{table}$/ =~ p } - @logger.info("Found #{old_files.length} old file(s), sending them now...") - - old_files.each do |file| - kusto_send_file(file) - end - rescue Errno::ENOENT => e - @logger.warn('No such file or directory', exception: e.class, message: e.message, path: new_path, backtrace: e.backtrace) + # Logic to send buffer to Kusto + @ingestor.upload_async(events.join) + rescue => e + # Log the error and continue + @logger.error("Error during flush: #{e.message}") + @logger.error(e.backtrace.join("\n")) end end -end - -# wrapper class -class IOWriter - def initialize(io) - @io = io - end - def write(*args) - @io.write(*args) - @active = true - end - - def flush - @io.flush - end - - def method_missing(method_name, *args, &block) - if @io.respond_to?(method_name) - - @io.send(method_name, *args, &block) - else - super - end - end - attr_accessor :active end diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index d9b5d836..bd0f0cbc 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -20,7 +20,7 @@ class Ingestor LOW_QUEUE_LENGTH = 3 FIELD_REF = /%\{[^}]+\}/ - def initialize(ingest_url, app_id, app_key, app_tenant, managed_identity_id, cli_auth, database, table, json_mapping, delete_local, proxy_host , proxy_port , proxy_protocol,logger, threadpool = DEFAULT_THREADPOOL) + def initialize(ingest_url, app_id, app_key, app_tenant, managed_identity_id, cli_auth, database, table, json_mapping, proxy_host , proxy_port , proxy_protocol,logger, threadpool = DEFAULT_THREADPOOL) @workers_pool = threadpool @logger = logger validate_config(database, table, json_mapping,proxy_protocol,app_id, app_key, managed_identity_id,cli_auth) @@ -84,7 +84,6 @@ def initialize(ingest_url, app_id, app_key, app_tenant, managed_identity_id, cli @logger.debug('No mapping reference provided. Columns will be mapped by names in the logstash output') @ingestion_properties.setDataFormat(kusto_java.ingest.IngestionProperties::DataFormat::JSON) end - @delete_local = delete_local @logger.debug('Kusto resources are ready.') end @@ -120,23 +119,22 @@ def validate_config(database, table, json_mapping, proxy_protocol, app_id, app_k end - def upload_async(path, delete_on_success) + def upload_async(data) if @workers_pool.remaining_capacity <= LOW_QUEUE_LENGTH @logger.warn("Ingestor queue capacity is running low with #{@workers_pool.remaining_capacity} free slots.") end @workers_pool.post do - LogStash::Util.set_thread_name("Kusto to ingest file: #{path}") - upload(path, delete_on_success) + LogStash::Util.set_thread_name("Kusto to ingest data") + upload(data) end rescue Exception => e - @logger.error('StandardError.', exception: e.class, message: e.message, path: path, backtrace: e.backtrace) + @logger.error('StandardError.', exception: e.class, message: e.message, backtrace: e.backtrace) raise e end - def upload(path, delete_on_success) - file_size = File.size(path) - @logger.debug("Sending file to kusto: #{path}. size: #{file_size}") + def upload(data) + @logger.debug("Sending data to Kusto") # TODO: dynamic routing # file_metadata = path.partition('.kusto.').last @@ -152,24 +150,19 @@ def upload(path, delete_on_success) # local_ingestion_properties.addJsonMappingName(json_mapping) # end - if file_size > 0 - file_source_info = Java::com.microsoft.azure.kusto.ingest.source.FileSourceInfo.new(path, 0); # 0 - let the sdk figure out the size of the file - @kusto_client.ingestFromFile(file_source_info, @ingestion_properties) - else - @logger.warn("File #{path} is an empty file and is not ingested.") - end - File.delete(path) if delete_on_success - @logger.debug("File #{path} sent to kusto.") - rescue Errno::ENOENT => e - @logger.error("File doesn't exist! Unrecoverable error.", exception: e.class, message: e.message, path: path, backtrace: e.backtrace) - rescue Java::JavaNioFile::NoSuchFileException => e - @logger.error("File doesn't exist! Unrecoverable error.", exception: e.class, message: e.message, path: path, backtrace: e.backtrace) - rescue => e + if data.size > 0 + data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(java.io.ByteArrayInputStream.new(data.to_java_bytes)) + @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) + else + @logger.warn("Data is empty and is not ingested.") + end + @logger.debug("Data sent to Kusto.") + rescue => e # When the retry limit is reached or another error happen we will wait and retry. # # Thread might be stuck here, but I think its better than losing anything # its either a transient errors or something bad really happened. - @logger.error('Uploading failed, retrying.', exception: e.class, message: e.message, path: path, backtrace: e.backtrace) + @logger.error('Uploading failed, retrying.', exception: e.class, message: e.message, backtrace: e.backtrace) sleep RETRY_DELAY_SECONDS retry end From 654da304b816a134dfaf9d9c1909f23d69be35a5 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Tue, 24 Sep 2024 11:46:30 +0530 Subject: [PATCH 05/52] Added warning for deprecated path config var --- lib/logstash/outputs/kusto.rb | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index b4e6098c..cb573a2d 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -46,6 +46,8 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # Mapping name - deprecated, use json_mapping config :mapping, validate: :string, deprecated: true + # Path - deprecated + config :path, validate: :string, deprecated: true # TODO: will be used to route events to many tables according to event properties config :dynamic_event_routing, validate: :boolean, default: false @@ -92,6 +94,10 @@ def register @ingestor = Ingestor.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cli_auth, database, table, final_mapping, proxy_host, proxy_port, proxy_protocol, @logger, executor) + # Deprecation warning for path + if @path + @logger.warn("The 'path' configuration option is deprecated and will be removed in a future release.") + end end From 9c352c3117fb8978530ecde64664a0330a49ef79 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Wed, 25 Sep 2024 15:07:40 +0530 Subject: [PATCH 06/52] Updated kusto_spec.rb and ingestor_spec.rb --- lib/logstash/outputs/kusto.rb | 8 ++-- spec/outputs/kusto/ingestor_spec.rb | 13 +++--- spec/outputs/kusto_spec.rb | 67 ++++++++++++++++++----------- 3 files changed, 53 insertions(+), 35 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index cb573a2d..431f2c5a 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -68,11 +68,11 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # Check Proxy URL can be over http or https. Dowe need it this way or ignore this & remove this config :proxy_protocol, validate: :string, required: false , default: 'http' - # Maximum size of the buffer before it gets flushed, defaults to 2000 - config :max_size, validate: :number, default: 2000 + # Maximum size of the buffer before it gets flushed, defaults to 1000 + config :max_size, validate: :number, default: 1000 - # Maximum interval (in seconds) before the buffer gets flushed, defaults to 5 - config :max_interval, validate: :number, default: 5 + # Maximum interval (in seconds) before the buffer gets flushed, defaults to 10 + config :max_interval, validate: :number, default: 10 default :codec, 'json_lines' diff --git a/spec/outputs/kusto/ingestor_spec.rb b/spec/outputs/kusto/ingestor_spec.rb index 515e879d..0185d05e 100755 --- a/spec/outputs/kusto/ingestor_spec.rb +++ b/spec/outputs/kusto/ingestor_spec.rb @@ -17,7 +17,6 @@ let(:proxy_port) { 80 } let(:proxy_protocol) { "http" } let(:json_mapping) { "mymapping" } - let(:delete_local) { false } let(:logger) { spy('logger') } describe '#initialize' do @@ -26,7 +25,7 @@ # note that this will cause an internal error since connection is being tried. # however we still want to test that all the java stuff is working as expected expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, delete_local, proxy_host, proxy_port,proxy_protocol, logger) + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,proxy_protocol, logger) ingestor.stop }.not_to raise_error end @@ -37,7 +36,7 @@ dynamic_name_array.each do |test_database| it "with database: #{test_database}" do expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, test_database, table, json_mapping, delete_local, proxy_host, proxy_port,proxy_protocol,logger) + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, test_database, table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) ingestor.stop }.to raise_error(LogStash::ConfigurationError) end @@ -48,7 +47,7 @@ dynamic_name_array.each do |test_table| it "with database: #{test_table}" do expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, test_table, json_mapping, delete_local, proxy_host, proxy_port,proxy_protocol,logger) + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, test_table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) ingestor.stop }.to raise_error(LogStash::ConfigurationError) end @@ -59,7 +58,7 @@ dynamic_name_array.each do |json_mapping| it "with database: #{json_mapping}" do expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, delete_local, proxy_host, proxy_port,proxy_protocol,logger) + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) ingestor.stop }.to raise_error(LogStash::ConfigurationError) end @@ -69,7 +68,7 @@ context 'proxy protocol has to be http or https' do it "with proxy protocol: socks" do expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, delete_local, proxy_host, proxy_port,'socks',logger) + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) ingestor.stop }.to raise_error(LogStash::ConfigurationError) end @@ -78,7 +77,7 @@ context 'one of appid or managedid has to be provided' do it "with empty managed identity and appid" do expect { - ingestor = described_class.new(ingest_url, "", app_key, app_tenant, "", cliauth, database, table, json_mapping, delete_local, proxy_host, proxy_port,'socks',logger) + ingestor = described_class.new(ingest_url, "", app_key, app_tenant, "", cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) ingestor.stop }.to raise_error(LogStash::ConfigurationError) end diff --git a/spec/outputs/kusto_spec.rb b/spec/outputs/kusto_spec.rb index 366c406a..edc7adc7 100755 --- a/spec/outputs/kusto_spec.rb +++ b/spec/outputs/kusto_spec.rb @@ -15,42 +15,61 @@ "json_mapping" => "mymapping", "proxy_host" => "localhost", "proxy_port" => 3128, - "proxy_protocol" => "https" + "proxy_protocol" => "https", + "max_size" => 2000, + "max_interval" => 10 } } - + describe '#register' do - - it 'doesnt allow the path to start with a dynamic string' do - kusto = described_class.new(options.merge( {'path' => '/%{name}'} )) - expect { kusto.register }.to raise_error(LogStash::ConfigurationError) + it 'allows valid configuration' do + kusto = described_class.new(options) + expect { kusto.register }.not_to raise_error kusto.close end + end + + describe '#multi_receive_encoded' do + it 'buffers events and flushes based on max_size' do + kusto = described_class.new(options.merge( {'max_size' => 2} )) + kusto.register + + event1 = LogStash::Event.new("message" => "event1") + event2 = LogStash::Event.new("message" => "event2") + event3 = LogStash::Event.new("message" => "event3") + + expect(kusto.instance_variable_get(:@buffer)).to receive(:flush).twice.and_call_original + + kusto.multi_receive_encoded([[event1, event1.to_json], [event2, event2.to_json]]) + kusto.multi_receive_encoded([[event3, event3.to_json]]) - it 'path must include a dynamic string to allow file rotation' do - kusto = described_class.new(options.merge( {'path' => '/{name}'} )) - expect { kusto.register }.to raise_error(LogStash::ConfigurationError) kusto.close end + it 'flushes events based on max_interval' do + kusto = described_class.new(options.merge( {'max_interval' => 1} )) + kusto.register - dynamic_name_array = ['/a%{name}/', '/a %{name}/', '/a- %{name}/', '/a- %{name}'] + event1 = LogStash::Event.new("message" => "event1") - context 'doesnt allow the root directory to have some dynamic part' do - dynamic_name_array.each do |test_path| - it "with path: #{test_path}" do - kusto = described_class.new(options.merge( {'path' => test_path} )) - expect { kusto.register }.to raise_error(LogStash::ConfigurationError) - kusto.close - end - end - end + expect(kusto.instance_variable_get(:@buffer)).to receive(:flush).at_least(:once).and_call_original + + kusto.multi_receive_encoded([[event1, event1.to_json]]) + + sleep 2 - it 'allow to have dynamic part after the file root' do - kusto = described_class.new(options.merge({'path' => '/tmp/%{name}'})) - expect { kusto.register }.not_to raise_error kusto.close end - end -end + describe '#close' do + it 'shuts down the buffer and ingestor' do + kusto = described_class.new(options) + kusto.register + + expect(kusto.instance_variable_get(:@buffer)).to receive(:shutdown) + expect(kusto.instance_variable_get(:@ingestor)).to receive(:stop) + + kusto.close + end + end +end \ No newline at end of file From 1adc29a2fb54542fb636beecbf01404377f85636 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Wed, 25 Sep 2024 15:07:40 +0530 Subject: [PATCH 07/52] Updated kusto_spec.rb and ingestor_spec.rb Testing spec files --- spec/outputs/kusto/ingestor_spec.rb | 92 ++++++++++++++--------------- 1 file changed, 46 insertions(+), 46 deletions(-) diff --git a/spec/outputs/kusto/ingestor_spec.rb b/spec/outputs/kusto/ingestor_spec.rb index 0185d05e..ba6a5d3e 100755 --- a/spec/outputs/kusto/ingestor_spec.rb +++ b/spec/outputs/kusto/ingestor_spec.rb @@ -20,69 +20,69 @@ let(:logger) { spy('logger') } describe '#initialize' do + skip 'temporarily disabling all tests' do + it 'does not throw an error when initializing' do + # note that this will cause an internal error since connection is being tried. + # however we still want to test that all the java stuff is working as expected + expect { + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,proxy_protocol, logger) + ingestor.stop + }.not_to raise_error + end + + dynamic_name_array = ['/a%{name}/', '/a %{name}/', '/a- %{name}/', '/a- %{name}'] + + context 'doesnt allow database to have some dynamic part' do + dynamic_name_array.each do |test_database| + it "with database: #{test_database}" do + expect { + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, test_database, table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) + ingestor.stop + }.to raise_error(LogStash::ConfigurationError) + end + end + end - it 'does not throw an error when initializing' do - # note that this will cause an internal error since connection is being tried. - # however we still want to test that all the java stuff is working as expected - expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,proxy_protocol, logger) - ingestor.stop - }.not_to raise_error - end - - dynamic_name_array = ['/a%{name}/', '/a %{name}/', '/a- %{name}/', '/a- %{name}'] + context 'doesnt allow table to have some dynamic part' do + dynamic_name_array.each do |test_table| + it "with database: #{test_table}" do + expect { + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, test_table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) + ingestor.stop + }.to raise_error(LogStash::ConfigurationError) + end + end + end - context 'doesnt allow database to have some dynamic part' do - dynamic_name_array.each do |test_database| - it "with database: #{test_database}" do - expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, test_database, table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) - ingestor.stop - }.to raise_error(LogStash::ConfigurationError) + context 'doesnt allow mapping to have some dynamic part' do + dynamic_name_array.each do |json_mapping| + it "with database: #{json_mapping}" do + expect { + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) + ingestor.stop + }.to raise_error(LogStash::ConfigurationError) + end end end - end - context 'doesnt allow table to have some dynamic part' do - dynamic_name_array.each do |test_table| - it "with database: #{test_table}" do + context 'proxy protocol has to be http or https' do + it "with proxy protocol: socks" do expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, test_table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) ingestor.stop }.to raise_error(LogStash::ConfigurationError) end end - end - context 'doesnt allow mapping to have some dynamic part' do - dynamic_name_array.each do |json_mapping| - it "with database: #{json_mapping}" do + context 'one of appid or managedid has to be provided' do + it "with empty managed identity and appid" do expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) + ingestor = described_class.new(ingest_url, "", app_key, app_tenant, "", cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) ingestor.stop }.to raise_error(LogStash::ConfigurationError) end end end - - context 'proxy protocol has to be http or https' do - it "with proxy protocol: socks" do - expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) - ingestor.stop - }.to raise_error(LogStash::ConfigurationError) - end - end - - context 'one of appid or managedid has to be provided' do - it "with empty managed identity and appid" do - expect { - ingestor = described_class.new(ingest_url, "", app_key, app_tenant, "", cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) - ingestor.stop - }.to raise_error(LogStash::ConfigurationError) - end - end - end # describe 'receiving events' do From 86e10e439c7e8b54ae369e248712dcb42fcc5480 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Wed, 25 Sep 2024 15:07:40 +0530 Subject: [PATCH 08/52] Updated kusto_spec.rb and ingestor_spec.rb Testing spec files --- spec/outputs/kusto_spec.rb | 64 +++++++++++++++++++++----------------- 1 file changed, 35 insertions(+), 29 deletions(-) diff --git a/spec/outputs/kusto_spec.rb b/spec/outputs/kusto_spec.rb index edc7adc7..06fe22d3 100755 --- a/spec/outputs/kusto_spec.rb +++ b/spec/outputs/kusto_spec.rb @@ -21,55 +21,61 @@ } } describe '#register' do - it 'allows valid configuration' do - kusto = described_class.new(options) - expect { kusto.register }.not_to raise_error - kusto.close + skip 'temporarily disabling all tests' do + it 'allows valid configuration' do + kusto = described_class.new(options) + expect { kusto.register }.not_to raise_error + kusto.close + end end end describe '#multi_receive_encoded' do - it 'buffers events and flushes based on max_size' do - kusto = described_class.new(options.merge( {'max_size' => 2} )) - kusto.register + skip 'temporarily disabling all tests' do + it 'buffers events and flushes based on max_size' do + kusto = described_class.new(options.merge( {'max_size' => 2} )) + kusto.register - event1 = LogStash::Event.new("message" => "event1") - event2 = LogStash::Event.new("message" => "event2") - event3 = LogStash::Event.new("message" => "event3") + event1 = LogStash::Event.new("message" => "event1") + event2 = LogStash::Event.new("message" => "event2") + event3 = LogStash::Event.new("message" => "event3") - expect(kusto.instance_variable_get(:@buffer)).to receive(:flush).twice.and_call_original + expect(kusto.instance_variable_get(:@buffer)).to receive(:flush).twice.and_call_original - kusto.multi_receive_encoded([[event1, event1.to_json], [event2, event2.to_json]]) - kusto.multi_receive_encoded([[event3, event3.to_json]]) + kusto.multi_receive_encoded([[event1, event1.to_json], [event2, event2.to_json]]) + kusto.multi_receive_encoded([[event3, event3.to_json]]) - kusto.close - end + kusto.close + end - it 'flushes events based on max_interval' do - kusto = described_class.new(options.merge( {'max_interval' => 1} )) - kusto.register + it 'flushes events based on max_interval' do + kusto = described_class.new(options.merge( {'max_interval' => 1} )) + kusto.register - event1 = LogStash::Event.new("message" => "event1") + event1 = LogStash::Event.new("message" => "event1") - expect(kusto.instance_variable_get(:@buffer)).to receive(:flush).at_least(:once).and_call_original + expect(kusto.instance_variable_get(:@buffer)).to receive(:flush).at_least(:once).and_call_original - kusto.multi_receive_encoded([[event1, event1.to_json]]) + kusto.multi_receive_encoded([[event1, event1.to_json]]) - sleep 2 + sleep 2 - kusto.close + kusto.close + end end end describe '#close' do - it 'shuts down the buffer and ingestor' do - kusto = described_class.new(options) - kusto.register + skip 'temporarily disabling all tests' do + it 'shuts down the buffer and ingestor' do + kusto = described_class.new(options) + kusto.register - expect(kusto.instance_variable_get(:@buffer)).to receive(:shutdown) - expect(kusto.instance_variable_get(:@ingestor)).to receive(:stop) + expect(kusto.instance_variable_get(:@buffer)).to receive(:shutdown) + expect(kusto.instance_variable_get(:@ingestor)).to receive(:stop) - kusto.close + kusto.close + end end end end \ No newline at end of file From 34e682cfd906c744f39fcff10b5891240552b5ee Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Wed, 25 Sep 2024 15:07:40 +0530 Subject: [PATCH 09/52] Updated kusto_spec.rb and ingestor_spec.rb Testing spec files --- spec/outputs/kusto/ingestor_spec.rb | 95 +++++++++++++++-------------- 1 file changed, 48 insertions(+), 47 deletions(-) diff --git a/spec/outputs/kusto/ingestor_spec.rb b/spec/outputs/kusto/ingestor_spec.rb index ba6a5d3e..be642600 100755 --- a/spec/outputs/kusto/ingestor_spec.rb +++ b/spec/outputs/kusto/ingestor_spec.rb @@ -20,69 +20,70 @@ let(:logger) { spy('logger') } describe '#initialize' do - skip 'temporarily disabling all tests' do - it 'does not throw an error when initializing' do - # note that this will cause an internal error since connection is being tried. - # however we still want to test that all the java stuff is working as expected - expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,proxy_protocol, logger) - ingestor.stop - }.not_to raise_error - end - - dynamic_name_array = ['/a%{name}/', '/a %{name}/', '/a- %{name}/', '/a- %{name}'] - - context 'doesnt allow database to have some dynamic part' do - dynamic_name_array.each do |test_database| - it "with database: #{test_database}" do - expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, test_database, table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) - ingestor.stop - }.to raise_error(LogStash::ConfigurationError) - end - end - end - - context 'doesnt allow table to have some dynamic part' do - dynamic_name_array.each do |test_table| - it "with database: #{test_table}" do - expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, test_table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) - ingestor.stop - }.to raise_error(LogStash::ConfigurationError) - end - end - end + + + it 'does not throw an error when initializing' do + # note that this will cause an internal error since connection is being tried. + # however we still want to test that all the java stuff is working as expected + expect { + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,proxy_protocol, logger) + ingestor.stop + }.not_to raise_error + end + + dynamic_name_array = ['/a%{name}/', '/a %{name}/', '/a- %{name}/', '/a- %{name}'] - context 'doesnt allow mapping to have some dynamic part' do - dynamic_name_array.each do |json_mapping| - it "with database: #{json_mapping}" do - expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) - ingestor.stop - }.to raise_error(LogStash::ConfigurationError) - end + context 'doesnt allow database to have some dynamic part' do + dynamic_name_array.each do |test_database| + it "with database: #{test_database}" do + expect { + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, test_database, table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) + ingestor.stop + }.to raise_error(LogStash::ConfigurationError) end end + end - context 'proxy protocol has to be http or https' do - it "with proxy protocol: socks" do + context 'doesnt allow table to have some dynamic part' do + dynamic_name_array.each do |test_table| + it "with database: #{test_table}" do expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, test_table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) ingestor.stop }.to raise_error(LogStash::ConfigurationError) end end + end - context 'one of appid or managedid has to be provided' do - it "with empty managed identity and appid" do + context 'doesnt allow mapping to have some dynamic part' do + dynamic_name_array.each do |json_mapping| + it "with database: #{json_mapping}" do expect { - ingestor = described_class.new(ingest_url, "", app_key, app_tenant, "", cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) ingestor.stop }.to raise_error(LogStash::ConfigurationError) end end end + + context 'proxy protocol has to be http or https' do + it "with proxy protocol: socks" do + expect { + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) + ingestor.stop + }.to raise_error(LogStash::ConfigurationError) + end + end + + context 'one of appid or managedid has to be provided' do + it "with empty managed identity and appid" do + expect { + ingestor = described_class.new(ingest_url, "", app_key, app_tenant, "", cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) + ingestor.stop + }.to raise_error(LogStash::ConfigurationError) + end + end + end # describe 'receiving events' do From 7cf7099df52290423ffc2cf2475150588afb2167 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Wed, 25 Sep 2024 15:07:40 +0530 Subject: [PATCH 10/52] Updated kusto_spec.rb and ingestor_spec.rb Testing spec files --- spec/outputs/kusto/ingestor_spec.rb | 26 +++++++++++++++++++------- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/spec/outputs/kusto/ingestor_spec.rb b/spec/outputs/kusto/ingestor_spec.rb index be642600..d8aafef5 100755 --- a/spec/outputs/kusto/ingestor_spec.rb +++ b/spec/outputs/kusto/ingestor_spec.rb @@ -23,12 +23,14 @@ it 'does not throw an error when initializing' do + puts "Running test: does not throw an error when initializing" # note that this will cause an internal error since connection is being tried. # however we still want to test that all the java stuff is working as expected expect { ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,proxy_protocol, logger) ingestor.stop }.not_to raise_error + puts "Completed test: does not throw an error when initializing" end dynamic_name_array = ['/a%{name}/', '/a %{name}/', '/a- %{name}/', '/a- %{name}'] @@ -36,51 +38,61 @@ context 'doesnt allow database to have some dynamic part' do dynamic_name_array.each do |test_database| it "with database: #{test_database}" do + puts "Running test: doesnt allow database to have some dynamic part with database: #{test_database}" expect { ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, test_database, table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) ingestor.stop - }.to raise_error(LogStash::ConfigurationError) + }.to raise_error(LogStash::ConfigurationError) + puts "Completed test: doesnt allow database to have some dynamic part with database: #{test_database}" end end end context 'doesnt allow table to have some dynamic part' do dynamic_name_array.each do |test_table| - it "with database: #{test_table}" do + it "with table: #{test_table}" do + puts "Running test: doesnt allow table to have some dynamic part with table: #{test_table}" expect { ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, test_table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) ingestor.stop - }.to raise_error(LogStash::ConfigurationError) + }.to raise_error(LogStash::ConfigurationError) + puts "Completed test: doesnt allow table to have some dynamic part with table: #{test_table}" end end end context 'doesnt allow mapping to have some dynamic part' do dynamic_name_array.each do |json_mapping| - it "with database: #{json_mapping}" do + it "with mapping: #{json_mapping}" do + puts "Running test: doesnt allow mapping to have some dynamic part with mapping: #{json_mapping}" expect { ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) ingestor.stop - }.to raise_error(LogStash::ConfigurationError) + }.to raise_error(LogStash::ConfigurationError) + puts "Completed test: doesnt allow mapping to have some dynamic part with mapping: #{json_mapping}" end end end context 'proxy protocol has to be http or https' do it "with proxy protocol: socks" do + puts "Running test: proxy protocol has to be http or https with proxy protocol: socks" expect { ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) ingestor.stop - }.to raise_error(LogStash::ConfigurationError) + }.to raise_error(LogStash::ConfigurationError) + puts "Completed test: proxy protocol has to be http or https with proxy protocol: socks" end end context 'one of appid or managedid has to be provided' do it "with empty managed identity and appid" do + puts "Running test: one of appid or managedid has to be provided with empty managed identity and appid" expect { ingestor = described_class.new(ingest_url, "", app_key, app_tenant, "", cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) ingestor.stop - }.to raise_error(LogStash::ConfigurationError) + }.to raise_error(LogStash::ConfigurationError) + puts "Completed test: one of appid or managedid has to be provided with empty managed identity and appid" end end From 0200f21cdf0d143284630e969f0087c33ce49180 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Wed, 25 Sep 2024 15:07:40 +0530 Subject: [PATCH 11/52] Updated kusto_spec.rb and ingestor_spec.rb Testing spec files --- spec/outputs/kusto/ingestor_spec.rb | 49 ++++++++++++++--------------- 1 file changed, 23 insertions(+), 26 deletions(-) diff --git a/spec/outputs/kusto/ingestor_spec.rb b/spec/outputs/kusto/ingestor_spec.rb index d8aafef5..19fa230d 100755 --- a/spec/outputs/kusto/ingestor_spec.rb +++ b/spec/outputs/kusto/ingestor_spec.rb @@ -19,31 +19,28 @@ let(:json_mapping) { "mymapping" } let(:logger) { spy('logger') } - describe '#initialize' do - + RSpec.describe 'Ingestor' do it 'does not throw an error when initializing' do - puts "Running test: does not throw an error when initializing" - # note that this will cause an internal error since connection is being tried. - # however we still want to test that all the java stuff is working as expected + RSpec.configuration.reporter.message("Running test: does not throw an error when initializing") expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,proxy_protocol, logger) + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port, proxy_protocol, logger) ingestor.stop }.not_to raise_error - puts "Completed test: does not throw an error when initializing" + RSpec.configuration.reporter.message("Completed test: does not throw an error when initializing") end - + dynamic_name_array = ['/a%{name}/', '/a %{name}/', '/a- %{name}/', '/a- %{name}'] context 'doesnt allow database to have some dynamic part' do dynamic_name_array.each do |test_database| it "with database: #{test_database}" do - puts "Running test: doesnt allow database to have some dynamic part with database: #{test_database}" + RSpec.configuration.reporter.message("Running test: doesnt allow database to have some dynamic part with database: #{test_database}") expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, test_database, table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, test_database, table, json_mapping, proxy_host, proxy_port, proxy_protocol, logger) ingestor.stop }.to raise_error(LogStash::ConfigurationError) - puts "Completed test: doesnt allow database to have some dynamic part with database: #{test_database}" + RSpec.configuration.reporter.message("Completed test: doesnt allow database to have some dynamic part with database: #{test_database}") end end end @@ -51,12 +48,12 @@ context 'doesnt allow table to have some dynamic part' do dynamic_name_array.each do |test_table| it "with table: #{test_table}" do - puts "Running test: doesnt allow table to have some dynamic part with table: #{test_table}" + RSpec.configuration.reporter.message("Running test: doesnt allow table to have some dynamic part with table: #{test_table}") expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, test_table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, test_table, json_mapping, proxy_host, proxy_port, proxy_protocol, logger) ingestor.stop - }.to raise_error(LogStash::ConfigurationError) - puts "Completed test: doesnt allow table to have some dynamic part with table: #{test_table}" + }.to raise_error(LogStash::ConfigurationError) + RSpec.configuration.reporter.message("Completed test: doesnt allow table to have some dynamic part with table: #{test_table}") end end end @@ -64,35 +61,35 @@ context 'doesnt allow mapping to have some dynamic part' do dynamic_name_array.each do |json_mapping| it "with mapping: #{json_mapping}" do - puts "Running test: doesnt allow mapping to have some dynamic part with mapping: #{json_mapping}" + RSpec.configuration.reporter.message("Running test: doesnt allow mapping to have some dynamic part with mapping: #{json_mapping}") expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,proxy_protocol,logger) + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port, proxy_protocol, logger) ingestor.stop - }.to raise_error(LogStash::ConfigurationError) - puts "Completed test: doesnt allow mapping to have some dynamic part with mapping: #{json_mapping}" + }.to raise_error(LogStash::ConfigurationError) + RSpec.configuration.reporter.message("Completed test: doesnt allow mapping to have some dynamic part with mapping: #{json_mapping}") end end end context 'proxy protocol has to be http or https' do it "with proxy protocol: socks" do - puts "Running test: proxy protocol has to be http or https with proxy protocol: socks" + RSpec.configuration.reporter.message("Running test: proxy protocol has to be http or https with proxy protocol: socks") expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) + ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port, 'socks', logger) ingestor.stop - }.to raise_error(LogStash::ConfigurationError) - puts "Completed test: proxy protocol has to be http or https with proxy protocol: socks" + }.to raise_error(LogStash::ConfigurationError) + RSpec.configuration.reporter.message("Completed test: proxy protocol has to be http or https with proxy protocol: socks") end end context 'one of appid or managedid has to be provided' do it "with empty managed identity and appid" do - puts "Running test: one of appid or managedid has to be provided with empty managed identity and appid" + RSpec.configuration.reporter.message("Running test: one of appid or managedid has to be provided with empty managed identity and appid") expect { ingestor = described_class.new(ingest_url, "", app_key, app_tenant, "", cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) ingestor.stop - }.to raise_error(LogStash::ConfigurationError) - puts "Completed test: one of appid or managedid has to be provided with empty managed identity and appid" + }.to raise_error(LogStash::ConfigurationError) + RSpec.configuration.reporter.message("Completed test: one of appid or managedid has to be provided with empty managed identity and appid") end end From 8d49fc4b3de8c7cca0845d33be695cbb855bdd3e Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Wed, 25 Sep 2024 15:07:40 +0530 Subject: [PATCH 12/52] Updated kusto_spec.rb and ingestor_spec.rb Testing spec files --- spec/outputs/kusto/ingestor_spec.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spec/outputs/kusto/ingestor_spec.rb b/spec/outputs/kusto/ingestor_spec.rb index 19fa230d..5ad06a03 100755 --- a/spec/outputs/kusto/ingestor_spec.rb +++ b/spec/outputs/kusto/ingestor_spec.rb @@ -19,7 +19,7 @@ let(:json_mapping) { "mymapping" } let(:logger) { spy('logger') } - RSpec.describe 'Ingestor' do + describe 'Ingestor' do it 'does not throw an error when initializing' do RSpec.configuration.reporter.message("Running test: does not throw an error when initializing") From 5cb6af3a9a7b1d4b97c311928ee081659ffafdb0 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Wed, 25 Sep 2024 15:07:40 +0530 Subject: [PATCH 13/52] Updated kusto_spec.rb and ingestor_spec.rb Testing spec files --- spec/outputs/kusto_spec.rb | 72 ++++++++++++++++++++------------------ 1 file changed, 37 insertions(+), 35 deletions(-) diff --git a/spec/outputs/kusto_spec.rb b/spec/outputs/kusto_spec.rb index 06fe22d3..eba4cb7b 100755 --- a/spec/outputs/kusto_spec.rb +++ b/spec/outputs/kusto_spec.rb @@ -21,61 +21,63 @@ } } describe '#register' do - skip 'temporarily disabling all tests' do - it 'allows valid configuration' do - kusto = described_class.new(options) - expect { kusto.register }.not_to raise_error - kusto.close - end + it 'allows valid configuration' do + RSpec.configuration.reporter.message("Running test: allows valid configuration") + kusto = described_class.new(options) + expect { kusto.register }.not_to raise_error + kusto.close + RSpec.configuration.reporter.message("Completed test: allows valid configuration") end end describe '#multi_receive_encoded' do - skip 'temporarily disabling all tests' do - it 'buffers events and flushes based on max_size' do - kusto = described_class.new(options.merge( {'max_size' => 2} )) - kusto.register + it 'buffers events and flushes based on max_size' do + RSpec.configuration.reporter.message("Running test: buffers events and flushes based on max_size") + kusto = described_class.new(options.merge({'max_size' => 2})) + kusto.register - event1 = LogStash::Event.new("message" => "event1") - event2 = LogStash::Event.new("message" => "event2") - event3 = LogStash::Event.new("message" => "event3") + event1 = LogStash::Event.new("message" => "event1") + event2 = LogStash::Event.new("message" => "event2") + event3 = LogStash::Event.new("message" => "event3") - expect(kusto.instance_variable_get(:@buffer)).to receive(:flush).twice.and_call_original + expect(kusto.instance_variable_get(:@buffer)).to receive(:flush).twice.and_call_original - kusto.multi_receive_encoded([[event1, event1.to_json], [event2, event2.to_json]]) - kusto.multi_receive_encoded([[event3, event3.to_json]]) + kusto.multi_receive_encoded([[event1, event1.to_json], [event2, event2.to_json]]) + kusto.multi_receive_encoded([[event3, event3.to_json]]) - kusto.close - end + kusto.close + RSpec.configuration.reporter.message("Completed test: buffers events and flushes based on max_size") + end - it 'flushes events based on max_interval' do - kusto = described_class.new(options.merge( {'max_interval' => 1} )) - kusto.register + it 'flushes events based on max_interval' do + RSpec.configuration.reporter.message("Running test: flushes events based on max_interval") + kusto = described_class.new(options.merge({'max_interval' => 1})) + kusto.register - event1 = LogStash::Event.new("message" => "event1") + event1 = LogStash::Event.new("message" => "event1") - expect(kusto.instance_variable_get(:@buffer)).to receive(:flush).at_least(:once).and_call_original + expect(kusto.instance_variable_get(:@buffer)).to receive(:flush).at_least(:once).and_call_original - kusto.multi_receive_encoded([[event1, event1.to_json]]) + kusto.multi_receive_encoded([[event1, event1.to_json]]) - sleep 2 + sleep 2 - kusto.close - end + kusto.close + RSpec.configuration.reporter.message("Completed test: flushes events based on max_interval") end end describe '#close' do - skip 'temporarily disabling all tests' do - it 'shuts down the buffer and ingestor' do - kusto = described_class.new(options) - kusto.register + it 'shuts down the buffer and ingestor' do + RSpec.configuration.reporter.message("Running test: shuts down the buffer and ingestor") + kusto = described_class.new(options) + kusto.register - expect(kusto.instance_variable_get(:@buffer)).to receive(:shutdown) - expect(kusto.instance_variable_get(:@ingestor)).to receive(:stop) + expect(kusto.instance_variable_get(:@buffer)).to receive(:shutdown) + expect(kusto.instance_variable_get(:@ingestor)).to receive(:stop) - kusto.close - end + kusto.close + RSpec.configuration.reporter.message("Completed test: shuts down the buffer and ingestor") end end end \ No newline at end of file From 5a859faef2e624133d9b1ed0dc77efbea1575f0b Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Wed, 25 Sep 2024 15:07:40 +0530 Subject: [PATCH 14/52] Updated kusto_spec.rb and ingestor_spec.rb Testing spec files --- lib/logstash/outputs/kusto.rb | 4 +++- .../outputs/kusto/custom_size_based_buffer.rb | 18 +++++++++++++++--- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index 431f2c5a..d63b1fe2 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -109,16 +109,18 @@ def multi_receive_encoded(events_and_encoded) end def close + @logger.info("Closing Kusto output plugin") @flusher.stop unless @flusher.nil? @cleaner.stop unless @cleaner.nil? @buffer.shutdown @ingestor.stop unless @ingestor.nil? + @logger.info("Kusto output plugin closed") end private def flush_buffer(events) return if events.empty? - + @logger.info("flush_buffer with #{events.size} events") begin # Logic to send buffer to Kusto @ingestor.upload_async(events.join) diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index 264131af..3b893c0c 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -1,3 +1,5 @@ +require 'logger' + module LogStash module Outputs class CustomSizeBasedBuffer @@ -11,13 +13,20 @@ def initialize(max_size, max_interval, &flush_callback) @shutdown = false @flusher_condition = ConditionVariable.new + # Initialize logger + @logger = Logger.new(STDOUT) + @logger.level = Logger::DEBUG + start_flusher_thread end def <<(event) @mutex.synchronize do @buffer << event - flush if @buffer.size >= @max_size + if @buffer.size >= @max_size + @logger.debug("Size-based flush triggered") + flush + end end end @@ -38,6 +47,7 @@ def start_flusher_thread @mutex.synchronize do break if @shutdown if Time.now - @last_flush_time >= @max_interval + @logger.debug("Time-based flush triggered") flush end @flusher_condition.wait(@mutex, @max_interval) # Wait for either the interval or shutdown signal @@ -50,6 +60,7 @@ def start_flusher_thread def flush_if_needed @mutex.synchronize do if Time.now - @last_flush_time >= @max_interval + @logger.debug("Time-based flush triggered in flush_if_needed") flush end end @@ -59,11 +70,12 @@ def flush return if @buffer.empty? begin + @logger.debug("Flushing buffer with #{@buffer.size} events") @flush_callback.call(@buffer) rescue => e # Log the error and continue, - puts "Error during flush: #{e.message}" - puts e.backtrace.join("\n") + @logger.error("Error during flush: #{e.message}") + @logger.error(e.backtrace.join("\n")) ensure @buffer.clear @last_flush_time = Time.now From 5cab14719b8dee5af0973a9543ffc238127bc8da Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Wed, 25 Sep 2024 15:07:40 +0530 Subject: [PATCH 15/52] Updated kusto_spec.rb and ingestor_spec.rb Testing spec files --- spec/outputs/kusto_spec.rb | 46 -------------------------------------- 1 file changed, 46 deletions(-) diff --git a/spec/outputs/kusto_spec.rb b/spec/outputs/kusto_spec.rb index eba4cb7b..028f5c57 100755 --- a/spec/outputs/kusto_spec.rb +++ b/spec/outputs/kusto_spec.rb @@ -20,52 +20,6 @@ "max_interval" => 10 } } - describe '#register' do - it 'allows valid configuration' do - RSpec.configuration.reporter.message("Running test: allows valid configuration") - kusto = described_class.new(options) - expect { kusto.register }.not_to raise_error - kusto.close - RSpec.configuration.reporter.message("Completed test: allows valid configuration") - end - end - - describe '#multi_receive_encoded' do - it 'buffers events and flushes based on max_size' do - RSpec.configuration.reporter.message("Running test: buffers events and flushes based on max_size") - kusto = described_class.new(options.merge({'max_size' => 2})) - kusto.register - - event1 = LogStash::Event.new("message" => "event1") - event2 = LogStash::Event.new("message" => "event2") - event3 = LogStash::Event.new("message" => "event3") - - expect(kusto.instance_variable_get(:@buffer)).to receive(:flush).twice.and_call_original - - kusto.multi_receive_encoded([[event1, event1.to_json], [event2, event2.to_json]]) - kusto.multi_receive_encoded([[event3, event3.to_json]]) - - kusto.close - RSpec.configuration.reporter.message("Completed test: buffers events and flushes based on max_size") - end - - it 'flushes events based on max_interval' do - RSpec.configuration.reporter.message("Running test: flushes events based on max_interval") - kusto = described_class.new(options.merge({'max_interval' => 1})) - kusto.register - - event1 = LogStash::Event.new("message" => "event1") - - expect(kusto.instance_variable_get(:@buffer)).to receive(:flush).at_least(:once).and_call_original - - kusto.multi_receive_encoded([[event1, event1.to_json]]) - - sleep 2 - - kusto.close - RSpec.configuration.reporter.message("Completed test: flushes events based on max_interval") - end - end describe '#close' do it 'shuts down the buffer and ingestor' do From ab495f55269c0f67eeb22d8d755225d3ad633532 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Wed, 25 Sep 2024 15:07:40 +0530 Subject: [PATCH 16/52] Updated kusto_spec.rb and ingestor_spec.rb Testing spec files --- e2e/e2e.rb | 2 ++ lib/logstash/outputs/kusto/custom_size_based_buffer.rb | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/e2e/e2e.rb b/e2e/e2e.rb index cabc1f45..5e0d2b76 100755 --- a/e2e/e2e.rb +++ b/e2e/e2e.rb @@ -41,6 +41,7 @@ def initialize database => "#{@database}" table => "#{@table_with_mapping}" json_mapping => "#{@mapping_name}" + max_size => 2 } kusto { path => "nomaptmp%{+YYYY-MM-dd-HH-mm}.txt" @@ -48,6 +49,7 @@ def initialize ingest_url => "#{@ingest_url}" database => "#{@database}" table => "#{@table_without_mapping}" + max_size => 3 } } } diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index 3b893c0c..a5d972ad 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -24,7 +24,7 @@ def <<(event) @mutex.synchronize do @buffer << event if @buffer.size >= @max_size - @logger.debug("Size-based flush triggered") + @logger.debug("Size-based flush triggered after #{@max_size} was reached") flush end end @@ -47,7 +47,7 @@ def start_flusher_thread @mutex.synchronize do break if @shutdown if Time.now - @last_flush_time >= @max_interval - @logger.debug("Time-based flush triggered") + @logger.debug("Time-based flush triggered after #{@max_interval} seconds") flush end @flusher_condition.wait(@mutex, @max_interval) # Wait for either the interval or shutdown signal From 76904bd8c31961771c92daed9de0124fff32a0bd Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Fri, 27 Sep 2024 14:18:27 +0530 Subject: [PATCH 17/52] Updated max_size config The max_size config now refers to the size of the buffer (defaults to 10MB) instead of number of events in the buffer --- lib/logstash/outputs/kusto.rb | 4 ++-- .../outputs/kusto/custom_size_based_buffer.rb | 12 ++++++++---- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index d63b1fe2..e79fe11c 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -68,8 +68,8 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # Check Proxy URL can be over http or https. Dowe need it this way or ignore this & remove this config :proxy_protocol, validate: :string, required: false , default: 'http' - # Maximum size of the buffer before it gets flushed, defaults to 1000 - config :max_size, validate: :number, default: 1000 + # Maximum size of the buffer before it gets flushed, defaults to 10MB + config :max_size, validate: :number, default: 10 # Maximum interval (in seconds) before the buffer gets flushed, defaults to 10 config :max_interval, validate: :number, default: 10 diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index a5d972ad..b5db8a57 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -4,10 +4,11 @@ module LogStash module Outputs class CustomSizeBasedBuffer def initialize(max_size, max_interval, &flush_callback) - @max_size = max_size + @max_size = max_size * 1024 * 1024 # Convert MB to bytes @max_interval = max_interval @flush_callback = flush_callback @buffer = [] + @buffer_size = 0 # Initialize buffer size @mutex = Mutex.new @last_flush_time = Time.now @shutdown = false @@ -22,9 +23,11 @@ def initialize(max_size, max_interval, &flush_callback) def <<(event) @mutex.synchronize do + event_size = event.bytesize @buffer << event - if @buffer.size >= @max_size - @logger.debug("Size-based flush triggered after #{@max_size} was reached") + @buffer_size += event_size + if @buffer_size >= @max_size + @logger.debug("Size-based flush triggered after #{@max_size} bytes was reached") flush end end @@ -70,7 +73,7 @@ def flush return if @buffer.empty? begin - @logger.debug("Flushing buffer with #{@buffer.size} events") + @logger.debug("Flushing buffer with #{@buffer.size} events and #{@buffer_size} bytes") @flush_callback.call(@buffer) rescue => e # Log the error and continue, @@ -78,6 +81,7 @@ def flush @logger.error(e.backtrace.join("\n")) ensure @buffer.clear + @buffer_size = 0 # Reset buffer size @last_flush_time = Time.now end end From 86dc8ae7912421850c9b62e42970125a3a816431 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Fri, 27 Sep 2024 14:18:27 +0530 Subject: [PATCH 18/52] Updated max_size config The max_size config now refers to the size of the buffer (defaults to 10MB) instead of number of events in the buffer --- e2e/e2e.rb | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/e2e/e2e.rb b/e2e/e2e.rb index 5e0d2b76..f969281f 100755 --- a/e2e/e2e.rb +++ b/e2e/e2e.rb @@ -41,7 +41,7 @@ def initialize database => "#{@database}" table => "#{@table_with_mapping}" json_mapping => "#{@mapping_name}" - max_size => 2 + max_size => 0.005 } kusto { path => "nomaptmp%{+YYYY-MM-dd-HH-mm}.txt" @@ -49,7 +49,7 @@ def initialize ingest_url => "#{@ingest_url}" database => "#{@database}" table => "#{@table_without_mapping}" - max_size => 3 + max_size => 0.004 } } } From 47f8b1a08be99bc76f4fb2021f91bf830b2a50de Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Wed, 2 Oct 2024 21:20:53 +0530 Subject: [PATCH 19/52] Added tests in kusto_spec.rb --- lib/logstash/outputs/kusto.rb | 10 ++- spec/outputs/kusto/ingestor_spec.rb | 44 ---------- spec/outputs/kusto_spec.rb | 120 ++++++++++++++++++++++++++++ 3 files changed, 127 insertions(+), 47 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index e79fe11c..6d830621 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -104,7 +104,11 @@ def register public def multi_receive_encoded(events_and_encoded) events_and_encoded.each do |event, encoded| - @buffer << encoded + begin + @buffer << encoded + rescue => e + @logger.error("Error processing event: #{e.message}") + end end end @@ -114,10 +118,10 @@ def close @cleaner.stop unless @cleaner.nil? @buffer.shutdown @ingestor.stop unless @ingestor.nil? - @logger.info("Kusto output plugin closed") + @logger.info("Kusto output plugin Closed") end - private + public def flush_buffer(events) return if events.empty? @logger.info("flush_buffer with #{events.size} events") diff --git a/spec/outputs/kusto/ingestor_spec.rb b/spec/outputs/kusto/ingestor_spec.rb index 5ad06a03..a077549b 100755 --- a/spec/outputs/kusto/ingestor_spec.rb +++ b/spec/outputs/kusto/ingestor_spec.rb @@ -92,50 +92,6 @@ RSpec.configuration.reporter.message("Completed test: one of appid or managedid has to be provided with empty managed identity and appid") end end - end - # describe 'receiving events' do - - # context 'with non-zero flush interval' do - # let(:temporary_output_file) { Stud::Temporary.pathname } - - # let(:event_count) { 100 } - # let(:flush_interval) { 5 } - - # let(:events) do - # event_count.times.map do |idx| - # LogStash::Event.new('subject' => idx) - # end - # end - - # let(:output) { described_class.new(options.merge( {'path' => temporary_output_file, 'flush_interval' => flush_interval, 'delete_temp_files' => false } )) } - - # before(:each) { output.register } - - # after(:each) do - # output.close - # File.exist?(temporary_output_file) && File.unlink(temporary_output_file) - # File.exist?(temporary_output_file + '.kusto') && File.unlink(temporary_output_file + '.kusto') - # end - - # it 'eventually flushes without receiving additional events' do - # output.multi_receive_encoded(events) - - # # events should not all be flushed just yet... - # expect(File.read(temporary_output_file)).to satisfy("have less than #{event_count} lines") do |contents| - # contents && contents.lines.count < event_count - # end - - # # wait for the flusher to run... - # sleep(flush_interval + 1) - - # # events should all be flushed - # expect(File.read(temporary_output_file)).to satisfy("have exactly #{event_count} lines") do |contents| - # contents && contents.lines.count == event_count - # end - # end - # end - - # end end diff --git a/spec/outputs/kusto_spec.rb b/spec/outputs/kusto_spec.rb index 028f5c57..631e086f 100755 --- a/spec/outputs/kusto_spec.rb +++ b/spec/outputs/kusto_spec.rb @@ -1,4 +1,5 @@ # encoding: utf-8 +require_relative "../spec_helpers.rb" require 'logstash/outputs/kusto' require 'logstash/codecs/plain' require 'logstash/event' @@ -20,6 +21,125 @@ "max_interval" => 10 } } + describe '#initialize' do + it 'initializes with the correct options' do + RSpec.configuration.reporter.message("Running test: initializes with the correct options") + kusto = described_class.new(options.merge("app_key" => LogStash::Util::Password.new("mykey"))) + expect(kusto.instance_variable_get(:@path)).to eq("./kusto_tst/%{+YYYY-MM-dd-HH-mm}") + expect(kusto.instance_variable_get(:@ingest_url)).to eq("https://ingest-sdkse2etest.eastus.kusto.windows.net/") + expect(kusto.instance_variable_get(:@app_id)).to eq("myid") + expect(kusto.instance_variable_get(:@app_key).value).to eq("mykey") + expect(kusto.instance_variable_get(:@app_tenant)).to eq("mytenant") + expect(kusto.instance_variable_get(:@database)).to eq("mydatabase") + expect(kusto.instance_variable_get(:@table)).to eq("mytable") + expect(kusto.instance_variable_get(:@json_mapping)).to eq("mymapping") + expect(kusto.instance_variable_get(:@proxy_host)).to eq("localhost") + expect(kusto.instance_variable_get(:@proxy_port)).to eq(3128) + expect(kusto.instance_variable_get(:@proxy_protocol)).to eq("https") + expect(kusto.instance_variable_get(:@max_size)).to eq(2000) + expect(kusto.instance_variable_get(:@max_interval)).to eq(10) + RSpec.configuration.reporter.message("Completed test: initializes with the correct options") + end + end + + describe '#multi_receive_encoded' do + it 'processes events and adds them to the buffer' do + RSpec.configuration.reporter.message("Running test: processes events and adds them to the buffer") + kusto = described_class.new(options) + kusto.register + + events = [LogStash::Event.new("message" => "test1"), LogStash::Event.new("message" => "test2")] + encoded_events = events.map { |e| [e, e.to_json] } + kusto.multi_receive_encoded(encoded_events) + + buffer = kusto.instance_variable_get(:@buffer) + expect(buffer.instance_variable_get(:@buffer).size).to eq(2) + RSpec.configuration.reporter.message("Completed test: processes events and adds them to the buffer") + end + + it 'handles errors during event processing' do + RSpec.configuration.reporter.message("Running test: handles errors during event processing") + kusto = described_class.new(options) + kusto.register + + allow(kusto.instance_variable_get(:@buffer)).to receive(:<<).and_raise(StandardError.new("Test error")) + events = [LogStash::Event.new("message" => "test1")] + encoded_events = events.map { |e| [e, e.to_json] } + + expect { kusto.multi_receive_encoded(encoded_events) }.not_to raise_error + RSpec.configuration.reporter.message("Completed test: handles errors during event processing") + end + end + + describe '#register' do + it 'raises an error for invalid configurations' do + RSpec.configuration.reporter.message("Running test: raises an error for invalid configurations") + invalid_options = options.merge("ingest_url" => nil) + expect { described_class.new(invalid_options).register }.to raise_error(LogStash::ConfigurationError) + RSpec.configuration.reporter.message("Completed test: raises an error for invalid configurations") + end + end + + describe '#flush_buffer' do + it 'handles errors during buffer flushing' do + RSpec.configuration.reporter.message("Running test: handles errors during buffer flushing") + kusto = described_class.new(options) + kusto.register + + allow(kusto.instance_variable_get(:@ingestor)).to receive(:upload_async).and_raise(StandardError.new("Test error")) + events = [LogStash::Event.new("message" => "test1")] + encoded_events = events.map { |e| [e, e.to_json] } + kusto.multi_receive_encoded(encoded_events) + + expect { kusto.flush_buffer(encoded_events) }.not_to raise_error + RSpec.configuration.reporter.message("Completed test: handles errors during buffer flushing") + end + + it 'flushes the buffer when max_size is reached' do + RSpec.configuration.reporter.message("Running test: flushes the buffer when max_size is reached") + kusto = described_class.new(options.merge("max_size" => 1)) # Set max_size to 1MB for testing + kusto.register + + events = [LogStash::Event.new("message" => "test1")] + encoded_events = events.map { |e| [e, e.to_json] } + expect(kusto.instance_variable_get(:@ingestor)).to receive(:upload_async).with(anything) + kusto.multi_receive_encoded(encoded_events) + kusto.flush_buffer(encoded_events) # Pass the encoded events + RSpec.configuration.reporter.message("Completed test: flushes the buffer when max_size is reached") + end + + it 'flushes the buffer when max_interval is reached' do + RSpec.configuration.reporter.message("Running test: flushes the buffer when max_interval is reached") + kusto = described_class.new(options.merge("max_interval" => 1)) # Set max_interval to 1 second for testing + kusto.register + + events = [LogStash::Event.new("message" => "test1")] + encoded_events = events.map { |e| [e, e.to_json] } + kusto.multi_receive_encoded(encoded_events) + sleep(2) # Wait for the interval to pass + + expect(kusto.instance_variable_get(:@ingestor)).to receive(:upload_async).with(anything) + kusto.flush_buffer(encoded_events) # Pass the encoded events + RSpec.configuration.reporter.message("Completed test: flushes the buffer when max_interval is reached") + end + + it 'eventually flushes without receiving additional events based on max_interval' do + RSpec.configuration.reporter.message("Running test: eventually flushes without receiving additional events based on max_interval") + kusto = described_class.new(options.merge("max_interval" => 1)) # Set max_interval to 1 second for testing + kusto.register + + events = [LogStash::Event.new("message" => "test1")] + encoded_events = events.map { |e| [e, e.to_json] } + kusto.multi_receive_encoded(encoded_events) + + # Wait for the interval to pass + sleep(2) + + expect(kusto.instance_variable_get(:@ingestor)).to receive(:upload_async).with(anything) + kusto.flush_buffer(encoded_events) # Pass the encoded events + RSpec.configuration.reporter.message("Completed test: eventually flushes without receiving additional events based on max_interval") + end + end describe '#close' do it 'shuts down the buffer and ingestor' do From bbf9ed42aee6334bbcee573df97ee978a5b5b43b Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Wed, 2 Oct 2024 21:20:53 +0530 Subject: [PATCH 20/52] Added tests in kusto_spec.rb --- lib/logstash/outputs/kusto.rb | 30 +++++++++++++++---- .../outputs/kusto/custom_size_based_buffer.rb | 8 ++++- 2 files changed, 31 insertions(+), 7 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index 6d830621..0fd4e161 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -22,7 +22,7 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # The Kusto endpoint for ingestion related communication. You can see it on the Azure Portal. config :ingest_url, validate: :string, required: true - # The following are the credentails used to connect to the Kusto service + # The following are the credentials used to connect to the Kusto service # application id config :app_id, validate: :string, required: false # application key (secret) @@ -65,7 +65,7 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # Port where the proxy runs , defaults to 80. Usually a value like 3128 config :proxy_port, validate: :number, required: false , default: 80 - # Check Proxy URL can be over http or https. Dowe need it this way or ignore this & remove this + # Check Proxy URL can be over http or https. Do we need it this way or ignore this & remove this config :proxy_protocol, validate: :string, required: false , default: 'http' # Maximum size of the buffer before it gets flushed, defaults to 10MB @@ -109,15 +109,33 @@ def multi_receive_encoded(events_and_encoded) rescue => e @logger.error("Error processing event: #{e.message}") end + begin + @buffer << encoded + rescue => e + @logger.error("Error processing event: #{e.message}") + end end end def close @logger.info("Closing Kusto output plugin") - @flusher.stop unless @flusher.nil? - @cleaner.stop unless @cleaner.nil? - @buffer.shutdown - @ingestor.stop unless @ingestor.nil? + + begin + @buffer.shutdown unless @buffer.nil? + @logger.info("Buffer shutdown") unless @buffer.nil? + rescue => e + @logger.error("Error shutting down buffer: #{e.message}") + @logger.error(e.backtrace.join("\n")) + end + + begin + @ingestor.stop unless @ingestor.nil? + @logger.info("Ingestor stopped") unless @ingestor.nil? + rescue => e + @logger.error("Error stopping ingestor: #{e.message}") + @logger.error(e.backtrace.join("\n")) + end + @logger.info("Kusto output plugin Closed") end diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index b5db8a57..8ca083e3 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -34,12 +34,15 @@ def <<(event) end def shutdown + @logger.info("Shutting down CustomSizeBasedBuffer") @mutex.synchronize do @shutdown = true @flusher_condition.signal # Wake up the flusher thread end @flusher_thread.join + @logger.info("Flusher thread joined") flush # Ensure final flush after shutdown + @logger.info("CustomSizeBasedBuffer shutdown complete") end private @@ -48,7 +51,10 @@ def start_flusher_thread @flusher_thread = Thread.new do loop do @mutex.synchronize do - break if @shutdown + if @shutdown + @logger.debug("Flusher thread exiting due to shutdown signal") + break + end if Time.now - @last_flush_time >= @max_interval @logger.debug("Time-based flush triggered after #{@max_interval} seconds") flush From 028eec9f21af6516edb09a62e27e7ca5e704714b Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Tue, 15 Oct 2024 22:47:34 +0530 Subject: [PATCH 21/52] Updated custom_size_based_buffer.rb --- lib/logstash/outputs/kusto.rb | 5 - .../outputs/kusto/custom_size_based_buffer.rb | 131 +++++++++--------- 2 files changed, 63 insertions(+), 73 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index 0fd4e161..3e444f10 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -109,11 +109,6 @@ def multi_receive_encoded(events_and_encoded) rescue => e @logger.error("Error processing event: #{e.message}") end - begin - @buffer << encoded - rescue => e - @logger.error("Error processing event: #{e.message}") - end end end diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index 8ca083e3..11de9a9a 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -1,95 +1,90 @@ require 'logger' +require 'thread' module LogStash module Outputs class CustomSizeBasedBuffer - def initialize(max_size, max_interval, &flush_callback) - @max_size = max_size * 1024 * 1024 # Convert MB to bytes - @max_interval = max_interval + def initialize(max_size_mb, max_interval, &flush_callback) + @buffer_config = { + max_size: max_size_mb * 1024 * 1024, # Convert MB to bytes + max_interval: max_interval, + logger: Logger.new(STDOUT) + } + @buffer_state = { + pending_items: [], + pending_size: 0, + last_flush: Time.now.to_i, + timer: Thread.new do + loop do + sleep(@buffer_config[:max_interval]) + buffer_flush(force: true) + end + end + } @flush_callback = flush_callback - @buffer = [] - @buffer_size = 0 # Initialize buffer size - @mutex = Mutex.new - @last_flush_time = Time.now @shutdown = false - @flusher_condition = ConditionVariable.new - - # Initialize logger - @logger = Logger.new(STDOUT) - @logger.level = Logger::DEBUG - - start_flusher_thread + @buffer_config[:logger].info("CustomSizeBasedBuffer initialized with max_size: #{max_size_mb} MB, max_interval: #{max_interval} seconds") end def <<(event) - @mutex.synchronize do - event_size = event.bytesize - @buffer << event - @buffer_size += event_size - if @buffer_size >= @max_size - @logger.debug("Size-based flush triggered after #{@max_size} bytes was reached") - flush - end + while buffer_full? do + sleep 0.1 end + + @buffer_state[:pending_items] << event + @buffer_state[:pending_size] += event.bytesize + + buffer_flush end def shutdown - @logger.info("Shutting down CustomSizeBasedBuffer") - @mutex.synchronize do - @shutdown = true - @flusher_condition.signal # Wake up the flusher thread - end - @flusher_thread.join - @logger.info("Flusher thread joined") - flush # Ensure final flush after shutdown - @logger.info("CustomSizeBasedBuffer shutdown complete") + @buffer_config[:logger].info("Shutting down buffer") + @shutdown = true + @buffer_state[:timer].kill + buffer_flush(final: true) end private - def start_flusher_thread - @flusher_thread = Thread.new do - loop do - @mutex.synchronize do - if @shutdown - @logger.debug("Flusher thread exiting due to shutdown signal") - break - end - if Time.now - @last_flush_time >= @max_interval - @logger.debug("Time-based flush triggered after #{@max_interval} seconds") - flush - end - @flusher_condition.wait(@mutex, @max_interval) # Wait for either the interval or shutdown signal - end - end - end + def buffer_full? + @buffer_state[:pending_size] >= @buffer_config[:max_size] end + def buffer_flush(options = {}) + force = options[:force] || options[:final] + final = options[:final] - def flush_if_needed - @mutex.synchronize do - if Time.now - @last_flush_time >= @max_interval - @logger.debug("Time-based flush triggered in flush_if_needed") - flush - end + if @buffer_state[:pending_size] == 0 + return 0 end - end - def flush - return if @buffer.empty? - - begin - @logger.debug("Flushing buffer with #{@buffer.size} events and #{@buffer_size} bytes") - @flush_callback.call(@buffer) - rescue => e - # Log the error and continue, - @logger.error("Error during flush: #{e.message}") - @logger.error(e.backtrace.join("\n")) - ensure - @buffer.clear - @buffer_size = 0 # Reset buffer size - @last_flush_time = Time.now + time_since_last_flush = Time.now.to_i - @buffer_state[:last_flush] + + if !force && @buffer_state[:pending_size] < @buffer_config[:max_size] && time_since_last_flush < @buffer_config[:max_interval] + return 0 end + + if force + @buffer_config[:logger].info("Time-based flush triggered after #{@buffer_config[:max_interval]} seconds") + elsif @buffer_state[:pending_size] >= @buffer_config[:max_size] + @buffer_config[:logger].info("Size-based flush triggered at #{@buffer_state[:pending_size]} bytes was reached") + end + + outgoing_items = @buffer_state[:pending_items] + outgoing_size = @buffer_state[:pending_size] + buffer_initialize + + @flush_callback.call(outgoing_items) # Pass the list of events to the callback + + @buffer_state[:last_flush] = Time.now.to_i + @buffer_config[:logger].info("Flush completed. Flushed #{outgoing_items.size} events") + + outgoing_items.size + end + + def buffer_initialize + @buffer_state[:pending_items] = [] + @buffer_state[:pending_size] = 0 end end end From fed3fc43263b4cacaf3f985b88b0dfa06a3fb1b3 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Tue, 15 Oct 2024 22:47:34 +0530 Subject: [PATCH 22/52] Updated custom_size_based_buffer.rb ** Updated the kusto_spec.rb test --- spec/outputs/kusto_spec.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spec/outputs/kusto_spec.rb b/spec/outputs/kusto_spec.rb index 631e086f..b276ad88 100755 --- a/spec/outputs/kusto_spec.rb +++ b/spec/outputs/kusto_spec.rb @@ -53,7 +53,7 @@ kusto.multi_receive_encoded(encoded_events) buffer = kusto.instance_variable_get(:@buffer) - expect(buffer.instance_variable_get(:@buffer).size).to eq(2) + expect(buffer.instance_variable_get(:@buffer_state)[:pending_items].size).to eq(2) RSpec.configuration.reporter.message("Completed test: processes events and adds them to the buffer") end From 7ca57b9773d8070bf2ee0b99c80e137ba7246ad1 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Wed, 16 Oct 2024 10:48:10 +0530 Subject: [PATCH 23/52] Updated custom_size_based_buffer.rb --- .../outputs/kusto/custom_size_based_buffer.rb | 58 ++++++++++++------- 1 file changed, 38 insertions(+), 20 deletions(-) diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index 11de9a9a..c371408e 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -23,6 +23,8 @@ def initialize(max_size_mb, max_interval, &flush_callback) } @flush_callback = flush_callback @shutdown = false + @pending_mutex = Mutex.new + @flush_mutex = Mutex.new @buffer_config[:logger].info("CustomSizeBasedBuffer initialized with max_size: #{max_size_mb} MB, max_interval: #{max_interval} seconds") end @@ -31,8 +33,10 @@ def <<(event) sleep 0.1 end - @buffer_state[:pending_items] << event - @buffer_state[:pending_size] += event.bytesize + @pending_mutex.synchronize do + @buffer_state[:pending_items] << event + @buffer_state[:pending_size] += event.bytesize + end buffer_flush end @@ -47,39 +51,53 @@ def shutdown private def buffer_full? - @buffer_state[:pending_size] >= @buffer_config[:max_size] + @pending_mutex.synchronize do + @buffer_state[:pending_size] >= @buffer_config[:max_size] + end end def buffer_flush(options = {}) force = options[:force] || options[:final] final = options[:final] - if @buffer_state[:pending_size] == 0 + if final + @flush_mutex.lock + elsif !@flush_mutex.try_lock return 0 end - time_since_last_flush = Time.now.to_i - @buffer_state[:last_flush] + items_flushed = 0 - if !force && @buffer_state[:pending_size] < @buffer_config[:max_size] && time_since_last_flush < @buffer_config[:max_interval] - return 0 - end + begin + @pending_mutex.synchronize do + return 0 if @buffer_state[:pending_size] == 0 - if force - @buffer_config[:logger].info("Time-based flush triggered after #{@buffer_config[:max_interval]} seconds") - elsif @buffer_state[:pending_size] >= @buffer_config[:max_size] - @buffer_config[:logger].info("Size-based flush triggered at #{@buffer_state[:pending_size]} bytes was reached") - end + time_since_last_flush = Time.now.to_i - @buffer_state[:last_flush] - outgoing_items = @buffer_state[:pending_items] - outgoing_size = @buffer_state[:pending_size] - buffer_initialize + return 0 if !force && @buffer_state[:pending_size] < @buffer_config[:max_size] && time_since_last_flush < @buffer_config[:max_interval] - @flush_callback.call(outgoing_items) # Pass the list of events to the callback + if force + @buffer_config[:logger].info("Time-based flush triggered after #{@buffer_config[:max_interval]} seconds") + elsif @buffer_state[:pending_size] >= @buffer_config[:max_size] + @buffer_config[:logger].info("Size-based flush triggered at #{@buffer_state[:pending_size]} bytes was reached") + end - @buffer_state[:last_flush] = Time.now.to_i - @buffer_config[:logger].info("Flush completed. Flushed #{outgoing_items.size} events") + outgoing_items = @buffer_state[:pending_items].dup + outgoing_size = @buffer_state[:pending_size] + buffer_initialize + + @flush_callback.call(outgoing_items) # Pass the list of events to the callback + + @buffer_state[:last_flush] = Time.now.to_i + @buffer_config[:logger].info("Flush completed. Flushed #{outgoing_items.size} events, #{outgoing_size} bytes") + + items_flushed = outgoing_items.size + end + ensure + @flush_mutex.unlock + end - outgoing_items.size + items_flushed end def buffer_initialize From 1f9591f950364c73b55996ad37ec1eace45da184 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Thu, 17 Oct 2024 16:38:27 +0530 Subject: [PATCH 24/52] Adds temp file buffer used during network downtime --- lib/logstash/outputs/kusto.rb | 9 +- .../outputs/kusto/custom_size_based_buffer.rb | 109 +++++++++++++++++- lib/logstash/outputs/kusto/ingestor.rb | 42 ++++--- spec/outputs/kusto_spec.rb | 29 +++-- 4 files changed, 160 insertions(+), 29 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index 3e444f10..17770e1a 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -77,8 +77,13 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base default :codec, 'json_lines' def register - # Initialize the custom buffer with size and interval - @buffer = LogStash::Outputs::CustomSizeBasedBuffer.new(@max_size, @max_interval) do |events| + # Set buffer_file to a valid file path + buffer_file = File.join(Dir.pwd, 'buffer', 'kusto_buffer.dat') + + # Ensure the buffer directory exists + FileUtils.mkdir_p(File.dirname(buffer_file)) + # Initialize the custom buffer with size, interval, and buffer file + @buffer = LogStash::Outputs::CustomSizeBasedBuffer.new(@max_size, @max_interval, buffer_file) do |events| flush_buffer(events) end diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index c371408e..6f182909 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -1,13 +1,17 @@ require 'logger' require 'thread' +require 'fileutils' module LogStash module Outputs class CustomSizeBasedBuffer - def initialize(max_size_mb, max_interval, &flush_callback) + def initialize(max_size_mb, max_interval, buffer_file, &flush_callback) + raise ArgumentError, "buffer_file cannot be nil" if buffer_file.nil? + @buffer_config = { max_size: max_size_mb * 1024 * 1024, # Convert MB to bytes max_interval: max_interval, + buffer_file: buffer_file, logger: Logger.new(STDOUT) } @buffer_state = { @@ -25,6 +29,7 @@ def initialize(max_size_mb, max_interval, &flush_callback) @shutdown = false @pending_mutex = Mutex.new @flush_mutex = Mutex.new + load_buffer_from_file @buffer_config[:logger].info("CustomSizeBasedBuffer initialized with max_size: #{max_size_mb} MB, max_interval: #{max_interval} seconds") end @@ -46,6 +51,7 @@ def shutdown @shutdown = true @buffer_state[:timer].kill buffer_flush(final: true) + clear_file_buffer end private @@ -69,30 +75,49 @@ def buffer_flush(options = {}) items_flushed = 0 begin + outgoing_items = [] + outgoing_size = 0 + @pending_mutex.synchronize do return 0 if @buffer_state[:pending_size] == 0 time_since_last_flush = Time.now.to_i - @buffer_state[:last_flush] - return 0 if !force && @buffer_state[:pending_size] < @buffer_config[:max_size] && time_since_last_flush < @buffer_config[:max_interval] + if !force && @buffer_state[:pending_size] < @buffer_config[:max_size] && time_since_last_flush < @buffer_config[:max_interval] + return 0 + end if force @buffer_config[:logger].info("Time-based flush triggered after #{@buffer_config[:max_interval]} seconds") elsif @buffer_state[:pending_size] >= @buffer_config[:max_size] @buffer_config[:logger].info("Size-based flush triggered at #{@buffer_state[:pending_size]} bytes was reached") + else + @buffer_config[:logger].info("Flush triggered without specific condition") end outgoing_items = @buffer_state[:pending_items].dup outgoing_size = @buffer_state[:pending_size] buffer_initialize + end + begin @flush_callback.call(outgoing_items) # Pass the list of events to the callback + clear_flushed_buffer_states(outgoing_items) unless ::File.zero?(@buffer_config[:buffer_file]) # Clear the flushed items from the file + rescue => e + @buffer_config[:logger].error("Flush failed: #{e.message}") + # Save the items to the file buffer in case of failure + @pending_mutex.synchronize do + @buffer_state[:pending_items] = outgoing_items + @buffer_state[:pending_items] + @buffer_state[:pending_size] += outgoing_size + save_buffer_to_file + end + raise e + end - @buffer_state[:last_flush] = Time.now.to_i - @buffer_config[:logger].info("Flush completed. Flushed #{outgoing_items.size} events, #{outgoing_size} bytes") + @buffer_state[:last_flush] = Time.now.to_i + @buffer_config[:logger].info("Flush completed. Flushed #{outgoing_items.size} events, #{outgoing_size} bytes") - items_flushed = outgoing_items.size - end + items_flushed = outgoing_items.size ensure @flush_mutex.unlock end @@ -104,6 +129,78 @@ def buffer_initialize @buffer_state[:pending_items] = [] @buffer_state[:pending_size] = 0 end + + def clear_flushed_buffer_states(flushed_items) + remaining_buffer_states = [] + ::File.foreach(@buffer_config[:buffer_file]) do |line| + begin + buffer_state = Marshal.load(line) + buffer_state[:pending_items] -= flushed_items + buffer_state[:pending_size] = buffer_state[:pending_items].sum(&:bytesize) + remaining_buffer_states << buffer_state unless buffer_state[:pending_items].empty? + rescue ArgumentError => e + @buffer_config[:logger].error("Failed to load buffer state: #{e.message}") + next + end + end + + ::File.open(@buffer_config[:buffer_file], 'w') do |file| + remaining_buffer_states.each do |state| + file.write(Marshal.dump(state) + "\n") + end + end + end + + def save_buffer_to_file + buffer_state_copy = @buffer_state.dup + buffer_state_copy.delete(:timer) # Exclude the Thread object from serialization + + ::FileUtils.mkdir_p(::File.dirname(@buffer_config[:buffer_file])) # Ensure directory exists + ::File.open(@buffer_config[:buffer_file], 'a') do |file| + file.write(Marshal.dump(buffer_state_copy) + "\n") + end + @buffer_config[:logger].info("Saved buffer state to file") + end + + def load_buffer_from_file + ::FileUtils.mkdir_p(::File.dirname(@buffer_config[:buffer_file])) # Ensure directory exists + ::File.open(@buffer_config[:buffer_file], 'a') {} # Create the file if it doesn't exist + + if ::File.file?(@buffer_config[:buffer_file]) && !::File.zero?(@buffer_config[:buffer_file]) + begin + @pending_mutex.synchronize do + buffer_states = [] + ::File.foreach(@buffer_config[:buffer_file]) do |line| + buffer_states << Marshal.load(line) + end + @buffer_state = buffer_states.reduce do |acc, state| + acc[:pending_items].concat(state[:pending_items]) + acc[:pending_size] += state[:pending_size] + acc + end + @buffer_state[:timer] = Thread.new do + loop do + sleep(@buffer_config[:max_interval]) + buffer_flush(force: true) + end + end + # Ensure the buffer does not flush immediately upon loading + @buffer_state[:last_flush] = Time.now.to_i + end + @buffer_config[:logger].info("Loaded buffer state from file") + rescue => e + @buffer_config[:logger].error("Failed to load buffer from file: #{e.message}") + buffer_initialize + end + else + buffer_initialize + end + end + + def clear_file_buffer + ::File.open(@buffer_config[:buffer_file], 'w') {} # Truncate the file + @buffer_config[:logger].info("File buffer cleared on shutdown") + end end end end \ No newline at end of file diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index bd0f0cbc..6a5aa79f 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -126,10 +126,15 @@ def upload_async(data) @workers_pool.post do LogStash::Util.set_thread_name("Kusto to ingest data") - upload(data) + begin + upload(data) + rescue => e + @logger.error('Error during async upload.', exception: e.class, message: e.message, backtrace: e.backtrace) + raise e + end end rescue Exception => e - @logger.error('StandardError.', exception: e.class, message: e.message, backtrace: e.backtrace) + @logger.error('StandardError in upload_async.', exception: e.class, message: e.message, backtrace: e.backtrace) raise e end @@ -150,21 +155,30 @@ def upload(data) # local_ingestion_properties.addJsonMappingName(json_mapping) # end + if data.size > 0 - data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(java.io.ByteArrayInputStream.new(data.to_java_bytes)) - @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) - else - @logger.warn("Data is empty and is not ingested.") - end - @logger.debug("Data sent to Kusto.") - rescue => e - # When the retry limit is reached or another error happen we will wait and retry. + data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(java.io.ByteArrayInputStream.new(data.to_java_bytes)) + @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) + else + @logger.warn("Data is empty and is not ingested.") + end + @logger.debug("Data sent to Kusto.") + rescue => e + # When the retry limit is reached or another error happens we will wait and retry. # - # Thread might be stuck here, but I think its better than losing anything - # its either a transient errors or something bad really happened. + # Thread might be stuck here, but I think it's better than losing anything + # it's either a transient error or something bad really happened. @logger.error('Uploading failed, retrying.', exception: e.class, message: e.message, backtrace: e.backtrace) - sleep RETRY_DELAY_SECONDS - retry + retry_count = 0 + max_retries = 5 + begin + sleep (2 ** retry_count) * RETRY_DELAY_SECONDS + retry_count += 1 + retry if retry_count <= max_retries + rescue => retry_error + @logger.error('Retrying failed.', exception: retry_error.class, message: retry_error.message, backtrace: retry_error.backtrace) + raise retry_error + end end def stop diff --git a/spec/outputs/kusto_spec.rb b/spec/outputs/kusto_spec.rb index b276ad88..1c34bf4b 100755 --- a/spec/outputs/kusto_spec.rb +++ b/spec/outputs/kusto_spec.rb @@ -50,13 +50,23 @@ events = [LogStash::Event.new("message" => "test1"), LogStash::Event.new("message" => "test2")] encoded_events = events.map { |e| [e, e.to_json] } - kusto.multi_receive_encoded(encoded_events) + # Temporarily disable automatic flushing for the test buffer = kusto.instance_variable_get(:@buffer) - expect(buffer.instance_variable_get(:@buffer_state)[:pending_items].size).to eq(2) + allow(buffer).to receive(:buffer_flush) + + # Clear the buffer before the test + buffer.instance_variable_set(:@buffer_state, { pending_items: [], pending_size: 0, last_flush: Time.now.to_i }) + + kusto.multi_receive_encoded(encoded_events) + + pending_items = buffer.instance_variable_get(:@buffer_state)[:pending_items] + RSpec.configuration.reporter.message("Pending items in buffer: #{pending_items.inspect}") + + expect(pending_items.size).to eq(2) RSpec.configuration.reporter.message("Completed test: processes events and adds them to the buffer") end - + it 'handles errors during event processing' do RSpec.configuration.reporter.message("Running test: handles errors during event processing") kusto = described_class.new(options) @@ -102,9 +112,14 @@ events = [LogStash::Event.new("message" => "test1")] encoded_events = events.map { |e| [e, e.to_json] } - expect(kusto.instance_variable_get(:@ingestor)).to receive(:upload_async).with(anything) + # Ensure upload_async is called only once + expect(kusto.instance_variable_get(:@ingestor)).to receive(:upload_async).with(anything).once kusto.multi_receive_encoded(encoded_events) - kusto.flush_buffer(encoded_events) # Pass the encoded events + + # Trigger the buffer flush manually + buffer = kusto.instance_variable_get(:@buffer) + buffer.send(:buffer_flush, force: true) + RSpec.configuration.reporter.message("Completed test: flushes the buffer when max_size is reached") end @@ -119,7 +134,7 @@ sleep(2) # Wait for the interval to pass expect(kusto.instance_variable_get(:@ingestor)).to receive(:upload_async).with(anything) - kusto.flush_buffer(encoded_events) # Pass the encoded events + kusto.flush_buffer(encoded_events) # Pass the encoded events here RSpec.configuration.reporter.message("Completed test: flushes the buffer when max_interval is reached") end @@ -136,7 +151,7 @@ sleep(2) expect(kusto.instance_variable_get(:@ingestor)).to receive(:upload_async).with(anything) - kusto.flush_buffer(encoded_events) # Pass the encoded events + kusto.flush_buffer(encoded_events) # Pass the encoded events here RSpec.configuration.reporter.message("Completed test: eventually flushes without receiving additional events based on max_interval") end end From b033dee2474c1879e808a51f0f02aca01dc05721 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Thu, 17 Oct 2024 17:16:40 +0530 Subject: [PATCH 25/52] Updated custom_size_based_buffer.rb Removed the temp file buffer and added retry to prevent data loss --- lib/logstash/outputs/kusto.rb | 9 +- .../outputs/kusto/custom_size_based_buffer.rb | 99 +++---------------- spec/outputs/kusto_spec.rb | 13 --- 3 files changed, 15 insertions(+), 106 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index 17770e1a..6cee01d4 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -77,13 +77,8 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base default :codec, 'json_lines' def register - # Set buffer_file to a valid file path - buffer_file = File.join(Dir.pwd, 'buffer', 'kusto_buffer.dat') - - # Ensure the buffer directory exists - FileUtils.mkdir_p(File.dirname(buffer_file)) # Initialize the custom buffer with size, interval, and buffer file - @buffer = LogStash::Outputs::CustomSizeBasedBuffer.new(@max_size, @max_interval, buffer_file) do |events| + @buffer = LogStash::Outputs::CustomSizeBasedBuffer.new(@max_size, @max_interval) do |events| flush_buffer(events) end @@ -147,9 +142,9 @@ def flush_buffer(events) # Logic to send buffer to Kusto @ingestor.upload_async(events.join) rescue => e - # Log the error and continue @logger.error("Error during flush: #{e.message}") @logger.error(e.backtrace.join("\n")) + raise e # Exception is raised to trigger the rescue block in buffer_flush end end diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index 6f182909..d610efbe 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -1,17 +1,13 @@ require 'logger' require 'thread' -require 'fileutils' module LogStash module Outputs class CustomSizeBasedBuffer - def initialize(max_size_mb, max_interval, buffer_file, &flush_callback) - raise ArgumentError, "buffer_file cannot be nil" if buffer_file.nil? - + def initialize(max_size_mb, max_interval, &flush_callback) @buffer_config = { max_size: max_size_mb * 1024 * 1024, # Convert MB to bytes max_interval: max_interval, - buffer_file: buffer_file, logger: Logger.new(STDOUT) } @buffer_state = { @@ -29,7 +25,6 @@ def initialize(max_size_mb, max_interval, buffer_file, &flush_callback) @shutdown = false @pending_mutex = Mutex.new @flush_mutex = Mutex.new - load_buffer_from_file @buffer_config[:logger].info("CustomSizeBasedBuffer initialized with max_size: #{max_size_mb} MB, max_interval: #{max_interval} seconds") end @@ -51,7 +46,6 @@ def shutdown @shutdown = true @buffer_state[:timer].kill buffer_flush(final: true) - clear_file_buffer end private @@ -73,6 +67,8 @@ def buffer_flush(options = {}) end items_flushed = 0 + max_retries = 5 + retries = 0 begin outgoing_items = [] @@ -101,17 +97,20 @@ def buffer_flush(options = {}) end begin + @buffer_config[:logger].info("Attempting to flush #{outgoing_items.size} items to the network") @flush_callback.call(outgoing_items) # Pass the list of events to the callback - clear_flushed_buffer_states(outgoing_items) unless ::File.zero?(@buffer_config[:buffer_file]) # Clear the flushed items from the file + @buffer_config[:logger].info("Successfully flushed #{outgoing_items.size} items to the network") rescue => e @buffer_config[:logger].error("Flush failed: #{e.message}") - # Save the items to the file buffer in case of failure - @pending_mutex.synchronize do - @buffer_state[:pending_items] = outgoing_items + @buffer_state[:pending_items] - @buffer_state[:pending_size] += outgoing_size - save_buffer_to_file + @buffer_config[:logger].error(e.backtrace.join("\n")) + retries += 1 + if retries <= max_retries + sleep 1 + retry + else + @buffer_config[:logger].error("Max retries reached. Data loss may occur.") + raise e end - raise e end @buffer_state[:last_flush] = Time.now.to_i @@ -129,78 +128,6 @@ def buffer_initialize @buffer_state[:pending_items] = [] @buffer_state[:pending_size] = 0 end - - def clear_flushed_buffer_states(flushed_items) - remaining_buffer_states = [] - ::File.foreach(@buffer_config[:buffer_file]) do |line| - begin - buffer_state = Marshal.load(line) - buffer_state[:pending_items] -= flushed_items - buffer_state[:pending_size] = buffer_state[:pending_items].sum(&:bytesize) - remaining_buffer_states << buffer_state unless buffer_state[:pending_items].empty? - rescue ArgumentError => e - @buffer_config[:logger].error("Failed to load buffer state: #{e.message}") - next - end - end - - ::File.open(@buffer_config[:buffer_file], 'w') do |file| - remaining_buffer_states.each do |state| - file.write(Marshal.dump(state) + "\n") - end - end - end - - def save_buffer_to_file - buffer_state_copy = @buffer_state.dup - buffer_state_copy.delete(:timer) # Exclude the Thread object from serialization - - ::FileUtils.mkdir_p(::File.dirname(@buffer_config[:buffer_file])) # Ensure directory exists - ::File.open(@buffer_config[:buffer_file], 'a') do |file| - file.write(Marshal.dump(buffer_state_copy) + "\n") - end - @buffer_config[:logger].info("Saved buffer state to file") - end - - def load_buffer_from_file - ::FileUtils.mkdir_p(::File.dirname(@buffer_config[:buffer_file])) # Ensure directory exists - ::File.open(@buffer_config[:buffer_file], 'a') {} # Create the file if it doesn't exist - - if ::File.file?(@buffer_config[:buffer_file]) && !::File.zero?(@buffer_config[:buffer_file]) - begin - @pending_mutex.synchronize do - buffer_states = [] - ::File.foreach(@buffer_config[:buffer_file]) do |line| - buffer_states << Marshal.load(line) - end - @buffer_state = buffer_states.reduce do |acc, state| - acc[:pending_items].concat(state[:pending_items]) - acc[:pending_size] += state[:pending_size] - acc - end - @buffer_state[:timer] = Thread.new do - loop do - sleep(@buffer_config[:max_interval]) - buffer_flush(force: true) - end - end - # Ensure the buffer does not flush immediately upon loading - @buffer_state[:last_flush] = Time.now.to_i - end - @buffer_config[:logger].info("Loaded buffer state from file") - rescue => e - @buffer_config[:logger].error("Failed to load buffer from file: #{e.message}") - buffer_initialize - end - else - buffer_initialize - end - end - - def clear_file_buffer - ::File.open(@buffer_config[:buffer_file], 'w') {} # Truncate the file - @buffer_config[:logger].info("File buffer cleared on shutdown") - end end end end \ No newline at end of file diff --git a/spec/outputs/kusto_spec.rb b/spec/outputs/kusto_spec.rb index 1c34bf4b..967f9a11 100755 --- a/spec/outputs/kusto_spec.rb +++ b/spec/outputs/kusto_spec.rb @@ -91,19 +91,6 @@ end describe '#flush_buffer' do - it 'handles errors during buffer flushing' do - RSpec.configuration.reporter.message("Running test: handles errors during buffer flushing") - kusto = described_class.new(options) - kusto.register - - allow(kusto.instance_variable_get(:@ingestor)).to receive(:upload_async).and_raise(StandardError.new("Test error")) - events = [LogStash::Event.new("message" => "test1")] - encoded_events = events.map { |e| [e, e.to_json] } - kusto.multi_receive_encoded(encoded_events) - - expect { kusto.flush_buffer(encoded_events) }.not_to raise_error - RSpec.configuration.reporter.message("Completed test: handles errors during buffer flushing") - end it 'flushes the buffer when max_size is reached' do RSpec.configuration.reporter.message("Running test: flushes the buffer when max_size is reached") From 71998af4a5d765ef97813a7a3888bc550010b58e Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Thu, 17 Oct 2024 17:16:40 +0530 Subject: [PATCH 26/52] Updated custom_size_based_buffer.rb Added exponential backoff to retries. Removed max_retries. --- .../outputs/kusto/custom_size_based_buffer.rb | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index d610efbe..1b8b0f6a 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -67,8 +67,7 @@ def buffer_flush(options = {}) end items_flushed = 0 - max_retries = 5 - retries = 0 + backoff_interval = 1 begin outgoing_items = [] @@ -103,14 +102,9 @@ def buffer_flush(options = {}) rescue => e @buffer_config[:logger].error("Flush failed: #{e.message}") @buffer_config[:logger].error(e.backtrace.join("\n")) - retries += 1 - if retries <= max_retries - sleep 1 - retry - else - @buffer_config[:logger].error("Max retries reached. Data loss may occur.") - raise e - end + sleep backoff_interval + backoff_interval = [backoff_interval * 2, 60].min # Exponential backoff with a max interval of 60 seconds + retry end @buffer_state[:last_flush] = Time.now.to_i From 9d574e3e1ac48e8c9c0e0e0d49d2be60065a151b Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Thu, 24 Oct 2024 22:10:48 +0530 Subject: [PATCH 27/52] Added temporary file buffer for persistant storage Updated upload_async and upload to raise errors during network downtime to trigger rescue block in flush buffer so the data sent for flushing can be restored and the flush can be attempted later. Added file buffer to store data ONLY when flush fails due to network issues. Once the network is back online the each file in the buffer is flushed and deleted first and the regular in-memory buffer is used post that. --- lib/logstash/outputs/kusto.rb | 2 +- .../outputs/kusto/custom_size_based_buffer.rb | 91 +++++++++++++++---- lib/logstash/outputs/kusto/ingestor.rb | 82 +++++++++-------- 3 files changed, 117 insertions(+), 58 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index 6cee01d4..1c663152 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -77,7 +77,7 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base default :codec, 'json_lines' def register - # Initialize the custom buffer with size, interval, and buffer file + # Initialize the custom buffer with size and interval @buffer = LogStash::Outputs::CustomSizeBasedBuffer.new(@max_size, @max_interval) do |events| flush_buffer(events) end diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index 1b8b0f6a..7c680266 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -1,5 +1,7 @@ require 'logger' require 'thread' +require 'fileutils' +require 'securerandom' module LogStash module Outputs @@ -8,24 +10,33 @@ def initialize(max_size_mb, max_interval, &flush_callback) @buffer_config = { max_size: max_size_mb * 1024 * 1024, # Convert MB to bytes max_interval: max_interval, + buffer_dir: './tmp/buffer_storage/', logger: Logger.new(STDOUT) } @buffer_state = { pending_items: [], pending_size: 0, last_flush: Time.now.to_i, - timer: Thread.new do - loop do - sleep(@buffer_config[:max_interval]) - buffer_flush(force: true) - end - end + timer: nil, + network_down: false } @flush_callback = flush_callback @shutdown = false @pending_mutex = Mutex.new @flush_mutex = Mutex.new + load_buffer_from_files @buffer_config[:logger].info("CustomSizeBasedBuffer initialized with max_size: #{max_size_mb} MB, max_interval: #{max_interval} seconds") + + # Start the timer thread after a delay to ensure initializations are completed + Thread.new do + sleep(10) + @buffer_state[:timer] = Thread.new do + loop do + sleep(@buffer_config[:max_interval]) + buffer_flush(force: true) + end + end + end end def <<(event) @@ -37,8 +48,6 @@ def <<(event) @buffer_state[:pending_items] << event @buffer_state[:pending_size] += event.bytesize end - - buffer_flush end def shutdown @@ -46,6 +55,7 @@ def shutdown @shutdown = true @buffer_state[:timer].kill buffer_flush(final: true) + clear_buffer_files end private @@ -67,7 +77,6 @@ def buffer_flush(options = {}) end items_flushed = 0 - backoff_interval = 1 begin outgoing_items = [] @@ -92,19 +101,18 @@ def buffer_flush(options = {}) outgoing_items = @buffer_state[:pending_items].dup outgoing_size = @buffer_state[:pending_size] - buffer_initialize + @buffer_state[:pending_items] = [] + @buffer_state[:pending_size] = 0 end begin - @buffer_config[:logger].info("Attempting to flush #{outgoing_items.size} items to the network") @flush_callback.call(outgoing_items) # Pass the list of events to the callback - @buffer_config[:logger].info("Successfully flushed #{outgoing_items.size} items to the network") + @buffer_state[:network_down] = false # Reset network status after successful flush + flush_buffer_files # Flush buffer files if any exist rescue => e @buffer_config[:logger].error("Flush failed: #{e.message}") - @buffer_config[:logger].error(e.backtrace.join("\n")) - sleep backoff_interval - backoff_interval = [backoff_interval * 2, 60].min # Exponential backoff with a max interval of 60 seconds - retry + @buffer_state[:network_down] = true + save_buffer_to_file(outgoing_items) end @buffer_state[:last_flush] = Time.now.to_i @@ -118,9 +126,54 @@ def buffer_flush(options = {}) items_flushed end - def buffer_initialize - @buffer_state[:pending_items] = [] - @buffer_state[:pending_size] = 0 + def save_buffer_to_file(events) + buffer_state_copy = { + pending_items: events, + pending_size: events.sum(&:bytesize) + } + + ::FileUtils.mkdir_p(@buffer_config[:buffer_dir]) # Ensure directory exists + file_path = ::File.join(@buffer_config[:buffer_dir], "buffer_state_#{Time.now.to_i}_#{SecureRandom.uuid}.log") + ::File.open(file_path, 'w') do |file| + file.write(Marshal.dump(buffer_state_copy)) + end + @buffer_config[:logger].info("Saved buffer state to file: #{file_path}") + end + + def load_buffer_from_files + Dir.glob(::File.join(@buffer_config[:buffer_dir], 'buffer_state_*.log')).each do |file_path| + begin + buffer_state = Marshal.load(::File.read(file_path)) + @buffer_state[:pending_items].concat(buffer_state[:pending_items]) + @buffer_state[:pending_size] += buffer_state[:pending_size] + ::File.delete(file_path) + rescue => e + @buffer_config[:logger].error("Failed to load buffer from file: #{e.message}") + end + end + @buffer_config[:logger].info("Loaded buffer state from files") + end + + def flush_buffer_files + Dir.glob(::File.join(@buffer_config[:buffer_dir], 'buffer_state_*.log')).each do |file_path| + begin + buffer_state = Marshal.load(::File.read(file_path)) + @buffer_config[:logger].info("Flushed from file: #{file_path}") + @flush_callback.call(buffer_state[:pending_items]) + ::File.delete(file_path) + @buffer_config[:logger].info("Flushed and deleted buffer state file: #{file_path}") + rescue => e + @buffer_config[:logger].error("Failed to flush buffer state file: #{e.message}") + break + end + end + end + + def clear_buffer_files + Dir.glob(::File.join(@buffer_config[:buffer_dir], 'buffer_state_*.log')).each do |file_path| + ::File.delete(file_path) + end + @buffer_config[:logger].info("Cleared all buffer state files") end end end diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index 6a5aa79f..e991d5a5 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -54,7 +54,6 @@ def initialize(ingest_url, app_id, app_key, app_tenant, managed_identity_id, cli kusto_java.data.auth.ConnectionStringBuilder.createWithAadApplicationCredentials(ingest_url, app_id, app_key.value, app_tenant) end end - # @logger.debug(Gem.loaded_specs.to_s) # Unfortunately there's no way to avoid using the gem/plugin name directly... name_for_tracing = "logstash-output-kusto:#{Gem.loaded_specs['logstash-output-kusto']&.version || "unknown"}" @@ -123,62 +122,69 @@ def upload_async(data) if @workers_pool.remaining_capacity <= LOW_QUEUE_LENGTH @logger.warn("Ingestor queue capacity is running low with #{@workers_pool.remaining_capacity} free slots.") end - + exception = nil @workers_pool.post do LogStash::Util.set_thread_name("Kusto to ingest data") begin upload(data) rescue => e @logger.error('Error during async upload.', exception: e.class, message: e.message, backtrace: e.backtrace) - raise e + exception = e end end + + # Wait for the task to complete and check for exceptions + @workers_pool.shutdown + @workers_pool.wait_for_termination + + if exception + @logger.error('StandardError in upload_async.', exception: exception.class, message: exception.message, backtrace: exception.backtrace) + raise exception + end rescue Exception => e @logger.error('StandardError in upload_async.', exception: e.class, message: e.message, backtrace: e.backtrace) raise e end def upload(data) - @logger.debug("Sending data to Kusto") - - # TODO: dynamic routing - # file_metadata = path.partition('.kusto.').last - # file_metadata_parts = file_metadata.split('.') - - # if file_metadata_parts.length == 3 - # # this is the number we expect - database, table, json_mapping - # database = file_metadata_parts[0] - # table = file_metadata_parts[1] - # json_mapping = file_metadata_parts[2] - - # local_ingestion_properties = Java::KustoIngestionProperties.new(database, table) - # local_ingestion_properties.addJsonMappingName(json_mapping) - # end - + @logger.info("Sending data to Kusto") if data.size > 0 - data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(java.io.ByteArrayInputStream.new(data.to_java_bytes)) - @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) + ingestionLatch = java.util.concurrent.CountDownLatch.new(1) + + Thread.new do + begin + data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(java.io.ByteArrayInputStream.new(data.to_java_bytes)) + ingestion_result = @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) + + # Check the ingestion status + status = ingestion_result.getIngestionStatusCollection.get(0) + if status.status != Java::com.microsoft.azure.kusto.ingest.result.OperationStatus::Queued + raise "Failed upload: #{status.errorCodeString}" + end + @logger.info("Final ingestion status: #{status.status}") + rescue => e + @logger.error('Error during ingestFromStream.', exception: e.class, message: e.message, backtrace: e.backtrace) + if e.message.include?("network") + raise e + end + ensure + ingestionLatch.countDown() + end + end + + # Wait for the ingestion to complete with a timeout + if !ingestionLatch.await(30, java.util.concurrent.TimeUnit::SECONDS) + @logger.error('Ingestion timed out, possible network issue.') + raise 'Ingestion timed out, possible network issue.' + end else @logger.warn("Data is empty and is not ingested.") end - @logger.debug("Data sent to Kusto.") + @logger.info("Data sent to Kusto.") rescue => e - # When the retry limit is reached or another error happens we will wait and retry. - # - # Thread might be stuck here, but I think it's better than losing anything - # it's either a transient error or something bad really happened. - @logger.error('Uploading failed, retrying.', exception: e.class, message: e.message, backtrace: e.backtrace) - retry_count = 0 - max_retries = 5 - begin - sleep (2 ** retry_count) * RETRY_DELAY_SECONDS - retry_count += 1 - retry if retry_count <= max_retries - rescue => retry_error - @logger.error('Retrying failed.', exception: retry_error.class, message: retry_error.message, backtrace: retry_error.backtrace) - raise retry_error - end + @logger.error('Uploading failed.', exception: e.class, message: e.message, backtrace: e.backtrace) + raise e # Raise the original error if ingestion fails end def stop @@ -186,4 +192,4 @@ def stop @workers_pool.wait_for_termination(nil) # block until its done end end -end +end \ No newline at end of file From 398a301e7620a4f4238aeecf19fd43cbe68fa355 Mon Sep 17 00:00:00 2001 From: Ramachandran G Date: Wed, 16 Oct 2024 17:46:03 +0530 Subject: [PATCH 28/52] * Refactor to classes --- lib/logstash/outputs/kusto.rb | 18 +- .../outputs/kusto/custom_size_based_buffer.rb | 42 ++--- lib/logstash/outputs/kusto/ingestor.rb | 73 +++------ lib/logstash/outputs/kusto/interval.rb | 126 +++++++------- .../kusto/kustoLogstashConfiguration.rb | 155 ++++++++++++++++++ spec/outputs/kusto/ingestor_spec.rb | 97 ----------- spec/outputs/kusto/interval_spec.rb | 69 ++++++++ .../kusto/kustoLogstashConfiguration_spec.rb | 106 ++++++++++++ 8 files changed, 444 insertions(+), 242 deletions(-) create mode 100644 lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb delete mode 100755 spec/outputs/kusto/ingestor_spec.rb create mode 100644 spec/outputs/kusto/interval_spec.rb create mode 100755 spec/outputs/kusto/kustoLogstashConfiguration_spec.rb diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index 1c663152..7d6fb2dd 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -7,6 +7,7 @@ require 'logstash/outputs/kusto/ingestor' require 'logstash/outputs/kusto/interval' require 'logstash/outputs/kusto/custom_size_based_buffer' +require 'logstash/outputs/kusto/kustoLogstashConfiguration' ## # This plugin sends messages to Azure Kusto in batches. @@ -91,13 +92,13 @@ def register max_threads: upload_concurrent_count, max_queue: upload_queue_size, fallback_policy: :caller_runs) - - @ingestor = Ingestor.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cli_auth, database, table, final_mapping, proxy_host, proxy_port, proxy_protocol, @logger, executor) - - # Deprecation warning for path - if @path - @logger.warn("The 'path' configuration option is deprecated and will be removed in a future release.") - end + + kusto_ingest_base = LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, database, table, final_mapping) + kusto_auth_base = LogStash::Outputs::KustoInternal::KustoAuthConfiguration.new(app_id, app_key, app_tenant, managed_identity, cli_auth) + kusto_proxy_base = LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new(proxy_host , proxy_port , proxy_protocol, false) + @kusto_logstash_configuration = LogStash::Outputs::KustoInternal::KustoLogstashConfiguration.new(kusto_ingest_base, kusto_auth_base , kusto_proxy_base, logger) + @ingestor = Ingestor.new(@kusto_logstash_configuration, @logger, executor) + end @@ -114,8 +115,7 @@ def multi_receive_encoded(events_and_encoded) def close @logger.info("Closing Kusto output plugin") - - begin + begin @buffer.shutdown unless @buffer.nil? @logger.info("Buffer shutdown") unless @buffer.nil? rescue => e diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index 7c680266..6a012276 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -39,10 +39,10 @@ def initialize(max_size_mb, max_interval, &flush_callback) end end - def <<(event) - while buffer_full? do - sleep 0.1 - end + def <<(event) + while buffer_full? do + sleep 0.1 + end @pending_mutex.synchronize do @buffer_state[:pending_items] << event @@ -58,23 +58,23 @@ def shutdown clear_buffer_files end - private + private - def buffer_full? - @pending_mutex.synchronize do - @buffer_state[:pending_size] >= @buffer_config[:max_size] - end - end + def buffer_full? + @pending_mutex.synchronize do + @buffer_state[:pending_size] >= @buffer_config[:max_size] + end + end - def buffer_flush(options = {}) - force = options[:force] || options[:final] - final = options[:final] + def buffer_flush(options = {}) + force = options[:force] || options[:final] + final = options[:final] - if final - @flush_mutex.lock - elsif !@flush_mutex.try_lock - return 0 - end + if final + @flush_mutex.lock + elsif !@flush_mutex.try_lock + return 0 + end items_flushed = 0 @@ -85,7 +85,7 @@ def buffer_flush(options = {}) @pending_mutex.synchronize do return 0 if @buffer_state[:pending_size] == 0 - time_since_last_flush = Time.now.to_i - @buffer_state[:last_flush] + time_since_last_flush = Time.now.to_i - @buffer_state[:last_flush] if !force && @buffer_state[:pending_size] < @buffer_config[:max_size] && time_since_last_flush < @buffer_config[:max_interval] return 0 @@ -123,8 +123,8 @@ def buffer_flush(options = {}) @flush_mutex.unlock end - items_flushed - end + items_flushed + end def save_buffer_to_file(events) buffer_state_copy = { diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index e991d5a5..392cb0a1 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -20,38 +20,39 @@ class Ingestor LOW_QUEUE_LENGTH = 3 FIELD_REF = /%\{[^}]+\}/ - def initialize(ingest_url, app_id, app_key, app_tenant, managed_identity_id, cli_auth, database, table, json_mapping, proxy_host , proxy_port , proxy_protocol,logger, threadpool = DEFAULT_THREADPOOL) + def initialize(kusto_logstash_configuration, logger, threadpool = DEFAULT_THREADPOOL) @workers_pool = threadpool @logger = logger - validate_config(database, table, json_mapping,proxy_protocol,app_id, app_key, managed_identity_id,cli_auth) + #Validate and assign + kusto_logstash_configuration.validate_config() + @kusto_logstash_configuration = kusto_logstash_configuration + @logger.info('Preparing Kusto resources.') kusto_java = Java::com.microsoft.azure.kusto apache_http = Java::org.apache.http - # kusto_connection_string = kusto_java.data.auth.ConnectionStringBuilder.createWithAadApplicationCredentials(ingest_url, app_id, app_key.value, app_tenant) - # If there is managed identity, use it. This means the AppId and AppKey are empty/nil - # If there is CLI Auth, use that instead of managed identity - is_managed_identity = (app_id.nil? && app_key.nil? && !cli_auth) + + is_managed_identity = @kusto_logstash_configuration.kusto_auth.is_managed_identity # If it is system managed identity, propagate the system identity - is_system_assigned_managed_identity = is_managed_identity && 0 == "system".casecmp(managed_identity_id) + is_system_assigned_managed_identity = @kusto_logstash_configuration.kusto_auth.is_system_assigned_managed_identity # Is it direct connection - is_direct_conn = (proxy_host.nil? || proxy_host.empty?) + is_direct_conn = @kusto_logstash_configuration.kusto_proxy.is_direct_conn # Create a connection string kusto_connection_string = if is_managed_identity if is_system_assigned_managed_identity @logger.info('Using system managed identity.') - kusto_java.data.auth.ConnectionStringBuilder.createWithAadManagedIdentity(ingest_url) + kusto_java.data.auth.ConnectionStringBuilder.createWithAadManagedIdentity(@kusto_logstash_configuration.kusto_ingest.ingest_url) else @logger.info('Using user managed identity.') - kusto_java.data.auth.ConnectionStringBuilder.createWithAadManagedIdentity(ingest_url, managed_identity_id) + kusto_java.data.auth.ConnectionStringBuilder.createWithAadManagedIdentity(@kusto_logstash_configuration.kusto_ingest.ingest_url, @kusto_logstash_configuration.kusto_ingest.managed_identity_id) end else - if cli_auth + if @kusto_logstash_configuration.kusto_auth.cli_auth @logger.warn('*Use of CLI Auth is only for dev-test scenarios. This is ***NOT RECOMMENDED*** for production*') - kusto_java.data.auth.ConnectionStringBuilder.createWithAzureCli(ingest_url) + kusto_java.data.auth.ConnectionStringBuilder.createWithAzureCli(@kusto_logstash_configuration.kusto_ingest.ingest_url) else @logger.info('Using app id and app key.') - kusto_java.data.auth.ConnectionStringBuilder.createWithAadApplicationCredentials(ingest_url, app_id, app_key.value, app_tenant) + kusto_java.data.auth.ConnectionStringBuilder.createWithAadApplicationCredentials(@kusto_logstash_configuration.kusto_ingest.ingest_url, @kusto_logstash_configuration.kusto_auth.app_id, @kusto_logstash_configuration.kusto_auth.app_key.value, @kusto_logstash_configuration.kusto_auth.app_tenant) end end @logger.debug(Gem.loaded_specs.to_s) @@ -62,22 +63,22 @@ def initialize(ingest_url, app_id, app_key, app_tenant, managed_identity_id, cli tuple_utils = Java::org.apache.commons.lang3.tuple # kusto_connection_string.setClientVersionForTracing(name_for_tracing) version_for_tracing=Gem.loaded_specs['logstash-output-kusto']&.version || "unknown" - kusto_connection_string.setConnectorDetails("Logstash",version_for_tracing.to_s,"","",false,"", tuple_utils.Pair.emptyArray()); + kusto_connection_string.setConnectorDetails("Logstash",version_for_tracing.to_s,name_for_tracing.to_s,version_for_tracing.to_s,false,"", tuple_utils.Pair.emptyArray()); @kusto_client = begin if is_direct_conn kusto_java.ingest.IngestClientFactory.createClient(kusto_connection_string) else - kusto_http_client_properties = kusto_java.data.HttpClientProperties.builder().proxy(apache_http.HttpHost.new(proxy_host,proxy_port,proxy_protocol)).build() + kusto_http_client_properties = kusto_java.data.HttpClientProperties.builder().proxy(apache_http.HttpHost.new(@kusto_logstash_configuration.kusto_proxy.proxy_host,@kusto_logstash_configuration.kusto_proxy.proxy_port,@kusto_logstash_configuration.kusto_proxy.proxy_protocol)).build() kusto_java.ingest.IngestClientFactory.createClient(kusto_connection_string, kusto_http_client_properties) end end - @ingestion_properties = kusto_java.ingest.IngestionProperties.new(database, table) - is_mapping_ref_provided = !(json_mapping.nil? || json_mapping.empty?) - if is_mapping_ref_provided - @logger.debug('Using mapping reference.', json_mapping) - @ingestion_properties.setIngestionMapping(json_mapping, kusto_java.ingest.IngestionMapping::IngestionMappingKind::JSON) + @ingestion_properties = kusto_java.ingest.IngestionProperties.new(@kusto_logstash_configuration.kusto_ingest.database, @kusto_logstash_configuration.kusto_ingest.table) + + if @kusto_logstash_configuration.kusto_ingest.is_mapping_ref_provided + @logger.debug('Using mapping reference.', @kusto_logstash_configuration.kusto_ingest.json_mapping) + @ingestion_properties.setIngestionMapping(@kusto_logstash_configuration.kusto_ingest.json_mapping, kusto_java.ingest.IngestionMapping::IngestionMappingKind::JSON) @ingestion_properties.setDataFormat(kusto_java.ingest.IngestionProperties::DataFormat::JSON) else @logger.debug('No mapping reference provided. Columns will be mapped by names in the logstash output') @@ -86,38 +87,6 @@ def initialize(ingest_url, app_id, app_key, app_tenant, managed_identity_id, cli @logger.debug('Kusto resources are ready.') end - def validate_config(database, table, json_mapping, proxy_protocol, app_id, app_key, managed_identity_id,cli_auth) - # Add an additional validation and fail this upfront - if app_id.nil? && app_key.nil? && managed_identity_id.nil? - if cli_auth - @logger.info('Using CLI Auth, this is only for dev-test scenarios. This is ***NOT RECOMMENDED*** for production') - else - @logger.error('managed_identity_id is not provided and app_id/app_key is empty.') - raise LogStash::ConfigurationError.new('managed_identity_id is not provided and app_id/app_key is empty.') - end - end - if database =~ FIELD_REF - @logger.error('database config value should not be dynamic.', database) - raise LogStash::ConfigurationError.new('database config value should not be dynamic.') - end - - if table =~ FIELD_REF - @logger.error('table config value should not be dynamic.', table) - raise LogStash::ConfigurationError.new('table config value should not be dynamic.') - end - - if json_mapping =~ FIELD_REF - @logger.error('json_mapping config value should not be dynamic.', json_mapping) - raise LogStash::ConfigurationError.new('json_mapping config value should not be dynamic.') - end - - if not(["https", "http"].include? proxy_protocol) - @logger.error('proxy_protocol has to be http or https.', proxy_protocol) - raise LogStash::ConfigurationError.new('proxy_protocol has to be http or https.') - end - - end - def upload_async(data) if @workers_pool.remaining_capacity <= LOW_QUEUE_LENGTH @logger.warn("Ingestor queue capacity is running low with #{@workers_pool.remaining_capacity} free slots.") diff --git a/lib/logstash/outputs/kusto/interval.rb b/lib/logstash/outputs/kusto/interval.rb index 33046309..6ba8d4a4 100755 --- a/lib/logstash/outputs/kusto/interval.rb +++ b/lib/logstash/outputs/kusto/interval.rb @@ -5,77 +5,77 @@ require 'logstash/errors' class LogStash::Outputs::Kusto < LogStash::Outputs::Base - ## - # Bare-bones utility for running a block of code at an interval. - # - class Interval - ## - # Initializes a new Interval with the given arguments and starts it - # before returning it. - # - # @param interval [Integer] (see: Interval#initialize) - # @param procsy [#call] (see: Interval#initialize) - # - # @return [Interval] - # - def self.start(interval, procsy) - new(interval, procsy).tap(&:start) - end + ## + # Bare-bones utility for running a block of code at an interval. + # + class Interval + ## + # Initializes a new Interval with the given arguments and starts it + # before returning it. + # + # @param interval [Integer] (see: Interval#initialize) + # @param procsy [#call] (see: Interval#initialize) + # + # @return [Interval] + # + def self.start(interval, procsy) + new(interval, procsy).tap(&:start) + end - ## - # @param interval [Integer]: time in seconds to wait between calling the given proc - # @param procsy [#call]: proc or lambda to call periodically; must not raise exceptions. - def initialize(interval, procsy) - @interval = interval - @procsy = procsy + ## + # @param interval [Integer]: time in seconds to wait between calling the given proc + # @param procsy [#call]: proc or lambda to call periodically; must not raise exceptions. + def initialize(interval, procsy) + @interval = interval + @procsy = procsy - # Mutex, ConditionVariable, etc. - @mutex = Mutex.new - @sleeper = ConditionVariable.new - end + # Mutex, ConditionVariable, etc. + @mutex = Mutex.new + @sleeper = ConditionVariable.new + end - ## - # Starts the interval, or returns if it has already been started. - # - # @return [void] - def start - @mutex.synchronize do - return if @thread && @thread.alive? + ## + # Starts the interval, or returns if it has already been started. + # + # @return [void] + def start + @mutex.synchronize do + return if @thread && @thread.alive? - @thread = Thread.new { run } - end - end + @thread = Thread.new { run } + end + end - ## - # Stop the interval. - # Does not interrupt if execution is in-progress. - def stop - @mutex.synchronize do - @stopped = true - end + ## + # Stop the interval. + # Does not interrupt if execution is in-progress. + def stop + @mutex.synchronize do + @stopped = true + end - @thread && @thread.join - end + @thread && @thread.join + end - ## - # @return [Boolean] - def alive? - @thread && @thread.alive? - end + ## + # @return [Boolean] + def alive? + @thread && @thread.alive? + end - private + private - def run - @mutex.synchronize do - loop do - @sleeper.wait(@mutex, @interval) - break if @stopped + def run + @mutex.synchronize do + loop do + @sleeper.wait(@mutex, @interval) + break if @stopped - @procsy.call - end - end - ensure - @sleeper.broadcast - end - end + @procsy.call + end + end + ensure + @sleeper.broadcast + end + end end diff --git a/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb b/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb new file mode 100644 index 00000000..0aecb874 --- /dev/null +++ b/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb @@ -0,0 +1,155 @@ +# encoding: utf-8 +# A class just having all the configurations wrapped into a seperate object +module LogStash + module Outputs + module KustoInternal + class KustoLogstashConfiguration + FIELD_REF = /%\{[^}]+\}/ + def initialize(kusto_ingest,kusto_auth, kusto_proxy, logger) + @logger = logger + @kusto_ingest = kusto_ingest + @kusto_auth = kusto_auth + @kusto_proxy = kusto_proxy + @logger.info("Kusto configuration initialized.") + end # def initialize + + # Configuration + def kusto_ingest + @kusto_ingest + end + def kusto_auth + @kusto_auth + end + def kusto_proxy + @kusto_proxy + end + + def validate_config() + # Add an additional validation and fail this upfront + if @kusto_auth.app_id.to_s.empty? && @kusto_auth.managed_identity_id.to_s.empty? && !@kusto_auth.cli_auth + @logger.error('managed_identity_id is not provided, cli_auth is false and app_id/app_key is empty.') + raise LogStash::ConfigurationError.new('managed_identity_id is not provided and app_id/app_key is empty.') + end + # If proxy AAD is required and the proxy configuration is not provided - fail + if @kusto_proxy.proxy_aad_only && @kusto_proxy.is_direct_conn + @logger.error('proxy_aad_only can be used only when proxy is configured.', @kusto_proxy.proxy_aad_only) + raise LogStash::ConfigurationError.new('proxy_aad_only can be used only when proxy is configured.') + end + + if @kusto_ingest.database =~ FIELD_REF + @logger.error('database config value should not be dynamic.', @kusto_ingest.database) + raise LogStash::ConfigurationError.new('database config value should not be dynamic.') + end + if @kusto_ingest.table =~ FIELD_REF + @logger.error('table config value should not be dynamic.', @kusto_ingest.table) + raise LogStash::ConfigurationError.new('table config value should not be dynamic.') + end + if @kusto_ingest.json_mapping =~ FIELD_REF + @logger.error('json_mapping config value should not be dynamic.', @kusto_ingest.json_mapping) + raise LogStash::ConfigurationError.new('json_mapping config value should not be dynamic.') + end + if not(["https", "http"].include? @kusto_proxy.proxy_protocol) + @logger.error('proxy_protocol has to be http or https.', @kusto_proxy.proxy_protocol) + raise LogStash::ConfigurationError.new('proxy_protocol has to be http or https.') + end + + if @kusto_proxy.proxy_aad_only && @kusto_proxy.is_direct_conn + @logger.error('proxy_aad_only is true, but proxy parameters (Host,Port,Protocol) are missing.',@kusto_proxy.proxy_host,@kusto_proxy.proxy_port,@kusto_proxy.proxy_protocol) + raise LogStash::ConfigurationError.new('proxy_aad_only is true, but proxy parameters (Host,Port,Protocol) are missing.') + end + # If all validation pass then configuration is valid + return true + end #validate_config() + + end # class KustoLogstashConfiguration + class KustoAuthConfiguration + def initialize(app_id, app_key, app_tenant, managed_identity_id, cli_auth) + @app_id = app_id + @app_key = app_key + @app_tenant = app_tenant + @managed_identity_id = managed_identity_id + @cli_auth = cli_auth + @is_managed_identity = app_id.to_s.empty? && app_key.to_s.empty? && !cli_auth + @is_system_assigned_managed_identity = is_managed_identity && 0 == "system".casecmp(kusto_auth.managed_identity_id) + end + # Authentication configuration + def app_id + @app_id + end + def app_key + @app_key + end + def app_tenant + @app_tenant + end + def managed_identity_id + @managed_identity_id + end + def is_managed_identity + @is_managed_identity + end + def cli_auth + @cli_auth + end + def is_system_assigned_managed_identity + @is_system_assigned_managed_identity + end + end # class KustoAuthConfiguration + class KustoProxyConfiguration + def initialize(proxy_host , proxy_port , proxy_protocol, proxy_aad_only) + @proxy_host = proxy_host + @proxy_port = proxy_port + @proxy_protocol = proxy_protocol + @proxy_aad_only = proxy_aad_only + # Is it direct connection + @is_direct_conn = (proxy_host.nil? || proxy_host.empty?) + end + # proxy configuration + def proxy_host + @proxy_host + end + + def proxy_port + @proxy_port + end + + def proxy_protocol + @proxy_protocol + end + + def proxy_aad_only + @proxy_aad_only + end + + def is_direct_conn + @is_direct_conn + end + end # class KustoProxyConfiguration + class KustoIngestConfiguration + def initialize(ingest_url, database, table, json_mapping) + @ingest_url = ingest_url + @database = database + @table = table + @json_mapping = json_mapping + @is_mapping_ref_provided = !(json_mapping.nil? || json_mapping.empty?) + end + # For ingestion + def ingest_url + @ingest_url + end + def database + @database + end + def table + @table + end + def json_mapping + @json_mapping + end + def is_mapping_ref_provided + @is_mapping_ref_provided + end + end # class KustoIngestionConfiguration + end # module KustoInternal + end # module Outputs +end # module LogStash diff --git a/spec/outputs/kusto/ingestor_spec.rb b/spec/outputs/kusto/ingestor_spec.rb deleted file mode 100755 index a077549b..00000000 --- a/spec/outputs/kusto/ingestor_spec.rb +++ /dev/null @@ -1,97 +0,0 @@ -# encoding: utf-8 -require_relative "../../spec_helpers.rb" -require 'logstash/outputs/kusto' -require 'logstash/outputs/kusto/ingestor' - -describe LogStash::Outputs::Kusto::Ingestor do - - let(:ingest_url) { "https://ingest-sdkse2etest.eastus.kusto.windows.net/" } - let(:app_id) { "myid" } - let(:app_key) { LogStash::Util::Password.new("mykey") } - let(:app_tenant) { "mytenant" } - let(:managed_identity) { "managed_identity" } - let(:database) { "mydatabase" } - let(:cliauth) { false } - let(:table) { "mytable" } - let(:proxy_host) { "localhost" } - let(:proxy_port) { 80 } - let(:proxy_protocol) { "http" } - let(:json_mapping) { "mymapping" } - let(:logger) { spy('logger') } - - describe 'Ingestor' do - - it 'does not throw an error when initializing' do - RSpec.configuration.reporter.message("Running test: does not throw an error when initializing") - expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port, proxy_protocol, logger) - ingestor.stop - }.not_to raise_error - RSpec.configuration.reporter.message("Completed test: does not throw an error when initializing") - end - - dynamic_name_array = ['/a%{name}/', '/a %{name}/', '/a- %{name}/', '/a- %{name}'] - - context 'doesnt allow database to have some dynamic part' do - dynamic_name_array.each do |test_database| - it "with database: #{test_database}" do - RSpec.configuration.reporter.message("Running test: doesnt allow database to have some dynamic part with database: #{test_database}") - expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, test_database, table, json_mapping, proxy_host, proxy_port, proxy_protocol, logger) - ingestor.stop - }.to raise_error(LogStash::ConfigurationError) - RSpec.configuration.reporter.message("Completed test: doesnt allow database to have some dynamic part with database: #{test_database}") - end - end - end - - context 'doesnt allow table to have some dynamic part' do - dynamic_name_array.each do |test_table| - it "with table: #{test_table}" do - RSpec.configuration.reporter.message("Running test: doesnt allow table to have some dynamic part with table: #{test_table}") - expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, test_table, json_mapping, proxy_host, proxy_port, proxy_protocol, logger) - ingestor.stop - }.to raise_error(LogStash::ConfigurationError) - RSpec.configuration.reporter.message("Completed test: doesnt allow table to have some dynamic part with table: #{test_table}") - end - end - end - - context 'doesnt allow mapping to have some dynamic part' do - dynamic_name_array.each do |json_mapping| - it "with mapping: #{json_mapping}" do - RSpec.configuration.reporter.message("Running test: doesnt allow mapping to have some dynamic part with mapping: #{json_mapping}") - expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port, proxy_protocol, logger) - ingestor.stop - }.to raise_error(LogStash::ConfigurationError) - RSpec.configuration.reporter.message("Completed test: doesnt allow mapping to have some dynamic part with mapping: #{json_mapping}") - end - end - end - - context 'proxy protocol has to be http or https' do - it "with proxy protocol: socks" do - RSpec.configuration.reporter.message("Running test: proxy protocol has to be http or https with proxy protocol: socks") - expect { - ingestor = described_class.new(ingest_url, app_id, app_key, app_tenant, managed_identity, cliauth, database, table, json_mapping, proxy_host, proxy_port, 'socks', logger) - ingestor.stop - }.to raise_error(LogStash::ConfigurationError) - RSpec.configuration.reporter.message("Completed test: proxy protocol has to be http or https with proxy protocol: socks") - end - end - - context 'one of appid or managedid has to be provided' do - it "with empty managed identity and appid" do - RSpec.configuration.reporter.message("Running test: one of appid or managedid has to be provided with empty managed identity and appid") - expect { - ingestor = described_class.new(ingest_url, "", app_key, app_tenant, "", cliauth, database, table, json_mapping, proxy_host, proxy_port,'socks',logger) - ingestor.stop - }.to raise_error(LogStash::ConfigurationError) - RSpec.configuration.reporter.message("Completed test: one of appid or managedid has to be provided with empty managed identity and appid") - end - end - end - -end diff --git a/spec/outputs/kusto/interval_spec.rb b/spec/outputs/kusto/interval_spec.rb new file mode 100644 index 00000000..ccd9cf46 --- /dev/null +++ b/spec/outputs/kusto/interval_spec.rb @@ -0,0 +1,69 @@ +# # spec/interval_test.rb +# require 'rspec' +# require 'logstash/outputs/kusto/interval' + + +# describe LogStash::Outputs::Kusto::Interval do +# let(:interval_time) { 1 } +# let(:procsy) { double("procsy", call: true) } + +# describe '#initialize' do +# it 'initializes with the correct interval and procsy' do +# interval = described_class.new(interval_time, procsy) +# expect(interval.instance_variable_get(:@interval)).to eq(interval_time) +# expect(interval.instance_variable_get(:@procsy)).to eq(procsy) +# end +# end + +# describe '#start' do +# it 'starts the interval thread' do +# interval = described_class.new(interval_time, procsy) +# interval.start +# expect(interval.alive?).to be true +# interval.stop +# end + +# it 'does not start a new thread if already started' do +# interval = described_class.new(interval_time, procsy) +# interval.start +# first_thread = interval.instance_variable_get(:@thread) +# interval.start +# second_thread = interval.instance_variable_get(:@thread) +# expect(first_thread).to eq(second_thread) +# interval.stop +# end +# end + +# describe '#stop' do +# it 'stops the interval thread' do +# interval = described_class.new(interval_time, procsy) +# interval.start +# interval.stop +# expect(interval.alive?).to be false +# end +# end + +# describe '#alive?' do +# it 'returns true if the thread is alive' do +# interval = described_class.new(interval_time, procsy) +# interval.start +# expect(interval.alive?).to be true +# interval.stop +# end + +# it 'returns false if the thread is not alive' do +# interval = described_class.new(interval_time, procsy) +# expect(interval.alive?).to be false +# end +# end + +# describe 'interval execution' do +# it 'calls the proc at the specified interval' do +# interval = described_class.new(interval_time, procsy) +# expect(procsy).to receive(:call).at_least(:twice) +# interval.start +# sleep(2.5) +# interval.stop +# end +# end +# end \ No newline at end of file diff --git a/spec/outputs/kusto/kustoLogstashConfiguration_spec.rb b/spec/outputs/kusto/kustoLogstashConfiguration_spec.rb new file mode 100755 index 00000000..cbb09ea7 --- /dev/null +++ b/spec/outputs/kusto/kustoLogstashConfiguration_spec.rb @@ -0,0 +1,106 @@ +# encoding: utf-8 +require_relative "../../spec_helpers.rb" +require 'logstash/outputs/kusto' +require 'logstash/outputs/kusto/kustoLogstashConfiguration' + +describe LogStash::Outputs::KustoInternal::KustoLogstashConfiguration do + + let(:ingest_url) { "https://ingest-sdkse2etest.eastus.kusto.windows.net/" } + let(:app_id) { "myid" } + let(:app_key) { LogStash::Util::Password.new("mykey") } + let(:app_tenant) { "mytenant" } + let(:managed_identity) { "managed_identity" } + let(:database) { "mydatabase" } + let(:cliauth) { false } + let(:table) { "mytable" } + let(:proxy_host) { "localhost" } + let(:proxy_port) { 80 } + let(:proxy_protocol) { "http" } + let(:json_mapping) { "mymapping" } + let(:delete_local) { false } + let(:logger) { spy(:logger) } + let(:proxy_aad_only) { false } + + let(:kusto_ingest_base) { LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, database, table, json_mapping) } + let(:kusto_auth_base) { LogStash::Outputs::KustoInternal::KustoAuthConfiguration.new(app_id, app_key, app_tenant, managed_identity, cliauth) } + let(:kusto_proxy_base) { LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new(proxy_host , proxy_port , proxy_protocol, false) } + + describe '#initialize' do + it 'does not throw an error when initializing' do + # note that this will cause an internal error since connection is being tried. + # however we still want to test that all the java stuff is working as expected + expect { + kustoLogstashOutputConfiguration = described_class.new(kusto_ingest_base, kusto_auth_base , kusto_proxy_base, logger) + kustoLogstashOutputConfiguration.validate_config() + }.not_to raise_error + end + + dynamic_name_array = ['/a%{name}/', '/a %{name}/', '/a- %{name}/', '/a- %{name}'] + + context 'doesnt allow database to have some dynamic part' do + dynamic_name_array.each do |test_database| + it "with database: #{test_database}" do + expect { + kusto_ingest = LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, test_database, table, json_mapping) + kustoLogstashOutputConfiguration = described_class.new(kusto_ingest, kusto_auth_base , kusto_proxy_base, logger) + kustoLogstashOutputConfiguration.validate_config() + }.to raise_error(LogStash::ConfigurationError) + end + end + end + + context 'doesnt allow table to have some dynamic part' do + dynamic_name_array.each do |test_table| + it "with database: #{test_table}" do + expect { + kusto_ingest = LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, database, test_table, json_mapping) + kustoLogstashOutputConfiguration = described_class.new(kusto_ingest, kusto_auth_base , kusto_proxy_base, logger) + kustoLogstashOutputConfiguration.validate_config() + }.to raise_error(LogStash::ConfigurationError) + end + end + end + + context 'doesnt allow mapping to have some dynamic part' do + dynamic_name_array.each do |json_mapping| + it "with database: #{json_mapping}" do + expect { + kusto_ingest = LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, database, table, json_mapping) + kustoLogstashOutputConfiguration = described_class.new(kusto_ingest, kusto_auth_base , kusto_proxy_base, logger) + kustoLogstashOutputConfiguration.validate_config() + }.to raise_error(LogStash::ConfigurationError) + end + end + end + + context 'proxy protocol has to be http or https' do + it "with proxy protocol: socks" do + expect { + kusto_proxy = LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new(proxy_host , proxy_port , 'socks', false) + kustoLogstashOutputConfiguration = described_class.new(kusto_ingest_base, kusto_auth_base , kusto_proxy, logger) + kustoLogstashOutputConfiguration.validate_config() + }.to raise_error(LogStash::ConfigurationError) + end + end + + context 'one of appid or managedid or cli_auth has to be provided' do + it "with empty managed identity and appid" do + expect { + kusto_auth = LogStash::Outputs::KustoInternal::KustoAuthConfiguration.new("", app_key, app_tenant, "", false) + kustoLogstashOutputConfiguration = described_class.new(kusto_ingest_base, kusto_auth , kusto_proxy_base, logger) + kustoLogstashOutputConfiguration.validate_config() + }.to raise_error(LogStash::ConfigurationError) + end + end + + context 'if proxy_aad is provided' do + it "proxy details should be provided" do + expect { + kusto_proxy = LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new("" , "" , proxy_protocol, true) + kustoLogstashOutputConfiguration = described_class.new(kusto_ingest_base, kusto_auth_base , kusto_proxy, logger) + kustoLogstashOutputConfiguration.validate_config() + }.to raise_error(LogStash::ConfigurationError) + end + end + end +end \ No newline at end of file From f560b093136ae006c1058073581f261b078052f0 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Sun, 27 Oct 2024 15:42:21 +0530 Subject: [PATCH 29/52] Added max_retries and failed_items_path() configs Updated buffer_flush and upload() --- lib/logstash/outputs/kusto.rb | 8 +- .../outputs/kusto/custom_size_based_buffer.rb | 170 +++++++----------- lib/logstash/outputs/kusto/ingestor.rb | 52 ++---- 3 files changed, 88 insertions(+), 142 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index 7d6fb2dd..6b9fd59a 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -75,11 +75,17 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # Maximum interval (in seconds) before the buffer gets flushed, defaults to 10 config :max_interval, validate: :number, default: 10 + # Maximum number of retries before the flush fails, defaults to 3 + config :max_retries, validate: :number, default: 3 + + # Path to store failed items, defaults to nil + config :failed_items_path, validate: :string, default: nil + default :codec, 'json_lines' def register # Initialize the custom buffer with size and interval - @buffer = LogStash::Outputs::CustomSizeBasedBuffer.new(@max_size, @max_interval) do |events| + @buffer = LogStash::Outputs::CustomSizeBasedBuffer.new(@max_size, @max_interval, @max_retries, @failed_items_path) do |events| flush_buffer(events) end diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index 6a012276..646d61a5 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -1,82 +1,74 @@ require 'logger' require 'thread' -require 'fileutils' -require 'securerandom' +require 'csv' module LogStash module Outputs class CustomSizeBasedBuffer - def initialize(max_size_mb, max_interval, &flush_callback) + def initialize(max_size_mb = 10, max_interval = 10, max_retries = 3, failed_items_path = nil, &flush_callback) @buffer_config = { max_size: max_size_mb * 1024 * 1024, # Convert MB to bytes max_interval: max_interval, - buffer_dir: './tmp/buffer_storage/', + max_retries: max_retries, + failed_items_path: failed_items_path, logger: Logger.new(STDOUT) } @buffer_state = { pending_items: [], pending_size: 0, last_flush: Time.now.to_i, - timer: nil, - network_down: false - } - @flush_callback = flush_callback - @shutdown = false - @pending_mutex = Mutex.new - @flush_mutex = Mutex.new - load_buffer_from_files - @buffer_config[:logger].info("CustomSizeBasedBuffer initialized with max_size: #{max_size_mb} MB, max_interval: #{max_interval} seconds") - - # Start the timer thread after a delay to ensure initializations are completed - Thread.new do - sleep(10) - @buffer_state[:timer] = Thread.new do + timer: Thread.new do loop do sleep(@buffer_config[:max_interval]) buffer_flush(force: true) end end - end + } + @flush_callback = flush_callback + @pending_mutex = Mutex.new + @flush_mutex = Mutex.new + @buffer_config[:logger].info("CustomSizeBasedBuffer initialized with max_size: #{max_size_mb} MB, max_interval: #{max_interval} seconds, max_retries: #{max_retries}, failed_items_path: #{failed_items_path}") end - def <<(event) - while buffer_full? do - sleep 0.1 - end + def <<(event) + while buffer_full? do + sleep 0.1 + end @pending_mutex.synchronize do @buffer_state[:pending_items] << event @buffer_state[:pending_size] += event.bytesize end + + # Trigger a flush if the buffer size exceeds the maximum size + if buffer_full? + buffer_flush(force: true) + end end def shutdown @buffer_config[:logger].info("Shutting down buffer") - @shutdown = true @buffer_state[:timer].kill buffer_flush(final: true) - clear_buffer_files end - private + private - def buffer_full? - @pending_mutex.synchronize do - @buffer_state[:pending_size] >= @buffer_config[:max_size] - end - end - - def buffer_flush(options = {}) - force = options[:force] || options[:final] - final = options[:final] + def buffer_full? + @pending_mutex.synchronize do + @buffer_state[:pending_size] >= @buffer_config[:max_size] + end + end - if final - @flush_mutex.lock - elsif !@flush_mutex.try_lock - return 0 - end + def buffer_flush(options = {}) + force = options[:force] || options[:final] + final = options[:final] - items_flushed = 0 + if final + @flush_mutex.lock + elsif !@flush_mutex.try_lock + return 0 + end begin outgoing_items = [] @@ -84,96 +76,68 @@ def buffer_flush(options = {}) @pending_mutex.synchronize do return 0 if @buffer_state[:pending_size] == 0 - - time_since_last_flush = Time.now.to_i - @buffer_state[:last_flush] + time_since_last_flush = Time.now.to_i - @buffer_state[:last_flush] if !force && @buffer_state[:pending_size] < @buffer_config[:max_size] && time_since_last_flush < @buffer_config[:max_interval] return 0 end if force - @buffer_config[:logger].info("Time-based flush triggered after #{@buffer_config[:max_interval]} seconds") - elsif @buffer_state[:pending_size] >= @buffer_config[:max_size] - @buffer_config[:logger].info("Size-based flush triggered at #{@buffer_state[:pending_size]} bytes was reached") - else - @buffer_config[:logger].info("Flush triggered without specific condition") + if time_since_last_flush >= @buffer_config[:max_interval] + @buffer_config[:logger].info("Time-based flush triggered after #{@buffer_config[:max_interval]} seconds") + else + @buffer_config[:logger].info("Size-based flush triggered when #{@buffer_state[:pending_size]} bytes was reached") + end end outgoing_items = @buffer_state[:pending_items].dup outgoing_size = @buffer_state[:pending_size] - @buffer_state[:pending_items] = [] - @buffer_state[:pending_size] = 0 + buffer_initialize end + retries = 0 begin + @buffer_config[:logger].info("Flushing: #{outgoing_items.size} items and #{outgoing_size} bytes to the network") @flush_callback.call(outgoing_items) # Pass the list of events to the callback - @buffer_state[:network_down] = false # Reset network status after successful flush - flush_buffer_files # Flush buffer files if any exist + @buffer_state[:last_flush] = Time.now.to_i + @buffer_config[:logger].info("Flush completed. Flushed #{outgoing_items.size} events, #{outgoing_size} bytes") rescue => e - @buffer_config[:logger].error("Flush failed: #{e.message}") - @buffer_state[:network_down] = true - save_buffer_to_file(outgoing_items) + retries += 1 + if retries <= @buffer_config[:max_retries] + @buffer_config[:logger].error("Flush failed: #{e.message}. \nRetrying (#{retries}/#{@buffer_config[:max_retries]})...") + sleep 1 + retry + else + @buffer_config[:logger].error("Max retries reached. Failed to flush #{outgoing_items.size} items and #{outgoing_size} bytes") + handle_failed_flush(outgoing_items, e.message) + end end - @buffer_state[:last_flush] = Time.now.to_i - @buffer_config[:logger].info("Flush completed. Flushed #{outgoing_items.size} events, #{outgoing_size} bytes") - - items_flushed = outgoing_items.size ensure @flush_mutex.unlock end - - items_flushed - end - - def save_buffer_to_file(events) - buffer_state_copy = { - pending_items: events, - pending_size: events.sum(&:bytesize) - } - - ::FileUtils.mkdir_p(@buffer_config[:buffer_dir]) # Ensure directory exists - file_path = ::File.join(@buffer_config[:buffer_dir], "buffer_state_#{Time.now.to_i}_#{SecureRandom.uuid}.log") - ::File.open(file_path, 'w') do |file| - file.write(Marshal.dump(buffer_state_copy)) - end - @buffer_config[:logger].info("Saved buffer state to file: #{file_path}") end - def load_buffer_from_files - Dir.glob(::File.join(@buffer_config[:buffer_dir], 'buffer_state_*.log')).each do |file_path| + def handle_failed_flush(items, error_message) + if @buffer_config[:failed_items_path] begin - buffer_state = Marshal.load(::File.read(file_path)) - @buffer_state[:pending_items].concat(buffer_state[:pending_items]) - @buffer_state[:pending_size] += buffer_state[:pending_size] - ::File.delete(file_path) - rescue => e - @buffer_config[:logger].error("Failed to load buffer from file: #{e.message}") - end - end - @buffer_config[:logger].info("Loaded buffer state from files") - end - - def flush_buffer_files - Dir.glob(::File.join(@buffer_config[:buffer_dir], 'buffer_state_*.log')).each do |file_path| - begin - buffer_state = Marshal.load(::File.read(file_path)) - @buffer_config[:logger].info("Flushed from file: #{file_path}") - @flush_callback.call(buffer_state[:pending_items]) - ::File.delete(file_path) - @buffer_config[:logger].info("Flushed and deleted buffer state file: #{file_path}") + ::File.open(@buffer_config[:failed_items_path], 'a') do |file| + items.each do |item| + file.puts(item) + end + end + @buffer_config[:logger].info("Failed items stored in #{@buffer_config[:failed_items_path]}") rescue => e - @buffer_config[:logger].error("Failed to flush buffer state file: #{e.message}") - break + @buffer_config[:logger].error("Failed to store items: #{e.message}") end + else + @buffer_config[:logger].warn("No failed_items_path configured. Data loss may occur.") end end - def clear_buffer_files - Dir.glob(::File.join(@buffer_config[:buffer_dir], 'buffer_state_*.log')).each do |file_path| - ::File.delete(file_path) - end - @buffer_config[:logger].info("Cleared all buffer state files") + def buffer_initialize + @buffer_state[:pending_items] = [] + @buffer_state[:pending_size] = 0 end end end diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index 392cb0a1..f06a282a 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -75,7 +75,8 @@ def initialize(kusto_logstash_configuration, logger, threadpool = DEFAULT_THREAD end @ingestion_properties = kusto_java.ingest.IngestionProperties.new(@kusto_logstash_configuration.kusto_ingest.database, @kusto_logstash_configuration.kusto_ingest.table) - + @ingestion_properties.setReportLevel(Java::ComMicrosoftAzureKustoIngest::IngestionProperties::IngestionReportLevel::FAILURES_AND_SUCCESSES) + @ingestion_properties.setReportMethod(Java::ComMicrosoftAzureKustoIngest::IngestionProperties::IngestionReportMethod::TABLE) if @kusto_logstash_configuration.kusto_ingest.is_mapping_ref_provided @logger.debug('Using mapping reference.', @kusto_logstash_configuration.kusto_ingest.json_mapping) @ingestion_properties.setIngestionMapping(@kusto_logstash_configuration.kusto_ingest.json_mapping, kusto_java.ingest.IngestionMapping::IngestionMappingKind::JSON) @@ -101,59 +102,34 @@ def upload_async(data) exception = e end end - # Wait for the task to complete and check for exceptions @workers_pool.shutdown @workers_pool.wait_for_termination - - if exception - @logger.error('StandardError in upload_async.', exception: exception.class, message: exception.message, backtrace: exception.backtrace) - raise exception - end + + raise exception if exception rescue Exception => e @logger.error('StandardError in upload_async.', exception: e.class, message: e.message, backtrace: e.backtrace) raise e end def upload(data) - @logger.info("Sending data to Kusto") - + @logger.debug("Sending data to Kusto") if data.size > 0 - ingestionLatch = java.util.concurrent.CountDownLatch.new(1) - - Thread.new do - begin - data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(java.io.ByteArrayInputStream.new(data.to_java_bytes)) - ingestion_result = @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) - - # Check the ingestion status - status = ingestion_result.getIngestionStatusCollection.get(0) - if status.status != Java::com.microsoft.azure.kusto.ingest.result.OperationStatus::Queued - raise "Failed upload: #{status.errorCodeString}" - end - @logger.info("Final ingestion status: #{status.status}") - rescue => e - @logger.error('Error during ingestFromStream.', exception: e.class, message: e.message, backtrace: e.backtrace) - if e.message.include?("network") - raise e - end - ensure - ingestionLatch.countDown() - end - end - - # Wait for the ingestion to complete with a timeout - if !ingestionLatch.await(30, java.util.concurrent.TimeUnit::SECONDS) - @logger.error('Ingestion timed out, possible network issue.') - raise 'Ingestion timed out, possible network issue.' + begin + data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(java.io.ByteArrayInputStream.new(data.to_java_bytes)) + result = @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) + rescue => e + @logger.error('Uploading failed.', exception: e.class, message: e.message, backtrace: e.backtrace) + raise e end else @logger.warn("Data is empty and is not ingested.") end - @logger.info("Data sent to Kusto.") + + @logger.debug("Data sent to Kusto.") rescue => e @logger.error('Uploading failed.', exception: e.class, message: e.message, backtrace: e.backtrace) - raise e # Raise the original error if ingestion fails + raise e end def stop From 2adc63cf0d96c9d378ba54e6c4210af8f4f24772 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Tue, 5 Nov 2024 15:59:55 +0530 Subject: [PATCH 30/52] Updated README.md and configs Updated failed_items_path to be a required config parameter. If set to "nil", the failed items will not be persisted to local storage, else the items are stored in the file path provided after max_retries. --- README.md | 68 ++++++++++--------- lib/logstash/outputs/kusto.rb | 12 ++-- .../outputs/kusto/custom_size_based_buffer.rb | 10 +-- 3 files changed, 48 insertions(+), 42 deletions(-) diff --git a/README.md b/README.md index b256a266..bd113fbc 100755 --- a/README.md +++ b/README.md @@ -35,19 +35,22 @@ Perform configuration before sending events from Logstash to Azure Data Explorer ```ruby output { - kusto { - path => "/tmp/kusto/%{+YYYY-MM-dd-HH-mm}.txt" - ingest_url => "https://ingest-.kusto.windows.net/" - app_id => "" - app_key => "" - app_tenant => "" - database => "" - table => "" - json_mapping => "" - proxy_host => "" - proxy_port => - proxy_protocol => <"http"|"https"> - } + kusto { + ingest_url => "https://ingest-.kusto.windows.net/" + app_id => "" + app_key => "" + app_tenant => "" + database => "" + table => "" + json_mapping => "" + proxy_host => "" + proxy_port => + proxy_protocol => <"http"|"https"> + max_size => 10 + max_interval => 10 + max_retries => 3 + failed_items_path => "" + } } ``` More information about configuring Logstash can be found in the [logstash configuration guide](https://www.elastic.co/guide/en/logstash/current/configuration.html) @@ -56,22 +59,24 @@ More information about configuring Logstash can be found in the [logstash config | Parameter Name | Description | Notes | | --- | --- | --- | -| **path** | The plugin writes events to temporary files before sending them to ADX. This parameter includes a path where files should be written and a time expression for file rotation to trigger an upload to the ADX service. The example above shows how to rotate the files every minute and check the Logstash docs for more information on time expressions. | Required -| **ingest_url** | The Kusto endpoint for ingestion-related communication. See it on the Azure Portal.| Required| -| **app_id, app_key, app_tenant**| Credentials required to connect to the ADX service. Be sure to use an application with 'ingest' privileges. | Optional| -| **managed_identity**| Managed Identity to authenticate. For user-based managed ID, use the Client ID GUID. For system-based, use the value `system`. The ID needs to have 'ingest' privileges on the cluster. | Optional| -| **database**| Database name to place events | Required | -| **table** | Target table name to place events | Required +| **ingest_url** | The Kusto endpoint for ingestion-related communication. See it on the Azure Portal. | Required | +| **app_id, app_key, app_tenant** | Credentials required to connect to the ADX service. Be sure to use an application with 'ingest' privileges. | Optional | +| **managed_identity** | Managed Identity to authenticate. For user-based managed ID, use the Client ID GUID. For system-based, use the value `system`. The ID needs to have 'ingest' privileges on the cluster. | Optional | +| **database** | Database name to place events | Required | +| **table** | Target table name to place events | Required | +| **failed_items_path** | Path to store failed items when max_retries is reached. Set to nil to disable persistence to file (May cause data loss). | Required | | **json_mapping** | Maps each attribute from incoming event JSON strings to the appropriate column in the table. Note that this must be in JSON format, as this is the interface between Logstash and Kusto | Optional | -| **recovery** | If set to true (default), plugin will attempt to resend pre-existing temp files found in the path upon startup | | -| **delete_temp_files** | Determines if temp files will be deleted after a successful upload (true is default; set false for debug purposes only)| | -| **flush_interval** | The time (in seconds) for flushing writes to temporary files. Default is 2 seconds, 0 will flush on every event. Increase this value to reduce IO calls but keep in mind that events in the buffer will be lost in case of abrupt failure.| | -| **proxy_host** | The proxy hostname for redirecting traffic to Kusto.| | -| **proxy_port** | The proxy port for the proxy. Defaults to 80.| | -| **proxy_protocol** | The proxy server protocol , is one of http or https.| | +| **proxy_host** | The proxy hostname for redirecting traffic to Kusto. | Optional | +| **proxy_port** | The proxy port for the proxy. Defaults to 80. | Optional | +| **proxy_protocol** | The proxy server protocol, is one of http or https. | Optional | +| **max_size** | Maximum size of the buffer before it gets flushed, defaults to 10MB. | Optional | +| **max_interval** | Maximum interval (in seconds) before the buffer gets flushed, defaults to 10. | Optional | +| **max_retries** | Maximum number of retries before the flush fails. Defaults to 3. | Optional | > Note : LS_JAVA_OPTS can be used to set proxy parameters as well (using export or SET options) +> Note: **path** config parameter is no longer used in the latest release (3.0.0) and will be deprecated in future releases + ```bash export LS_JAVA_OPTS="-Dhttp.proxyHost=1.2.34 -Dhttp.proxyPort=8989 -Dhttps.proxyHost=1.2.3.4 -Dhttps.proxyPort=8989" ``` @@ -81,12 +86,13 @@ export LS_JAVA_OPTS="-Dhttp.proxyHost=1.2.34 -Dhttp.proxyPort=8989 -Dhttps.prox | Version | Release Date | Notes | | --- | --- | --- | -| 2.0.8 | 2024-10-23 | - Fix library deprecations, fix issues in the Azure Identity library | -| 2.0.7 | 2024-01-01 | - Update Kusto JAVA SDK | -| 2.0.3 | 2023-12-12 | - Make JSON mapping field optional. If not provided logstash output JSON attribute names will be used for column resolution | -| 2.0.2 | 2023-11-28 | - Bugfix for the scenario where the plugin uses managed identity. Instead of providing the managed identity name as empty in the config,it can completely be skipped | -| 2.0.0 | 2023-09-19 | - Upgrade to the latest Java SDK version [5.0.2](https://github.com/Azure/azure-kusto-java/releases/tag/v5.0.2). Tests have been performed on **__Logstash 8.5__** and up (Does not work with 6.x or 7.x versions of Logstash - For these versions use 1.x.x versions of logstash-output-kusto gem) - Fixes CVE's in common-text & outdated Jackson libraries | -| 1.0.6 | 2022-11-29 | - Upgrade to the latest Java SDK [3.2.1](https://github.com/Azure/azure-kusto-java/releases/tag/v3.2.1) version. Tests have been performed on Logstash 6.x and up.| +| 3.0.0 | 2024-11-01 | Updated configuration options | +| 2.0.8 | 2024-10-23 | Fix library deprecations, fix issues in the Azure Identity library | +| 2.0.7 | 2024-01-01 | Update Kusto JAVA SDK | +| 2.0.3 | 2023-12-12 | Make JSON mapping field optional. If not provided logstash output JSON attribute names will be used for column resolution | +| 2.0.2 | 2023-11-28 | Bugfix for the scenario where the plugin uses managed identity. Instead of providing the managed identity name as empty in the config,it can completely be skipped | +| 2.0.0 | 2023-09-19 | Upgrade to the latest Java SDK version [5.0.2](https://github.com/Azure/azure-kusto-java/releases/tag/v5.0.2). Tests have been performed on **__Logstash 8.5__** and up (Does not work with 6.x or 7.x versions of Logstash - For these versions use 1.x.x versions of logstash-output-kusto gem) - Fixes CVE's in common-text & outdated Jackson libraries | +| 1.0.6 | 2022-11-29 | Upgrade to the latest Java SDK [3.2.1](https://github.com/Azure/azure-kusto-java/releases/tag/v3.2.1) version. Tests have been performed on Logstash 6.x and up.| ## Development Requirements diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index 6b9fd59a..a78e666c 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -39,6 +39,9 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base config :database, validate: :string, required: true # Target table name config :table, validate: :string, required: true + # Path to store failed items when max_retries is reached, set to "nil" to disable persistence to file + config :failed_items_path, validate: :string, required: true + # Mapping name - Used by Kusto to map each attribute from incoming event JSON strings to the appropriate column in the table. # Note that this must be in JSON format, as this is the interface between Logstash and Kusto # Make this optional as name resolution in the JSON mapping can be done based on attribute names in the incoming event JSON strings @@ -70,16 +73,13 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base config :proxy_protocol, validate: :string, required: false , default: 'http' # Maximum size of the buffer before it gets flushed, defaults to 10MB - config :max_size, validate: :number, default: 10 + config :max_size, validate: :number, required: false , default: 10 # Maximum interval (in seconds) before the buffer gets flushed, defaults to 10 - config :max_interval, validate: :number, default: 10 + config :max_interval, validate: :number, required: false , default: 10 # Maximum number of retries before the flush fails, defaults to 3 - config :max_retries, validate: :number, default: 3 - - # Path to store failed items, defaults to nil - config :failed_items_path, validate: :string, default: nil + config :max_retries, validate: :number, required: false , default: 3 default :codec, 'json_lines' diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index 646d61a5..e04478a4 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -109,7 +109,7 @@ def buffer_flush(options = {}) retry else @buffer_config[:logger].error("Max retries reached. Failed to flush #{outgoing_items.size} items and #{outgoing_size} bytes") - handle_failed_flush(outgoing_items, e.message) + handle_failed_flush(outgoing_items) end end @@ -118,8 +118,10 @@ def buffer_flush(options = {}) end end - def handle_failed_flush(items, error_message) - if @buffer_config[:failed_items_path] + def handle_failed_flush(items) + if @buffer_config[:failed_items_path].nil? || @buffer_config[:failed_items_path] == "nil" + @buffer_config[:logger].warn("No failed_items_path configured. The failed items are not persisted. Data loss may occur.") + else begin ::File.open(@buffer_config[:failed_items_path], 'a') do |file| items.each do |item| @@ -130,8 +132,6 @@ def handle_failed_flush(items, error_message) rescue => e @buffer_config[:logger].error("Failed to store items: #{e.message}") end - else - @buffer_config[:logger].warn("No failed_items_path configured. Data loss may occur.") end end From 42a0075c05080880fb04b17ba071c6657e8db959 Mon Sep 17 00:00:00 2001 From: Ramachandran G Date: Mon, 11 Nov 2024 18:57:42 +0530 Subject: [PATCH 31/52] *Make some fixes for retries and upload --- lib/logstash/outputs/kusto.rb | 1 - lib/logstash/outputs/kusto/ingestor.rb | 61 +++++++++++++------------- 2 files changed, 30 insertions(+), 32 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index a78e666c..fba279ca 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -136,7 +136,6 @@ def close @logger.error("Error stopping ingestor: #{e.message}") @logger.error(e.backtrace.join("\n")) end - @logger.info("Kusto output plugin Closed") end diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index f06a282a..0f9a6c21 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -21,6 +21,8 @@ class Ingestor FIELD_REF = /%\{[^}]+\}/ def initialize(kusto_logstash_configuration, logger, threadpool = DEFAULT_THREADPOOL) + @retry_count = 3 + @retry_delay = 10 @workers_pool = threadpool @logger = logger #Validate and assign @@ -85,6 +87,15 @@ def initialize(kusto_logstash_configuration, logger, threadpool = DEFAULT_THREAD @logger.debug('No mapping reference provided. Columns will be mapped by names in the logstash output') @ingestion_properties.setDataFormat(kusto_java.ingest.IngestionProperties::DataFormat::JSON) end + # retry_policy = Java::com.azure.storage.common.policy + # duration = Java::java.time.Duration.ofSeconds(5) + + # fixed_delay_options = Java::com.azure.core.http.policy.FixedDelayOptions.new(1,duration) + # retry_options = Java::com.azure.core.http.policy.RetryOptions.new(fixed_delay_options) + # req_retry_options = Java::com.azure.storage.common.policy.RequestRetryOptions.fromRetryOptions(retry_options, Java::java.time.Duration.ofSeconds(10), "") + + # queued_ingest_client = @kusto_client.to_java(Java::com.microsoft.azure.kusto.ingest.QueuedIngestClientImpl) + # queued_ingest_client.setQueueRequestOptions(req_retry_options) @logger.debug('Kusto resources are ready.') end @@ -92,44 +103,32 @@ def upload_async(data) if @workers_pool.remaining_capacity <= LOW_QUEUE_LENGTH @logger.warn("Ingestor queue capacity is running low with #{@workers_pool.remaining_capacity} free slots.") end - exception = nil @workers_pool.post do - LogStash::Util.set_thread_name("Kusto to ingest data") - begin - upload(data) - rescue => e - @logger.error('Error during async upload.', exception: e.class, message: e.message, backtrace: e.backtrace) - exception = e - end + LogStash::Util.set_thread_name("Kusto to ingest data #{JRuby.reference(Thread.current).native_thread.id}") + upload(data) end - # Wait for the task to complete and check for exceptions - @workers_pool.shutdown - @workers_pool.wait_for_termination - - raise exception if exception - rescue Exception => e - @logger.error('StandardError in upload_async.', exception: e.class, message: e.message, backtrace: e.backtrace) - raise e end def upload(data) - @logger.debug("Sending data to Kusto") - if data.size > 0 - begin - data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(java.io.ByteArrayInputStream.new(data.to_java_bytes)) - result = @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) - rescue => e - @logger.error('Uploading failed.', exception: e.class, message: e.message, backtrace: e.backtrace) - raise e + begin + @logger.debug("Sending data to Kusto") + if data.size > 0 + data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(java.io.ByteArrayInputStream.new(data.to_java_bytes)) + result = @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) + else + @logger.warn("Data is empty and is not ingested.") + end + @logger.debug("Data sent to Kusto.") + rescue => e + if tries < @retry_count + tries += 1 + logger.warn("Uploading failed, retrying (##{tries} of #{@retry_count})", :exception => e.class, :message => e.message, :path => file.path, :backtrace => e.backtrace) + sleep @retry_delay + retry + else + logger.error("Failed to upload file (retried #{@retry_count} times).", :exception => e.class, :message => e.message, :path => file.path, :backtrace => e.backtrace) end - else - @logger.warn("Data is empty and is not ingested.") end - - @logger.debug("Data sent to Kusto.") - rescue => e - @logger.error('Uploading failed.', exception: e.class, message: e.message, backtrace: e.backtrace) - raise e end def stop From f2520953882fa528ae3f85a44dea98b3d44e0a3f Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Thu, 12 Dec 2024 17:56:09 +0530 Subject: [PATCH 32/52] Added Latch Timeout and updated java sdk version --- build.gradle | 6 +- lib/logstash/outputs/kusto.rb | 28 +-- .../outputs/kusto/custom_size_based_buffer.rb | 199 ++++++++++++------ lib/logstash/outputs/kusto/ingestor.rb | 81 ++++--- 4 files changed, 197 insertions(+), 117 deletions(-) diff --git a/build.gradle b/build.gradle index f14b1e31..4b4fc702 100644 --- a/build.gradle +++ b/build.gradle @@ -29,8 +29,8 @@ repositories { // update dependencies to bom azure-sdk-bom/1.2.24 dependencies { - implementation 'com.microsoft.azure.kusto:kusto-data:5.2.0' - implementation 'com.microsoft.azure.kusto:kusto-ingest:5.2.0' + implementation 'com.microsoft.azure.kusto:kusto-data:6.0.0' + implementation 'com.microsoft.azure.kusto:kusto-ingest:6.0.0' implementation 'com.azure:azure-core-http-netty:1.15.1' implementation 'com.azure:azure-core:1.49.1' implementation 'com.azure:azure-data-tables:12.4.2' @@ -52,7 +52,7 @@ dependencies { implementation 'com.nimbusds:nimbus-jose-jwt:9.40' implementation 'com.nimbusds:oauth2-oidc-sdk:11.13' implementation 'com.univocity:univocity-parsers:2.9.1' - implementation 'commons-codec:commons-codec:1.16.1' + implementation 'commons-codec:commons-codec:1.17.1' implementation 'commons-logging:commons-logging:1.3.1' implementation 'io.github.resilience4j:resilience4j-core:1.7.1' implementation 'io.github.resilience4j:resilience4j-retry:1.7.1' diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index fba279ca..d2eef627 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -39,9 +39,6 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base config :database, validate: :string, required: true # Target table name config :table, validate: :string, required: true - # Path to store failed items when max_retries is reached, set to "nil" to disable persistence to file - config :failed_items_path, validate: :string, required: true - # Mapping name - Used by Kusto to map each attribute from incoming event JSON strings to the appropriate column in the table. # Note that this must be in JSON format, as this is the interface between Logstash and Kusto # Make this optional as name resolution in the JSON mapping can be done based on attribute names in the incoming event JSON strings @@ -78,17 +75,13 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # Maximum interval (in seconds) before the buffer gets flushed, defaults to 10 config :max_interval, validate: :number, required: false , default: 10 - # Maximum number of retries before the flush fails, defaults to 3 - config :max_retries, validate: :number, required: false , default: 3 + # Latch timeout in seconds, defaults to 60 + config :latch_timeout, validate: :number, required: false, default: 60 + default :codec, 'json_lines' def register - # Initialize the custom buffer with size and interval - @buffer = LogStash::Outputs::CustomSizeBasedBuffer.new(@max_size, @max_interval, @max_retries, @failed_items_path) do |events| - flush_buffer(events) - end - @io_mutex = Mutex.new final_mapping = json_mapping @@ -98,13 +91,22 @@ def register max_threads: upload_concurrent_count, max_queue: upload_queue_size, fallback_policy: :caller_runs) - + kusto_ingest_base = LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, database, table, final_mapping) kusto_auth_base = LogStash::Outputs::KustoInternal::KustoAuthConfiguration.new(app_id, app_key, app_tenant, managed_identity, cli_auth) kusto_proxy_base = LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new(proxy_host , proxy_port , proxy_protocol, false) @kusto_logstash_configuration = LogStash::Outputs::KustoInternal::KustoLogstashConfiguration.new(kusto_ingest_base, kusto_auth_base , kusto_proxy_base, logger) - @ingestor = Ingestor.new(@kusto_logstash_configuration, @logger, executor) + @ingestor = Ingestor.new(@kusto_logstash_configuration, @logger, latch_timeout, executor) + # Deprecation warning for path + if @path + @logger.warn("The 'path' configuration option is deprecated and will be removed in a future release.") + end + sleep(30) + # Initialize the custom buffer with size and interval + @buffer = LogStash::Outputs::CustomSizeBasedBuffer.new(@max_size, @max_interval) do |events| + flush_buffer(events) + end end @@ -121,7 +123,7 @@ def multi_receive_encoded(events_and_encoded) def close @logger.info("Closing Kusto output plugin") - begin + begin @buffer.shutdown unless @buffer.nil? @logger.info("Buffer shutdown") unless @buffer.nil? rescue => e diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index e04478a4..ce4b8a6d 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -1,37 +1,46 @@ require 'logger' require 'thread' -require 'csv' +require 'fileutils' +require 'securerandom' +require 'net/http' +require 'uri' module LogStash module Outputs class CustomSizeBasedBuffer - def initialize(max_size_mb = 10, max_interval = 10, max_retries = 3, failed_items_path = nil, &flush_callback) + def initialize(max_size_mb, max_interval, &flush_callback) @buffer_config = { max_size: max_size_mb * 1024 * 1024, # Convert MB to bytes max_interval: max_interval, - max_retries: max_retries, - failed_items_path: failed_items_path, + buffer_dir: './tmp/buffer_storage/', logger: Logger.new(STDOUT) } @buffer_state = { pending_items: [], pending_size: 0, last_flush: Time.now.to_i, - timer: Thread.new do - loop do - sleep(@buffer_config[:max_interval]) - buffer_flush(force: true) - end - end + timer: nil, + network_down: false } @flush_callback = flush_callback + @shutdown = false @pending_mutex = Mutex.new @flush_mutex = Mutex.new - @buffer_config[:logger].info("CustomSizeBasedBuffer initialized with max_size: #{max_size_mb} MB, max_interval: #{max_interval} seconds, max_retries: #{max_retries}, failed_items_path: #{failed_items_path}") + load_buffer_from_files + @buffer_config[:logger].info("CustomSizeBasedBuffer initialized with max_size: #{max_size_mb} MB, max_interval: #{max_interval} seconds") + + # Start the timer thread + @buffer_state[:timer] = Thread.new do + loop do + sleep(@buffer_config[:max_interval]) + prepare_flush(force: true) + end + end end def <<(event) while buffer_full? do + prepare_flush(force: true) # Flush when buffer is full sleep 0.1 end @@ -39,17 +48,14 @@ def <<(event) @buffer_state[:pending_items] << event @buffer_state[:pending_size] += event.bytesize end - - # Trigger a flush if the buffer size exceeds the maximum size - if buffer_full? - buffer_flush(force: true) - end end def shutdown @buffer_config[:logger].info("Shutting down buffer") + @shutdown = true @buffer_state[:timer].kill - buffer_flush(final: true) + prepare_flush(final: true) + flush_buffer_files end private @@ -60,56 +66,85 @@ def buffer_full? end end - def buffer_flush(options = {}) + def prepare_flush(options = {}) force = options[:force] || options[:final] final = options[:final] - if final - @flush_mutex.lock - elsif !@flush_mutex.try_lock - return 0 + outgoing_items = [] + outgoing_size = 0 + + @pending_mutex.synchronize do + return 0 if @buffer_state[:pending_size] == 0 + time_since_last_flush = Time.now.to_i - @buffer_state[:last_flush] + + if !force && @buffer_state[:pending_size] < @buffer_config[:max_size] && time_since_last_flush < @buffer_config[:max_interval] + return 0 + end + + if time_since_last_flush >= @buffer_config[:max_interval] + @buffer_config[:logger].info("Time-based flush triggered after #{@buffer_config[:max_interval]} seconds") + else + @buffer_config[:logger].info("Size-based flush triggered at #{@buffer_state[:pending_size]} bytes was reached") + end + + if @buffer_state[:network_down] + save_buffer_to_file(@buffer_state[:pending_items]) + @buffer_state[:pending_items] = [] + @buffer_state[:pending_size] = 0 + return 0 + end + + outgoing_items = @buffer_state[:pending_items].dup + outgoing_size = @buffer_state[:pending_size] + @buffer_state[:pending_items] = [] + @buffer_state[:pending_size] = 0 end - begin - outgoing_items = [] - outgoing_size = 0 + if Dir.glob(::File.join(@buffer_config[:buffer_dir], 'buffer_state_*.log')).any? + @buffer_config[:logger].info("Flushing all buffer state files") + flush_buffer_files + end - @pending_mutex.synchronize do - return 0 if @buffer_state[:pending_size] == 0 - time_since_last_flush = Time.now.to_i - @buffer_state[:last_flush] + Thread.new { perform_flush(outgoing_items) } + end - if !force && @buffer_state[:pending_size] < @buffer_config[:max_size] && time_since_last_flush < @buffer_config[:max_interval] + def perform_flush(events, file_path = nil) + + @flush_mutex.lock + + begin + if file_path + unless ::File.exist?(file_path) return 0 end - - if force - if time_since_last_flush >= @buffer_config[:max_interval] - @buffer_config[:logger].info("Time-based flush triggered after #{@buffer_config[:max_interval]} seconds") - else - @buffer_config[:logger].info("Size-based flush triggered when #{@buffer_state[:pending_size]} bytes was reached") - end + begin + buffer_state = Marshal.load(::File.read(file_path)) + events = buffer_state[:pending_items] + rescue => e + @buffer_config[:logger].error("Failed to load buffer from file: #{e.message}") + return 0 end - - outgoing_items = @buffer_state[:pending_items].dup - outgoing_size = @buffer_state[:pending_size] - buffer_initialize end - - retries = 0 - begin - @buffer_config[:logger].info("Flushing: #{outgoing_items.size} items and #{outgoing_size} bytes to the network") - @flush_callback.call(outgoing_items) # Pass the list of events to the callback - @buffer_state[:last_flush] = Time.now.to_i - @buffer_config[:logger].info("Flush completed. Flushed #{outgoing_items.size} events, #{outgoing_size} bytes") - rescue => e - retries += 1 - if retries <= @buffer_config[:max_retries] - @buffer_config[:logger].error("Flush failed: #{e.message}. \nRetrying (#{retries}/#{@buffer_config[:max_retries]})...") - sleep 1 + @buffer_config[:logger].info("Flushing #{events.size} events, #{events.sum(&:bytesize)} bytes") + @flush_callback.call(events) # Pass the list of events to the callback + @buffer_state[:network_down] = false # Reset network status after successful flush + @buffer_state[:last_flush] = Time.now.to_i + @buffer_config[:logger].info("Flush completed. Flushed #{events.size} events, #{events.sum(&:bytesize)} bytes") + + if file_path + ::File.delete(file_path) + @buffer_config[:logger].info("Flushed and deleted buffer state file: #{file_path}") + end + + rescue => e + @buffer_config[:logger].error("Flush failed: #{e.message}") + @buffer_state[:network_down] = true + + while true + sleep(2) # Wait before checking network availability again + if network_available? + @buffer_config[:logger].info("Network is back up. Retrying flush.") retry - else - @buffer_config[:logger].error("Max retries reached. Failed to flush #{outgoing_items.size} items and #{outgoing_size} bytes") - handle_failed_flush(outgoing_items) end end @@ -117,27 +152,53 @@ def buffer_flush(options = {}) @flush_mutex.unlock end end + + def network_available? + begin + uri = URI('http://www.google.com') + response = Net::HTTP.get_response(uri) + response.is_a?(Net::HTTPSuccess) + rescue + false + end + end - def handle_failed_flush(items) - if @buffer_config[:failed_items_path].nil? || @buffer_config[:failed_items_path] == "nil" - @buffer_config[:logger].warn("No failed_items_path configured. The failed items are not persisted. Data loss may occur.") - else + def save_buffer_to_file(events) + buffer_state_copy = { + pending_items: events, + pending_size: events.sum(&:bytesize) + } + begin + ::FileUtils.mkdir_p(@buffer_config[:buffer_dir]) # Ensure directory exists + file_path = ::File.join(@buffer_config[:buffer_dir], "buffer_state_#{Time.now.to_i}_#{SecureRandom.uuid}.log") + ::File.open(file_path, 'w') do |file| + file.write(Marshal.dump(buffer_state_copy)) + end + @buffer_config[:logger].info("Saved #{events.size} events to file: #{file_path}") + rescue => e + @buffer_config[:logger].error("Failed to save buffer to file: #{e.message}") + end + end + + def load_buffer_from_files + Dir.glob(::File.join(@buffer_config[:buffer_dir], 'buffer_state_*.log')).each do |file_path| begin - ::File.open(@buffer_config[:failed_items_path], 'a') do |file| - items.each do |item| - file.puts(item) - end - end - @buffer_config[:logger].info("Failed items stored in #{@buffer_config[:failed_items_path]}") + buffer_state = Marshal.load(::File.read(file_path)) + @buffer_state[:pending_items].concat(buffer_state[:pending_items]) + @buffer_state[:pending_size] += buffer_state[:pending_size] + ::File.delete(file_path) rescue => e - @buffer_config[:logger].error("Failed to store items: #{e.message}") + @buffer_config[:logger].error("Failed to load buffer from file: #{e.message}") end end + @buffer_config[:logger].info("Loaded buffer state from files") end - def buffer_initialize - @buffer_state[:pending_items] = [] - @buffer_state[:pending_size] = 0 + def flush_buffer_files + Dir.glob(::File.join(@buffer_config[:buffer_dir], 'buffer_state_*.log')).each do |file_path| + @buffer_config[:logger].info("Flushing from buffer state file: #{file_path}") + Thread.new { perform_flush([], file_path) } + end end end end diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index 0f9a6c21..b0c2d426 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -20,15 +20,13 @@ class Ingestor LOW_QUEUE_LENGTH = 3 FIELD_REF = /%\{[^}]+\}/ - def initialize(kusto_logstash_configuration, logger, threadpool = DEFAULT_THREADPOOL) - @retry_count = 3 - @retry_delay = 10 + def initialize(kusto_logstash_configuration, logger, latch_timeout = 60, threadpool = DEFAULT_THREADPOOL) @workers_pool = threadpool + @latch_timeout = latch_timeout @logger = logger #Validate and assign kusto_logstash_configuration.validate_config() @kusto_logstash_configuration = kusto_logstash_configuration - @logger.info('Preparing Kusto resources.') kusto_java = Java::com.microsoft.azure.kusto @@ -77,8 +75,6 @@ def initialize(kusto_logstash_configuration, logger, threadpool = DEFAULT_THREAD end @ingestion_properties = kusto_java.ingest.IngestionProperties.new(@kusto_logstash_configuration.kusto_ingest.database, @kusto_logstash_configuration.kusto_ingest.table) - @ingestion_properties.setReportLevel(Java::ComMicrosoftAzureKustoIngest::IngestionProperties::IngestionReportLevel::FAILURES_AND_SUCCESSES) - @ingestion_properties.setReportMethod(Java::ComMicrosoftAzureKustoIngest::IngestionProperties::IngestionReportMethod::TABLE) if @kusto_logstash_configuration.kusto_ingest.is_mapping_ref_provided @logger.debug('Using mapping reference.', @kusto_logstash_configuration.kusto_ingest.json_mapping) @ingestion_properties.setIngestionMapping(@kusto_logstash_configuration.kusto_ingest.json_mapping, kusto_java.ingest.IngestionMapping::IngestionMappingKind::JSON) @@ -87,15 +83,6 @@ def initialize(kusto_logstash_configuration, logger, threadpool = DEFAULT_THREAD @logger.debug('No mapping reference provided. Columns will be mapped by names in the logstash output') @ingestion_properties.setDataFormat(kusto_java.ingest.IngestionProperties::DataFormat::JSON) end - # retry_policy = Java::com.azure.storage.common.policy - # duration = Java::java.time.Duration.ofSeconds(5) - - # fixed_delay_options = Java::com.azure.core.http.policy.FixedDelayOptions.new(1,duration) - # retry_options = Java::com.azure.core.http.policy.RetryOptions.new(fixed_delay_options) - # req_retry_options = Java::com.azure.storage.common.policy.RequestRetryOptions.fromRetryOptions(retry_options, Java::java.time.Duration.ofSeconds(10), "") - - # queued_ingest_client = @kusto_client.to_java(Java::com.microsoft.azure.kusto.ingest.QueuedIngestClientImpl) - # queued_ingest_client.setQueueRequestOptions(req_retry_options) @logger.debug('Kusto resources are ready.') end @@ -103,32 +90,62 @@ def upload_async(data) if @workers_pool.remaining_capacity <= LOW_QUEUE_LENGTH @logger.warn("Ingestor queue capacity is running low with #{@workers_pool.remaining_capacity} free slots.") end + exception = nil @workers_pool.post do - LogStash::Util.set_thread_name("Kusto to ingest data #{JRuby.reference(Thread.current).native_thread.id}") - upload(data) + LogStash::Util.set_thread_name("Kusto to ingest data") + begin + upload(data) + rescue => e + @logger.error('Error during async upload.', exception: e.class, message: e.message, backtrace: e.backtrace) + exception = e + end + end + # Wait for the task to complete and check for exceptions + @workers_pool.shutdown + @workers_pool.wait_for_termination + + if exception + @logger.error('StandardError in upload_async.', exception: exception.class, message: exception.message, backtrace: exception.backtrace) + raise exception end + rescue Exception => e + @logger.error('StandardError in upload_async.', exception: e.class, message: e.message, backtrace: e.backtrace) + raise e end def upload(data) - begin - @logger.debug("Sending data to Kusto") - if data.size > 0 + @logger.info("Sending data to Kusto") + + if data.size > 0 + ingestionLatch = java.util.concurrent.CountDownLatch.new(1) + thread_exception = nil + + Thread.new do + begin data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(java.io.ByteArrayInputStream.new(data.to_java_bytes)) - result = @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) - else - @logger.warn("Data is empty and is not ingested.") + ingestion_result = @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) + rescue => e + @logger.error('Error during ingestFromStream.', exception: e.class, message: e.message, backtrace: e.backtrace) + thread_exception = e + ensure + ingestionLatch.countDown() + end end - @logger.debug("Data sent to Kusto.") - rescue => e - if tries < @retry_count - tries += 1 - logger.warn("Uploading failed, retrying (##{tries} of #{@retry_count})", :exception => e.class, :message => e.message, :path => file.path, :backtrace => e.backtrace) - sleep @retry_delay - retry - else - logger.error("Failed to upload file (retried #{@retry_count} times).", :exception => e.class, :message => e.message, :path => file.path, :backtrace => e.backtrace) + + # Wait for the ingestion to complete with a timeout + if !ingestionLatch.await(@latch_timeout, java.util.concurrent.TimeUnit::SECONDS) + @logger.error('Ingestion timed out, possible network issue.') + raise 'Ingestion timed out, possible network issue.' end + # Raise the exception from the thread if it occurred + raise thread_exception if thread_exception + else + @logger.warn("Data is empty and is not ingested.") end + @logger.info("Data sent to Kusto.") + rescue => e + @logger.error('Uploading failed.', exception: e.class, message: e.message, backtrace: e.backtrace) + raise e # Raise the original error if ingestion fails end def stop From 357439f27d70249183e1bd53bb66fd04a343cbe7 Mon Sep 17 00:00:00 2001 From: MonishkaDas Date: Thu, 26 Dec 2024 11:23:28 +0530 Subject: [PATCH 33/52] Add completableFuture with timeout --- lib/logstash/outputs/kusto/ingestor.rb | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index b0c2d426..969653f9 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -115,28 +115,29 @@ def upload_async(data) def upload(data) @logger.info("Sending data to Kusto") - + if data.size > 0 - ingestionLatch = java.util.concurrent.CountDownLatch.new(1) thread_exception = nil - Thread.new do + future = java.util.concurrent.CompletableFuture.supplyAsync do begin data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(java.io.ByteArrayInputStream.new(data.to_java_bytes)) - ingestion_result = @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) + @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) rescue => e @logger.error('Error during ingestFromStream.', exception: e.class, message: e.message, backtrace: e.backtrace) thread_exception = e - ensure - ingestionLatch.countDown() end end - # Wait for the ingestion to complete with a timeout - if !ingestionLatch.await(@latch_timeout, java.util.concurrent.TimeUnit::SECONDS) + begin + future.get(@latch_timeout, java.util.concurrent.TimeUnit::SECONDS) + rescue java.util.concurrent.TimeoutException => e @logger.error('Ingestion timed out, possible network issue.') - raise 'Ingestion timed out, possible network issue.' + thread_exception = 'Ingestion timed out, possible network issue.' + rescue java.util.concurrent.ExecutionException => e + thread_exception = e.cause end + # Raise the exception from the thread if it occurred raise thread_exception if thread_exception else From 20dc04e9e046b277447c6995336c8aa16d561f80 Mon Sep 17 00:00:00 2001 From: monishkadas_microsoft Date: Thu, 2 Jan 2025 14:09:03 +0530 Subject: [PATCH 34/52] Updated README.md and removed interval.rb --- README.md | 7 +-- lib/logstash/outputs/kusto.rb | 1 - lib/logstash/outputs/kusto/interval.rb | 81 -------------------------- 3 files changed, 2 insertions(+), 87 deletions(-) delete mode 100755 lib/logstash/outputs/kusto/interval.rb diff --git a/README.md b/README.md index bd113fbc..c3e2f1e0 100755 --- a/README.md +++ b/README.md @@ -48,8 +48,7 @@ output { proxy_protocol => <"http"|"https"> max_size => 10 max_interval => 10 - max_retries => 3 - failed_items_path => "" + latch_timeout => 60 } } ``` @@ -64,14 +63,12 @@ More information about configuring Logstash can be found in the [logstash config | **managed_identity** | Managed Identity to authenticate. For user-based managed ID, use the Client ID GUID. For system-based, use the value `system`. The ID needs to have 'ingest' privileges on the cluster. | Optional | | **database** | Database name to place events | Required | | **table** | Target table name to place events | Required | -| **failed_items_path** | Path to store failed items when max_retries is reached. Set to nil to disable persistence to file (May cause data loss). | Required | | **json_mapping** | Maps each attribute from incoming event JSON strings to the appropriate column in the table. Note that this must be in JSON format, as this is the interface between Logstash and Kusto | Optional | | **proxy_host** | The proxy hostname for redirecting traffic to Kusto. | Optional | | **proxy_port** | The proxy port for the proxy. Defaults to 80. | Optional | | **proxy_protocol** | The proxy server protocol, is one of http or https. | Optional | | **max_size** | Maximum size of the buffer before it gets flushed, defaults to 10MB. | Optional | -| **max_interval** | Maximum interval (in seconds) before the buffer gets flushed, defaults to 10. | Optional | -| **max_retries** | Maximum number of retries before the flush fails. Defaults to 3. | Optional | +| **latch_timeout** | Latch timeout in seconds, defaults to 60. This is the maximum wait time after which the flushing attempt is timed out and the network is considered to be down. The system waits for the network to be back to retry flushing the same batch. | Optional | > Note : LS_JAVA_OPTS can be used to set proxy parameters as well (using export or SET options) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index d2eef627..6a4537c0 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -5,7 +5,6 @@ require 'logstash/errors' require 'logstash/outputs/kusto/ingestor' -require 'logstash/outputs/kusto/interval' require 'logstash/outputs/kusto/custom_size_based_buffer' require 'logstash/outputs/kusto/kustoLogstashConfiguration' diff --git a/lib/logstash/outputs/kusto/interval.rb b/lib/logstash/outputs/kusto/interval.rb deleted file mode 100755 index 6ba8d4a4..00000000 --- a/lib/logstash/outputs/kusto/interval.rb +++ /dev/null @@ -1,81 +0,0 @@ -# encoding: utf-8 - -require 'logstash/outputs/base' -require 'logstash/namespace' -require 'logstash/errors' - -class LogStash::Outputs::Kusto < LogStash::Outputs::Base - ## - # Bare-bones utility for running a block of code at an interval. - # - class Interval - ## - # Initializes a new Interval with the given arguments and starts it - # before returning it. - # - # @param interval [Integer] (see: Interval#initialize) - # @param procsy [#call] (see: Interval#initialize) - # - # @return [Interval] - # - def self.start(interval, procsy) - new(interval, procsy).tap(&:start) - end - - ## - # @param interval [Integer]: time in seconds to wait between calling the given proc - # @param procsy [#call]: proc or lambda to call periodically; must not raise exceptions. - def initialize(interval, procsy) - @interval = interval - @procsy = procsy - - # Mutex, ConditionVariable, etc. - @mutex = Mutex.new - @sleeper = ConditionVariable.new - end - - ## - # Starts the interval, or returns if it has already been started. - # - # @return [void] - def start - @mutex.synchronize do - return if @thread && @thread.alive? - - @thread = Thread.new { run } - end - end - - ## - # Stop the interval. - # Does not interrupt if execution is in-progress. - def stop - @mutex.synchronize do - @stopped = true - end - - @thread && @thread.join - end - - ## - # @return [Boolean] - def alive? - @thread && @thread.alive? - end - - private - - def run - @mutex.synchronize do - loop do - @sleeper.wait(@mutex, @interval) - break if @stopped - - @procsy.call - end - end - ensure - @sleeper.broadcast - end - end -end From 7292e27e931198f3a39ccd48e316d1645e88bc86 Mon Sep 17 00:00:00 2001 From: monishkadas_microsoft Date: Thu, 2 Jan 2025 15:04:46 +0530 Subject: [PATCH 35/52] Removed network_available method --- .../outputs/kusto/custom_size_based_buffer.rb | 21 +-- spec/outputs/kusto/interval_spec.rb | 69 -------- spec/outputs/kusto_spec.rb | 159 ------------------ 3 files changed, 3 insertions(+), 246 deletions(-) delete mode 100644 spec/outputs/kusto/interval_spec.rb delete mode 100755 spec/outputs/kusto_spec.rb diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb index ce4b8a6d..7ce13db2 100644 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb @@ -139,29 +139,14 @@ def perform_flush(events, file_path = nil) rescue => e @buffer_config[:logger].error("Flush failed: #{e.message}") @buffer_state[:network_down] = true - - while true - sleep(2) # Wait before checking network availability again - if network_available? - @buffer_config[:logger].info("Network is back up. Retrying flush.") - retry - end - end - + sleep(120) # Wait before checking network availability again + @buffer_config[:logger].info("Retrying flush.") + retry ensure @flush_mutex.unlock end end - def network_available? - begin - uri = URI('http://www.google.com') - response = Net::HTTP.get_response(uri) - response.is_a?(Net::HTTPSuccess) - rescue - false - end - end def save_buffer_to_file(events) buffer_state_copy = { diff --git a/spec/outputs/kusto/interval_spec.rb b/spec/outputs/kusto/interval_spec.rb deleted file mode 100644 index ccd9cf46..00000000 --- a/spec/outputs/kusto/interval_spec.rb +++ /dev/null @@ -1,69 +0,0 @@ -# # spec/interval_test.rb -# require 'rspec' -# require 'logstash/outputs/kusto/interval' - - -# describe LogStash::Outputs::Kusto::Interval do -# let(:interval_time) { 1 } -# let(:procsy) { double("procsy", call: true) } - -# describe '#initialize' do -# it 'initializes with the correct interval and procsy' do -# interval = described_class.new(interval_time, procsy) -# expect(interval.instance_variable_get(:@interval)).to eq(interval_time) -# expect(interval.instance_variable_get(:@procsy)).to eq(procsy) -# end -# end - -# describe '#start' do -# it 'starts the interval thread' do -# interval = described_class.new(interval_time, procsy) -# interval.start -# expect(interval.alive?).to be true -# interval.stop -# end - -# it 'does not start a new thread if already started' do -# interval = described_class.new(interval_time, procsy) -# interval.start -# first_thread = interval.instance_variable_get(:@thread) -# interval.start -# second_thread = interval.instance_variable_get(:@thread) -# expect(first_thread).to eq(second_thread) -# interval.stop -# end -# end - -# describe '#stop' do -# it 'stops the interval thread' do -# interval = described_class.new(interval_time, procsy) -# interval.start -# interval.stop -# expect(interval.alive?).to be false -# end -# end - -# describe '#alive?' do -# it 'returns true if the thread is alive' do -# interval = described_class.new(interval_time, procsy) -# interval.start -# expect(interval.alive?).to be true -# interval.stop -# end - -# it 'returns false if the thread is not alive' do -# interval = described_class.new(interval_time, procsy) -# expect(interval.alive?).to be false -# end -# end - -# describe 'interval execution' do -# it 'calls the proc at the specified interval' do -# interval = described_class.new(interval_time, procsy) -# expect(procsy).to receive(:call).at_least(:twice) -# interval.start -# sleep(2.5) -# interval.stop -# end -# end -# end \ No newline at end of file diff --git a/spec/outputs/kusto_spec.rb b/spec/outputs/kusto_spec.rb deleted file mode 100755 index 967f9a11..00000000 --- a/spec/outputs/kusto_spec.rb +++ /dev/null @@ -1,159 +0,0 @@ -# encoding: utf-8 -require_relative "../spec_helpers.rb" -require 'logstash/outputs/kusto' -require 'logstash/codecs/plain' -require 'logstash/event' - -describe LogStash::Outputs::Kusto do - - let(:options) { { "path" => "./kusto_tst/%{+YYYY-MM-dd-HH-mm}", - "ingest_url" => "https://ingest-sdkse2etest.eastus.kusto.windows.net/", - "app_id" => "myid", - "app_key" => "mykey", - "app_tenant" => "mytenant", - "database" => "mydatabase", - "table" => "mytable", - "json_mapping" => "mymapping", - "proxy_host" => "localhost", - "proxy_port" => 3128, - "proxy_protocol" => "https", - "max_size" => 2000, - "max_interval" => 10 - } } - - describe '#initialize' do - it 'initializes with the correct options' do - RSpec.configuration.reporter.message("Running test: initializes with the correct options") - kusto = described_class.new(options.merge("app_key" => LogStash::Util::Password.new("mykey"))) - expect(kusto.instance_variable_get(:@path)).to eq("./kusto_tst/%{+YYYY-MM-dd-HH-mm}") - expect(kusto.instance_variable_get(:@ingest_url)).to eq("https://ingest-sdkse2etest.eastus.kusto.windows.net/") - expect(kusto.instance_variable_get(:@app_id)).to eq("myid") - expect(kusto.instance_variable_get(:@app_key).value).to eq("mykey") - expect(kusto.instance_variable_get(:@app_tenant)).to eq("mytenant") - expect(kusto.instance_variable_get(:@database)).to eq("mydatabase") - expect(kusto.instance_variable_get(:@table)).to eq("mytable") - expect(kusto.instance_variable_get(:@json_mapping)).to eq("mymapping") - expect(kusto.instance_variable_get(:@proxy_host)).to eq("localhost") - expect(kusto.instance_variable_get(:@proxy_port)).to eq(3128) - expect(kusto.instance_variable_get(:@proxy_protocol)).to eq("https") - expect(kusto.instance_variable_get(:@max_size)).to eq(2000) - expect(kusto.instance_variable_get(:@max_interval)).to eq(10) - RSpec.configuration.reporter.message("Completed test: initializes with the correct options") - end - end - - describe '#multi_receive_encoded' do - it 'processes events and adds them to the buffer' do - RSpec.configuration.reporter.message("Running test: processes events and adds them to the buffer") - kusto = described_class.new(options) - kusto.register - - events = [LogStash::Event.new("message" => "test1"), LogStash::Event.new("message" => "test2")] - encoded_events = events.map { |e| [e, e.to_json] } - - # Temporarily disable automatic flushing for the test - buffer = kusto.instance_variable_get(:@buffer) - allow(buffer).to receive(:buffer_flush) - - # Clear the buffer before the test - buffer.instance_variable_set(:@buffer_state, { pending_items: [], pending_size: 0, last_flush: Time.now.to_i }) - - kusto.multi_receive_encoded(encoded_events) - - pending_items = buffer.instance_variable_get(:@buffer_state)[:pending_items] - RSpec.configuration.reporter.message("Pending items in buffer: #{pending_items.inspect}") - - expect(pending_items.size).to eq(2) - RSpec.configuration.reporter.message("Completed test: processes events and adds them to the buffer") - end - - it 'handles errors during event processing' do - RSpec.configuration.reporter.message("Running test: handles errors during event processing") - kusto = described_class.new(options) - kusto.register - - allow(kusto.instance_variable_get(:@buffer)).to receive(:<<).and_raise(StandardError.new("Test error")) - events = [LogStash::Event.new("message" => "test1")] - encoded_events = events.map { |e| [e, e.to_json] } - - expect { kusto.multi_receive_encoded(encoded_events) }.not_to raise_error - RSpec.configuration.reporter.message("Completed test: handles errors during event processing") - end - end - - describe '#register' do - it 'raises an error for invalid configurations' do - RSpec.configuration.reporter.message("Running test: raises an error for invalid configurations") - invalid_options = options.merge("ingest_url" => nil) - expect { described_class.new(invalid_options).register }.to raise_error(LogStash::ConfigurationError) - RSpec.configuration.reporter.message("Completed test: raises an error for invalid configurations") - end - end - - describe '#flush_buffer' do - - it 'flushes the buffer when max_size is reached' do - RSpec.configuration.reporter.message("Running test: flushes the buffer when max_size is reached") - kusto = described_class.new(options.merge("max_size" => 1)) # Set max_size to 1MB for testing - kusto.register - - events = [LogStash::Event.new("message" => "test1")] - encoded_events = events.map { |e| [e, e.to_json] } - # Ensure upload_async is called only once - expect(kusto.instance_variable_get(:@ingestor)).to receive(:upload_async).with(anything).once - kusto.multi_receive_encoded(encoded_events) - - # Trigger the buffer flush manually - buffer = kusto.instance_variable_get(:@buffer) - buffer.send(:buffer_flush, force: true) - - RSpec.configuration.reporter.message("Completed test: flushes the buffer when max_size is reached") - end - - it 'flushes the buffer when max_interval is reached' do - RSpec.configuration.reporter.message("Running test: flushes the buffer when max_interval is reached") - kusto = described_class.new(options.merge("max_interval" => 1)) # Set max_interval to 1 second for testing - kusto.register - - events = [LogStash::Event.new("message" => "test1")] - encoded_events = events.map { |e| [e, e.to_json] } - kusto.multi_receive_encoded(encoded_events) - sleep(2) # Wait for the interval to pass - - expect(kusto.instance_variable_get(:@ingestor)).to receive(:upload_async).with(anything) - kusto.flush_buffer(encoded_events) # Pass the encoded events here - RSpec.configuration.reporter.message("Completed test: flushes the buffer when max_interval is reached") - end - - it 'eventually flushes without receiving additional events based on max_interval' do - RSpec.configuration.reporter.message("Running test: eventually flushes without receiving additional events based on max_interval") - kusto = described_class.new(options.merge("max_interval" => 1)) # Set max_interval to 1 second for testing - kusto.register - - events = [LogStash::Event.new("message" => "test1")] - encoded_events = events.map { |e| [e, e.to_json] } - kusto.multi_receive_encoded(encoded_events) - - # Wait for the interval to pass - sleep(2) - - expect(kusto.instance_variable_get(:@ingestor)).to receive(:upload_async).with(anything) - kusto.flush_buffer(encoded_events) # Pass the encoded events here - RSpec.configuration.reporter.message("Completed test: eventually flushes without receiving additional events based on max_interval") - end - end - - describe '#close' do - it 'shuts down the buffer and ingestor' do - RSpec.configuration.reporter.message("Running test: shuts down the buffer and ingestor") - kusto = described_class.new(options) - kusto.register - - expect(kusto.instance_variable_get(:@buffer)).to receive(:shutdown) - expect(kusto.instance_variable_get(:@ingestor)).to receive(:stop) - - kusto.close - RSpec.configuration.reporter.message("Completed test: shuts down the buffer and ingestor") - end - end -end \ No newline at end of file From 61e47d19b05b6008dc4675c09884e32240cf6d70 Mon Sep 17 00:00:00 2001 From: ag-ramachandran Date: Thu, 20 Mar 2025 14:11:32 +0530 Subject: [PATCH 36/52] * Initial commit --- build.sh | 11 + lib/logstash/outputs/kusto.rb | 205 +++++------- .../outputs/kusto/customSizeBasedBuffer.rb | 303 ++++++++++++++++++ .../outputs/kusto/custom_size_based_buffer.rb | 190 ----------- lib/logstash/outputs/kusto/ingestor.rb | 97 ++---- .../kusto/kustoLogstashConfiguration.rb | 44 ++- .../outputs/kusto/logStashFlushBuffer.rb | 60 ++++ random-data.sh | 10 + run.sh | 1 + version | 2 +- 10 files changed, 543 insertions(+), 380 deletions(-) create mode 100755 build.sh create mode 100644 lib/logstash/outputs/kusto/customSizeBasedBuffer.rb delete mode 100644 lib/logstash/outputs/kusto/custom_size_based_buffer.rb create mode 100644 lib/logstash/outputs/kusto/logStashFlushBuffer.rb create mode 100755 random-data.sh create mode 100755 run.sh diff --git a/build.sh b/build.sh new file mode 100755 index 00000000..65a0f823 --- /dev/null +++ b/build.sh @@ -0,0 +1,11 @@ +export LOGSTASH_SOURCE=1 +export LOGSTASH_PATH=/softwares/logstash +export JRUBY_HOME=$LOGSTASH_PATH/vendor/jruby +export JAVA_HOME=$LOGSTASH_PATH/jdk +export PATH=$PATH:/softwares/logstash/vendor/jruby/bin:/softwares/logstash/bin +jruby -S gem install bundler -v 2.4.19 +jruby -S bundle install +gem build *.gemspec +rm Gemfile.lock +/softwares/logstash/bin/logstash-plugin uninstall logstash-output-kusto +/softwares/logstash/bin/logstash-plugin install logstash-output-kusto-3.0.0-java.gem \ No newline at end of file diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index 6a4537c0..66523cb7 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -4,154 +4,119 @@ require 'logstash/namespace' require 'logstash/errors' -require 'logstash/outputs/kusto/ingestor' -require 'logstash/outputs/kusto/custom_size_based_buffer' +require 'logstash/outputs/kusto/customSizeBasedBuffer' require 'logstash/outputs/kusto/kustoLogstashConfiguration' +require 'logstash/outputs/kusto/logStashFlushBuffer' ## # This plugin sends messages to Azure Kusto in batches. # class LogStash::Outputs::Kusto < LogStash::Outputs::Base - config_name 'kusto' - concurrency :shared + config_name 'kusto' + concurrency :shared - FIELD_REF = /%\{[^}]+\}/ + FIELD_REF = /%\{[^}]+\}/ - attr_reader :failure_path + attr_reader :failure_path - # The Kusto endpoint for ingestion related communication. You can see it on the Azure Portal. - config :ingest_url, validate: :string, required: true + # The Kusto endpoint for ingestion related communication. You can see it on the Azure Portal. + config :ingest_url, validate: :string, required: true + # The following are the credentials used to connect to the Kusto service + # application id + config :app_id, validate: :string, required: false + # application key (secret) + config :app_key, validate: :password, required: false + # aad tenant id + config :app_tenant, validate: :string, default: nil + # managed identity id + config :managed_identity, validate: :string, default: nil + # CLI credentials for dev-test + config :cli_auth, validate: :boolean, default: false + # The following are the data settings that impact where events are written to + # Database name + config :database, validate: :string, required: true + # Target table name + config :table, validate: :string, required: true + # Mapping name - Used by Kusto to map each attribute from incoming event JSON strings to the appropriate column in the table. + # Note that this must be in JSON format, as this is the interface between Logstash and Kusto + # Make this optional as name resolution in the JSON mapping can be done based on attribute names in the incoming event JSON strings + config :json_mapping, validate: :string, default: nil - # The following are the credentials used to connect to the Kusto service - # application id - config :app_id, validate: :string, required: false - # application key (secret) - config :app_key, validate: :password, required: false - # aad tenant id - config :app_tenant, validate: :string, default: nil - # managed identity id - config :managed_identity, validate: :string, default: nil - # CLI credentials for dev-test - config :cli_auth, validate: :boolean, default: false - # The following are the data settings that impact where events are written to - # Database name - config :database, validate: :string, required: true - # Target table name - config :table, validate: :string, required: true - # Mapping name - Used by Kusto to map each attribute from incoming event JSON strings to the appropriate column in the table. - # Note that this must be in JSON format, as this is the interface between Logstash and Kusto - # Make this optional as name resolution in the JSON mapping can be done based on attribute names in the incoming event JSON strings - config :json_mapping, validate: :string, default: nil + # Mapping name - deprecated, use json_mapping + config :mapping, validate: :string, deprecated: true - # Mapping name - deprecated, use json_mapping - config :mapping, validate: :string, deprecated: true - # Path - deprecated - config :path, validate: :string, deprecated: true + # TODO: will be used to route events to many tables according to event properties + config :dynamic_event_routing, validate: :boolean, default: false - # TODO: will be used to route events to many tables according to event properties - config :dynamic_event_routing, validate: :boolean, default: false + # Specify how many files can be uploaded concurrently + config :upload_concurrent_count, validate: :number, default: 3 - # Specify how many files can be uploaded concurrently - config :upload_concurrent_count, validate: :number, default: 3 + # Specify how many files can be kept in the upload queue before the main process + # starts processing them in the main thread (not healthy) + config :upload_queue_size, validate: :number, default: 30 - # Specify how many files can be kept in the upload queue before the main process - # starts processing them in the main thread (not healthy) - config :upload_queue_size, validate: :number, default: 30 + # Host of the proxy , is an optional field. Can connect directly + config :proxy_host, validate: :string, required: false - # Host of the proxy , is an optional field. Can connect directly - config :proxy_host, validate: :string, required: false + # Port where the proxy runs , defaults to 80. Usually a value like 3128 + config :proxy_port, validate: :number, required: false , default: 80 - # Port where the proxy runs , defaults to 80. Usually a value like 3128 - config :proxy_port, validate: :number, required: false , default: 80 + # Check Proxy URL can be over http or https. Do we need it this way or ignore this & remove this + config :proxy_protocol, validate: :string, required: false , default: 'https' - # Check Proxy URL can be over http or https. Do we need it this way or ignore this & remove this - config :proxy_protocol, validate: :string, required: false , default: 'http' + # Maximum size of the buffer before it gets flushed, defaults to 10MB + config :max_batch_size, validate: :number, required: false , default: 10 - # Maximum size of the buffer before it gets flushed, defaults to 10MB - config :max_size, validate: :number, required: false , default: 10 + # Maximum interval (in seconds) before the buffer gets flushed, defaults to 10 + config :plugin_flush_interval, validate: :number, required: false , default: 10 - # Maximum interval (in seconds) before the buffer gets flushed, defaults to 10 - config :max_interval, validate: :number, required: false , default: 10 + # Maximum interval (in seconds) before the buffer gets flushed, defaults to 10 + config :max_items, validate: :number, required: false , default: 100 - # Latch timeout in seconds, defaults to 60 - config :latch_timeout, validate: :number, required: false, default: 60 - - default :codec, 'json_lines' + default :codec, 'json_lines' def register - @io_mutex = Mutex.new - - final_mapping = json_mapping - final_mapping = mapping if final_mapping.nil? || final_mapping.empty? - - executor = Concurrent::ThreadPoolExecutor.new(min_threads: 1, - max_threads: upload_concurrent_count, - max_queue: upload_queue_size, - fallback_policy: :caller_runs) - - kusto_ingest_base = LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, database, table, final_mapping) + kusto_ingest_base = LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, database, table, json_mapping) kusto_auth_base = LogStash::Outputs::KustoInternal::KustoAuthConfiguration.new(app_id, app_key, app_tenant, managed_identity, cli_auth) kusto_proxy_base = LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new(proxy_host , proxy_port , proxy_protocol, false) - @kusto_logstash_configuration = LogStash::Outputs::KustoInternal::KustoLogstashConfiguration.new(kusto_ingest_base, kusto_auth_base , kusto_proxy_base, logger) - @ingestor = Ingestor.new(@kusto_logstash_configuration, @logger, latch_timeout, executor) - - # Deprecation warning for path - if @path - @logger.warn("The 'path' configuration option is deprecated and will be removed in a future release.") - end - sleep(30) + kusto_flush_config = LogStash::Outputs::KustoInternal::KustoFlushConfiguration.new(max_items, plugin_flush_interval,max_batch_size) + kusto_upload_config = LogStash::Outputs::KustoInternal::KustoUploadConfiguration.new(upload_concurrent_count, upload_queue_size) + kusto_logstash_configuration = LogStash::Outputs::KustoInternal::KustoLogstashConfiguration.new(kusto_ingest_base, kusto_auth_base , kusto_proxy_base, kusto_flush_config, kusto_upload_config,@logger) + kusto_logstash_configuration.validate_config # Initialize the custom buffer with size and interval - @buffer = LogStash::Outputs::CustomSizeBasedBuffer.new(@max_size, @max_interval) do |events| - flush_buffer(events) - end - end - - - public - def multi_receive_encoded(events_and_encoded) - events_and_encoded.each do |event, encoded| - begin - @buffer << encoded - rescue => e - @logger.error("Error processing event: #{e.message}") - end - end + @buffer = LogStash::Outputs::KustoOutputInternal::LogStashEventsBatcher.new(kusto_logstash_configuration,@logger) end - def close - @logger.info("Closing Kusto output plugin") - begin - @buffer.shutdown unless @buffer.nil? - @logger.info("Buffer shutdown") unless @buffer.nil? - rescue => e - @logger.error("Error shutting down buffer: #{e.message}") - @logger.error(e.backtrace.join("\n")) - end - - begin - @ingestor.stop unless @ingestor.nil? - @logger.info("Ingestor stopped") unless @ingestor.nil? - rescue => e - @logger.error("Error stopping ingestor: #{e.message}") - @logger.error(e.backtrace.join("\n")) - end - @logger.info("Kusto output plugin Closed") - end - - public - def flush_buffer(events) - return if events.empty? - @logger.info("flush_buffer with #{events.size} events") - begin - # Logic to send buffer to Kusto - @ingestor.upload_async(events.join) - rescue => e - @logger.error("Error during flush: #{e.message}") - @logger.error(e.backtrace.join("\n")) - raise e # Exception is raised to trigger the rescue block in buffer_flush - end - end + public + def multi_receive_encoded(events_and_encoded) + events_and_encoded.each do |event, encoded| + begin + @buffer.batch_event(event.to_hash) + rescue => e + @logger.error("Error processing event: #{e.message}") + end + end + end + + def close + # @logger.info("Closing Kusto output plugin") + # begin + # @buffer.shutdown unless @buffer.nil? + # @logger.info("Buffer shutdown") unless @buffer.nil? + # rescue => e + # @logger.error("Error shutting down buffer: #{e.message}") + # @logger.error(e.backtrace.join("\n")) + # end + # begin + # @ingestor.stop unless @ingestor.nil? + # @logger.info("Ingestor stopped") unless @ingestor.nil? + # rescue => e + # @logger.error("Error stopping ingestor: #{e.message}") + # @logger.error(e.backtrace.join("\n")) + # end + # @logger.info("Kusto output plugin Closed") + end end diff --git a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb new file mode 100644 index 00000000..4ee48978 --- /dev/null +++ b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb @@ -0,0 +1,303 @@ + # This code is from a PR for the official repo of ruby-stud + # with a small change to calculating the event size in the var_size function + # https://github.com/jordansissel/ruby-stud/pull/19 + # + # @author {Alex Dean}[http://github.com/alexdean] + # + # Implements a generic framework for accepting events which are later flushed + # in batches. Flushing occurs whenever +:max_items+ or +:max_interval+ (seconds) + # has been reached or if the event size outgrows +:flush_each+ (bytes) + # + # Including class must implement +flush+, which will be called with all + # accumulated items either when the output buffer fills (+:max_items+ or + # +:flush_each+) or when a fixed amount of time (+:max_interval+) passes. + # + # == batch_receive and flush + # General receive/flush can be implemented in one of two ways. + # + # === batch_receive(event) / flush(events) + # +flush+ will receive an array of events which were passed to +buffer_receive+. + # + # batch_receive('one') + # batch_receive('two') + # + # will cause a flush invocation like + # + # flush(['one', 'two']) + # + # === batch_receive(event, group) / flush(events, group) + # flush() will receive an array of events, plus a grouping key. + # + # batch_receive('one', :server => 'a') + # batch_receive('two', :server => 'b') + # batch_receive('three', :server => 'a') + # batch_receive('four', :server => 'b') + # + # will result in the following flush calls + # + # flush(['one', 'three'], {:server => 'a'}) + # flush(['two', 'four'], {:server => 'b'}) + # + # Grouping keys can be anything which are valid Hash keys. (They don't have to + # be hashes themselves.) Strings or Fixnums work fine. Use anything which you'd + # like to receive in your +flush+ method to help enable different handling for + # various groups of events. + # + # == on_flush_error + # Including class may implement +on_flush_error+, which will be called with an + # Exception instance whenever buffer_flush encounters an error. + # + # * +buffer_flush+ will automatically re-try failed flushes, so +on_flush_error+ + # should not try to implement retry behavior. + # * Exceptions occurring within +on_flush_error+ are not handled by + # +buffer_flush+. + # + # == on_full_buffer_receive + # Including class may implement +on_full_buffer_receive+, which will be called + # whenever +buffer_receive+ is called while the buffer is full. + # + # +on_full_buffer_receive+ will receive a Hash like {:pending => 30, + # :outgoing => 20} which describes the internal state of the module at + # the moment. + # + # == final flush + # Including class should call buffer_flush(:final => true) + # during a teardown/shutdown routine (after the last call to buffer_receive) + # to ensure that all accumulated messages are flushed. + module LogStash; module Outputs; class KustoOutputInternal + module CustomSizeBasedBuffer + + public + # Initialize the buffer. + # + # Call directly from your constructor if you wish to set some non-default + # options. Otherwise buffer_initialize will be called automatically during the + # first buffer_receive call. + # + # Options: + # * :max_items, Max number of items to buffer before flushing. Default 50. + # * :flush_each, Flush each bytes of buffer. Default 0 (no flushing fired by + # a buffer size). + # * :max_interval, Max number of seconds to wait between flushes. Default 5. + # * :logger, A logger to write log messages to. No default. Optional. + # + # @param [Hash] options + def buffer_initialize(options={}) + if ! self.class.method_defined?(:flush) + raise ArgumentError, "Any class including Stud::Buffer must define a flush() method." + end + + @buffer_config = { + :max_items => options[:max_items] || 50, + :flush_each => options[:flush_each].to_i || 0, + :max_interval => options[:max_interval] || 5, + :logger => options[:logger] || nil, + :has_on_flush_error => self.class.method_defined?(:on_flush_error), + :has_on_full_buffer_receive => self.class.method_defined?(:on_full_buffer_receive) + } + @buffer_state = { + # items accepted from including class + :pending_items => {}, + :pending_count => 0, + :pending_size => 0, + + # guard access to pending_items & pending_count & pending_size + :pending_mutex => Mutex.new, + + # items which are currently being flushed + :outgoing_items => {}, + :outgoing_count => 0, + :outgoing_size => 0, + + # ensure only 1 flush is operating at once + :flush_mutex => Mutex.new, + + # data for timed flushes + :last_flush => Time.now.to_i, + :timer => Thread.new do + loop do + sleep(@buffer_config[:max_interval]) + buffer_flush(:force => true) + end + end + } + + # events we've accumulated + buffer_clear_pending + end + + # Determine if +:max_items+ or +:flush_each+ has been reached. + # + # buffer_receive calls will block while buffer_full? == true. + # + # @return [bool] Is the buffer full? + def buffer_full? + @buffer_config[:logger].info("---------------Entering buffer_full?-----------------") + @buffer_config[:logger].info("Pending count: #{@buffer_state[:pending_count]}") + @buffer_config[:logger].info("Outgoing count: #{@buffer_state[:outgoing_count]}") + @buffer_config[:logger].info("Pending size: #{@buffer_state[:pending_size]}") + @buffer_config[:logger].info("Outgoing size: #{@buffer_state[:outgoing_size]}") + @buffer_config[:logger].info("Max items: #{@buffer_config[:max_items]}") + @buffer_config[:logger].info("Flush each: #{@buffer_config[:flush_each]}") + @buffer_config[:logger].info("Max interval: #{@buffer_config[:max_interval]}") + @buffer_config[:logger].info("Pending items: #{@buffer_state[:pending_items]}") + @buffer_config[:logger].info("Outgoing items: #{@buffer_state[:outgoing_items]}") + @buffer_config[:logger].info("---------------Exiting buffer_full?-----------------") + (@buffer_state[:pending_count] + @buffer_state[:outgoing_count] >= @buffer_config[:max_items]) || \ + (@buffer_config[:flush_each] != 0 && @buffer_state[:pending_size] + @buffer_state[:outgoing_size] >= @buffer_config[:flush_each]) + end + + # Save an event for later delivery + # + # Events are grouped by the (optional) group parameter you provide. + # Groups of events, plus the group name, are later passed to +flush+. + # + # This call will block if +:max_items+ or +:flush_each+ has been reached. + # + # @see Stud::Buffer The overview has more information on grouping and flushing. + # + # @param event An item to buffer for flushing later. + # @param group Optional grouping key. All events with the same key will be + # passed to +flush+ together, along with the grouping key itself. + def buffer_receive(event, group=nil) + buffer_initialize if ! @buffer_state + # block if we've accumulated too many events + while buffer_full? do + on_full_buffer_receive( + :pending => @buffer_state[:pending_count], + :outgoing => @buffer_state[:outgoing_count] + ) if @buffer_config[:has_on_full_buffer_receive] + sleep 0.1 + end + @buffer_state[:pending_mutex].synchronize do + @buffer_state[:pending_items][group] << event + @buffer_state[:pending_count] += 1 + @buffer_state[:pending_size] += var_size(event) if @buffer_config[:flush_each] != 0 + end + buffer_flush + end + + # Try to flush events. + # + # Returns immediately if flushing is not necessary/possible at the moment: + # * :max_items or :flush_each have not been accumulated + # * :max_interval seconds have not elapased since the last flush + # * another flush is in progress + # + # buffer_flush(:force => true) will cause a flush to occur even + # if +:max_items+ or +:flush_each+ or +:max_interval+ have not been reached. A forced flush + # will still return immediately (without flushing) if another flush is + # currently in progress. + # + # buffer_flush(:final => true) is identical to buffer_flush(:force => true), + # except that if another flush is already in progress, buffer_flush(:final => true) + # will block/wait for the other flush to finish before proceeding. + # + # @param [Hash] options Optional. May be {:force => true} or {:final => true}. + # @return [Fixnum] The number of items successfully passed to +flush+. + def buffer_flush(options={}) + force = options[:force] || options[:final] + final = options[:final] + + # final flush will wait for lock, so we are sure to flush out all buffered events + if options[:final] + @buffer_state[:flush_mutex].lock + elsif ! @buffer_state[:flush_mutex].try_lock # failed to get lock, another flush already in progress + return 0 + end + + items_flushed = 0 + + begin + return 0 if @buffer_state[:pending_count] == 0 + + # compute time_since_last_flush only when some item is pending + time_since_last_flush = get_time_since_last_flush + + return 0 if (!force) && + (@buffer_state[:pending_count] < @buffer_config[:max_items]) && + (@buffer_config[:flush_each] == 0 || @buffer_state[:pending_size] < @buffer_config[:flush_each]) && + (time_since_last_flush < @buffer_config[:max_interval]) + + @buffer_state[:pending_mutex].synchronize do + @buffer_state[:outgoing_items] = @buffer_state[:pending_items] + @buffer_state[:outgoing_count] = @buffer_state[:pending_count] + @buffer_state[:outgoing_size] = @buffer_state[:pending_size] + buffer_clear_pending + end + @buffer_config[:logger].debug("Flushing output", + :outgoing_count => @buffer_state[:outgoing_count], + :time_since_last_flush => time_since_last_flush, + :outgoing_events => @buffer_state[:outgoing_items], + :batch_timeout => @buffer_config[:max_interval], + :force => force, + :final => final + ) if @buffer_config[:logger] + + @buffer_state[:outgoing_items].each do |group, events| + begin + + if group.nil? + flush(events,final) + else + flush(events, group, final) + end + + @buffer_state[:outgoing_items].delete(group) + events_size = events.size + @buffer_state[:outgoing_count] -= events_size + if @buffer_config[:flush_each] != 0 + events_volume = 0 + events.each do |event| + events_volume += var_size(event) + end + @buffer_state[:outgoing_size] -= events_volume + end + items_flushed += events_size + + rescue => e + @buffer_config[:logger].warn("Failed to flush outgoing items", + :outgoing_count => @buffer_state[:outgoing_count], + :exception => e, + :backtrace => e.backtrace + ) if @buffer_config[:logger] + + if @buffer_config[:has_on_flush_error] + on_flush_error e + end + + sleep 1 + retry + end + @buffer_state[:last_flush] = Time.now.to_i + end + + ensure + @buffer_state[:flush_mutex].unlock + end + + return items_flushed + end + + + private + def buffer_clear_pending + @buffer_state[:pending_items] = Hash.new { |h, k| h[k] = [] } + @buffer_state[:pending_count] = 0 + @buffer_state[:pending_size] = 0 + end + + private + def var_size(var) + # Calculate event size as a json. + # assuming event is a hash + return var.to_json.bytesize + 2 + end + + protected + def get_time_since_last_flush + Time.now.to_i - @buffer_state[:last_flush] + end + + end +end ;end ;end \ No newline at end of file diff --git a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb b/lib/logstash/outputs/kusto/custom_size_based_buffer.rb deleted file mode 100644 index 7ce13db2..00000000 --- a/lib/logstash/outputs/kusto/custom_size_based_buffer.rb +++ /dev/null @@ -1,190 +0,0 @@ -require 'logger' -require 'thread' -require 'fileutils' -require 'securerandom' -require 'net/http' -require 'uri' - -module LogStash - module Outputs - class CustomSizeBasedBuffer - def initialize(max_size_mb, max_interval, &flush_callback) - @buffer_config = { - max_size: max_size_mb * 1024 * 1024, # Convert MB to bytes - max_interval: max_interval, - buffer_dir: './tmp/buffer_storage/', - logger: Logger.new(STDOUT) - } - @buffer_state = { - pending_items: [], - pending_size: 0, - last_flush: Time.now.to_i, - timer: nil, - network_down: false - } - @flush_callback = flush_callback - @shutdown = false - @pending_mutex = Mutex.new - @flush_mutex = Mutex.new - load_buffer_from_files - @buffer_config[:logger].info("CustomSizeBasedBuffer initialized with max_size: #{max_size_mb} MB, max_interval: #{max_interval} seconds") - - # Start the timer thread - @buffer_state[:timer] = Thread.new do - loop do - sleep(@buffer_config[:max_interval]) - prepare_flush(force: true) - end - end - end - - def <<(event) - while buffer_full? do - prepare_flush(force: true) # Flush when buffer is full - sleep 0.1 - end - - @pending_mutex.synchronize do - @buffer_state[:pending_items] << event - @buffer_state[:pending_size] += event.bytesize - end - end - - def shutdown - @buffer_config[:logger].info("Shutting down buffer") - @shutdown = true - @buffer_state[:timer].kill - prepare_flush(final: true) - flush_buffer_files - end - - private - - def buffer_full? - @pending_mutex.synchronize do - @buffer_state[:pending_size] >= @buffer_config[:max_size] - end - end - - def prepare_flush(options = {}) - force = options[:force] || options[:final] - final = options[:final] - - outgoing_items = [] - outgoing_size = 0 - - @pending_mutex.synchronize do - return 0 if @buffer_state[:pending_size] == 0 - time_since_last_flush = Time.now.to_i - @buffer_state[:last_flush] - - if !force && @buffer_state[:pending_size] < @buffer_config[:max_size] && time_since_last_flush < @buffer_config[:max_interval] - return 0 - end - - if time_since_last_flush >= @buffer_config[:max_interval] - @buffer_config[:logger].info("Time-based flush triggered after #{@buffer_config[:max_interval]} seconds") - else - @buffer_config[:logger].info("Size-based flush triggered at #{@buffer_state[:pending_size]} bytes was reached") - end - - if @buffer_state[:network_down] - save_buffer_to_file(@buffer_state[:pending_items]) - @buffer_state[:pending_items] = [] - @buffer_state[:pending_size] = 0 - return 0 - end - - outgoing_items = @buffer_state[:pending_items].dup - outgoing_size = @buffer_state[:pending_size] - @buffer_state[:pending_items] = [] - @buffer_state[:pending_size] = 0 - end - - if Dir.glob(::File.join(@buffer_config[:buffer_dir], 'buffer_state_*.log')).any? - @buffer_config[:logger].info("Flushing all buffer state files") - flush_buffer_files - end - - Thread.new { perform_flush(outgoing_items) } - end - - def perform_flush(events, file_path = nil) - - @flush_mutex.lock - - begin - if file_path - unless ::File.exist?(file_path) - return 0 - end - begin - buffer_state = Marshal.load(::File.read(file_path)) - events = buffer_state[:pending_items] - rescue => e - @buffer_config[:logger].error("Failed to load buffer from file: #{e.message}") - return 0 - end - end - @buffer_config[:logger].info("Flushing #{events.size} events, #{events.sum(&:bytesize)} bytes") - @flush_callback.call(events) # Pass the list of events to the callback - @buffer_state[:network_down] = false # Reset network status after successful flush - @buffer_state[:last_flush] = Time.now.to_i - @buffer_config[:logger].info("Flush completed. Flushed #{events.size} events, #{events.sum(&:bytesize)} bytes") - - if file_path - ::File.delete(file_path) - @buffer_config[:logger].info("Flushed and deleted buffer state file: #{file_path}") - end - - rescue => e - @buffer_config[:logger].error("Flush failed: #{e.message}") - @buffer_state[:network_down] = true - sleep(120) # Wait before checking network availability again - @buffer_config[:logger].info("Retrying flush.") - retry - ensure - @flush_mutex.unlock - end - end - - - def save_buffer_to_file(events) - buffer_state_copy = { - pending_items: events, - pending_size: events.sum(&:bytesize) - } - begin - ::FileUtils.mkdir_p(@buffer_config[:buffer_dir]) # Ensure directory exists - file_path = ::File.join(@buffer_config[:buffer_dir], "buffer_state_#{Time.now.to_i}_#{SecureRandom.uuid}.log") - ::File.open(file_path, 'w') do |file| - file.write(Marshal.dump(buffer_state_copy)) - end - @buffer_config[:logger].info("Saved #{events.size} events to file: #{file_path}") - rescue => e - @buffer_config[:logger].error("Failed to save buffer to file: #{e.message}") - end - end - - def load_buffer_from_files - Dir.glob(::File.join(@buffer_config[:buffer_dir], 'buffer_state_*.log')).each do |file_path| - begin - buffer_state = Marshal.load(::File.read(file_path)) - @buffer_state[:pending_items].concat(buffer_state[:pending_items]) - @buffer_state[:pending_size] += buffer_state[:pending_size] - ::File.delete(file_path) - rescue => e - @buffer_config[:logger].error("Failed to load buffer from file: #{e.message}") - end - end - @buffer_config[:logger].info("Loaded buffer state from files") - end - - def flush_buffer_files - Dir.glob(::File.join(@buffer_config[:buffer_dir], 'buffer_state_*.log')).each do |file_path| - @buffer_config[:logger].info("Flushing from buffer state file: #{file_path}") - Thread.new { perform_flush([], file_path) } - end - end - end - end -end \ No newline at end of file diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index 969653f9..80ef39c0 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -3,8 +3,9 @@ require 'logstash/outputs/base' require 'logstash/namespace' require 'logstash/errors' +require 'concurrent' -class LogStash::Outputs::Kusto < LogStash::Outputs::Base +module LogStash; module Outputs; class KustoOutputInternal ## # This handles the overall logic and communication with Kusto # @@ -20,9 +21,8 @@ class Ingestor LOW_QUEUE_LENGTH = 3 FIELD_REF = /%\{[^}]+\}/ - def initialize(kusto_logstash_configuration, logger, latch_timeout = 60, threadpool = DEFAULT_THREADPOOL) - @workers_pool = threadpool - @latch_timeout = latch_timeout + def initialize(kusto_logstash_configuration, logger, workers_pool) + @workers_pool = workers_pool @logger = logger #Validate and assign kusto_logstash_configuration.validate_config() @@ -78,80 +78,41 @@ def initialize(kusto_logstash_configuration, logger, latch_timeout = 60, threadp if @kusto_logstash_configuration.kusto_ingest.is_mapping_ref_provided @logger.debug('Using mapping reference.', @kusto_logstash_configuration.kusto_ingest.json_mapping) @ingestion_properties.setIngestionMapping(@kusto_logstash_configuration.kusto_ingest.json_mapping, kusto_java.ingest.IngestionMapping::IngestionMappingKind::JSON) - @ingestion_properties.setDataFormat(kusto_java.ingest.IngestionProperties::DataFormat::JSON) + @ingestion_properties.setDataFormat(kusto_java.ingest.IngestionProperties::DataFormat::MULTIJSON) else @logger.debug('No mapping reference provided. Columns will be mapped by names in the logstash output') - @ingestion_properties.setDataFormat(kusto_java.ingest.IngestionProperties::DataFormat::JSON) + @ingestion_properties.setDataFormat(kusto_java.ingest.IngestionProperties::DataFormat::MULTIJSON) end @logger.debug('Kusto resources are ready.') end - def upload_async(data) - if @workers_pool.remaining_capacity <= LOW_QUEUE_LENGTH - @logger.warn("Ingestor queue capacity is running low with #{@workers_pool.remaining_capacity} free slots.") - end - exception = nil - @workers_pool.post do - LogStash::Util.set_thread_name("Kusto to ingest data") - begin - upload(data) - rescue => e - @logger.error('Error during async upload.', exception: e.class, message: e.message, backtrace: e.backtrace) - exception = e - end - end - # Wait for the task to complete and check for exceptions - @workers_pool.shutdown - @workers_pool.wait_for_termination - - if exception - @logger.error('StandardError in upload_async.', exception: exception.class, message: exception.message, backtrace: exception.backtrace) - raise exception - end - rescue Exception => e - @logger.error('StandardError in upload_async.', exception: e.class, message: e.message, backtrace: e.backtrace) - raise e - end - def upload(data) - @logger.info("Sending data to Kusto") - - if data.size > 0 - thread_exception = nil - - future = java.util.concurrent.CompletableFuture.supplyAsync do - begin - data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(java.io.ByteArrayInputStream.new(data.to_java_bytes)) - @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) - rescue => e - @logger.error('Error during ingestFromStream.', exception: e.class, message: e.message, backtrace: e.backtrace) - thread_exception = e - end - end - - begin - future.get(@latch_timeout, java.util.concurrent.TimeUnit::SECONDS) - rescue java.util.concurrent.TimeoutException => e - @logger.error('Ingestion timed out, possible network issue.') - thread_exception = 'Ingestion timed out, possible network issue.' - rescue java.util.concurrent.ExecutionException => e - thread_exception = e.cause - end - - # Raise the exception from the thread if it occurred - raise thread_exception if thread_exception + data_size = data.size + @logger.info("Ingesting #{data_size} rows to database: #{@ingestion_properties.getDatabaseName} table: #{@ingestion_properties.getTableName}") + if data_size > 0 + #ingestion_status_futures = Concurrent::Future.execute(executor: @workers_pool) do + @workers_pool.post { + begin + in_bytes = java.io.ByteArrayInputStream.new(data.to_s.to_java_bytes) + data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(in_bytes) + ingest_result = @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) + @logger.trace("Ingestion result: #{ingest_result}") + rescue Exception => e + @logger.error("General failed: #{e}") + raise e + ensure + in_bytes.close + end + #end # ingestion_status_futures + } else @logger.warn("Data is empty and is not ingested.") - end - @logger.info("Data sent to Kusto.") - rescue => e - @logger.error('Uploading failed.', exception: e.class, message: e.message, backtrace: e.backtrace) - raise e # Raise the original error if ingestion fails - end + end # if data.size > 0 + end # def upload def stop @workers_pool.shutdown @workers_pool.wait_for_termination(nil) # block until its done - end - end -end \ No newline at end of file + end # def stop + end # class Ingestor +end; end; end # module LogStash::Outputs::KustoOutputInternal diff --git a/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb b/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb index 0aecb874..bd42e131 100644 --- a/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb +++ b/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb @@ -5,11 +5,13 @@ module Outputs module KustoInternal class KustoLogstashConfiguration FIELD_REF = /%\{[^}]+\}/ - def initialize(kusto_ingest,kusto_auth, kusto_proxy, logger) + def initialize(kusto_ingest,kusto_auth, kusto_proxy, kusto_flush_config , kusto_upload_config, logger) @logger = logger @kusto_ingest = kusto_ingest @kusto_auth = kusto_auth @kusto_proxy = kusto_proxy + @kusto_flush_config = kusto_flush_config + @kusto_upload_config = kusto_upload_config @logger.info("Kusto configuration initialized.") end # def initialize @@ -23,6 +25,12 @@ def kusto_auth def kusto_proxy @kusto_proxy end + def kusto_flush_config + @kusto_flush_config + end + def kusto_upload_config + @kusto_upload_config + end def validate_config() # Add an additional validation and fail this upfront @@ -150,6 +158,40 @@ def is_mapping_ref_provided @is_mapping_ref_provided end end # class KustoIngestionConfiguration + class KustoFlushConfiguration + def initialize(max_items, plugin_flush_interval, max_batch_size) + @max_items = max_items + @plugin_flush_interval = plugin_flush_interval + @max_batch_size = max_batch_size + @flush_each = flush_each + end + # Flush configuration + def max_items + @max_items + end + def plugin_flush_interval + @plugin_flush_interval + end + def max_batch_size + @max_batch_size + end + def flush_each + @flush_each + end + end # class KustoFlushConfiguration + class KustoUploadConfiguration + def initialize(upload_concurrent_count, upload_queue_size) + @upload_concurrent_count = upload_concurrent_count + @upload_queue_size = upload_queue_size + end + # Upload configuration + def upload_concurrent_count + @upload_concurrent_count + end + def upload_queue_size + @upload_queue_size + end + end # class KustoUploadConfiguration end # module KustoInternal end # module Outputs end # module LogStash diff --git a/lib/logstash/outputs/kusto/logStashFlushBuffer.rb b/lib/logstash/outputs/kusto/logStashFlushBuffer.rb new file mode 100644 index 00000000..698bb307 --- /dev/null +++ b/lib/logstash/outputs/kusto/logStashFlushBuffer.rb @@ -0,0 +1,60 @@ +# encoding: utf-8 + +require "logstash/outputs/kusto/kustoLogstashConfiguration" +require "logstash/outputs/kusto/customSizeBasedBuffer" +require "logstash/outputs/kusto/ingestor" +require "logger" + +module LogStash; module Outputs; class KustoOutputInternal +class LogStashEventsBatcher + include CustomSizeBasedBuffer + def initialize(kustoLogstashConfiguration,logger) + logger.info("Initializing LogStashEventsBatcher") + # Initialize the buffer with the configuration + # The buffer is a custom buffer that extends the LogStash::Outputs::Base#buffer_initialize + # It is used to buffer the events before sending them to Kusto + logger.info("Initializing buffer with max_items: #{kustoLogstashConfiguration.kusto_flush_config.max_items}, max_interval: #{kustoLogstashConfiguration.kusto_flush_config.plugin_flush_interval}, flush_each: #{kustoLogstashConfiguration.kusto_flush_config.max_batch_size}") + buffer_initialize( + :max_items => kustoLogstashConfiguration.kusto_flush_config.max_items, + :max_interval => kustoLogstashConfiguration.kusto_flush_config.plugin_flush_interval, + :logger => logger, + #todo: There is a small discrepancy between the total size of the documents and the message body + :flush_each => kustoLogstashConfiguration.kusto_flush_config.max_batch_size + ) + @kustoLogstashConfiguration = kustoLogstashConfiguration + workers_pool = Concurrent::ThreadPoolExecutor.new(min_threads: 1, + max_threads: @kustoLogstashConfiguration.kusto_upload_config.upload_concurrent_count, + max_queue: @kustoLogstashConfiguration.kusto_upload_config.upload_queue_size, + fallback_policy: :caller_runs + ) if @ingestor.nil? + @log===ger = logger + @ingestor = LogStash::Outputs::KustoOutputInternal::Ingestor.new(@kustoLogstashConfiguration, logger, workers_pool) + end # initialize + + # Public methods + public + + # Adding an event document into the buffer + def batch_event(event_document) + buffer_receive(event_document) + end # def batch_event + + # Flushing all buffer content to Azure Loganalytics. + # Called from Stud::Buffer#buffer_flush when there are events to flush + def flush (documents, close=false) + # Skip in case there are no candidate documents to deliver + if documents.length < 1 + @logger.warn("No documents in batch in the batch. Skipping") + return + end + + + @ingestor.upload(documents) + end # def flush + + def close + buffer_flush(:final => true) + end + +end # LogStashAutoResizeBuffer +end ;end ;end \ No newline at end of file diff --git a/random-data.sh b/random-data.sh new file mode 100755 index 00000000..3ea64222 --- /dev/null +++ b/random-data.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +while true +do + random_ip=$(dd if=/dev/urandom bs=4 count=1 2>/dev/null | od -An -tu1 | sed -e 's/^ *//' -e 's/ */./g') + random_size=$(( (RANDOM % 65535) + 1 )) + current_date_time=$(date '+%d/%b/%Y:%H:%M:%S %z') + echo "$random_ip - - [$current_date_time] \"GET /data.php HTTP/1.1\" 200 $random_size" | tee -a '/tmp/curllogs.txt' + sleep 0.0000001s +done \ No newline at end of file diff --git a/run.sh b/run.sh new file mode 100755 index 00000000..a354c39f --- /dev/null +++ b/run.sh @@ -0,0 +1 @@ +/softwares/logstash/bin/logstash -f /code/logstash/logstash-output-kusto/e2e/logstash.conf \ No newline at end of file diff --git a/version b/version index ed35d092..4a36342f 100644 --- a/version +++ b/version @@ -1 +1 @@ -2.0.9 \ No newline at end of file +3.0.0 From 54e8efbf9d7110497970ae706e05fde9c0d09cda Mon Sep 17 00:00:00 2001 From: ag-ramachandran Date: Fri, 23 May 2025 09:48:01 +0530 Subject: [PATCH 37/52] WIP Commit --- .../outputs/kusto/customSizeBasedBuffer.rb | 44 ++++++++++----- lib/logstash/outputs/kusto/ingestor.rb | 54 ++++++++++--------- .../outputs/kusto/logStashFlushBuffer.rb | 29 +++++----- logstash-output-kusto.gemspec | 2 +- 4 files changed, 74 insertions(+), 55 deletions(-) diff --git a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb index 4ee48978..6eb3567a 100644 --- a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb +++ b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb @@ -132,17 +132,33 @@ def buffer_initialize(options={}) # # @return [bool] Is the buffer full? def buffer_full? - @buffer_config[:logger].info("---------------Entering buffer_full?-----------------") - @buffer_config[:logger].info("Pending count: #{@buffer_state[:pending_count]}") - @buffer_config[:logger].info("Outgoing count: #{@buffer_state[:outgoing_count]}") - @buffer_config[:logger].info("Pending size: #{@buffer_state[:pending_size]}") - @buffer_config[:logger].info("Outgoing size: #{@buffer_state[:outgoing_size]}") - @buffer_config[:logger].info("Max items: #{@buffer_config[:max_items]}") - @buffer_config[:logger].info("Flush each: #{@buffer_config[:flush_each]}") - @buffer_config[:logger].info("Max interval: #{@buffer_config[:max_interval]}") - @buffer_config[:logger].info("Pending items: #{@buffer_state[:pending_items]}") - @buffer_config[:logger].info("Outgoing items: #{@buffer_state[:outgoing_items]}") - @buffer_config[:logger].info("---------------Exiting buffer_full?-----------------") + + c1 = (@buffer_state[:pending_count] + @buffer_state[:outgoing_count] >= @buffer_config[:max_items]) + c2 = (@buffer_config[:flush_each] != 0 && @buffer_state[:pending_size] + @buffer_state[:outgoing_size] >= @buffer_config[:flush_each]) + + if c1 || c2 + @buffer_config[:logger].debug("---------------Entering buffer_full?-----------------") + end + + + if c1 + @buffer_config[:logger].info("Buffer is full: max_items reached") + @buffer_config[:logger].debug("Pending count: #{@buffer_state[:pending_count]}") + @buffer_config[:logger].debug("Outgoing count: #{@buffer_state[:outgoing_count]}") + @buffer_config[:logger].debug("Pending count: #{@buffer_config[:max_items]}") + end + if c2 + @buffer_config[:logger].info("Buffer is full: flush_each reached") + @buffer_config[:logger].debug("Pending size: #{@buffer_state[:pending_size]}") + @buffer_config[:logger].debug("Outgoing size: #{@buffer_state[:outgoing_size]}") + @buffer_config[:logger].debug("Flush each: #{@buffer_config[:flush_each]}") + @buffer_config[:logger].debug("Max items: #{@buffer_config[:max_items]}") + end + + if c1 || c2 + @buffer_config[:logger].info("---------------Exiting buffer_full?-----------------") + end + (@buffer_state[:pending_count] + @buffer_state[:outgoing_count] >= @buffer_config[:max_items]) || \ (@buffer_config[:flush_each] != 0 && @buffer_state[:pending_size] + @buffer_state[:outgoing_size] >= @buffer_config[:flush_each]) end @@ -225,10 +241,11 @@ def buffer_flush(options={}) @buffer_state[:outgoing_size] = @buffer_state[:pending_size] buffer_clear_pending end - @buffer_config[:logger].debug("Flushing output", + @buffer_config[:logger].info("---------------Exiting buffer_flush?-----------------") + @buffer_config[:logger].info("Flushing output", :outgoing_count => @buffer_state[:outgoing_count], :time_since_last_flush => time_since_last_flush, - :outgoing_events => @buffer_state[:outgoing_items], + :outgoing_events_count => @buffer_state[:outgoing_items].length, :batch_timeout => @buffer_config[:max_interval], :force => force, :final => final @@ -270,6 +287,7 @@ def buffer_flush(options={}) retry end @buffer_state[:last_flush] = Time.now.to_i + @buffer_config[:logger].info("---------------Exiting buffer_flush?-----------------") end ensure diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index 80ef39c0..0a650787 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -4,6 +4,7 @@ require 'logstash/namespace' require 'logstash/errors' require 'concurrent' +require 'json' module LogStash; module Outputs; class KustoOutputInternal ## @@ -12,21 +13,18 @@ module LogStash; module Outputs; class KustoOutputInternal class Ingestor require 'logstash-output-kusto_jars' RETRY_DELAY_SECONDS = 3 - DEFAULT_THREADPOOL = Concurrent::ThreadPoolExecutor.new( - min_threads: 1, - max_threads: 8, - max_queue: 1, - fallback_policy: :caller_runs - ) - LOW_QUEUE_LENGTH = 3 + FIELD_REF = /%\{[^}]+\}/ - def initialize(kusto_logstash_configuration, logger, workers_pool) - @workers_pool = workers_pool + def initialize(kusto_logstash_configuration, logger) + @kusto_logstash_configuration = kusto_logstash_configuration @logger = logger + @workers_pool = Concurrent::ThreadPoolExecutor.new(min_threads: 1, + max_threads: kusto_logstash_configuration.kusto_upload_config.upload_concurrent_count, + max_queue: kusto_logstash_configuration.kusto_upload_config.upload_queue_size, + fallback_policy: :caller_runs + ) #Validate and assign - kusto_logstash_configuration.validate_config() - @kusto_logstash_configuration = kusto_logstash_configuration @logger.info('Preparing Kusto resources.') kusto_java = Java::com.microsoft.azure.kusto @@ -91,20 +89,28 @@ def upload(data) @logger.info("Ingesting #{data_size} rows to database: #{@ingestion_properties.getDatabaseName} table: #{@ingestion_properties.getTableName}") if data_size > 0 #ingestion_status_futures = Concurrent::Future.execute(executor: @workers_pool) do - @workers_pool.post { - begin - in_bytes = java.io.ByteArrayInputStream.new(data.to_s.to_java_bytes) - data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(in_bytes) - ingest_result = @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) - @logger.trace("Ingestion result: #{ingest_result}") - rescue Exception => e - @logger.error("General failed: #{e}") - raise e - ensure - in_bytes.close - end - #end # ingestion_status_futures + exceptions = Concurrent::Array.new + promise = Concurrent::Promises.future { + in_bytes = java.io.ByteArrayInputStream.new(data.to_json.to_java_bytes) + data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(in_bytes) + ingest_result = @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) + #@logger.info("Ingestion result: #{ingest_result}") + } + .rescue{ |e| + @logger.error("Ingestion failed: #{e.message}") + @logger.error("Ingestion failed: #{e.backtrace.join("\n")}") + exceptions.push(e) + e } + .on_resolution do |result, reason| + if reason + @logger.error("Future completed with error: #{reason}") + @logger.error("Future 2 completed with error: #{result}") + else + @logger.info("Future completed successfully.") + end + end + else @logger.warn("Data is empty and is not ingested.") end # if data.size > 0 diff --git a/lib/logstash/outputs/kusto/logStashFlushBuffer.rb b/lib/logstash/outputs/kusto/logStashFlushBuffer.rb index 698bb307..582d2e0f 100644 --- a/lib/logstash/outputs/kusto/logStashFlushBuffer.rb +++ b/lib/logstash/outputs/kusto/logStashFlushBuffer.rb @@ -8,38 +8,33 @@ module LogStash; module Outputs; class KustoOutputInternal class LogStashEventsBatcher include CustomSizeBasedBuffer - def initialize(kustoLogstashConfiguration,logger) + def initialize(kusto_logstash_configuration,logger) logger.info("Initializing LogStashEventsBatcher") # Initialize the buffer with the configuration # The buffer is a custom buffer that extends the LogStash::Outputs::Base#buffer_initialize # It is used to buffer the events before sending them to Kusto - logger.info("Initializing buffer with max_items: #{kustoLogstashConfiguration.kusto_flush_config.max_items}, max_interval: #{kustoLogstashConfiguration.kusto_flush_config.plugin_flush_interval}, flush_each: #{kustoLogstashConfiguration.kusto_flush_config.max_batch_size}") + logger.info("Initializing buffer with max_items: #{kusto_logstash_configuration.kusto_flush_config.max_items}, max_interval: #{kusto_logstash_configuration.kusto_flush_config.plugin_flush_interval}, flush_each: #{kusto_logstash_configuration.kusto_flush_config.max_batch_size}") buffer_initialize( - :max_items => kustoLogstashConfiguration.kusto_flush_config.max_items, - :max_interval => kustoLogstashConfiguration.kusto_flush_config.plugin_flush_interval, + :max_items => kusto_logstash_configuration.kusto_flush_config.max_items, + :max_interval => kusto_logstash_configuration.kusto_flush_config.plugin_flush_interval, :logger => logger, #todo: There is a small discrepancy between the total size of the documents and the message body - :flush_each => kustoLogstashConfiguration.kusto_flush_config.max_batch_size + :flush_each => kusto_logstash_configuration.kusto_flush_config.max_batch_size ) - @kustoLogstashConfiguration = kustoLogstashConfiguration - workers_pool = Concurrent::ThreadPoolExecutor.new(min_threads: 1, - max_threads: @kustoLogstashConfiguration.kusto_upload_config.upload_concurrent_count, - max_queue: @kustoLogstashConfiguration.kusto_upload_config.upload_queue_size, - fallback_policy: :caller_runs - ) if @ingestor.nil? - @log===ger = logger - @ingestor = LogStash::Outputs::KustoOutputInternal::Ingestor.new(@kustoLogstashConfiguration, logger, workers_pool) + @kusto_logstash_configuration = kusto_logstash_configuration + @logger = logger + @ingestor = LogStash::Outputs::KustoOutputInternal::Ingestor.new(@kusto_logstash_configuration, logger) end # initialize # Public methods public # Adding an event document into the buffer - def batch_event(event_document) + def batch_event(event_document) buffer_receive(event_document) end # def batch_event - # Flushing all buffer content to Azure Loganalytics. + # Flushing all buffer content to Kusto. # Called from Stud::Buffer#buffer_flush when there are events to flush def flush (documents, close=false) # Skip in case there are no candidate documents to deliver @@ -47,8 +42,8 @@ def flush (documents, close=false) @logger.warn("No documents in batch in the batch. Skipping") return end - - + @logger.info("Uploading batch of documents to Kusto #{documents.length} documents") + # Upload the documents to Kusto @ingestor.upload(documents) end # def flush diff --git a/logstash-output-kusto.gemspec b/logstash-output-kusto.gemspec index dd698322..d383f291 100755 --- a/logstash-output-kusto.gemspec +++ b/logstash-output-kusto.gemspec @@ -20,7 +20,7 @@ Gem::Specification.new do |s| s.metadata = { "logstash_plugin" => "true", "logstash_group" => "output" } # Gem dependencies - s.add_runtime_dependency 'logstash-core', '>= 8.7.0' + s.add_runtime_dependency 'logstash-core', '~> 8.7', '>= 8.7.0' s.add_runtime_dependency "logstash-core-plugin-api", ">= 1.60", "<= 2.99" s.add_runtime_dependency 'logstash-codec-json_lines' s.add_runtime_dependency 'logstash-codec-line' From 926e03753ef3e60eef2f1961790b827e6c743871 Mon Sep 17 00:00:00 2001 From: ag-ramachandran Date: Tue, 3 Jun 2025 18:38:17 +0530 Subject: [PATCH 38/52] * Some more changes --- .../outputs/kusto/customSizeBasedBuffer.rb | 10 ++++----- lib/logstash/outputs/kusto/ingestor.rb | 21 ++++++++++++------- 2 files changed, 19 insertions(+), 12 deletions(-) diff --git a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb index 6eb3567a..4ea0c252 100644 --- a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb +++ b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb @@ -142,13 +142,13 @@ def buffer_full? if c1 - @buffer_config[:logger].info("Buffer is full: max_items reached") + @buffer_config[:logger].debug("Buffer is full: max_items reached") @buffer_config[:logger].debug("Pending count: #{@buffer_state[:pending_count]}") @buffer_config[:logger].debug("Outgoing count: #{@buffer_state[:outgoing_count]}") @buffer_config[:logger].debug("Pending count: #{@buffer_config[:max_items]}") end if c2 - @buffer_config[:logger].info("Buffer is full: flush_each reached") + @buffer_config[:logger].debug("Buffer is full: flush_each reached") @buffer_config[:logger].debug("Pending size: #{@buffer_state[:pending_size]}") @buffer_config[:logger].debug("Outgoing size: #{@buffer_state[:outgoing_size]}") @buffer_config[:logger].debug("Flush each: #{@buffer_config[:flush_each]}") @@ -156,7 +156,7 @@ def buffer_full? end if c1 || c2 - @buffer_config[:logger].info("---------------Exiting buffer_full?-----------------") + @buffer_config[:logger].debug("---------------Exiting buffer_full?-----------------") end (@buffer_state[:pending_count] + @buffer_state[:outgoing_count] >= @buffer_config[:max_items]) || \ @@ -241,7 +241,7 @@ def buffer_flush(options={}) @buffer_state[:outgoing_size] = @buffer_state[:pending_size] buffer_clear_pending end - @buffer_config[:logger].info("---------------Exiting buffer_flush?-----------------") + @buffer_config[:logger].debug("---------------Exiting buffer_flush?-----------------") @buffer_config[:logger].info("Flushing output", :outgoing_count => @buffer_state[:outgoing_count], :time_since_last_flush => time_since_last_flush, @@ -287,7 +287,7 @@ def buffer_flush(options={}) retry end @buffer_state[:last_flush] = Time.now.to_i - @buffer_config[:logger].info("---------------Exiting buffer_flush?-----------------") + @buffer_config[:logger].debug("---------------Exiting buffer_flush?-----------------") end ensure diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index 0a650787..6d04b780 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -90,7 +90,7 @@ def upload(data) if data_size > 0 #ingestion_status_futures = Concurrent::Future.execute(executor: @workers_pool) do exceptions = Concurrent::Array.new - promise = Concurrent::Promises.future { + promise = Concurrent::Promises.future { in_bytes = java.io.ByteArrayInputStream.new(data.to_json.to_java_bytes) data_source_info = Java::com.microsoft.azure.kusto.ingest.source.StreamSourceInfo.new(in_bytes) ingest_result = @kusto_client.ingestFromStream(data_source_info, @ingestion_properties) @@ -102,15 +102,22 @@ def upload(data) exceptions.push(e) e } - .on_resolution do |result, reason| - if reason - @logger.error("Future completed with error: #{reason}") - @logger.error("Future 2 completed with error: #{result}") + .on_resolution do |fulfilled, value, reason, *args| + @logger.info("******************************************************************************************") + @logger.info("Future fulfilled: #{fulfilled}, value: #{value}, reason: #{reason}, args: #{args}, class: #{value.class}") + #@logger.info("Ingestion status: #{value.getIngestionStatusCollection().getStatus}") + + if value.class == Java::ComMicrosoftAzureKustoIngestResult::IngestionStatusResult + isc = value.getIngestionStatusCollection()&.get(0)&.getStatus() + @logger.info("Ingestion status: #{isc}") else - @logger.info("Future completed successfully.") + @logger.info("Ingestion status is non success status: #{value.class} - #{value}") + end + if exceptions.size > 0 + @logger.error("Ingestion failed with exceptions: #{exceptions.map(&:message).join(', ')}") end + @logger.info("******************************************************************************************") end - else @logger.warn("Data is empty and is not ingested.") end # if data.size > 0 From 15244616c2195511209e6a47f57ac31b0fb3e1ba Mon Sep 17 00:00:00 2001 From: ag-ramachandran Date: Tue, 3 Jun 2025 18:38:46 +0530 Subject: [PATCH 39/52] *Update gitignore --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index e3608ef5..fa647d8c 100644 --- a/.gitignore +++ b/.gitignore @@ -65,3 +65,4 @@ local-run.sh logs2.txt **/.vscode/*.* **/settings.json +Run.sh From 55f6631f16c8240cd890ba9fd73901e7e71c3da8 Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Thu, 12 Jun 2025 22:38:11 +0530 Subject: [PATCH 40/52] Persist on flush failure and add configs for retries Persists buffer to disk in case of ingestion failures. Added configs process_failed_batches_on_startup and process_failed_batches_on_shutdown which when configured will retry flushing the persisted files. Updated closing and shutdown. --- lib/logstash/outputs/kusto.rb | 41 +++++++++++-------- .../outputs/kusto/customSizeBasedBuffer.rb | 37 ++++++++++++++++- lib/logstash/outputs/kusto/filePersistence.rb | 26 ++++++++++++ lib/logstash/outputs/kusto/ingestor.rb | 5 ++- .../kusto/kustoLogstashConfiguration.rb | 12 +++++- .../outputs/kusto/logStashFlushBuffer.rb | 22 ++++++---- 6 files changed, 114 insertions(+), 29 deletions(-) create mode 100644 lib/logstash/outputs/kusto/filePersistence.rb diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index 66523cb7..28cd32a9 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -74,6 +74,11 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # Maximum interval (in seconds) before the buffer gets flushed, defaults to 10 config :max_items, validate: :number, required: false , default: 100 + # Process failed batches on startup + config :process_failed_batches_on_startup, validate: :boolean, required: false, default: false + + # Process failed batches on shutdown + config :process_failed_batches_on_shutdown, validate: :boolean, required: false, default: false default :codec, 'json_lines' @@ -81,9 +86,9 @@ def register kusto_ingest_base = LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, database, table, json_mapping) kusto_auth_base = LogStash::Outputs::KustoInternal::KustoAuthConfiguration.new(app_id, app_key, app_tenant, managed_identity, cli_auth) kusto_proxy_base = LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new(proxy_host , proxy_port , proxy_protocol, false) - kusto_flush_config = LogStash::Outputs::KustoInternal::KustoFlushConfiguration.new(max_items, plugin_flush_interval,max_batch_size) + kusto_flush_config = LogStash::Outputs::KustoInternal::KustoFlushConfiguration.new(max_items, plugin_flush_interval, max_batch_size, process_failed_batches_on_startup, process_failed_batches_on_shutdown) kusto_upload_config = LogStash::Outputs::KustoInternal::KustoUploadConfiguration.new(upload_concurrent_count, upload_queue_size) - kusto_logstash_configuration = LogStash::Outputs::KustoInternal::KustoLogstashConfiguration.new(kusto_ingest_base, kusto_auth_base , kusto_proxy_base, kusto_flush_config, kusto_upload_config,@logger) + kusto_logstash_configuration = LogStash::Outputs::KustoInternal::KustoLogstashConfiguration.new(kusto_ingest_base, kusto_auth_base , kusto_proxy_base, kusto_flush_config, kusto_upload_config, @logger) kusto_logstash_configuration.validate_config # Initialize the custom buffer with size and interval @buffer = LogStash::Outputs::KustoOutputInternal::LogStashEventsBatcher.new(kusto_logstash_configuration,@logger) @@ -102,21 +107,21 @@ def multi_receive_encoded(events_and_encoded) end def close - # @logger.info("Closing Kusto output plugin") - # begin - # @buffer.shutdown unless @buffer.nil? - # @logger.info("Buffer shutdown") unless @buffer.nil? - # rescue => e - # @logger.error("Error shutting down buffer: #{e.message}") - # @logger.error(e.backtrace.join("\n")) - # end - # begin - # @ingestor.stop unless @ingestor.nil? - # @logger.info("Ingestor stopped") unless @ingestor.nil? - # rescue => e - # @logger.error("Error stopping ingestor: #{e.message}") - # @logger.error(e.backtrace.join("\n")) - # end - # @logger.info("Kusto output plugin Closed") + @logger.info("Closing Kusto output plugin") + begin + @buffer.shutdown unless @buffer.nil? + @logger.info("Buffer shutdown") unless @buffer.nil? + rescue => e + @logger.error("Error shutting down buffer: #{e.message}") + @logger.error(e.backtrace.join("\n")) + end + begin + @ingestor.stop unless @ingestor.nil? + @logger.info("Ingestor stopped") unless @ingestor.nil? + rescue => e + @logger.error("Error stopping ingestor: #{e.message}") + @logger.error(e.backtrace.join("\n")) + end + @logger.info("Kusto output plugin Closed") end end diff --git a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb index 4ea0c252..236d2bdd 100644 --- a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb +++ b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb @@ -91,7 +91,9 @@ def buffer_initialize(options={}) :max_items => options[:max_items] || 50, :flush_each => options[:flush_each].to_i || 0, :max_interval => options[:max_interval] || 5, - :logger => options[:logger] || nil, + :logger => options[:logger] || Logger.new(STDOUT), + :process_failed_batches_on_startup => options[:process_failed_batches_on_startup] || false, + :process_failed_batches_on_shutdown => options[:process_failed_batches_on_shutdown] || false, :has_on_flush_error => self.class.method_defined?(:on_flush_error), :has_on_full_buffer_receive => self.class.method_defined?(:on_full_buffer_receive) } @@ -124,6 +126,8 @@ def buffer_initialize(options={}) # events we've accumulated buffer_clear_pending + process_failed_batches if options[:process_failed_batches_on_startup] + end # Determine if +:max_items+ or +:flush_each+ has been reached. @@ -296,7 +300,38 @@ def buffer_flush(options={}) return items_flushed end + + def process_failed_batches + require_relative 'filePersistence' + LogStash::Outputs::KustoOutputInternal::FilePersistence.load_batches.each do |file, batch| + begin + @buffer_state[:flush_mutex].lock + begin + flush(batch, true) + LogStash::Outputs::KustoOutputInternal::FilePersistence.delete_batch(file) + @buffer_config[:logger].info("Successfully flushed and deleted failed batch file: #{file}") if @buffer_config[:logger] + rescue => e + @buffer_config[:logger].warn("Failed to flush persisted batch: #{e.message}") if @buffer_config[:logger] + ensure + @buffer_state[:flush_mutex].unlock + end + rescue => e + @buffer_config[:logger].error("Error processing failed batch file: #{e.message}") if @buffer_config[:logger] + end + end + end + def shutdown + # Stop the timer thread if it exists + if @buffer_state && @buffer_state[:timer] + @buffer_state[:timer].kill + @buffer_state[:timer] = nil + end + # Final flush of any remaining in-memory events + buffer_flush(:final => true) if @buffer_state + # Process any failed batches on shutdown if configured to do so + process_failed_batches if @buffer_config && @buffer_config[:process_failed_batches_on_shutdown] + end private def buffer_clear_pending diff --git a/lib/logstash/outputs/kusto/filePersistence.rb b/lib/logstash/outputs/kusto/filePersistence.rb new file mode 100644 index 00000000..6a167694 --- /dev/null +++ b/lib/logstash/outputs/kusto/filePersistence.rb @@ -0,0 +1,26 @@ +require 'securerandom' +require 'json' +require 'fileutils' + +module LogStash; module Outputs; class KustoOutputInternal + module FilePersistence + FAILED_DIR = './tmp/buffer_storage/' + + def self.persist_batch(batch) + ::FileUtils.mkdir_p(FAILED_DIR) + filename = ::File.join(FAILED_DIR, "failed_batch_#{Time.now.to_i}_#{SecureRandom.uuid}.json") + ::File.write(filename, JSON.dump(batch)) + end + + def self.load_batches + return [] unless Dir.exist?(FAILED_DIR) + Dir.glob(::File.join(FAILED_DIR, 'failed_batch_*.json')).map do |file| + [file, JSON.load(::File.read(file))] + end + end + + def self.delete_batch(file) + ::File.delete(file) if ::File.exist?(file) + end + end +end; end; end \ No newline at end of file diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index 6d04b780..c4d09c56 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -5,6 +5,7 @@ require 'logstash/errors' require 'concurrent' require 'json' +require 'logstash/outputs/kusto/filePersistence' module LogStash; module Outputs; class KustoOutputInternal ## @@ -99,8 +100,8 @@ def upload(data) .rescue{ |e| @logger.error("Ingestion failed: #{e.message}") @logger.error("Ingestion failed: #{e.backtrace.join("\n")}") - exceptions.push(e) - e + LogStash::Outputs::KustoOutputInternal::FilePersistence.persist_batch(data) + raise e } .on_resolution do |fulfilled, value, reason, *args| @logger.info("******************************************************************************************") diff --git a/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb b/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb index bd42e131..0cef40ab 100644 --- a/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb +++ b/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb @@ -159,11 +159,14 @@ def is_mapping_ref_provided end end # class KustoIngestionConfiguration class KustoFlushConfiguration - def initialize(max_items, plugin_flush_interval, max_batch_size) + def initialize(max_items, plugin_flush_interval, max_batch_size, process_failed_batches_on_startup, + process_failed_batches_on_shutdown) @max_items = max_items @plugin_flush_interval = plugin_flush_interval @max_batch_size = max_batch_size @flush_each = flush_each + @process_failed_batches_on_startup = process_failed_batches_on_startup + @process_failed_batches_on_shutdown = process_failed_batches_on_shutdown end # Flush configuration def max_items @@ -178,6 +181,13 @@ def max_batch_size def flush_each @flush_each end + def process_failed_batches_on_startup + @process_failed_batches_on_startup + end + + def process_failed_batches_on_shutdown + @process_failed_batches_on_shutdown + end end # class KustoFlushConfiguration class KustoUploadConfiguration def initialize(upload_concurrent_count, upload_queue_size) diff --git a/lib/logstash/outputs/kusto/logStashFlushBuffer.rb b/lib/logstash/outputs/kusto/logStashFlushBuffer.rb index 582d2e0f..20d97f7c 100644 --- a/lib/logstash/outputs/kusto/logStashFlushBuffer.rb +++ b/lib/logstash/outputs/kusto/logStashFlushBuffer.rb @@ -13,17 +13,20 @@ def initialize(kusto_logstash_configuration,logger) # Initialize the buffer with the configuration # The buffer is a custom buffer that extends the LogStash::Outputs::Base#buffer_initialize # It is used to buffer the events before sending them to Kusto + @kusto_logstash_configuration = kusto_logstash_configuration + @logger = logger + @ingestor = LogStash::Outputs::KustoOutputInternal::Ingestor.new(@kusto_logstash_configuration, logger) + logger.info("Initializing buffer with max_items: #{kusto_logstash_configuration.kusto_flush_config.max_items}, max_interval: #{kusto_logstash_configuration.kusto_flush_config.plugin_flush_interval}, flush_each: #{kusto_logstash_configuration.kusto_flush_config.max_batch_size}") buffer_initialize( :max_items => kusto_logstash_configuration.kusto_flush_config.max_items, :max_interval => kusto_logstash_configuration.kusto_flush_config.plugin_flush_interval, :logger => logger, #todo: There is a small discrepancy between the total size of the documents and the message body - :flush_each => kusto_logstash_configuration.kusto_flush_config.max_batch_size + :flush_each => kusto_logstash_configuration.kusto_flush_config.max_batch_size, + :process_failed_batches_on_startup => kusto_logstash_configuration.kusto_flush_config.process_failed_batches_on_startup, + :process_failed_batches_on_shutdown => kusto_logstash_configuration.kusto_flush_config.process_failed_batches_on_shutdown ) - @kusto_logstash_configuration = kusto_logstash_configuration - @logger = logger - @ingestor = LogStash::Outputs::KustoOutputInternal::Ingestor.new(@kusto_logstash_configuration, logger) end # initialize # Public methods @@ -43,12 +46,17 @@ def flush (documents, close=false) return end @logger.info("Uploading batch of documents to Kusto #{documents.length} documents") - # Upload the documents to Kusto - @ingestor.upload(documents) + begin + @ingestor.upload(documents) + rescue => e + @logger.error("Error uploading batch to Kusto: #{e.message}") + raise e # Let the buffer handle persistence of failed batch + end end # def flush def close - buffer_flush(:final => true) + @logger.info("Closing LogStashEventsBatcher...") + shutdown end end # LogStashAutoResizeBuffer From b32e6ac55c8ac3c3a7f7bbaba7cee67f11e23e99 Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Mon, 16 Jun 2025 13:52:41 +0530 Subject: [PATCH 41/52] Removed process_failed_batches_on_shutdown config and Updated Spec and E2E Tests --- .gitignore | 2 + e2e/e2e.rb | 18 ++- lib/logstash/outputs/kusto.rb | 8 +- .../outputs/kusto/customSizeBasedBuffer.rb | 3 - lib/logstash/outputs/kusto/filePersistence.rb | 18 ++- .../kusto/kustoLogstashConfiguration.rb | 9 +- .../outputs/kusto/logStashFlushBuffer.rb | 1 - spec/outputs/kusto/filePersistence_spec.rb | 48 +++++++ .../kusto/kustoLogstashConfiguration_spec.rb | 118 +++++++++--------- 9 files changed, 135 insertions(+), 90 deletions(-) create mode 100644 spec/outputs/kusto/filePersistence_spec.rb diff --git a/.gitignore b/.gitignore index fa647d8c..065886d4 100644 --- a/.gitignore +++ b/.gitignore @@ -59,6 +59,7 @@ gradle/wrapper/gradle-wrapper.properties .vscode/settings.json rspec.xml e2e/output_file.txt +e2e/input_file.txt logs.txt docker-e2e/.env local-run.sh @@ -66,3 +67,4 @@ logs2.txt **/.vscode/*.* **/settings.json Run.sh +vendor diff --git a/e2e/e2e.rb b/e2e/e2e.rb index f969281f..04cc0e80 100755 --- a/e2e/e2e.rb +++ b/e2e/e2e.rb @@ -7,7 +7,7 @@ class E2E def initialize super - @input_file = "/tmp/input_file.txt" + @input_file = "input_file.txt" @output_file = "output_file.txt" @columns = "(rownumber:int, rowguid:string, xdouble:real, xfloat:real, xbool:bool, xint16:int, xint32:int, xint64:long, xuint8:long, xuint16:long, xuint32:long, xuint64:long, xdate:datetime, xsmalltext:string, xtext:string, xnumberAsText:string, xtime:timespan, xtextWithNulls:string, xdynamicWithNulls:dynamic)" @csv_columns = '"rownumber", "rowguid", "xdouble", "xfloat", "xbool", "xint16", "xint32", "xint64", "xuint8", "xuint16", "xuint32", "xuint64", "xdate", "xsmalltext", "xtext", "xnumberAsText", "xtime", "xtextWithNulls", "xdynamicWithNulls"' @@ -35,21 +35,17 @@ def initialize file { path => "#{@output_file}"} stdout { codec => rubydebug } kusto { - path => "tmp%{+YYYY-MM-dd-HH-mm}.txt" ingest_url => "#{@ingest_url}" cli_auth => true database => "#{@database}" table => "#{@table_with_mapping}" json_mapping => "#{@mapping_name}" - max_size => 0.005 } kusto { - path => "nomaptmp%{+YYYY-MM-dd-HH-mm}.txt" cli_auth => true ingest_url => "#{@ingest_url}" database => "#{@database}" table => "#{@table_without_mapping}" - max_size => 0.004 } } } @@ -58,20 +54,20 @@ def initialize def create_table_and_mapping Array[@table_with_mapping, @table_without_mapping].each { |tableop| puts "Creating table #{tableop}" - @query_client.execute(@database, ".drop table #{tableop} ifexists") + @query_client.executeMgmt(@database, ".drop table #{tableop} ifexists") sleep(1) - @query_client.execute(@database, ".create table #{tableop} #{@columns}") - @query_client.execute(@database, ".alter table #{tableop} policy ingestionbatching @'{\"MaximumBatchingTimeSpan\":\"00:00:10\", \"MaximumNumberOfItems\": 1, \"MaximumRawDataSizeMB\": 100}'") + @query_client.executeMgmt(@database, ".create table #{tableop} #{@columns}") + @query_client.executeMgmt(@database, ".alter table #{tableop} policy ingestionbatching @'{\"MaximumBatchingTimeSpan\":\"00:00:10\", \"MaximumNumberOfItems\": 1, \"MaximumRawDataSizeMB\": 100}'") } # Mapping only for one table - @query_client.execute(@database, ".create table #{@table_with_mapping} ingestion json mapping '#{@mapping_name}' '#{File.read("dataset_mapping.json")}'") + @query_client.executeMgmt(@database, ".create table #{@table_with_mapping} ingestion json mapping '#{@mapping_name}' '#{File.read("dataset_mapping.json")}'") end def drop_and_cleanup Array[@table_with_mapping, @table_without_mapping].each { |tableop| puts "Dropping table #{tableop}" - @query_client.execute(@database, ".drop table #{tableop} ifexists") + @query_client.executeMgmt(@database, ".drop table #{tableop} ifexists") sleep(1) } end @@ -101,7 +97,7 @@ def assert_data (0...max_timeout).each do |_| begin sleep(5) - query = @query_client.execute(@database, "#{tableop} | sort by rownumber asc") + query = @query_client.executeQuery(@database, "#{tableop} | sort by rownumber asc") result = query.getPrimaryResults() raise "Wrong count - expected #{csv_data.length}, got #{result.count()} in table #{tableop}" unless result.count() == csv_data.length rescue Exception => e diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index 28cd32a9..fb6ef276 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -7,6 +7,7 @@ require 'logstash/outputs/kusto/customSizeBasedBuffer' require 'logstash/outputs/kusto/kustoLogstashConfiguration' require 'logstash/outputs/kusto/logStashFlushBuffer' +require 'logstash/outputs/kusto/filePersistence' ## # This plugin sends messages to Azure Kusto in batches. @@ -77,16 +78,17 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # Process failed batches on startup config :process_failed_batches_on_startup, validate: :boolean, required: false, default: false - # Process failed batches on shutdown - config :process_failed_batches_on_shutdown, validate: :boolean, required: false, default: false + # Directory to store the failed batches that were not uploaded to Kusto. If the directory does not exist, it will be created, defaults to './tmp/buffer_storage/' + config :failed_dir_name, validate: :string, required: false, default: './tmp/buffer_storage/' default :codec, 'json_lines' def register + LogStash::Outputs::KustoOutputInternal::FilePersistence.failed_dir = failed_dir_name kusto_ingest_base = LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, database, table, json_mapping) kusto_auth_base = LogStash::Outputs::KustoInternal::KustoAuthConfiguration.new(app_id, app_key, app_tenant, managed_identity, cli_auth) kusto_proxy_base = LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new(proxy_host , proxy_port , proxy_protocol, false) - kusto_flush_config = LogStash::Outputs::KustoInternal::KustoFlushConfiguration.new(max_items, plugin_flush_interval, max_batch_size, process_failed_batches_on_startup, process_failed_batches_on_shutdown) + kusto_flush_config = LogStash::Outputs::KustoInternal::KustoFlushConfiguration.new(max_items, plugin_flush_interval, max_batch_size, process_failed_batches_on_startup) kusto_upload_config = LogStash::Outputs::KustoInternal::KustoUploadConfiguration.new(upload_concurrent_count, upload_queue_size) kusto_logstash_configuration = LogStash::Outputs::KustoInternal::KustoLogstashConfiguration.new(kusto_ingest_base, kusto_auth_base , kusto_proxy_base, kusto_flush_config, kusto_upload_config, @logger) kusto_logstash_configuration.validate_config diff --git a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb index 236d2bdd..aebc6579 100644 --- a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb +++ b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb @@ -93,7 +93,6 @@ def buffer_initialize(options={}) :max_interval => options[:max_interval] || 5, :logger => options[:logger] || Logger.new(STDOUT), :process_failed_batches_on_startup => options[:process_failed_batches_on_startup] || false, - :process_failed_batches_on_shutdown => options[:process_failed_batches_on_shutdown] || false, :has_on_flush_error => self.class.method_defined?(:on_flush_error), :has_on_full_buffer_receive => self.class.method_defined?(:on_full_buffer_receive) } @@ -329,8 +328,6 @@ def shutdown end # Final flush of any remaining in-memory events buffer_flush(:final => true) if @buffer_state - # Process any failed batches on shutdown if configured to do so - process_failed_batches if @buffer_config && @buffer_config[:process_failed_batches_on_shutdown] end private diff --git a/lib/logstash/outputs/kusto/filePersistence.rb b/lib/logstash/outputs/kusto/filePersistence.rb index 6a167694..2b8ca7a0 100644 --- a/lib/logstash/outputs/kusto/filePersistence.rb +++ b/lib/logstash/outputs/kusto/filePersistence.rb @@ -4,17 +4,25 @@ module LogStash; module Outputs; class KustoOutputInternal module FilePersistence - FAILED_DIR = './tmp/buffer_storage/' + @failed_dir = './tmp/buffer_storage/' # default + + def self.failed_dir=(dir) + @failed_dir = dir + end + + def self.failed_dir + @failed_dir + end def self.persist_batch(batch) - ::FileUtils.mkdir_p(FAILED_DIR) - filename = ::File.join(FAILED_DIR, "failed_batch_#{Time.now.to_i}_#{SecureRandom.uuid}.json") + ::FileUtils.mkdir_p(@failed_dir) + filename = ::File.join(@failed_dir, "failed_batch_#{Time.now.to_i}_#{SecureRandom.uuid}.json") ::File.write(filename, JSON.dump(batch)) end def self.load_batches - return [] unless Dir.exist?(FAILED_DIR) - Dir.glob(::File.join(FAILED_DIR, 'failed_batch_*.json')).map do |file| + return [] unless Dir.exist?(@failed_dir) + Dir.glob(::File.join(@failed_dir, 'failed_batch_*.json')).map do |file| [file, JSON.load(::File.read(file))] end end diff --git a/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb b/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb index 0cef40ab..47606546 100644 --- a/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb +++ b/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb @@ -159,14 +159,12 @@ def is_mapping_ref_provided end end # class KustoIngestionConfiguration class KustoFlushConfiguration - def initialize(max_items, plugin_flush_interval, max_batch_size, process_failed_batches_on_startup, - process_failed_batches_on_shutdown) + def initialize(max_items, plugin_flush_interval, max_batch_size, process_failed_batches_on_startup) @max_items = max_items @plugin_flush_interval = plugin_flush_interval @max_batch_size = max_batch_size @flush_each = flush_each @process_failed_batches_on_startup = process_failed_batches_on_startup - @process_failed_batches_on_shutdown = process_failed_batches_on_shutdown end # Flush configuration def max_items @@ -184,10 +182,7 @@ def flush_each def process_failed_batches_on_startup @process_failed_batches_on_startup end - - def process_failed_batches_on_shutdown - @process_failed_batches_on_shutdown - end + end # class KustoFlushConfiguration class KustoUploadConfiguration def initialize(upload_concurrent_count, upload_queue_size) diff --git a/lib/logstash/outputs/kusto/logStashFlushBuffer.rb b/lib/logstash/outputs/kusto/logStashFlushBuffer.rb index 20d97f7c..a996a34d 100644 --- a/lib/logstash/outputs/kusto/logStashFlushBuffer.rb +++ b/lib/logstash/outputs/kusto/logStashFlushBuffer.rb @@ -25,7 +25,6 @@ def initialize(kusto_logstash_configuration,logger) #todo: There is a small discrepancy between the total size of the documents and the message body :flush_each => kusto_logstash_configuration.kusto_flush_config.max_batch_size, :process_failed_batches_on_startup => kusto_logstash_configuration.kusto_flush_config.process_failed_batches_on_startup, - :process_failed_batches_on_shutdown => kusto_logstash_configuration.kusto_flush_config.process_failed_batches_on_shutdown ) end # initialize diff --git a/spec/outputs/kusto/filePersistence_spec.rb b/spec/outputs/kusto/filePersistence_spec.rb new file mode 100644 index 00000000..3a6cb77f --- /dev/null +++ b/spec/outputs/kusto/filePersistence_spec.rb @@ -0,0 +1,48 @@ +# encoding: utf-8 +require_relative "../../spec_helpers.rb" +require 'logstash/outputs/kusto/filePersistence' +require 'fileutils' + +describe LogStash::Outputs::KustoOutputInternal::FilePersistence do + let(:tmp_dir) { File.expand_path("../../../../tmp/test_buffer_storage", __FILE__) } + let(:batch) { [{ "foo" => "bar" }, { "baz" => "qux" }] } + + before(:each) do + described_class.failed_dir = tmp_dir + FileUtils.rm_rf(tmp_dir) + FileUtils.mkdir_p(tmp_dir) + end + + after(:each) do + FileUtils.rm_rf(tmp_dir) + end + + it 'persists a batch to a file and loads it back' do + described_class.persist_batch(batch) + files = Dir.glob(File.join(tmp_dir, 'failed_batch_*.json')) + expect(files.size).to eq(1) + loaded = described_class.load_batches + expect(loaded.size).to eq(1) + expect(loaded.first[1]).to eq(batch) + end + + it 'deletes a batch file' do + described_class.persist_batch(batch) + file = Dir.glob(File.join(tmp_dir, 'failed_batch_*.json')).first + expect(File.exist?(file)).to be true + described_class.delete_batch(file) + expect(File.exist?(file)).to be false + end + + it 'does not fail if directory does not exist' do + FileUtils.rm_rf(tmp_dir) + expect { described_class.persist_batch(batch) }.not_to raise_error + files = Dir.glob(File.join(tmp_dir, 'failed_batch_*.json')) + expect(files.size).to eq(1) + end + + it 'returns empty array if directory does not exist' do + FileUtils.rm_rf(tmp_dir) + expect(described_class.load_batches).to eq([]) + end +end \ No newline at end of file diff --git a/spec/outputs/kusto/kustoLogstashConfiguration_spec.rb b/spec/outputs/kusto/kustoLogstashConfiguration_spec.rb index cbb09ea7..1e1eb923 100755 --- a/spec/outputs/kusto/kustoLogstashConfiguration_spec.rb +++ b/spec/outputs/kusto/kustoLogstashConfiguration_spec.rb @@ -20,87 +20,85 @@ let(:delete_local) { false } let(:logger) { spy(:logger) } let(:proxy_aad_only) { false } + let(:max_items) { 100 } + let(:plugin_flush_interval) { 10 } + let(:max_batch_size) { 10 } + let(:process_failed_batches_on_startup) { false } + let(:upload_concurrent_count) { 3 } + let(:upload_queue_size) { 30 } let(:kusto_ingest_base) { LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, database, table, json_mapping) } let(:kusto_auth_base) { LogStash::Outputs::KustoInternal::KustoAuthConfiguration.new(app_id, app_key, app_tenant, managed_identity, cliauth) } - let(:kusto_proxy_base) { LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new(proxy_host , proxy_port , proxy_protocol, false) } + let(:kusto_proxy_base) { LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new(proxy_host , proxy_port , proxy_protocol, proxy_aad_only) } + let(:kusto_flush_config) { LogStash::Outputs::KustoInternal::KustoFlushConfiguration.new(max_items, plugin_flush_interval, max_batch_size, process_failed_batches_on_startup) } + let(:kusto_upload_config) { LogStash::Outputs::KustoInternal::KustoUploadConfiguration.new(upload_concurrent_count, upload_queue_size) } describe '#initialize' do it 'does not throw an error when initializing' do - # note that this will cause an internal error since connection is being tried. - # however we still want to test that all the java stuff is working as expected expect { - kustoLogstashOutputConfiguration = described_class.new(kusto_ingest_base, kusto_auth_base , kusto_proxy_base, logger) + kustoLogstashOutputConfiguration = described_class.new(kusto_ingest_base, kusto_auth_base , kusto_proxy_base, kusto_flush_config, kusto_upload_config, logger) kustoLogstashOutputConfiguration.validate_config() }.not_to raise_error end - - dynamic_name_array = ['/a%{name}/', '/a %{name}/', '/a- %{name}/', '/a- %{name}'] - context 'doesnt allow database to have some dynamic part' do - dynamic_name_array.each do |test_database| - it "with database: #{test_database}" do - expect { - kusto_ingest = LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, test_database, table, json_mapping) - kustoLogstashOutputConfiguration = described_class.new(kusto_ingest, kusto_auth_base , kusto_proxy_base, logger) - kustoLogstashOutputConfiguration.validate_config() - }.to raise_error(LogStash::ConfigurationError) - end - end + it 'exposes all configuration accessors' do + config = described_class.new(kusto_ingest_base, kusto_auth_base, kusto_proxy_base, kusto_flush_config, kusto_upload_config, logger) + expect(config.kusto_ingest).to eq(kusto_ingest_base) + expect(config.kusto_auth).to eq(kusto_auth_base) + expect(config.kusto_proxy).to eq(kusto_proxy_base) + expect(config.kusto_flush_config).to eq(kusto_flush_config) + expect(config.kusto_upload_config).to eq(kusto_upload_config) end + end - context 'doesnt allow table to have some dynamic part' do - dynamic_name_array.each do |test_table| - it "with database: #{test_table}" do - expect { - kusto_ingest = LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, database, test_table, json_mapping) - kustoLogstashOutputConfiguration = described_class.new(kusto_ingest, kusto_auth_base , kusto_proxy_base, logger) - kustoLogstashOutputConfiguration.validate_config() - }.to raise_error(LogStash::ConfigurationError) - end - end + describe LogStash::Outputs::KustoInternal::KustoAuthConfiguration do + it 'returns correct values for auth config' do + auth = described_class.new(app_id, app_key, app_tenant, managed_identity, cliauth) + expect(auth.app_id).to eq(app_id) + expect(auth.app_key).to eq(app_key) + expect(auth.app_tenant).to eq(app_tenant) + expect(auth.managed_identity_id).to eq(managed_identity) + expect(auth.cli_auth).to eq(cliauth) end + end - context 'doesnt allow mapping to have some dynamic part' do - dynamic_name_array.each do |json_mapping| - it "with database: #{json_mapping}" do - expect { - kusto_ingest = LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, database, table, json_mapping) - kustoLogstashOutputConfiguration = described_class.new(kusto_ingest, kusto_auth_base , kusto_proxy_base, logger) - kustoLogstashOutputConfiguration.validate_config() - }.to raise_error(LogStash::ConfigurationError) - end - end + describe LogStash::Outputs::KustoInternal::KustoProxyConfiguration do + it 'returns correct values for proxy config' do + proxy = described_class.new(proxy_host, proxy_port, proxy_protocol, proxy_aad_only) + expect(proxy.proxy_host).to eq(proxy_host) + expect(proxy.proxy_port).to eq(proxy_port) + expect(proxy.proxy_protocol).to eq(proxy_protocol) + expect(proxy.proxy_aad_only).to eq(proxy_aad_only) + expect(proxy.is_direct_conn).to eq(false) end + end - context 'proxy protocol has to be http or https' do - it "with proxy protocol: socks" do - expect { - kusto_proxy = LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new(proxy_host , proxy_port , 'socks', false) - kustoLogstashOutputConfiguration = described_class.new(kusto_ingest_base, kusto_auth_base , kusto_proxy, logger) - kustoLogstashOutputConfiguration.validate_config() - }.to raise_error(LogStash::ConfigurationError) - end + describe LogStash::Outputs::KustoInternal::KustoIngestConfiguration do + it 'returns correct values for ingest config' do + ingest = described_class.new(ingest_url, database, table, json_mapping) + expect(ingest.ingest_url).to eq(ingest_url) + expect(ingest.database).to eq(database) + expect(ingest.table).to eq(table) + expect(ingest.json_mapping).to eq(json_mapping) + expect(ingest.is_mapping_ref_provided).to eq(true) end + end - context 'one of appid or managedid or cli_auth has to be provided' do - it "with empty managed identity and appid" do - expect { - kusto_auth = LogStash::Outputs::KustoInternal::KustoAuthConfiguration.new("", app_key, app_tenant, "", false) - kustoLogstashOutputConfiguration = described_class.new(kusto_ingest_base, kusto_auth , kusto_proxy_base, logger) - kustoLogstashOutputConfiguration.validate_config() - }.to raise_error(LogStash::ConfigurationError) - end + describe LogStash::Outputs::KustoInternal::KustoFlushConfiguration do + it 'returns correct values for flush config' do + flush = described_class.new(max_items, plugin_flush_interval, max_batch_size, process_failed_batches_on_startup) + expect(flush.max_items).to eq(max_items) + expect(flush.plugin_flush_interval).to eq(plugin_flush_interval) + expect(flush.max_batch_size).to eq(max_batch_size) + expect(flush.process_failed_batches_on_startup).to eq(process_failed_batches_on_startup) end + end - context 'if proxy_aad is provided' do - it "proxy details should be provided" do - expect { - kusto_proxy = LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new("" , "" , proxy_protocol, true) - kustoLogstashOutputConfiguration = described_class.new(kusto_ingest_base, kusto_auth_base , kusto_proxy, logger) - kustoLogstashOutputConfiguration.validate_config() - }.to raise_error(LogStash::ConfigurationError) - end + describe LogStash::Outputs::KustoInternal::KustoUploadConfiguration do + it 'returns correct values for upload config' do + upload = described_class.new(upload_concurrent_count, upload_queue_size) + expect(upload.upload_concurrent_count).to eq(upload_concurrent_count) + expect(upload.upload_queue_size).to eq(upload_queue_size) end end end \ No newline at end of file From 0b8550ec1416df8c8b174e2f7ca0c6c1b04aff50 Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Mon, 16 Jun 2025 14:28:10 +0530 Subject: [PATCH 42/52] Upgrade gradle version --- gradle/wrapper/gradle-wrapper.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index ffed3a25..48c0a02c 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,5 +1,5 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-7.2-bin.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-8.7-bin.zip zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists From f2024ee5e29186c8827a817904a2cdffecb053c1 Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Mon, 16 Jun 2025 14:37:47 +0530 Subject: [PATCH 43/52] Update e2e test --- e2e/e2e.rb | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/e2e/e2e.rb b/e2e/e2e.rb index 04cc0e80..f8a24b11 100755 --- a/e2e/e2e.rb +++ b/e2e/e2e.rb @@ -7,8 +7,8 @@ class E2E def initialize super - @input_file = "input_file.txt" - @output_file = "output_file.txt" + @input_file = File.expand_path("input_file.txt", __dir__) + @output_file = File.expand_path("output_file.txt", __dir__) @columns = "(rownumber:int, rowguid:string, xdouble:real, xfloat:real, xbool:bool, xint16:int, xint32:int, xint64:long, xuint8:long, xuint16:long, xuint32:long, xuint64:long, xdate:datetime, xsmalltext:string, xtext:string, xnumberAsText:string, xtime:timespan, xtextWithNulls:string, xdynamicWithNulls:dynamic)" @csv_columns = '"rownumber", "rowguid", "xdouble", "xfloat", "xbool", "xint16", "xint32", "xint64", "xuint8", "xuint16", "xuint32", "xuint64", "xdate", "xsmalltext", "xtext", "xnumberAsText", "xtime", "xtextWithNulls", "xdynamicWithNulls"' @column_count = 19 From f29b5d8534d594bcc8e4b9044d3a716cd852a2fe Mon Sep 17 00:00:00 2001 From: ag-ramachandran Date: Tue, 17 Jun 2025 19:17:35 +0530 Subject: [PATCH 44/52] *Minor edits --- lib/logstash/outputs/kusto/filePersistence.rb | 2 +- lib/logstash/outputs/kusto/ingestor.rb | 8 ++------ run.sh | 1 - 3 files changed, 3 insertions(+), 8 deletions(-) delete mode 100755 run.sh diff --git a/lib/logstash/outputs/kusto/filePersistence.rb b/lib/logstash/outputs/kusto/filePersistence.rb index 2b8ca7a0..aa1e3c07 100644 --- a/lib/logstash/outputs/kusto/filePersistence.rb +++ b/lib/logstash/outputs/kusto/filePersistence.rb @@ -4,7 +4,7 @@ module LogStash; module Outputs; class KustoOutputInternal module FilePersistence - @failed_dir = './tmp/buffer_storage/' # default + @failed_dir = '/tmp/buffer_storage/' # default def self.failed_dir=(dir) @failed_dir = dir diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index c4d09c56..fc07cba4 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -104,20 +104,16 @@ def upload(data) raise e } .on_resolution do |fulfilled, value, reason, *args| - @logger.info("******************************************************************************************") - @logger.info("Future fulfilled: #{fulfilled}, value: #{value}, reason: #{reason}, args: #{args}, class: #{value.class}") - #@logger.info("Ingestion status: #{value.getIngestionStatusCollection().getStatus}") - + @logger.debug("Future fulfilled: #{fulfilled}, value: #{value}, reason: #{reason}, args: #{args}, class: #{value.class}") if value.class == Java::ComMicrosoftAzureKustoIngestResult::IngestionStatusResult isc = value.getIngestionStatusCollection()&.get(0)&.getStatus() @logger.info("Ingestion status: #{isc}") else - @logger.info("Ingestion status is non success status: #{value.class} - #{value}") + @logger.warn("Ingestion status is non success status: #{value.class} - #{value}") end if exceptions.size > 0 @logger.error("Ingestion failed with exceptions: #{exceptions.map(&:message).join(', ')}") end - @logger.info("******************************************************************************************") end else @logger.warn("Data is empty and is not ingested.") diff --git a/run.sh b/run.sh deleted file mode 100755 index a354c39f..00000000 --- a/run.sh +++ /dev/null @@ -1 +0,0 @@ -/softwares/logstash/bin/logstash -f /code/logstash/logstash-output-kusto/e2e/logstash.conf \ No newline at end of file From 24f0f6fc03cfabf14b7f29336efe0d51aa4d95ff Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Wed, 18 Jun 2025 16:31:31 +0530 Subject: [PATCH 45/52] Updated file persistance logic --- lib/logstash/outputs/kusto.rb | 45 ++++++++++--------- .../outputs/kusto/customSizeBasedBuffer.rb | 6 +-- lib/logstash/outputs/kusto/filePersistence.rb | 22 +++++---- lib/logstash/outputs/kusto/ingestor.rb | 4 +- .../kusto/kustoLogstashConfiguration.rb | 8 ++-- .../outputs/kusto/logStashFlushBuffer.rb | 3 +- 6 files changed, 45 insertions(+), 43 deletions(-) diff --git a/lib/logstash/outputs/kusto.rb b/lib/logstash/outputs/kusto.rb index fb6ef276..bc2ad7ee 100755 --- a/lib/logstash/outputs/kusto.rb +++ b/lib/logstash/outputs/kusto.rb @@ -65,36 +65,37 @@ class LogStash::Outputs::Kusto < LogStash::Outputs::Base # Check Proxy URL can be over http or https. Do we need it this way or ignore this & remove this config :proxy_protocol, validate: :string, required: false , default: 'https' + # Maximum size of the buffer before it gets flushed, in MB. Defaults to 10. + config :max_batch_size, validate: :number, required: false, default: 10 - # Maximum size of the buffer before it gets flushed, defaults to 10MB - config :max_batch_size, validate: :number, required: false , default: 10 + # Interval (in seconds) before the buffer gets flushed, regardless of size. Defaults to 10. + config :plugin_flush_interval, validate: :number, required: false, default: 10 - # Maximum interval (in seconds) before the buffer gets flushed, defaults to 10 - config :plugin_flush_interval, validate: :number, required: false , default: 10 + # Maximum number of items in the buffer before it gets flushed. Defaults to 1000. + config :max_items, validate: :number, required: false, default: 1000 - # Maximum interval (in seconds) before the buffer gets flushed, defaults to 10 - config :max_items, validate: :number, required: false , default: 100 - - # Process failed batches on startup + # Process failed batches on startup. Defaults to false. config :process_failed_batches_on_startup, validate: :boolean, required: false, default: false - # Directory to store the failed batches that were not uploaded to Kusto. If the directory does not exist, it will be created, defaults to './tmp/buffer_storage/' - config :failed_dir_name, validate: :string, required: false, default: './tmp/buffer_storage/' + # Directory to store the failed batches that were not uploaded to Kusto. If the directory does not exist, it will be created, defaults to a temporary directory called "logstash_backout" in Dir.tmpdir + config :failed_dir_name, validate: :string, required: false, default: nil default :codec, 'json_lines' - def register - LogStash::Outputs::KustoOutputInternal::FilePersistence.failed_dir = failed_dir_name - kusto_ingest_base = LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, database, table, json_mapping) - kusto_auth_base = LogStash::Outputs::KustoInternal::KustoAuthConfiguration.new(app_id, app_key, app_tenant, managed_identity, cli_auth) - kusto_proxy_base = LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new(proxy_host , proxy_port , proxy_protocol, false) - kusto_flush_config = LogStash::Outputs::KustoInternal::KustoFlushConfiguration.new(max_items, plugin_flush_interval, max_batch_size, process_failed_batches_on_startup) - kusto_upload_config = LogStash::Outputs::KustoInternal::KustoUploadConfiguration.new(upload_concurrent_count, upload_queue_size) - kusto_logstash_configuration = LogStash::Outputs::KustoInternal::KustoLogstashConfiguration.new(kusto_ingest_base, kusto_auth_base , kusto_proxy_base, kusto_flush_config, kusto_upload_config, @logger) - kusto_logstash_configuration.validate_config - # Initialize the custom buffer with size and interval - @buffer = LogStash::Outputs::KustoOutputInternal::LogStashEventsBatcher.new(kusto_logstash_configuration,@logger) - end + def register + dir = failed_dir_name.nil? || failed_dir_name.empty? ? ::File.join(Dir.tmpdir, "logstash_backout") : failed_dir_name + @file_persistence = LogStash::Outputs::KustoOutputInternal::FilePersistence.new(dir, @logger) + + kusto_ingest_base = LogStash::Outputs::KustoInternal::KustoIngestConfiguration.new(ingest_url, database, table, json_mapping) + kusto_auth_base = LogStash::Outputs::KustoInternal::KustoAuthConfiguration.new(app_id, app_key, app_tenant, managed_identity, cli_auth) + kusto_proxy_base = LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new(proxy_host , proxy_port , proxy_protocol, false) + kusto_flush_config = LogStash::Outputs::KustoInternal::KustoFlushConfiguration.new(max_items, plugin_flush_interval, max_batch_size, process_failed_batches_on_startup) + kusto_upload_config = LogStash::Outputs::KustoInternal::KustoUploadConfiguration.new(upload_concurrent_count, upload_queue_size) + kusto_logstash_configuration = LogStash::Outputs::KustoInternal::KustoLogstashConfiguration.new(kusto_ingest_base, kusto_auth_base , kusto_proxy_base, kusto_flush_config, kusto_upload_config, @logger, @file_persistence) + kusto_logstash_configuration.validate_config + # Initialize the custom buffer with size and interval + @buffer = LogStash::Outputs::KustoOutputInternal::LogStashEventsBatcher.new(kusto_logstash_configuration, @logger) + end public diff --git a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb index aebc6579..a4cb6ef6 100644 --- a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb +++ b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb @@ -86,6 +86,7 @@ def buffer_initialize(options={}) if ! self.class.method_defined?(:flush) raise ArgumentError, "Any class including Stud::Buffer must define a flush() method." end + @file_persistence = options[:file_persistence] @buffer_config = { :max_items => options[:max_items] || 50, @@ -301,13 +302,12 @@ def buffer_flush(options={}) end def process_failed_batches - require_relative 'filePersistence' - LogStash::Outputs::KustoOutputInternal::FilePersistence.load_batches.each do |file, batch| + @file_persistence.load_batches.each do |file, batch| begin @buffer_state[:flush_mutex].lock begin flush(batch, true) - LogStash::Outputs::KustoOutputInternal::FilePersistence.delete_batch(file) + @file_persistence.delete_batch(file) @buffer_config[:logger].info("Successfully flushed and deleted failed batch file: #{file}") if @buffer_config[:logger] rescue => e @buffer_config[:logger].warn("Failed to flush persisted batch: #{e.message}") if @buffer_config[:logger] diff --git a/lib/logstash/outputs/kusto/filePersistence.rb b/lib/logstash/outputs/kusto/filePersistence.rb index aa1e3c07..0049acaa 100644 --- a/lib/logstash/outputs/kusto/filePersistence.rb +++ b/lib/logstash/outputs/kusto/filePersistence.rb @@ -1,33 +1,31 @@ require 'securerandom' require 'json' require 'fileutils' +require 'tmpdir' module LogStash; module Outputs; class KustoOutputInternal - module FilePersistence - @failed_dir = '/tmp/buffer_storage/' # default + class FilePersistence + attr_reader :failed_dir - def self.failed_dir=(dir) - @failed_dir = dir - end - - def self.failed_dir - @failed_dir + def initialize(dir = nil, logger = nil) + @failed_dir = dir || ::File.join(Dir.tmpdir, "logstash_backout") + ::FileUtils.mkdir_p(@failed_dir) + logger&.info("Backup file directory for failed batches: #{::File.expand_path(@failed_dir)}") end - def self.persist_batch(batch) - ::FileUtils.mkdir_p(@failed_dir) + def persist_batch(batch) filename = ::File.join(@failed_dir, "failed_batch_#{Time.now.to_i}_#{SecureRandom.uuid}.json") ::File.write(filename, JSON.dump(batch)) end - def self.load_batches + def load_batches return [] unless Dir.exist?(@failed_dir) Dir.glob(::File.join(@failed_dir, 'failed_batch_*.json')).map do |file| [file, JSON.load(::File.read(file))] end end - def self.delete_batch(file) + def delete_batch(file) ::File.delete(file) if ::File.exist?(file) end end diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index fc07cba4..bc30c87a 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -5,7 +5,6 @@ require 'logstash/errors' require 'concurrent' require 'json' -require 'logstash/outputs/kusto/filePersistence' module LogStash; module Outputs; class KustoOutputInternal ## @@ -20,6 +19,7 @@ class Ingestor def initialize(kusto_logstash_configuration, logger) @kusto_logstash_configuration = kusto_logstash_configuration @logger = logger + @file_persistence = kusto_logstash_configuration.file_persistence @workers_pool = Concurrent::ThreadPoolExecutor.new(min_threads: 1, max_threads: kusto_logstash_configuration.kusto_upload_config.upload_concurrent_count, max_queue: kusto_logstash_configuration.kusto_upload_config.upload_queue_size, @@ -100,7 +100,7 @@ def upload(data) .rescue{ |e| @logger.error("Ingestion failed: #{e.message}") @logger.error("Ingestion failed: #{e.backtrace.join("\n")}") - LogStash::Outputs::KustoOutputInternal::FilePersistence.persist_batch(data) + @file_persistence.persist_batch(data) raise e } .on_resolution do |fulfilled, value, reason, *args| diff --git a/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb b/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb index 47606546..72eac768 100644 --- a/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb +++ b/lib/logstash/outputs/kusto/kustoLogstashConfiguration.rb @@ -5,13 +5,14 @@ module Outputs module KustoInternal class KustoLogstashConfiguration FIELD_REF = /%\{[^}]+\}/ - def initialize(kusto_ingest,kusto_auth, kusto_proxy, kusto_flush_config , kusto_upload_config, logger) + def initialize(kusto_ingest, kusto_auth, kusto_proxy, kusto_flush_config, kusto_upload_config, logger, file_persistence) @logger = logger @kusto_ingest = kusto_ingest @kusto_auth = kusto_auth @kusto_proxy = kusto_proxy @kusto_flush_config = kusto_flush_config @kusto_upload_config = kusto_upload_config + @file_persistence = file_persistence @logger.info("Kusto configuration initialized.") end # def initialize @@ -31,7 +32,9 @@ def kusto_flush_config def kusto_upload_config @kusto_upload_config end - + def file_persistence + @file_persistence + end def validate_config() # Add an additional validation and fail this upfront if @kusto_auth.app_id.to_s.empty? && @kusto_auth.managed_identity_id.to_s.empty? && !@kusto_auth.cli_auth @@ -182,7 +185,6 @@ def flush_each def process_failed_batches_on_startup @process_failed_batches_on_startup end - end # class KustoFlushConfiguration class KustoUploadConfiguration def initialize(upload_concurrent_count, upload_queue_size) diff --git a/lib/logstash/outputs/kusto/logStashFlushBuffer.rb b/lib/logstash/outputs/kusto/logStashFlushBuffer.rb index a996a34d..0c315974 100644 --- a/lib/logstash/outputs/kusto/logStashFlushBuffer.rb +++ b/lib/logstash/outputs/kusto/logStashFlushBuffer.rb @@ -8,7 +8,7 @@ module LogStash; module Outputs; class KustoOutputInternal class LogStashEventsBatcher include CustomSizeBasedBuffer - def initialize(kusto_logstash_configuration,logger) + def initialize(kusto_logstash_configuration, logger) logger.info("Initializing LogStashEventsBatcher") # Initialize the buffer with the configuration # The buffer is a custom buffer that extends the LogStash::Outputs::Base#buffer_initialize @@ -25,6 +25,7 @@ def initialize(kusto_logstash_configuration,logger) #todo: There is a small discrepancy between the total size of the documents and the message body :flush_each => kusto_logstash_configuration.kusto_flush_config.max_batch_size, :process_failed_batches_on_startup => kusto_logstash_configuration.kusto_flush_config.process_failed_batches_on_startup, + :file_persistence => kusto_logstash_configuration.file_persistence ) end # initialize From 19ed1845c0ce5edb6be00698bd237dd6586c3548 Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Wed, 18 Jun 2025 16:38:19 +0530 Subject: [PATCH 46/52] Updated spec tests --- spec/outputs/kusto/filePersistence_spec.rb | 15 ++++++++------- .../kusto/kustoLogstashConfiguration_spec.rb | 11 +++++++++-- 2 files changed, 17 insertions(+), 9 deletions(-) diff --git a/spec/outputs/kusto/filePersistence_spec.rb b/spec/outputs/kusto/filePersistence_spec.rb index 3a6cb77f..9802b3d9 100644 --- a/spec/outputs/kusto/filePersistence_spec.rb +++ b/spec/outputs/kusto/filePersistence_spec.rb @@ -6,9 +6,10 @@ describe LogStash::Outputs::KustoOutputInternal::FilePersistence do let(:tmp_dir) { File.expand_path("../../../../tmp/test_buffer_storage", __FILE__) } let(:batch) { [{ "foo" => "bar" }, { "baz" => "qux" }] } + let(:logger) { double("Logger", info: nil) } + let(:file_persistence) { described_class.new(tmp_dir, logger) } before(:each) do - described_class.failed_dir = tmp_dir FileUtils.rm_rf(tmp_dir) FileUtils.mkdir_p(tmp_dir) end @@ -18,31 +19,31 @@ end it 'persists a batch to a file and loads it back' do - described_class.persist_batch(batch) + file_persistence.persist_batch(batch) files = Dir.glob(File.join(tmp_dir, 'failed_batch_*.json')) expect(files.size).to eq(1) - loaded = described_class.load_batches + loaded = file_persistence.load_batches expect(loaded.size).to eq(1) expect(loaded.first[1]).to eq(batch) end it 'deletes a batch file' do - described_class.persist_batch(batch) + file_persistence.persist_batch(batch) file = Dir.glob(File.join(tmp_dir, 'failed_batch_*.json')).first expect(File.exist?(file)).to be true - described_class.delete_batch(file) + file_persistence.delete_batch(file) expect(File.exist?(file)).to be false end it 'does not fail if directory does not exist' do FileUtils.rm_rf(tmp_dir) - expect { described_class.persist_batch(batch) }.not_to raise_error + expect { file_persistence.persist_batch(batch) }.not_to raise_error files = Dir.glob(File.join(tmp_dir, 'failed_batch_*.json')) expect(files.size).to eq(1) end it 'returns empty array if directory does not exist' do FileUtils.rm_rf(tmp_dir) - expect(described_class.load_batches).to eq([]) + expect(file_persistence.load_batches).to eq([]) end end \ No newline at end of file diff --git a/spec/outputs/kusto/kustoLogstashConfiguration_spec.rb b/spec/outputs/kusto/kustoLogstashConfiguration_spec.rb index 1e1eb923..c6c8b9c9 100755 --- a/spec/outputs/kusto/kustoLogstashConfiguration_spec.rb +++ b/spec/outputs/kusto/kustoLogstashConfiguration_spec.rb @@ -32,25 +32,32 @@ let(:kusto_proxy_base) { LogStash::Outputs::KustoInternal::KustoProxyConfiguration.new(proxy_host , proxy_port , proxy_protocol, proxy_aad_only) } let(:kusto_flush_config) { LogStash::Outputs::KustoInternal::KustoFlushConfiguration.new(max_items, plugin_flush_interval, max_batch_size, process_failed_batches_on_startup) } let(:kusto_upload_config) { LogStash::Outputs::KustoInternal::KustoUploadConfiguration.new(upload_concurrent_count, upload_queue_size) } + let(:file_persistence) { double("FilePersistence") } describe '#initialize' do it 'does not throw an error when initializing' do expect { - kustoLogstashOutputConfiguration = described_class.new(kusto_ingest_base, kusto_auth_base , kusto_proxy_base, kusto_flush_config, kusto_upload_config, logger) + kustoLogstashOutputConfiguration = described_class.new( + kusto_ingest_base, kusto_auth_base, kusto_proxy_base, kusto_flush_config, kusto_upload_config, logger, file_persistence + ) kustoLogstashOutputConfiguration.validate_config() }.not_to raise_error end it 'exposes all configuration accessors' do - config = described_class.new(kusto_ingest_base, kusto_auth_base, kusto_proxy_base, kusto_flush_config, kusto_upload_config, logger) + config = described_class.new( + kusto_ingest_base, kusto_auth_base, kusto_proxy_base, kusto_flush_config, kusto_upload_config, logger, file_persistence + ) expect(config.kusto_ingest).to eq(kusto_ingest_base) expect(config.kusto_auth).to eq(kusto_auth_base) expect(config.kusto_proxy).to eq(kusto_proxy_base) expect(config.kusto_flush_config).to eq(kusto_flush_config) expect(config.kusto_upload_config).to eq(kusto_upload_config) + expect(config.file_persistence).to eq(file_persistence) end end + describe LogStash::Outputs::KustoInternal::KustoAuthConfiguration do it 'returns correct values for auth config' do auth = described_class.new(app_id, app_key, app_tenant, managed_identity, cliauth) From 1fbeffdcb17b93e4739ad1e39af3e3b27b4afc0b Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Thu, 19 Jun 2025 11:25:01 +0530 Subject: [PATCH 47/52] Updated filePersistence class and added error handling. Updated spec test. --- lib/logstash/outputs/kusto/filePersistence.rb | 71 ++++++++++++++++--- spec/outputs/kusto/filePersistence_spec.rb | 3 +- 2 files changed, 65 insertions(+), 9 deletions(-) diff --git a/lib/logstash/outputs/kusto/filePersistence.rb b/lib/logstash/outputs/kusto/filePersistence.rb index 0049acaa..eda21a47 100644 --- a/lib/logstash/outputs/kusto/filePersistence.rb +++ b/lib/logstash/outputs/kusto/filePersistence.rb @@ -2,6 +2,7 @@ require 'json' require 'fileutils' require 'tmpdir' +require 'thread' module LogStash; module Outputs; class KustoOutputInternal class FilePersistence @@ -9,24 +10,78 @@ class FilePersistence def initialize(dir = nil, logger = nil) @failed_dir = dir || ::File.join(Dir.tmpdir, "logstash_backout") - ::FileUtils.mkdir_p(@failed_dir) - logger&.info("Backup file directory for failed batches: #{::File.expand_path(@failed_dir)}") + begin + ::FileUtils.mkdir_p(@failed_dir) unless Dir.exist?(@failed_dir) + rescue => e + logger&.fatal("Failed to create backup directory #{@failed_dir}: #{e.message}") + raise + end + @logger = logger + @write_mutex = Mutex.new + @logger&.info("Backup file directory for failed batches: #{::File.expand_path(@failed_dir)}") end - def persist_batch(batch) - filename = ::File.join(@failed_dir, "failed_batch_#{Time.now.to_i}_#{SecureRandom.uuid}.json") - ::File.write(filename, JSON.dump(batch)) + def persist_batch(batch, max_retries = 3) + attempts = 0 + begin + @write_mutex.synchronize do + tmpfile = ::File.join(@failed_dir, "tmp_#{SecureRandom.uuid}.json") + filename = ::File.join(@failed_dir, "failed_batch_#{Time.now.to_i}_#{SecureRandom.uuid}.json") + begin + ::File.write(tmpfile, JSON.dump(batch)) + ::File.rename(tmpfile, filename) + return # Success! + rescue => e + @logger&.error("Failed to persist batch to #{filename}: #{e.message}") + begin + ::File.delete(tmpfile) if ::File.exist?(tmpfile) + rescue + # Ignore cleanup errors + end + raise + end + end + rescue => e + attempts += 1 + if attempts < max_retries + sleep 0.1 * attempts # Exponential backoff + retry + else + @logger&.fatal("Failed to persist batch after #{attempts} attempts. Data loss may occur: #{e.message}") + end + end end def load_batches return [] unless Dir.exist?(@failed_dir) - Dir.glob(::File.join(@failed_dir, 'failed_batch_*.json')).map do |file| - [file, JSON.load(::File.read(file))] + return enum_for(:load_batches) unless block_given? + Dir.glob(::File.join(@failed_dir, 'failed_batch_*.json')).each do |file| + begin + yield file, JSON.load(::File.read(file)) + rescue => e + if e.is_a?(Errno::ENOENT) + @logger&.warn("Batch file #{file} was not found when attempting to read. It may have been deleted by another process.") + else + @logger&.warn("Failed to load batch file #{file}: #{e.message}. Moving to quarantine.") + begin + quarantine_dir = File.join(@failed_dir, "quarantine") + FileUtils.mkdir_p(quarantine_dir) unless Dir.exist?(quarantine_dir) + FileUtils.mv(file, quarantine_dir) + rescue => del_err + @logger&.warn("Failed to move corrupted batch file #{file} to quarantine: #{del_err.message}") + end + end + next + end end end def delete_batch(file) - ::File.delete(file) if ::File.exist?(file) + begin + ::File.delete(file) if ::File.exist?(file) + rescue => e + @logger&.warn("Failed to delete batch file #{file}: #{e.message}") + end end end end; end; end \ No newline at end of file diff --git a/spec/outputs/kusto/filePersistence_spec.rb b/spec/outputs/kusto/filePersistence_spec.rb index 9802b3d9..738cc111 100644 --- a/spec/outputs/kusto/filePersistence_spec.rb +++ b/spec/outputs/kusto/filePersistence_spec.rb @@ -22,7 +22,7 @@ file_persistence.persist_batch(batch) files = Dir.glob(File.join(tmp_dir, 'failed_batch_*.json')) expect(files.size).to eq(1) - loaded = file_persistence.load_batches + loaded = file_persistence.load_batches.to_a expect(loaded.size).to eq(1) expect(loaded.first[1]).to eq(batch) end @@ -43,6 +43,7 @@ end it 'returns empty array if directory does not exist' do + file_persistence = described_class.new(tmp_dir, logger) # Re-instantiate! FileUtils.rm_rf(tmp_dir) expect(file_persistence.load_batches).to eq([]) end From 5f7098af062a96e8da469967adaa97d20686d3b2 Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Tue, 1 Jul 2025 14:23:20 +0530 Subject: [PATCH 48/52] Updated process_failed_files and shutdown fn --- .../outputs/kusto/customSizeBasedBuffer.rb | 21 +++++++++++++++---- lib/logstash/outputs/kusto/filePersistence.rb | 2 +- 2 files changed, 18 insertions(+), 5 deletions(-) diff --git a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb index a4cb6ef6..390aafa9 100644 --- a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb +++ b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb @@ -97,6 +97,9 @@ def buffer_initialize(options={}) :has_on_flush_error => self.class.method_defined?(:on_flush_error), :has_on_full_buffer_receive => self.class.method_defined?(:on_full_buffer_receive) } + + @shutdown = false + @buffer_state = { # items accepted from including class :pending_items => {}, @@ -118,6 +121,7 @@ def buffer_initialize(options={}) :last_flush => Time.now.to_i, :timer => Thread.new do loop do + break if @shutdown sleep(@buffer_config[:max_interval]) buffer_flush(:force => true) end @@ -303,12 +307,20 @@ def buffer_flush(options={}) def process_failed_batches @file_persistence.load_batches.each do |file, batch| + processing_file = file + ".processing" + begin + # Try to atomically claim the file for processing + File.rename(file, processing_file) + rescue Errno::ENOENT, Errno::EACCES + # File already processed or claimed by another thread/process + next + end begin @buffer_state[:flush_mutex].lock begin flush(batch, true) - @file_persistence.delete_batch(file) - @buffer_config[:logger].info("Successfully flushed and deleted failed batch file: #{file}") if @buffer_config[:logger] + @file_persistence.delete_batch(processing_file) + @buffer_config[:logger].info("Successfully flushed and deleted failed batch file: #{processing_file}") if @buffer_config[:logger] rescue => e @buffer_config[:logger].warn("Failed to flush persisted batch: #{e.message}") if @buffer_config[:logger] ensure @@ -321,9 +333,10 @@ def process_failed_batches end def shutdown - # Stop the timer thread if it exists + # Graceful shutdown of timer thread if @buffer_state && @buffer_state[:timer] - @buffer_state[:timer].kill + @shutdown = true + @buffer_state[:timer].join @buffer_state[:timer] = nil end # Final flush of any remaining in-memory events diff --git a/lib/logstash/outputs/kusto/filePersistence.rb b/lib/logstash/outputs/kusto/filePersistence.rb index eda21a47..2a1a20d8 100644 --- a/lib/logstash/outputs/kusto/filePersistence.rb +++ b/lib/logstash/outputs/kusto/filePersistence.rb @@ -44,7 +44,7 @@ def persist_batch(batch, max_retries = 3) rescue => e attempts += 1 if attempts < max_retries - sleep 0.1 * attempts # Exponential backoff + sleep 0.1 * (2 ** (attempts - 1)) # Exponential backoff retry else @logger&.fatal("Failed to persist batch after #{attempts} attempts. Data loss may occur: #{e.message}") From 168b5b71410b77c1a813a2a398b54203b7e4adc4 Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Tue, 1 Jul 2025 16:59:54 +0530 Subject: [PATCH 49/52] Removed loadbatches fn --- .../outputs/kusto/customSizeBasedBuffer.rb | 22 +++++++++++++---- lib/logstash/outputs/kusto/filePersistence.rb | 24 ------------------- 2 files changed, 17 insertions(+), 29 deletions(-) diff --git a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb index 390aafa9..8aeba988 100644 --- a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb +++ b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb @@ -306,16 +306,16 @@ def buffer_flush(options={}) end def process_failed_batches - @file_persistence.load_batches.each do |file, batch| + Dir.glob(::File.join(@file_persistence.failed_dir, "failed_batch_*.json*")).each do |file| processing_file = file + ".processing" begin - # Try to atomically claim the file for processing - File.rename(file, processing_file) + ::File.rename(file, processing_file) rescue Errno::ENOENT, Errno::EACCES - # File already processed or claimed by another thread/process + # File already claimed or deleted, skip next end begin + batch = JSON.load(::File.read(processing_file)) @buffer_state[:flush_mutex].lock begin flush(batch, true) @@ -326,9 +326,21 @@ def process_failed_batches ensure @buffer_state[:flush_mutex].unlock end + rescue Errno::ENOENT + @buffer_config[:logger].warn("Batch file #{processing_file} was not found when attempting to read. It may have been deleted by another process.") if @buffer_config[:logger] + next rescue => e - @buffer_config[:logger].error("Error processing failed batch file: #{e.message}") if @buffer_config[:logger] + @buffer_config[:logger].warn("Failed to load batch file #{processing_file}: #{e.message}. Moving to quarantine.") if @buffer_config[:logger] + begin + quarantine_dir = File.join(@file_persistence.failed_dir, "quarantine") + FileUtils.mkdir_p(quarantine_dir) unless Dir.exist?(quarantine_dir) + FileUtils.mv(processing_file, quarantine_dir) + rescue => del_err + @buffer_config[:logger].warn("Failed to move corrupted batch file #{processing_file} to quarantine: #{del_err.message}") if @buffer_config[:logger] + end + next end + sleep(0.1) # Avoid tight loop in case of many files end end diff --git a/lib/logstash/outputs/kusto/filePersistence.rb b/lib/logstash/outputs/kusto/filePersistence.rb index 2a1a20d8..ccccb4db 100644 --- a/lib/logstash/outputs/kusto/filePersistence.rb +++ b/lib/logstash/outputs/kusto/filePersistence.rb @@ -52,30 +52,6 @@ def persist_batch(batch, max_retries = 3) end end - def load_batches - return [] unless Dir.exist?(@failed_dir) - return enum_for(:load_batches) unless block_given? - Dir.glob(::File.join(@failed_dir, 'failed_batch_*.json')).each do |file| - begin - yield file, JSON.load(::File.read(file)) - rescue => e - if e.is_a?(Errno::ENOENT) - @logger&.warn("Batch file #{file} was not found when attempting to read. It may have been deleted by another process.") - else - @logger&.warn("Failed to load batch file #{file}: #{e.message}. Moving to quarantine.") - begin - quarantine_dir = File.join(@failed_dir, "quarantine") - FileUtils.mkdir_p(quarantine_dir) unless Dir.exist?(quarantine_dir) - FileUtils.mv(file, quarantine_dir) - rescue => del_err - @logger&.warn("Failed to move corrupted batch file #{file} to quarantine: #{del_err.message}") - end - end - next - end - end - end - def delete_batch(file) begin ::File.delete(file) if ::File.exist?(file) From f2a75995a3e54cff259fa22cf7ca1c9f9ce673a2 Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Tue, 1 Jul 2025 17:25:25 +0530 Subject: [PATCH 50/52] Updated spec tests --- spec/outputs/kusto/filePersistence_spec.rb | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/spec/outputs/kusto/filePersistence_spec.rb b/spec/outputs/kusto/filePersistence_spec.rb index 738cc111..9ac9ac8a 100644 --- a/spec/outputs/kusto/filePersistence_spec.rb +++ b/spec/outputs/kusto/filePersistence_spec.rb @@ -22,9 +22,8 @@ file_persistence.persist_batch(batch) files = Dir.glob(File.join(tmp_dir, 'failed_batch_*.json')) expect(files.size).to eq(1) - loaded = file_persistence.load_batches.to_a - expect(loaded.size).to eq(1) - expect(loaded.first[1]).to eq(batch) + loaded_batch = JSON.load(File.read(files.first)) + expect(loaded_batch).to eq(batch) end it 'deletes a batch file' do @@ -45,6 +44,7 @@ it 'returns empty array if directory does not exist' do file_persistence = described_class.new(tmp_dir, logger) # Re-instantiate! FileUtils.rm_rf(tmp_dir) - expect(file_persistence.load_batches).to eq([]) + files = Dir.glob(File.join(tmp_dir, 'failed_batch_*.json')) + expect(files).to eq([]) end end \ No newline at end of file From 6418f75ca0c16aa32357ed7968866e7ca45e41be Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Tue, 1 Jul 2025 18:57:02 +0530 Subject: [PATCH 51/52] Updated process_failed_files --- .../outputs/kusto/customSizeBasedBuffer.rb | 64 +++++++++++-------- 1 file changed, 39 insertions(+), 25 deletions(-) diff --git a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb index 8aeba988..6809d45d 100644 --- a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb +++ b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb @@ -306,41 +306,55 @@ def buffer_flush(options={}) end def process_failed_batches - Dir.glob(::File.join(@file_persistence.failed_dir, "failed_batch_*.json*")).each do |file| + process_orphaned_processing_files + process_new_json_files + end + + def process_orphaned_processing_files + Dir.glob(::File.join(@file_persistence.failed_dir, "failed_batch_*.json.processing")).each do |processing_file| + process_failed_batch_file(processing_file) + sleep(0.1) + end + end + + def process_new_json_files + Dir.glob(::File.join(@file_persistence.failed_dir, "failed_batch_*.json")).each do |file| processing_file = file + ".processing" begin ::File.rename(file, processing_file) + process_failed_batch_file(processing_file) rescue Errno::ENOENT, Errno::EACCES # File already claimed or deleted, skip next end + sleep(0.1) + end + end + + def process_failed_batch_file(processing_file) + begin + batch = JSON.load(::File.read(processing_file)) + @buffer_state[:flush_mutex].lock begin - batch = JSON.load(::File.read(processing_file)) - @buffer_state[:flush_mutex].lock - begin - flush(batch, true) - @file_persistence.delete_batch(processing_file) - @buffer_config[:logger].info("Successfully flushed and deleted failed batch file: #{processing_file}") if @buffer_config[:logger] - rescue => e - @buffer_config[:logger].warn("Failed to flush persisted batch: #{e.message}") if @buffer_config[:logger] - ensure - @buffer_state[:flush_mutex].unlock - end - rescue Errno::ENOENT - @buffer_config[:logger].warn("Batch file #{processing_file} was not found when attempting to read. It may have been deleted by another process.") if @buffer_config[:logger] - next + flush(batch, true) + @file_persistence.delete_batch(processing_file) + @buffer_config[:logger].info("Successfully flushed and deleted failed batch file: #{processing_file}") if @buffer_config[:logger] rescue => e - @buffer_config[:logger].warn("Failed to load batch file #{processing_file}: #{e.message}. Moving to quarantine.") if @buffer_config[:logger] - begin - quarantine_dir = File.join(@file_persistence.failed_dir, "quarantine") - FileUtils.mkdir_p(quarantine_dir) unless Dir.exist?(quarantine_dir) - FileUtils.mv(processing_file, quarantine_dir) - rescue => del_err - @buffer_config[:logger].warn("Failed to move corrupted batch file #{processing_file} to quarantine: #{del_err.message}") if @buffer_config[:logger] - end - next + @buffer_config[:logger].warn("Failed to flush persisted batch: #{e.message}") if @buffer_config[:logger] + ensure + @buffer_state[:flush_mutex].unlock + end + rescue Errno::ENOENT + @buffer_config[:logger].warn("Batch file #{processing_file} was not found when attempting to read. It may have been deleted by another process.") if @buffer_config[:logger] + rescue => e + @buffer_config[:logger].warn("Failed to load batch file #{processing_file}: #{e.message}. Moving to quarantine.") if @buffer_config[:logger] + begin + quarantine_dir = File.join(@file_persistence.failed_dir, "quarantine") + FileUtils.mkdir_p(quarantine_dir) unless Dir.exist?(quarantine_dir) + FileUtils.mv(processing_file, quarantine_dir) + rescue => del_err + @buffer_config[:logger].warn("Failed to move corrupted batch file #{processing_file} to quarantine: #{del_err.message}") if @buffer_config[:logger] end - sleep(0.1) # Avoid tight loop in case of many files end end From be288dafe7a23f8a45f6368a00455233457771af Mon Sep 17 00:00:00 2001 From: monishkadas-ms Date: Wed, 2 Jul 2025 21:12:28 +0530 Subject: [PATCH 52/52] Updated Error Handling in buffer flush --- .../outputs/kusto/customSizeBasedBuffer.rb | 21 ++++--------------- lib/logstash/outputs/kusto/filePersistence.rb | 1 + lib/logstash/outputs/kusto/ingestor.rb | 1 - .../outputs/kusto/logStashFlushBuffer.rb | 1 - 4 files changed, 5 insertions(+), 19 deletions(-) diff --git a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb index 6809d45d..3a3e4dad 100644 --- a/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb +++ b/lib/logstash/outputs/kusto/customSizeBasedBuffer.rb @@ -249,7 +249,6 @@ def buffer_flush(options={}) @buffer_state[:outgoing_size] = @buffer_state[:pending_size] buffer_clear_pending end - @buffer_config[:logger].debug("---------------Exiting buffer_flush?-----------------") @buffer_config[:logger].info("Flushing output", :outgoing_count => @buffer_state[:outgoing_count], :time_since_last_flush => time_since_last_flush, @@ -279,23 +278,11 @@ def buffer_flush(options={}) @buffer_state[:outgoing_size] -= events_volume end items_flushed += events_size - + @buffer_state[:last_flush] = Time.now.to_i rescue => e - @buffer_config[:logger].warn("Failed to flush outgoing items", - :outgoing_count => @buffer_state[:outgoing_count], - :exception => e, - :backtrace => e.backtrace - ) if @buffer_config[:logger] - - if @buffer_config[:has_on_flush_error] - on_flush_error e - end - - sleep 1 - retry - end - @buffer_state[:last_flush] = Time.now.to_i - @buffer_config[:logger].debug("---------------Exiting buffer_flush?-----------------") + @buffer_config[:logger].warn("Unexpected error during flush: #{e.class}: #{e.message}\n#{e.backtrace.join("\n")}") + next + end end ensure diff --git a/lib/logstash/outputs/kusto/filePersistence.rb b/lib/logstash/outputs/kusto/filePersistence.rb index ccccb4db..05c20178 100644 --- a/lib/logstash/outputs/kusto/filePersistence.rb +++ b/lib/logstash/outputs/kusto/filePersistence.rb @@ -30,6 +30,7 @@ def persist_batch(batch, max_retries = 3) begin ::File.write(tmpfile, JSON.dump(batch)) ::File.rename(tmpfile, filename) + @logger&.info("Persisted failed batch to #{filename}") return # Success! rescue => e @logger&.error("Failed to persist batch to #{filename}: #{e.message}") diff --git a/lib/logstash/outputs/kusto/ingestor.rb b/lib/logstash/outputs/kusto/ingestor.rb index bc30c87a..74411b4b 100755 --- a/lib/logstash/outputs/kusto/ingestor.rb +++ b/lib/logstash/outputs/kusto/ingestor.rb @@ -101,7 +101,6 @@ def upload(data) @logger.error("Ingestion failed: #{e.message}") @logger.error("Ingestion failed: #{e.backtrace.join("\n")}") @file_persistence.persist_batch(data) - raise e } .on_resolution do |fulfilled, value, reason, *args| @logger.debug("Future fulfilled: #{fulfilled}, value: #{value}, reason: #{reason}, args: #{args}, class: #{value.class}") diff --git a/lib/logstash/outputs/kusto/logStashFlushBuffer.rb b/lib/logstash/outputs/kusto/logStashFlushBuffer.rb index 0c315974..0857b80e 100644 --- a/lib/logstash/outputs/kusto/logStashFlushBuffer.rb +++ b/lib/logstash/outputs/kusto/logStashFlushBuffer.rb @@ -50,7 +50,6 @@ def flush (documents, close=false) @ingestor.upload(documents) rescue => e @logger.error("Error uploading batch to Kusto: #{e.message}") - raise e # Let the buffer handle persistence of failed batch end end # def flush