Skip to content
Snippets Groups Projects
  1. May 17, 2017
    • Vladimir Davydov's avatar
      vinyl: don't use data left from previous vylog records · cd4ed98b
      Vladimir Davydov authored
      vy_recovery_iterate() doesn't clean vy_log_record before proceeding to
      the next log entry. As a result, a run record passed to the callback by
      vy_recovery_iterate() contains extra info left from the index this run
      is for: index_lsn, index_id and space_id. We use this in gc and backup
      callbacks to format file name. The problem is vy_recovery_iterate() is
      also used internally for log rotation. Including extra keys in records
      doesn't result in writing them to the log file on rotation, because per
      each record type we have a mask of keys corresponding to the record
      (vy_log_key_mask). In order to allow optional keys in vylog, the
      following patch will change the meaning of the mask so that it only
      contains mandatory keys, while a key will be written to the log only if
      its value differs from the default (similarly to request_encode). Thus,
      to avoid writing keys not relevant to a record type we need to clean
      vy_log_record within vy_recovery_iterate() before jumping to the next
      record. So this patch weans gc and backup from exploiting this feature -
      let them save index_id and space_id in the context, as we do on
      replication and recovery.
      cd4ed98b
    • Vladislav Shpilevoy's avatar
      Implement info_append_double · 24707c0c
      Vladislav Shpilevoy authored
      Info_append_double is needed to print vinyl index.info.bloom_fpr later.
      24707c0c
    • Vladimir Davydov's avatar
      vinyl: zap vy_index::path · 124bac9a
      Vladimir Davydov authored
      It was helpful when a vinyl index could have a custom path. Currently,
      it's forbidden, so we can format index path in place.
      124bac9a
    • Vladimir Davydov's avatar
      vinyl: zap vy_index::name · ae80dbc1
      Vladimir Davydov authored
      We can format it in place when needed.
      ae80dbc1
    • Vladimir Davydov's avatar
      vinyl: introduce quota timeout · 70a610d1
      Vladimir Davydov authored
      There must be a reasonable timeout on quota wait time, otherwise we risk
      throttling a client forever, e.g. in case of a disk error. This patch
      introduces quota timeout. The timeout is configured via vinyl_timeout
      configuration option and set to 60 seconds by default.
      
      Closes #2014
      70a610d1
    • Vladimir Davydov's avatar
      vinyl: split quota callback function · 883f16a7
      Vladimir Davydov authored
      Use separate function for each kind of callback. The next patch will add
      an argument and a return value to the throttle callback, so it isn't apt
      anymore to use the only callback to handle all cases.
      883f16a7
    • Vladimir Davydov's avatar
      vinyl: reserve quota before using memory · c8ea5fc7
      Vladimir Davydov authored
      Currently, we reserve quota after allocating memory, which can result in
      exceeding the memory limit, e.g. after the following script
      
          box.cfg{vinyl_memory = 1024 * 1024}
          s = box.schema.space.create('test', {engine = 'vinyl'})
          s:create_index('pk')
          pad = string.rep('x', 2 * box.cfg.vinyl_memory / 3)
          s:auto_increment{pad}
          s:auto_increment{pad}
      
      is done, box.info.vinyl().memory.used reports that 1447330 bytes are
      allocated. Fix this by reserving quota before allocation. A test is
      added later in the series.
      c8ea5fc7
    • Vladimir Davydov's avatar
      vinyl: add test checking that we don't exceed quota on recovery · c06fc47a
      Vladimir Davydov authored
      Vinyl dump is disabled during local recovery from WAL. This is OK,
      because we shouldn't exceed the quota provided we don't replay
      statements that were dumped to disk before restart. Check that.
      c06fc47a
  2. May 16, 2017
  3. May 15, 2017
    • Vladimir Davydov's avatar
      vinyl: fix index->stmt_count incremented twice on upsert · 13dd8a41
      Vladimir Davydov authored
      Although upsert optimizers (both sync and background) replace the
      last UPSERT with a REPLACE and do not insert new statements, they
      use vy_index_insert_stmt(), which increments index->stmt_count.
      As a result, index->stmt_count is incremented twice.
      
      Closes #2421
      13dd8a41
  4. May 12, 2017
    • Roman Tsisyk's avatar
    • Roman Tsisyk's avatar
      Disable buggy test/unit/guard.test · 808fe987
      Roman Tsisyk authored
      See #2429
      808fe987
    • Vladislav Shpilevoy's avatar
      1dcc60c0
    • Roman Tsisyk's avatar
      Debian: bump tarantool-common dependency · a89cc747
      Roman Tsisyk authored
      Force tarantool-common upgrade to support systemd notifications.
      
      + Add missing "s" suffix to TimeoutStartSec= option.
      
      See #1923
      a89cc747
    • Roman Tsisyk's avatar
      vinyl: add a workaround for cache · 2f8e0204
      Roman Tsisyk authored
      Check that there are no statements between prev_stmt and stmt
      in the cache on vy_cache_add() before trying to build a chain.
      This workaround makes vy_cache_add() more fool-proof for cases
      when vy_read_iterator skips some keys during restoration.
      2f8e0204
    • Vladislav Shpilevoy's avatar
      vinyl: fix error in mem_restore on prepared statement · a0db136e
      Vladislav Shpilevoy authored
      Between prepare and commit of a transaction it is possible
      that some prepared statements are read from mem.
      Prepared statement has abnomal lsn > MAX_LSN.
      
      If version of the index is changed after return of a
      prepared statement and the iterator is placed to read view,
      the restart and restore on a such prepared statement could
      lead to restore on LSN bigger than vlsn.
      
      Example:
      
              FIBER 1                        FIBER 2
             box.begin()
       replace {1}, lsn=MAX_LSN+1
       replace {2}, lsn=MAX_LSN+1
          prepare vy_tx ...
                              ->->->      open iterator
                                    read {1},lsn=MAX_LSN+1 from mem
                                           yield ...
                              <-<-<-
          commit vy_tx
       replace {1, 1},lsn=100
      send iterator to read view
                              ->->->
                                       iterator read view = 100
                                index version changed - restore on last_stmt
                                    last_stmt = {1},lsn=MAX_LSN+1
                                    last_stmt LSN > iterator VLSN
                                       ??????????????????????
      
      Lets return in such situations the next key.
      a0db136e
    • Vladislav Shpilevoy's avatar
      vinyl: fix mem restore in case of not started mem iterator · d21971cd
      Vladislav Shpilevoy authored
      If vy_mem_iterator was not started, the vy_mem_iterator_restore had been
      able to iterate to the same key, as the target of restore, and then call
      vy_mem_iterator_next_lsn_impl to find the older lsn.
      
      But the vy_mem_iterator_start_from did not set 'search_started' flag and
      next_lsn_impl would restart the mem iterator again, to the first statement,
      which could be incorrect result of restore.
      
      Example:
      
      cache statements: {1}, {2}, {3}
      mem statements: {1}, {2}, {3}
      iterator: GE
      key: {1}
      
      - merge_iterator returns all cached data and then tries to restore mem on {3}.
      - then mem_iterator during restore calls vy_mem_iterator_start_from and sees tuple {3}.
      - then mem_iterator tries to find older lsn and calls vy_mem_iterator_next_lsn_impl,
        which sees 'search_started = false' and restarts iterator to {1}.
      - then read_iterator returns {1} after {3} - ERROR!
      
      Lets set 'search_started' flag inside vy_mem_iterator_start_from.
      d21971cd
    • Vladislav Shpilevoy's avatar
      vinyl: fix return of same statements from read_iterator · a03a4a99
      Vladislav Shpilevoy authored
      Under heavy load it is possible, that mem_iterator_restore
      inside read_iterator_next returns statement with the same key,
      as the previous one.
      
      Check keys in vy_read_iterator_merge_next_key.
      a03a4a99
    • Vladislav Shpilevoy's avatar
      vinyl: fix restore of started mem_iterator to not null last_stmt · 5f5b63d0
      Vladislav Shpilevoy authored
      If the mem_iterator is started, but hasn't any curr_stmt (maybe it
      was finished), it could return its first statement regardless of last_stmt.
      
      Lets call start_from() instead of start() in case of not null last_stmt.
      5f5b63d0
    • Alexandr Lyapunov's avatar
      vinyl: fix restoration of mem iterator and avoid unnecessary restore calls. · 9f341871
      Alexandr Lyapunov authored
      Mem iterator restoration in case of LE and LT was completelly wrong.
      Additionally I added more asserts and found that restoration is
      called too frequently.
      Fix'em both.
      
      fix #2207
      9f341871
    • bigbes's avatar
    • bigbes's avatar
      Adding table.deepcopy() · b1d805d3
      bigbes authored
      b1d805d3
    • Vladimir Davydov's avatar
      vinyl: fix quota watermark calculation · a74219d5
      Vladimir Davydov authored
      The code calculating the quota watermark was written long ago, when we
      didn't have the common memory level and didn't use lsregion allocator
      and hence it was possible to free all memory used by any range. Things
      have changed drastically since then: now it's impossible to free memory
      occupied by a range or even an index, because statements are shared
      between indexes, so the scheduler effectively dumps all memory on
      exceeding the quota. Fix the quota calculation accordingly.
      a74219d5
    • Vladimir Davydov's avatar
      vinyl: yield properly · aa0ebf46
      Vladimir Davydov authored
      We should use fiber_sleep(0) to yield periodically, not
      fiber_reschedule() - the latter doesn't advance the event loop.
      aa0ebf46
    • Roman Tsisyk's avatar
      03882982
    • Georgy Kirichenko's avatar
      Unit test for fibers stack guards · 826ccd20
      Georgy Kirichenko authored
      826ccd20
    • Vladimir Davydov's avatar
      vinyl: add test for partial dump · e5f6f0d1
      Vladimir Davydov authored
      Check that we correctly recover in case not all indexes of the same
      space were dumped.
      e5f6f0d1
    • Vladimir Davydov's avatar
      vinyl: increment ref counter of pinned index · 40989528
      Vladimir Davydov authored
      We remove the primary index from the scheduler while we are dumping a
      secondary index of the same space ('pin' it). On dump task completion we
      add it back. By that time, the primary index could have been dropped, in
      which case we may get use-after-free in vy_scheduler_unpin_index(). To
      avoid it we should increment a reference counter of a pinned index.
      40989528
    • Vladimir Davydov's avatar
      errinj: ERRINJ_U64 => ERRINJ_INT · ff064d10
      Vladimir Davydov authored
      I need to set errinj parameter to a negative value (errinj.param < 0 -
      errinj is disabled, errinj.param = X >= 0 - errinj is enabled for X).
      I don't think it's a good idea to introduce int64_t in addtion to
      uint64_t, because int64_t should be enough for error injection. So,
      let's change ERRINJ_U64 (and errinj.u64param) to ERRINJ_INT
      (errinj.iparam) of type int64_t.
      ff064d10
    • Vladimir Davydov's avatar
      vinyl: use double for timeout injections · 685ec3d7
      Vladimir Davydov authored
      Use double instead of uint64 as an argument of the scheduler and squash
      timeout injections, because they are converted to double anyway. Also,
      rename ERRINJ_VINYL_SCHED_TIMEOUT to ERRINJ_VY_SCHED_TIMEOUT to match
      other vinyl error injections.
      685ec3d7
  5. May 11, 2017
    • Vladimir Davydov's avatar
      vinyl: get rid of vy_log_record->signature · 405608a5
      Vladimir Davydov authored
      Each record of the metadata log has signature. Normally (except for
      VY_LOG_FORGET_RUN), signature is assigned to a record when it is written
      to the log (by vy_log_write()) - then it is set to the signature of the
      last checkpoint. VY_LOG_FORGET_RUN records inherit the signature of the
      corresponding VY_LOG_DROP_RUN.
      
      The signature is used for two purposes:
      
       1. Loading snapshot. Vylog does not use separate files for snapshot and
          log, instead it appends records written after a checkpoint to the
          file created by the checkpoint. The appended records will have
          signature equal to the signature of the checkpoint while older
          records will have signatures of previous checkpoints, so we can
          filter out records corresponding to the checkpoint.
      
       2. Garbage collection. The box.internal.gc.run() method takes LSN of
          the oldest snapshot to save as an argument. The signature allows us
          to filter runs that were deleted before a given LSN.
      
      Actually, #1 doesn't need a signature - instead we could write a marker
      of the end of the snapshot to the vylog file, and it would be easier to
      comprehend. So, the signature is really needed only by #2, but then
      there's no point in appending it to each record, because garbage
      collection only affects runs, i.e. only VY_LOG_DROP_RUN needs it.
      
      So this patch removes the signature and instead introduces:
      
       - a new record type VY_LOG_SNAPSHOT, which is written right after the
         last record of the snapshot;
      
       - a new key VY_LOG_KEY_GC_LSN exclusive for VY_LOG_DROP_RUN records,
         which is set to the LSN of the last checkpoint using the deleted run.
      
      With this patch vy_recovery_new() has two arguments, 'signature' and
      'only_snapshot' flag: 'signature' is the signature of the vylog file to
      load (basically, the file name); 'only_snapshot' is a flag indicating
      that only records corresponding to the snapshot must be loaded. Also,
      run files that should be preserved by garbage collection are now
      filtered by vy_collect_garbage() using the new key (not by signature in
      vy_recovery_new() as before).
      405608a5
    • Vladimir Davydov's avatar
      vinyl: move vy_scheduler definition to the top · d66f637a
      Vladimir Davydov authored
      Since the vy_scheduler struct is defined in the middle of the source
      file, we have to pass scheduler->task_pool and scheduler->generation to
      vy_task_dump_new, vy_task_compact_new, vy_index_rotate_mem, and
      vy_tx_write_prepare instead of accessing them directly, although we do
      have a pointer to the scheduler in these functions and even call its
      methods there. In the next patch I'm running into this again. I grew
      tired of it, let's instead move the definition of the vy_scheduler
      struct closer to the beginning of the file.
      d66f637a
    • Vladimir Davydov's avatar
      vinyl: log all deletions explicitly · 803a8428
      Vladimir Davydov authored
      Currently, a VY_LOG_DELETE_RANGE record implies VY_LOG_DELETE_SLICE for
      each slice of the deleted range and a VY_LOG_DROP_INDEX record implices
      VY_LOG_DELETE_RANGE and VY_LOG_DROP_RUN for each range and run of the
      dropped index. A bad thing about it is that vylog doesn't behave like a
      space and therefore it will be difficult to convert it to one once
      generic metadata management (aka xctl) is introduced. Let's log all
      deletions explicitly.
      803a8428
    • Vladimir Davydov's avatar
      vinyl: ensure dropped indexes are not removed from vy_recovery until rotation · 6b2f0acd
      Vladimir Davydov authored
      Currently, a vy_index_recovery_info object can be freed when vy_recovery
      is loaded - see
      
        vy_recovery_new
          vy_recovery_process_record
            vy_recovery_drop_index
      
      This happens if the corresponding index is dropped (VY_LOG_DROP_INDEX)
      and it has no ranges/slices/runs. Currently, this can only happen if log
      rotation took place after the index was dropped, because an index always
      has at least one range and VY_LOG_DELETE_RANGE isn't written before
      VY_LOG_DROP_INDEX (it's assumed implicitly). The following patch is
      going to require VY_LOG_DELETE_RANGE to be written explicitly before
      VY_LOG_DROP_INDEX (and VY_LOG_DELETE_SLICE before VY_LOG_DELETE_RANGE as
      well) to make vylog similar to a table. So we must postpone an index
      deletion from vy_recovery until rotation, otherwise the following piece
      of code executed right before restart
      
        s = box.schema.space.create('test', {engine = 'vinyl'})
        s:create_index('pk')
        s:drop()
      
      will result in a recovery error:
      
        can't initialize storage: Invalid VYLOG file: Index 6005 not registered
      
      because vy_recovery won't contain the dropped index on WAL replay.
      6b2f0acd
    • Vladimir Davydov's avatar
      vinyl: use the same id counter for ranges, runs, and slices · f2339171
      Vladimir Davydov authored
      There's no point in maintaining a separate counter per each of them.
      f2339171
    • Nick Zavaritsky's avatar
      Add a guard page to the end fiber stack · f3155daa
      Nick Zavaritsky authored
      mprotect() the last 4kB page of a 64kB fiber stack in order to detect
      stack overflows.
      
      Closes #2411
      f3155daa
    • bigbes's avatar
      systemd notify related bug fix · f2d1b295
      bigbes authored
      On older version of systemd (systemd 219) 0 means infinity on newer
      version. To make one script compatible with both versions we'll replace
      infinity with 86400
      
      error message:
      ```
      May 11 10:09:42 jessie systemd[1]:
      [/lib/systemd/system/tarantool@.service:57] Failed to parse usec_t
      value, ignoring: infinity
      ```
      f2d1b295
    • Vladimir Davydov's avatar
      vinyl: fix run leak when compacted slices share the same run · 499f33fe
      Vladimir Davydov authored
      If a range has two or more slices of the same run (this can happen after
      coalescing) and there are no slices of the run in other ranges, we will
      leak the run on compaction, because slice->run->slice_count will be
      greater than 1.
      499f33fe
    • Vladimir Davydov's avatar
      vinyl: turn vy_run->slice_count into reference counter · 4c05f5e1
      Vladimir Davydov authored
      That is, delete the run automatically once it hits 0. This simplifies
      slice deletion - we don't need to check against run->slice->slice_count
      and delete run manually any more. A run is usually referenced only by
      slices created for it. The only exception is dump/compaction task
      creating a run - it keeps the reference counter elevated until it is
      completed.
      4c05f5e1
    • Konstantin Osipov's avatar
Loading