- May 17, 2017
-
-
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.
-
Vladislav Shpilevoy authored
Info_append_double is needed to print vinyl index.info.bloom_fpr later.
-
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.
-
Vladimir Davydov authored
We can format it in place when needed.
-
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
-
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.
-
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.
-
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.
-
- May 16, 2017
-
-
Alexandr Lyapunov authored
-
- May 15, 2017
-
-
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
-
- May 12, 2017
-
-
Roman Tsisyk authored
See #2429
-
Vladislav Shpilevoy authored
-
Roman Tsisyk authored
Force tarantool-common upgrade to support systemd notifications. + Add missing "s" suffix to TimeoutStartSec= option. See #1923
-
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.
-
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.
-
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.
-
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.
-
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.
-
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
-
bigbes authored
-
bigbes authored
-
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.
-
Vladimir Davydov authored
We should use fiber_sleep(0) to yield periodically, not fiber_reschedule() - the latter doesn't advance the event loop.
-
Roman Tsisyk authored
See 56462bca
-
Georgy Kirichenko authored
-
Vladimir Davydov authored
Check that we correctly recover in case not all indexes of the same space were dumped.
-
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.
-
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.
-
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.
- May 11, 2017
-
-
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).
-
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.
-
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.
-
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.
-
Vladimir Davydov authored
There's no point in maintaining a separate counter per each of them.
-
Nick Zavaritsky authored
mprotect() the last 4kB page of a 64kB fiber stack in order to detect stack overflows. Closes #2411
-
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 ```
-
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.
-
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.
-
Konstantin Osipov authored
-