Commit Graph

47 Commits

Author SHA1 Message Date
guybe7 f49ff156ec
Add RM_PublishMessageShard (#10543)
since PUBLISH and SPUBLISH use different dictionaries for channels and clients,
and we already have an API for PUBLISH, it only makes sense to have one for SPUBLISH

Add test coverage and unifying some test infrastructure.
2022-04-17 15:43:22 +03:00
Harkrishn Patro 9f8885760b
Sharded pubsub implementation (#8621)
This commit implements a sharded pubsub implementation based off of shard channels.

Co-authored-by: Harkrishn Patro <harkrisp@amazon.com>
Co-authored-by: Madelyn Olson <madelyneolson@gmail.com>
2022-01-02 16:54:47 -08:00
guybe7 43e736f79b
Treat subcommands as commands (#9504)
## Intro

The purpose is to allow having different flags/ACL categories for
subcommands (Example: CONFIG GET is ok-loading but CONFIG SET isn't)

We create a small command table for every command that has subcommands
and each subcommand has its own flags, etc. (same as a "regular" command)

This commit also unites the Redis and the Sentinel command tables

## Affected commands

CONFIG
Used to have "admin ok-loading ok-stale no-script"
Changes:
1. Dropped "ok-loading" in all except GET (this doesn't change behavior since
there were checks in the code doing that)

XINFO
Used to have "read-only random"
Changes:
1. Dropped "random" in all except CONSUMERS

XGROUP
Used to have "write use-memory"
Changes:
1. Dropped "use-memory" in all except CREATE and CREATECONSUMER

COMMAND
No changes.

MEMORY
Used to have "random read-only"
Changes:
1. Dropped "random" in PURGE and USAGE

ACL
Used to have "admin no-script ok-loading ok-stale"
Changes:
1. Dropped "admin" in WHOAMI, GENPASS, and CAT

LATENCY
No changes.

MODULE
No changes.

SLOWLOG
Used to have "admin random ok-loading ok-stale"
Changes:
1. Dropped "random" in RESET

OBJECT
Used to have "read-only random"
Changes:
1. Dropped "random" in ENCODING and REFCOUNT

SCRIPT
Used to have "may-replicate no-script"
Changes:
1. Dropped "may-replicate" in all except FLUSH and LOAD

CLIENT
Used to have "admin no-script random ok-loading ok-stale"
Changes:
1. Dropped "random" in all except INFO and LIST
2. Dropped "admin" in ID, TRACKING, CACHING, GETREDIR, INFO, SETNAME, GETNAME, and REPLY

STRALGO
No changes.

PUBSUB
No changes.

CLUSTER
Changes:
1. Dropped "admin in countkeysinslots, getkeysinslot, info, nodes, keyslot, myid, and slots

SENTINEL
No changes.

(note that DEBUG also fits, but we decided not to convert it since it's for
debugging and anyway undocumented)

## New sub-command
This commit adds another element to the per-command output of COMMAND,
describing the list of subcommands, if any (in the same structure as "regular" commands)
Also, it adds a new subcommand:
```
COMMAND LIST [FILTERBY (MODULE <module-name>|ACLCAT <cat>|PATTERN <pattern>)]
```
which returns a set of all commands (unless filters), but excluding subcommands.

## Module API
A new module API, RM_CreateSubcommand, was added, in order to allow
module writer to define subcommands

## ACL changes:
1. Now, that each subcommand is actually a command, each has its own ACL id.
2. The old mechanism of allowed_subcommands is redundant
(blocking/allowing a subcommand is the same as blocking/allowing a regular command),
but we had to keep it, to support the widespread usage of allowed_subcommands
to block commands with certain args, that aren't subcommands (e.g. "-select +select|0").
3. I have renamed allowed_subcommands to allowed_firstargs to emphasize the difference.
4. Because subcommands are commands in ACL too, you can now use "-" to block subcommands
(e.g. "+client -client|kill"), which wasn't possible in the past.
5. It is also possible to use the allowed_firstargs mechanism with subcommand.
For example: `+config -config|set +config|set|loglevel` will block all CONFIG SET except
for setting the log level.
6. All of the ACL changes above required some amount of refactoring.

## Misc
1. There are two approaches: Either each subcommand has its own function or all
   subcommands use the same function, determining what to do according to argv[0].
   For now, I took the former approaches only with CONFIG and COMMAND,
   while other commands use the latter approach (for smaller blamelog diff).
2. Deleted memoryGetKeys: It is no longer needed because MEMORY USAGE now uses the "range" key spec.
4. Bugfix: GETNAME was missing from CLIENT's help message.
5. Sentinel and Redis now use the same table, with the same function pointer.
   Some commands have a different implementation in Sentinel, so we redirect
   them (these are ROLE, PUBLISH, and INFO).
6. Command stats now show the stats per subcommand (e.g. instead of stats just
   for "config" you will have stats for "config|set", "config|get", etc.)
7. It is now possible to use COMMAND directly on subcommands:
   COMMAND INFO CONFIG|GET (The pipeline syntax was inspired from ACL, and
   can be used in functions lookupCommandBySds and lookupCommandByCString)
8. STRALGO is now a container command (has "help")

## Breaking changes:
1. Command stats now show the stats per subcommand (see (5) above)
2021-10-20 11:52:57 +03:00
yoav-steinberg 2753429c99
Client eviction (#8687)
### Description
A mechanism for disconnecting clients when the sum of all connected clients is above a
configured limit. This prevents eviction or OOM caused by accumulated used memory
between all clients. It's a complimentary mechanism to the `client-output-buffer-limit`
mechanism which takes into account not only a single client and not only output buffers
but rather all memory used by all clients.

#### Design
The general design is as following:
* We track memory usage of each client, taking into account all memory used by the
  client (query buffer, output buffer, parsed arguments, etc...). This is kept up to date
  after reading from the socket, after processing commands and after writing to the socket.
* Based on the used memory we sort all clients into buckets. Each bucket contains all
  clients using up up to x2 memory of the clients in the bucket below it. For example up
  to 1m clients, up to 2m clients, up to 4m clients, ...
* Before processing a command and before sleep we check if we're over the configured
  limit. If we are we start disconnecting clients from larger buckets downwards until we're
  under the limit.

#### Config
`maxmemory-clients` max memory all clients are allowed to consume, above this threshold
we disconnect clients.
This config can either be set to 0 (meaning no limit), a size in bytes (possibly with MB/GB
suffix), or as a percentage of `maxmemory` by using the `%` suffix (e.g. setting it to `10%`
would mean 10% of `maxmemory`).

#### Important code changes
* During the development I encountered yet more situations where our io-threads access
  global vars. And needed to fix them. I also had to handle keeps the clients sorted into the
  memory buckets (which are global) while their memory usage changes in the io-thread.
  To achieve this I decided to simplify how we check if we're in an io-thread and make it
  much more explicit. I removed the `CLIENT_PENDING_READ` flag used for checking
  if the client is in an io-thread (it wasn't used for anything else) and just used the global
  `io_threads_op` variable the same way to check during writes.
* I optimized the cleanup of the client from the `clients_pending_read` list on client freeing.
  We now store a pointer in the `client` struct to this list so we don't need to search in it
  (`pending_read_list_node`).
* Added `evicted_clients` stat to `INFO` command.
* Added `CLIENT NO-EVICT ON|OFF` sub command to exclude a specific client from the
  client eviction mechanism. Added corrosponding 'e' flag in the client info string.
* Added `multi-mem` field in the client info string to show how much memory is used up
  by buffered multi commands.
* Client `tot-mem` now accounts for buffered multi-commands, pubsub patterns and
  channels (partially), tracking prefixes (partially).
* CLIENT_CLOSE_ASAP flag is now handled in a new `beforeNextClient()` function so
  clients will be disconnected between processing different clients and not only before sleep.
  This new function can be used in the future for work we want to do outside the command
  processing loop but don't want to wait for all clients to be processed before we get to it.
  Specifically I wanted to handle output-buffer-limit related closing before we process client
  eviction in case the two race with each other.
* Added a `DEBUG CLIENT-EVICTION` command to print out info about the client eviction
  buckets.
* Each client now holds a pointer to the client eviction memory usage bucket it belongs to
  and listNode to itself in that bucket for quick removal.
* Global `io_threads_op` variable now can contain a `IO_THREADS_OP_IDLE` value
  indicating no io-threading is currently being executed.
* In order to track memory used by each clients in real-time we can't rely on updating
  these stats in `clientsCron()` alone anymore. So now I call `updateClientMemUsage()`
  (used to be `clientsCronTrackClientsMemUsage()`) after command processing, after
  writing data to pubsub clients, after writing the output buffer and after reading from the
  socket (and maybe other places too). The function is written to be fast.
* Clients are evicted if needed (with appropriate log line) in `beforeSleep()` and before
  processing a command (before performing oom-checks and key-eviction).
* All clients memory usage buckets are grouped as follows:
  * All clients using less than 64k.
  * 64K..128K
  * 128K..256K
  * ...
  * 2G..4G
  * All clients using 4g and up.
* Added client-eviction.tcl with a bunch of tests for the new mechanism.
* Extended maxmemory.tcl to test the interaction between maxmemory and
  maxmemory-clients settings.
* Added an option to flag a numeric configuration variable as a "percent", this means that
  if we encounter a '%' after the number in the config file (or config set command) we
  consider it as valid. Such a number is store internally as a negative value. This way an
  integer value can be interpreted as either a percent (negative) or absolute value (positive).
  This is useful for example if some numeric configuration can optionally be set to a percentage
  of something else.

Co-authored-by: Oran Agra <oran@redislabs.com>
2021-09-23 14:02:16 +03:00
Uri Shachar c7e502a07b
Cleaning up the cluster interface by moving almost all related declar… (#9080)
* Cleaning up the cluster interface by moving almost all related declarations into cluster.h
(no logic change -- just moving declarations/definitions around)

This initial effort leaves two items out of scope - the configuration parsing into the server
struct and the internals exposed by the clusterNode struct.

* Remove unneeded declarations of dictSds*
Ideally all the dictSds functionality would move from server.c into a dedicated module
so we can avoid the duplication in redis-benchmark/cli

* Move crc16 back into server.h, will be moved out once we create a seperate header file for
hashing functions
2021-06-15 20:35:13 -07:00
Binbin 0bfccc55e2
Fixed some typos, add a spell check ci and others minor fix (#8890)
This PR adds a spell checker CI action that will fail future PRs if they introduce typos and spelling mistakes.
This spell checker is based on blacklist of common spelling mistakes, so it will not catch everything,
but at least it is also unlikely to cause false positives.

Besides that, the PR also fixes many spelling mistakes and types, not all are a result of the spell checker we use.

Here's a summary of other changes:
1. Scanned the entire source code and fixes all sorts of typos and spelling mistakes (including missing or extra spaces).
2. Outdated function / variable / argument names in comments
3. Fix outdated keyspace masks error log when we check `config.notify-keyspace-events` in loadServerConfigFromString.
4. Trim the white space at the end of line in `module.c`. Check: https://github.com/redis/redis/pull/7751
5. Some outdated https link URLs.
6. Fix some outdated comment. Such as:
    - In README: about the rdb, we used to said create a `thread`, change to `process`
    - dbRandomKey function coment (about the dictGetRandomKey, change to dictGetFairRandomKey)
    - notifyKeyspaceEvent fucntion comment (add type arg)
    - Some others minor fix in comment (Most of them are incorrectly quoted by variable names)
7. Modified the error log so that users can easily distinguish between TCP and TLS in `changeBindAddr`
2021-06-10 15:39:33 +03:00
Huang Zhw e138698e54
make processCommand check publish channel permissions. (#8534)
Add publish channel permissions check in processCommand.

processCommand didn't check publish channel permissions, so we can
queue a publish command in a transaction. But when exec the transaction,
it will fail with -NOPERM.

We also union keys/commands/channels permissions check togegher in
ACLCheckAllPerm. Remove pubsubCheckACLPermissionsOrReply in 
publishCommand/subscribeCommand/psubscribeCommand. Always 
check permissions in processCommand/execCommand/
luaRedisGenericCommand.
2021-03-26 14:10:01 +03:00
Viktor Söderqvist 6122f1c450
Shared reusable client for RM_Call() (#8516)
A single client pointer is added in the server struct. This is
initialized by the first RM_Call() and reused for every subsequent
RM_Call() except if it's already in use, which means that it's not
used for (recursive) module calls to modules. For these, a new
"fake" client is created each time.

Other changes:
* Avoid allocating a dict iterator in pubsubUnsubscribeAllChannels
  when not needed
2021-02-28 14:11:18 +02:00
Harkrishn Patro 303465af35
Remove redundant pubsub list to store the patterns. (#8472)
Remove redundant pubsub list to store the patterns.
2021-02-17 14:13:50 -08:00
Itamar Haber 9dcdc7e79a
HELP subcommand, continued (#5531)
* man-like consistent long formatting
* Uppercases commands, subcommands and options
* Adds 'HELP' to HELP for all
* Lexicographical order
* Uses value notation and other .md likeness
* Moves const char *help to top
* Keeps it under 80 chars
* Misc help typos, consistent conjuctioning (i.e return and not returns)
* Uses addReplySubcommandSyntaxError(c) all over

Signed-off-by: Itamar Haber <itamar@redislabs.com>
2021-01-04 17:02:57 +02:00
Itamar Haber c1b1e8c329
Adds pub/sub channel patterns to ACL (#7993)
Fixes #7923.

This PR appropriates the special `&` symbol (because `@` and `*` are taken),
followed by a literal value or pattern for describing the Pub/Sub patterns that
an ACL user can interact with. It is similar to the existing key patterns
mechanism in function (additive) and implementation (copy-pasta). It also adds
the allchannels and resetchannels ACL keywords, naturally.

The default user is given allchannels permissions, whereas new users get
whatever is defined by the acl-pubsub-default configuration directive. For
backward compatibility in 6.2, the default of this directive is allchannels but
this is likely to be changed to resetchannels in the next major version for
stronger default security settings.

Unless allchannels is set for the user, channel access permissions are checked
as follows :
* Calls to both PUBLISH and SUBSCRIBE will fail unless a pattern matching the
  argumentative channel name(s) exists for the user.
* Calls to PSUBSCRIBE will fail unless the pattern(s) provided as an argument
  literally exist(s) in the user's list.

Such failures are logged to the ACL log.

Runtime changes to channel permissions for a user with existing subscribing
clients cause said clients to disconnect unless the new permissions permit the
connections to continue. Note, however, that PSUBSCRIBErs' patterns are matched
literally, so given the change bar:* -> b*, pattern subscribers to bar:* will be
disconnected.

Notes/questions:
* UNSUBSCRIBE, PUNSUBSCRIBE and PUBSUB remain unprotected due to lack of reasons
  for touching them.
2020-12-01 14:21:39 +02:00
Meir Shpilraien (Spielrein) d87a0d0286
Unified MULTI, LUA, and RM_Call with respect to blocking commands (#8025)
Blocking command should not be used with MULTI, LUA, and RM_Call. This is because,
the caller, who executes the command in this context, expects a reply.

Today, LUA and MULTI have a special (and different) treatment to blocking commands:

LUA   - Most commands are marked with no-script flag which are checked when executing
and command from LUA, commands that are not marked (like XREAD) verify that their
blocking mode is not used inside LUA (by checking the CLIENT_LUA client flag).
MULTI - Command that is going to block, first verify that the client is not inside
multi (by checking the CLIENT_MULTI client flag). If the client is inside multi, they
return a result which is a match to the empty key with no timeout (for example blpop
inside MULTI will act as lpop)
For modules that perform RM_Call with blocking command, the returned results type is
REDISMODULE_REPLY_UNKNOWN and the caller can not really know what happened.

Disadvantages of the current state are:

No unified approach, LUA, MULTI, and RM_Call, each has a different treatment
Module can not safely execute blocking command (and get reply or error).
Though It is true that modules are not like LUA or MULTI and should be smarter not
to execute blocking commands on RM_Call, sometimes you want to execute a command base
on client input (for example if you create a module that provides a new scripting
language like javascript or python).
While modules (on modules command) can check for REDISMODULE_CTX_FLAGS_LUA or
REDISMODULE_CTX_FLAGS_MULTI to know not to block the client, there is no way to
check if the command came from another module using RM_Call. So there is no way
for a module to know not to block another module RM_Call execution.

This commit adds a way to unify the treatment for blocking clients by introducing
a new CLIENT_DENY_BLOCKING client flag. On LUA, MULTI, and RM_Call the new flag
turned on to signify that the client should not be blocked. A blocking command
verifies that the flag is turned off before blocking. If a blocking command sees
that the CLIENT_DENY_BLOCKING flag is on, it's not blocking and return results
which are matches to empty key with no timeout (as MULTI does today).

The new flag is checked on the following commands:

List blocking commands: BLPOP, BRPOP, BRPOPLPUSH, BLMOVE,
Zset blocking commands: BZPOPMIN, BZPOPMAX
Stream blocking commands: XREAD, XREADGROUP
SUBSCRIBE, PSUBSCRIBE, MONITOR
In addition, the new flag is turned on inside the AOF client, we do not want to
block the AOF client to prevent deadlocks and commands ordering issues (and there
is also an existing assert in the code that verifies it).

To keep backward compatibility on LUA, all the no-script flags on existing commands
were kept untouched. In addition, a LUA special treatment on XREAD and XREADGROUP was kept.

To keep backward compatibility on MULTI (which today allows SUBSCRIBE, and PSUBSCRIBE).
We added a special treatment on those commands to allow executing them on MULTI.

The only backward compatibility issue that this PR introduces is that now MONITOR
is not allowed inside MULTI.

Tests were added to verify blocking commands are not blocking the client on LUA, MULTI,
or RM_Call. Tests were added to verify the module can check for CLIENT_DENY_BLOCKING flag.

Co-authored-by: Oran Agra <oran@redislabs.com>
Co-authored-by: Itamar Haber <itamar@redislabs.com>
2020-11-17 18:58:55 +02:00
antirez 1b4bc60999 Merge branch 'pubsub_patterns_boost' of https://github.com/leeyiw/redis into leeyiw-pubsub_patterns_boost 2020-03-31 12:40:08 +02:00
antirez f53cc00c09 Tracking: always reply with an array of keys. 2020-02-10 13:42:18 +01:00
antirez eaaac08892 RESP3: Pubsub messages in new push format if client is in RESP3 mode. 2019-01-09 17:00:30 +01:00
antirez bc75a94e2d RESP3: pubsub messages API completely refactored. 2019-01-09 17:00:30 +01:00
antirez 798a329192 RESP3: extract code to send pubsub messages into functions. 2019-01-09 17:00:30 +01:00
antirez 317f8b9d38 RESP3: most null replies converted. 2019-01-09 17:00:29 +01:00
antirez dcbd40cea4 RESP3: Use new aggregate reply API in cluster.c. 2019-01-09 17:00:29 +01:00
antirez 2edcafb35d addReplySubSyntaxError() renamed to addReplySubcommandSyntaxError(). 2018-07-02 18:49:34 +02:00
Itamar Haber fefde6e3e4 Capitalizes subcommands & orders lexicographically 2018-06-09 21:03:52 +03:00
Itamar Haber c199280edb Globally applies addReplySubSyntaxError 2018-06-07 18:39:36 +03:00
伯成 dfb12f0628 Boost up performance for redis PUB-SUB patterns matching
If lots of clients PSUBSCRIBE to same patterns, multiple pattens matching will take place. This commit change it into just one single pattern matching by using a `dict *` to store the unique pattern and which clients subscribe to it.
2018-03-01 11:46:56 +08:00
antirez 522760fac7 Change indentation and other minor details of PR #4489.
The main change introduced by this commit is pretending that help
arrays are more text than code, thus indenting them at level 0. This
improves readability, and is an old practice when defining arrays of
C strings describing text.

Additionally a few useless return statements are removed, and the HELP
subcommand capitalized when printed to the user.
2017-12-06 12:05:14 +01:00
Itamar Haber 59d52f7fab Standardizes the 'help' subcommand
This adds a new `addReplyHelp` helper that's used by commands
when returning a help text. The following commands have been
touched: DEBUG, OBJECT, COMMAND, PUBSUB, SCRIPT and SLOWLOG.

WIP

Fix entry command table entry for OBJECT for HELP option.

After #4472 the command may have just 2 arguments.

Improve OBJECT HELP descriptions.

See #4472.

WIP 2

WIP 3
2017-11-28 21:15:45 +02:00
antirez 32f80e2f1b RDMF: More consistent define names. 2015-07-27 14:37:58 +02:00
antirez 2d9e3eb107 RDMF: redisAssert -> serverAssert. 2015-07-26 15:29:53 +02:00
antirez 554bd0e7bd RDMF: use client instead of redisClient, like Disque. 2015-07-26 15:20:52 +02:00
antirez cef054e868 RDMF (Redis/Disque merge friendlyness) refactoring WIP 1. 2015-07-26 15:17:18 +02:00
Matt Stancliff 25791550e0 pubsub: Return integers for NUMSUB, not strings
Also adds test for numsub — due to tcl being tcl,
it doesn't capture the "numberness" of the fix,
but now we at least have one test case for numsub.

Closes #1561
2014-08-08 11:19:37 +02:00
antirez 59cf0824d9 PubSub clients refactoring and new PUBSUB flag.
The code tested many times if a client had active Pub/Sub subscriptions
by checking the length of a list and dictionary where the patterns and
channels are stored. This was substituted with a client flag called
REDIS_PUBSUB that is simpler to test for. Moreover in order to manage
this flag some code was refactored.

This commit is believed to have no effects in the behavior of the
server.
2014-07-16 17:34:07 +02:00
antirez 82b53c650c struct dictEntry -> dictEntry. 2014-03-20 16:20:37 +01:00
antirez 344a065d51 Cluster: don't propagate PUBLISH two times.
PUBLISH both published messages via Cluster bus and replication when
cluster was enabled, resulting in duplicated message in the slave.
2014-02-10 16:00:27 +01:00
antirez 515a26bbc1 New API to force propagation.
The old REDIS_CMD_FORCE_REPLICATION flag was removed from the
implementation of Redis, now there is a new API to force specific
executions of a command to be propagated to AOF / Replication link:

    void forceCommandPropagation(int flags);

The new API is also compatible with Lua scripting, so a script that will
execute commands that are forced to be propagated, will also be
propagated itself accordingly even if no change to data is operated.

As a side effect, this new design fixes the issue with scripts not able
to propagate PUBLISH to slaves (issue #873).
2013-06-21 12:07:53 +02:00
antirez 519c9e11d1 Allow PUBSUB NUMSUB without channels.
The result is an empty list but it is handy to call it programmatically.
2013-06-20 15:34:56 +02:00
antirez 455563faec PUBSUB command implemented.
Currently it implements three subcommands:

PUBSUB CHANNELS [<pattern>]    List channels with non-zero subscribers.
PUBSUB NUMSUB [channel_1 ...]  List number of subscribers for channels.
PUBSUB NUMPAT                  Return number of subscribed patterns.
2013-06-20 15:32:00 +02:00
Gengliang Wang 042ed270c8 Removed useless "return" statements in pubsub.c
(original commit message edited)
2013-03-06 16:49:20 +01:00
antirez 2039f1a38a UNSUBSCRIBE and PUNSUBSCRIBE: always provide a reply.
UNSUBSCRIBE and PUNSUBSCRIBE commands are designed to mass-unsubscribe
the client respectively all the channels and patters if called without
arguments.

However when these functions are called without arguments, but there are
no channels or patters we are subscribed to, the old behavior was to
don't reply at all.

This behavior is broken, as every command should always reply.
Also it is possible that we are no longer subscribed to a channels but we
are subscribed to patters or the other way around, and the client should
be notified with the correct number of subscriptions.

Also it is not pretty that sometimes we did not receive a reply at all
in a redis-cli session from these commands, blocking redis-cli trying
to read the reply.

This fixes issue #714.
2013-01-21 19:02:26 +01:00
guiquanz 9d09ce3981 Fixed many typos. 2013-01-19 10:59:44 +01:00
antirez 4365e5b2d3 BSD license added to every C source and header file. 2012-11-08 18:31:32 +01:00
antirez 355f859134 Use less memory when emitting the protocol, by using more shared objects for commonly emitted parts of the protocol. 2012-02-04 08:58:37 +01:00
antirez c0ba9ebe13 dict.c API names modified to be more coincise and consistent. 2011-11-08 17:07:55 +01:00
antirez c563ce463b propagate PUBLISH messages using the redis cluster nodes bus. Still need to process the incoming packets of that type. Work in progress. 2011-10-07 15:37:34 +02:00
antirez eab0e26e03 replaced redisAssert() with redisAssertWithInfo() in a shitload of places. 2011-10-04 18:43:03 +02:00
antirez efc3408748 Fix for Pub/Sub system, introduced in Redis 2.2.6 with the new copy-on-write safe iterator semantics. In the hope this is the last bug I introduced this way. 2011-05-25 12:32:15 +02:00
antirez 0f49d6b049 minor aesthetic change 2010-07-01 15:14:25 +02:00
antirez e2641e09cc redis.c split into many different C files.
networking related stuff moved into networking.c

moved more code

more work on layout of source code

SDS instantaneuos memory saving. By Pieter and Salvatore at VMware ;)

cleanly compiling again after the first split, now splitting it in more C files

moving more things around... work in progress

split replication code

splitting more

Sets split

Hash split

replication split

even more splitting

more splitting

minor change
2010-07-01 14:38:51 +02:00