Skip to content
Snippets Groups Projects
  1. Apr 22, 2022
  2. Apr 21, 2022
    • Vladimir Davydov's avatar
      alter: move space_format_decode to field_def.h · 12320247
      Vladimir Davydov authored
      So that we can reuse it for online space upgrade.
      
      NO_DOC=refactoring
      NO_TEST=refactoring
      NO_CHANGELOG=refactoring
      12320247
    • Vladimir Davydov's avatar
      alter: raise ER_WRONG_SPACE_FORMAT instead of ER_CREATE/ALTER_SPACE · 6845a705
      Vladimir Davydov authored
      This is needed to factor out the function that decodes space format so
      that it can be used for online space upgrade. It's not a big deal,
      because we already use ER_WRONG_SPACE_OPTIONS if space options specified
      on space creation/alter are wrong so using ER_WRONG_SPACE_FORMAT would
      only be consistent.
      
      NO_DOC=minor change in error message
      NO_CHANGELOG=minor change in error message
      6845a705
    • Vladimir Davydov's avatar
      alter: use ER_WRONG_INDEX_PARTS for wrong parts · 502e4fea
      Vladimir Davydov authored
      Currently, we use ER_WRONG_INDEX_OPTIONS instead, which is confusing.
      Also, let's simplify the error message: remove the reference to the
      deprecated index parts format.
      
      NO_DOC=minor change in error message
      NO_CHANGELOG=minor change in error message
      502e4fea
    • Vladimir Davydov's avatar
      alter: drop field_no from ER_WRONG_XXX_OPTIONS errors · 2a83f9e5
      Vladimir Davydov authored
      opts_decode takes errcode and field_no, which is passed to diag_set,
      so all code paths that use opts_decode must set a field_no in an error
      message. This is stupid, because often there's no reasonable field_no to
      report. As a result, for ER_WRONG_INDEX_OPTIONS, ER_WRONG_SPACE_OPTIONS,
      and ER_WRONG_COLLATION we use either 0 or field_no of the field in a
      system space, which means nothing to users. When there's an error in
      index parts we pass the index part no for field_no ("(field X)" where X
      is an index part number), which is confusing. The only error for which
      field_no makes sense is ER_WRONG_SPACE_FORMAT, but it doesn't justify
      passing around field_no for other errors and complicating common code
      paths.  Let's drop the field_no everywhere for simplicity - the user can
      pin-point the erroneous field without it by inspecting fields/parts.
      
      NO_DOC=minor change in error message
      NO_CHANGELOG=minor change in error message
      2a83f9e5
    • Vladimir Davydov's avatar
      alter: don't malloc space_opts.sql in space_opts_decode · 46b3962e
      Vladimir Davydov authored
      Options created with space_opts_decode are never explictily freed,
      because they are supposed to use only region memory. For some reason,
      we duplicate space_opts.sql in space_opts_decodes, which results in
      a memory leak. Remove this code.
      
      NO_DOC=bug fix
      NO_TEST=memory leak
      NO_CHANGELOG=minor bug
      46b3962e
    • Mergen Imeev's avatar
      sql: fixes for new built-in functions · ba2379f8
      Mergen Imeev authored
      Due to full-ci a few problems were found.
      
      NO_DOC=Fixes for fill-ci
      NO_TEST=Fixes for fill-ci
      NO_CHANGELOG=Fixes for full-ci
      ba2379f8
    • Mergen Imeev's avatar
      sql: introduce built-in function DATE_PART() · 90951de0
      Mergen Imeev authored
      Part of #6773
      
      @TarantoolBot document
      Title: SQL built-in function DATE_PART()
      
      The DATE_PART() function returns the requested information from a
      DATETIME value. It takes two arguments, the first one tells us what
      information is being requested, the second is a DATETIME value. Here is
      a list of supported values of the first argument and what information is
      returned:
      'millennium' - milennium
      'century' - century
      'decade' - decade
      'year' - year
      'quarter' - quarter of year
      'month' - month of year
      'week' - week of year
      'day' - day of month
      'dow' - day of week
      'doy' - day of year
      'hour' - hour of day
      'minute' - minute of hour
      'second' - second of min
      'millisecond' - millisecond of second
      'microsecond' - microsecond of second
      'nanosecond' - nanosecond of second
      'epoch' - epoch
      'timezone_offset' - timezone offset
      90951de0
    • Mergen Imeev's avatar
      sql: introduce built-in function NOW() · 0243cdc0
      Mergen Imeev authored
      Part of #6773
      
      NO_CHANGELOG=It will be added in another commit.
      
      @TarantoolBot document
      Title: SQL built-in function NOW()
      
      The NOW() function returns the current date and time as a DATETIME
      value. If this function is called more than once in a query, it returns
      the same result until the query completes, unless a yield has occurred.
      On yield the value returned by NOW() changes.
      0243cdc0
  3. Apr 20, 2022
    • Ilya Verbin's avatar
      box: fix space creation with fkey pointing to itself · 054a339a
      Ilya Verbin authored
      This patch enables creation of a space with a foreign key pointing
      to the same space. Such spaces are marked with a `selfpin` flag in
      the `space_cache_pin_list`. If a space is pinned only by selfpins,
      it is allowed to be deleted from the space cache.
      
      When an old space is replaced by the new one, it's self-pinned for the
      second time, then all old pins are moved to the new space's pin list
      (see `space_cache_repin_pinned`), then the first self-pin is removed
      by `space_cleanup_constraints` of the old space.
      
      Closes #6961
      
      NO_DOC=bugfix
      054a339a
    • Ilya Verbin's avatar
      box: do tuple_check_constraint only if validate is set · 82bfeffe
      Ilya Verbin authored
      Part of #6961
      
      NO_DOC=bugfix
      NO_CHANGELOG=see later commits
      NO_TEST=see later commits
      82bfeffe
    • Vladimir Davydov's avatar
      sql: drop extern_alloc flag of sql_expr_delete · 63dd2f33
      Vladimir Davydov authored
      It's always false.
      
      I don't understand why it was added in the first place. Initially, it
      was introduced by commit b9c9792d ("sql: move default col values to
      Tarantool's core"). The flag was set to true only in Tarantool core for
      some reason. Then there was commit 7c6c572c ("sql: Fix DEFAULTs AST
      memory leak on alter") that claimed there was a memory leak because the
      flag was set it to true and switched it to false in all places except
      one: when a space_def was destroyed it was still set to true. I dropped
      usage of space_expr_dup and switched the flag to false in the previous
      commit, and everything seems to work just fine now.
      
      NO_DOC=refactoring
      NO_TEST=refactoring
      NO_CHANGELOG=refactoring
      63dd2f33
    • Vladimir Davydov's avatar
      Move default_value_expr from field_def to tuple_field · 3dcc3fd6
      Vladimir Davydov authored
      field_def, like any other _def struct, is supposed to be plain and
      simple - it should map data stored in the _space system spaces as is.
      Complex data structures produced from it, like field dictionary or
      compiled SQL expression should be stored in tuple_field. This simplifies
      reasoning about the lifetime of this structure:
       - If it's allocated on the region, as it is the case, when the
         structure is created from MsgPack data, it will live until the
         region is truncated. It doesn't need to be destroyed in this case.
       - If it's allocated on malloc (by a dup function), it should be
         explicitly deleted with a delete function.
      
      Why we are doing this now: to implement space upgrade, we'll need to add
      a new member to space_opts struct. The new member will be a struct that
      may store an array of field_def. The problem is space_opts created on
      the region from MsgPack data is never deleted (see alter.cc), which
      means that the expression stored in field_def could leak.
      
      Note: in order to avoid tuple -> sql dependency, we have to call
      sql_expr_compile and sql_expr_delete from tuple_format.c via a callback
      set by sql_init.
      
      NO_DOC=refactoring
      NO_TEST=refactoring
      NO_CHANGELOG=refactoring
      3dcc3fd6
  4. Apr 19, 2022
    • Serge Petrenko's avatar
      relay: synchronize raft and promote term dispatch · c842448e
      Serge Petrenko authored
      When processing promote/demote row from wal in relay, process tx
      endpoint until last sent raft term is greater or equal to term in
      promote/demote request.
      
      Part of #6754
      
      @TarantoolBot document
      Title: box.info.synchro.queue.term added
      
      Added `term` field to `box.info.synchro.queue`. It contains term of the
      last PROMOTE. It is usually equal to `box.info.election.term` but may be
      less than election term when new round of elections started, but no one
      promoted yet.
      c842448e
    • Serge Petrenko's avatar
      refactoring: remove relay_restart_recovery as unneeded · 6336ea26
      Serge Petrenko authored
      As stated in its comment, this routine was needed back in the time when
      raft followers ignored data coming from a non-leader. This is not the
      case anymore. Now the followers accept the data from any source, so
      there's no need to try and re-send some data which accumulated while the
      node hasn't yet broadcast its leadership.
      
      Part of #6754
      
      NO_DOC=dead code removal
      NO_CHANGELOG=dead code removal
      NO_TEST=dead code removal
      6336ea26
    • Boris Stepanenko's avatar
      relay: split relay endpoint into two: tx and wal. · 1ddd3f2a
      Boris Stepanenko authored
      Makes it possible to synchronise processing of messages, received from
      tx and read from wal in relay thread.
      
      Part of #6754
      
      NO_DOC=Internal changes
      NO_CHANGELOG=Internal changes
      NO_TEST=Internal changes
      1ddd3f2a
    • Georgiy Lebedev's avatar
      core: introduce parent backtrace collection · 3116c4e7
      Georgiy Lebedev authored
      With backtrace implementation reworked in afc100d, we can now
      efficiently collect unwinding information about parents at fiber
      creation time:
      
      * Add `core_backtrace_cat` helper for concatenating backtraces.
      * Add 'parent_bt' field to `struct fiber` for storing fiber parent's
        backtrace.
      * Add `fiber_parent_backtrace_enabled` and corresponding C/Lua toggles for
        controlling parent backtrace collection for newly created fibers.
      
      Closes #4002
      
      @TarantoolBot document
      Title: new handles for fiber parent backtrace feature
      
      local fiber = require('fiber')
      local log = require('log')
      local yaml = require('yaml')
      
      local function foo()
          log.info("%s", yaml.encode(fiber.info()[fiber.self()[id]].backtrace))
      end
      
      -- Parent backtrace collection feature is disable by default.
      fiber.parent_backtrace_enable()
      -- Backtrace will contain also parent's backtrace.
      fiber.create(foo)
      
      fiber.parent_backtrace_disable()
      -- Backtrace will not contain parent's backtrace.
      fiber.create(foo)
      
      local function bar()
          -- Fibers created before enabling parent backtrace feature will not be
          -- contained in backtrace.
          fiber.parent_backtrace_enable()
          fiber.create(foo)
      end
      
      -- Grandchild will not contain child's backtrace.
      fiber.create(bar)
      3116c4e7
    • Georgiy Lebedev's avatar
      core: re-enable backtrace feature on AARCH64 · ef486a09
      Georgiy Lebedev authored
      After fixing libunwind issues in 7dc9fe44, libcoro fibers issues in
      761053f0 and c8ad49f0, we can now re-enable backtrace feature on
      AARCH64.
      
      For some not investigated reason `unw_backtrace` does not collect any
      ips when "-mbranch-protection" compile feature is enabled: provide a
      workaround for this case.
      
      Closes #6060
      
      NO_DOC=build system
      NO_TEST=build system
      ef486a09
    • Georgiy Lebedev's avatar
      core: refactor backtrace component · 8ce76364
      Georgiy Lebedev authored
      Split backtrace component into two parts: Core and Lua — Core backtrace
      is now completely independent from Lua.
      
      Remove `proc_cache` from backtrace, as libunwind caches unwinding
      information internally, see:
      https://www.nongnu.org/libunwind/man/libunwind(3).html.
      
      Rework backtrace implementation: instead of in-place unwinding, collect
      sufficient unwinding information using fast `unw_backtrace` function in
      order to resolve function names and locations on-demand later on — split
      collection and processing of stack frames.
      
      GNU libunwind does not support macOS platform, so we have to stick with
      LLVM libunwind, which has a lot more limited API: because of this we
      have to use some workarounds.
      
      Needed for #4002
      
      NO_CHANGELOG=refactoring
      NO_DOC=refactoring
      NO_TEST=refactoring
      8ce76364
    • Georgiy Lebedev's avatar
      core: refactor C++ ABI usage · 12bcf7e3
      Georgiy Lebedev authored
      src/lib/core/backtrace.cc is C++ compiled only because it uses
      `abi::__cxa_demangle`: refactor the C++ ABI usage into a separate module,
      provide a C wrapper to it and make the backtrace component to be C
      compiled.
      
      Needed for #4002
      
      NO_CHANGELOG=refactoring
      NO_DOC=refactoring
      NO_TEST=refactoring
      12bcf7e3
    • Georgiy Lebedev's avatar
      core: make backtrace_collect_frame ABI compliant · d1e626d1
      Georgiy Lebedev authored
      In case a context switch is required for backtrace collection, we use
      inline assembly to set up the target fiber's stack: we also call another
      function inside from it — this means potentially any registers could get
      clobbered. In order to follow platform ABIs, we need to add all registers,
      the state of which we do not restore, to the clobber list.
      
      For details, see this discussion: https://stackoverflow.com/questions/37502841/calling-printf-in-extended-inline-asm/37503773#37503773
      Also, check out a similar case in our code base: https://github.com/tarantool/tarantool/blob/bb3ed32166688be0b1410a2971bc8380a47be74c/third_party/coro/coro.c#L128-L137
      
      While we are here, also drop architectures other than officially supported
      ones for context switching.
      
      Needed for #4002
      
      NO_CHANGELOG=internal fix, no behaviour changed
      NO_DOC=internal fix, no behaviour changed
      NO_TEST=clobber list of inline assembly
      d1e626d1
  5. Apr 18, 2022
    • Serge Petrenko's avatar
      test: speedup replication/election_qsync_stress · 9c0496fd
      Serge Petrenko authored
      The test ran for ~40 seconds, and instance logs were filled with
      "on_shutdown triggers timed out" messages on each node shutdown.
      
      This is because the test left a lingering netbox connection with a started
      but not finished insert into a synchronous space with huge
      replication_synchro_quorum. Since the introduction of graceful shutdown
      Tarantool tries to wait for a timeout (3 seconds by default) before
      stopping the instance, when there are unfinished requests.
      There is always an unfinished request in this test by design, so there's
      no point in waiting for its completion.
      Close the connection explicitly to save a lot of time.
      
      The test includes stopping instances 10 times in a loop, so the test was
      slowed down by 30 seconds.
      
      At the moment of writing, the only replication* test, which also has
      this issue, is replication-luatest/no_quorum_test.lua.
      
      This was tested by setting on_shutdown_trigger_timeout to 300 seconds
      and finding hanged tests.
      
      The no_quorum test is deliberately not fixed, since it doesn't make use
      of net_box explicitly, and is mysteriously fixed by reordering rows with
      server creation. So this might be a bug in luatest. See the following
      comment for details:
      https://github.com/tarantool/tarantool/issues/6820#issuecomment-1082914500
      
      Part-of #6820
      
      NO_DOC=test fix
      NO_CHANGELOG=test fix
      9c0496fd
  6. Apr 15, 2022
    • Vladimir Davydov's avatar
      box: init audit log before recovery · b494e762
      Vladimir Davydov authored
      To audit DML events, we need to install triggers on user spaces not only
      when a space is created, but also when it's recovered. Let's move audit
      log initialization before recovery so we can handle it.
      
      NO_TEST=ee
      NO_DOC=refactoring
      NO_CHANGELOG=refactoring
      b494e762
    • Yaroslav Lobankov's avatar
      ci: run sysbench 10 times to get average result · a9915553
      Yaroslav Lobankov authored
      Let's run the sysbench benchmark 10 times in a row to get average
      numbers and minimise fluctuations in results. 10 iterations take about
      1h which is more than an acceptable result.
      
      NO_DOC=ci
      NO_TEST=ci
      NO_CHANGELOG=ci
      a9915553
    • Yaroslav Lobankov's avatar
      ci: run sysbench against 2.8 branch · fa363b6d
      Yaroslav Lobankov authored
      This patch makes the perf_sysbench.yml workflow run against the '2.8'
      branch. Unfortunately, we need to keep the logic for the '2.8' branch
      in the 'master' branch workflow because scheduled workflows cannot be
      run on non-default branches even they reside in them [1].
      
      So 'matrix' is used to support workflow runs against the '2.8' branch.
      
      [1] https://docs.github.com/en/actions/using-workflows/events-that-trigger-workflows#schedule
      
      NO_DOC=ci
      NO_TEST=ci
      NO_CHANGELOG=ci
      fa363b6d
    • Alexander Turenko's avatar
      lua: track memory occupied by runtime tuples · b56e012b
      Alexander Turenko authored
      See the documentation request below for details.
      
      Fixes #5872
      
      @TarantoolBot document
      Title: New `box.runtime.info().tuple` metric
      
      It allows to track memory allocated for tuples on runtime arena. It does
      not count tuples owned by memtx and vinyl, but tracks so called runtime
      tuples. The most common example is a tuple created by the
      `box.tuple.new(<...>)` function.
      
      Example:
      
      ```lua
      tarantool> box.runtime.info().tuple -- 0
      tarantool> box.tuple.new({})
      tarantool> box.runtime.info().tuple -- 160
      tarantool> box.tuple.new({})
      tarantool> box.runtime.info().tuple -- 320
      tarantool> collectgarbage()
      tarantool> box.runtime.info().tuple -- 160
      ```
      b56e012b
  7. Apr 14, 2022
    • Yaroslav Lobankov's avatar
      ci: add publishing perf metrics (perf_sysbench) · 8641990b
      Yaroslav Lobankov authored
      This change adds publishing the performance metrics to InfluxDB database
      measured by the sysbench benchmark.
      
      NO_DOC=ci
      NO_TEST=ci
      NO_CHANGELOG=ci
      8641990b
    • Vladimir Davydov's avatar
      Fix box.session.peer in box.session.on_disconnect triggers · 0ada48d7
      Vladimir Davydov authored
      iproto_connection_close() closes the connection fd and then pushes
      the disconnect message to tx, which runs session disconnect triggers.
      As a result, box.session.peer() return nil, when called from a
      box.session.on_disconnect() trigger, which prevents us from using the
      trigger for auditing the disconnect event in the EE version.
      
      Postponing close() until after the disconnect triggers have finished
      wouldn't help, because on OS X and BSD, in contrast to Linux,
      getpeername() fails if the socket was closed by the other end.
      
      To fix this issue, let's store the peer address in the session meta.
      
      Note, we need to remove box.session.peer() check from box-tap/session
      test. The check was added by commit d68050b9 ("Fix on_disconnect
      trigger"), which fixed a crash on attempt to use box.session.peer() from
      box.session.on_disconnect() trigger. This patch doesn't reintroduce the
      crash, but instead fixes the bug in a more extensive way and adds a
      dedicated test.
      
      Closes #7014
      
      NO_DOC=bug fix
      0ada48d7
    • Alexander Turenko's avatar
      static-build: bump openssl from 1.1.1f to 1.1.1n · 7e5db095
      Alexander Turenko authored
      Just regular update to bring openssl security fixes into tarantool.
      
      Fixes #6947
      
      NO_TEST=security update of a dependency
      NO_DOC=security update of a dependency
      7e5db095
  8. Apr 13, 2022
    • Vladislav Shpilevoy's avatar
      limbo: don't clear txn sync flag in case of fail · f9299f24
      Vladislav Shpilevoy authored
      The limbo cleared TXN_WAIT_SYNC and TXN_WAIT_ACK flags for all
      removed entries - succeeded and failed. For succeeded it is fine.
      For failed it was not.
      
      The reason is that a transaction could be rolled back after a
      successful WAL write but before its waiting fiber wakes up. Then
      on wakeup the fiber wouldn't not see TXN_WAIT_SYNC flag and assert
      that the transaction signature >= 0. It wasn't true for txns
      rolled back due to synchro-reasons like a foreign PROMOTE not
      including this transaction.
      
      The patch makes so a failed transaction keeps its TXN_WAIT_SYNC
      flag so as its owner fiber on wakeup would reach
      txn_limbo_wait_complete(), notice the bad signature, and follow
      the rollback-path.
      
      TXN_WAIT_ACK is dropped, because the transaction owner otherwise
      would try to call txn_limbo_ack() for the transaction even if the
      limbo doesn't belong to the instance anymore.
      
      An alternative solution would be to check signature value for all
      transactions even when journal_entry->res is >= 0. But that would
      slow down the common path even for non-synchro transactions.
      
      Closes #6842
      
      NO_DOC=Bugfix
      f9299f24
    • Vladislav Shpilevoy's avatar
      limbo: assign LSN to entries right after WAL write · 993f0f9a
      Vladislav Shpilevoy authored
      LSN to a limbo entry used to be assigned either before a WAL write
      when the txn was from an applier, or one event loop iteration
      after a WAL write for instance's own transactions.
      
      The latter means that the WAl write callback only did
      fiber_wakeup() on a fiber which is supposed to assign the LSN
      later.
      
      That made possible a bug when a remote PROMOTE was received during
      a local txn WAL write. They were written to WAL in that order. But
      PROMOTE WAL write was handled *before* the txn's WAL write. That
      led to a crash, because PROMOTE processing wasn't ready to the
      limbo having entries without an LSN.
      
      The reason why it happened was that the finished WAL batches, even
      if are sent to WAL far from each other in time, still can be
      processed in TX thread all at once, without yields. They just keep
      stacking in an inter-thread queue until TX thread picks them up.
      If TX thread is slow enough, the WAL batches will form bigger
      "batches of batches" in this WAL->TX queue.
      
      When it happened for a txn + PROMOTE, the txn WAL write trigger
      only called fiber_wakeup() without LSN assign. The PROMOTE WAL
      write trigger was applied right away without yields, didn't meet
      its assumptions, and crashed.
      
      The patch makes an LSN be assigned to a limbo entry right in WAL
      write trigger. The cost of this is to store limbo entry as a
      member in struct txn.
      
      The patch only fixes the issue for PROMOTE covering the older
      transaction. The not covering case is still failing, subject of
      another commit.
      
      A side effect which allowed to make the patch a bit simpler - LSN
      is assigned to all limbo entries now, even to the non-synchro
      ones.
      
      The alternative solution was to create an on WAL write trigger
      for each synchro transaction and store the limbo entry in its data
      field. But that is more complicated. I decided it is time to add
      the entry to the txn. For non-synchro transactions it shouldn't
      add any cost, because the limbo entry is only touched for newly
      allocated txns (which eventually will stop being allocated and
      will only be reused from txn_cache); for synchro transactions on
      their normal path; for all transactions on failure path.
      
      Another alternative solution was to make limbo's latch a
      read-write latch. "Readers" would be new transactions until they
      finish WAL write, "writers" would be PROMOTE, DEMOTE, probably
      ROLLBACK. That way a PROMOTE wouldn't start until all limbo
      entries have LSNs. But that looks like an overkill. At least for
      this issue.
      
      Part of #6842
      
      NO_DOC=Bugfix
      NO_CHANGELOG=To be added later
      993f0f9a
    • Vladislav Shpilevoy's avatar
      limbo: rollback new txns during foreign promote · 5370882f
      Vladislav Shpilevoy authored
      Before the patch a limbo owner could try to add new transaction to
      it while processing a remote PROMOTE. That led to a crash - commit
      of PROMOTE after the WAL write expected that it would empty the
      whole queue, which didn't work for records not having an LSN yet.
      
      Interestingly, all worked fine when PROMOTE was local - it turns
      the limbo into the auto-rollback mode. It didn't happen for
      requests coming from the applier.
      
      This patch extends synchro request execution pipeline - now there
      is a preparation step. It is done both for local and for remote
      synchro requests the same. For PROMOTE it activates auto-rollback
      mode.
      
      DEMOTE might seem the same, but it doesn't look affected. To
      receive a foreign DEMOTE the node would need not to be the queue
      owner. Hence it wouldn't be able to append new synchro txns during
      the DEMOTE WAL write.
      
      Part of #6842
      
      NO_DOC=Bugfix
      NO_CHANGELOG=To be added later
      5370882f
    • Vladislav Shpilevoy's avatar
      luatest: introduce server:wait_vclock_of(other) · 64de84bc
      Vladislav Shpilevoy authored
      It waits for vclock of another server. Save several lines of
      relatively frequently used code. Is going to be used even more in
      following patches.
      
      Needed for #6842
      
      NO_DOC=Refactoring
      NO_TEST=Refactoring
      NO_CHANGELOG=Refactoring
      64de84bc
    • Vladislav Shpilevoy's avatar
      limbo: fix typo in txn_limbo_read_promote() · f1430a68
      Vladislav Shpilevoy authored
      It used the global txn_limbo in one place instead of the limbo
      passed as an argument.
      
      NO_DOC=Refactoring
      NO_TEST=Refactoring
      NO_CHANGELOG=Refactoring
      f1430a68
    • Vladislav Shpilevoy's avatar
      limbo: encapsulate request inside write_de/promote · 38c31ea1
      Vladislav Shpilevoy authored
      box_issue_promote/demote used to make WAL write and synchro
      command apply in 2 separate synchro requests. Soon these APIs are
      going to change a little. This patch moves the request creation
      into the limbo internals so as box wouldn't care about request
      structs.
      
      Box now calls a single txn_limbo_write_promote/demote() function
      which writes and applies a request under the hood.
      
      Part of #6842
      
      NO_DOC=Refactoring
      NO_TEST=Refactoring
      NO_CHANGELOG=Refactoring
      38c31ea1
    • Vladislav Shpilevoy's avatar
      limbo: rename txn_limbo_apply · 47cc4837
      Vladislav Shpilevoy authored
      The new name is txn_limbo_req_commit(). The reason is that the
      future commits will introduce txn_limbo_req_prepare() and
      txn_limbo_req_rollback().
      
      The reason is that at least PROMOTE and DEMOTE requests need to
      signal their start somehow before going into WAL so as new
      synchronous transactions couldn't interfere. It is already done
      in txn_limbo_write_promote() and txn_limbo_write_demote(). But not
      done when a synchro command comes from an applier.
      
      The new pipeline will be that both local and remote synchro
      commands will make a prepare() before WAL write, and then either
      commit() or rollback() of the request + commit/rollback() of the
      limbo state (txn_limbo_begin/commit/rollback).
      
      Essentially, txn_limbo_req_*() will be similar to txn_stmt_*(),
      and txn_limbo_begin/commit/rollback() will be similar to
      txn_begin/commit/rollback().
      
      That is also going to be needed to remove panic() from limbo WAL
      write failures.
      
      Part of #6842
      
      NO_DOC=Refactoring
      NO_TEST=Refactoring
      NO_CHANGELOG=Refactoring
      47cc4837
    • Vladimir Davydov's avatar
      box: add internal box_on_select trigger · cbc3daed
      Vladimir Davydov authored
      The trigger is invoked on min/max/get/select/iterator. The trigger
      callback is passed a context with space, index, iterator type, and key
      (MsgPack array). The trigger is run only if access and sanity checks
      have passed. It will be used for auditing SELECT events in the EE
      version.
      
      NO_TEST=ee
      NO_DOC=internal
      NO_CHANGELOG=internal
      cbc3daed
Loading