From d4488d5a507341d511878718aecff8ecb88a6dec Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Mon, 23 Aug 2021 13:17:25 +0200 Subject: [PATCH 01/62] Add async_connect and async_send methods and add specific specs for Fiber.scheduler Fixes #342 --- lib/pg/connection.rb | 187 ++++++++++++++++++++-- spec/helpers.rb | 2 + spec/helpers/scheduler.rb | 241 +++++++++++++++++++++++++++++ spec/helpers/tcp_gate_scheduler.rb | 186 ++++++++++++++++++++++ spec/pg/connection_sync_spec.rb | 6 +- spec/pg/scheduler_spec.rb | 97 ++++++++++++ 6 files changed, 703 insertions(+), 16 deletions(-) create mode 100644 spec/helpers/scheduler.rb create mode 100644 spec/helpers/tcp_gate_scheduler.rb create mode 100644 spec/pg/scheduler_spec.rb diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index d3335d0b6..ab97dcd32 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -3,6 +3,7 @@ require 'pg' unless defined?( PG ) require 'uri' +require 'io/wait' # The PostgreSQL connection class. The interface for this class is based on # {libpq}[http://www.postgresql.org/docs/9.2/interactive/libpq.html], the C @@ -284,20 +285,178 @@ def ssl_attributes end end - REDIRECT_METHODS = { - :exec => [:async_exec, :sync_exec], - :query => [:async_exec, :sync_exec], - :exec_params => [:async_exec_params, :sync_exec_params], - :prepare => [:async_prepare, :sync_prepare], - :exec_prepared => [:async_exec_prepared, :sync_exec_prepared], - :describe_portal => [:async_describe_portal, :sync_describe_portal], - :describe_prepared => [:async_describe_prepared, :sync_describe_prepared], - } - - def self.async_api=(enable) - REDIRECT_METHODS.each do |ali, (async, sync)| - remove_method(ali) if method_defined?(ali) - alias_method( ali, enable ? async : sync ) + private def wait_for_flush + # From https://www.postgresql.org/docs/13/libpq-async.html + # After sending any command or data on a nonblocking connection, call PQflush. If it returns 1, wait for the socket to become read- or write-ready. If it becomes write-ready, call PQflush again. If it becomes read-ready, call PQconsumeInput , then call PQflush again. Repeat until PQflush returns 0. (It is necessary to check for read-ready and drain the input with PQconsumeInput , because the server can block trying to send us data, e.g., NOTICE messages, and won't read our data until we read its.) Once PQflush returns 0, wait for the socket to be read-ready and then read the response as described above. + + until flush() + # wait for the socket to become read- or write-ready + + if Fiber.respond_to?(:scheduler) && Fiber.scheduler + # If a scheduler is set use it directly. + # This is necessary since IO.select isn't passed to the scheduler. + events = Fiber.scheduler.io_wait(socket_io, IO::READABLE | IO::WRITABLE, nil) + if (events & IO::READABLE) > 0 + consume_input + end + else + readable, writable = IO.select([socket_io], [socket_io]) + if readable.any? + consume_input + end + end + end + end + + def async_exec(*args) + discard_results + async_send_query(*args) + + block() + res = get_last_result + + if block_given? + begin + return yield(res) + ensure + res.clear + end + end + res + end + + def async_exec_params(*args) + discard_results + + if args[1].nil? + # TODO: pg_deprecated(3, ("forwarding async_exec_params to async_exec is deprecated")); + async_send_query(*args) + else + async_send_query_params(*args) + end + + block() + res = get_last_result + + if block_given? + begin + return yield(res) + ensure + res.clear + end + end + res + end + + alias sync_send_query send_query + def async_send_query(*args, &block) + sync_send_query(*args) + wait_for_flush + end + + alias sync_send_query_params send_query_params + def async_send_query_params(*args, &block) + sync_send_query_params(*args) + wait_for_flush + end + + # In async_api=false mode all send calls run directly on libpq. + # Blocking vs. nonblocking state can be changed in libpq. + alias sync_setnonblocking setnonblocking + + # In async_api=true mode (default) all send calls run nonblocking. + # The difference is that setnonblocking(true) disables automatic handling of would-block cases. + def async_setnonblocking(enabled) + singleton_class.async_send_api = !enabled + sync_setnonblocking(true) + end + + # sync/async isnonblocking methods are switched by async_setnonblocking() + alias sync_isnonblocking isnonblocking + def async_isnonblocking + false + end + + + class << self + alias sync_connect new + + def async_connect(*args, **kwargs) + conn = PG::Connection.connect_start( *args, **kwargs ) or + raise(PG::Error, "Unable to create a new connection") + raise(PG::ConnectionBad, conn.error_message) if conn.status == PG::CONNECTION_BAD + + # Now grab a reference to the underlying socket so we know when the connection is established + socket = conn.socket_io + + # Track the progress of the connection, waiting for the socket to become readable/writable before polling it + poll_status = PG::PGRES_POLLING_WRITING + until poll_status == PG::PGRES_POLLING_OK || + poll_status == PG::PGRES_POLLING_FAILED + + # If the socket needs to read, wait 'til it becomes readable to poll again + case poll_status + when PG::PGRES_POLLING_READING + socket.wait_readable + + # ...and the same for when the socket needs to write + when PG::PGRES_POLLING_WRITING + socket.wait_writable + end + + # Check to see if it's finished or failed yet + poll_status = conn.connect_poll + end + + raise(PG::ConnectionBad, conn.error_message) unless conn.status == PG::CONNECTION_OK + + # Set connection to nonblocking to handle all blocking states in ruby. + # That way a fiber scheduler is able to handle IO requests. + conn.sync_setnonblocking(true) + conn.set_default_encoding + + conn + end + + REDIRECT_CLASS_METHODS = { + :new => [:async_connect, :sync_connect], + } + + REDIRECT_SEND_METHODS = { + :send_query => [:async_send_query, :sync_send_query], + :send_query_params => [:async_send_query_params, :sync_send_query_params], + :isnonblocking => [:async_isnonblocking, :sync_isnonblocking], + :nonblocking? => [:async_isnonblocking, :sync_isnonblocking], + } + REDIRECT_METHODS = { + :exec => [:async_exec, :sync_exec], + :query => [:async_exec, :sync_exec], + :exec_params => [:async_exec_params, :sync_exec_params], + :prepare => [:async_prepare, :sync_prepare], + :exec_prepared => [:async_exec_prepared, :sync_exec_prepared], + :describe_portal => [:async_describe_portal, :sync_describe_portal], + :describe_prepared => [:async_describe_prepared, :sync_describe_prepared], + :setnonblocking => [:async_setnonblocking, :sync_setnonblocking], + } + + def async_send_api=(enable) + REDIRECT_SEND_METHODS.each do |ali, (async, sync)| + undef_method(ali) if method_defined?(ali) + alias_method( ali, enable ? async : sync ) + end + end + + def async_api=(enable) + self.async_send_api = enable + REDIRECT_METHODS.each do |ali, (async, sync)| + remove_method(ali) if method_defined?(ali) + alias_method( ali, enable ? async : sync ) + end + REDIRECT_CLASS_METHODS.each do |ali, (async, sync)| + singleton_class.remove_method(ali) if method_defined?(ali) + # TODO: send is necessary for ruby < 2.5 + singleton_class.send(:alias_method, ali, enable ? async : sync ) + end end end diff --git a/spec/helpers.rb b/spec/helpers.rb index 60090d9ca..2edb83f50 100644 --- a/spec/helpers.rb +++ b/spec/helpers.rb @@ -4,6 +4,8 @@ require 'rspec' require 'shellwords' require 'pg' +require_relative 'helpers/scheduler.rb' +require_relative 'helpers/tcp_gate_scheduler.rb' DEFAULT_TEST_DIR_STR = File.join(Dir.pwd, "tmp_test_specs") TEST_DIR_STR = ENV['RUBY_PG_TEST_DIR'] || DEFAULT_TEST_DIR_STR diff --git a/spec/helpers/scheduler.rb b/spec/helpers/scheduler.rb new file mode 100644 index 000000000..a58946877 --- /dev/null +++ b/spec/helpers/scheduler.rb @@ -0,0 +1,241 @@ +# frozen_string_literal: true + +# This file is copied from https://github.com/ruby/ruby/blob/5e9598baea97c53757f12713bacc7f19f315c846/test/fiber/scheduler.rb + +# This is an example and simplified scheduler for test purposes. +# It is not efficient for a large number of file descriptors as it uses IO.select(). +# Production Fiber schedulers should use epoll/kqueue/etc. + +require 'fiber' +require 'socket' + +begin + require 'io/nonblock' +rescue LoadError + # Ignore. +end + +module Helpers +class Scheduler + def initialize + @readable = {} + @writable = {} + @waiting = {} + + @closed = false + + @lock = Thread::Mutex.new + @blocking = 0 + @ready = [] + + @urgent = IO.pipe + end + + attr :readable + attr :writable + attr :waiting + + def next_timeout + _fiber, timeout = @waiting.min_by{|key, value| value} + + if timeout + offset = timeout - current_time + + if offset < 0 + return 0 + else + return offset + end + end + end + + def run + #$stderr.puts [__method__, Fiber.current].inspect + + while @readable.any? or @writable.any? or @waiting.any? or @blocking.positive? + # Can only handle file descriptors up to 1024... + readable, writable = IO.select(@readable.keys + [@urgent.first], @writable.keys, [], next_timeout) + + puts "readable: #{readable}" if readable&.any? + puts "writable: #{writable}" if writable&.any? + + selected = {} + readable&.each do |io| + if fiber = @readable.delete(io) + @writable.delete(io) if @writable[io] == fiber + selected[fiber] = IO::READABLE + elsif io == @urgent.first + @urgent.first.read_nonblock(1024) + end + end + + writable&.each do |io| + if fiber = @writable.delete(io) + @readable.delete(io) if @readable[io] == fiber + selected[fiber] = selected.fetch(fiber, 0) | IO::WRITABLE + end + end + + selected.each do |fiber, events| + fiber.resume(events) + end + + if @waiting.any? + time = current_time + waiting, @waiting = @waiting, {} + + waiting.each do |fiber, timeout| + if fiber.alive? + if timeout <= time + fiber.resume + else + @waiting[fiber] = timeout + end + end + end + end + + if @ready.any? + ready = nil + + @lock.synchronize do + ready, @ready = @ready, [] + end + + ready.each do |fiber| + fiber.resume + end + end + end + end + + def close + # $stderr.puts [__method__, Fiber.current].inspect + + raise "Scheduler already closed!" if @closed + + self.run + ensure + if @urgent + @urgent.each(&:close) + @urgent = nil + end + + @closed = true + + # We freeze to detect any unintended modifications after the scheduler is closed: + self.freeze + end + + def closed? + @closed + end + + def current_time + Process.clock_gettime(Process::CLOCK_MONOTONIC) + end + + def timeout_after(duration, klass, message, &block) + fiber = Fiber.current + + self.fiber do + sleep(duration) + + if fiber&.alive? + fiber.raise(klass, message) + end + end + + begin + yield(duration) + ensure + fiber = nil + end + end + + def process_wait(pid, flags) + # $stderr.puts [__method__, pid, flags, Fiber.current].inspect + + # This is a very simple way to implement a non-blocking wait: + Thread.new do + Process::Status.wait(pid, flags) + end.value + end + + def io_wait(io, events, duration) + #$stderr.puts [__method__, io, events, duration, Fiber.current].inspect + + unless (events & IO::READABLE).zero? + @readable[io] = Fiber.current + end + + unless (events & IO::WRITABLE).zero? + @writable[io] = Fiber.current + end + + Fiber.yield + end + + # Used for Kernel#sleep and Thread::Mutex#sleep + def kernel_sleep(duration = nil) + # $stderr.puts [__method__, duration, Fiber.current].inspect + + self.block(:sleep, duration) + + return true + end + + # Used when blocking on synchronization (Thread::Mutex#lock, + # Thread::Queue#pop, Thread::SizedQueue#push, ...) + def block(blocker, timeout = nil) + # $stderr.puts [__method__, blocker, timeout].inspect + + if timeout + @waiting[Fiber.current] = current_time + timeout + begin + Fiber.yield + ensure + # Remove from @waiting in the case #unblock was called before the timeout expired: + @waiting.delete(Fiber.current) + end + else + @blocking += 1 + begin + Fiber.yield + ensure + @blocking -= 1 + end + end + end + + # Used when synchronization wakes up a previously-blocked fiber + # (Thread::Mutex#unlock, Thread::Queue#push, ...). + # This might be called from another thread. + def unblock(blocker, fiber) + # $stderr.puts [__method__, blocker, fiber].inspect + # $stderr.puts blocker.backtrace.inspect + # $stderr.puts fiber.backtrace.inspect + + @lock.synchronize do + @ready << fiber + end + + io = @urgent.last + io.write_nonblock('.') + end + + def fiber(&block) + fiber = Fiber.new(blocking: false, &block) + + fiber.resume + + return fiber + end + + def address_resolve(hostname) + Thread.new do + Addrinfo.getaddrinfo(hostname, nil).map(&:ip_address).uniq + end.value + end +end +end diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb new file mode 100644 index 000000000..4706a980d --- /dev/null +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -0,0 +1,186 @@ +# frozen_string_literal: true + +# This is an example and simplified scheduler for test purposes. +# It is not efficient for a large number of file descriptors as it uses IO.select(). +# Production Fiber schedulers should use epoll/kqueue/etc. + +module Helpers +class TcpGateScheduler < Scheduler + class Connection + attr_reader :internal_io + attr_reader :external_io + + def initialize(internal_io, external_host, external_port, debug: false) + @internal_io = internal_io + @external_host = external_host + @external_port = external_port + @external_io = nil + @pending_connect = false + @pending_read = false + @pending_write = false + @debug = debug + end + + def print_data(desc, data) + return unless @debug + if data.bytesize >= 90 + sdata = data[0..90] + puts "#{desc}: #{sdata} (... #{data.bytesize} bytes)" + else + puts "#{desc}: #{data} (#{data.bytesize} bytes)" + end + end + + def puts(*args) + return unless @debug + super + end + + def connect + # Not yet connected? + if !@external_io && !@pending_connect + @pending_connect = true + Fiber.schedule do + @external_io = TCPSocket.new(@external_host, @external_port) + @pending_connect = false + end + end + end + + def read + connect + + if @external_io && !@pending_read + @pending_read = true + Fiber.schedule do + begin + read_str = @external_io.readpartial(1000) + print_data("read fd:#{@external_io.fileno}->#{@internal_io.fileno}", read_str) + @internal_io.write(read_str) + rescue EOFError + puts "read_eof from fd:#{@external_io.fileno}" + @internal_io.close_write + end + @pending_read = false + end + end + end + + def write(amount=5) + connect + if @external_io + if @pending_write + @pending_write += amount + else + @pending_write = amount + Fiber.schedule do + # transfer up to 5*65536 bytes + # this should be enough to trigger writability on the observed connection + loop do + len = 65536 + begin + read_str = @internal_io.readpartial(len) + print_data("write fd:#{@internal_io.fileno}->#{@external_io.fileno}", read_str) + @external_io.write(read_str) + rescue EOFError + puts "write_eof from fd:#{@internal_io.fileno}" + @external_io.close_write + end + @pending_write -= 1 + break if !read_str || read_str.bytesize < len || @pending_write <= 0 + end + @pending_write = false + end + end + end + end + end + + def initialize(external_host:, external_port:, internal_host: 'localhost', internal_port: 0, debug: false) + super() + @started = false + @connections = [] + @server_io = TCPServer.new(internal_host, internal_port) + @external_host = external_host + @external_port = external_port + @finish = false + @debug = debug + end + + def finish + @finish = true + TCPSocket.new('localhost', internal_port).close + end + + def internal_port + @server_io.local_address.ip_port + end + + def puts(*args) + return unless @debug + super + end + + def io_wait(io, events, duration) + #$stderr.puts [:IO_WAIT, io, events, duration, Fiber.current].inspect + + begin + sock = TCPSocket.for_fd(io.fileno) + sock.autoclose = false + remote_address = sock.remote_address + rescue Errno::ENOTCONN + end + + unless @started + @started = true + Fiber.schedule do + # Wait for new connections to the TCP gate + while client=@server_io.accept + break if @finish + conn = Connection.new(client, @external_host, @external_port) + @connections << conn + end + end + end + + # Remove old connections + @connections.reject! do |conn| + conn.internal_io.closed? || conn.external_io&.closed? + end + + # Some IO call is waiting for data by rb_wait_for_single_fd() or so. + # Is it on our intercepted IO? + # Inspect latest connections first, since closed connections aren't removed immediately. + if cidx=@connections.rindex { |g| g.internal_io.local_address.to_s == remote_address.to_s } + conn = @connections[cidx] + puts "trigger: fd:#{io.fileno} #{{addr: remote_address, events: events}}" + # Success! Our observed client IO waits for some data to be readable or writable. + # The IO function running on the observed IO did make proper use of some ruby wait function. + # As a reward we provide some data to read or write. + # + # To the contrary: + # If the blocking IO function doesn't make use of ruby wait functions, then it won't get any data and starve as a result. + + if (events & IO::WRITABLE) > 0 + conn.write + + if (events & IO::READABLE) > 0 + conn.read + end + else + if (events & IO::READABLE) > 0 + # The write handler is called here because writes usually succeed without waiting for writablility. + # In this case the callback wait_io(IO::WRITABLE) isn't called, so that we don't get a trigger to transfer data. + # But after sending some data the caller usually waits for some answer to read. + # Therefore trigger transfer of all pending written data. + conn.write(99999) + + conn.read + end + end + end + + super + end +end +end diff --git a/spec/pg/connection_sync_spec.rb b/spec/pg/connection_sync_spec.rb index 52fd22469..1b8ed5e7b 100644 --- a/spec/pg/connection_sync_spec.rb +++ b/spec/pg/connection_sync_spec.rb @@ -4,11 +4,13 @@ require_relative '../helpers' context "running with sync_* methods" do - before :each do + before :all do + @conn.finish PG::Connection.async_api = false + @conn = connect_testing_db end - after :each do + after :all do PG::Connection.async_api = true end diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb new file mode 100644 index 000000000..a1f73f2b1 --- /dev/null +++ b/spec/pg/scheduler_spec.rb @@ -0,0 +1,97 @@ +# -*- rspec -*- +#encoding: utf-8 + +require_relative '../helpers' + +$scheduler_timeout = false + +context "with a Fiber scheduler" do + + def setup + # Run examples with gated scheduler + sched = Helpers::TcpGateScheduler.new(external_host: 'localhost', external_port: ENV['PGPORT'].to_i) + Fiber.set_scheduler(sched) + @conninfo_gate = @conninfo.gsub(/(^| )port=\d+/, " port=#{sched.internal_port}") + + # Run examples with default scheduler + #Fiber.set_scheduler(Helpers::Scheduler.new) + #@conninfo_gate = @conninfo + + # Run examples without scheduler + #def Fiber.schedule; yield; end + #@conninfo_gate = @conninfo + end + + def teardown + Fiber.set_scheduler(nil) + end + + def stop_scheduler + if Fiber.scheduler && Fiber.scheduler.respond_to?(:finish) + Fiber.scheduler.finish + end + end + + def thread_with_timeout(timeout) + th = Thread.new do + yield + end + unless th.join(timeout) + th.kill + $scheduler_timeout = true + raise("scheduler timeout in:\n#{th.backtrace.join("\n")}") + end + end + + it "connects to a server" do + thread_with_timeout(10) do + setup + Fiber.schedule do + conn = PG.connect(@conninfo_gate) + + res = conn.exec_params("SELECT 7", []) + expect(res.values).to eq([["7"]]) + + conn.finish + stop_scheduler + end + end + end + + it "waits when sending data" do + thread_with_timeout(10) do + setup + Fiber.schedule do + conn = PG.connect(@conninfo_gate) + + data = "x" * 1000 * 1000 * 10 + res = conn.exec_params("SELECT LENGTH($1)", [data]) + expect(res.values).to eq([[data.length.to_s]]) + + conn.finish + stop_scheduler + end + end + end + + it "connects several times" do + thread_with_timeout(10) do + setup + Fiber.schedule do + 3.times do + conn = PG.connect(@conninfo_gate) + conn.finish + end + stop_scheduler + end + end + end +end + +# Do not wait for threads doing blocking calls at the process shutdown. +# Instead exit immediately after printing the rspec report, if we know there are pending IO calls, which do not react on ruby interrupts. +END{ + if $scheduler_timeout + exit!(1) + end +} From f4b7d12cd54557117b06498ec3716d0ebf4263a2 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 24 Aug 2021 14:01:02 +0200 Subject: [PATCH 02/62] More verbose CI output --- .appveyor.yml | 2 +- .github/workflows/source-gem.yml | 4 +++- .travis.yml | 2 +- Rakefile | 2 +- spec/pg/scheduler_spec.rb | 2 +- 5 files changed, 7 insertions(+), 5 deletions(-) diff --git a/.appveyor.yml b/.appveyor.yml index f8efda44f..5b6807943 100644 --- a/.appveyor.yml +++ b/.appveyor.yml @@ -24,7 +24,7 @@ install: build_script: - bundle exec rake -rdevkit compile --trace test_script: - - bundle exec rake test + - bundle exec rake test PG_DEBUG=1 environment: matrix: - ruby_version: "head" diff --git a/.github/workflows/source-gem.yml b/.github/workflows/source-gem.yml index 977f9361c..2959dbf20 100644 --- a/.github/workflows/source-gem.yml +++ b/.github/workflows/source-gem.yml @@ -111,7 +111,9 @@ jobs: - run: gem install --local *.gem --verbose - name: Run specs - run: ruby -rpg -S rspec spec/**/*_spec.rb + env: + PG_DEBUG: 1 + run: ruby -rpg -S rspec spec/**/*_spec.rb -cfdoc - name: Print logs if job failed if: ${{ failure() && matrix.os == 'windows' }} diff --git a/.travis.yml b/.travis.yml index 9b5a5b387..bfa7756c6 100644 --- a/.travis.yml +++ b/.travis.yml @@ -38,7 +38,7 @@ before_install: - export PATH=/usr/lib/postgresql/$PGVERSION/bin:$PATH script: - - bundle exec rake compile test + - bundle exec rake compile test PG_DEBUG=1 after_failure: - "find tmp -name mkmf.log | xargs cat" diff --git a/Rakefile b/Rakefile index f1c764793..bf35de41c 100644 --- a/Rakefile +++ b/Rakefile @@ -68,7 +68,7 @@ Rake::ExtensionTask.new do |ext| end end -RSpec::Core::RakeTask.new(:spec).rspec_opts = "--profile" +RSpec::Core::RakeTask.new(:spec).rspec_opts = "--profile -cfdoc" task :test => :spec # Use the fivefish formatter for docs generated from development checkout diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index a1f73f2b1..ef718c1d9 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -9,7 +9,7 @@ def setup # Run examples with gated scheduler - sched = Helpers::TcpGateScheduler.new(external_host: 'localhost', external_port: ENV['PGPORT'].to_i) + sched = Helpers::TcpGateScheduler.new(external_host: 'localhost', external_port: ENV['PGPORT'].to_i, debug: ENV['PG_DEBUG']=='1') Fiber.set_scheduler(sched) @conninfo_gate = @conninfo.gsub(/(^| )port=\d+/, " port=#{sched.internal_port}") From e86d7af46140b98843b31c078ba3a23670696dcd Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 24 Aug 2021 17:29:17 +0200 Subject: [PATCH 03/62] Exclude scheduler spec from ruby < 3 --- spec/helpers.rb | 1 + spec/pg/scheduler_spec.rb | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/spec/helpers.rb b/spec/helpers.rb index 2edb83f50..f80e8b166 100644 --- a/spec/helpers.rb +++ b/spec/helpers.rb @@ -392,6 +392,7 @@ def wait_for_flush(conn) config.filter_run_excluding( :postgresql_96 ) if PG.library_version < 90600 config.filter_run_excluding( :postgresql_10 ) if PG.library_version < 100000 config.filter_run_excluding( :postgresql_12 ) if PG.library_version < 120000 + config.filter_run_excluding( :scheduler ) if RUBY_VERSION < "3.0" ### Automatically set up and tear down the database config.before(:suite) do |*args| diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index ef718c1d9..c07467c81 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -5,7 +5,7 @@ $scheduler_timeout = false -context "with a Fiber scheduler" do +context "with a Fiber scheduler", :scheduler do def setup # Run examples with gated scheduler From f3ec80cec42ccd86029b968013d9b99e1878067b Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 24 Aug 2021 19:10:11 +0200 Subject: [PATCH 04/62] Scheduler: Improve debugging --- spec/helpers/tcp_gate_scheduler.rb | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index 4706a980d..319811fa7 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -23,11 +23,11 @@ def initialize(internal_io, external_host, external_port, debug: false) def print_data(desc, data) return unless @debug - if data.bytesize >= 90 - sdata = data[0..90] - puts "#{desc}: #{sdata} (... #{data.bytesize} bytes)" + if data.bytesize >= 70 + sdata = data[0..70] + puts "#{desc}: #{sdata.inspect} (... #{data.bytesize} bytes)" else - puts "#{desc}: #{data} (#{data.bytesize} bytes)" + puts "#{desc}: #{data.inspect} (#{data.bytesize} bytes)" end end @@ -42,6 +42,7 @@ def connect @pending_connect = true Fiber.schedule do @external_io = TCPSocket.new(@external_host, @external_port) + puts "connected to external: #{@external_io.inspect}" @pending_connect = false end end @@ -137,7 +138,8 @@ def io_wait(io, events, duration) # Wait for new connections to the TCP gate while client=@server_io.accept break if @finish - conn = Connection.new(client, @external_host, @external_port) + conn = Connection.new(client, @external_host, @external_port, debug: @debug) + puts "accept new observed connection: #{conn.internal_io.inspect}" @connections << conn end end From ad901ec4e7f46a40dbd0ad5715f2b570b55cf50e Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 24 Aug 2021 20:46:23 +0200 Subject: [PATCH 05/62] Fixup: wait for connect --- spec/helpers/tcp_gate_scheduler.rb | 63 ++++++++++++++++-------------- 1 file changed, 34 insertions(+), 29 deletions(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index 319811fa7..f77467987 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -38,22 +38,27 @@ def puts(*args) def connect # Not yet connected? - if !@external_io && !@pending_connect - @pending_connect = true - Fiber.schedule do + if !@external_io + if !@pending_connect + @pending_connect = Queue.new @external_io = TCPSocket.new(@external_host, @external_port) - puts "connected to external: #{@external_io.inspect}" + @pending_connect.close @pending_connect = false + puts "connected to external: #{@external_io.inspect}" + else + # connection is being established -> wait for it before doing read/write + @pending_connect.pop end end end def read - connect - - if @external_io && !@pending_read + if !@pending_read @pending_read = true + Fiber.schedule do + connect + begin read_str = @external_io.readpartial(1000) print_data("read fd:#{@external_io.fileno}->#{@internal_io.fileno}", read_str) @@ -68,30 +73,30 @@ def read end def write(amount=5) - connect - if @external_io - if @pending_write - @pending_write += amount - else - @pending_write = amount - Fiber.schedule do - # transfer up to 5*65536 bytes - # this should be enough to trigger writability on the observed connection - loop do - len = 65536 - begin - read_str = @internal_io.readpartial(len) - print_data("write fd:#{@internal_io.fileno}->#{@external_io.fileno}", read_str) - @external_io.write(read_str) - rescue EOFError - puts "write_eof from fd:#{@internal_io.fileno}" - @external_io.close_write - end - @pending_write -= 1 - break if !read_str || read_str.bytesize < len || @pending_write <= 0 + if @pending_write + @pending_write += amount + else + @pending_write = amount + + Fiber.schedule do + connect + + # transfer up to 5*65536 bytes + # this should be enough to trigger writability on the observed connection + loop do + len = 65536 + begin + read_str = @internal_io.readpartial(len) + print_data("write fd:#{@internal_io.fileno}->#{@external_io.fileno}", read_str) + @external_io.write(read_str) + rescue EOFError + puts "write_eof from fd:#{@internal_io.fileno}" + @external_io.close_write end - @pending_write = false + @pending_write -= 1 + break if !read_str || read_str.bytesize < len || @pending_write <= 0 end + @pending_write = false end end end From 1ec69387758e312969ff6c17c62e27d042d5245e Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 24 Aug 2021 22:13:59 +0200 Subject: [PATCH 06/62] Work around issues on Windows IO#readpartial and PG::Connection#block don't call the scheduler on Windows. --- lib/pg/connection.rb | 22 ++++++++++++++++++---- spec/helpers/tcp_gate_scheduler.rb | 12 ++++++++++-- 2 files changed, 28 insertions(+), 6 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index ab97dcd32..292efe7eb 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -308,12 +308,27 @@ def ssl_attributes end end + private def wait_for_query_result + result = nil + loop do + # Buffer any incoming data on the socket until a full result is ready. + consume_input + while is_busy + socket_io.wait_readable + consume_input + end + + # Fetch the next result. If there isn't one, the query is finished + result = get_result || break + end + result + end + def async_exec(*args) discard_results async_send_query(*args) - block() - res = get_last_result + res = wait_for_query_result if block_given? begin @@ -335,8 +350,7 @@ def async_exec_params(*args) async_send_query_params(*args) end - block() - res = get_last_result + res = wait_for_query_result if block_given? begin diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index f77467987..04e8f481c 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -60,9 +60,13 @@ def read connect begin - read_str = @external_io.readpartial(1000) + puts "read fd:#{@external_io.fileno}->#{@internal_io.fileno} start" + read_str = @external_io.read_nonblock(1000) print_data("read fd:#{@external_io.fileno}->#{@internal_io.fileno}", read_str) @internal_io.write(read_str) + rescue IO::WaitReadable, Errno::EINTR + @external_io.wait_readable + retry rescue EOFError puts "read_eof from fd:#{@external_io.fileno}" @internal_io.close_write @@ -86,9 +90,13 @@ def write(amount=5) loop do len = 65536 begin - read_str = @internal_io.readpartial(len) + puts "write fd:#{@internal_io.fileno}->#{@external_io.fileno} start" + read_str = @internal_io.read_nonblock(len) print_data("write fd:#{@internal_io.fileno}->#{@external_io.fileno}", read_str) @external_io.write(read_str) + rescue IO::WaitReadable, Errno::EINTR + @internal_io.wait_readable + retry rescue EOFError puts "write_eof from fd:#{@internal_io.fileno}" @external_io.close_write From b92ef92354f198f2f4bbcd7bcc1efd218946269f Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 24 Aug 2021 22:52:46 +0200 Subject: [PATCH 07/62] Fix workaround on COPY tests --- lib/pg/connection.rb | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index 292efe7eb..d5e1c9ae1 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -309,7 +309,7 @@ def ssl_attributes end private def wait_for_query_result - result = nil + cur = prev = nil loop do # Buffer any incoming data on the socket until a full result is ready. consume_input @@ -319,9 +319,19 @@ def ssl_attributes end # Fetch the next result. If there isn't one, the query is finished - result = get_result || break + cur = get_result || break + + prev.clear if prev + prev = cur + + status = cur.result_status + break if status == PGRES_COPY_OUT || status == PGRES_COPY_IN || status == PGRES_COPY_BOTH + + if prev + prev.check + end end - result + prev end def async_exec(*args) From b67792077256058ff921b4a6b165d38263c61dcb Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 25 Aug 2021 14:15:24 +0200 Subject: [PATCH 08/62] We can use io.wait to wait for readability or writability --- lib/pg/connection.rb | 16 +++------------- 1 file changed, 3 insertions(+), 13 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index d5e1c9ae1..4ef390d07 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -291,19 +291,9 @@ def ssl_attributes until flush() # wait for the socket to become read- or write-ready - - if Fiber.respond_to?(:scheduler) && Fiber.scheduler - # If a scheduler is set use it directly. - # This is necessary since IO.select isn't passed to the scheduler. - events = Fiber.scheduler.io_wait(socket_io, IO::READABLE | IO::WRITABLE, nil) - if (events & IO::READABLE) > 0 - consume_input - end - else - readable, writable = IO.select([socket_io], [socket_io]) - if readable.any? - consume_input - end + events = socket_io.wait(nil, :read_write) + if !events.is_a?(Integer) || (events & IO::READABLE) > 0 + consume_input end end end From e4ec261c27a28d07d7d6715cbba6d600de308214 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 25 Aug 2021 16:02:15 +0200 Subject: [PATCH 09/62] Better workaround on Windows --- lib/pg/connection.rb | 38 ++++++++++++++++++-------------------- spec/pg/connection_spec.rb | 2 +- 2 files changed, 19 insertions(+), 21 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index 4ef390d07..a77f6a377 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -298,37 +298,34 @@ def ssl_attributes end end - private def wait_for_query_result - cur = prev = nil - loop do + if RUBY_PLATFORM =~ /mingw|mswin/ + def block(timeout=nil) + # pgconn_block in the C ext isn't Fiber.scheduler compatible on Windows. + # So use ruby code and socket_io.wait_readable instead. + # Buffer any incoming data on the socket until a full result is ready. consume_input - while is_busy - socket_io.wait_readable - consume_input - end - - # Fetch the next result. If there isn't one, the query is finished - cur = get_result || break - - prev.clear if prev - prev = cur - status = cur.result_status - break if status == PGRES_COPY_OUT || status == PGRES_COPY_IN || status == PGRES_COPY_BOTH + if timeout + aborttime = Time.now + timeout + end - if prev - prev.check + while is_busy + unless socket_io.wait_readable(timeout && [0.0, aborttime - Time.now].max) + return false + end + consume_input end + return true end - prev end def async_exec(*args) discard_results async_send_query(*args) - res = wait_for_query_result + block + res = get_last_result if block_given? begin @@ -350,7 +347,8 @@ def async_exec_params(*args) async_send_query_params(*args) end - res = wait_for_query_result + block + res = get_last_result if block_given? begin diff --git a/spec/pg/connection_spec.rb b/spec/pg/connection_spec.rb index 9c4d38cdb..3ba2e6b08 100644 --- a/spec/pg/connection_spec.rb +++ b/spec/pg/connection_spec.rb @@ -1195,7 +1195,7 @@ def interrupt_thread(exc=nil) end serv.close expect{ conn.block }.to raise_error(PG::ConnectionBad, /server closed the connection unexpectedly/) - expect{ conn.block }.to raise_error(PG::ConnectionBad, /can't get socket descriptor/) + expect{ conn.block }.to raise_error(PG::ConnectionBad, /can't get socket descriptor|connection not open/) end it "sets the fallback_application_name on new connections" do From 8a7fab16fb72e7670d7bd95698df95149ae55fc6 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 25 Aug 2021 16:02:54 +0200 Subject: [PATCH 10/62] More scheduler specs --- spec/pg/scheduler_spec.rb | 46 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index c07467c81..6b381bcac 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -86,6 +86,52 @@ def thread_with_timeout(timeout) end end end + + it "can retrieve several results" do + thread_with_timeout(10) do + setup + Fiber.schedule do + conn = PG.connect(@conninfo_gate) + + res = conn.send_query <<-EOT + SELECT generate_series(0,999), NULL; + SELECT 1000, pg_sleep(0.1); + EOT + + conn.block + res = conn.get_result + expect( res.values.length ).to eq( 1000 ) + + conn.block + res = conn.get_result + expect( res.values ).to eq( [["1000", ""]] ) + + res = conn.get_result + expect( res ).to be_nil + + conn.finish + stop_scheduler + end + end + end + + it "can retrieve the last one of several results" do + thread_with_timeout(10) do + setup + Fiber.schedule do + conn = PG.connect(@conninfo_gate) + + res = conn.exec <<-EOT + SELECT 1, NULL; + SELECT 3, pg_sleep(0.1); + EOT + expect( res.values ).to eq( [["3", ""]] ) + + conn.finish + stop_scheduler + end + end + end end # Do not wait for threads doing blocking calls at the process shutdown. From 80e3a746563895df7c22022fb94dfa62ed19cf42 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 25 Aug 2021 17:25:30 +0200 Subject: [PATCH 11/62] Revert "We can use io.wait to wait for readability or writability" This reverts commit d947fe3ddd8e798918e4a03b48e40fb876657d03. IO#wait doesn't return the event integer, we need to trigger consume_input. Also IO#wait API has changed in ruby-3.0 and is ugly. --- lib/pg/connection.rb | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index a77f6a377..835ec9e8c 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -291,9 +291,19 @@ def ssl_attributes until flush() # wait for the socket to become read- or write-ready - events = socket_io.wait(nil, :read_write) - if !events.is_a?(Integer) || (events & IO::READABLE) > 0 - consume_input + + if Fiber.respond_to?(:scheduler) && Fiber.scheduler + # If a scheduler is set use it directly. + # This is necessary since IO.select isn't passed to the scheduler. + events = Fiber.scheduler.io_wait(socket_io, IO::READABLE | IO::WRITABLE, nil) + if (events & IO::READABLE) > 0 + consume_input + end + else + readable, writable = IO.select([socket_io], [socket_io]) + if readable.any? + consume_input + end end end end From 99664bf80d230c59aa3f89b1b8e1ebb32631c85c Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 25 Aug 2021 17:27:22 +0200 Subject: [PATCH 12/62] Add ruby-3.0 to verify scheduler on it's first ruby release. --- .github/workflows/source-gem.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/source-gem.yml b/.github/workflows/source-gem.yml index 2959dbf20..c9e876a25 100644 --- a/.github/workflows/source-gem.yml +++ b/.github/workflows/source-gem.yml @@ -40,6 +40,9 @@ jobs: - os: ubuntu ruby: "head" PGVER: "13" + - os: ubuntu + ruby: "3.0" + PGVER: "12" - os: ubuntu ruby: "2.4" PGVER: "9.3" From e075048ae1a67efe40680ed0aef62f0e77aa304d Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 25 Aug 2021 17:48:06 +0200 Subject: [PATCH 13/62] Add some domumantation to the TcpGateScheduler --- spec/helpers/tcp_gate_scheduler.rb | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index 04e8f481c..a430cca9e 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -1,8 +1,21 @@ # frozen_string_literal: true -# This is an example and simplified scheduler for test purposes. -# It is not efficient for a large number of file descriptors as it uses IO.select(). -# Production Fiber schedulers should use epoll/kqueue/etc. +# This is a special scheduler for testing compatibility to Fiber.scheduler of functions using a TCP connection. +# +# It works as a gate between the client and the server. +# Data is transferred only, when the scheduler receives wait_io requests. +# The TCP communication in a C extension can be verified in a (mostly) timing insensitive way. +# If a call does IO but doesn't call the scheduler, the test will block and can be caught by an external timeout. +# +# PG.connect +# port:5444 TcpGateScheduler DB +# ------------- ---------------------------------------- -------- +# | scheduler | | TCPServer TCPSocket | | | +# | specs |----->| port 5444 port 5432|----->|Server| +# ------------- ^ | | | port | +# '------- wait_readable: <-send data-- | | 5432 | +# observe fd| wait_writable: --send data-> | -------- +# ---------------------------------------- module Helpers class TcpGateScheduler < Scheduler From ecc5e187cf8d4f82f47af534a10807be1303e414 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 25 Aug 2021 21:47:15 +0200 Subject: [PATCH 14/62] Add async_get_result and async_get_last_result --- lib/pg/connection.rb | 20 ++++++++++++++++---- spec/pg/scheduler_spec.rb | 2 -- 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index 835ec9e8c..7a3ffa55e 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -334,8 +334,7 @@ def async_exec(*args) discard_results async_send_query(*args) - block - res = get_last_result + res = async_get_last_result if block_given? begin @@ -357,8 +356,7 @@ def async_exec_params(*args) async_send_query_params(*args) end - block - res = get_last_result + res = async_get_last_result if block_given? begin @@ -370,6 +368,18 @@ def async_exec_params(*args) res end + alias sync_get_result get_result + def async_get_result(*args) + block + sync_get_result + end + + alias sync_get_last_result get_last_result + def async_get_last_result(*args) + block + sync_get_last_result + end + alias sync_send_query send_query def async_send_query(*args, &block) sync_send_query(*args) @@ -459,6 +469,8 @@ def async_connect(*args, **kwargs) :describe_portal => [:async_describe_portal, :sync_describe_portal], :describe_prepared => [:async_describe_prepared, :sync_describe_prepared], :setnonblocking => [:async_setnonblocking, :sync_setnonblocking], + :get_result => [:async_get_result, :sync_get_result], + :get_last_result => [:async_get_last_result, :sync_get_last_result], } def async_send_api=(enable) diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index 6b381bcac..f1c33e703 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -98,11 +98,9 @@ def thread_with_timeout(timeout) SELECT 1000, pg_sleep(0.1); EOT - conn.block res = conn.get_result expect( res.values.length ).to eq( 1000 ) - conn.block res = conn.get_result expect( res.values ).to eq( [["1000", ""]] ) From f69cd621ca8569cfd198619caa5df2ab25da230b Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Thu, 26 Aug 2021 13:55:22 +0200 Subject: [PATCH 15/62] Use monotonic time source on Windows It is faster and should be more reliable (avoiding leap seconds etc). --- lib/pg/connection.rb | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index 7a3ffa55e..69f808222 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -317,11 +317,13 @@ def block(timeout=nil) consume_input if timeout - aborttime = Time.now + timeout + aborttime = Process.clock_gettime(Process::CLOCK_MONOTONIC) + timeout end while is_busy - unless socket_io.wait_readable(timeout && [0.0, aborttime - Time.now].max) + tm = timeout && + [0.0, aborttime - Process.clock_gettime(Process::CLOCK_MONOTONIC)].max + unless socket_io.wait_readable(tm) return false end consume_input From 74ca4560dd57e994ce138c9767bea013acc84440 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sat, 28 Aug 2021 08:52:05 +0200 Subject: [PATCH 16/62] Disable GVL unlock/lock mechanism Since we do all blocking operations through ruby mechanisms now, we don't need to unlock the GVL any longer. GVL unlocking was introduced in pg-0.15.0 as a way to allow other ruby threads to be executed while blocking operations. Unfortunately it turned out that the blocked ruby thread doesn't react to signals and that this method is incompatible to Fiber.scheduler. This doesn't remove the unlock/lock mechanism from the sources yet, to allow easy comparison. My plan is to remove the code entirely before the next release. --- ext/gvl_wrappers.h | 15 +++++++++++++-- spec/pg/connection_spec.rb | 6 +++--- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/ext/gvl_wrappers.h b/ext/gvl_wrappers.h index 364a5c35e..f84b5a8e9 100644 --- a/ext/gvl_wrappers.h +++ b/ext/gvl_wrappers.h @@ -46,7 +46,7 @@ return NULL; \ } -#define DEFINE_GVL_STUB(name, when_non_void, rettype, lastparamtype, lastparamname) \ +#define nono_DEFINE_GVL_STUB(name, when_non_void, rettype, lastparamtype, lastparamname) \ rettype gvl_##name(FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST3) lastparamtype lastparamname){ \ struct gvl_wrapper_##name##_params params = { \ {FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST1) lastparamname}, when_non_void((rettype)0) \ @@ -55,6 +55,12 @@ when_non_void( return params.retval; ) \ } +#define DEFINE_GVL_STUB(name, when_non_void, rettype, lastparamtype, lastparamname) \ + rettype gvl_##name(FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST3) lastparamtype lastparamname){ \ + when_non_void( return ) \ + name( FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST1) lastparamname ); \ + } + #define DEFINE_GVL_STUB_DECL(name, when_non_void, rettype, lastparamtype, lastparamname) \ rettype gvl_##name(FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST3) lastparamtype lastparamname); @@ -66,7 +72,7 @@ return NULL; \ } -#define DEFINE_GVLCB_STUB(name, when_non_void, rettype, lastparamtype, lastparamname) \ +#define nono_DEFINE_GVLCB_STUB(name, when_non_void, rettype, lastparamtype, lastparamname) \ rettype gvl_##name(FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST3) lastparamtype lastparamname){ \ struct gvl_wrapper_##name##_params params = { \ {FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST1) lastparamname}, when_non_void((rettype)0) \ @@ -74,6 +80,11 @@ rb_thread_call_with_gvl(gvl_##name##_skeleton, ¶ms); \ when_non_void( return params.retval; ) \ } +#define DEFINE_GVLCB_STUB(name, when_non_void, rettype, lastparamtype, lastparamname) \ + rettype gvl_##name(FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST3) lastparamtype lastparamname){ \ + when_non_void( return ) \ + name( FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST1) lastparamname ); \ + } #define GVL_TYPE_VOID(string) #define GVL_TYPE_NONVOID(string) string diff --git a/spec/pg/connection_spec.rb b/spec/pg/connection_spec.rb index 3ba2e6b08..b70f83999 100644 --- a/spec/pg/connection_spec.rb +++ b/spec/pg/connection_spec.rb @@ -570,7 +570,7 @@ def interrupt_thread(exc=nil) sleep 0.1 Process.kill("USR2", Process.pid) end - @conn.exec("select pg_sleep(0.3)") + @conn.async_exec("select pg_sleep(0.3)") expect( signal_received ).to be_truthy end @@ -1850,7 +1850,7 @@ def interrupt_thread(exc=nil) context "OS thread support" do it "Connection#exec shouldn't block a second thread" do t = Thread.new do - @conn.exec( "select pg_sleep(1)" ) + @conn.async_exec( "select pg_sleep(1)" ) end sleep 0.1 @@ -1864,7 +1864,7 @@ def interrupt_thread(exc=nil) t = Thread.new do serv = TCPServer.new( '127.0.0.1', 54320 ) expect { - described_class.new( '127.0.0.1', 54320, "", "", "me", "xxxx", "somedb" ) + described_class.async_connect( '127.0.0.1', 54320, "", "", "me", "xxxx", "somedb" ) }.to raise_error(PG::ConnectionBad, /server closed the connection unexpectedly/) end From 00e0ca461b73d3fb260a70675b3eb2cfb268303a Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sun, 29 Aug 2021 11:21:24 +0200 Subject: [PATCH 17/62] Add scheduler compatible get_copy_data --- lib/pg/connection.rb | 14 ++++++++++++++ spec/pg/scheduler_spec.rb | 21 +++++++++++++++++++++ 2 files changed, 35 insertions(+) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index 69f808222..54f075680 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -382,6 +382,19 @@ def async_get_last_result(*args) sync_get_last_result end + alias sync_get_copy_data get_copy_data + def async_get_copy_data(async=false, decoder=nil) + if async + return sync_get_copy_data(async, decoder) + else + while (res=sync_get_copy_data(true, decoder)) == false + socket_io.wait_readable + consume_input + end + return res + end + end + alias sync_send_query send_query def async_send_query(*args, &block) sync_send_query(*args) @@ -473,6 +486,7 @@ def async_connect(*args, **kwargs) :setnonblocking => [:async_setnonblocking, :sync_setnonblocking], :get_result => [:async_get_result, :sync_get_result], :get_last_result => [:async_get_last_result, :sync_get_last_result], + :get_copy_data => [:async_get_copy_data, :sync_get_copy_data], } def async_send_api=(enable) diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index f1c33e703..79ad170fd 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -130,6 +130,27 @@ def thread_with_timeout(timeout) end end end + + it "can receive COPY data" do + thread_with_timeout(10) do + setup + Fiber.schedule do + conn = PG.connect(@conninfo_gate) + + rows = [] + conn.copy_data( "COPY (SELECT generate_series(0,999)::TEXT UNION ALL SELECT pg_sleep(1)::TEXT || '1000') TO STDOUT" ) do |res| + res = nil + 1002.times do + rows << conn.get_copy_data + end + end + expect( rows ).to eq( 1001.times.map{|i| "#{i}\n" } + [nil] ) + + conn.finish + stop_scheduler + end + end + end end # Do not wait for threads doing blocking calls at the process shutdown. From b3fea3066d8627fa3759ef76ecd703906143ee42 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sun, 29 Aug 2021 14:12:26 +0200 Subject: [PATCH 18/62] Improve markup in docs --- ext/pg_connection.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ext/pg_connection.c b/ext/pg_connection.c index 3caab35e3..c4af6e5e9 100644 --- a/ext/pg_connection.c +++ b/ext/pg_connection.c @@ -2575,8 +2575,8 @@ pgconn_put_copy_data(int argc, VALUE *argv, VALUE self) * forces the COPY command to fail with the string * _error_message_. * - * Returns true if the end-of-data was sent, false if it was - * not sent (false is only possible if the connection + * Returns true if the end-of-data was sent, *false* if it was + * not sent (*false* is only possible if the connection * is in nonblocking mode, and this command would block). */ static VALUE From d72606e2d0c18798996a6a11a40adb0c3fd7d87e Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sun, 29 Aug 2021 14:13:40 +0200 Subject: [PATCH 19/62] TCP-Gate: Improve write handling Write until the observed fd gets writeable again and abort at this point in time. This is necessary to reliable trigger the next writable-event. The amount of data to read from internal_io until a writeable state is triggered on the other end increases with the number of bytes written. This way we make sure, that only the minimum number of bytes is written, that is necessary to get the nonblocking mechanisms working. --- spec/helpers/tcp_gate_scheduler.rb | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index a430cca9e..01e044d66 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -89,11 +89,9 @@ def read end end - def write(amount=5) - if @pending_write - @pending_write += amount - else - @pending_write = amount + def write(until_writeable: false) + if !@pending_write + @pending_write = true Fiber.schedule do connect @@ -107,6 +105,11 @@ def write(amount=5) read_str = @internal_io.read_nonblock(len) print_data("write fd:#{@internal_io.fileno}->#{@external_io.fileno}", read_str) @external_io.write(read_str) + if until_writeable + res = IO.select(nil, [until_writeable], nil, 0) + break if res + end + rescue IO::WaitReadable, Errno::EINTR @internal_io.wait_readable retry @@ -114,8 +117,7 @@ def write(amount=5) puts "write_eof from fd:#{@internal_io.fileno}" @external_io.close_write end - @pending_write -= 1 - break if !read_str || read_str.bytesize < len || @pending_write <= 0 + break if !read_str || read_str.bytesize < len end @pending_write = false end @@ -190,7 +192,7 @@ def io_wait(io, events, duration) # If the blocking IO function doesn't make use of ruby wait functions, then it won't get any data and starve as a result. if (events & IO::WRITABLE) > 0 - conn.write + conn.write(until_writeable: io) if (events & IO::READABLE) > 0 conn.read @@ -201,7 +203,7 @@ def io_wait(io, events, duration) # In this case the callback wait_io(IO::WRITABLE) isn't called, so that we don't get a trigger to transfer data. # But after sending some data the caller usually waits for some answer to read. # Therefore trigger transfer of all pending written data. - conn.write(99999) + conn.write conn.read end From d273782c755119fc13b676527e4d555440b5f62b Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sun, 29 Aug 2021 14:23:12 +0200 Subject: [PATCH 20/62] Implement async_put_copy_data/async_put_copy_end --- lib/pg/connection.rb | 20 ++++++++++++++++++++ spec/pg/scheduler_spec.rb | 27 +++++++++++++++++++++++++++ 2 files changed, 47 insertions(+) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index 54f075680..1f1f8eacb 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -424,6 +424,23 @@ def async_isnonblocking false end + alias sync_put_copy_data put_copy_data + def async_put_copy_data(buffer, encoder=nil) + until sync_put_copy_data(buffer, encoder) + wait_for_flush + end + wait_for_flush + true + end + alias sync_put_copy_end put_copy_end + def async_put_copy_end(*args) + until sync_put_copy_end(*args) + wait_for_flush + end + wait_for_flush + true + end + class << self alias sync_connect new @@ -469,11 +486,14 @@ def async_connect(*args, **kwargs) :new => [:async_connect, :sync_connect], } + # These methods are affected by PQsetnonblocking REDIRECT_SEND_METHODS = { :send_query => [:async_send_query, :sync_send_query], :send_query_params => [:async_send_query_params, :sync_send_query_params], :isnonblocking => [:async_isnonblocking, :sync_isnonblocking], :nonblocking? => [:async_isnonblocking, :sync_isnonblocking], + :put_copy_data => [:async_put_copy_data, :sync_put_copy_data], + :put_copy_end => [:async_put_copy_end, :sync_put_copy_end], } REDIRECT_METHODS = { :exec => [:async_exec, :sync_exec], diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index 79ad170fd..299b68e54 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -151,6 +151,33 @@ def thread_with_timeout(timeout) end end end + + it "can send lots of data per put_copy_data" do + thread_with_timeout(60) do + setup + Fiber.schedule do + conn = PG.connect(@conninfo_gate) + + conn.exec <<-EOSQL + CREATE TEMP TABLE copytable (col1 TEXT); + EOSQL + + res = nil + conn.copy_data( "COPY copytable FROM STDOUT CSV" ) do + data = "x" * 1000 * 1000 + data << "\n" + 50.times do + res = conn.put_copy_data(data) + break if res == false + end + end + expect( res ).to be_truthy + conn.finish + + stop_scheduler + end + end + end end # Do not wait for threads doing blocking calls at the process shutdown. From 8d452a20adce6ab24dfbedd2faed98e58437d5a3 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Fri, 3 Sep 2021 21:26:50 +0200 Subject: [PATCH 21/62] Fix race condition with write data transfer If a write request without stopping on writablility comes in, make sure, that the pending transfer doesn't abort prematurely. Otherwise a read request might wait for data although not all data is written. --- spec/helpers/tcp_gate_scheduler.rb | 25 ++++++++++++++++++------- 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index 01e044d66..4bea88d5a 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -73,7 +73,6 @@ def read connect begin - puts "read fd:#{@external_io.fileno}->#{@internal_io.fileno} start" read_str = @external_io.read_nonblock(1000) print_data("read fd:#{@external_io.fileno}->#{@internal_io.fileno}", read_str) @internal_io.write(read_str) @@ -92,22 +91,28 @@ def read def write(until_writeable: false) if !@pending_write @pending_write = true + @until_writeable = until_writeable Fiber.schedule do + puts "start write #{until_writeable ? "until #{until_writeable.inspect} is writeable" : "all pending data"}" connect - # transfer up to 5*65536 bytes - # this should be enough to trigger writability on the observed connection + # transfer data blocks of up to 65536 bytes + # until the observed connection is writable again or + # no data left to read loop do len = 65536 begin - puts "write fd:#{@internal_io.fileno}->#{@external_io.fileno} start" read_str = @internal_io.read_nonblock(len) print_data("write fd:#{@internal_io.fileno}->#{@external_io.fileno}", read_str) + sleep 0 @external_io.write(read_str) - if until_writeable - res = IO.select(nil, [until_writeable], nil, 0) - break if res + if @until_writeable + res = IO.select(nil, [@until_writeable], nil, 0) + if res + puts "stop writing - #{@until_writeable.inspect} is writable again" + break + end end rescue IO::WaitReadable, Errno::EINTR @@ -119,8 +124,14 @@ def write(until_writeable: false) end break if !read_str || read_str.bytesize < len end + @until_writeable = false @pending_write = false end + + elsif until_writeable == false + # If a write request without stopping on writablility comes in, + # make sure, that the pending transfer doesn't abort prematurely. + @until_writeable = false end end end From 3f63c1f34cd8979ad9481d016e6c03eb99314798 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Fri, 3 Sep 2021 21:57:29 +0200 Subject: [PATCH 22/62] Treat IO.select errors as timeout This happens when the underlying library closed the observed socket. That way regular read until EOF and close_write is called. --- spec/helpers/tcp_gate_scheduler.rb | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index 4bea88d5a..d495715ac 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -94,7 +94,7 @@ def write(until_writeable: false) @until_writeable = until_writeable Fiber.schedule do - puts "start write #{until_writeable ? "until #{until_writeable.inspect} is writeable" : "all pending data"}" + puts "start write #{@until_writeable ? "until #{@until_writeable.inspect} is writeable" : "all pending data"}" connect # transfer data blocks of up to 65536 bytes @@ -108,7 +108,7 @@ def write(until_writeable: false) sleep 0 @external_io.write(read_str) if @until_writeable - res = IO.select(nil, [@until_writeable], nil, 0) + res = IO.select(nil, [@until_writeable], nil, 0) rescue nil if res puts "stop writing - #{@until_writeable.inspect} is writable again" break From f2afb65c67dfdf2ebf1b73cb90cf1a2d74a2f254 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sat, 4 Sep 2021 12:31:06 +0200 Subject: [PATCH 23/62] Scheduler: Make sure connections are closed at finish Otherwise lingering connections might remain open. --- spec/helpers/tcp_gate_scheduler.rb | 91 ++++++++++++++++++++---------- 1 file changed, 62 insertions(+), 29 deletions(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index d495715ac..c46c0097f 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -65,36 +65,57 @@ def connect end end - def read + # transfer data in read direction + # + # Option `transfer_until` can be (higher to lower priority): + # :eof => transfer until channel is closed + # false => transfer only one block + # + # The method does nothing if a transfer is already pending, but might raise the transfer_until option, if the requested priority is higher than the pending transfer. + def read( transfer_until: ) if !@pending_read @pending_read = true + @transfer_until = transfer_until Fiber.schedule do connect begin - read_str = @external_io.read_nonblock(1000) - print_data("read fd:#{@external_io.fileno}->#{@internal_io.fileno}", read_str) - @internal_io.write(read_str) - rescue IO::WaitReadable, Errno::EINTR - @external_io.wait_readable - retry - rescue EOFError - puts "read_eof from fd:#{@external_io.fileno}" - @internal_io.close_write - end + begin + read_str = @external_io.read_nonblock(1000) + print_data("read fd:#{@external_io.fileno}->#{@internal_io.fileno}", read_str) + @internal_io.write(read_str) + rescue IO::WaitReadable, Errno::EINTR + @external_io.wait_readable + retry + rescue EOFError + puts "read_eof from fd:#{@external_io.fileno}" + @internal_io.close_write + break + end + end while @transfer_until @pending_read = false end + elsif transfer_until == :eof + @transfer_until = transfer_until end end - def write(until_writeable: false) + # transfer data in write direction + # + # Option `transfer_until` can be (higher to lower priority): + # :eof => transfer until channel is closed + # :nodata => transfer until no immediate data is available + # IO object => transfer until IO is writeable + # + # The method does nothing if a transfer is already pending, but might raise the transfer_until option, if the requested priority is higher than the pending transfer. + def write( transfer_until: ) if !@pending_write @pending_write = true - @until_writeable = until_writeable + @transfer_until = transfer_until Fiber.schedule do - puts "start write #{@until_writeable ? "until #{@until_writeable.inspect} is writeable" : "all pending data"}" + puts "start write #{@transfer_until ? "until #{@transfer_until.inspect} is writeable" : "all pending data"}" connect # transfer data blocks of up to 65536 bytes @@ -107,10 +128,10 @@ def write(until_writeable: false) print_data("write fd:#{@internal_io.fileno}->#{@external_io.fileno}", read_str) sleep 0 @external_io.write(read_str) - if @until_writeable - res = IO.select(nil, [@until_writeable], nil, 0) rescue nil + if @transfer_until.is_a?(IO) + res = IO.select(nil, [@transfer_until], nil, 0) rescue nil if res - puts "stop writing - #{@until_writeable.inspect} is writable again" + puts "stop writing - #{@transfer_until.inspect} is writable again" break end end @@ -121,19 +142,27 @@ def write(until_writeable: false) rescue EOFError puts "write_eof from fd:#{@internal_io.fileno}" @external_io.close_write + break end - break if !read_str || read_str.bytesize < len + break if @transfer_until != :eof && (!read_str || read_str.bytesize < len) end @until_writeable = false @pending_write = false end - elsif until_writeable == false + elsif (transfer_until == :nodata && @transfer_until.is_a?(IO)) || + transfer_until == :eof # If a write request without stopping on writablility comes in, # make sure, that the pending transfer doesn't abort prematurely. - @until_writeable = false + @transfer_until = transfer_until end end + + # Make sure all data is transferred and both connections are closed. + def finish + write transfer_until: :eof + read transfer_until: :eof + end end def initialize(external_host:, external_port:, internal_host: 'localhost', internal_port: 0, debug: false) @@ -176,10 +205,14 @@ def io_wait(io, events, duration) Fiber.schedule do # Wait for new connections to the TCP gate while client=@server_io.accept - break if @finish - conn = Connection.new(client, @external_host, @external_port, debug: @debug) - puts "accept new observed connection: #{conn.internal_io.inspect}" - @connections << conn + if @finish + @connections.each(&:finish) + break + else + conn = Connection.new(client, @external_host, @external_port, debug: @debug) + puts "accept new observed connection: #{conn.internal_io.inspect}" + @connections << conn + end end end end @@ -203,20 +236,20 @@ def io_wait(io, events, duration) # If the blocking IO function doesn't make use of ruby wait functions, then it won't get any data and starve as a result. if (events & IO::WRITABLE) > 0 - conn.write(until_writeable: io) + conn.write(transfer_until: io) if (events & IO::READABLE) > 0 - conn.read + conn.read(transfer_until: false) end else if (events & IO::READABLE) > 0 - # The write handler is called here because writes usually succeed without waiting for writablility. + # Call the write handler here because writes usually succeed without waiting for writablility. # In this case the callback wait_io(IO::WRITABLE) isn't called, so that we don't get a trigger to transfer data. # But after sending some data the caller usually waits for some answer to read. # Therefore trigger transfer of all pending written data. - conn.write + conn.write(transfer_until: :nodata) - conn.read + conn.read(transfer_until: false) end end end From 87e430daffe1c8c02c7d099a20e5803f0d440e7f Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sat, 4 Sep 2021 15:00:28 +0200 Subject: [PATCH 24/62] Scheduler: Handle forcibly closed connections A PostgreSQL server on Windows always closes the connection with RST instead of FIN. This is a knwon behaviour as described here: https://github.com/jackc/pgx/issues/637 --- spec/helpers/tcp_gate_scheduler.rb | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index c46c0097f..9d05ccc36 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -88,7 +88,7 @@ def read( transfer_until: ) rescue IO::WaitReadable, Errno::EINTR @external_io.wait_readable retry - rescue EOFError + rescue EOFError, Errno::ECONNRESET puts "read_eof from fd:#{@external_io.fileno}" @internal_io.close_write break @@ -139,7 +139,7 @@ def write( transfer_until: ) rescue IO::WaitReadable, Errno::EINTR @internal_io.wait_readable retry - rescue EOFError + rescue EOFError, Errno::ECONNRESET puts "write_eof from fd:#{@internal_io.fileno}" @external_io.close_write break From f68e06eeff21a596420b8d3fa3084ec30063dd5d Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sat, 4 Sep 2021 15:05:06 +0200 Subject: [PATCH 25/62] Scheduler: Better debug output --- spec/helpers/tcp_gate_scheduler.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index 9d05ccc36..8220fcfee 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -115,7 +115,7 @@ def write( transfer_until: ) @transfer_until = transfer_until Fiber.schedule do - puts "start write #{@transfer_until ? "until #{@transfer_until.inspect} is writeable" : "all pending data"}" + puts "start write #{@transfer_until.is_a?(IO) ? "until #{@transfer_until.inspect} is writeable" : "until #{@transfer_until}"}" connect # transfer data blocks of up to 65536 bytes From 82eb48bf1098e08bba48a00c209302735f452123 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sat, 4 Sep 2021 15:05:23 +0200 Subject: [PATCH 26/62] Remove duplicated code in tests --- spec/pg/scheduler_spec.rb | 148 ++++++++++++++------------------------ 1 file changed, 55 insertions(+), 93 deletions(-) diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index 299b68e54..3ba6d19a3 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -43,14 +43,13 @@ def thread_with_timeout(timeout) end end - it "connects to a server" do - thread_with_timeout(10) do + def run_with_scheduler(timeout=10) + thread_with_timeout(timeout) do setup Fiber.schedule do conn = PG.connect(@conninfo_gate) - res = conn.exec_params("SELECT 7", []) - expect(res.values).to eq([["7"]]) + yield conn conn.finish stop_scheduler @@ -58,124 +57,87 @@ def thread_with_timeout(timeout) end end - it "waits when sending data" do - thread_with_timeout(10) do - setup - Fiber.schedule do - conn = PG.connect(@conninfo_gate) - - data = "x" * 1000 * 1000 * 10 - res = conn.exec_params("SELECT LENGTH($1)", [data]) - expect(res.values).to eq([[data.length.to_s]]) + it "connects to a server" do + run_with_scheduler do |conn| + res = conn.exec_params("SELECT 7", []) + expect(res.values).to eq([["7"]]) + end + end - conn.finish - stop_scheduler - end + it "waits when sending data" do + run_with_scheduler do |conn| + data = "x" * 1000 * 1000 * 10 + res = conn.exec_params("SELECT LENGTH($1)", [data]) + expect(res.values).to eq([[data.length.to_s]]) end end it "connects several times" do - thread_with_timeout(10) do - setup - Fiber.schedule do - 3.times do - conn = PG.connect(@conninfo_gate) - conn.finish - end - stop_scheduler + run_with_scheduler do + 3.times do + conn = PG.connect(@conninfo_gate) + conn.finish end end end it "can retrieve several results" do - thread_with_timeout(10) do - setup - Fiber.schedule do - conn = PG.connect(@conninfo_gate) - - res = conn.send_query <<-EOT - SELECT generate_series(0,999), NULL; - SELECT 1000, pg_sleep(0.1); - EOT + run_with_scheduler do |conn| + res = conn.send_query <<-EOT + SELECT generate_series(0,999), NULL; + SELECT 1000, pg_sleep(0.1); + EOT - res = conn.get_result - expect( res.values.length ).to eq( 1000 ) + res = conn.get_result + expect( res.values.length ).to eq( 1000 ) - res = conn.get_result - expect( res.values ).to eq( [["1000", ""]] ) + res = conn.get_result + expect( res.values ).to eq( [["1000", ""]] ) - res = conn.get_result - expect( res ).to be_nil - - conn.finish - stop_scheduler - end + res = conn.get_result + expect( res ).to be_nil end end it "can retrieve the last one of several results" do - thread_with_timeout(10) do - setup - Fiber.schedule do - conn = PG.connect(@conninfo_gate) - - res = conn.exec <<-EOT - SELECT 1, NULL; - SELECT 3, pg_sleep(0.1); - EOT - expect( res.values ).to eq( [["3", ""]] ) - - conn.finish - stop_scheduler - end + run_with_scheduler do |conn| + res = conn.exec <<-EOT + SELECT 1, NULL; + SELECT 3, pg_sleep(0.1); + EOT + expect( res.values ).to eq( [["3", ""]] ) end end it "can receive COPY data" do - thread_with_timeout(10) do - setup - Fiber.schedule do - conn = PG.connect(@conninfo_gate) - - rows = [] - conn.copy_data( "COPY (SELECT generate_series(0,999)::TEXT UNION ALL SELECT pg_sleep(1)::TEXT || '1000') TO STDOUT" ) do |res| - res = nil - 1002.times do - rows << conn.get_copy_data - end + run_with_scheduler do |conn| + rows = [] + conn.copy_data( "COPY (SELECT generate_series(0,999)::TEXT UNION ALL SELECT pg_sleep(1)::TEXT || '1000') TO STDOUT" ) do |res| + res = nil + 1002.times do + rows << conn.get_copy_data end - expect( rows ).to eq( 1001.times.map{|i| "#{i}\n" } + [nil] ) - - conn.finish - stop_scheduler end + expect( rows ).to eq( 1001.times.map{|i| "#{i}\n" } + [nil] ) end end it "can send lots of data per put_copy_data" do - thread_with_timeout(60) do - setup - Fiber.schedule do - conn = PG.connect(@conninfo_gate) - - conn.exec <<-EOSQL - CREATE TEMP TABLE copytable (col1 TEXT); - EOSQL - - res = nil - conn.copy_data( "COPY copytable FROM STDOUT CSV" ) do - data = "x" * 1000 * 1000 - data << "\n" - 50.times do - res = conn.put_copy_data(data) - break if res == false - end + run_with_scheduler(60) do |conn| + conn.exec <<-EOSQL + CREATE TEMP TABLE copytable (col1 TEXT); + EOSQL + + res = nil + conn.copy_data( "COPY copytable FROM STDOUT CSV" ) do + data = "x" * 1000 * 1000 + data << "\n" + 50.times do + res = conn.put_copy_data(data) + break if res == false end - expect( res ).to be_truthy - conn.finish - - stop_scheduler end + expect( res ).to be_truthy end end end From 15d23ffb161657fdcf7299c03b5327afce7c671d Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Mon, 6 Sep 2021 20:40:53 +0200 Subject: [PATCH 27/62] Make discard_results scheduler friendly --- ext/pg_connection.c | 45 +++++++++++++++++++++++++++++++++------ spec/pg/scheduler_spec.rb | 39 +++++++++++++++++++++++++++++++++ 2 files changed, 78 insertions(+), 6 deletions(-) diff --git a/ext/pg_connection.c b/ext/pg_connection.c index c4af6e5e9..c2967a99d 100644 --- a/ext/pg_connection.c +++ b/ext/pg_connection.c @@ -3113,22 +3113,55 @@ static VALUE pgconn_discard_results(VALUE self) { PGconn *conn = pg_get_pgconn(self); + VALUE socket_io; + + if( PQtransactionStatus(conn) == PQTRANS_IDLE ) { + return Qnil; + } + + socket_io = pgconn_socket_io(self); + + for(;;) { + PGresult *cur; + + /* pgconn_block() raises an exception in case of errors. + * To avoid this call rb_io_wait() and PQconsumeInput() without rb_raise(). + */ + while( gvl_PQisBusy(conn) ){ + rb_io_wait(socket_io, RB_INT2NUM(RUBY_IO_READABLE), Qnil); + if ( PQconsumeInput(conn) == 0 ) + return Qfalse; + } + + cur = gvl_PQgetResult(conn); + if( cur == NULL) break; - PGresult *cur; - while ((cur = gvl_PQgetResult(conn)) != NULL) { int status = PQresultStatus(cur); PQclear(cur); if (status == PGRES_COPY_IN){ gvl_PQputCopyEnd(conn, "COPY terminated by new PQexec"); } if (status == PGRES_COPY_OUT){ - char *buffer = NULL; - while( gvl_PQgetCopyData(conn, &buffer, 0) > 0) - PQfreemem(buffer); + for(;;) { + char *buffer = NULL; + int st = gvl_PQgetCopyData(conn, &buffer, 1); + if( st == 0 ) { + /* would block -> wait for readable data */ + rb_io_wait(socket_io, RB_INT2NUM(RUBY_IO_READABLE), Qnil); + if ( PQconsumeInput(conn) == 0 ) + return Qfalse; + } else if( st > 0 ) { + /* some data retrieved -> discard it */ + PQfreemem(buffer); + } else { + /* no more data */ + break; + } + } } } - return Qnil; + return Qtrue; } /* diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index 3ba6d19a3..70a7306fd 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -140,6 +140,45 @@ def run_with_scheduler(timeout=10) expect( res ).to be_truthy end end + + it "discards any pending results" do + run_with_scheduler do |conn| + conn.send_query("SELECT 5") + res = conn.exec("SELECT 6") + expect( res.values ).to eq( [["6"]] ) + end + end + + it "can discard_results after query" do + run_with_scheduler do |conn| + conn.send_query("SELECT 7") + conn.discard_results + conn.send_query("SELECT 8") + res = conn.get_result + expect( res.values ).to eq( [["8"]] ) + end + end + + it "can discard_results after COPY FROM STDIN" do + run_with_scheduler do |conn| + conn.exec( "CREATE TEMP TABLE copytable (col1 TEXT)" ) + conn.exec( "COPY copytable FROM STDIN" ) + conn.discard_results + conn.send_query("SELECT 2") + res = conn.get_result + expect( res.values ).to eq( [["2"]] ) + end + end + + it "can discard_results after COPY TO STDOUT" do + run_with_scheduler do |conn| + conn.exec("COPY (SELECT generate_series(0,999)::TEXT UNION ALL SELECT pg_sleep(1)::TEXT || '1000') TO STDOUT" ) + conn.discard_results + conn.send_query("SELECT 3") + res = conn.get_result + expect( res.values ).to eq( [["3"]] ) + end + end end # Do not wait for threads doing blocking calls at the process shutdown. From 5df6a712ce2baf406ba8877acf620a42713adfb3 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 7 Sep 2021 07:59:11 +0200 Subject: [PATCH 28/62] Fix compat to ruby < 3.0 Older rubies require the use of the rb_io_t structure for wait functions. But older rb_io_wait_* functions are soft-deprecated, since they have to re-wrap the IO objects for the scheduler. Moreover they don't fully support nonblocking behavior on Windows. --- ext/extconf.rb | 1 + ext/pg_connection.c | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+) diff --git a/ext/extconf.rb b/ext/extconf.rb index 9a795d3eb..da8bea8a6 100755 --- a/ext/extconf.rb +++ b/ext/extconf.rb @@ -142,6 +142,7 @@ module PG have_func 'timegm' have_func 'rb_gc_adjust_memory_usage' # since ruby-2.4 have_func 'rb_gc_mark_movable' # since ruby-2.7 +have_func 'rb_io_wait' # since ruby-3.0 # unistd.h confilicts with ruby/win32.h when cross compiling for win32 and ruby 1.9.1 have_header 'unistd.h' diff --git a/ext/pg_connection.c b/ext/pg_connection.c index c2967a99d..f7a278f11 100644 --- a/ext/pg_connection.c +++ b/ext/pg_connection.c @@ -3101,6 +3101,24 @@ pgconn_get_last_result(VALUE self) return rb_pgresult; } +#if !defined(HAVE_RB_IO_WAIT) + +typedef enum { + RUBY_IO_READABLE = RB_WAITFD_IN, + RUBY_IO_WRITABLE = RB_WAITFD_OUT, + RUBY_IO_PRIORITY = RB_WAITFD_PRI, +} rb_io_event_t; + +#define rb_io_wait(io, event, timeout) do { \ + rb_io_t *fptr; \ + GetOpenFile((io), fptr); \ + if( (event) == RB_INT2NUM(RUBY_IO_READABLE) ) \ + rb_io_wait_readable(fptr->fd); \ + else \ + rb_io_wait_writable(fptr->fd); \ +} while(0) +#endif + /* * call-seq: * conn.discard_results() From 63e00c2bc88709a9f83c311b4db218932515f240 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 7 Sep 2021 12:03:08 +0200 Subject: [PATCH 29/62] ID values can be allocated in Init This avoids a hash table lookup at runtime. --- ext/pg_connection.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/ext/pg_connection.c b/ext/pg_connection.c index f7a278f11..8199f96a5 100644 --- a/ext/pg_connection.c +++ b/ext/pg_connection.c @@ -12,6 +12,7 @@ VALUE rb_cPGconn; static ID s_id_encode; +static ID s_id_autoclose_set; static VALUE sym_type, sym_format, sym_value; static VALUE sym_symbol, sym_string, sym_static_symbol; @@ -916,7 +917,6 @@ pgconn_socket_io(VALUE self) { int sd; int ruby_sd; - ID id_autoclose = rb_intern("autoclose="); t_pg_connection *this = pg_get_connection_safe( self ); VALUE socket_io = this->socket_io; @@ -934,7 +934,7 @@ pgconn_socket_io(VALUE self) socket_io = rb_funcall( rb_cIO, rb_intern("for_fd"), 2, INT2NUM(ruby_sd), INT2NUM(2 /* File::RDWR */) ); /* Disable autoclose feature */ - rb_funcall( socket_io, id_autoclose, 1, Qfalse ); + rb_funcall( socket_io, s_id_autoclose_set, 1, Qfalse ); this->socket_io = socket_io; } @@ -4201,6 +4201,7 @@ void init_pg_connection() { s_id_encode = rb_intern("encode"); + s_id_autoclose_set = rb_intern("autoclose="); sym_type = ID2SYM(rb_intern("type")); sym_format = ID2SYM(rb_intern("format")); sym_value = ID2SYM(rb_intern("value")); From 6c885e8ba0b2f0f83d0476c186d69aa88ee7e2c4 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 7 Sep 2021 12:06:04 +0200 Subject: [PATCH 30/62] Use rb_io_wait if available and remove Windows specific wait functions Fall back to rb_wait_for_single_fd() on ruby < 3.0, which works on Windows as well. --- ext/pg_connection.c | 145 +++++++++---------------------------------- lib/pg/connection.rb | 24 ------- 2 files changed, 31 insertions(+), 138 deletions(-) diff --git a/ext/pg_connection.c b/ext/pg_connection.c index 8199f96a5..5a7271642 100644 --- a/ext/pg_connection.c +++ b/ext/pg_connection.c @@ -2289,103 +2289,43 @@ pgconn_notifies(VALUE self) return hash; } -/* Win32 + Ruby 1.9+ */ -#if defined( _WIN32 ) -/* - * On Windows, use platform-specific strategies to wait for the socket - * instead of rb_wait_for_single_fd(). - */ - -int rb_w32_wait_events( HANDLE *events, int num, DWORD timeout ); - -static void * -wait_socket_readable( PGconn *conn, struct timeval *ptimeout, void *(*is_readable)(PGconn *) ) -{ - int sd = PQsocket( conn ); - void *retval; - struct timeval aborttime={0,0}, currtime, waittime; - DWORD timeout_milisec = INFINITE; - DWORD wait_ret; - WSAEVENT hEvent; - - if ( sd < 0 ) - rb_raise(rb_eConnectionBad, "PQsocket() can't get socket descriptor"); - - hEvent = WSACreateEvent(); - /* Check for connection errors (PQisBusy is true on connection errors) */ - if( PQconsumeInput(conn) == 0 ) { - WSACloseEvent( hEvent ); - rb_raise( rb_eConnectionBad, "PQconsumeInput() %s", PQerrorMessage(conn) ); - } - - if ( ptimeout ) { - gettimeofday(&currtime, NULL); - timeradd(&currtime, ptimeout, &aborttime); - } - - while ( !(retval=is_readable(conn)) ) { - if ( WSAEventSelect(sd, hEvent, FD_READ|FD_CLOSE) == SOCKET_ERROR ) { - WSACloseEvent( hEvent ); - rb_raise( rb_eConnectionBad, "WSAEventSelect socket error: %d", WSAGetLastError() ); - } - - if ( ptimeout ) { - gettimeofday(&currtime, NULL); - timersub(&aborttime, &currtime, &waittime); - timeout_milisec = (DWORD)( waittime.tv_sec * 1e3 + waittime.tv_usec / 1e3 ); - } - - /* Is the given timeout valid? */ - if( !ptimeout || (waittime.tv_sec >= 0 && waittime.tv_usec >= 0) ){ - /* Wait for the socket to become readable before checking again */ - wait_ret = rb_w32_wait_events( &hEvent, 1, timeout_milisec ); - } else { - wait_ret = WAIT_TIMEOUT; - } +#if !defined(HAVE_RB_IO_WAIT) - if ( wait_ret == WAIT_TIMEOUT ) { - WSACloseEvent( hEvent ); - return NULL; - } else if ( wait_ret == WAIT_OBJECT_0 ) { - /* The event we were waiting for. */ - } else if ( wait_ret == WAIT_OBJECT_0 + 1) { - /* This indicates interruption from timer thread, GC, exception - * from other threads etc... */ - rb_thread_check_ints(); - } else if ( wait_ret == WAIT_FAILED ) { - WSACloseEvent( hEvent ); - rb_raise( rb_eConnectionBad, "Wait on socket error (WaitForMultipleObjects): %lu", GetLastError() ); - } else { - WSACloseEvent( hEvent ); - rb_raise( rb_eConnectionBad, "Wait on socket abandoned (WaitForMultipleObjects)" ); - } +typedef enum { + RUBY_IO_READABLE = RB_WAITFD_IN, + RUBY_IO_WRITABLE = RB_WAITFD_OUT, + RUBY_IO_PRIORITY = RB_WAITFD_PRI, +} rb_io_event_t; - /* Check for connection errors (PQisBusy is true on connection errors) */ - if ( PQconsumeInput(conn) == 0 ) { - WSACloseEvent( hEvent ); - rb_raise( rb_eConnectionBad, "PQconsumeInput() %s", PQerrorMessage(conn) ); - } +static VALUE +rb_io_wait(VALUE io, VALUE events, VALUE timeout) { + rb_io_t *fptr; + struct timeval waittime; + int res; + + GetOpenFile((io), fptr); + if( !NIL_P(timeout) ){ + waittime.tv_sec = (time_t)(NUM2DBL(timeout)); + waittime.tv_usec = (time_t)(NUM2DBL(timeout) - (double)waittime.tv_sec); } + res = rb_wait_for_single_fd(fptr->fd, NUM2UINT(events), NIL_P(timeout) ? NULL : &waittime); - WSACloseEvent( hEvent ); - return retval; + return UINT2NUM(res); } - -#else - -/* non Win32 */ +#endif static void * -wait_socket_readable( PGconn *conn, struct timeval *ptimeout, void *(*is_readable)(PGconn *)) +wait_socket_readable( VALUE self, struct timeval *ptimeout, void *(*is_readable)(PGconn *)) { - int sd = PQsocket( conn ); - int ret; + VALUE socket_io; + VALUE ret; void *retval; struct timeval aborttime={0,0}, currtime, waittime; + VALUE wait_timeout = Qnil; + PGconn *conn = pg_get_pgconn(self); - if ( sd < 0 ) - rb_raise(rb_eConnectionBad, "PQsocket() can't get socket descriptor"); + socket_io = pgconn_socket_io(self); /* Check for connection errors (PQisBusy is true on connection errors) */ if ( PQconsumeInput(conn) == 0 ) @@ -2400,22 +2340,19 @@ wait_socket_readable( PGconn *conn, struct timeval *ptimeout, void *(*is_readabl if ( ptimeout ) { gettimeofday(&currtime, NULL); timersub(&aborttime, &currtime, &waittime); + wait_timeout = DBL2NUM((double)(waittime.tv_sec) + (double)(waittime.tv_usec) / 1000000.0); } /* Is the given timeout valid? */ if( !ptimeout || (waittime.tv_sec >= 0 && waittime.tv_usec >= 0) ){ /* Wait for the socket to become readable before checking again */ - ret = rb_wait_for_single_fd( sd, RB_WAITFD_IN, ptimeout ? &waittime : NULL ); + ret = rb_io_wait(socket_io, RB_INT2NUM(RUBY_IO_READABLE), wait_timeout); } else { - ret = 0; - } - - if ( ret < 0 ){ - rb_sys_fail( "rb_wait_for_single_fd()" ); + ret = Qfalse; } /* Return false if the select() timed out */ - if ( ret == 0 ){ + if ( ret == Qfalse ){ return NULL; } @@ -2429,8 +2366,6 @@ wait_socket_readable( PGconn *conn, struct timeval *ptimeout, void *(*is_readabl } -#endif - static void * notify_readable(PGconn *conn) { @@ -2468,7 +2403,7 @@ pgconn_wait_for_notify(int argc, VALUE *argv, VALUE self) ptimeout = &timeout; } - pnotification = (PGnotify*) wait_socket_readable( this->pgconn, ptimeout, notify_readable); + pnotification = (PGnotify*) wait_socket_readable( self, ptimeout, notify_readable); /* Return nil if the select timed out */ if ( !pnotification ) return Qnil; @@ -3050,7 +2985,7 @@ pgconn_block( int argc, VALUE *argv, VALUE self ) { ptimeout = &timeout; } - ret = wait_socket_readable( conn, ptimeout, get_result_readable); + ret = wait_socket_readable( self, ptimeout, get_result_readable); if( !ret ) return Qfalse; @@ -3101,24 +3036,6 @@ pgconn_get_last_result(VALUE self) return rb_pgresult; } -#if !defined(HAVE_RB_IO_WAIT) - -typedef enum { - RUBY_IO_READABLE = RB_WAITFD_IN, - RUBY_IO_WRITABLE = RB_WAITFD_OUT, - RUBY_IO_PRIORITY = RB_WAITFD_PRI, -} rb_io_event_t; - -#define rb_io_wait(io, event, timeout) do { \ - rb_io_t *fptr; \ - GetOpenFile((io), fptr); \ - if( (event) == RB_INT2NUM(RUBY_IO_READABLE) ) \ - rb_io_wait_readable(fptr->fd); \ - else \ - rb_io_wait_writable(fptr->fd); \ -} while(0) -#endif - /* * call-seq: * conn.discard_results() diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index 1f1f8eacb..e60fade19 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -308,30 +308,6 @@ def ssl_attributes end end - if RUBY_PLATFORM =~ /mingw|mswin/ - def block(timeout=nil) - # pgconn_block in the C ext isn't Fiber.scheduler compatible on Windows. - # So use ruby code and socket_io.wait_readable instead. - - # Buffer any incoming data on the socket until a full result is ready. - consume_input - - if timeout - aborttime = Process.clock_gettime(Process::CLOCK_MONOTONIC) + timeout - end - - while is_busy - tm = timeout && - [0.0, aborttime - Process.clock_gettime(Process::CLOCK_MONOTONIC)].max - unless socket_io.wait_readable(tm) - return false - end - consume_input - end - return true - end - end - def async_exec(*args) discard_results async_send_query(*args) From 5ef9bae3e70245b1ac0772ed10d597f41d1641a1 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 7 Sep 2021 12:57:49 +0200 Subject: [PATCH 31/62] Remove now unused variable --- ext/pg_connection.c | 2 -- 1 file changed, 2 deletions(-) diff --git a/ext/pg_connection.c b/ext/pg_connection.c index 5a7271642..c5c4aa579 100644 --- a/ext/pg_connection.c +++ b/ext/pg_connection.c @@ -2970,8 +2970,6 @@ get_result_readable(PGconn *conn) */ static VALUE pgconn_block( int argc, VALUE *argv, VALUE self ) { - PGconn *conn = pg_get_pgconn( self ); - struct timeval timeout; struct timeval *ptimeout = NULL; VALUE timeout_in; From 47999468a4f90080a939a9f6b80192e55a974fbc Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 7 Sep 2021 13:17:36 +0200 Subject: [PATCH 32/62] Implement async_reset method using the nonblocking libpq API --- lib/pg/connection.rb | 22 ++++++++++++++++------ spec/pg/scheduler_spec.rb | 11 +++++++++++ 2 files changed, 27 insertions(+), 6 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index e60fade19..ff4e6fbc7 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -417,15 +417,16 @@ def async_put_copy_end(*args) true end + alias sync_reset reset + def async_reset + reset_start + self.class.send(:async_connect_reset, self, :reset_poll) + end class << self alias sync_connect new - def async_connect(*args, **kwargs) - conn = PG::Connection.connect_start( *args, **kwargs ) or - raise(PG::Error, "Unable to create a new connection") - raise(PG::ConnectionBad, conn.error_message) if conn.status == PG::CONNECTION_BAD - + private def async_connect_reset(conn, poll_meth) # Now grab a reference to the underlying socket so we know when the connection is established socket = conn.socket_io @@ -445,7 +446,7 @@ def async_connect(*args, **kwargs) end # Check to see if it's finished or failed yet - poll_status = conn.connect_poll + poll_status = conn.send( poll_meth ) end raise(PG::ConnectionBad, conn.error_message) unless conn.status == PG::CONNECTION_OK @@ -458,6 +459,14 @@ def async_connect(*args, **kwargs) conn end + def async_connect(*args, **kwargs) + conn = PG::Connection.connect_start(*args, **kwargs ) or + raise(PG::Error, "Unable to create a new connection") + raise(PG::ConnectionBad, conn.error_message) if conn.status == PG::CONNECTION_BAD + + async_connect_reset(conn, :connect_poll) + end + REDIRECT_CLASS_METHODS = { :new => [:async_connect, :sync_connect], } @@ -483,6 +492,7 @@ def async_connect(*args, **kwargs) :get_result => [:async_get_result, :sync_get_result], :get_last_result => [:async_get_last_result, :sync_get_last_result], :get_copy_data => [:async_get_copy_data, :sync_get_copy_data], + :reset => [:async_reset, :sync_reset], } def async_send_api=(enable) diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index 70a7306fd..300b04b1f 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -81,6 +81,17 @@ def run_with_scheduler(timeout=10) end end + it "can reset the connection" do + run_with_scheduler do + conn = PG.connect(@conninfo_gate) + conn.exec("SET search_path TO test1") + expect( conn.exec("SHOW search_path").values ).to eq( [["test1"]] ) + conn.reset + expect( conn.exec("SHOW search_path").values ).to eq( [['"$user", public']] ) + conn.finish + end + end + it "can retrieve several results" do run_with_scheduler do |conn| res = conn.send_query <<-EOT From 1fc4eceeee33e7fcb756a3890ffdb9dac7a84c89 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 7 Sep 2021 14:44:35 +0200 Subject: [PATCH 33/62] Move async_exec methods back to C This moves async_exec back to C and enabled other query methods like async_exec_prepared to be fully scheduler compatible. This also removes async/sync_send_* method variations in favor of using a flag to enable this. The performance of all query methods should now be similar to pg-1.2. --- ext/pg.h | 2 + ext/pg_connection.c | 33 ++++++++++ lib/pg/connection.rb | 135 +++++++++----------------------------- spec/pg/scheduler_spec.rb | 29 +++++++- 4 files changed, 94 insertions(+), 105 deletions(-) diff --git a/ext/pg.h b/ext/pg.h index 02d6c4e81..c41afbfd4 100644 --- a/ext/pg.h +++ b/ext/pg.h @@ -113,6 +113,8 @@ typedef struct { int enc_idx : PG_ENC_IDX_BITS; /* flags controlling Symbol/String field names */ unsigned int flags : 2; + /* enable automatic flushing of send data at the end of send_query calls */ + unsigned int flush_data : 1; #if defined(_WIN32) /* File descriptor to be used for rb_w32_unwrap_io_handle() */ diff --git a/ext/pg_connection.c b/ext/pg_connection.c index c5c4aa579..a8313f10c 100644 --- a/ext/pg_connection.c +++ b/ext/pg_connection.c @@ -21,6 +21,7 @@ static PQnoticeProcessor default_notice_processor = NULL; static VALUE pgconn_finish( VALUE ); static VALUE pgconn_set_default_encoding( VALUE self ); +static VALUE pgconn_wait_for_flush( VALUE self ); static void pgconn_set_internal_encoding_index( VALUE ); static const rb_data_type_t pg_connection_type; @@ -1811,6 +1812,7 @@ pgconn_send_query(int argc, VALUE *argv, VALUE self) rb_iv_set(error, "@connection", self); rb_exc_raise(error); } + pgconn_wait_for_flush( self ); return Qnil; } @@ -1888,6 +1890,7 @@ pgconn_send_query_params(int argc, VALUE *argv, VALUE self) rb_iv_set(error, "@connection", self); rb_exc_raise(error); } + pgconn_wait_for_flush( self ); return Qnil; } @@ -1951,6 +1954,7 @@ pgconn_send_prepare(int argc, VALUE *argv, VALUE self) rb_iv_set(error, "@connection", self); rb_exc_raise(error); } + pgconn_wait_for_flush( self ); return Qnil; } @@ -2019,6 +2023,7 @@ pgconn_send_query_prepared(int argc, VALUE *argv, VALUE self) rb_iv_set(error, "@connection", self); rb_exc_raise(error); } + pgconn_wait_for_flush( self ); return Qnil; } @@ -2040,6 +2045,7 @@ pgconn_send_describe_prepared(VALUE self, VALUE stmt_name) rb_iv_set(error, "@connection", self); rb_exc_raise(error); } + pgconn_wait_for_flush( self ); return Qnil; } @@ -2062,6 +2068,7 @@ pgconn_send_describe_portal(VALUE self, VALUE portal) rb_iv_set(error, "@connection", self); rb_exc_raise(error); } + pgconn_wait_for_flush( self ); return Qnil; } @@ -2291,6 +2298,7 @@ pgconn_notifies(VALUE self) #if !defined(HAVE_RB_IO_WAIT) +/* For compat with ruby < 3.0 */ typedef enum { RUBY_IO_READABLE = RB_WAITFD_IN, @@ -2365,6 +2373,29 @@ wait_socket_readable( VALUE self, struct timeval *ptimeout, void *(*is_readable) return retval; } +static VALUE +pgconn_wait_for_flush( VALUE self ){ + if( !pg_get_connection_safe(self)->flush_data ) + return Qnil; + + while( pgconn_flush(self) == Qfalse ){ + /* wait for the socket to become read- or write-ready */ + int events; + VALUE socket_io = pgconn_socket_io(self); + events = RB_NUM2INT(rb_io_wait(socket_io, RB_INT2NUM(RUBY_IO_READABLE | RUBY_IO_WRITABLE), Qnil)); + + if (events & RUBY_IO_READABLE) + pgconn_consume_input(self); + } + return Qnil; +} + +static VALUE +pgconn_flush_data_set( VALUE self, VALUE enabled ){ + t_pg_connection *conn = pg_get_connection(self); + conn->flush_data = RTEST(enabled); + return enabled; +} static void * notify_readable(PGconn *conn) @@ -4254,6 +4285,8 @@ init_pg_connection() rb_define_method(rb_cPGconn, "set_client_encoding", pgconn_set_client_encoding, 1); rb_define_alias(rb_cPGconn, "client_encoding=", "set_client_encoding"); rb_define_method(rb_cPGconn, "block", pgconn_block, -1); + rb_define_private_method(rb_cPGconn, "wait_for_flush", pgconn_wait_for_flush, 0); + rb_define_private_method(rb_cPGconn, "flush_data=", pgconn_flush_data_set, 1); rb_define_method(rb_cPGconn, "wait_for_notify", pgconn_wait_for_notify, -1); rb_define_alias(rb_cPGconn, "notifies_wait", "wait_for_notify"); rb_define_method(rb_cPGconn, "quote_ident", pgconn_s_quote_ident, 1); diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index ff4e6fbc7..01be0185c 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -285,67 +285,6 @@ def ssl_attributes end end - private def wait_for_flush - # From https://www.postgresql.org/docs/13/libpq-async.html - # After sending any command or data on a nonblocking connection, call PQflush. If it returns 1, wait for the socket to become read- or write-ready. If it becomes write-ready, call PQflush again. If it becomes read-ready, call PQconsumeInput , then call PQflush again. Repeat until PQflush returns 0. (It is necessary to check for read-ready and drain the input with PQconsumeInput , because the server can block trying to send us data, e.g., NOTICE messages, and won't read our data until we read its.) Once PQflush returns 0, wait for the socket to be read-ready and then read the response as described above. - - until flush() - # wait for the socket to become read- or write-ready - - if Fiber.respond_to?(:scheduler) && Fiber.scheduler - # If a scheduler is set use it directly. - # This is necessary since IO.select isn't passed to the scheduler. - events = Fiber.scheduler.io_wait(socket_io, IO::READABLE | IO::WRITABLE, nil) - if (events & IO::READABLE) > 0 - consume_input - end - else - readable, writable = IO.select([socket_io], [socket_io]) - if readable.any? - consume_input - end - end - end - end - - def async_exec(*args) - discard_results - async_send_query(*args) - - res = async_get_last_result - - if block_given? - begin - return yield(res) - ensure - res.clear - end - end - res - end - - def async_exec_params(*args) - discard_results - - if args[1].nil? - # TODO: pg_deprecated(3, ("forwarding async_exec_params to async_exec is deprecated")); - async_send_query(*args) - else - async_send_query_params(*args) - end - - res = async_get_last_result - - if block_given? - begin - return yield(res) - ensure - res.clear - end - end - res - end - alias sync_get_result get_result def async_get_result(*args) block @@ -371,18 +310,6 @@ def async_get_copy_data(async=false, decoder=nil) end end - alias sync_send_query send_query - def async_send_query(*args, &block) - sync_send_query(*args) - wait_for_flush - end - - alias sync_send_query_params send_query_params - def async_send_query_params(*args, &block) - sync_send_query_params(*args) - wait_for_flush - end - # In async_api=false mode all send calls run directly on libpq. # Blocking vs. nonblocking state can be changed in libpq. alias sync_setnonblocking setnonblocking @@ -391,6 +318,7 @@ def async_send_query_params(*args, &block) # The difference is that setnonblocking(true) disables automatic handling of would-block cases. def async_setnonblocking(enabled) singleton_class.async_send_api = !enabled + self.flush_data = !enabled sync_setnonblocking(true) end @@ -420,51 +348,52 @@ def async_put_copy_end(*args) alias sync_reset reset def async_reset reset_start - self.class.send(:async_connect_reset, self, :reset_poll) + async_connect_reset(:reset_poll) end - class << self - alias sync_connect new + private def async_connect_reset(poll_meth) + # Now grab a reference to the underlying socket so we know when the connection is established + socket = socket_io - private def async_connect_reset(conn, poll_meth) - # Now grab a reference to the underlying socket so we know when the connection is established - socket = conn.socket_io + # Track the progress of the connection, waiting for the socket to become readable/writable before polling it + poll_status = PG::PGRES_POLLING_WRITING + until poll_status == PG::PGRES_POLLING_OK || + poll_status == PG::PGRES_POLLING_FAILED - # Track the progress of the connection, waiting for the socket to become readable/writable before polling it - poll_status = PG::PGRES_POLLING_WRITING - until poll_status == PG::PGRES_POLLING_OK || - poll_status == PG::PGRES_POLLING_FAILED + # If the socket needs to read, wait 'til it becomes readable to poll again + case poll_status + when PG::PGRES_POLLING_READING + socket.wait_readable - # If the socket needs to read, wait 'til it becomes readable to poll again - case poll_status - when PG::PGRES_POLLING_READING - socket.wait_readable + # ...and the same for when the socket needs to write + when PG::PGRES_POLLING_WRITING + socket.wait_writable + end - # ...and the same for when the socket needs to write - when PG::PGRES_POLLING_WRITING - socket.wait_writable - end + # Check to see if it's finished or failed yet + poll_status = send( poll_meth ) + end - # Check to see if it's finished or failed yet - poll_status = conn.send( poll_meth ) - end + raise(PG::ConnectionBad, error_message) unless status == PG::CONNECTION_OK - raise(PG::ConnectionBad, conn.error_message) unless conn.status == PG::CONNECTION_OK + # Set connection to nonblocking to handle all blocking states in ruby. + # That way a fiber scheduler is able to handle IO requests. + sync_setnonblocking(true) + self.flush_data = true + set_default_encoding - # Set connection to nonblocking to handle all blocking states in ruby. - # That way a fiber scheduler is able to handle IO requests. - conn.sync_setnonblocking(true) - conn.set_default_encoding + self + end - conn - end + class << self + alias sync_connect new def async_connect(*args, **kwargs) conn = PG::Connection.connect_start(*args, **kwargs ) or raise(PG::Error, "Unable to create a new connection") raise(PG::ConnectionBad, conn.error_message) if conn.status == PG::CONNECTION_BAD - async_connect_reset(conn, :connect_poll) + conn.send(:async_connect_reset, :connect_poll) end REDIRECT_CLASS_METHODS = { @@ -473,8 +402,6 @@ def async_connect(*args, **kwargs) # These methods are affected by PQsetnonblocking REDIRECT_SEND_METHODS = { - :send_query => [:async_send_query, :sync_send_query], - :send_query_params => [:async_send_query_params, :sync_send_query_params], :isnonblocking => [:async_isnonblocking, :sync_isnonblocking], :nonblocking? => [:async_isnonblocking, :sync_isnonblocking], :put_copy_data => [:async_put_copy_data, :sync_put_copy_data], diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index 300b04b1f..d41c792e3 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -64,7 +64,7 @@ def run_with_scheduler(timeout=10) end end - it "waits when sending data" do + it "waits when sending query data" do run_with_scheduler do |conn| data = "x" * 1000 * 1000 * 10 res = conn.exec_params("SELECT LENGTH($1)", [data]) @@ -190,6 +190,33 @@ def run_with_scheduler(timeout=10) expect( res.values ).to eq( [["3"]] ) end end + + it "should convert strings and parameters to #prepare and #exec_prepared" do + run_with_scheduler do |conn| + conn.prepare("weiß1", "VALUES( LENGTH($1), 'grün')") + data = "x" * 1000 * 1000 * 10 + r = conn.exec_prepared("weiß1", [data]) + expect( r.values ).to eq( [[data.length.to_s, 'grün']] ) + end + end + + it "should convert strings to #describe_prepared" do + run_with_scheduler do |conn| + conn.prepare("weiß2", "VALUES(123)") + r = conn.describe_prepared("weiß2") + expect( r.nfields ).to eq( 1 ) + end + end + + it "should convert strings to #describe_portal" do + run_with_scheduler do |conn| + conn.transaction do + conn.exec "DECLARE cörsör CURSOR FOR VALUES(1,2,3)" + r = conn.describe_portal("cörsör") + expect( r.nfields ).to eq( 3 ) + end + end + end end # Do not wait for threads doing blocking calls at the process shutdown. From 7944e1a867722b0683ef7eaf148841abf1dd72ad Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 7 Sep 2021 15:50:42 +0200 Subject: [PATCH 34/62] Add an option to extconf.rb to enable nogvl-wrapping of libpq functions. Let's see how good the new default of not releasing GVL works. In case it doesn't this should re-enable it: gem inst pg -- --enable-nogvl --- ext/extconf.rb | 4 ++++ ext/gvl_wrappers.c | 4 ++++ ext/gvl_wrappers.h | 15 ++++++++++++--- 3 files changed, 20 insertions(+), 3 deletions(-) diff --git a/ext/extconf.rb b/ext/extconf.rb index da8bea8a6..8ed3cbbe7 100755 --- a/ext/extconf.rb +++ b/ext/extconf.rb @@ -15,6 +15,10 @@ ENV['PATH'] = "#{pgdir}/bin" + File::PATH_SEPARATOR + ENV['PATH'] end +if enable_config("nogvl") + $defs.push( "-DENABLE_NOGVL" ) +end + if enable_config("windows-cross") # Avoid dependency to external libgcc.dll on x86-mingw32 $LDFLAGS << " -static-libgcc" diff --git a/ext/gvl_wrappers.c b/ext/gvl_wrappers.c index eee8ff5ba..9e7554f51 100644 --- a/ext/gvl_wrappers.c +++ b/ext/gvl_wrappers.c @@ -9,9 +9,13 @@ char *PQencryptPasswordConn(PGconn *conn, const char *passwd, const char *user, const char *algorithm){return NULL;} #endif +#ifdef ENABLE_NOGVL FOR_EACH_BLOCKING_FUNCTION( DEFINE_GVL_WRAPPER_STRUCT ); FOR_EACH_BLOCKING_FUNCTION( DEFINE_GVL_SKELETON ); +#endif FOR_EACH_BLOCKING_FUNCTION( DEFINE_GVL_STUB ); +#ifdef ENABLE_NOGVL FOR_EACH_CALLBACK_FUNCTION( DEFINE_GVL_WRAPPER_STRUCT ); FOR_EACH_CALLBACK_FUNCTION( DEFINE_GVLCB_SKELETON ); +#endif FOR_EACH_CALLBACK_FUNCTION( DEFINE_GVLCB_STUB ); diff --git a/ext/gvl_wrappers.h b/ext/gvl_wrappers.h index f84b5a8e9..5ea102cbf 100644 --- a/ext/gvl_wrappers.h +++ b/ext/gvl_wrappers.h @@ -17,6 +17,10 @@ #include +#ifdef RUBY_EXTCONF_H +# include RUBY_EXTCONF_H +#endif + #define DEFINE_PARAM_LIST1(type, name) \ name, @@ -46,7 +50,8 @@ return NULL; \ } -#define nono_DEFINE_GVL_STUB(name, when_non_void, rettype, lastparamtype, lastparamname) \ +#ifdef ENABLE_NOGVL +#define DEFINE_GVL_STUB(name, when_non_void, rettype, lastparamtype, lastparamname) \ rettype gvl_##name(FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST3) lastparamtype lastparamname){ \ struct gvl_wrapper_##name##_params params = { \ {FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST1) lastparamname}, when_non_void((rettype)0) \ @@ -54,12 +59,13 @@ rb_thread_call_without_gvl(gvl_##name##_skeleton, ¶ms, RUBY_UBF_IO, 0); \ when_non_void( return params.retval; ) \ } - +#else #define DEFINE_GVL_STUB(name, when_non_void, rettype, lastparamtype, lastparamname) \ rettype gvl_##name(FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST3) lastparamtype lastparamname){ \ when_non_void( return ) \ name( FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST1) lastparamname ); \ } +#endif #define DEFINE_GVL_STUB_DECL(name, when_non_void, rettype, lastparamtype, lastparamname) \ rettype gvl_##name(FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST3) lastparamtype lastparamname); @@ -72,7 +78,8 @@ return NULL; \ } -#define nono_DEFINE_GVLCB_STUB(name, when_non_void, rettype, lastparamtype, lastparamname) \ +#ifdef ENABLE_NOGVL +#define DEFINE_GVLCB_STUB(name, when_non_void, rettype, lastparamtype, lastparamname) \ rettype gvl_##name(FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST3) lastparamtype lastparamname){ \ struct gvl_wrapper_##name##_params params = { \ {FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST1) lastparamname}, when_non_void((rettype)0) \ @@ -80,11 +87,13 @@ rb_thread_call_with_gvl(gvl_##name##_skeleton, ¶ms); \ when_non_void( return params.retval; ) \ } +#else #define DEFINE_GVLCB_STUB(name, when_non_void, rettype, lastparamtype, lastparamname) \ rettype gvl_##name(FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST3) lastparamtype lastparamname){ \ when_non_void( return ) \ name( FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST1) lastparamname ); \ } +#endif #define GVL_TYPE_VOID(string) #define GVL_TYPE_NONVOID(string) string From 56a764e81f7928f79e8d285ff26a7d773ccac82e Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 7 Sep 2021 19:35:09 +0200 Subject: [PATCH 35/62] Catch EINVAL while remote_address This happens sometimes on MacOS. Errno::EINVAL: Invalid argument - getpeername(2) --- spec/helpers/tcp_gate_scheduler.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index 8220fcfee..20b25ddaf 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -197,7 +197,7 @@ def io_wait(io, events, duration) sock = TCPSocket.for_fd(io.fileno) sock.autoclose = false remote_address = sock.remote_address - rescue Errno::ENOTCONN + rescue Errno::ENOTCONN, Errno::EINVAL end unless @started From 3b2fa5e26c16b76a3aafb1fc23d9f5bee596400d Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Fri, 10 Sep 2021 20:53:08 +0200 Subject: [PATCH 36/62] Add async_set_client_encoding which is compatible to scheduler --- ext/pg_connection.c | 43 ++++++++++++++++++++++++++++----------- lib/pg/connection.rb | 2 ++ spec/pg/scheduler_spec.rb | 22 ++++++++++++++++++++ 3 files changed, 55 insertions(+), 12 deletions(-) diff --git a/ext/pg_connection.c b/ext/pg_connection.c index a8313f10c..0198dc252 100644 --- a/ext/pg_connection.c +++ b/ext/pg_connection.c @@ -2906,9 +2906,11 @@ pgconn_get_client_encoding(VALUE self) /* * call-seq: - * conn.set_client_encoding( encoding ) + * conn.sync_set_client_encoding( encoding ) * - * Sets the client encoding to the _encoding_ String. + * This function has the same behavior as #async_set_client_encoding, but is implemented using the synchronous command processing API of libpq. + * See #async_exec for the differences between the two API variants. + * It's not recommended to use explicit sync or async variants but #set_client_encoding instead, unless you have a good reason to do so. */ static VALUE pgconn_set_client_encoding(VALUE self, VALUE str) @@ -3853,16 +3855,33 @@ pgconn_external_encoding(VALUE self) return rb_enc_from_encoding( enc ); } +/* + * call-seq: + * conn.set_client_encoding( encoding ) + * + * Sets the client encoding to the _encoding_ String. + */ +static VALUE +pgconn_async_set_client_encoding(VALUE self, VALUE encname) +{ + VALUE query_format, query; + + Check_Type(encname, T_STRING); + query_format = rb_str_new_cstr("set client_encoding to '%s'"); + query = rb_funcall(query_format, rb_intern("%"), 1, encname); + + pgconn_async_exec(1, &query, self); + pgconn_set_internal_encoding_index( self ); + + return Qnil; +} static VALUE pgconn_set_client_encoding_async1( VALUE args ) { VALUE self = ((VALUE*)args)[0]; VALUE encname = ((VALUE*)args)[1]; - VALUE query_format = rb_str_new_cstr("set client_encoding to '%s'"); - VALUE query = rb_funcall(query_format, rb_intern("%"), 1, encname); - - pgconn_async_exec(1, &query, self); + pgconn_async_set_client_encoding(self, encname); return 0; } @@ -3877,9 +3896,9 @@ pgconn_set_client_encoding_async2( VALUE arg, VALUE ex ) static VALUE -pgconn_set_client_encoding_async( VALUE self, const char *encname ) +pgconn_set_client_encoding_async( VALUE self, VALUE encname ) { - VALUE args[] = { self, rb_str_new_cstr(encname) }; + VALUE args[] = { self, encname }; return rb_rescue(pgconn_set_client_encoding_async1, (VALUE)&args, pgconn_set_client_encoding_async2, Qnil); } @@ -3901,10 +3920,9 @@ pgconn_set_default_encoding( VALUE self ) if (( enc = rb_default_internal_encoding() )) { encname = pg_get_rb_encoding_as_pg_encoding( enc ); - if ( pgconn_set_client_encoding_async(self, encname) != 0 ) + if ( pgconn_set_client_encoding_async(self, rb_str_new_cstr(encname)) != 0 ) rb_warning( "Failed to set the default_internal encoding to %s: '%s'", encname, PQerrorMessage(conn) ); - pgconn_set_internal_encoding_index( self ); return rb_enc_from_encoding( enc ); } else { pgconn_set_internal_encoding_index( self ); @@ -4282,8 +4300,9 @@ init_pg_connection() /****** PG::Connection INSTANCE METHODS: Other ******/ rb_define_method(rb_cPGconn, "get_client_encoding", pgconn_get_client_encoding, 0); - rb_define_method(rb_cPGconn, "set_client_encoding", pgconn_set_client_encoding, 1); - rb_define_alias(rb_cPGconn, "client_encoding=", "set_client_encoding"); + rb_define_method(rb_cPGconn, "sync_set_client_encoding", pgconn_set_client_encoding, 1); + rb_define_method(rb_cPGconn, "async_set_client_encoding", pgconn_async_set_client_encoding, 1); + rb_define_alias(rb_cPGconn, "async_client_encoding=", "async_set_client_encoding"); rb_define_method(rb_cPGconn, "block", pgconn_block, -1); rb_define_private_method(rb_cPGconn, "wait_for_flush", pgconn_wait_for_flush, 0); rb_define_private_method(rb_cPGconn, "flush_data=", pgconn_flush_data_set, 1); diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index 01be0185c..d2160acd6 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -420,6 +420,8 @@ def async_connect(*args, **kwargs) :get_last_result => [:async_get_last_result, :sync_get_last_result], :get_copy_data => [:async_get_copy_data, :sync_get_copy_data], :reset => [:async_reset, :sync_reset], + :set_client_encoding => [:async_set_client_encoding, :sync_set_client_encoding], + :client_encoding= => [:async_set_client_encoding, :sync_set_client_encoding], } def async_send_api=(enable) diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index d41c792e3..58ddc95fb 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -64,6 +64,28 @@ def run_with_scheduler(timeout=10) end end + it "connects to a server with setting default encoding" do + Encoding.default_internal = Encoding::ISO8859_3 + begin + run_with_scheduler do |conn| + res = conn.exec_params("SELECT 8", []) + expect(res.getvalue(0,0).encoding).to eq(Encoding::ISO8859_3) + expect( conn.get_client_encoding ).to eq( "LATIN3" ) + end + ensure + Encoding.default_internal = nil + end + end + + it "can set_client_encoding" do + run_with_scheduler do |conn| + expect( conn.set_client_encoding('iso8859-4') ).to eq( nil ) + expect( conn.get_client_encoding ).to eq( "LATIN4" ) + conn.client_encoding = 'iso8859-2' + expect( conn.get_client_encoding ).to eq( "LATIN2" ) + end + end + it "waits when sending query data" do run_with_scheduler do |conn| data = "x" * 1000 * 1000 * 10 From 5bc7f7c23edadfdf7a279804f9b503f3529fe998 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sun, 12 Sep 2021 21:19:14 +0200 Subject: [PATCH 37/62] Remove PostgreSQL version guard since we don't support PostgreSQL-9.0 --- lib/pg/connection.rb | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index d2160acd6..65a97e416 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -41,11 +41,7 @@ def self::parse_connect_args( *args ) option_string = '' options = {} - # Parameter 'fallback_application_name' was introduced in PostgreSQL 9.0 - # together with PQescapeLiteral(). - if PG::Connection.instance_methods.find {|m| m.to_sym == :escape_literal } - options[:fallback_application_name] = $0.sub( /^(.{30}).{4,}(.{30})$/ ){ $1+"..."+$2 } - end + options[:fallback_application_name] = $0.sub( /^(.{30}).{4,}(.{30})$/ ){ $1+"..."+$2 } if args.length == 1 case args.first From 4a9b33460bfd84a499423115fcf9ccc95a913185 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sun, 12 Sep 2021 21:25:34 +0200 Subject: [PATCH 38/62] Avoid PG.connect blocking while address resolution This is done by resolving the address in ruby and providing hostaddr= as described in libpq docs. Test requires ruby-3.1+ to intercept address resolution through the scheduler. --- lib/pg/connection.rb | 62 +++++++++++++++++++++++--- spec/helpers.rb | 1 + spec/helpers/tcp_gate_scheduler.rb | 8 ++++ spec/pg/connection_spec.rb | 71 +++++++++++++++++++++++++----- spec/pg/scheduler_spec.rb | 10 +++++ 5 files changed, 135 insertions(+), 17 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index 65a97e416..b2abfe643 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -26,17 +26,46 @@ class PG::Connection CONNECT_ARGUMENT_ORDER = %w[host port options tty dbname user password] - ### Quote the given +value+ for use in a connection-parameter string. - def self::quote_connstr( value ) + ### Quote a single +value+ for use in a connection-parameter string. + def self.quote_connstr( value ) return "'" + value.to_s.gsub( /[\\']/ ) {|m| '\\' + m } + "'" end + # Convert Hash options to connection String + # + # Values are properly quoted and escaped. + def self.connect_hash_to_string( hash ) + hash.map { |k,v| "#{k}=#{quote_connstr(v)}" }.join( ' ' ) + end + + # Decode a connection string to Hash options + # + # Value are properly unquoted and unescaped. + def self.connect_string_to_hash( str ) + options = {} + key = nil + value = String.new + str.scan(/\G\s*(?>([^\s\\\']+)\s*=\s*|([^\s\\\']+)|'((?:[^\'\\]|\\.)*)'|(\\.?)|(\S))(\s|\z)?/m) do + |k, word, sq, esc, garbage, sep| + raise ArgumentError, "unterminated quoted string in connection info string: #{str.inspect}" if garbage + if k + key = k + else + value << (word || (sq || esc).gsub(/\\(.)/, '\\1')) + end + if sep + raise "missing = after #{value.inspect}" unless key + options[key.to_sym] = value + key = nil + value = String.new + end + end + options + end ### Parse the connection +args+ into a connection-parameter string. See PG::Connection.new ### for valid arguments. def self::parse_connect_args( *args ) - return '' if args.empty? - hash_arg = args.last.is_a?( Hash ) ? args.pop : {} option_string = '' options = {} @@ -65,7 +94,28 @@ def self::parse_connect_args( *args ) end end - options.merge!( hash_arg ) + options.merge!( hash_arg.transform_keys(&:to_sym) ) + + # Resolve DNS in Ruby to avoid blocking state while connecting, when it ... + if (host = options[:host] || connect_string_to_hash(option_string)[:host]) && + # isn't UnixSocket + !host.empty? && !host.start_with?("/") && + # isn't a path on Windows + (RUBY_PLATFORM !~ /mingw|mswin/ || host !~ /\A\w:[\/\\]/) + + options[:hostaddr] ||= if + Fiber.respond_to?(:scheduler) && + Fiber.scheduler && + RUBY_VERSION < '3.1.' + + # Use pure Ruby address resolver to avoid blocking of the scheduler. + # `IPSocket.getaddress` isn't fiber aware before ruby-3.1. + require "resolv" + Resolv.getaddress(host) + else + IPSocket.getaddress(host) + end + end if uri uri.host = nil if options[:host] @@ -77,7 +127,7 @@ def self::parse_connect_args( *args ) return uri.to_s.sub( /^#{uri.scheme}:(?!\/\/)/, "#{uri.scheme}://" ) else option_string += ' ' unless option_string.empty? && options.empty? - return option_string + options.map { |k,v| "#{k}=#{quote_connstr(v)}" }.join( ' ' ) + return option_string + connect_hash_to_string(options) end end diff --git a/spec/helpers.rb b/spec/helpers.rb index f80e8b166..cd03c0ccb 100644 --- a/spec/helpers.rb +++ b/spec/helpers.rb @@ -393,6 +393,7 @@ def wait_for_flush(conn) config.filter_run_excluding( :postgresql_10 ) if PG.library_version < 100000 config.filter_run_excluding( :postgresql_12 ) if PG.library_version < 120000 config.filter_run_excluding( :scheduler ) if RUBY_VERSION < "3.0" + config.filter_run_excluding( :scheduler_address_resolve ) if RUBY_VERSION < "3.1" ### Automatically set up and tear down the database config.before(:suite) do |*args| diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index 20b25ddaf..146a20959 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -256,5 +256,13 @@ def io_wait(io, events, duration) super end + + # Rewrite the hostname to verify that address resolution goes through the scheduler. + def address_resolve(hostname) + if hostname =~ /\Ascheduler-(.*)/ + hostname = $1 + end + super(hostname) + end end end diff --git a/spec/pg/connection_spec.rb b/spec/pg/connection_spec.rb index b70f83999..e380a4d65 100644 --- a/spec/pg/connection_spec.rb +++ b/spec/pg/connection_spec.rb @@ -14,27 +14,70 @@ expect( ObjectSpace.memsize_of(@conn) ).to be > DATA_OBJ_MEMSIZE end - it "can create a connection option string from a Hash of options" do + it "encode and decode Hash to connection string to Hash" do + hash = { + :host => 'pgsql.example.com', + :dbname => 'db01', + 'sslmode' => 'require', + 'somekey' => '', + 'password' => "\\ \t\n\"'", + } + optstring = described_class.connect_hash_to_string(hash) + res = described_class.connect_string_to_hash(optstring) + + expect( res ).to eq( hash.transform_keys(&:to_sym) ) + end + + it "decode option string to Hash" do + optstring = "host=c:\\\\pipe password = \\\\\\'\" " + res = described_class.connect_string_to_hash(optstring) + + expect( res ).to eq({ + host: 'c:\pipe', + password: "\\'\"", + }) + end + + it "shouldn't resolve absolute path in connection option string" do + optstring = described_class.parse_connect_args( + :host => '/var/socket' + ) + expect( optstring ).to match( /(^|\s)host='\/var\/socket'/ ) + expect( optstring ).not_to match( /hostaddr/ ) + end + + it "shouldn't resolve Windows path in connection option string", :windows do + optstring = described_class.parse_connect_args( + :host => "C:\\pipe\\00" + ) + expect( optstring ).to match( /(^|\s)host='C:\\\\pipe\\\\00'/ ) + expect( optstring ).not_to match( /hostaddr/ ) + end + + it "can create a connection option string from a Hash parameter" do optstring = described_class.parse_connect_args( :host => 'pgsql.example.com', :dbname => 'db01', - 'sslmode' => 'require' + 'sslmode' => 'require', + 'hostaddr' => '1.2.3.4' ) expect( optstring ).to be_a( String ) expect( optstring ).to match( /(^|\s)host='pgsql.example.com'/ ) expect( optstring ).to match( /(^|\s)dbname='db01'/ ) expect( optstring ).to match( /(^|\s)sslmode='require'/ ) + expect( optstring ).to match( /(^|\s)hostaddr='1.2.3.4'/ ) end it "can create a connection option string from positional parameters" do - optstring = described_class.parse_connect_args( 'pgsql.example.com', nil, '-c geqo=off', nil, + optstring = described_class.parse_connect_args( 'localhost', nil, '-c geqo=off', nil, 'sales' ) expect( optstring ).to be_a( String ) - expect( optstring ).to match( /(^|\s)host='pgsql.example.com'/ ) + expect( optstring ).to match( /(^|\s)host='localhost'/ ) expect( optstring ).to match( /(^|\s)dbname='sales'/ ) expect( optstring ).to match( /(^|\s)options='-c geqo=off'/ ) + expect( optstring ).to match( /(^|\s)hostaddr='(::1|127.0.0.1)'/ ) expect( optstring ).to_not match( /port=/ ) expect( optstring ).to_not match( /tty=/ ) @@ -42,30 +85,32 @@ it "can create a connection option string from a mix of positional and hash parameters" do optstring = described_class.parse_connect_args( 'pgsql.example.com', - :dbname => 'licensing', :user => 'jrandom' ) + :dbname => 'licensing', :user => 'jrandom', + 'hostaddr' => '1.2.3.4' ) expect( optstring ).to be_a( String ) expect( optstring ).to match( /(^|\s)host='pgsql.example.com'/ ) expect( optstring ).to match( /(^|\s)dbname='licensing'/ ) expect( optstring ).to match( /(^|\s)user='jrandom'/ ) + expect( optstring ).to match( /(^|\s)hostaddr='1.2.3.4'/ ) end it "can create a connection option string from an option string and a hash" do - optstring = described_class.parse_connect_args( 'dbname=original', :user => 'jrandom' ) + optstring = described_class.parse_connect_args( 'dbname=original', :user => 'jrandom', 'host' => 'localhost' ) expect( optstring ).to be_a( String ) expect( optstring ).to match( /(^|\s)dbname=original/ ) expect( optstring ).to match( /(^|\s)user='jrandom'/ ) + expect( optstring ).to match( /(^|\s)hostaddr='(::1|127.0.0.1)'/ ) end it "escapes single quotes and backslashes in connection parameters" do expect( - described_class.parse_connect_args( "DB 'browser' \\" ) - ).to match( /host='DB \\'browser\\' \\\\'/ ) - + described_class.parse_connect_args( password: "DB 'browser' \\" ) + ).to match( /password='DB \\'browser\\' \\\\'/ ) end - let(:uri) { 'postgresql://user:pass@pgsql.example.com:222/db01?sslmode=require' } + let(:uri) { 'postgresql://user:pass@pgsql.example.com:222/db01?sslmode=require&hostaddr=4.3.2.1' } it "can connect using a URI" do string = described_class.parse_connect_args( uri ) @@ -88,7 +133,9 @@ expect( string ).to match( %r{^postgresql://user:pass@pgsql.example.com:222/db01\?} ) expect( string ).to match( %r{\?.*sslmode=require} ) expect( string ).to match( %r{\?.*connect_timeout=2} ) + end + it "can create a connection URI from a URI and prefers the hash" do string = described_class.parse_connect_args( uri, :user => 'a', :password => 'b', @@ -103,6 +150,7 @@ expect( string ).to match( %r{\?.*host=localhost} ) expect( string ).to match( %r{\?.*port=555} ) expect( string ).to match( %r{\?.*dbname=x} ) + expect( string ).to match( %r{\?.*hostaddr=(%3A%3A1|127.0.0.1)} ) end it "can create a connection URI with a non-standard domain socket directory" do @@ -120,7 +168,7 @@ end it "connects with defaults if no connection parameters are given" do - expect( described_class.parse_connect_args ).to eq( '' ) + expect( described_class.parse_connect_args ).to match( /fallback_application_name='[^']+'/ ) end it "connects successfully with connection string" do @@ -134,6 +182,7 @@ expect( string ).to match( %r{(^|\s)host=localhost} ) expect( string ).to match( %r{(^|\s)port=555} ) expect( string ).to match( %r{(^|\s)dbname=test} ) + expect( string ).to match( %r{(^|\s)hostaddr='(::1|127.0.0.1)'} ) end it "connects successfully with connection string" do diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index 58ddc95fb..b34fa7936 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -103,6 +103,16 @@ def run_with_scheduler(timeout=10) end end + it "can connect with DNS lookup", :scheduler_address_resolve do + run_with_scheduler do + conninfo = @conninfo_gate.gsub(/(^| )host=\w+/, " host=scheduler-localhost") + conn = PG.connect(conninfo) + opt = conn.conninfo.find { |info| info[:keyword] == 'host' } + expect( opt[:val] ).to eq( 'scheduler-localhost' ) + conn.finish + end + end + it "can reset the connection" do run_with_scheduler do conn = PG.connect(@conninfo_gate) From 1c538cf23325640cdb6f3a1ccfca9eeb6985214c Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sun, 12 Sep 2021 21:49:03 +0200 Subject: [PATCH 39/62] Avoid double fault, when the connection couldn't be established --- spec/helpers.rb | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/spec/helpers.rb b/spec/helpers.rb index cd03c0ccb..5fb2f9bbd 100644 --- a/spec/helpers.rb +++ b/spec/helpers.rb @@ -38,8 +38,10 @@ def self::included( mod ) end mod.after( :all ) do - check_for_lingering_connections( @conn ) - @conn.finish + if @conn + check_for_lingering_connections( @conn ) + @conn.finish + end end end From 8cab8a1effeac5292fc769964f7da2e5d86f24d1 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Mon, 13 Sep 2021 10:26:21 +0200 Subject: [PATCH 40/62] Avoid compiler warning about mixed declaration and code --- ext/pg_connection.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ext/pg_connection.c b/ext/pg_connection.c index 0198dc252..4061ebfa0 100644 --- a/ext/pg_connection.c +++ b/ext/pg_connection.c @@ -3089,6 +3089,7 @@ pgconn_discard_results(VALUE self) for(;;) { PGresult *cur; + int status; /* pgconn_block() raises an exception in case of errors. * To avoid this call rb_io_wait() and PQconsumeInput() without rb_raise(). @@ -3102,7 +3103,7 @@ pgconn_discard_results(VALUE self) cur = gvl_PQgetResult(conn); if( cur == NULL) break; - int status = PQresultStatus(cur); + status = PQresultStatus(cur); PQclear(cur); if (status == PGRES_COPY_IN){ gvl_PQputCopyEnd(conn, "COPY terminated by new PQexec"); From cacf429fa9fb6164e75d70ec7c765ade53de1e50 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Mon, 13 Sep 2021 10:27:24 +0200 Subject: [PATCH 41/62] Bump required ruby version to 2.5 since we broke 2.4 again (by Hash#transform_keys) --- .appveyor.yml | 2 +- .github/workflows/binary-gems.yml | 2 +- .github/workflows/source-gem.yml | 4 ++-- .travis.yml | 2 +- Rakefile.cross | 2 +- lib/pg/connection.rb | 3 +-- pg.gemspec | 2 +- 7 files changed, 8 insertions(+), 9 deletions(-) diff --git a/.appveyor.yml b/.appveyor.yml index 5b6807943..ed5022519 100644 --- a/.appveyor.yml +++ b/.appveyor.yml @@ -31,6 +31,6 @@ environment: RUBYDOWNLOAD: x86 PGVERSION: 10.16-1-windows PGVER: 10 - - ruby_version: "24" + - ruby_version: "25" PGVERSION: 9.3.25-1-windows PGVER: 9.3 diff --git a/.github/workflows/binary-gems.yml b/.github/workflows/binary-gems.yml index cba4eceaf..b86b99970 100644 --- a/.github/workflows/binary-gems.yml +++ b/.github/workflows/binary-gems.yml @@ -40,7 +40,7 @@ jobs: - ruby: "3.0" PGVERSION: 13.2-1-windows-x64 PGVER: "13" - - ruby: "2.4" + - ruby: "2.5" PGVERSION: 10.16-1-windows PGVER: "10" diff --git a/.github/workflows/source-gem.yml b/.github/workflows/source-gem.yml index c9e876a25..a99a0a80c 100644 --- a/.github/workflows/source-gem.yml +++ b/.github/workflows/source-gem.yml @@ -34,7 +34,7 @@ jobs: PGVERSION: 13.2-1-windows-x64 PGVER: "13" - os: windows - ruby: "2.4" + ruby: "2.5" PGVERSION: 9.3.25-1-windows-x64 PGVER: "9.3" - os: ubuntu @@ -44,7 +44,7 @@ jobs: ruby: "3.0" PGVER: "12" - os: ubuntu - ruby: "2.4" + ruby: "2.5" PGVER: "9.3" - os: ubuntu ruby: "truffleruby" diff --git a/.travis.yml b/.travis.yml index bfa7756c6..5340558b8 100644 --- a/.travis.yml +++ b/.travis.yml @@ -17,7 +17,7 @@ matrix: script: | docker run --rm -t --network=host ruby-pg - - rvm: "2.4" + - rvm: "2.5" env: - "PGVERSION=9.3" - rvm: ruby-head diff --git a/Rakefile.cross b/Rakefile.cross index 9e9edfa82..c3c7e8cf9 100644 --- a/Rakefile.cross +++ b/Rakefile.cross @@ -291,7 +291,7 @@ CrossLibraries.each do |xlib| RakeCompilerDock.sh <<-EOT, platform: platform (cp build/gem/gem-*.pem ~/.gem/ || true) && bundle install --local && - rake native:#{platform} pkg/#{$gem_spec.full_name}-#{platform}.gem MAKE="make -j`nproc`" + rake native:#{platform} pkg/#{$gem_spec.full_name}-#{platform}.gem MAKE="make -j`nproc`" RUBY_CC_VERSION=3.0.0:2.7.0:2.6.0:2.5.0 EOT end desc "Build the windows binary gems" diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index b2abfe643..1b3286e5d 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -485,8 +485,7 @@ def async_api=(enable) end REDIRECT_CLASS_METHODS.each do |ali, (async, sync)| singleton_class.remove_method(ali) if method_defined?(ali) - # TODO: send is necessary for ruby < 2.5 - singleton_class.send(:alias_method, ali, enable ? async : sync ) + singleton_class.alias_method(ali, enable ? async : sync ) end end end diff --git a/pg.gemspec b/pg.gemspec index 0948c0dd8..c7212b60b 100644 --- a/pg.gemspec +++ b/pg.gemspec @@ -13,7 +13,7 @@ Gem::Specification.new do |spec| spec.description = "Pg is the Ruby interface to the PostgreSQL RDBMS. It works with PostgreSQL 9.3 and later." spec.homepage = "https://github.com/ged/ruby-pg" spec.license = "BSD-2-Clause" - spec.required_ruby_version = ">= 2.4" + spec.required_ruby_version = ">= 2.5" spec.metadata["homepage_uri"] = spec.homepage spec.metadata["source_code_uri"] = "https://github.com/ged/ruby-pg" From c324d0b5f94104d25494152e27fb3ed34ea3183d Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Mon, 13 Sep 2021 10:55:42 +0200 Subject: [PATCH 42/62] Add test case for invalid argument to #connect_string_to_hash and gather tests into "describe" blocks. --- lib/pg/connection.rb | 2 +- spec/pg/connection_spec.rb | 330 +++++++++++++++++++------------------ 2 files changed, 172 insertions(+), 160 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index 1b3286e5d..34b6ff873 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -54,7 +54,7 @@ def self.connect_string_to_hash( str ) value << (word || (sq || esc).gsub(/\\(.)/, '\\1')) end if sep - raise "missing = after #{value.inspect}" unless key + raise ArgumentError, "missing = after #{value.inspect}" unless key options[key.to_sym] = value key = nil value = String.new diff --git a/spec/pg/connection_spec.rb b/spec/pg/connection_spec.rb index e380a4d65..0c1bc568d 100644 --- a/spec/pg/connection_spec.rb +++ b/spec/pg/connection_spec.rb @@ -14,175 +14,210 @@ expect( ObjectSpace.memsize_of(@conn) ).to be > DATA_OBJ_MEMSIZE end - it "encode and decode Hash to connection string to Hash" do - hash = { - :host => 'pgsql.example.com', - :dbname => 'db01', - 'sslmode' => 'require', - 'somekey' => '', - 'password' => "\\ \t\n\"'", - } - optstring = described_class.connect_hash_to_string(hash) - res = described_class.connect_string_to_hash(optstring) + describe "PG::Connection#connect_string_to_hash" do + it "encode and decode Hash to connection string to Hash" do + hash = { + :host => 'pgsql.example.com', + :dbname => 'db01', + 'sslmode' => 'require', + 'somekey' => '', + 'password' => "\\ \t\n\"'", + } + optstring = described_class.connect_hash_to_string(hash) + res = described_class.connect_string_to_hash(optstring) - expect( res ).to eq( hash.transform_keys(&:to_sym) ) - end + expect( res ).to eq( hash.transform_keys(&:to_sym) ) + end - it "decode option string to Hash" do - optstring = "host=c:\\\\pipe password = \\\\\\'\" " - res = described_class.connect_string_to_hash(optstring) + it "decode option string to Hash" do + optstring = "host=overwritten host=c:\\\\pipe password = \\\\\\'\" " + res = described_class.connect_string_to_hash(optstring) - expect( res ).to eq({ - host: 'c:\pipe', - password: "\\'\"", - }) - end + expect( res ).to eq({ + host: 'c:\pipe', + password: "\\'\"", + }) + end - it "shouldn't resolve absolute path in connection option string" do - optstring = described_class.parse_connect_args( - :host => '/var/socket' - ) - expect( optstring ).to match( /(^|\s)host='\/var\/socket'/ ) - expect( optstring ).not_to match( /hostaddr/ ) - end + it "raises error when decoding invalid option string" do + optstring = "host='abc" + expect{ described_class.connect_string_to_hash(optstring) }.to raise_error(ArgumentError, /unterminated quoted string/) - it "shouldn't resolve Windows path in connection option string", :windows do - optstring = described_class.parse_connect_args( - :host => "C:\\pipe\\00" - ) - expect( optstring ).to match( /(^|\s)host='C:\\\\pipe\\\\00'/ ) - expect( optstring ).not_to match( /hostaddr/ ) + optstring = "host" + expect{ described_class.connect_string_to_hash(optstring) }.to raise_error(ArgumentError, /missing = after/) + end end - it "can create a connection option string from a Hash parameter" do - optstring = described_class.parse_connect_args( - :host => 'pgsql.example.com', - :dbname => 'db01', - 'sslmode' => 'require', - 'hostaddr' => '1.2.3.4' - ) + describe "PG::Connection#parse_connect_args" do + it "shouldn't resolve absolute path in connection option string" do + optstring = described_class.parse_connect_args( + :host => '/var/socket' + ) + expect( optstring ).to match( /(^|\s)host='\/var\/socket'/ ) + expect( optstring ).not_to match( /hostaddr/ ) + end - expect( optstring ).to be_a( String ) - expect( optstring ).to match( /(^|\s)host='pgsql.example.com'/ ) - expect( optstring ).to match( /(^|\s)dbname='db01'/ ) - expect( optstring ).to match( /(^|\s)sslmode='require'/ ) - expect( optstring ).to match( /(^|\s)hostaddr='1.2.3.4'/ ) - end + it "shouldn't resolve Windows path in connection option string", :windows do + optstring = described_class.parse_connect_args( + :host => "C:\\pipe\\00" + ) + expect( optstring ).to match( /(^|\s)host='C:\\\\pipe\\\\00'/ ) + expect( optstring ).not_to match( /hostaddr/ ) + end - it "can create a connection option string from positional parameters" do - optstring = described_class.parse_connect_args( 'localhost', nil, '-c geqo=off', nil, - 'sales' ) + it "can create a connection option string from a Hash parameter" do + optstring = described_class.parse_connect_args( + :host => 'pgsql.example.com', + :dbname => 'db01', + 'sslmode' => 'require', + 'hostaddr' => '1.2.3.4' + ) - expect( optstring ).to be_a( String ) - expect( optstring ).to match( /(^|\s)host='localhost'/ ) - expect( optstring ).to match( /(^|\s)dbname='sales'/ ) - expect( optstring ).to match( /(^|\s)options='-c geqo=off'/ ) - expect( optstring ).to match( /(^|\s)hostaddr='(::1|127.0.0.1)'/ ) + expect( optstring ).to be_a( String ) + expect( optstring ).to match( /(^|\s)host='pgsql.example.com'/ ) + expect( optstring ).to match( /(^|\s)dbname='db01'/ ) + expect( optstring ).to match( /(^|\s)sslmode='require'/ ) + expect( optstring ).to match( /(^|\s)hostaddr='1.2.3.4'/ ) + end - expect( optstring ).to_not match( /port=/ ) - expect( optstring ).to_not match( /tty=/ ) - end + it "can create a connection option string from positional parameters" do + optstring = described_class.parse_connect_args( 'localhost', nil, '-c geqo=off', nil, + 'sales' ) - it "can create a connection option string from a mix of positional and hash parameters" do - optstring = described_class.parse_connect_args( 'pgsql.example.com', - :dbname => 'licensing', :user => 'jrandom', - 'hostaddr' => '1.2.3.4' ) + expect( optstring ).to be_a( String ) + expect( optstring ).to match( /(^|\s)host='localhost'/ ) + expect( optstring ).to match( /(^|\s)dbname='sales'/ ) + expect( optstring ).to match( /(^|\s)options='-c geqo=off'/ ) + expect( optstring ).to match( /(^|\s)hostaddr='(::1|127.0.0.1)'/ ) - expect( optstring ).to be_a( String ) - expect( optstring ).to match( /(^|\s)host='pgsql.example.com'/ ) - expect( optstring ).to match( /(^|\s)dbname='licensing'/ ) - expect( optstring ).to match( /(^|\s)user='jrandom'/ ) - expect( optstring ).to match( /(^|\s)hostaddr='1.2.3.4'/ ) - end + expect( optstring ).to_not match( /port=/ ) + expect( optstring ).to_not match( /tty=/ ) + end - it "can create a connection option string from an option string and a hash" do - optstring = described_class.parse_connect_args( 'dbname=original', :user => 'jrandom', 'host' => 'localhost' ) + it "can create a connection option string from a mix of positional and hash parameters" do + optstring = described_class.parse_connect_args( 'pgsql.example.com', + :dbname => 'licensing', :user => 'jrandom', + 'hostaddr' => '1.2.3.4' ) - expect( optstring ).to be_a( String ) - expect( optstring ).to match( /(^|\s)dbname=original/ ) - expect( optstring ).to match( /(^|\s)user='jrandom'/ ) - expect( optstring ).to match( /(^|\s)hostaddr='(::1|127.0.0.1)'/ ) - end + expect( optstring ).to be_a( String ) + expect( optstring ).to match( /(^|\s)host='pgsql.example.com'/ ) + expect( optstring ).to match( /(^|\s)dbname='licensing'/ ) + expect( optstring ).to match( /(^|\s)user='jrandom'/ ) + expect( optstring ).to match( /(^|\s)hostaddr='1.2.3.4'/ ) + end - it "escapes single quotes and backslashes in connection parameters" do - expect( - described_class.parse_connect_args( password: "DB 'browser' \\" ) - ).to match( /password='DB \\'browser\\' \\\\'/ ) - end + it "can create a connection option string from an option string and a hash" do + optstring = described_class.parse_connect_args( 'dbname=original', :user => 'jrandom', 'host' => 'localhost' ) - let(:uri) { 'postgresql://user:pass@pgsql.example.com:222/db01?sslmode=require&hostaddr=4.3.2.1' } + expect( optstring ).to be_a( String ) + expect( optstring ).to match( /(^|\s)dbname=original/ ) + expect( optstring ).to match( /(^|\s)user='jrandom'/ ) + expect( optstring ).to match( /(^|\s)hostaddr='(::1|127.0.0.1)'/ ) + end - it "can connect using a URI" do - string = described_class.parse_connect_args( uri ) + it "escapes single quotes and backslashes in connection parameters" do + expect( + described_class.parse_connect_args( password: "DB 'browser' \\" ) + ).to match( /password='DB \\'browser\\' \\\\'/ ) + end - expect( string ).to be_a( String ) - expect( string ).to match( %r{^postgresql://user:pass@pgsql.example.com:222/db01\?} ) - expect( string ).to match( %r{\?.*sslmode=require} ) + let(:uri) { 'postgresql://user:pass@pgsql.example.com:222/db01?sslmode=require&hostaddr=4.3.2.1' } - string = described_class.parse_connect_args( URI.parse(uri) ) + it "can connect using a URI" do + string = described_class.parse_connect_args( uri ) - expect( string ).to be_a( String ) - expect( string ).to match( %r{^postgresql://user:pass@pgsql.example.com:222/db01\?} ) - expect( string ).to match( %r{\?.*sslmode=require} ) - end + expect( string ).to be_a( String ) + expect( string ).to match( %r{^postgresql://user:pass@pgsql.example.com:222/db01\?} ) + expect( string ).to match( %r{\?.*sslmode=require} ) - it "can create a connection URI from a URI and a hash" do - string = described_class.parse_connect_args( uri, :connect_timeout => 2 ) + string = described_class.parse_connect_args( URI.parse(uri) ) - expect( string ).to be_a( String ) - expect( string ).to match( %r{^postgresql://user:pass@pgsql.example.com:222/db01\?} ) - expect( string ).to match( %r{\?.*sslmode=require} ) - expect( string ).to match( %r{\?.*connect_timeout=2} ) - end + expect( string ).to be_a( String ) + expect( string ).to match( %r{^postgresql://user:pass@pgsql.example.com:222/db01\?} ) + expect( string ).to match( %r{\?.*sslmode=require} ) + end - it "can create a connection URI from a URI and prefers the hash" do - string = described_class.parse_connect_args( uri, - :user => 'a', - :password => 'b', - :host => 'localhost', - :port => 555, - :dbname => 'x' ) + it "can create a connection URI from a URI and a hash" do + string = described_class.parse_connect_args( uri, :connect_timeout => 2 ) - expect( string ).to be_a( String ) - expect( string ).to match( %r{^postgresql://\?} ) - expect( string ).to match( %r{\?.*user=a} ) - expect( string ).to match( %r{\?.*password=b} ) - expect( string ).to match( %r{\?.*host=localhost} ) - expect( string ).to match( %r{\?.*port=555} ) - expect( string ).to match( %r{\?.*dbname=x} ) - expect( string ).to match( %r{\?.*hostaddr=(%3A%3A1|127.0.0.1)} ) - end + expect( string ).to be_a( String ) + expect( string ).to match( %r{^postgresql://user:pass@pgsql.example.com:222/db01\?} ) + expect( string ).to match( %r{\?.*sslmode=require} ) + expect( string ).to match( %r{\?.*connect_timeout=2} ) + end - it "can create a connection URI with a non-standard domain socket directory" do - string = described_class.parse_connect_args( 'postgresql://%2Fvar%2Flib%2Fpostgresql/dbname' ) + it "can create a connection URI from a URI and prefers the hash" do + string = described_class.parse_connect_args( uri, + :user => 'a', + :password => 'b', + :host => 'localhost', + :port => 555, + :dbname => 'x' ) - expect( string ).to be_a( String ) - expect( string ).to match( %r{^postgresql://%2Fvar%2Flib%2Fpostgresql/dbname} ) + expect( string ).to be_a( String ) + expect( string ).to match( %r{^postgresql://\?} ) + expect( string ).to match( %r{\?.*user=a} ) + expect( string ).to match( %r{\?.*password=b} ) + expect( string ).to match( %r{\?.*host=localhost} ) + expect( string ).to match( %r{\?.*port=555} ) + expect( string ).to match( %r{\?.*dbname=x} ) + expect( string ).to match( %r{\?.*hostaddr=(%3A%3A1|127.0.0.1)} ) + end - string = described_class. - parse_connect_args( 'postgresql:///dbname', :host => '/var/lib/postgresql' ) + it "can create a connection URI with a non-standard domain socket directory" do + string = described_class.parse_connect_args( 'postgresql://%2Fvar%2Flib%2Fpostgresql/dbname' ) - expect( string ).to be_a( String ) - expect( string ).to match( %r{^postgresql:///dbname\?} ) - expect( string ).to match( %r{\?.*host=%2Fvar%2Flib%2Fpostgresql} ) - end + expect( string ).to be_a( String ) + expect( string ).to match( %r{^postgresql://%2Fvar%2Flib%2Fpostgresql/dbname} ) - it "connects with defaults if no connection parameters are given" do - expect( described_class.parse_connect_args ).to match( /fallback_application_name='[^']+'/ ) - end + string = described_class. + parse_connect_args( 'postgresql:///dbname', :host => '/var/lib/postgresql' ) - it "connects successfully with connection string" do - conninfo_with_colon_in_password = "host=localhost user=a port=555 dbname=test password=a:a" + expect( string ).to be_a( String ) + expect( string ).to match( %r{^postgresql:///dbname\?} ) + expect( string ).to match( %r{\?.*host=%2Fvar%2Flib%2Fpostgresql} ) + end - string = described_class.parse_connect_args( conninfo_with_colon_in_password ) + it "connects with defaults if no connection parameters are given" do + expect( described_class.parse_connect_args ).to match( /fallback_application_name='[^']+'/ ) + end + + it "connects successfully with connection string" do + conninfo_with_colon_in_password = "host=localhost user=a port=555 dbname=test password=a:a" + + string = described_class.parse_connect_args( conninfo_with_colon_in_password ) + + expect( string ).to be_a( String ) + expect( string ).to match( %r{(^|\s)user=a} ) + expect( string ).to match( %r{(^|\s)password=a:a} ) + expect( string ).to match( %r{(^|\s)host=localhost} ) + expect( string ).to match( %r{(^|\s)port=555} ) + expect( string ).to match( %r{(^|\s)dbname=test} ) + expect( string ).to match( %r{(^|\s)hostaddr='(::1|127.0.0.1)'} ) + end + + it "sets the fallback_application_name on new connections" do + conn_string = PG::Connection.parse_connect_args( 'dbname=test' ) - expect( string ).to be_a( String ) - expect( string ).to match( %r{(^|\s)user=a} ) - expect( string ).to match( %r{(^|\s)password=a:a} ) - expect( string ).to match( %r{(^|\s)host=localhost} ) - expect( string ).to match( %r{(^|\s)port=555} ) - expect( string ).to match( %r{(^|\s)dbname=test} ) - expect( string ).to match( %r{(^|\s)hostaddr='(::1|127.0.0.1)'} ) + conn_name = conn_string[ /application_name='(.*?)'/, 1 ] + expect( conn_name ).to include( $0[0..10] ) + expect( conn_name ).to include( $0[-10..-1] ) + expect( conn_name.length ).to be <= 64 + end + + it "sets a shortened fallback_application_name on new connections" do + old_0 = $0 + begin + $0 = "/this/is/a/very/long/path/with/many/directories/to/our/beloved/ruby" + conn_string = PG::Connection.parse_connect_args( 'dbname=test' ) + conn_name = conn_string[ /application_name='(.*?)'/, 1 ] + expect( conn_name ).to include( $0[0..10] ) + expect( conn_name ).to include( $0[-10..-1] ) + expect( conn_name.length ).to be <= 64 + ensure + $0 = old_0 + end + end end it "connects successfully with connection string" do @@ -1247,29 +1282,6 @@ def interrupt_thread(exc=nil) expect{ conn.block }.to raise_error(PG::ConnectionBad, /can't get socket descriptor|connection not open/) end - it "sets the fallback_application_name on new connections" do - conn_string = PG::Connection.parse_connect_args( 'dbname=test' ) - - conn_name = conn_string[ /application_name='(.*?)'/, 1 ] - expect( conn_name ).to include( $0[0..10] ) - expect( conn_name ).to include( $0[-10..-1] ) - expect( conn_name.length ).to be <= 64 - end - - it "sets a shortened fallback_application_name on new connections" do - old_0 = $0 - begin - $0 = "/this/is/a/very/long/path/with/many/directories/to/our/beloved/ruby" - conn_string = PG::Connection.parse_connect_args( 'dbname=test' ) - conn_name = conn_string[ /application_name='(.*?)'/, 1 ] - expect( conn_name ).to include( $0[0..10] ) - expect( conn_name ).to include( $0[-10..-1] ) - expect( conn_name.length ).to be <= 64 - ensure - $0 = old_0 - end - end - it "calls the block supplied to wait_for_notify with the notify payload if it accepts " + "any number of arguments" do From 5e8c911cc4a55b5b2bc059b3ed0f340c033bc2cf Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Mon, 13 Sep 2021 16:09:40 +0200 Subject: [PATCH 43/62] Allow specification of multiple PostgreSQL hosts to connect to Unfortunately the uri gem from stdlib isn't compatible with postgresql://host1:port1,host2:port2 syntax. Therefore we introduce an extended regexp for parsing the URI. Previously we allowed changes to the URI by overwriting URI parts with hash values. This is no longer possible and this is a specification change. But I think it's bad practice and a questionable feature. Now only additions to the passed URI or connection string are possible. That way hostaddr and fallback_application_name are automatically appended. This patch also expands address resolution for multiple hosts. Fixes #387 --- ext/pg_connection.c | 5 +- lib/pg/connection.rb | 108 +++++++++++++++++++++++-------------- spec/pg/connection_spec.rb | 57 +++++++++++--------- 3 files changed, 105 insertions(+), 65 deletions(-) diff --git a/ext/pg_connection.c b/ext/pg_connection.c index 4061ebfa0..745944015 100644 --- a/ext/pg_connection.c +++ b/ext/pg_connection.c @@ -286,7 +286,7 @@ pgconn_s_allocate( VALUE klass ) * PG::Connection.new * * # As a Hash - * PG::Connection.new( :dbname => 'test', :port => 5432 ) + * PG::Connection.new( dbname: 'test', port: 5432 ) * * # As a String * PG::Connection.new( "dbname=test port=5432" ) @@ -294,6 +294,9 @@ pgconn_s_allocate( VALUE klass ) * # As an Array * PG::Connection.new( nil, 5432, nil, nil, 'test', nil, nil ) * + * # As an URI + * PG::Connection.new( "postgresql://user:pass@pgsql.example.com:5432/testdb?sslmode=require" ) + * * If the Ruby default internal encoding is set (i.e., Encoding.default_internal != nil), the * connection will have its +client_encoding+ set accordingly. * diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index 34b6ff873..18ae4c242 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -63,71 +63,99 @@ def self.connect_string_to_hash( str ) options end - ### Parse the connection +args+ into a connection-parameter string. See PG::Connection.new - ### for valid arguments. - def self::parse_connect_args( *args ) - hash_arg = args.last.is_a?( Hash ) ? args.pop : {} - option_string = '' - options = {} + # URI defined in RFC3986 + # This regexp is modified to allow host to specify multiple comma separated components captured as + # Taken from: https://github.com/ruby/ruby/blob/be04006c7d2f9aeb7e9d8d09d945b3a9c7850202/lib/uri/rfc3986_parser.rb#L6 + POSTGRESQL_URI = /\A(?(?[A-Za-z][+\-.0-9A-Za-z]*):(?\/\/(?(?:(?(?:%\h\h|[!$&-.0-;=A-Z_a-z~])*)@)?(?(?(?\[(?:(?(?:\h{1,4}:){6}(?\h{1,4}:\h{1,4}|(?(?[1-9]\d|1\d{2}|2[0-4]\d|25[0-5]|\d)\.\g\.\g\.\g))|::(?:\h{1,4}:){5}\g|\h{1,4}?::(?:\h{1,4}:){4}\g|(?:(?:\h{1,4}:)?\h{1,4})?::(?:\h{1,4}:){3}\g|(?:(?:\h{1,4}:){,2}\h{1,4})?::(?:\h{1,4}:){2}\g|(?:(?:\h{1,4}:){,3}\h{1,4})?::\h{1,4}:\g|(?:(?:\h{1,4}:){,4}\h{1,4})?::\g|(?:(?:\h{1,4}:){,5}\h{1,4})?::\h{1,4}|(?:(?:\h{1,4}:){,6}\h{1,4})?::)|(?v\h+\.[!$&-.0-;=A-Z_a-z~]+))\])|\g|(?(?:%\h\h|[!$&-.0-9;=A-Z_a-z~])+))?(?::(?\d*))?(?:,\g(?::\g)?)*))(?(?:\/(?(?:%\h\h|[!$&-.0-;=@-Z_a-z~])*))*)|(?\/(?:(?(?:%\h\h|[!$&-.0-;=@-Z_a-z~])+)(?:\/\g)*)?)|(?\g(?:\/\g)*)|(?))(?:\?(?[^#]*))?(?:\#(?(?:%\h\h|[!$&-.0-;=@-Z_a-z~\/?])*))?)\z/ - options[:fallback_application_name] = $0.sub( /^(.{30}).{4,}(.{30})$/ ){ $1+"..."+$2 } + # Parse the connection +args+ into a connection-parameter string. + # See PG::Connection.new for valid arguments. + # + # It accepts: + # * an option String kind of "host=name port=5432" + # * an option Hash kind of {host: "name", port: 5432} + # * URI string + # * URI object + # * positional arguments + # + # The method adds the option "hostaddr" and "fallback_application_name" if they aren't already set. + # The URI and the options string is passed through and "hostaddr" as well as "fallback_application_name" + # are added to the end. + def self::parse_connect_args( *args ) + hash_arg = args.last.is_a?( Hash ) ? args.pop.transform_keys(&:to_sym) : {} + option_string = "" + iopts = {} if args.length == 1 case args.first - when URI, /\A#{URI::ABS_URI_REF}\z/ - uri = URI(args.first) - options.merge!( Hash[URI.decode_www_form( uri.query )] ) if uri.query + when URI, POSTGRESQL_URI + uri = args.first.to_s + uri_match = POSTGRESQL_URI.match(uri) + if uri_match['query'] + iopts = URI.decode_www_form(uri_match['query']).to_h.transform_keys(&:to_sym) + end + # extract "host1,host2" from "host1:5432,host2:5432" + iopts[:host] = uri_match['hostports'].split(",").map { |hp| hp.split(":", 2)[0] }.join(",") + oopts = {} when /=/ # Option string style option_string = args.first.to_s + iopts = connect_string_to_hash(option_string) + oopts = {} else - # Positional parameters - options[CONNECT_ARGUMENT_ORDER.first.to_sym] = args.first + # Positional parameters (only host given) + iopts[CONNECT_ARGUMENT_ORDER.first.to_sym] = args.first + oopts = iopts.dup end else + # Positional parameters max = CONNECT_ARGUMENT_ORDER.length raise ArgumentError, "Extra positional parameter %d: %p" % [ max + 1, args[max] ] if args.length > max CONNECT_ARGUMENT_ORDER.zip( args ) do |(k,v)| - options[ k.to_sym ] = v if v + iopts[ k.to_sym ] = v if v end + oopts = iopts.dup end - options.merge!( hash_arg.transform_keys(&:to_sym) ) + iopts.merge!( hash_arg ) + oopts.merge!( hash_arg ) # Resolve DNS in Ruby to avoid blocking state while connecting, when it ... - if (host = options[:host] || connect_string_to_hash(option_string)[:host]) && - # isn't UnixSocket - !host.empty? && !host.start_with?("/") && - # isn't a path on Windows - (RUBY_PLATFORM !~ /mingw|mswin/ || host !~ /\A\w:[\/\\]/) - - options[:hostaddr] ||= if - Fiber.respond_to?(:scheduler) && - Fiber.scheduler && - RUBY_VERSION < '3.1.' - - # Use pure Ruby address resolver to avoid blocking of the scheduler. - # `IPSocket.getaddress` isn't fiber aware before ruby-3.1. - require "resolv" - Resolv.getaddress(host) - else - IPSocket.getaddress(host) + if (host=iopts[:host]) && !iopts[:hostaddr] + hostaddrs = host.split(",").map do |mhost| + if !mhost.empty? && !mhost.start_with?("/") && # isn't UnixSocket + # isn't a path on Windows + (RUBY_PLATFORM !~ /mingw|mswin/ || mhost !~ /\A\w:[\/\\]/) + + if Fiber.respond_to?(:scheduler) && + Fiber.scheduler && + RUBY_VERSION < '3.1.' + + # Use pure Ruby address resolver to avoid blocking of the scheduler. + # `IPSocket.getaddress` isn't fiber aware before ruby-3.1. + require "resolv" + Resolv.getaddress(mhost) rescue '' + else + IPSocket.getaddress(mhost) rescue '' + end + end end + oopts[:hostaddr] = hostaddrs.join(",") if hostaddrs.any? + end + + if !iopts[:fallback_application_name] + oopts[:fallback_application_name] = $0.sub( /^(.{30}).{4,}(.{30})$/ ){ $1+"..."+$2 } end if uri - uri.host = nil if options[:host] - uri.port = nil if options[:port] - uri.user = nil if options[:user] - uri.password = nil if options[:password] - uri.path = '' if options[:dbname] - uri.query = URI.encode_www_form( options ) - return uri.to_s.sub( /^#{uri.scheme}:(?!\/\/)/, "#{uri.scheme}://" ) + uri += uri_match['query'] ? "&" : "?" + uri += URI.encode_www_form( oopts ) + return uri else - option_string += ' ' unless option_string.empty? && options.empty? - return option_string + connect_hash_to_string(options) + option_string += ' ' unless option_string.empty? && oopts.empty? + return option_string + connect_hash_to_string(oopts) end end diff --git a/spec/pg/connection_spec.rb b/spec/pg/connection_spec.rb index 0c1bc568d..47917efec 100644 --- a/spec/pg/connection_spec.rb +++ b/spec/pg/connection_spec.rb @@ -96,7 +96,7 @@ it "can create a connection option string from a mix of positional and hash parameters" do optstring = described_class.parse_connect_args( 'pgsql.example.com', - :dbname => 'licensing', :user => 'jrandom', + :dbname => 'licensing', :user => 'jrandom', 'hostaddr' => '1.2.3.4' ) expect( optstring ).to be_a( String ) @@ -107,12 +107,13 @@ end it "can create a connection option string from an option string and a hash" do - optstring = described_class.parse_connect_args( 'dbname=original', :user => 'jrandom', 'host' => 'localhost' ) + optstring = described_class.parse_connect_args( 'dbname=original', :user => 'jrandom', + 'host' => 'www.ruby-lang.org,nonexisting-domaiiin.xyz,localhost' ) expect( optstring ).to be_a( String ) expect( optstring ).to match( /(^|\s)dbname=original/ ) expect( optstring ).to match( /(^|\s)user='jrandom'/ ) - expect( optstring ).to match( /(^|\s)hostaddr='(::1|127.0.0.1)'/ ) + expect( optstring ).to match( /(^|\s)hostaddr='\d+\.\d+\.\d+\.\d+,,(::1|127\.0\.0\.1)'/ ) end it "escapes single quotes and backslashes in connection parameters" do @@ -123,7 +124,7 @@ let(:uri) { 'postgresql://user:pass@pgsql.example.com:222/db01?sslmode=require&hostaddr=4.3.2.1' } - it "can connect using a URI" do + it "accepts an URI" do string = described_class.parse_connect_args( uri ) expect( string ).to be_a( String ) @@ -137,34 +138,25 @@ expect( string ).to match( %r{\?.*sslmode=require} ) end - it "can create a connection URI from a URI and a hash" do - string = described_class.parse_connect_args( uri, :connect_timeout => 2 ) + it "accepts an URI and adds parameters from hash" do + string = described_class.parse_connect_args( uri + "&fallback_application_name=testapp", :connect_timeout => 2 ) expect( string ).to be_a( String ) expect( string ).to match( %r{^postgresql://user:pass@pgsql.example.com:222/db01\?} ) - expect( string ).to match( %r{\?.*sslmode=require} ) - expect( string ).to match( %r{\?.*connect_timeout=2} ) + expect( string ).to match( %r{\?sslmode=require&} ) + expect( string ).to match( %r{\?.*&fallback_application_name=testapp&} ) + expect( string ).to match( %r{\?.*&connect_timeout=2$} ) end - it "can create a connection URI from a URI and prefers the hash" do - string = described_class.parse_connect_args( uri, - :user => 'a', - :password => 'b', - :host => 'localhost', - :port => 555, - :dbname => 'x' ) + it "accepts an URI and adds hostaddr" do + uri = 'postgresql://www.ruby-lang.org,nonexisting-domaiiin.xyz,localhost' + string = described_class.parse_connect_args( uri ) expect( string ).to be_a( String ) - expect( string ).to match( %r{^postgresql://\?} ) - expect( string ).to match( %r{\?.*user=a} ) - expect( string ).to match( %r{\?.*password=b} ) - expect( string ).to match( %r{\?.*host=localhost} ) - expect( string ).to match( %r{\?.*port=555} ) - expect( string ).to match( %r{\?.*dbname=x} ) - expect( string ).to match( %r{\?.*hostaddr=(%3A%3A1|127.0.0.1)} ) + expect( string ).to match( %r{^postgresql://www.ruby-lang.org,nonexisting-domaiiin.xyz,localhost\?hostaddr=\d+\.\d+\.\d+\.\d+%2C%2C(%3A%3A1|127\.0\.0\.1)} ) end - it "can create a connection URI with a non-standard domain socket directory" do + it "accepts an URI with a non-standard domain socket directory" do string = described_class.parse_connect_args( 'postgresql://%2Fvar%2Flib%2Fpostgresql/dbname' ) expect( string ).to be_a( String ) @@ -193,7 +185,7 @@ expect( string ).to match( %r{(^|\s)host=localhost} ) expect( string ).to match( %r{(^|\s)port=555} ) expect( string ).to match( %r{(^|\s)dbname=test} ) - expect( string ).to match( %r{(^|\s)hostaddr='(::1|127.0.0.1)'} ) + expect( string ).to match( %r{(^|\s)hostaddr='(::1|127\.0\.0\.1)'} ) end it "sets the fallback_application_name on new connections" do @@ -262,6 +254,23 @@ end end + it "connects using URI with multiple hosts", :postgresql_10 do + uri = "postgres://localhost:#{@port},127.0.0.1:#{@port}/test?keepalives=1" + tmpconn = described_class.connect( uri ) + expect( tmpconn.status ).to eq( PG::CONNECTION_OK ) + expect( tmpconn.conninfo_hash[:host] ).to eq( "localhost,127.0.0.1" ) + expect( tmpconn.conninfo_hash[:hostaddr] ).to match( /\A(::1|127\.0\.0\.1),(::1|127\.0\.0\.1)\z/ ) + tmpconn.finish + end + + it "connects using Hash with multiple hosts", :postgresql_10 do + tmpconn = described_class.connect( host: "127.0.0.1,localhost", port: @port, dbname: "test" ) + expect( tmpconn.status ).to eq( PG::CONNECTION_OK ) + expect( tmpconn.conninfo_hash[:host] ).to eq( "127.0.0.1,localhost" ) + expect( tmpconn.conninfo_hash[:hostaddr] ).to match( /\A(::1|127\.0\.0\.1),(::1|127\.0\.0\.1)\z/ ) + tmpconn.finish + end + it "can connect asynchronously" do tmpconn = described_class.connect_start( @conninfo ) expect( tmpconn ).to be_a( described_class ) From 238ddd2e10c5e4b531ff4b522a946dc81550632f Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Tue, 14 Sep 2021 12:57:05 +0200 Subject: [PATCH 44/62] Add async_cancel as a nonblocking version of conn#cancel There's no nonblocking version of PQcancel(), so that we have to implement our own, to make it scheduler compatible. There's PQbackendPID() to retrieve the necessary process ID from libpq, but unfortunately there is no public function to retrieve the cancel key. Therefore we mimic the C struct pg_cancel of libpq in order to access the key from the opaque pointer. It is stable since 17 years, so that it's not expected to change soon. "be_key" is behind "salen" in "struct SockAddr", which has a size that depend on the outcome of the PostgreSQL configure script. We define it as "socklen_t" for now. Let's see how well this works on different platforms. To implement our own cancel method, the new method conn.backend_key is added. And this changes conn.socket_io to a BasicSocket instead of an IO. This allows to call methods like remote_address without any "for_fd" re-wrapping. --- ext/pg.h | 1 + ext/pg_connection.c | 50 ++++++++++++++++++++++++++++++++++++++- lib/pg/connection.rb | 27 +++++++++++++++++++++ spec/pg/scheduler_spec.rb | 8 +++++++ 4 files changed, 85 insertions(+), 1 deletion(-) diff --git a/ext/pg.h b/ext/pg.h index c41afbfd4..8fdf5bff2 100644 --- a/ext/pg.h +++ b/ext/pg.h @@ -11,6 +11,7 @@ #include #if !defined(_WIN32) # include +# include #endif #if defined(HAVE_UNISTD_H) && !defined(_WIN32) # include diff --git a/ext/pg_connection.c b/ext/pg_connection.c index 745944015..2a9676b38 100644 --- a/ext/pg_connection.c +++ b/ext/pg_connection.c @@ -922,6 +922,7 @@ pgconn_socket_io(VALUE self) int sd; int ruby_sd; t_pg_connection *this = pg_get_connection_safe( self ); + VALUE cSocket; VALUE socket_io = this->socket_io; if ( !RTEST(socket_io) ) { @@ -935,7 +936,8 @@ pgconn_socket_io(VALUE self) ruby_sd = sd; #endif - socket_io = rb_funcall( rb_cIO, rb_intern("for_fd"), 2, INT2NUM(ruby_sd), INT2NUM(2 /* File::RDWR */) ); + cSocket = rb_const_get(rb_cObject, rb_intern("BasicSocket")); + socket_io = rb_funcall( cSocket, rb_intern("for_fd"), 1, INT2NUM(ruby_sd)); /* Disable autoclose feature */ rb_funcall( socket_io, s_id_autoclose_set, 1, Qfalse ); @@ -960,6 +962,51 @@ pgconn_backend_pid(VALUE self) return INT2NUM(PQbackendPID(pg_get_pgconn(self))); } +typedef struct +{ + struct sockaddr_storage addr; + socklen_t salen; +} SockAddr; + +/* Copy of struct pg_cancel from libpq-int.h + * + * See https://github.com/postgres/postgres/blame/master/src/interfaces/libpq/libpq-int.h#L577-L586 + */ +struct pg_cancel +{ + SockAddr raddr; /* Remote address */ + int be_pid; /* PID of backend --- needed for cancels */ + int be_key; /* key of backend --- needed for cancels */ +}; + +/* + * call-seq: + * conn.backend_key() -> Integer + * + * Returns the key of the backend server process for this connection. + * This key can be used to cancel queries on the server. + */ +static VALUE +pgconn_backend_key(VALUE self) +{ + int be_key; + struct pg_cancel *cancel; + PGconn *conn = pg_get_pgconn(self); + + cancel = (struct pg_cancel*)PQgetCancel(conn); + if(cancel == NULL) + rb_raise(rb_ePGerror,"Invalid connection!"); + + if( cancel->be_pid != PQbackendPID(conn) ) + rb_raise(rb_ePGerror,"Unexpected binary struct layout - please file a bug report at ruby-pg!"); + + be_key = cancel->be_key; + + PQfreeCancel(cancel); + + return INT2NUM(be_key); +} + /* * call-seq: * conn.connection_needs_password() -> Boolean @@ -4227,6 +4274,7 @@ init_pg_connection() rb_define_method(rb_cPGconn, "socket", pgconn_socket, 0); rb_define_method(rb_cPGconn, "socket_io", pgconn_socket_io, 0); rb_define_method(rb_cPGconn, "backend_pid", pgconn_backend_pid, 0); + rb_define_method(rb_cPGconn, "backend_key", pgconn_backend_key, 0); rb_define_method(rb_cPGconn, "connection_needs_password", pgconn_connection_needs_password, 0); rb_define_method(rb_cPGconn, "connection_used_password", pgconn_connection_used_password, 0); /* rb_define_method(rb_cPGconn, "getssl", pgconn_getssl, 0); */ diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index 18ae4c242..dd22a2612 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -4,6 +4,7 @@ require 'pg' unless defined?( PG ) require 'uri' require 'io/wait' +require 'socket' # The PostgreSQL connection class. The interface for this class is based on # {libpq}[http://www.postgresql.org/docs/9.2/interactive/libpq.html], the C @@ -425,6 +426,31 @@ def async_reset async_connect_reset(:reset_poll) end + alias sync_cancel cancel + def async_cancel + cl = socket_io.remote_address.connect + be_pid = backend_pid + be_key = backend_key + # Send CANCEL_REQUEST_CODE and parameters + cl.write([0x10, 1234, 5678, be_pid, be_key].pack("NnnNN")) + + # Wait for the postmaster to close the connection, which indicates that it's processed the request. + # cl.read should be enough, but read isn't scheduler compatible on Windows. + # Work around by using read_nonblock. + begin + cl.read_nonblock(1) + rescue IO::WaitReadable, Errno::EINTR + cl.wait_readable + retry + rescue EOFError + end + + cl.close + nil + rescue SystemCallError => err + err.to_s + end + private def async_connect_reset(poll_meth) # Now grab a reference to the underlying socket so we know when the connection is established socket = socket_io @@ -496,6 +522,7 @@ def async_connect(*args, **kwargs) :reset => [:async_reset, :sync_reset], :set_client_encoding => [:async_set_client_encoding, :sync_set_client_encoding], :client_encoding= => [:async_set_client_encoding, :sync_set_client_encoding], + :cancel => [:async_cancel, :sync_cancel], } def async_send_api=(enable) diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index b34fa7936..38ff77899 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -249,6 +249,14 @@ def run_with_scheduler(timeout=10) end end end + + it "can cancel a query" do + run_with_scheduler do |conn| + conn.send_query "SELECT pg_sleep(5)" + conn.cancel + expect{ conn.get_last_result }.to raise_error(PG::QueryCanceled) + end + end end # Do not wait for threads doing blocking calls at the process shutdown. From 5b10f80ad95415d0ac97a059decdb2f2d5576077 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 15 Sep 2021 16:52:57 +0200 Subject: [PATCH 45/62] better method name async_connect_or_reset --- lib/pg/connection.rb | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index dd22a2612..a6f4ba56c 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -423,7 +423,7 @@ def async_put_copy_end(*args) alias sync_reset reset def async_reset reset_start - async_connect_reset(:reset_poll) + async_connect_or_reset(:reset_poll) end alias sync_cancel cancel @@ -451,7 +451,7 @@ def async_cancel err.to_s end - private def async_connect_reset(poll_meth) + private def async_connect_or_reset(poll_meth) # Now grab a reference to the underlying socket so we know when the connection is established socket = socket_io @@ -491,9 +491,10 @@ class << self def async_connect(*args, **kwargs) conn = PG::Connection.connect_start(*args, **kwargs ) or raise(PG::Error, "Unable to create a new connection") + raise(PG::ConnectionBad, conn.error_message) if conn.status == PG::CONNECTION_BAD - conn.send(:async_connect_reset, :connect_poll) + conn.send(:async_connect_or_reset, :connect_poll) end REDIRECT_CLASS_METHODS = { From 004235fc078ac5487d959efc9e4552423ae5fd2e Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 15 Sep 2021 16:55:32 +0200 Subject: [PATCH 46/62] Improve TcpGateScheduler - compare local_address and remote_address, since only one is ambitious - better logging --- lib/pg/connection.rb | 2 +- spec/helpers/scheduler.rb | 8 +--- spec/helpers/tcp_gate_scheduler.rb | 59 ++++++++++++++++++++++-------- 3 files changed, 45 insertions(+), 24 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index a6f4ba56c..34f81e512 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -435,7 +435,7 @@ def async_cancel cl.write([0x10, 1234, 5678, be_pid, be_key].pack("NnnNN")) # Wait for the postmaster to close the connection, which indicates that it's processed the request. - # cl.read should be enough, but read isn't scheduler compatible on Windows. + # cl.read(1) should be enough, but read isn't scheduler compatible on Windows. # Work around by using read_nonblock. begin cl.read_nonblock(1) diff --git a/spec/helpers/scheduler.rb b/spec/helpers/scheduler.rb index a58946877..5f4578323 100644 --- a/spec/helpers/scheduler.rb +++ b/spec/helpers/scheduler.rb @@ -39,13 +39,7 @@ def next_timeout _fiber, timeout = @waiting.min_by{|key, value| value} if timeout - offset = timeout - current_time - - if offset < 0 - return 0 - else - return offset - end + [0, timeout - current_time].max end end diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index 146a20959..6119d9c19 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -22,6 +22,7 @@ class TcpGateScheduler < Scheduler class Connection attr_reader :internal_io attr_reader :external_io + attr_accessor :observed_fd def initialize(internal_io, external_host, external_port, debug: false) @internal_io = internal_io @@ -34,6 +35,13 @@ def initialize(internal_io, external_host, external_port, debug: false) @debug = debug end + def other_side_of?(local_address, remote_address) + internal_io.local_address.to_s == remote_address.to_s && internal_io.remote_address.to_s == local_address.to_s + rescue Errno::ENOTCONN + # internal_io.remote_address fails, if connection is already half closed + false + end + def print_data(desc, data) return unless @debug if data.bytesize >= 70 @@ -57,7 +65,7 @@ def connect @external_io = TCPSocket.new(@external_host, @external_port) @pending_connect.close @pending_connect = false - puts "connected to external: #{@external_io.inspect}" + puts "connected ext:#{@external_io.inspect} (belongs to int:#{@internal_io.fileno})" else # connection is being established -> wait for it before doing read/write @pending_connect.pop @@ -83,13 +91,13 @@ def read( transfer_until: ) begin begin read_str = @external_io.read_nonblock(1000) - print_data("read fd:#{@external_io.fileno}->#{@internal_io.fileno}", read_str) + print_data("read-transfer #{read_fds}", read_str) @internal_io.write(read_str) rescue IO::WaitReadable, Errno::EINTR @external_io.wait_readable retry rescue EOFError, Errno::ECONNRESET - puts "read_eof from fd:#{@external_io.fileno}" + puts "read_eof from #{read_fds}" @internal_io.close_write break end @@ -105,7 +113,7 @@ def read( transfer_until: ) # # Option `transfer_until` can be (higher to lower priority): # :eof => transfer until channel is closed - # :nodata => transfer until no immediate data is available + # :wouldblock => transfer until no immediate data is available # IO object => transfer until IO is writeable # # The method does nothing if a transfer is already pending, but might raise the transfer_until option, if the requested priority is higher than the pending transfer. @@ -115,7 +123,6 @@ def write( transfer_until: ) @transfer_until = transfer_until Fiber.schedule do - puts "start write #{@transfer_until.is_a?(IO) ? "until #{@transfer_until.inspect} is writeable" : "until #{@transfer_until}"}" connect # transfer data blocks of up to 65536 bytes @@ -125,7 +132,7 @@ def write( transfer_until: ) len = 65536 begin read_str = @internal_io.read_nonblock(len) - print_data("write fd:#{@internal_io.fileno}->#{@external_io.fileno}", read_str) + print_data("write-transfer #{write_fds}", read_str) sleep 0 @external_io.write(read_str) if @transfer_until.is_a?(IO) @@ -140,7 +147,7 @@ def write( transfer_until: ) @internal_io.wait_readable retry rescue EOFError, Errno::ECONNRESET - puts "write_eof from fd:#{@internal_io.fileno}" + puts "write_eof from #{write_fds}" @external_io.close_write break end @@ -150,7 +157,7 @@ def write( transfer_until: ) @pending_write = false end - elsif (transfer_until == :nodata && @transfer_until.is_a?(IO)) || + elsif (transfer_until == :wouldblock && @transfer_until.is_a?(IO)) || transfer_until == :eof # If a write request without stopping on writablility comes in, # make sure, that the pending transfer doesn't abort prematurely. @@ -158,8 +165,17 @@ def write( transfer_until: ) end end + def read_fds + "ext:#{@external_io&.fileno || '-'}->int:#{@internal_io.fileno} obs:#{observed_fd}" + end + + def write_fds + "int:#{@internal_io.fileno}->ext:#{@external_io&.fileno || '-'} obs:#{observed_fd}" + end + # Make sure all data is transferred and both connections are closed. def finish + puts "finish transfers #{write_fds} and #{read_fds}" write transfer_until: :eof read transfer_until: :eof end @@ -174,6 +190,7 @@ def initialize(external_host:, external_port:, internal_host: 'localhost', inter @external_port = external_port @finish = false @debug = debug + puts "TcpGate server listening: #{@server_io.inspect}" end def finish @@ -196,6 +213,7 @@ def io_wait(io, events, duration) begin sock = TCPSocket.for_fd(io.fileno) sock.autoclose = false + local_address = sock.local_address remote_address = sock.remote_address rescue Errno::ENOTCONN, Errno::EINVAL end @@ -210,7 +228,7 @@ def io_wait(io, events, duration) break else conn = Connection.new(client, @external_host, @external_port, debug: @debug) - puts "accept new observed connection: #{conn.internal_io.inspect}" + puts "accept new int:#{conn.internal_io.inspect} from #{conn.internal_io.remote_address.inspect} server fd:#{@server_io.fileno}" @connections << conn end end @@ -225,9 +243,8 @@ def io_wait(io, events, duration) # Some IO call is waiting for data by rb_wait_for_single_fd() or so. # Is it on our intercepted IO? # Inspect latest connections first, since closed connections aren't removed immediately. - if cidx=@connections.rindex { |g| g.internal_io.local_address.to_s == remote_address.to_s } + if cidx=@connections.rindex { _1.other_side_of?(local_address, remote_address) } conn = @connections[cidx] - puts "trigger: fd:#{io.fileno} #{{addr: remote_address, events: events}}" # Success! Our observed client IO waits for some data to be readable or writable. # The IO function running on the observed IO did make proper use of some ruby wait function. # As a reward we provide some data to read or write. @@ -235,20 +252,30 @@ def io_wait(io, events, duration) # To the contrary: # If the blocking IO function doesn't make use of ruby wait functions, then it won't get any data and starve as a result. + # compare and store the fileno for debugging + if conn.observed_fd && conn.observed_fd != io.fileno + raise "observed fd changed: old:#{conn.observed_fd} new:#{io.fileno}" + end + conn.observed_fd = io.fileno + if (events & IO::WRITABLE) > 0 + puts "write-trigger from fd:#{io.fileno} #{conn.write_fds} until #{io.fileno} writeable" conn.write(transfer_until: io) if (events & IO::READABLE) > 0 + puts "read-trigger from fd:#{io.fileno} #{conn.read_fds} single block" conn.read(transfer_until: false) end else if (events & IO::READABLE) > 0 - # Call the write handler here because writes usually succeed without waiting for writablility. - # In this case the callback wait_io(IO::WRITABLE) isn't called, so that we don't get a trigger to transfer data. - # But after sending some data the caller usually waits for some answer to read. - # Therefore trigger transfer of all pending written data. - conn.write(transfer_until: :nodata) + puts "write-trigger from fd:#{io.fileno} #{conn.write_fds} until wouldblock" + # Many applications wait for writablility only in a would-block case. + # Then we get no trigger although data was written to the observed IO. + # After writing some data the caller usually waits for some answer to read. + # We take this event as a trigger to transfer of all pending written data. + conn.write(transfer_until: :wouldblock) + puts "read-trigger from fd:#{io.fileno} #{conn.read_fds} single block" conn.read(transfer_until: false) end end From b92a08ddd9d955051bcc101365444024108499a7 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 15 Sep 2021 16:57:35 +0200 Subject: [PATCH 47/62] Add timeout support to Scheduler#io_wait and fix cancel test The cancel test was racy, since there was no safety about arrival of the sent query. --- spec/helpers/scheduler.rb | 7 +++++++ spec/pg/scheduler_spec.rb | 1 + 2 files changed, 8 insertions(+) diff --git a/spec/helpers/scheduler.rb b/spec/helpers/scheduler.rb index 5f4578323..7e8ae7dfc 100644 --- a/spec/helpers/scheduler.rb +++ b/spec/helpers/scheduler.rb @@ -167,7 +167,14 @@ def io_wait(io, events, duration) @writable[io] = Fiber.current end + if duration + @waiting[Fiber.current] = current_time + duration + end + Fiber.yield + ensure + # Remove from @waiting in the case event occured before the timeout expired: + @waiting.delete(Fiber.current) if duration end # Used for Kernel#sleep and Thread::Mutex#sleep diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index 38ff77899..ab20dea8d 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -253,6 +253,7 @@ def run_with_scheduler(timeout=10) it "can cancel a query" do run_with_scheduler do |conn| conn.send_query "SELECT pg_sleep(5)" + conn.block(0.01) # trigger transmission to the server conn.cancel expect{ conn.get_last_result }.to raise_error(PG::QueryCanceled) end From e82ade83e52798a39a49b621f8e7e49b9f58081b Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 15 Sep 2021 16:59:37 +0200 Subject: [PATCH 48/62] Change test to run 3 PG.connect concurrently Since this should trigger different kind of errors. --- spec/pg/scheduler_spec.rb | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index ab20dea8d..4c5b2e3a3 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -94,11 +94,17 @@ def run_with_scheduler(timeout=10) end end - it "connects several times" do + it "connects several times concurrently" do run_with_scheduler do + q = Queue.new 3.times do - conn = PG.connect(@conninfo_gate) - conn.finish + Fiber.schedule do + conn = PG.connect(@conninfo_gate) + conn.finish + q << true + end + end.times do + q.pop end end end From e9d1f93fc90e87dff2e50f891ff24f730310c9a6 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 15 Sep 2021 17:03:49 +0200 Subject: [PATCH 49/62] Ensure conn.block is called before each conn.get_result call call.block returns, if at least one result was received. The server always sends all results at once, so that isn't immediately visible, even if a multi-statement query was sent. But if there is some network starvarion, then the subsequent conn.get_result world block. --- ext/pg_connection.c | 70 ++++++++++++++++++++++-------- lib/pg/connection.rb | 6 --- spec/helpers/tcp_gate_scheduler.rb | 4 +- 3 files changed, 56 insertions(+), 24 deletions(-) diff --git a/ext/pg_connection.c b/ext/pg_connection.c index 2a9676b38..2987d3bcf 100644 --- a/ext/pg_connection.c +++ b/ext/pg_connection.c @@ -3075,6 +3075,42 @@ pgconn_block( int argc, VALUE *argv, VALUE self ) { } +/* + * call-seq: + * conn.sync_get_last_result( ) -> PG::Result + * + * This function has the same behavior as #async_get_last_result, but is implemented using the synchronous command processing API of libpq. + * See #async_exec for the differences between the two API variants. + * It's not recommended to use explicit sync or async variants but #get_last_result instead, unless you have a good reason to do so. + */ +static VALUE +pgconn_get_last_result(VALUE self) +{ + PGconn *conn = pg_get_pgconn(self); + VALUE rb_pgresult = Qnil; + PGresult *cur, *prev; + + + cur = prev = NULL; + while ((cur = gvl_PQgetResult(conn)) != NULL) { + int status; + + if (prev) PQclear(prev); + prev = cur; + + status = PQresultStatus(cur); + if (status == PGRES_COPY_OUT || status == PGRES_COPY_IN || status == PGRES_COPY_BOTH) + break; + } + + if (prev) { + rb_pgresult = pg_new_result( prev, self ); + pg_result_check(rb_pgresult); + } + + return rb_pgresult; +} + /* * call-seq: * conn.get_last_result( ) -> PG::Result @@ -3090,17 +3126,22 @@ pgconn_block( int argc, VALUE *argv, VALUE self ) { * one result. */ static VALUE -pgconn_get_last_result(VALUE self) +pgconn_async_get_last_result(VALUE self) { PGconn *conn = pg_get_pgconn(self); VALUE rb_pgresult = Qnil; PGresult *cur, *prev; - - cur = prev = NULL; - while ((cur = gvl_PQgetResult(conn)) != NULL) { + cur = prev = NULL; + for(;;) { int status; + pgconn_block( 0, NULL, self ); /* wait for input (without blocking) before reading the last result */ + + cur = gvl_PQgetResult(conn); + if (cur == NULL) + break; + if (prev) PQclear(prev); prev = cur; @@ -3215,8 +3256,7 @@ pgconn_async_exec(int argc, VALUE *argv, VALUE self) pgconn_discard_results( self ); pgconn_send_query( argc, argv, self ); - pgconn_block( 0, NULL, self ); /* wait for input (without blocking) before reading the last result */ - rb_pgresult = pgconn_get_last_result( self ); + rb_pgresult = pgconn_async_get_last_result( self ); if ( rb_block_given_p() ) { return rb_ensure( rb_yield, rb_pgresult, pg_result_clear, rb_pgresult ); @@ -3288,8 +3328,7 @@ pgconn_async_exec_params(int argc, VALUE *argv, VALUE self) } else { pgconn_send_query_params( argc, argv, self ); } - pgconn_block( 0, NULL, self ); /* wait for input (without blocking) before reading the last result */ - rb_pgresult = pgconn_get_last_result( self ); + rb_pgresult = pgconn_async_get_last_result( self ); if ( rb_block_given_p() ) { return rb_ensure( rb_yield, rb_pgresult, pg_result_clear, rb_pgresult ); @@ -3327,8 +3366,7 @@ pgconn_async_prepare(int argc, VALUE *argv, VALUE self) pgconn_discard_results( self ); pgconn_send_prepare( argc, argv, self ); - pgconn_block( 0, NULL, self ); /* wait for input (without blocking) before reading the last result */ - rb_pgresult = pgconn_get_last_result( self ); + rb_pgresult = pgconn_async_get_last_result( self ); if ( rb_block_given_p() ) { return rb_ensure( rb_yield, rb_pgresult, pg_result_clear, rb_pgresult ); @@ -3381,8 +3419,7 @@ pgconn_async_exec_prepared(int argc, VALUE *argv, VALUE self) pgconn_discard_results( self ); pgconn_send_query_prepared( argc, argv, self ); - pgconn_block( 0, NULL, self ); /* wait for input (without blocking) before reading the last result */ - rb_pgresult = pgconn_get_last_result( self ); + rb_pgresult = pgconn_async_get_last_result( self ); if ( rb_block_given_p() ) { return rb_ensure( rb_yield, rb_pgresult, pg_result_clear, rb_pgresult ); @@ -3406,8 +3443,7 @@ pgconn_async_describe_portal(VALUE self, VALUE portal) pgconn_discard_results( self ); pgconn_send_describe_portal( self, portal ); - pgconn_block( 0, NULL, self ); /* wait for input (without blocking) before reading the last result */ - rb_pgresult = pgconn_get_last_result( self ); + rb_pgresult = pgconn_async_get_last_result( self ); if ( rb_block_given_p() ) { return rb_ensure( rb_yield, rb_pgresult, pg_result_clear, rb_pgresult ); @@ -3431,8 +3467,7 @@ pgconn_async_describe_prepared(VALUE self, VALUE stmt_name) pgconn_discard_results( self ); pgconn_send_describe_prepared( self, stmt_name ); - pgconn_block( 0, NULL, self ); /* wait for input (without blocking) before reading the last result */ - rb_pgresult = pgconn_get_last_result( self ); + rb_pgresult = pgconn_async_get_last_result( self ); if ( rb_block_given_p() ) { return rb_ensure( rb_yield, rb_pgresult, pg_result_clear, rb_pgresult ); @@ -4361,7 +4396,8 @@ init_pg_connection() rb_define_method(rb_cPGconn, "wait_for_notify", pgconn_wait_for_notify, -1); rb_define_alias(rb_cPGconn, "notifies_wait", "wait_for_notify"); rb_define_method(rb_cPGconn, "quote_ident", pgconn_s_quote_ident, 1); - rb_define_method(rb_cPGconn, "get_last_result", pgconn_get_last_result, 0); + rb_define_method(rb_cPGconn, "sync_get_last_result", pgconn_get_last_result, 0); + rb_define_method(rb_cPGconn, "async_get_last_result", pgconn_async_get_last_result, 0); #ifdef HAVE_PQENCRYPTPASSWORDCONN rb_define_method(rb_cPGconn, "encrypt_password", pgconn_encrypt_password, -1); #endif diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index 34f81e512..ef96ea3d6 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -366,12 +366,6 @@ def async_get_result(*args) sync_get_result end - alias sync_get_last_result get_last_result - def async_get_last_result(*args) - block - sync_get_last_result - end - alias sync_get_copy_data get_copy_data def async_get_copy_data(async=false, decoder=nil) if async diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index 6119d9c19..42941fa70 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -90,7 +90,9 @@ def read( transfer_until: ) begin begin - read_str = @external_io.read_nonblock(1000) + # 140 bytes transfer is required to trigger an error in spec "can cancel a query", when get_last_error doesn't wait for readability between PQgetResult calls. + # TODO: Make an explicit spec for this case. + read_str = @external_io.read_nonblock(140) print_data("read-transfer #{read_fds}", read_str) @internal_io.write(read_str) rescue IO::WaitReadable, Errno::EINTR From 72b77cbdd81901063088f07ba3f599f2960bc911 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 15 Sep 2021 21:32:53 +0200 Subject: [PATCH 50/62] Use Socket class instead of BasicSocket This fixes compatibility to Truffleruby and seems to be a better choice anyway. --- ext/pg_connection.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ext/pg_connection.c b/ext/pg_connection.c index 2987d3bcf..b5554e316 100644 --- a/ext/pg_connection.c +++ b/ext/pg_connection.c @@ -936,7 +936,7 @@ pgconn_socket_io(VALUE self) ruby_sd = sd; #endif - cSocket = rb_const_get(rb_cObject, rb_intern("BasicSocket")); + cSocket = rb_const_get(rb_cObject, rb_intern("Socket")); socket_io = rb_funcall( cSocket, rb_intern("for_fd"), 1, INT2NUM(ruby_sd)); /* Disable autoclose feature */ From 1e40851f6b665f8333f306cfc10eb45021ed54df Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 15 Sep 2021 21:36:24 +0200 Subject: [PATCH 51/62] TcpGateScheduler: some refine on debug prints --- spec/helpers/tcp_gate_scheduler.rb | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index 42941fa70..bfd852ad9 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -210,7 +210,7 @@ def puts(*args) end def io_wait(io, events, duration) - #$stderr.puts [:IO_WAIT, io, events, duration, Fiber.current].inspect + puts "io_wait(#{io.inspect}, #{events}, #{duration}) from #{caller[0]}" begin sock = TCPSocket.for_fd(io.fileno) @@ -261,23 +261,23 @@ def io_wait(io, events, duration) conn.observed_fd = io.fileno if (events & IO::WRITABLE) > 0 - puts "write-trigger from fd:#{io.fileno} #{conn.write_fds} until #{io.fileno} writeable" + puts "write-trigger #{conn.write_fds} until #{io.fileno} writeable" conn.write(transfer_until: io) if (events & IO::READABLE) > 0 - puts "read-trigger from fd:#{io.fileno} #{conn.read_fds} single block" + puts "read-trigger #{conn.read_fds} single block" conn.read(transfer_until: false) end else if (events & IO::READABLE) > 0 - puts "write-trigger from fd:#{io.fileno} #{conn.write_fds} until wouldblock" + puts "write-trigger #{conn.write_fds} until wouldblock" # Many applications wait for writablility only in a would-block case. # Then we get no trigger although data was written to the observed IO. # After writing some data the caller usually waits for some answer to read. # We take this event as a trigger to transfer of all pending written data. conn.write(transfer_until: :wouldblock) - puts "read-trigger from fd:#{io.fileno} #{conn.read_fds} single block" + puts "read-trigger #{conn.read_fds} single block" conn.read(transfer_until: false) end end From fc4407160ba5b5c45dac2d44a8a3fc76f78d8763 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Wed, 15 Sep 2021 21:45:29 +0200 Subject: [PATCH 52/62] Compat with Macos --- spec/helpers/tcp_gate_scheduler.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index bfd852ad9..c8b60fd63 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -37,7 +37,7 @@ def initialize(internal_io, external_host, external_port, debug: false) def other_side_of?(local_address, remote_address) internal_io.local_address.to_s == remote_address.to_s && internal_io.remote_address.to_s == local_address.to_s - rescue Errno::ENOTCONN + rescue Errno::ENOTCONN, Errno::EINVAL # internal_io.remote_address fails, if connection is already half closed false end From 29084ea9314cb75beb91d0ff677ae8c8a675ee46 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Thu, 16 Sep 2021 10:00:01 +0200 Subject: [PATCH 53/62] Avoid recursive calls of puts As happend on Windows: 1) with a Fiber scheduler can send lots of data per put_copy_data Failure/Error: super ThreadError: deadlock; recursive locking ./spec/helpers/tcp_gate_scheduler.rb:209:in `write' ./spec/helpers/tcp_gate_scheduler.rb:209:in `puts' ./spec/helpers/tcp_gate_scheduler.rb:209:in `puts' ./spec/helpers/tcp_gate_scheduler.rb:209:in `puts' ./spec/helpers/tcp_gate_scheduler.rb:213:in `io_wait' ./spec/helpers/tcp_gate_scheduler.rb:209:in `write' ./spec/helpers/tcp_gate_scheduler.rb:209:in `puts' ./spec/helpers/tcp_gate_scheduler.rb:209:in `puts' ./spec/helpers/tcp_gate_scheduler.rb:209:in `puts' ./spec/helpers/tcp_gate_scheduler.rb:213:in `io_wait' ./lib/pg/connection.rb:405:in `wait_for_flush' ./lib/pg/connection.rb:405:in `async_put_copy_data' ./spec/pg/scheduler_spec.rb:185:in `block (5 levels) in ' ./spec/pg/scheduler_spec.rb:184:in `times' ./spec/pg/scheduler_spec.rb:184:in `block (4 levels) in ' ./lib/pg/connection.rb:242:in `copy_data' ./spec/pg/scheduler_spec.rb:181:in `block (3 levels) in ' ./spec/pg/scheduler_spec.rb:52:in `block (2 levels) in run_with_scheduler' --- spec/helpers/tcp_gate_scheduler.rb | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index c8b60fd63..4d89ff8c5 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -192,6 +192,7 @@ def initialize(external_host:, external_port:, internal_host: 'localhost', inter @external_port = external_port @finish = false @debug = debug + @in_puts = false puts "TcpGate server listening: #{@server_io.inspect}" end @@ -205,8 +206,11 @@ def internal_port end def puts(*args) - return unless @debug + return if !@debug || @in_puts # Avoid recursive calls of puts + @in_puts = true super + ensure + @in_puts = false end def io_wait(io, events, duration) From 581c7807c330c479695d84e5f6178459de10d1ad Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Thu, 16 Sep 2021 13:00:50 +0200 Subject: [PATCH 54/62] TcpGateScheduler: Gather events on fds, that are not yet accepted ... and respect them after accept. It can happen, that a connection is established and data is sent, before the TcpGateScheduler accepts the internal_io. In this case the events got lost and data wasn't transferred. --- spec/helpers/tcp_gate_scheduler.rb | 31 +++++++++++++++++++++++++++--- 1 file changed, 28 insertions(+), 3 deletions(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index 4d89ff8c5..92b357fae 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -36,7 +36,7 @@ def initialize(internal_io, external_host, external_port, debug: false) end def other_side_of?(local_address, remote_address) - internal_io.local_address.to_s == remote_address.to_s && internal_io.remote_address.to_s == local_address.to_s + internal_io.local_address.to_s == remote_address && internal_io.remote_address.to_s == local_address rescue Errno::ENOTCONN, Errno::EINVAL # internal_io.remote_address fails, if connection is already half closed false @@ -183,10 +183,13 @@ def finish end end + UnknownConnection = Struct.new :fileno, :events + def initialize(external_host:, external_port:, internal_host: 'localhost', internal_port: 0, debug: false) super() @started = false @connections = [] + @unknown_connections = {} @server_io = TCPServer.new(internal_host, internal_port) @external_host = external_host @external_port = external_port @@ -219,8 +222,8 @@ def io_wait(io, events, duration) begin sock = TCPSocket.for_fd(io.fileno) sock.autoclose = false - local_address = sock.local_address - remote_address = sock.remote_address + local_address = sock.local_address.to_s + remote_address = sock.remote_address.to_s rescue Errno::ENOTCONN, Errno::EINVAL end @@ -236,6 +239,20 @@ def io_wait(io, events, duration) conn = Connection.new(client, @external_host, @external_port, debug: @debug) puts "accept new int:#{conn.internal_io.inspect} from #{conn.internal_io.remote_address.inspect} server fd:#{@server_io.fileno}" @connections << conn + + # Have there been any events on the connection before accept? + if uconn=@unknown_connections.delete([conn.internal_io.remote_address.to_s, conn.internal_io.local_address.to_s]) + conn.observed_fd = uconn.fileno + + if (uconn.events & IO::WRITABLE) > 0 + puts "late-write-trigger #{conn.write_fds} until wouldblock" + conn.write(transfer_until: :wouldblock) + end + if (uconn.events & IO::READABLE) > 0 + puts "late-read-trigger #{conn.read_fds} single block" + conn.read(transfer_until: false) + end + end end end end @@ -285,6 +302,14 @@ def io_wait(io, events, duration) conn.read(transfer_until: false) end end + else + # Maybe the connection is not yet accepted. + # We store it to do the transfer after accept arrived. + if uc=@unknown_connections[[local_address, remote_address]] + uc.events |= events + else + @unknown_connections[[local_address, remote_address]] = UnknownConnection.new(io.fileno, events) + end end super From 7bdf4a4991f11a62fe369f111ba91e7a8cca6e90 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Thu, 16 Sep 2021 16:25:59 +0200 Subject: [PATCH 55/62] Windows: Add a workaround for nonworking nonblocking-IO on Windows It is questionable how valueable Fiber.scheduler is on Windows, given that most IO doesn't go through the scheduler. But this way we satisfy our test suite at least. --- lib/pg/connection.rb | 38 +++++++++++++++++++++++++------------- 1 file changed, 25 insertions(+), 13 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index ef96ea3d6..ca194d5be 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -422,21 +422,33 @@ def async_reset alias sync_cancel cancel def async_cancel - cl = socket_io.remote_address.connect be_pid = backend_pid be_key = backend_key - # Send CANCEL_REQUEST_CODE and parameters - cl.write([0x10, 1234, 5678, be_pid, be_key].pack("NnnNN")) - - # Wait for the postmaster to close the connection, which indicates that it's processed the request. - # cl.read(1) should be enough, but read isn't scheduler compatible on Windows. - # Work around by using read_nonblock. - begin - cl.read_nonblock(1) - rescue IO::WaitReadable, Errno::EINTR - cl.wait_readable - retry - rescue EOFError + cancel_request = [0x10, 1234, 5678, be_pid, be_key].pack("NnnNN") + + if Fiber.respond_to?(:scheduler) && Fiber.scheduler && RUBY_PLATFORM =~ /mingw|mswin/ + # Ruby's nonblocking IO is not really supported on Windows. + # We work around by using threads and explicit calls to wait_readable/wait_writable. + cl = Thread.new(socket_io.remote_address) { |ra| ra.connect }.value + begin + cl.write_nonblock(cancel_request) + rescue IO::WaitReadable, Errno::EINTR + cl.wait_writable + retry + end + begin + cl.read_nonblock(1) + rescue IO::WaitReadable, Errno::EINTR + cl.wait_readable + retry + rescue EOFError + end + else + cl = socket_io.remote_address.connect + # Send CANCEL_REQUEST_CODE and parameters + cl.write(cancel_request) + # Wait for the postmaster to close the connection, which indicates that it's processed the request. + cl.read(1) end cl.close From 45e179ab4612fbc366a225076d22d57895aceb6b Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sat, 18 Sep 2021 17:22:08 +0200 Subject: [PATCH 56/62] Add workaround for Truffleruby and revert conn.socket_io to BasicSocket This is a workaround for truffleruby < 21.3.0. The proposed upstream fix is here: https://github.com/oracle/truffleruby/pull/2444 Although it works with any of the Socket classes, revert to BasicSocket, since this is the common base class of TCPSocket and UNIXSocket. --- ext/pg_connection.c | 2 +- lib/pg/connection.rb | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/ext/pg_connection.c b/ext/pg_connection.c index b5554e316..2987d3bcf 100644 --- a/ext/pg_connection.c +++ b/ext/pg_connection.c @@ -936,7 +936,7 @@ pgconn_socket_io(VALUE self) ruby_sd = sd; #endif - cSocket = rb_const_get(rb_cObject, rb_intern("Socket")); + cSocket = rb_const_get(rb_cObject, rb_intern("BasicSocket")); socket_io = rb_funcall( cSocket, rb_intern("for_fd"), 1, INT2NUM(ruby_sd)); /* Disable autoclose feature */ diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index ca194d5be..bb6ca5552 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -443,6 +443,24 @@ def async_cancel retry rescue EOFError end + elsif RUBY_ENGINE == 'truffleruby' + begin + cl = socket_io.remote_address.connect + rescue NotImplementedError + # Workaround for truffleruby < 21.3.0 + cl2 = Socket.for_fd(socket_io.fileno) + cl2.autoclose = false + adr = cl2.remote_address + if adr.ip? + cl = TCPSocket.new(adr.ip_address, adr.ip_port) + cl.autoclose = false + else + cl = UNIXSocket.new(adr.unix_path) + cl.autoclose = false + end + end + cl.write(cancel_request) + cl.read(1) else cl = socket_io.remote_address.connect # Send CANCEL_REQUEST_CODE and parameters From 72d5b32db3578b2d7ca11c8f7abe72e7ba1c258f Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sun, 19 Sep 2021 14:43:31 +0200 Subject: [PATCH 57/62] Implement async_encrypt_password It retrieves the passowrd algorithm in a scheduler compatible way, if it isn't passed as the third parameter. --- lib/pg/connection.rb | 14 ++++++++++++++ spec/pg/scheduler_spec.rb | 9 +++++++++ 2 files changed, 23 insertions(+) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index bb6ca5552..ec528a5b9 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -414,6 +414,14 @@ def async_put_copy_end(*args) true end + if method_defined? :encrypt_password + alias sync_encrypt_password encrypt_password + def async_encrypt_password( password, username, algorithm=nil ) + algorithm ||= exec("SHOW password_encryption").getvalue(0,0) + sync_encrypt_password(password, username, algorithm) + end + end + alias sync_reset reset def async_reset reset_start @@ -550,6 +558,12 @@ def async_connect(*args, **kwargs) :cancel => [:async_cancel, :sync_cancel], } + if PG::Connection.instance_methods.include? :async_encrypt_password + REDIRECT_METHODS.merge!({ + :encrypt_password => [:async_encrypt_password, :sync_encrypt_password], + }) + end + def async_send_api=(enable) REDIRECT_SEND_METHODS.each do |ali, (async, sync)| undef_method(ali) if method_defined?(ali) diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index 4c5b2e3a3..d91407eb9 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -264,6 +264,15 @@ def run_with_scheduler(timeout=10) expect{ conn.get_last_result }.to raise_error(PG::QueryCanceled) end end + + it "can encrypt_password", :postgresql_10 do + run_with_scheduler do |conn| + res = conn.encrypt_password "passw", "myuser" + expect( res ).to match( /\S+/ ) + res = conn.encrypt_password "passw", "myuser", "md5" + expect( res ).to eq( "md57883f68fde2c10fdabfb7640c74cf1a7" ) + end + end end # Do not wait for threads doing blocking calls at the process shutdown. From 7fe6dac931d5fcdf5e793c44c2a9435549c1440a Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Sun, 19 Sep 2021 21:58:20 +0200 Subject: [PATCH 58/62] Add scheduler aware Connection.ping version --- lib/pg/connection.rb | 12 ++++++++++++ spec/pg/scheduler_spec.rb | 9 +++++++++ 2 files changed, 21 insertions(+) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index ec528a5b9..25e7ce285 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -529,8 +529,20 @@ def async_connect(*args, **kwargs) conn.send(:async_connect_or_reset, :connect_poll) end + alias sync_ping ping + def async_ping(*args) + if Fiber.respond_to?(:scheduler) && Fiber.scheduler + # Run PQping in a second thread to avoid blocking of the scheduler. + # Unfortunately there's no nonblocking way to run ping. + Thread.new { sync_ping(*args) }.value + else + sync_ping(*args) + end + end + REDIRECT_CLASS_METHODS = { :new => [:async_connect, :sync_connect], + :ping => [:async_ping, :sync_ping], } # These methods are affected by PQsetnonblocking diff --git a/spec/pg/scheduler_spec.rb b/spec/pg/scheduler_spec.rb index d91407eb9..fa607588c 100644 --- a/spec/pg/scheduler_spec.rb +++ b/spec/pg/scheduler_spec.rb @@ -273,6 +273,15 @@ def run_with_scheduler(timeout=10) expect( res ).to eq( "md57883f68fde2c10fdabfb7640c74cf1a7" ) end end + + it "can ping server" do + run_with_scheduler do |conn| + # ping doesn't trigger the scheduler, but runs in a second thread. + # This is why @conninfo is used instead of @conninfo_gate + ping = PG::Connection.ping(@conninfo) + expect( ping ).to eq( PG::PQPING_OK ) + end + end end # Do not wait for threads doing blocking calls at the process shutdown. From 6b4cd3f7133eff9d0a3943311a599e88b63922e2 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Mon, 20 Sep 2021 21:49:04 +0200 Subject: [PATCH 59/62] Enable GVL unlocking per default and rename it Option --disable-gvl-unlock is more understandable. Benchmarks didn't show a measurable difference between unlocking enabled/disabled, so keeping it enabled is safer. Should there still be any blocking function calls, GVL unlocking allows to run ruby threads in parallel. --- ext/extconf.rb | 7 +++++-- ext/gvl_wrappers.c | 4 ++-- ext/gvl_wrappers.h | 4 ++-- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/ext/extconf.rb b/ext/extconf.rb index 8ed3cbbe7..566e73116 100755 --- a/ext/extconf.rb +++ b/ext/extconf.rb @@ -15,8 +15,11 @@ ENV['PATH'] = "#{pgdir}/bin" + File::PATH_SEPARATOR + ENV['PATH'] end -if enable_config("nogvl") - $defs.push( "-DENABLE_NOGVL" ) +if enable_config("gvl-unlock", true) + $defs.push( "-DENABLE_GVL_UNLOCK" ) + $stderr.puts "Calling libpq with GVL unlocked" +else + $stderr.puts "Calling libpq with GVL locked" end if enable_config("windows-cross") diff --git a/ext/gvl_wrappers.c b/ext/gvl_wrappers.c index 9e7554f51..f570913b5 100644 --- a/ext/gvl_wrappers.c +++ b/ext/gvl_wrappers.c @@ -9,12 +9,12 @@ char *PQencryptPasswordConn(PGconn *conn, const char *passwd, const char *user, const char *algorithm){return NULL;} #endif -#ifdef ENABLE_NOGVL +#ifdef ENABLE_GVL_UNLOCK FOR_EACH_BLOCKING_FUNCTION( DEFINE_GVL_WRAPPER_STRUCT ); FOR_EACH_BLOCKING_FUNCTION( DEFINE_GVL_SKELETON ); #endif FOR_EACH_BLOCKING_FUNCTION( DEFINE_GVL_STUB ); -#ifdef ENABLE_NOGVL +#ifdef ENABLE_GVL_UNLOCK FOR_EACH_CALLBACK_FUNCTION( DEFINE_GVL_WRAPPER_STRUCT ); FOR_EACH_CALLBACK_FUNCTION( DEFINE_GVLCB_SKELETON ); #endif diff --git a/ext/gvl_wrappers.h b/ext/gvl_wrappers.h index 5ea102cbf..ce226c8b4 100644 --- a/ext/gvl_wrappers.h +++ b/ext/gvl_wrappers.h @@ -50,7 +50,7 @@ return NULL; \ } -#ifdef ENABLE_NOGVL +#ifdef ENABLE_GVL_UNLOCK #define DEFINE_GVL_STUB(name, when_non_void, rettype, lastparamtype, lastparamname) \ rettype gvl_##name(FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST3) lastparamtype lastparamname){ \ struct gvl_wrapper_##name##_params params = { \ @@ -78,7 +78,7 @@ return NULL; \ } -#ifdef ENABLE_NOGVL +#ifdef ENABLE_GVL_UNLOCK #define DEFINE_GVLCB_STUB(name, when_non_void, rettype, lastparamtype, lastparamname) \ rettype gvl_##name(FOR_EACH_PARAM_OF_##name(DEFINE_PARAM_LIST3) lastparamtype lastparamname){ \ struct gvl_wrapper_##name##_params params = { \ From 0871be7c82b98b52dc641a46111f2904095b9159 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Mon, 20 Sep 2021 22:05:29 +0200 Subject: [PATCH 60/62] TcpGate: Ignore EBADF error It happens on Windows sometimes, when writing to stdout per puts. In this case `io` isn't a socket and treating it as such results in EBADF: Errno::EBADF: Bad file descriptor - not a socket file descriptor ./spec/helpers/tcp_gate_scheduler.rb:223:in `for_fd' ./spec/helpers/tcp_gate_scheduler.rb:223:in `io_wait' ./spec/helpers/tcp_gate_scheduler.rb:214:in `write' ./spec/helpers/tcp_gate_scheduler.rb:214:in `puts' --- spec/helpers/tcp_gate_scheduler.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spec/helpers/tcp_gate_scheduler.rb b/spec/helpers/tcp_gate_scheduler.rb index 92b357fae..5e06e95e3 100644 --- a/spec/helpers/tcp_gate_scheduler.rb +++ b/spec/helpers/tcp_gate_scheduler.rb @@ -224,7 +224,7 @@ def io_wait(io, events, duration) sock.autoclose = false local_address = sock.local_address.to_s remote_address = sock.remote_address.to_s - rescue Errno::ENOTCONN, Errno::EINVAL + rescue Errno::ENOTCONN, Errno::EINVAL, Errno::EBADF end unless @started From 7191132cef58271f74b0e4b997319fca7c1e8d38 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Fri, 1 Oct 2021 13:08:31 +0200 Subject: [PATCH 61/62] Replace Resolv.getaddress by IPSocket to avoid inconsistencies The resolv library resolves differently than the system resolver. In general it prefers IPv4 over IPv6. This leads to the situation, that the server socket of the TcpGateScheduler is bound to IPv6, but PG.connect tries to use IPv4 instead. This happened on Windows-10 and Windows-Server 2016 with Ruby-3.0 and with no entries in the /etc/hosts file. I tried to align the results of resolv and system library, but didn't find out how the system resolver decides between IPv4 and IPv6. So the workaround is to use a second thread and use the system resolver on Ruby-3.0. --- lib/pg/connection.rb | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/lib/pg/connection.rb b/lib/pg/connection.rb index 25e7ce285..3efedf5df 100644 --- a/lib/pg/connection.rb +++ b/lib/pg/connection.rb @@ -134,10 +134,9 @@ def self::parse_connect_args( *args ) Fiber.scheduler && RUBY_VERSION < '3.1.' - # Use pure Ruby address resolver to avoid blocking of the scheduler. + # Use a second thread to avoid blocking of the scheduler. # `IPSocket.getaddress` isn't fiber aware before ruby-3.1. - require "resolv" - Resolv.getaddress(mhost) rescue '' + Thread.new{ IPSocket.getaddress(mhost) rescue '' }.value else IPSocket.getaddress(mhost) rescue '' end From 1b155a9a0622e9bfca9eae57311be7ea178f6552 Mon Sep 17 00:00:00 2001 From: Lars Kanis Date: Fri, 1 Oct 2021 18:11:15 +0200 Subject: [PATCH 62/62] Allow truffleruby to fail on github actions and add it to travis-ci Truffleruby-21.1.0 currently fails on Github Actions like here: https://github.com/larskanis/ruby-pg/runs/3766520041?check_suite_focus=true However it works with the same version on my local laptop, on travis-ci and with the current truffleruby-head version on Github. Since it looks like some issue that's already fixed, this commit allows truffleruby to fail on github for now. --- .github/workflows/source-gem.yml | 2 ++ .travis.yml | 7 ++++++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/.github/workflows/source-gem.yml b/.github/workflows/source-gem.yml index a99a0a80c..7ad81c856 100644 --- a/.github/workflows/source-gem.yml +++ b/.github/workflows/source-gem.yml @@ -113,7 +113,9 @@ jobs: - run: bundle install - run: gem install --local *.gem --verbose + - name: Run specs + continue-on-error: ${{ matrix.ruby == 'truffleruby' }} env: PG_DEBUG: 1 run: ruby -rpg -S rspec spec/**/*_spec.rb -cfdoc diff --git a/.travis.yml b/.travis.yml index 5340558b8..ef86428f8 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,5 +1,5 @@ sudo: required -dist: xenial +dist: focal services: - docker language: ruby @@ -20,9 +20,14 @@ matrix: - rvm: "2.5" env: - "PGVERSION=9.3" + # Use Ubuntu-16.04 since postgresql-9.3 depends on openssl-1.0.0, which isn't available in 20.04 + dist: xenial - rvm: ruby-head env: - "PGVERSION=13" + - rvm: truffleruby + env: + - "PGVERSION=13" allow_failures: - rvm: ruby-head