Skip to content
Snippets Groups Projects
  1. Apr 12, 2019
    • Vladislav Shpilevoy's avatar
      swim: factor out 'update' part of swim_member_upsert() · 1eb82afc
      Vladislav Shpilevoy authored
      Move 'update' logic into a separate function, because in the next
      commits it is going to become more complicated due to payload
      introduction, and it would be undesirable to clog the upsert()
      function with payload-specific code.
      
      Part of #3234
      1eb82afc
    • Vladislav Shpilevoy's avatar
      swim: replace event_bin and member_bin with the passport · 17f895ed
      Vladislav Shpilevoy authored
      Event_bin and member_bin binary packet structures were designed
      separately for different purposes. Initially the event_bin was
      thought having the same fields as passport + optional old UUID +
      optional payload. On the other hand, member_bin was supposed to
      store the passport + mandatory payload.
      
      But old UUID was cut off in favour of another way of UUID update.
      And payload appeared to be optional in both anti-entropy and
      dissemination. It means, that member_bin and event_bin are not
      needed anymore as separate structures. This commit replaces them
      with the passport completely.
      
      Part of #3234
      17f895ed
    • Vladislav Shpilevoy's avatar
      swim: factor out MP_BIN decoding from swim_decode_uuid · 61b0bd5a
      Vladislav Shpilevoy authored
      The new function is swim_decode_bin(), and is going to be used
      to safely decode payloads - arbitrary binary data disseminated
      alongside with all the other SWIM member attributes.
      
      Part of #3234
      61b0bd5a
    • Alexander Turenko's avatar
      net.box: fix 'unique' index flag in net.box schema · f1f6433b
      Alexander Turenko authored
      Before this commit it always returns false.
      
      Fixes #4091.
      f1f6433b
    • Cyrill Gorcunov's avatar
      fiber: Define constants for reserved fids · 4bb7b332
      Cyrill Gorcunov authored
      Opencoded constants are not good for long time
      support, make it named one. Moreover there was
      a typo in comment, fid = 0 is reserved as well.
      4bb7b332
    • Cyrill Gorcunov's avatar
      fiber: Drop unused FIBER_CALL_STACK · 9e7b460c
      Cyrill Gorcunov authored
      The constant is leftover from 08585902
      9e7b460c
    • Serge Petrenko's avatar
      test: rework box/on_shutdown test · c0260529
      Serge Petrenko authored
      The test is flaky under high load (e.g. when is run in parallel with a
      lot of workers). Make it less dependent on arbitrary timeouts to improve
      stability.
      
      Part of #4134
      c0260529
    • Serge Petrenko's avatar
      test: extract on_shutdown tests from box/misc · 70ea9998
      Serge Petrenko authored
      This part of the test is flaky when tests are run in parallel, besides,
      it is quite big on its own, so extract it into a separate file to add
      more flexibility in running tests and to make finding problems easier.
      
      Part of #4134
      70ea9998
    • Nikita Pettik's avatar
      sql: increment rowcount of FK alteration · 3bd13cf5
      Nikita Pettik authored
      Before this patch SQL statement which involves FK constraints creation
      or drop didn't increment rowcount:
      
      box.execute("ALTER TABLE t ADD CONSTRAINT fk1 FOREIGN KEY (b) REFERENCES parent (a);")
      ---
      - rowcount: 0
      ...
      
      This patch fixes this misbehaviour: accidentally VDBE was forgotten to
      enable counting changes during ALTER TABLE ADD/DROP constraint.
      
      Closes #4130
      3bd13cf5
    • Cyrill Gorcunov's avatar
      fiber: Drop redundant memset call · 264beb8b
      Cyrill Gorcunov authored
      When we allocate new fiber we are clearing the whole
      structure right after, so no need to call memset again,
      coro context is already full of zeros.
      
      Note the coro context is close to 1K size and redundat
      memset here is really a penalty.
      264beb8b
    • avtikhon's avatar
      test: disable flaky performance test · 962c2cae
      avtikhon authored
      Disabled wal_off/iterator_lt_gt.test.lua test due to performance
      test need to be reorganized into separate mode at the standalone
      host. Currently this test doesn't show any issue, but breaks the
      testing some time, with errors like:
      
      [010] wal_off/iterator_lt_gt.test.lua                                 [ fail ]
      [010]
      [010] Test failed! Result content mismatch:
      [010] --- wal_off/iterator_lt_gt.result	Fri Apr 12 10:30:43 2019
      [010] +++ wal_off/iterator_lt_gt.reject	Fri Apr 12 10:36:30 2019
      [010] @@ -79,7 +79,9 @@
      [010]  ...
      [010]  too_longs
      [010]  ---
      [010] -- []
      [010] +- - 'Some of the iterators takes too long to position: 0.074278'
      [010] +  - 'Some of the iterators takes too long to position: 0.11786'
      [010] +  - 'Some of the iterators takes too long to position: 0.053848'
      [010]  ...
      [010]  s:drop()
      [010]  ---
      [010]
      [010] Last 15 lines of Tarantool Log file [Instance "wal"][/tarantool/test/var/010_wal_off/wal.log]:
      
      See #2539
      962c2cae
    • Konstantin Osipov's avatar
      iproto: reduce effects of input buffer fragmentation on large cfg.readahead · a58b9bb8
      Konstantin Osipov authored
      When cfg.readahead is large, iproto_reset_input() has a tendency to
      leave all input buffers large enough for a long time. On the other hand,
      the input buffer is not recycled until its maximal size is reached.
      This leaves to a case when we keep shifting the read position towards
      the end of the buffer, fragmenting memory and growing it to readahead
      size, even if input packets and batches are actually small.
      
      Suggested by Alexander Turenko.
      a58b9bb8
    • Vladimir Davydov's avatar
      vinyl: improve dump start/stop logging · 363ab8e6
      Vladimir Davydov authored
      When initiating memory dump, print how much memory is going to be
      dumped, expected dump rate, ETA, and recent write rate. Upon dump
      completion, print observed dump rate in addition to dump size and
      duration. This should help debugging stalls on memory quota.
      
      Example:
      
       | 2019-04-12 12:03:25.092 [30948] main/115/lua I> dumping 39659424 bytes, expected rate 6.0 MB/s, ETA 6.3 s, recent write rate 4.2 MB/s
       | 2019-04-12 12:03:25.101 [30948] main/106/vinyl.scheduler I> 512/1: dump started
       | 2019-04-12 12:03:25.102 [30948] vinyl.dump.0/104/task I> writing `./512/1/00000000000000000008.run'
       | 2019-04-12 12:03:26.487 [30948] vinyl.dump.0/104/task I> writing `./512/1/00000000000000000008.index'
       | 2019-04-12 12:03:26.547 [30948] main/106/vinyl.scheduler I> 512/1: dump completed
       | 2019-04-12 12:03:26.551 [30948] main/106/vinyl.scheduler I> 512/0: dump started
       | 2019-04-12 12:03:26.553 [30948] vinyl.dump.0/105/task I> writing `./512/0/00000000000000000010.run'
       | 2019-04-12 12:03:28.026 [30948] vinyl.dump.0/105/task I> writing `./512/0/00000000000000000010.index'
       | 2019-04-12 12:03:28.100 [30948] main/106/vinyl.scheduler I> 512/0: dump completed
       | 2019-04-12 12:03:28.100 [30948] main/106/vinyl.scheduler I> dumped 33554332 bytes in 3.0 s, rate 10.6 MB/s
      363ab8e6
    • Vladimir Davydov's avatar
      vinyl: account statements skipped on read · 779fa706
      Vladimir Davydov authored
      After we retrieve a statement from a secondary index, we always do
      a lookup in the primary index to get the full tuple corresponding to
      the found secondary key. It may turn out that the full tuple doesn't
      match the secondary key, which means the key was overwritten, but
      the DELETE statement hasn't been propagated yet (aka deferred DELETE).
      Currently, there's no way to figure out how often this happens as all
      tuples read from an LSM tree are accounted under 'get' counter.
      
      So this patch splits 'get' in two: 'get', which now accounts only
      tuples actually returned to the user, and 'skip', which accounts
      skipped tuples.
      779fa706
  2. Apr 11, 2019
    • Vladislav Shpilevoy's avatar
      test: add srand(time(NULL)) to swim unit tests · 3de1456f
      Vladislav Shpilevoy authored
      Appeared, that it is not called. But probably it should be, in
      order to catch more errors.
      3de1456f
    • Vladimir Davydov's avatar
      vinyl: take into account primary key lookup in latency accounting · b5734069
      Vladimir Davydov authored
      Currently, latency accounting and warning lives in vy_point_lookup and
      vy_read_iterator_next. As a result, we don't take into account full by
      partial tuple lookup in it while it can take quite a while, especially
      if there are lots of deferred DELETE statements we have to skip. So this
      patch moves latency accounting to the upper level, namely to vy_get and
      vinyl_iterator_{primary,secondary}_next.
      
      Note, as a side effect, now we always print full tuples to the log on
      "too long" warning. Besides, we strip LSN and statement type as those
      don't make much sense anymore.
      b5734069
    • Vladimir Davydov's avatar
      box: account index.pairs in box.stat.SELECT · 7275ad6b
      Vladimir Davydov authored
      box.stat.SELECT accounts index.get and index.select, but not
      index.pairs, which is confusing since pairs() may be used even
      more often than select() in a Lua application.
      7275ad6b
    • Vladislav Shpilevoy's avatar
      swim: keep encoded round message cached · cf0ddeb8
      Vladislav Shpilevoy authored
      During a SWIM round a message is being handed out consisting of
      at most 4 sections. Parts of the message change rarely along with
      a member attribute update, or with removal of a member. So it is
      possible to cache the message and send it during several round
      steps in a row. Or even do not rebuild it the whole round.
      
      Part of #3234
      cf0ddeb8
    • Konstantin Osipov's avatar
      sql: as a temporary hack, coerce typeof() return values with nosql types · cdde8aea
      Konstantin Osipov authored
      SQL is still using a sqlite legacy enum and not enum field_type from
      NoSQL to identify types. This creates a mess with type identification,
      when the original column/literal type is lost during expression
      evaluation.
      Until we have proper type arithmetics and preserve field_type in
      expressions, coerce the string return value of typeof() functions, which
      queries SQL expression value type, with the closest nosql type name.
      
      Rename:
          real -> number
          text -> string
          blob -> scalar
      cdde8aea
    • Vladislav Shpilevoy's avatar
      swim: fix typos in the code · 003e0ff6
      Vladislav Shpilevoy authored
      After turning on a spell checker there were found lots of typos.
      The commit fixes them.
      003e0ff6
    • Vladislav Shpilevoy's avatar
      test: fix SWIM test number · 10b4401c
      Vladislav Shpilevoy authored
      During merge it was accidentally set to too low number.
      
      Follow up 8fe05fdd
      (swim: expose ping broadcast API)
      10b4401c
    • Vladislav Shpilevoy's avatar
      swim: expose ping broadcast API · 8fe05fdd
      Vladislav Shpilevoy authored
      The previous commit has introduced an API to broadcast SWIM
      packets. This commit harnesses it in orider to allow user to do
      initial discovery in a cluster, when member tables are empty, and
      UUIDs aren't ready at hand.
      
      Part of #3234
      8fe05fdd
    • Vladislav Shpilevoy's avatar
      swim: introduce broadcast tasks · 38917980
      Vladislav Shpilevoy authored
      When a cluster is just created, no one knows anyone. Broadcast
      helps to establish some initial relationships between members.
      
      This commit introduces only an interface to create broadcast
      tasks from SWIM code. The next commit uses this interface to
      implement ping broadcast.
      
      Part of #3234
      38917980
    • Vladislav Shpilevoy's avatar
      swim: on address update increment incarnation · 0519a5fc
      Vladislav Shpilevoy authored
      In the original SWIM paper the incarnation is just a way of
      refuting old statuses, nothing more. It is not designed as a
      versioning system of a member and its non-status attributes. But
      Tarantool harnesses the incarnation for wider range of tasks.
      In Tarantool's implementation the incarnation (in theory) refutes
      old statuses, old payloads, old addresses.
      
      But appeared, that before the patch an address update did not
      touch incarnation. Because of that it was possible to rewrite a
      new address with the old one back. The patch fixes it with a mere
      increment of incarnation on each address update.
      
      The fix is simple because the current SWIM implementation
      always carries the tuple {incarnation, status, address} together,
      as a one big attribute. It is not so for payloads, so for them an
      analogous fix will be much more tricky.
      
      Follow-up for f510dc6f
      (swim: introduce failure detection component)
      0519a5fc
    • Vladimir Davydov's avatar
      replication: allow to change instance id during join · 319ab0ae
      Vladimir Davydov authored
      Before rebootstrapping a replica, the admin may delete it from the
      _cluster space on the master. If he doesn't make a checkpoint after
      that, rebootstrap will fail with
      
        E> ER_LOCAL_INSTANCE_ID_IS_READ_ONLY: The local instance id 2 is read-only
      
      This is sort of unexpected. Let's fix this issue by allowing replicas to
      change their id during join.
      
      A note about replication/misc test. The test used this error to check
      that a master instance doesn't crash in case a replica fails to
      bootstrap. However, we can simply set mismatching replicaset UUIDs to
      get the same effect.
      
      Closes #4107
      319ab0ae
    • Vladimir Davydov's avatar
      replication: fix garbage collection logic · 4824b00e
      Vladimir Davydov authored
      Currently, the garbage collector works with vclock signatures and
      doesn't take into account vclock components. This works as long as
      the caller (i.e. relay) makes sure that it doesn't advance a consumer
      associated with a replica unless its acknowledged vclock is greater
      than or equal to the vclock of a WAL file fed to it. The bug is that
      it does not - it only compares vclock signatures. As a result, if
      a replica has some local changes or changes pulled from other members
      of the cluster, which render its signature greater, the master may
      remove files that are still needed by the replica, permanently breaking
      replication and requiring rebootstrap.
      
      I guess the proper fix would be teaching the garbage collector
      operate on vclock components rather than signatures, but it's rather
      difficult to implement. This patch is a quick fix, which simply
      replaces vclock signature comparison in relay with vclock_compare.
      
      Closes #4106
      4824b00e
    • Vladimir Davydov's avatar
      Revert "replication: update replica gc state on subscribe" · 766cd3e1
      Vladimir Davydov authored
      This reverts commit b5b4809c.
      
      The commit reverted by this patch made relay advance the consumer
      associated with a replica right on subscribe. This is wrong, because the
      current implementation of the garbage collector operates on vclock
      signatures so that if a replica reconnects with a greater signature than
      it had when it last disconnected (e.g. due to replica local changes or
      changes pulled from other members of the cluster), the garbage collector
      may delete WAL files still needed by the replica, breaking replication.
      
      There are two ways to fix this problem. The first and the most difficult
      way is to teach the garbage collector to work with vclocks, i.e. rather
      than simply sorting all consumers by signature and using the smallest
      signature for garbage collection, maintain a vclock each component of
      which is the minimum among corresponding components of all registered
      consumers.
      
      The second (easy) way is to advance a consumer only if its acknowledged
      vclock is greater than or equal to the vclock of a WAL fed to it. This
      way the garbage collector still works with vclock signatures and it's
      a responsibility of the caller (i.e. relay) to ensure that consumers are
      advanced correctly.
      
      I took on the second way for now, because I couldn't figure out an
      efficient way to implement the first. This implies reverting the above
      mentioned commit and reopening #4034 - sporadic replication/gc.test.lua
      failure - which will have to be fixed some other way.
      
      See the next patch for the rest of the fix and the test.
      
      Needed for #4106
      766cd3e1
    • Vladimir Davydov's avatar
      vinyl: don't compress L1 runs · 00b6ea52
      Vladimir Davydov authored
      L1 runs are usually the most frequently read and smallest runs at the
      same time so we gain nothing by compressing them.
      
      Closes #2389
      00b6ea52
    • Vladimir Davydov's avatar
      xlog: cleanup setting of write options · 35fc8afd
      Vladimir Davydov authored
      The way xlog write options (sync_interval and others) are set is a mess:
      if an xlog is created with xlog_create(), we overwrite them explicitly;
      if an xlog is created with xdir_create_xlog(), we inherit parameters
      from the xdir, which sets them depending on the xdir type (SNAP, XLOG,
      or VYLOG), but sometimes we overwrite them explicitly as well. The more
      options we add, the worse it gets.
      
      To clean it up, let's add an auxiliary structure combining all xlog
      write options and pass it to xlog_create() and xdir_create() everywhere.
      35fc8afd
  3. Apr 10, 2019
    • Vladimir Davydov's avatar
      test: fix vinyl/errinj_ddl failure · f41d2999
      Vladimir Davydov authored
      The test fixes the following two test failures:
      
       | --- vinyl/errinj_ddl.result	Tue Mar 19 17:52:48 2019
       | +++ vinyl/errinj_ddl.reject	Tue Mar 19 19:05:36 2019
       | @@ -358,7 +358,7 @@
       | ...
       | s.index.sk:stat().memory.rows
       | ---
       | -- 27
       | +- 23
       | ...
       | test_run:cmd('restart server default')
       | fiber = require('fiber')
      
      This happens, because creation of the test index can happen later than
      we expect. Fix it by adding an appropriate wait_cond.
      
       | --- vinyl/errinj_ddl.result	Tue Mar 19 17:52:48 2019
       | +++ vinyl/errinj_ddl.reject	Tue Mar 19 18:07:55 2019
       | @@ -504,6 +504,7 @@
       | ...
       | _ = s1:create_index('sk', {parts = {2, 'unsigned'}})
       | ---
       | +- error: Tuple field 2 required by space format is missing
       | ...
       | errinj.set("ERRINJ_VY_READ_PAGE_TIMEOUT", 0)
       | ---
      
      This one is due to a test transaction completing before DDL starts so
      that the transaction isn't aborted by DDL, as we expect. Fix it by
      making sure the transaction won't commit before DDL starts, again with
      the aid of wait_cond.
      
       | --- vinyl/errinj_ddl.result     Wed Apr 10 18:59:57 2019
       | +++ vinyl/errinj_ddl.reject     Wed Apr 10 19:05:35 2019
       | @@ -779,7 +779,7 @@
       |  ...
       |  ch1:get()
       |  ---
       | -- Transaction has been aborted by conflict
       | +- Duplicate key exists in unique index 'i1' in space 'test'
       |  ...
       |  ch2:get()
       |  ---
      
      This test case fails, because we use a timeout to stall reading DML
      operations. This was initially a bad call, because under severe load
      (e.g. parallel test run), the timeout may fire before we get to execute
      the DDL request, which is supposed to abort the DML operations, in which
      case they won't be aborted. Fix this by replacing the timeout with a
      delay, as we should have done right from the start.
      
      Closes #4056
      Closes #4057
      f41d2999
    • Vladimir Davydov's avatar
      test: fix vinyl/errinj_stat again · b9a05d09
      Vladimir Davydov authored
      Another failure this time:
      
       | [024] —- vinyl/errinj_stat.result  Wed Apr 10 14:21:34 2019
       | [024] +++ vinyl/errinj_stat.reject  Wed Apr 10 14:24:15 2019
       | [024] @@ -220,7 +220,7 @@
       | [024]  ...
       | [024]  box.snapshot()
       | [024]  —-
       | [024] — error: Error injection 'vinyl dump'
       | [024] +- error: Snapshot is already in progress
       | [024]  ...
       | [024]  stat = box.stat.vinyl().scheduler
       | [024]  —-
       | [024] @@ -231,7 +231,7 @@
       | [024]  ...
       | [024]  stat.tasks_failed > 0
       | [024]  —-
       | [024] — true
       | [024] +- false
       | [024]  ...
       | [024]  errinj.set('ERRINJ_VY_RUN_WRITE', false)
       | [024]  —-
      
      Hope it's the last time we fix it.
      
      Follow-up commit f4b80bcf ("test: fix vinyl/errinj_stat failure").
      b9a05d09
    • Vladimir Davydov's avatar
      test: fix vinyl/errinj_stat failure · f4b80bcf
      Vladimir Davydov authored
      The patch fixes the following test failures:
      
       | [022] --- vinyl/errinj_stat.result	Tue Mar 19 17:52:48 2019
       | [022] +++ vinyl/errinj_stat.reject	Wed Mar 20 08:08:41 2019
       | [022] @@ -229,7 +229,7 @@
       | [022] ...
       | [022] stat.tasks_inprogress == 0
       | [022] ---
       | [022] -- true
       | [022] +- false
       | [022] ...
       | [022] stat.tasks_completed == 1
       | [022] ---
      
       | [013] --- vinyl/errinj_stat.result	Tue Mar 19 17:52:48 2019
       | [013] +++ vinyl/errinj_stat.reject	Wed Mar 20 08:11:15 2019
       | [013] @@ -168,7 +168,7 @@
       | [013] ...
       | [013] stat.tasks_inprogress > 0
       | [013] ---
       | [013] -- true
       | [013] +- false
       | [013] ...
       | [013] stat.tasks_completed == 0
       | [013] ---
       | [013] @@ -183,7 +183,7 @@
       | [013] ...
       | [013] box.stat.vinyl().scheduler.tasks_inprogress > 0
       | [013] ---
       | [013] -- true
       | [013] +- false
       | [013] ...
       | [013] errinj.set('ERRINJ_VY_RUN_WRITE_DELAY', false)
       | [013] ---
      
      The problem occurred, because the test didn't make sure that an
      asynchronous dump/compaction task has actually started/completed.
      Even box.snapshot() doesn't guarantee that a dump task is complete,
      in fact. This patch adds wait_cond's to guarantee the test never
      fails like that anymore.
      
      Closes #4059
      Closes #4060
      f4b80bcf
    • Alexander Tikhonov's avatar
      test: wait for xlog/snap/log file changes · def75c88
      Alexander Tikhonov authored
      When a system in under heavy load (say, when tests are run in parallel)
      it is possible that disc writes stalls for some time. This can cause a
      fail of a check that a test performs, so now we retry such checks during
      60 seconds until a condition will be met.
      
      This change targets replication test suite.
      Unverified
      def75c88
    • Alexander Turenko's avatar
      test: increase timeouts in replication/errinj · e257eb27
      Alexander Turenko authored
      Needed for parallel running of the test suite.
      
      Use default replication_connect_timeout (30 seconds) instead of 0.5
      seconds. This don't changes meaning of the test cases.
      
      Increase replication_timeout from 0.01 to 0.1.
      
      These changes allow to run the test 100 times in 50 parallel jobs
      successfully.
      Unverified
      e257eb27
    • Alexander Turenko's avatar
      test: increase timeouts in replication/misc · 697caa6b
      Alexander Turenko authored
      All changes are needed to eliminate sporadic fails when testing is run
      with, say, 30 parallel jobs.
      
      First, replication_connect_timeout is increased to 30 seconds. This
      parameter doesn't change meaning of the test cases.
      
      Second, increase replication_timeout from 0.01 to 0.03. We usually set
      it to 0.1 in tests, but a duration of the gh-3160 test case ('Send
      heartbeats if there are changes from a remote master only') is around
      100 * replication_timeout seconds and we don't want to make this test
      much longer. Runs of the test case (w/o other ones that are in
      replication/mics.test.lua) in 30 parallel jobs show that 0.03 is enough
      for the gh-3160 case to pass stably and hopefully enough for the
      following test cases too.
      Unverified
      697caa6b
    • Alexander Turenko's avatar
      test: allow to run replication/misc multiple times · 7a2c31d3
      Alexander Turenko authored
      It allows to run `./test-run.py -j 1 replication/misc <...>
      replication/misc` that can be useful when debugging a flaky problem.
      
      This ability was broken after after 7474c14e ('test: enable cleaning of
      a test environment'), because test-run starts to clean package.loaded
      between runs and so each time the test is run it calls ffi.cdef() under
      require('rlimit'). This ffi.cdef() call defines a structure, so a second
      and following attempts to call the ffi.cdef() will give a Lua error.
      
      This commit does not change anything in regular testing, because each
      test runs once (if other is not stated in a configuration list).
      Unverified
      7a2c31d3
    • Vladislav Shpilevoy's avatar
      swim: make UUID update smoother and faster · d98f3d52
      Vladislav Shpilevoy authored
      Before this patch UUID update was the same as introduction of a
      new member and waiting until the 'old' is dropped as 'dead' by
      the failure detection component. It could take 2.5 minutes with
      the default ack timeout. What is more, with GC turned off it
      would always result in never deleted old UUID.
      
      The patch on a UUID update marks the old UUID as 'left' member.
      In the best and most common case it guarantees that old UUID will
      be dropped not later than after 2 complete rounds, and marked as
      'left' everywhere for log(cluster_size) round steps. Even with GC
      turned off.
      
      Part of #3234
      d98f3d52
    • Vladislav Shpilevoy's avatar
      swim: introduce quit message · 3702686e
      Vladislav Shpilevoy authored
      Quit allows to gracefully leave a cluster. Other members will not
      consider the quited instance as dead, and will drop it much
      earlier than it would happen via failure detection.
      
      Quit works as follows: a special message is sent to each member.
      Members, got that message, will mark the source as 'left' and
      will keep and disseminate that change for one round. In the best
      case after one round the left member will be marked as such in
      the whole cluster. 'Left' member will not be added back because,
      it is prohibited explicitly to add new 'left' members.
      
      Part of #3234
      3702686e
    • Vladislav Shpilevoy's avatar
      test: process IO swim test events before protocol's ones · 21705c08
      Vladislav Shpilevoy authored
      Before that patch the swim test event loop worked like this: pop
      a new event, set the global watch to its deadline, process the
      event, repeat until the deadlines are the same. These events
      usually generate IO events, which are processed next. But after
      swim_quit() will be introduced, it is possible to insert new IO
      events before protocol's events like round steps and ack checks.
      
      Because of that it would be impossible to process new IO events
      only, with timeout = 0, or with timeout > 0, but without changing
      the global clock.
      
      For example, a typical test would try to call swim_quit() on a
      swim instance, and expect that it has sent all the quit messages
      without delays immediately. But before this patch it would be
      necessary to run at least one swim round to get to the IO
      processing.
      
      The patch splits protocol's events and IO events processing logic
      into two functions and calls them explicitly in
      swim_wait_timeout() - the main function to check something in the
      swim tests.
      
      Part of #3234
      21705c08
    • Vladislav Shpilevoy's avatar
      test: on close of swim fake fd send its packets, not drop · 3f0a4e25
      Vladislav Shpilevoy authored
      The packets originator has already got an OK status and expects
      these messages sent even if the originator is closed right after
      that. This commit does the TCP-way and sends all the pending
      messages before actually closing the fake fd.
      
      Part of #3234
      3f0a4e25
Loading