Skip to content
Snippets Groups Projects
  1. Jun 07, 2018
    • Vladimir Davydov's avatar
      vinyl: do not yield on dump completion · 6605a5a4
      Vladimir Davydov authored
      The fact that we may yield after we added a new slice created by dump,
      but before we removed the dumped in-memory index from the LSM tree
      complicates read iterator logic, as it has to detect such a case and
      filter out sources that contain duplicates. This logic relies on the
      fact that no two slices of the same range intersect by LSN. For the
      sake of ALTER we have to relax this limitation, as statements inserted
      during index build can have arbitrary (not monotonically growing) LSNs,
      so the no-LSN-intersection property won't be fulfilled for whole slices,
      only for individual keys. Since there shouldn't be more than 1000 ranges
      in the same LSM tree, yielding doesn't make much sense as iteration over
      the whole range tree should be pretty fast. Besides, dump isn't done
      frequently. That said, let's remove yielding altogether.
      
      Needed for #1653
      6605a5a4
    • Vladimir Davydov's avatar
      vinyl: factor out vy_check_is_unique_secondary · 93f71dae
      Vladimir Davydov authored
      We need to check unique constraint when building a new index. So let's
      factor out this helper function and pass space_name, index_name, and
      read view to it explicitly (because index_name_by_id isn't going to work
      for an index that is under construction and there's no tx when we are
      building a new index). Suggested by @Gerold103.
      
      Needed for #1653
      93f71dae
    • Vladislav Shpilevoy's avatar
      session: enable box.session.push in local console · 5bb2fb7f
      Vladislav Shpilevoy authored
      It is quite simple - just use stdout file descriptor as the
      destination for push messages. It is needed to make remote and
      local console be similar.
      5bb2fb7f
  2. Jun 06, 2018
    • Vladislav Shpilevoy's avatar
      session: introduce text box.session.push · 6d99d51e
      Vladislav Shpilevoy authored
      box.session.push allows to send some intermediate results in the
      scope of main request with no finalizing it. Messages can be
      sent over text and binary protocol. This patch allows to send
      text pushes.
      
      Text push is a YAML document tagged with '!push!' handle and
      'tag:tarantool.io/push,2018' prefix. YAML tags is a standard way
      to define a type of the document.
      
      Console received push message just prints it to the stdout (or
      sends to a next console, if it is remote console too).
      
      Part of #2677
      6d99d51e
    • Konstantin Osipov's avatar
      diag: preserve errno in diag_set() and sio_socketname() · a79ae4ba
      Konstantin Osipov authored
      malloc() or getsockname() may reset the errno.
      Let's ensure we preserve errno when setting the diagnostics.
      a79ae4ba
    • Vladimir Davydov's avatar
      vinyl: simplify index hash building during vylog recovery · 68f9f39f
      Vladimir Davydov authored
      For the sake of recovery, we keep all vy_lsm_recovery_info objects in
      vy_recovery::index_id_hash, which maps space_id/index_id to the latest
      index incarnation. We build this hash while loading records from vylog,
      which is rather difficult for understanding, because an LSM tree may
      have a counterpart created by incomplete ALTER (see ->prepared) so we
      have to link and unlink those objects very carefully. Let's simplify the
      procedure by postponing hash building until we have loaded all records.
      68f9f39f
    • Vladimir Davydov's avatar
      vinyl: allow to commit statements to mem in arbitrary order · f2e30507
      Vladimir Davydov authored
      vy_mem_commit_stmt() expects statements to be committed in the order
      of increasing LSN. Although this condition holds now, it won't once
      we start using this function for building indexes. So let's remove
      this limitation.
      
      Needed for #1653
      f2e30507
    • Konstantin Osipov's avatar
      47105cd9
    • Vladimir Davydov's avatar
      vinyl: bump mem version after committing statement · 2ae0902e
      Vladimir Davydov authored
      Since commit 1e1c1fdb ("vinyl: make read iterator always return
      newest tuple version") vinyl read iterator guarantees that any tuple it
      returns is the newest version in the iterator read view. However, if we
      don't bump mem version after assigning LSN to a mem statement, a read
      iterator using committed_read_view might not see it and return a stale
      tuple. Currently, there's no code that relies on this iterator feature,
      but we will need it for building new indexes. Without this patch, build
      (introduced later in the series) might return inconsistent results.
      
      Needed for #1653
      2ae0902e
    • Vladimir Davydov's avatar
      vinyl: log new index before WAL write on DDL · dd0827ba
      Vladimir Davydov authored
      Currently, we write new indexes to vylog only after successful WAL
      write (see vinyl_index_commit_create). This is incompatible with space
      ALTER - the problem is during ALTER vinyl may need to create new run
      files, which we need to track in order not to leave garbage if ALTER
      fails or tarantool exits before ALTER is complete.
      
      So this patch splits index creation in two stages, prepare and commit.
      The 'commit' stage is represented by existing VY_LOG_CREATE_LSM record,
      which is written from index_vtab::commit_create callback, just like
      before. For the 'prepare' stage we introduce a new record type,
      VY_LOG_REPARE_LSM, written from index_vtab::add_primary_key and
      index_vtab::build_index callbacks, i.e. before WAL write. For now, we
      don't write anything to prepared, but uncommitted indexes (this will be
      done later), but we do add prepared indexes to the scheduler so that
      they can be dumped and compacted. If ALTER fails, we drop prepared
      indexes in index_vtab::abort_create callback. Prepared but uncommitted
      indexes are ignored by backup and replication and cleaned up from vylog
      on restart.
      
      Note, we have to rework vinyl/errinj_vylog test in this patch, because
      index creation (and hence space truncation) commands now fail on vylog
      error, i.e. a situation when the same index is dropped and recreated
      multiple times in xlog without having corresponding records in vylog is
      now impossible.
      
      Also, space truncation is not linearizable for vinyl anymore as it may
      yield before WAL write, while trying to prepare an index in vylog. This
      is OK - we never promised it is. Remove the corresponding test case.
      
      Needed for #1653
      dd0827ba
  3. Jun 04, 2018
    • Vladimir Davydov's avatar
      vinyl: apply box.cfg.snap_io_rate_limit to dump/compaction · 62830170
      Vladimir Davydov authored
      Vinyl worker threads can consume all disk bandwidth while performing
      dump or compaction, thus stalling DML requests, which also need some
      disk bandwidth for WAL. Memtx has a similar problem - it needs to write
      snapshot files. In case of memtx, we cope with this problem by limiting
      the write rate with box.cfg.snap_io_rate_limit option. Let's reuse this
      option for limiting vinyl dump/compaction rate.
      
      Closes #3220
      62830170
    • Vladimir Davydov's avatar
      xlog: use ev_sleep instead of fiber_sleep for rate limiting · 5d5f0d1c
      Vladimir Davydov authored
      fiber_sleep() works only if the current thread was created with
      cord_costart(). Since vinyl worker threads don't need fibers, they
      are created with cord_start() and hence can't use fiber_sleep().
      So to be able to limit rate of vinyl dump/compaction, we have to
      use ev_sleep() instead of fiber_sleep() in xlog. This is fine by
      other xlog writers, because they don't use fibers either, neither
      they should as xlogs are written without coio.
      
      Needed for #3220
      5d5f0d1c
  4. Jun 02, 2018
  5. Jun 01, 2018
    • Vladimir Davydov's avatar
      box: allow to specify the checkpoint to backup · d2272132
      Vladimir Davydov authored
      Even though we store last box.cfg.checkpoint_count checkpoints, we can
      only restore to the last one, because the backup API only allows to
      backup the last checkpoint. This patch adds an optional argument to
      box.backup.start() which specifies the index of the checkpoint to
      backup. If it is omitted or is 0, box.backup.start() will return files
      corresponding to the last checkpoint. If it is 1, it will back the
      previous checkpoint, and so on.
      
      Closes #3410
      d2272132
    • Vladimir Davydov's avatar
      box: add sanity check for memtx_memory and vinyl_memory cfg options · 951a03de
      Vladimir Davydov authored
      Follow-up #2634
      Closes #3439
      951a03de
    • Konstantin Osipov's avatar
    • Vladimir Davydov's avatar
      Rename index.info to index.stat · 04b790fd
      Vladimir Davydov authored
      index.info() is about statistics so let's rename it to index.stat().
      Since index.info() is not documented, we don't need to leave the old
      alias.
      
      Follow-up #3277
      04b790fd
    • Vladimir Davydov's avatar
      Move box.info.vinyl to box.stat.vinyl · b9119f2e
      Vladimir Davydov authored
      All global statistics are meant to be in box.stat module so let's move
      box.info.vinyl to box.stat.vinyl, but leave the old alias - we will
      remove it in 2.0. Patch all tests to use the new alias.
      
      Closes #3277
      b9119f2e
    • Konstantin Osipov's avatar
    • Vladimir Davydov's avatar
      vinyl: fix compaction vs checkpoint race resulting in invalid gc · b25e3168
      Vladimir Davydov authored
      The callback invoked upon compaction completion uses checkpoint_last()
      to determine whether compacted runs may be deleted: if the max LSN
      stored in a compacted run (run->dump_lsn) is greater than the LSN of the
      last checkpoint (gc_lsn) then the run doesn't belong to the last
      checkpoint and hence is safe to delete, see commit 35db70fa ("vinyl:
      remove runs not referenced by any checkpoint immediately").
      
      The problem is checkpoint_last() isn't synced with vylog rotation - it
      returns the signature of the last successfully created memtx snapshot
      and is updated in memtx_engine_commit_checkpoint() after vylog is
      rotated. If a compaction task completes after vylog is rotated but
      before snap file is renamed, it will assume that compacted runs do not
      belong to the last checkpoint, although they do (as they have been
      appended to the rotated vylog), and delete them.
      
      To eliminate this race, let's use vylog signature instead of snap
      signature in vy_task_compact_complete().
      
      Closes #3437
      b25e3168
    • Vladimir Davydov's avatar
      box: make gc info public · e9e927fe
      Vladimir Davydov authored
      Information about garbage collection internal state is quite useful when
      it comes to answering the question why my xlog files are not removed so
      let's move it from box.inernal.gc.info() to box.info.gc().
      e9e927fe
    • Konstantin Osipov's avatar
    • Konstantin Osipov's avatar
      port: ensure Lua port can be re-used · 18d87adf
      Konstantin Osipov authored
      Prepare port_lua for reuse in box.session.push()
      Ensure port_lua_dump* can be used with an unprepared
      Lua stack by moving the push of encoder from box_lua_call()
      to port_lua_dump().
      
      Use tarantool_L as a temporary lua stack to run the encoder.
      Remove encode_ctx and reuse the port state to pass
      parameters to the encoder.
      18d87adf
  6. May 31, 2018
    • Vladislav Shpilevoy's avatar
      port: rename dump() into dump_msgpack() · e4d3825d
      Vladislav Shpilevoy authored
      In #2677 a port must be able to dump both msgpack into obuf for
      binary box.session.push(), and text with no obuf for text
      box.session.push.
      e4d3825d
    • Vladislav Shpilevoy's avatar
      session: introduce session vtab and meta · 145f72d7
      Vladislav Shpilevoy authored
      box.session.push implementation depends on session type -
      console session must send YAML tagged text, binary session must
      send MessagePack via another thread, other sessions must return
      error.
      
      Add virtual table to a session with 'push', 'fd' and 'sync'
      functions.
      
      The same virtual table together with struct session meta can be
      used to use memory of struct session more effectively. Before the
      patch session stored sync and fd as attributes, but:
      * fd was duplicated for iproto, which already has fd in
        connection;
      * sync is used only by iproto, and just occupies 8 byte in other
        sessions;
      * after the #2677 session additionaly must be able to store
        iproto connection pointer.
      
      Struct session meta uses C union to store either iproto, or
      console, or another meta, but not all of them together.
      
      Part of #2677
      145f72d7
    • Vladislav Shpilevoy's avatar
      console: use Lua C API to do formatting for console · ac715203
      Vladislav Shpilevoy authored
      YAML formatting C API is needed for #2677, where it will be used
      to send text pushes and prints as tagged YAML documents.
      
      Push in the next patches is implemented as a virtual C method of
      struct session, so it is necessary to be able format YAML from C.
      
      Needed for #2677
      ac715203
    • Vladislav Shpilevoy's avatar
      session: move salt into iproto connection · 860d6b3f
      Vladislav Shpilevoy authored
      Session salt is 32 random bytes, that are used to encode password
      when a user is authorized. The salt is not used in non-binary
      sessions, and can be moved to iproto connection.
      860d6b3f
    • Vladislav Shpilevoy's avatar
      yaml: introduce yaml.decode tag_only option · 567769d0
      Vladislav Shpilevoy authored
      Yaml.decode tag_only option allows to decode a single tag of a
      YAML document. For #2677 it is needed to detect different push
      types in text console: print pushes via console.print, and actual
      pushes via box.session.push.
      
      To distinguish them YAML tags will be used. A client console for
      each message will try to find a tag. If a tag is absent, then the
      message is a simple response to a request.
      
      If a tag is !print!, then the document consists of a single
      string, that must be printed. Such a document must be decoded to
      get the printed string. So the calls sequence is
      yaml.decode(tag_only) + yaml.decode. The reason why a print
      message must be decoded is that a print() result on a server side
      can be not well-formatted YAML, and must be encoded into it to be
      correctly sent. For example, when I do on a server side something
      like this:
      
      console.print('very bad YAML string')
      
      The result of a print() is not a YAML document, and to be sent it
      must be encoded into YAML on a server side.
      
      If a tag is !push!, then the document is sent via
      box.session.push, and must not be decoded. It can be just printed
      or ignored or something.
      
      Needed for #2677
      567769d0
    • Vladislav Shpilevoy's avatar
      lua: merge encode_tagged into encode options · b2da28f8
      Vladislav Shpilevoy authored
      Encode_tagged is a workaround to be able to pass options to
      yaml.encode().
      
      Before the patch yaml.encode() in fact has this signature:
      yaml.encode(...). So it was impossible to add any options to this
      function - all of them would be treated as the parameters. But
      documentation says: https://tarantool.io/en/doc/1.9/reference/reference_lua/yaml.html?highlight=yaml#lua-function.yaml.encode
      that the function has this signature: yaml.encode(value).
      
      I hope if anyone uses yaml.encode(), he does it according to the
      documentation. And I can add the {tag_prefix, tag_handle} options
      to yaml.encode() and remove yaml.encode_tagged() workaround.
      b2da28f8
    • Vladimir Davydov's avatar
      vinyl: fix false-positive assertion at exit · ff02157f
      Vladimir Davydov authored
      latch_destroy() and fiber_cond_destroy() are basically no-op. All they
      do is check that latch/cond is not used. When a global latch or cond
      object is destroyed at exit, it may still have users and this is OK as
      we don't stop fibers at exit. In vinyl this results in the following
      false-positive assertion failures:
      
        src/latch.h:81: latch_destroy: Assertion `l->owner == NULL' failed.
      
        src/fiber_cond.c:49: fiber_cond_destroy: Assertion `rlist_empty(&c->waiters)' failed.
      
      Remove "destruction" of vy_log::latch to suppress the first one. Wake up
      all fibers waiting on vy_quota::cond before destruction to suppress the
      second one. Add some test cases.
      
      Closes #3412
      ff02157f
  7. May 30, 2018
    • Vladislav Shpilevoy's avatar
      yaml: introduce yaml.encode_tagged · ddcd95a0
      Vladislav Shpilevoy authored
      Encode_tagged allows to define one global YAML tag for a
      document. Tagged YAML documents are going to be used for
      console text pushes to distinguish actual box.session.push() from
      console.print(). The first will have tag !push, and the
      second - !print.
      ddcd95a0
    • Vladimir Davydov's avatar
      engine: constify vclock argument · 088e3e24
      Vladimir Davydov authored
      None of engine_wait_checkpoint, engine_commit_checkpoint, engine_join,
      engine_backup needs to modify the vclock argument.
      088e3e24
Loading